hqlcppds.cpp 164 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510351135123513351435153516351735183519352035213522352335243525352635273528352935303531353235333534353535363537353835393540354135423543354435453546354735483549355035513552355335543555355635573558355935603561356235633564356535663567356835693570357135723573357435753576357735783579358035813582358335843585358635873588358935903591359235933594359535963597359835993600360136023603360436053606360736083609361036113612361336143615361636173618361936203621362236233624362536263627362836293630363136323633363436353636363736383639364036413642364336443645364636473648364936503651365236533654365536563657365836593660366136623663366436653666366736683669367036713672367336743675367636773678367936803681368236833684368536863687368836893690369136923693369436953696369736983699370037013702370337043705370637073708370937103711371237133714371537163717371837193720372137223723372437253726372737283729373037313732373337343735373637373738373937403741374237433744374537463747374837493750375137523753375437553756375737583759376037613762376337643765376637673768376937703771377237733774377537763777377837793780378137823783378437853786378737883789379037913792379337943795379637973798379938003801380238033804380538063807380838093810381138123813381438153816381738183819382038213822382338243825382638273828382938303831383238333834383538363837383838393840384138423843384438453846384738483849385038513852385338543855385638573858385938603861386238633864386538663867386838693870387138723873387438753876387738783879388038813882388338843885388638873888388938903891389238933894389538963897389838993900390139023903390439053906390739083909391039113912391339143915391639173918391939203921392239233924392539263927392839293930393139323933393439353936393739383939394039413942394339443945394639473948394939503951395239533954395539563957395839593960396139623963396439653966396739683969397039713972397339743975397639773978397939803981398239833984398539863987398839893990399139923993399439953996399739983999400040014002400340044005400640074008400940104011401240134014401540164017401840194020402140224023402440254026402740284029403040314032403340344035403640374038403940404041404240434044404540464047404840494050405140524053405440554056405740584059406040614062406340644065406640674068406940704071407240734074407540764077407840794080408140824083408440854086408740884089409040914092409340944095409640974098409941004101410241034104410541064107410841094110411141124113411441154116411741184119412041214122412341244125412641274128412941304131413241334134413541364137413841394140414141424143414441454146414741484149415041514152415341544155415641574158415941604161416241634164416541664167416841694170417141724173417441754176417741784179418041814182418341844185418641874188418941904191419241934194419541964197419841994200420142024203420442054206420742084209421042114212421342144215421642174218421942204221422242234224422542264227422842294230423142324233423442354236423742384239424042414242424342444245424642474248424942504251425242534254425542564257425842594260426142624263426442654266426742684269427042714272427342744275427642774278427942804281428242834284428542864287428842894290429142924293429442954296429742984299430043014302430343044305430643074308430943104311431243134314431543164317431843194320432143224323432443254326432743284329433043314332433343344335433643374338433943404341434243434344434543464347434843494350435143524353435443554356435743584359436043614362436343644365436643674368436943704371437243734374437543764377437843794380438143824383438443854386438743884389439043914392439343944395439643974398439944004401440244034404440544064407440844094410441144124413441444154416441744184419442044214422442344244425442644274428442944304431443244334434443544364437443844394440444144424443444444454446444744484449445044514452445344544455445644574458445944604461446244634464446544664467446844694470447144724473447444754476447744784479448044814482448344844485448644874488448944904491449244934494449544964497449844994500450145024503450445054506450745084509451045114512451345144515451645174518451945204521452245234524452545264527452845294530453145324533453445354536453745384539454045414542454345444545454645474548454945504551455245534554455545564557455845594560456145624563456445654566456745684569457045714572457345744575457645774578457945804581458245834584458545864587458845894590459145924593459445954596459745984599460046014602460346044605460646074608460946104611461246134614461546164617461846194620462146224623462446254626462746284629463046314632463346344635463646374638463946404641464246434644464546464647464846494650465146524653465446554656465746584659466046614662466346644665
  1. /*##############################################################################
  2. Copyright (C) 2011 HPCC Systems.
  3. All rights reserved. This program is free software: you can redistribute it and/or modify
  4. it under the terms of the GNU Affero General Public License as
  5. published by the Free Software Foundation, either version 3 of the
  6. License, or (at your option) any later version.
  7. This program is distributed in the hope that it will be useful,
  8. but WITHOUT ANY WARRANTY; without even the implied warranty of
  9. MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
  10. GNU Affero General Public License for more details.
  11. You should have received a copy of the GNU Affero General Public License
  12. along with this program. If not, see <http://www.gnu.org/licenses/>.
  13. ############################################################################## */
  14. #include "jliball.hpp"
  15. #include "hql.hpp"
  16. #include "platform.h"
  17. #include "jlib.hpp"
  18. #include "jmisc.hpp"
  19. #include "jstream.ipp"
  20. #include "jdebug.hpp"
  21. #include "hql.hpp"
  22. #include "hqlthql.hpp"
  23. #include "hqlhtcpp.ipp"
  24. #include "hqlttcpp.ipp"
  25. #include "hqlutil.hpp"
  26. #include "hqlthql.hpp"
  27. #include "hqlpmap.hpp"
  28. #include "hqlattr.hpp"
  29. #include "hqlwcpp.hpp"
  30. #include "hqlcpputil.hpp"
  31. #include "hqltcppc.ipp"
  32. #include "hqlopt.hpp"
  33. #include "hqlfold.hpp"
  34. #include "hqlcerrors.hpp"
  35. #include "hqlcatom.hpp"
  36. #include "hqlresource.hpp"
  37. #include "hqlregex.ipp"
  38. #include "hqlsource.ipp"
  39. #include "hqlcse.ipp"
  40. #include "hqlgraph.ipp"
  41. #include "hqlccommon.hpp"
  42. #include "hqliter.ipp"
  43. #include "hqlinline.hpp"
  44. #include "hqlusage.hpp"
  45. #include "hqlcppds.hpp"
  46. #define MAX_FIXED_SIZE_RAW 1024
  47. #define INLINE_TABLE_EXPAND_LIMIT 4
  48. void addGraphIdAttribute(ActivityInstance * instance, BuildCtx & ctx, IHqlExpression * graphId)
  49. {
  50. SubGraphInfo * match = matchActiveGraph(ctx, graphId);
  51. assertex(match);
  52. instance->addAttributeInt("_graphId", match->graphId);
  53. }
  54. //===========================================================================
  55. void HqlCppTranslator::doBuildRowIfBranch(BuildCtx & initctx, BuildCtx & ctx, BoundRow * targetRow, IHqlExpression * branchExpr)
  56. {
  57. IHqlExpression * targetRowExpr = targetRow->queryBound();
  58. Owned<IReferenceSelector> rowSelector = buildNewRow(ctx, branchExpr);
  59. Owned<BoundRow> boundRow = rowSelector->getRow(ctx);
  60. OwnedHqlExpr rowExpr = getPointer(boundRow->queryBound());
  61. OwnedHqlExpr castRow = createValue(no_implicitcast, targetRowExpr->getType(), LINK(rowExpr));
  62. ctx.addAssign(targetRowExpr, castRow);
  63. if (rowSelector->isConditional())
  64. targetRow->setConditional(true);
  65. }
  66. IReferenceSelector * HqlCppTranslator::doBuildRowIf(BuildCtx & ctx, IHqlExpression * expr)
  67. {
  68. OwnedHqlExpr foldedCond = foldHqlExpression(expr->queryChild(0));
  69. if (foldedCond->queryValue())
  70. {
  71. unsigned branch = (foldedCond->queryValue()->getBoolValue()) ? 1 : 2;
  72. return buildNewRow(ctx, expr->queryChild(branch));
  73. }
  74. IHqlExpression * trueBranch = expr->queryChild(1);
  75. IHqlExpression * falseBranch = expr->queryChild(2);
  76. //Ideally should have a constant modifier on the following row...
  77. Owned<ITypeInfo> rowType = makeReferenceModifier(expr->getType());
  78. OwnedHqlExpr rowExpr = ctx.getTempDeclare(rowType, NULL);
  79. Owned<BoundRow> row = createBoundRow(expr->queryBody(), rowExpr);
  80. //MORE: Need casts because cursor may be (probably are) constant, but temporary isn't
  81. //should find out by looking at the a const modifier on the cursor.
  82. BuildCtx condctx(ctx);
  83. IHqlStmt * cond = buildFilterViaExpr(condctx, foldedCond);
  84. //Mark the context as conditional after the filter test, so any temporaries from the filter aren't affected.
  85. condctx.associateExpr(queryConditionalRowMarker(), rowExpr);
  86. doBuildRowIfBranch(ctx, condctx, row, trueBranch);
  87. condctx.selectElse(cond);
  88. condctx.associateExpr(queryConditionalRowMarker(), rowExpr);
  89. doBuildRowIfBranch(ctx, condctx, row, falseBranch);
  90. ctx.associate(*row);
  91. return createReferenceSelector(row);
  92. }
  93. IReferenceSelector * HqlCppTranslator::doBuildRowDeserializeRow(BuildCtx & ctx, IHqlExpression * expr)
  94. {
  95. IHqlExpression * srcRow = expr->queryChild(0);
  96. IHqlExpression * record = expr->queryRecord();
  97. Owned<BoundRow> tempRow = declareLinkedRow(ctx, expr, false);
  98. CHqlBoundTarget target;
  99. target.expr.set(tempRow->queryBound());
  100. HqlExprArray args;
  101. args.append(*createRowAllocator(ctx, record));
  102. args.append(*createRowSerializer(ctx, record, deserializerAtom));
  103. args.append(*LINK(srcRow));
  104. Owned<ITypeInfo> resultType = makeReferenceModifier(makeAttributeModifier(makeRowType(record->getType()), getLinkCountedAttr()));
  105. OwnedHqlExpr call = bindFunctionCall(rtlDeserializeRowAtom, args, resultType);
  106. buildExprAssign(ctx, target, call);
  107. ctx.associate(*tempRow);
  108. return createReferenceSelector(tempRow);
  109. }
  110. void HqlCppTranslator::buildConstRow(IHqlExpression * record, IHqlExpression * rowData, CHqlBoundExpr & bound)
  111. {
  112. OwnedHqlExpr marker = createAttribute(rowAtom, LINK(record), LINK(rowData));
  113. BuildCtx declareCtx(*code, literalAtom);
  114. if (declareCtx.getMatchExpr(marker, bound))
  115. return;
  116. //MORE: This probably needs to go in the header as well...
  117. Owned<ITypeInfo> rowType = makeConstantModifier(makeRowType(record->getType()));
  118. StringBuffer rowName;
  119. getUniqueId(rowName.append("r"));
  120. //Generate two variables to cope with the different ways the data is interpreted.
  121. //Would prefer it to be cleaner... row value would need an associated size
  122. unsigned dataSize = rowData->queryType()->getSize();
  123. OwnedITypeInfo declareType;
  124. OwnedHqlExpr initializer;
  125. if (options.staticRowsUseStringInitializer)
  126. {
  127. //Generates smaller code (and probably more efficient representation in the c++ compiler
  128. //const byte[5+1] = "Hello"; need an extra byte for the implicit \0
  129. declareType.setown(makeDataType(dataSize+1));
  130. initializer.set(rowData);
  131. }
  132. else
  133. {
  134. //Following is strictly correct, but much larger.
  135. //const byte[5] = { 'H','e','l','l','o' };
  136. declareType.set(rowData->queryType());
  137. initializer.setown(createValue(no_create_initializer, rowData->getType(), LINK(rowData)));
  138. }
  139. //MORE: Currently these are marked as const rows, but not generated as such
  140. OwnedHqlExpr boundDeclare = createVariable(rowName, makeConstantModifier(LINK(declareType)));
  141. OwnedHqlExpr boundRow = createVariable(rowName, LINK(rowType));
  142. declareCtx.addDeclare(boundDeclare, initializer);
  143. if (options.spanMultipleCpp)
  144. {
  145. BuildCtx protoctx(*code, mainprototypesAtom);
  146. protoctx.addDeclareExternal(boundDeclare);
  147. }
  148. bound.length.setown(getSizetConstant(dataSize));
  149. bound.expr.set(boundRow);
  150. declareCtx.associateExpr(marker, bound);
  151. }
  152. bool HqlCppTranslator::doBuildRowConstantTransform(IHqlExpression * transform, CHqlBoundExpr & bound)
  153. {
  154. if (!transform->isConstant() || !options.generateStaticInlineTables)
  155. return false;
  156. OwnedHqlExpr constRow = createConstantRowExpr(transform);
  157. if (!constRow || !canGenerateStringInline(constRow->queryType()->getSize()))
  158. return false;
  159. buildConstRow(transform->queryRecord(), constRow, bound);
  160. return true;
  161. }
  162. IReferenceSelector * HqlCppTranslator::doBuildRowCreateRow(BuildCtx & ctx, IHqlExpression * expr)
  163. {
  164. CHqlBoundExpr bound;
  165. if (!doBuildRowConstantTransform(expr->queryChild(0), bound))
  166. return doBuildRowViaTemp(ctx, expr);
  167. BoundRow * row = bindConstantRow(ctx, expr, bound);
  168. return createReferenceSelector(row);
  169. }
  170. BoundRow * HqlCppTranslator::bindConstantRow(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & bound)
  171. {
  172. BoundRow * row = bindRow(ctx, expr, bound.expr);
  173. //MORE: This should be done more cleanly
  174. OwnedHqlExpr sizeOfRow = createSizeof(row->querySelector());
  175. ctx.associateExpr(sizeOfRow, bound.length);
  176. return row;
  177. }
  178. bool HqlCppTranslator::doBuildRowConstantNull(IHqlExpression * expr, CHqlBoundExpr & bound)
  179. {
  180. if (!options.generateStaticInlineTables)
  181. return false;
  182. IHqlExpression * record = expr->queryRecord();
  183. OwnedHqlExpr constRow = createConstantNullRowExpr(record);
  184. if (!constRow)
  185. return false;
  186. buildConstRow(record, constRow, bound);
  187. return true;
  188. }
  189. IReferenceSelector * HqlCppTranslator::doBuildRowNull(BuildCtx & ctx, IHqlExpression * expr)
  190. {
  191. CHqlBoundExpr bound;
  192. if (!doBuildRowConstantNull(expr, bound))
  193. return doBuildRowViaTemp(ctx, expr);
  194. BoundRow * row = bindRow(ctx, expr, bound.expr);
  195. return createReferenceSelector(row);
  196. }
  197. IReferenceSelector * HqlCppTranslator::doBuildRowViaTemp(BuildCtx & ctx, IHqlExpression * expr)
  198. {
  199. HqlExprAssociation * match;
  200. if (expr->isDataset())
  201. match = ctx.queryAssociation(expr->queryNormalizedSelector(), AssocCursor, NULL);
  202. else
  203. match = ctx.queryAssociation(expr, AssocRow, NULL);
  204. if (match)
  205. {
  206. BoundRow * row = (BoundRow *)match;
  207. return createReferenceSelector(row, expr);
  208. }
  209. Owned<BoundRow> tempRow = declareTempRow(ctx, ctx, expr);
  210. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  211. Owned<IReferenceSelector> createdRef = createReferenceSelector(rowBuilder);
  212. buildRowAssign(ctx, createdRef, expr);
  213. finalizeTempRow(ctx, tempRow, rowBuilder);
  214. ctx.associate(*tempRow);
  215. return createReferenceSelector(tempRow);
  216. }
  217. void HqlCppTranslator::buildDefaultRow(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & bound)
  218. {
  219. OwnedHqlExpr clearExpr = createRow(no_null, LINK(expr->queryRecord()), createAttribute(clearAtom));
  220. BoundRow * matchedRow = (BoundRow *)ctx.queryAssociation(clearExpr, AssocRow, NULL);
  221. if (!matchedRow)
  222. {
  223. if (doBuildRowConstantNull(expr, bound))
  224. {
  225. bindRow(ctx, clearExpr, bound.expr);
  226. }
  227. else
  228. {
  229. BuildCtx * declarectx = &ctx;
  230. BuildCtx * callCtx = &ctx;
  231. getInvariantMemberContext(ctx, &declarectx, &callCtx, true, false);
  232. Owned<BoundRow> tempRow = declareTempRow(*declarectx, *callCtx, clearExpr);
  233. Owned<BoundRow> rowBuilder = createRowBuilder(*callCtx, tempRow);
  234. OwnedHqlExpr size = createVariable(LINK(sizetType));
  235. OwnedHqlExpr clearCall = createClearRowCall(*callCtx, rowBuilder);
  236. callCtx->addDeclare(size, clearCall);
  237. OwnedHqlExpr sizeOfRow = createSizeof(rowBuilder->querySelector());
  238. callCtx->associateExpr(sizeOfRow, size);
  239. finalizeTempRow(*callCtx, tempRow, rowBuilder);
  240. declarectx->associate(*tempRow);
  241. bound.expr.set(tempRow->queryBound());
  242. }
  243. }
  244. else
  245. bound.expr.set(matchedRow->queryBound());
  246. //yuk yuk, hack. If called from a const context then need to make the reference unconst.
  247. //The real fix is to implement real const tracking throughout the code generator, but that is far from trivial.
  248. //rkc39.hql is an example...
  249. if (ctx.queryMatchExpr(constantMemberMarkerExpr))
  250. bound.expr.setown(createValue(no_cast, makeReferenceModifier(bound.expr->getType()), getPointer(bound.expr)));
  251. }
  252. void HqlCppTranslator::buildNullRow(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & bound)
  253. {
  254. bound.expr.setown(createValue(no_nullptr, makeRowReferenceType(expr)));
  255. }
  256. IReferenceSelector * HqlCppTranslator::doBuildRowFromXML(BuildCtx & ctx, IHqlExpression * expr)
  257. {
  258. // assertex(supportsLinkCountedRows);
  259. Owned<ITypeInfo> overrideType = setLinkCountedAttr(expr->queryType(), true);
  260. Owned<ITypeInfo> utf8Type = makeUtf8Type(UNKNOWN_LENGTH, NULL);
  261. IHqlExpression * record = expr->queryRecord();
  262. OwnedHqlExpr ds = createDataset(no_anon, LINK(record));
  263. StringBuffer xmlInstanceName, xmlFactoryName, s;
  264. bool usesContents = false;
  265. getUniqueId(xmlInstanceName.append("xml"));
  266. buildXmlReadTransform(ds, xmlFactoryName, usesContents);
  267. OwnedHqlExpr curActivityId = getCurrentActivityId(ctx);
  268. s.append("Owned<IXmlToRowTransformer> ").append(xmlInstanceName).append(" = ").append(xmlFactoryName).append("(ctx,");
  269. generateExprCpp(s, curActivityId).append(");");
  270. ctx.addQuoted(s);
  271. HqlExprArray args;
  272. args.append(*createRowAllocator(ctx, record));
  273. args.append(*ensureExprType(expr->queryChild(1), utf8Type));
  274. args.append(*createQuoted(xmlInstanceName, makeBoolType()));
  275. args.append(*createConstant(expr->hasProperty(trimAtom)));
  276. OwnedHqlExpr function = bindFunctionCall(createRowFromXmlAtom, args, overrideType);
  277. CHqlBoundExpr bound;
  278. buildExpr(ctx, function, bound);
  279. Owned<ITypeInfo> rowType = makeReferenceModifier(LINK(overrideType));
  280. OwnedHqlExpr rowExpr = ctx.getTempDeclare(rowType, NULL);
  281. Owned<BoundRow> row = createBoundRow(expr, rowExpr);
  282. ctx.associate(*row); // associate here because it is compared inside the loop
  283. OwnedHqlExpr defaultRowPtr = getPointer(bound.expr);
  284. ctx.addAssign(rowExpr, defaultRowPtr);
  285. return createReferenceSelector(row);
  286. }
  287. //NB: If this is a dataset operation, this function assumes that any parent datasets are already in scope
  288. // i.e. processed in buildDatasetAssign()
  289. // the one exception is aggregate because it needs to treat its input differently.
  290. IReferenceSelector * HqlCppTranslator::buildNewOrActiveRow(BuildCtx & ctx, IHqlExpression * expr, bool isNew)
  291. {
  292. if (isNew)
  293. return buildNewRow(ctx, expr);
  294. else
  295. return buildActiveRow(ctx, expr);
  296. }
  297. IReferenceSelector * HqlCppTranslator::buildNewRow(BuildCtx & ctx, IHqlExpression * expr)
  298. {
  299. assertex(!expr->isDataset());
  300. BoundRow * match = static_cast<BoundRow *>(ctx.queryAssociation(expr, AssocRow, NULL));
  301. if (match)
  302. return createReferenceSelector(match, expr);
  303. BoundRow * row = NULL;
  304. node_operator op = expr->getOperator();
  305. switch (op)
  306. {
  307. case no_activerow:
  308. return buildActiveRow(ctx, expr->queryChild(0));
  309. case no_if:
  310. return doBuildRowIf(ctx, expr);
  311. case no_id2blob:
  312. return doBuildRowIdToBlob(ctx, expr, true);
  313. case no_index:
  314. case no_selectnth:
  315. return buildDatasetIndex(ctx, expr);
  316. case no_selectmap:
  317. return buildDatasetSelectMap(ctx, expr);
  318. case no_left:
  319. case no_right:
  320. case no_self:
  321. case no_top:
  322. case no_activetable:
  323. return buildActiveRow(ctx, expr);
  324. case no_fromxml:
  325. return doBuildRowFromXML(ctx, expr);
  326. case no_serialize:
  327. if (isDummySerializeDeserialize(expr))
  328. return buildNewRow(ctx, expr->queryChild(0)->queryChild(0));
  329. return doBuildRowViaTemp(ctx, expr);
  330. case no_deserialize:
  331. if (isDummySerializeDeserialize(expr))
  332. return buildNewRow(ctx, expr->queryChild(0)->queryChild(0));
  333. return doBuildRowDeserializeRow(ctx, expr);
  334. case no_deref:
  335. {
  336. //Untested
  337. CHqlBoundExpr bound;
  338. buildExpr(ctx, expr, bound);
  339. row = bindRow(ctx, expr, bound.expr);
  340. break;
  341. }
  342. case no_createrow:
  343. return doBuildRowCreateRow(ctx, expr);
  344. case no_newusertable:
  345. case no_hqlproject:
  346. case no_temprow:
  347. case no_projectrow:
  348. return doBuildRowViaTemp(ctx, expr);
  349. case no_null:
  350. return doBuildRowNull(ctx, expr);
  351. case no_typetransfer:
  352. {
  353. CHqlBoundExpr bound;
  354. IHqlExpression * value = expr->queryChild(1);
  355. if (value->isDatarow())
  356. buildAddress(ctx, value, bound);
  357. else
  358. buildExpr(ctx, value, bound);
  359. OwnedHqlExpr cursorExpr = createValue(no_implicitcast, makeReferenceModifier(expr->getType()), LINK(bound.expr));
  360. row = bindRow(ctx, expr, cursorExpr);
  361. break;
  362. }
  363. case no_getresult:
  364. {
  365. IHqlExpression * seqAttr = expr->queryProperty(sequenceAtom);
  366. IHqlExpression * nameAttr = expr->queryProperty(namedAtom);
  367. IHqlExpression * record = expr->queryRecord();
  368. OwnedHqlExpr serializedRecord = getSerializedForm(record);
  369. OwnedHqlExpr temp = createDatasetF(no_getresult, LINK(serializedRecord), LINK(seqAttr), LINK(nameAttr), NULL);
  370. OwnedHqlExpr row = createRow(no_selectnth, LINK(temp), createComma(getSizetConstant(1), createAttribute(noBoundCheckAtom)));
  371. row.setown(ensureDeserialized(row, expr->queryType()));
  372. return buildNewRow(ctx, row);
  373. }
  374. case no_matchattr:
  375. return doBuildRowMatchAttr(ctx, expr);
  376. case no_matchrow:
  377. return doBuildRowMatchRow(ctx, expr, true);
  378. case no_getgraphresult:
  379. case no_call:
  380. case no_externalcall:
  381. case no_alias:
  382. case no_translated:
  383. case no_libraryinput:
  384. {
  385. CHqlBoundExpr bound;
  386. buildExpr(ctx, expr, bound);
  387. Owned<ITypeInfo> rawType = removeModifier(expr->queryType(), typemod_ref);
  388. OwnedHqlExpr cursorExpr = createValue(no_implicitcast, makeReferenceModifier(LINK(rawType)), LINK(bound.expr));
  389. row = bindRow(ctx, expr, cursorExpr);
  390. if (bound.length)
  391. {
  392. OwnedHqlExpr sizeOfRow = createSizeof(row->querySelector());
  393. ctx.associateExpr(sizeOfRow, bound.length);
  394. }
  395. //We could associate the original expression to allow better cse for child datasets in transforms, but it doesn't actually improve any examples
  396. //IHqlExpression * original = queryPropertyChild(expr, _original_Atom, 0);
  397. //if (original)
  398. // bindRow(ctx, original, cursorExpr)->setResultAlias();
  399. break;//return createReferenceSelector(cursor);
  400. }
  401. case no_comma:
  402. case no_compound:
  403. buildStmt(ctx, expr->queryChild(0));
  404. return buildNewRow(ctx, expr->queryChild(1));
  405. case no_select:
  406. {
  407. #ifdef _DEBUG
  408. IHqlExpression * field = expr->queryChild(1);
  409. #endif
  410. Owned<IReferenceSelector> selector;
  411. if (isNewSelector(expr))
  412. selector.setown(buildNewRow(ctx, expr->queryChild(0)));
  413. else
  414. selector.setown(buildActiveRow(ctx, expr->queryChild(0)));
  415. return selector->select(ctx, expr);
  416. }
  417. //If called because known to be a single row.
  418. case no_datasetfromrow:
  419. case no_nofold:
  420. case no_nohoist:
  421. case no_section:
  422. case no_sectioninput:
  423. return buildNewRow(ctx, expr->queryChild(0));
  424. case no_skip:
  425. {
  426. buildStmt(ctx, expr);
  427. OwnedHqlExpr null = createNullExpr(expr);
  428. return buildNewRow(ctx, null);
  429. }
  430. case no_alias_scope:
  431. {
  432. expandAliasScope(ctx, expr);
  433. return buildNewRow(ctx, expr->queryChild(0));
  434. }
  435. case no_split:
  436. throwUnexpected();
  437. //not at all sure about this.....
  438. return buildNewRow(ctx, expr->queryChild(0));
  439. default:
  440. {
  441. HqlExprAssociation * match;
  442. if (expr->isDataset())
  443. match = ctx.queryAssociation(expr->queryNormalizedSelector(), AssocCursor, NULL);
  444. else
  445. match = ctx.queryAssociation(expr, AssocRow, NULL);
  446. if (match)
  447. {
  448. BoundRow * row = (BoundRow *)match;
  449. IReferenceSelector * alias = row->queryAlias();
  450. if (alias)
  451. return LINK(alias);
  452. return createReferenceSelector(row, expr);
  453. }
  454. UNIMPLEMENTED_XY("row", getOpString(expr->getOperator()));
  455. }
  456. }
  457. assertex(row);
  458. return createReferenceSelector(row);
  459. }
  460. IReferenceSelector * HqlCppTranslator::buildActiveRow(BuildCtx & ctx, IHqlExpression * expr)
  461. {
  462. node_operator op = expr->getOperator();
  463. switch (op)
  464. {
  465. case no_left:
  466. case no_right:
  467. case no_self:
  468. case no_top:
  469. case no_activetable:
  470. case no_selfref: // shouldn't ever occur...
  471. //All selectors should be listed here...
  472. break;
  473. case no_activerow:
  474. return buildActiveRow(ctx, expr->queryChild(0));
  475. default:
  476. if (!expr->isDataset() && !expr->isDictionary())
  477. return buildNewRow(ctx, expr);
  478. break;
  479. }
  480. HqlExprAssociation * match = ctx.queryAssociation(expr->queryNormalizedSelector(), AssocCursor, NULL);
  481. if (match)
  482. {
  483. BoundRow * row = (BoundRow *)match;
  484. IReferenceSelector * alias = row->queryAlias();
  485. if (alias)
  486. return LINK(alias);
  487. return createReferenceSelector(row, expr);
  488. }
  489. switch (op)
  490. {
  491. case no_select:
  492. {
  493. #ifdef _DEBUG
  494. IHqlExpression * field = expr->queryChild(1);
  495. #endif
  496. Owned<IReferenceSelector> selector = buildNewOrActiveRow(ctx, expr->queryChild(0), isNewSelector(expr));
  497. return selector->select(ctx, expr);
  498. }
  499. case no_id2blob:
  500. return doBuildRowIdToBlob(ctx, expr, false);
  501. }
  502. StringBuffer tablename;
  503. getExprIdentifier(tablename, expr);
  504. traceExpression("Dataset not found", expr);
  505. RowAssociationIterator iter(ctx);
  506. ForEach(iter)
  507. {
  508. BoundRow & cur = iter.get();
  509. traceExpression("BoundCursor:", cur.querySelector());
  510. }
  511. throwError1(HQLERR_DatasetNotActive, tablename.str());
  512. return NULL; //remove warning about control paths
  513. }
  514. //---------------------------------------------------------------------------
  515. void HqlCppTranslator::doBuildExprAggregate(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt)
  516. {
  517. OwnedHqlExpr normalized = normalizeAnyDatasetAliases(expr);
  518. if (expr != normalized)
  519. {
  520. buildExpr(ctx, normalized, tgt);
  521. return;
  522. }
  523. node_operator op = expr->getOperator();
  524. ITypeInfo * type = expr->queryType();
  525. ITypeInfo * tempType = op == no_count ? unsignedType : type;
  526. LoopInvariantHelper helper;
  527. BuildCtx aggctx(ctx);
  528. if (options.optimizeLoopInvariant)
  529. helper.getBestContext(aggctx, expr);
  530. CHqlBoundTarget result;
  531. createTempFor(aggctx, tempType, result, typemod_none, FormatNatural);
  532. doBuildAssignAggregate(aggctx, result, expr);
  533. tgt.setFromTarget(result);
  534. if (!isSameBasicType(type, tempType))
  535. tgt.expr.setown(createValue(no_implicitcast, LINK(type), tgt.expr.getClear()));
  536. if (expr->isPure())
  537. aggctx.associateExpr(expr, tgt);
  538. }
  539. void HqlCppTranslator::doBuildAssignAggregateLoop(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr, IHqlExpression * dataset, IHqlExpression * doneFirstVar)
  540. {
  541. node_operator op = expr->getOperator();
  542. switch (op)
  543. {
  544. case no_exists:
  545. {
  546. OwnedHqlExpr optimized = queryOptimizedExists(ctx, expr, dataset);
  547. if (optimized)
  548. {
  549. assignBound(ctx, target, optimized);
  550. return;
  551. }
  552. break;
  553. }
  554. case no_count:
  555. {
  556. CHqlBoundExpr temp;
  557. if (canBuildOptimizedCount(ctx, dataset, temp))
  558. {
  559. OwnedHqlExpr thisCount = temp.getTranslatedExpr();
  560. buildIncrementAssign(ctx, target, thisCount);
  561. return;
  562. }
  563. break;
  564. }
  565. }
  566. switch (dataset->getOperator())
  567. {
  568. case no_if:
  569. {
  570. BuildCtx subctx(ctx);
  571. IHqlStmt * stmt = buildFilterViaExpr(subctx, dataset->queryChild(0));
  572. doBuildAssignAggregateLoop(subctx, target, expr, dataset->queryChild(1), doneFirstVar);
  573. subctx.selectElse(stmt);
  574. doBuildAssignAggregateLoop(subctx, target, expr, dataset->queryChild(2), doneFirstVar);
  575. return;
  576. }
  577. case no_addfiles:
  578. {
  579. doBuildAssignAggregateLoop(ctx, target, expr, dataset->queryChild(0), doneFirstVar);
  580. doBuildAssignAggregateLoop(ctx, target, expr, dataset->queryChild(1), doneFirstVar);
  581. return;
  582. }
  583. case no_null:
  584. return;
  585. }
  586. LinkedHqlExpr arg = expr->queryChild(1);
  587. IHqlExpression * oldDataset = expr->queryChild(0);
  588. //If no_if or no_addfiles has been optimized above then the selector for the argument will have changed => map it.
  589. if (arg && (dataset != oldDataset))
  590. arg.setown(replaceSelector(arg, oldDataset, dataset));
  591. bool needToBreak = (op == no_exists);
  592. if (needToBreak)
  593. {
  594. //if it can have at most one row (fairly strange code!) then don't add a break
  595. //unless it was deliberately a choosen to restrict the number of iterations.
  596. if (hasNoMoreRowsThan(dataset, 1) && (dataset->getOperator() != no_choosen))
  597. needToBreak = false;
  598. }
  599. BuildCtx loopctx(ctx);
  600. buildDatasetIterate(loopctx, dataset, needToBreak);
  601. switch (op)
  602. {
  603. case no_exists:
  604. buildExprAssign(loopctx, target, queryBoolExpr(true));
  605. if (needToBreak)
  606. loopctx.addBreak();
  607. break;
  608. case no_count:
  609. {
  610. OwnedHqlExpr inc = createValue(no_postinc, makeVoidType(), LINK(target.expr));
  611. loopctx.addExpr(inc);
  612. break;
  613. }
  614. case no_sum:
  615. {
  616. OwnedHqlExpr cseArg = options.spotCSE ? spotScalarCSE(arg) : LINK(arg);
  617. buildIncrementAssign(loopctx, target, cseArg);
  618. break;
  619. }
  620. case no_min:
  621. case no_max:
  622. {
  623. BuildCtx maxctx(loopctx);
  624. OwnedHqlExpr resultExpr = target.getTranslatedExpr();
  625. OwnedHqlExpr cseArg = options.spotCSE ? spotScalarCSE(arg) : LINK(arg);
  626. OwnedHqlExpr simpleArg = buildSimplifyExpr(loopctx, cseArg);
  627. OwnedHqlExpr test = createBoolExpr((op == no_min) ? no_lt : no_gt, LINK(simpleArg), LINK(resultExpr));
  628. if (doneFirstVar)
  629. {
  630. IHqlExpression * firstTest = createValue(no_not, makeBoolType(), LINK(doneFirstVar));
  631. test.setown(createBoolExpr(no_or, firstTest, test.getClear()));
  632. }
  633. buildFilter(maxctx, test);
  634. buildExprAssign(maxctx, target, simpleArg);
  635. if (doneFirstVar)
  636. buildAssignToTemp(maxctx, doneFirstVar, queryBoolExpr(true));
  637. break;
  638. }
  639. default:
  640. assertex(!"unknown aggregate on child datasets");
  641. break;
  642. }
  643. }
  644. bool assignAggregateDirect(const CHqlBoundTarget & target, IHqlExpression * expr)
  645. {
  646. node_operator op = expr->getOperator();
  647. ITypeInfo * type = expr->queryType();
  648. ITypeInfo * tempType = op == no_count ? unsignedType : type;
  649. if (!isSameUnqualifiedType(target.queryType(), tempType))
  650. return false;
  651. //For exists/count/sum use a temporary variable, and then assign rather than accumulating directly in the target
  652. switch (op)
  653. {
  654. case no_sum:
  655. if (type->getTypeCode() != type_int)
  656. break;
  657. //fall through
  658. case no_exists:
  659. case no_count:
  660. if (target.expr->getOperator() != no_variable)
  661. return false;
  662. break;
  663. }
  664. return true;
  665. }
  666. static bool isNullValueMinimumValue(ITypeInfo * type)
  667. {
  668. switch (type->getTypeCode())
  669. {
  670. case type_int:
  671. case type_swapint:
  672. case type_decimal:
  673. return !type->isSigned();
  674. case type_data:
  675. case type_qstring:
  676. return true;
  677. }
  678. return false;
  679. }
  680. void HqlCppTranslator::doBuildAssignAggregate(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * _expr)
  681. {
  682. OwnedHqlExpr expr = normalizeAnyDatasetAliases(_expr);
  683. if (assignAggregateDirect(target, expr))
  684. {
  685. IHqlExpression * dataset = expr->queryChild(0);
  686. OwnedHqlExpr resultExpr = target.getTranslatedExpr();
  687. node_operator op = expr->getOperator();
  688. switch (op)
  689. {
  690. case no_exists:
  691. buildExprAssign(ctx, target, queryBoolExpr(false));
  692. break;
  693. default:
  694. {
  695. OwnedHqlExpr null = createNullExpr(target.queryType());
  696. buildExprAssign(ctx, target, null);
  697. break;
  698. }
  699. }
  700. OwnedHqlExpr doneFirstVar;
  701. if ((op == no_min) || ((op == no_max) && !isNullValueMinimumValue(target.queryType())))
  702. {
  703. doneFirstVar.setown(ctx.getTempDeclare(queryBoolType(), queryBoolExpr(false)));
  704. }
  705. doBuildAssignAggregateLoop(ctx, target, expr, dataset, doneFirstVar);
  706. }
  707. else
  708. {
  709. doBuildExprAssign(ctx, target, expr);
  710. }
  711. }
  712. //---------------------------------------------------------------------------
  713. bool HqlCppTranslator::canBuildOptimizedCount(BuildCtx & ctx, IHqlExpression * dataset, CHqlBoundExpr & tgt)
  714. {
  715. switch (dataset->getOperator())
  716. {
  717. case no_select:
  718. {
  719. if (isMultiLevelDatasetSelector(dataset, false))
  720. return false;
  721. Owned<IReferenceSelector> selector = buildReference(ctx, dataset);
  722. CHqlBoundExpr temp;
  723. selector->get(ctx, temp);
  724. tgt.expr.setown(getBoundCount(temp));
  725. return true;
  726. }
  727. break;
  728. default:
  729. if (!options.tempDatasetsUseLinkedRows)
  730. break;
  731. if (!alwaysEvaluatesToBound(dataset))
  732. break;
  733. //fall through
  734. case no_rows:
  735. case no_null:
  736. {
  737. CHqlBoundExpr temp;
  738. buildDataset(ctx, dataset, temp, FormatNatural);
  739. tgt.expr.setown(getBoundCount(temp));
  740. return true;
  741. }
  742. }
  743. #if 0
  744. //This is improves a few obscure cases (normally in the global context). I'm not convinced it is worth the extra cycles.
  745. //Could also remove the bound.count test.
  746. CHqlBoundExpr bound;
  747. if (ctx.getMatchExpr(dataset, bound) && bound.count)
  748. {
  749. tgt.expr.setown(getBoundCount(bound));
  750. return true;
  751. }
  752. #endif
  753. return false;
  754. }
  755. void HqlCppTranslator::doBuildExprCount(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt)
  756. {
  757. if (expr->hasProperty(keyedAtom))
  758. throwError(HQLERR_KeyedCountNonKeyable);
  759. IHqlExpression * dataset = expr->queryChild(0);
  760. CHqlBoundExpr temp;
  761. if (canBuildOptimizedCount(ctx, dataset, temp))
  762. {
  763. OwnedHqlExpr translated = temp.getTranslatedExpr();
  764. OwnedHqlExpr cast = ensureExprType(translated, expr->queryType());
  765. buildExpr(ctx, cast, tgt);
  766. }
  767. else
  768. doBuildExprAggregate(ctx, expr, tgt);
  769. }
  770. //---------------------------------------------------------------------------
  771. IHqlExpression * HqlCppTranslator::queryOptimizedExists(BuildCtx & ctx, IHqlExpression * expr, IHqlExpression * dataset)
  772. {
  773. node_operator dsOp = dataset->getOperator();
  774. //really this is isSimple()
  775. CHqlBoundExpr optimized;
  776. bool canOptimizeCount = canBuildOptimizedCount(ctx, dataset, optimized);
  777. node_operator op = (expr->getOperator() == no_exists) ? no_ne : no_eq;
  778. bool specialCase = false;
  779. switch (dsOp)
  780. {
  781. case no_select:
  782. specialCase = canOptimizeCount;
  783. break;
  784. default:
  785. specialCase = !canOptimizeCount && alwaysEvaluatesToBound(dataset);
  786. break;
  787. }
  788. if (specialCase)
  789. {
  790. CHqlBoundExpr temp;
  791. buildDataset(ctx, dataset, temp, FormatNatural);
  792. IHqlExpression * test;
  793. if (temp.count)
  794. test = LINK(temp.count);
  795. else
  796. test = getBoundSize(temp);
  797. return createBoolExpr(op, test, createConstant(test->queryType()->castFrom(false, 0)));
  798. }
  799. else if (canOptimizeCount)
  800. {
  801. IHqlExpression * count = optimized.expr;
  802. return createBoolExpr(op, LINK(count), createConstant(count->queryType()->castFrom(false, 0)));
  803. }
  804. return NULL;
  805. }
  806. void HqlCppTranslator::doBuildExprExists(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt)
  807. {
  808. OwnedHqlExpr optimized = queryOptimizedExists(ctx, expr, expr->queryChild(0));
  809. if (optimized)
  810. tgt.expr.setown(optimized.getClear());
  811. else
  812. doBuildExprAggregate(ctx, expr, tgt);
  813. }
  814. //---------------------------------------------------------------------------
  815. static IHqlExpression * createMinMax(node_operator compareOp, ITypeInfo * type, IHqlExpression * left, IHqlExpression * right)
  816. {
  817. return createValue(no_if, LINK(type),
  818. createBoolExpr(compareOp, LINK(left), LINK(right)),
  819. LINK(left), LINK(right));
  820. }
  821. bool HqlCppTranslator::doBuildAggregateMinMaxList(BuildCtx & ctx, const CHqlBoundTarget * target, IHqlExpression * expr, IHqlExpression * list, CHqlBoundExpr * tgt, node_operator compareOp)
  822. {
  823. OwnedHqlExpr value;
  824. unsigned max = list->numChildren();
  825. switch(max)
  826. {
  827. case 0:
  828. value.setown(createNullExpr(expr));
  829. break;
  830. case 1:
  831. value.set(list->queryChild(0));
  832. break;
  833. case 2:
  834. case 3:
  835. {
  836. OwnedHqlExpr simple[3];
  837. for (unsigned i=0; i < max; i++)
  838. {
  839. CHqlBoundExpr bound;
  840. buildSimpleExpr(ctx, list->queryChild(i), bound);
  841. simple[i].setown(bound.getTranslatedExpr());
  842. }
  843. ITypeInfo * type = expr->queryType();
  844. if (max == 2)
  845. value.setown(createMinMax(compareOp, type, simple[0], simple[1]));
  846. else
  847. {
  848. OwnedHqlExpr cmp02 = createMinMax(compareOp, type, simple[0], simple[2]);
  849. OwnedHqlExpr cmp12 = createMinMax(compareOp, type, simple[1], simple[2]);
  850. value.setown(createValue(no_if, expr->getType(),
  851. createBoolExpr(compareOp, LINK(simple[0]), LINK(simple[1])),
  852. LINK(cmp02), LINK(cmp12)));
  853. }
  854. }
  855. }
  856. if (value)
  857. {
  858. buildExprOrAssign(ctx, target, value, tgt);
  859. return true;
  860. }
  861. {
  862. CHqlBoundTarget temp;
  863. createTempFor(ctx, expr, temp);
  864. buildExprAssign(ctx, temp, list->queryChild(0));
  865. OwnedHqlExpr best = temp.getTranslatedExpr();
  866. for (unsigned i=1; i < list->numChildren(); i++)
  867. {
  868. CHqlBoundExpr bound;
  869. buildSimpleExpr(ctx, list->queryChild(i), bound);
  870. OwnedHqlExpr simple = bound.getTranslatedExpr();
  871. BuildCtx subctx(ctx);
  872. OwnedHqlExpr cond = createBoolExpr(compareOp, LINK(simple), LINK(best));
  873. buildFilter(subctx, cond);
  874. buildExprAssign(subctx, temp, simple);
  875. }
  876. buildExprOrAssign(ctx, target, best, tgt);
  877. return true;
  878. }
  879. }
  880. void HqlCppTranslator::doBuildAggregateList(BuildCtx & ctx, const CHqlBoundTarget * target, IHqlExpression * expr, CHqlBoundExpr * tgt)
  881. {
  882. OwnedHqlExpr list = normalizeListCasts(expr->queryChild(0));
  883. if (list->getOperator() == no_alias_scope)
  884. {
  885. expandAliasScope(ctx, list);
  886. list.set(list->queryChild(0));
  887. }
  888. node_operator aggOp;
  889. switch (expr->getOperator())
  890. {
  891. case no_existslist:
  892. {
  893. //Fixed length lists should have been optimized away
  894. CHqlBoundExpr bound;
  895. buildExpr(ctx, list, bound);
  896. OwnedHqlExpr test;
  897. if (bound.count)
  898. test.set(bound.count);
  899. else
  900. test.setown(getBoundLength(bound));
  901. OwnedHqlExpr value = createValue(no_ne, makeBoolType(), LINK(test), ensureExprType(queryZero(), test->queryType()));
  902. OwnedHqlExpr translated = createTranslated(value);
  903. buildExprOrAssign(ctx, target, translated, tgt);
  904. return;
  905. }
  906. case no_countlist:
  907. {
  908. //Fixed length lists should have been optimized away
  909. CHqlBoundExpr bound;
  910. buildExpr(ctx, list, bound);
  911. OwnedHqlExpr test = getBoundCount(bound);
  912. OwnedHqlExpr value = ensureExprType(test, expr->queryType());
  913. OwnedHqlExpr translated = createTranslated(value);
  914. buildExprOrAssign(ctx, target, translated, tgt);
  915. return;
  916. }
  917. case no_sumlist:
  918. aggOp = no_sumgroup;
  919. if (list->getOperator() == no_list)
  920. {
  921. HqlExprArray args;
  922. ITypeInfo * exprType = expr->queryType();
  923. ForEachChild(i, list)
  924. args.append(*ensureExprType(list->queryChild(i), exprType));
  925. OwnedHqlExpr balanced = createBalanced(no_add, exprType, args);
  926. if (!balanced)
  927. balanced.setown(ensureExprType(queryZero(), exprType));
  928. buildExprOrAssign(ctx, target, balanced, tgt);
  929. return;
  930. }
  931. //special case fixed length lists
  932. break;
  933. case no_minlist:
  934. aggOp = no_mingroup;
  935. if (list->getOperator() == no_list)
  936. {
  937. if (doBuildAggregateMinMaxList(ctx, target, expr, list, tgt, no_lt))
  938. return;
  939. }
  940. break;
  941. case no_maxlist:
  942. aggOp = no_maxgroup;
  943. if (list->getOperator() == no_list)
  944. {
  945. if (doBuildAggregateMinMaxList(ctx, target, expr, list, tgt, no_gt))
  946. return;
  947. }
  948. break;
  949. default:
  950. throwUnexpectedOp(expr->getOperator());
  951. }
  952. ITypeInfo * elemType = list->queryType()->queryChildType();
  953. if (!elemType) elemType = defaultIntegralType;
  954. //Default implementation in terms of a dataset
  955. OwnedHqlExpr field = createField(valueAtom, LINK(elemType), NULL);
  956. OwnedHqlExpr record = createRecord(field);
  957. OwnedHqlExpr ds = createDataset(no_temptable, LINK(list), LINK(record));
  958. OwnedHqlExpr aggField = createField(valueAtom, expr->getType(), NULL);
  959. OwnedHqlExpr aggRecord = createRecord(aggField);
  960. OwnedHqlExpr self = createSelector(no_self, aggRecord, NULL);
  961. OwnedHqlExpr aggExpr = createValue(aggOp, expr->getType(), createSelectExpr(LINK(ds), LINK(field)));
  962. OwnedHqlExpr aggAssign = createAssign(createSelectExpr(LINK(self), LINK(aggField)), LINK(aggExpr));
  963. OwnedHqlExpr aggTransform = createValue(no_newtransform, makeTransformType(aggRecord->getType()), LINK(aggAssign));
  964. OwnedHqlExpr agg = createDataset(no_newaggregate, LINK(ds), createComma(LINK(aggRecord), LINK(aggTransform)));
  965. OwnedHqlExpr result = createNewSelectExpr(createRow(no_selectnth, LINK(agg), createConstantOne()), LINK(aggField));
  966. buildExprOrAssign(ctx, target, result, tgt);
  967. }
  968. //---------------------------------------------------------------------------
  969. static HqlTransformerInfo graphIndependanceCheckerInfo("GraphIndependanceChecker");
  970. class GraphIndependanceChecker : public NewHqlTransformer
  971. {
  972. public:
  973. GraphIndependanceChecker(IHqlExpression * _graph) : NewHqlTransformer(graphIndependanceCheckerInfo), graph(_graph) { independent = true; }
  974. void analyseExpr(IHqlExpression * expr)
  975. {
  976. if (!independent || alreadyVisited(expr))
  977. return;
  978. switch (expr->getOperator())
  979. {
  980. case no_getgraphresult:
  981. case no_getgraphloopresultset:
  982. case no_getgraphloopresult:
  983. if (expr->queryChild(1) == graph)
  984. {
  985. independent = false;
  986. return;
  987. }
  988. break;
  989. }
  990. NewHqlTransformer::analyseExpr(expr);
  991. }
  992. inline bool isIndependent() const { return independent; }
  993. protected:
  994. LinkedHqlExpr graph;
  995. bool independent;
  996. };
  997. bool isGraphIndependent(IHqlExpression * expr, IHqlExpression * graph)
  998. {
  999. switch (expr->getOperator())
  1000. {
  1001. case no_constant:
  1002. return true;
  1003. }
  1004. GraphIndependanceChecker checker(graph);
  1005. checker.analyse(expr, 0);
  1006. return checker.isIndependent();
  1007. }
  1008. ///--------------------------------------------------------------------------------------------------------------------
  1009. IHqlExpression * createCounterAsGraphResult(IHqlExpression * counter, IHqlExpression * represents, unsigned seq)
  1010. {
  1011. OwnedHqlExpr value = createScalarFromGraphResult(counter->queryType(), unsignedType, represents, seq);
  1012. OwnedHqlExpr internalAttr = createAttribute(internalAtom);
  1013. return createAlias(value, internalAttr);
  1014. }
  1015. ChildGraphExprBuilder::ChildGraphExprBuilder(unsigned _numInputs)
  1016. : numInputs(_numInputs)
  1017. {
  1018. numOutputs=0;
  1019. represents.setown(createAttribute(graphAtom, createUniqueId()));
  1020. resultsExpr.setown(createAttribute(resultsAtom, LINK(represents)));
  1021. }
  1022. IHqlExpression * ChildGraphExprBuilder::addDataset(IHqlExpression * expr)
  1023. {
  1024. OwnedHqlExpr resultNumExpr;
  1025. ForEachItemIn(i, results)
  1026. {
  1027. IHqlExpression & curSetResult = results.item(i);
  1028. if (expr->queryBody() == curSetResult.queryChild(0)->queryBody())
  1029. {
  1030. resultNumExpr.set(curSetResult.queryChild(2));
  1031. break;
  1032. }
  1033. }
  1034. if (!resultNumExpr)
  1035. {
  1036. resultNumExpr.setown(getSizetConstant(numResults()));
  1037. results.append(*createValue(no_setgraphresult, makeVoidType(), LINK(expr), LINK(represents), LINK(resultNumExpr)));
  1038. numOutputs++;
  1039. }
  1040. HqlExprArray args;
  1041. args.append(*LINK(expr->queryRecord()));
  1042. args.append(*LINK(represents));
  1043. args.append(*LINK(resultNumExpr));
  1044. if (isGrouped(expr))
  1045. args.append(*createAttribute(groupedAtom));
  1046. if (!expr->isDataset())
  1047. args.append(*createAttribute(rowAtom));
  1048. args.append(*createAttribute(externalAtom, LINK(resultsExpr)));
  1049. args.append(*createAttribute(_original_Atom, LINK(expr)));
  1050. IHqlExpression * recordCountAttr = queryRecordCountInfo(expr);
  1051. if (recordCountAttr)
  1052. args.append(*LINK(recordCountAttr));
  1053. OwnedHqlExpr ret = createDataset(no_getgraphresult, args);
  1054. if (expr->isDatarow())
  1055. ret.setown(createRow(no_selectnth, LINK(ret), createComma(getSizetConstant(1), createAttribute(noBoundCheckAtom))));
  1056. return ret.getClear();
  1057. }
  1058. void ChildGraphExprBuilder::addAction(IHqlExpression * expr)
  1059. {
  1060. results.append(*LINK(expr));
  1061. }
  1062. unsigned ChildGraphExprBuilder::addInput()
  1063. {
  1064. unsigned id = numResults();
  1065. numInputs++;
  1066. return id;
  1067. }
  1068. IHqlExpression * ChildGraphExprBuilder::getGraph()
  1069. {
  1070. HqlExprArray args;
  1071. args.append(*LINK(represents));
  1072. args.append(*getSizetConstant(numResults()));
  1073. args.append(*createActionList(results));
  1074. return createValue(no_childquery, makeVoidType(), args);
  1075. }
  1076. //---------------------------------------------------------------------------
  1077. // Child dataset processing
  1078. ChildGraphBuilder::ChildGraphBuilder(HqlCppTranslator & _translator, IHqlExpression * subgraph)
  1079. : translator(_translator)
  1080. {
  1081. represents.set(subgraph->queryChild(0));
  1082. id = translator.nextActivityId();
  1083. appendUniqueId(instanceName.append("child"), id);
  1084. instanceExpr.setown(createQuoted(instanceName, makeBoolType()));
  1085. resultsExpr.setown(createAttribute(resultsAtom, LINK(represents)));
  1086. StringBuffer s;
  1087. resultInstanceExpr.setown(createQuoted(appendUniqueId(s.append("res"), id), makeBoolType()));
  1088. numResults = (unsigned)getIntValue(subgraph->queryChild(1));
  1089. IHqlExpression * actions = subgraph->queryChild(2);
  1090. actions->unwindList(results, no_actionlist);
  1091. }
  1092. void ChildGraphBuilder::generateGraph(BuildCtx & ctx)
  1093. {
  1094. BuildCtx graphctx(ctx);
  1095. //Make sure at least one results - because currently that's how we determine if new resourcing is being used
  1096. //Remove this line once all engines use the new child queries exclusively
  1097. if (numResults == 0) numResults++;
  1098. OwnedHqlExpr query = createActionList(results);
  1099. OwnedHqlExpr resourced = translator.getResourcedChildGraph(graphctx, query, represents, numResults, no_none);
  1100. Owned<ParentExtract> extractBuilder = translator.createExtractBuilder(graphctx, PETchild, represents, resourced, true);
  1101. if (!translator.queryOptions().serializeRowsetInExtract)
  1102. extractBuilder->setAllowDestructor();
  1103. translator.beginExtract(graphctx, extractBuilder);
  1104. translator.doBuildThorSubGraph(graphctx, resourced, SubGraphChild, id, represents);
  1105. EvalContext * instance = translator.queryEvalContext(graphctx);
  1106. OwnedHqlExpr retInstanceExpr;
  1107. if (instance && !translator.insideOnCreate(graphctx))
  1108. retInstanceExpr.setown(instance->createGraphLookup(id, true));
  1109. else
  1110. retInstanceExpr.setown(translator.doCreateGraphLookup(graphctx, graphctx, id, "this", true));
  1111. assertex(retInstanceExpr == instanceExpr);
  1112. CHqlBoundExpr boundExtract;
  1113. extractBuilder->endCreateExtract(boundExtract);
  1114. HqlExprArray args;
  1115. args.append(*LINK(instanceExpr));
  1116. args.append(*createTranslated(boundExtract.length));
  1117. args.append(*boundExtract.getTranslatedExpr());
  1118. OwnedHqlExpr call = translator.bindFunctionCall(evaluateChildQueryInstanceAtom, args);
  1119. CHqlBoundExpr bound;
  1120. translator.buildExpr(graphctx, call, bound);
  1121. StringBuffer s;
  1122. s.append("Owned<IEclGraphResults> ");
  1123. translator.generateExprCpp(s, resultInstanceExpr);
  1124. s.append(" = ");
  1125. translator.generateExprCpp(s, bound.expr);
  1126. s.append(";");
  1127. graphctx.addQuoted(s);
  1128. translator.endExtract(graphctx, extractBuilder);
  1129. ctx.associateExpr(resultsExpr, resultInstanceExpr);
  1130. }
  1131. void ChildGraphBuilder::generatePrefetchGraph(BuildCtx & _ctx, OwnedHqlExpr * retGraphExpr)
  1132. {
  1133. BuildCtx ctx(_ctx);
  1134. ctx.addGroup();
  1135. BuildCtx aliasctx(ctx);
  1136. aliasctx.addGroup();
  1137. OwnedHqlExpr query = createActionList(results);
  1138. OwnedHqlExpr resourced = translator.getResourcedChildGraph(ctx, query, represents, numResults, no_none);
  1139. Owned<ParentExtract> extractBuilder = translator.createExtractBuilder(ctx, PETchild, represents, resourced, false);
  1140. createBuilderAlias(aliasctx, extractBuilder);
  1141. translator.beginExtract(ctx, extractBuilder);
  1142. translator.doBuildThorSubGraph(ctx, resourced, SubGraphChild, id, represents);
  1143. EvalContext * instance = translator.queryEvalContext(ctx);
  1144. OwnedHqlExpr retInstanceExpr;
  1145. assertex(instance && !translator.insideOnCreate(ctx));
  1146. retInstanceExpr.setown(instance->createGraphLookup(id, true));
  1147. assertex(retInstanceExpr == instanceExpr);
  1148. retGraphExpr->setown(retInstanceExpr.getClear());
  1149. }
  1150. void ChildGraphBuilder::createBuilderAlias(BuildCtx & ctx, ParentExtract * extractBuilder)
  1151. {
  1152. StringBuffer s;
  1153. s.append("rtlRowBuilder & ");
  1154. translator.generateExprCpp(s, extractBuilder->queryExtractName());
  1155. s.append(" = builder;");
  1156. ctx.addQuoted(s);
  1157. }
  1158. unique_id_t ChildGraphBuilder::buildLoopBody(BuildCtx & ctx, bool multiInstance)
  1159. {
  1160. BuildCtx subctx(ctx);
  1161. subctx.addGroup();
  1162. OwnedHqlExpr query = createActionList(results);
  1163. OwnedHqlExpr resourced = translator.getResourcedChildGraph(ctx, query, represents, numResults, no_loop);
  1164. //Add a flag to indicate multi instance
  1165. if (multiInstance)
  1166. resourced.setown(appendOwnedOperand(resourced, createAttribute(multiInstanceAtom)));
  1167. bool isGlobalThorLoop = translator.targetThor() && !translator.insideChildQuery(ctx);
  1168. Owned<ParentExtract> extractBuilder = isGlobalThorLoop ? translator.createExtractBuilder(ctx, PETloop, represents, GraphRemote, false)
  1169. : translator.createExtractBuilder(ctx, PETloop, represents, resourced, false);
  1170. createBuilderAlias(subctx, extractBuilder);
  1171. translator.beginExtract(ctx, extractBuilder);
  1172. translator.doBuildThorSubGraph(ctx, resourced, SubGraphChild, id, represents);
  1173. translator.endExtract(ctx, extractBuilder);
  1174. return id;
  1175. }
  1176. static HqlTransformerInfo graphLoopReplacerInfo("GraphLoopReplacer");
  1177. class GraphLoopReplacer : public NewHqlTransformer
  1178. {
  1179. public:
  1180. GraphLoopReplacer(IHqlExpression * _rowsid, IHqlExpression * _represents, IHqlExpression * _counter, bool _isParallel) :
  1181. NewHqlTransformer(graphLoopReplacerInfo), rowsid(_rowsid), represents(_represents), counter(_counter), isParallel(_isParallel)
  1182. {
  1183. }
  1184. virtual IHqlExpression * createTransformed(IHqlExpression * expr)
  1185. {
  1186. switch (expr->getOperator())
  1187. {
  1188. case no_counter:
  1189. if (expr->queryBody() == counter)
  1190. {
  1191. if (isParallel)
  1192. {
  1193. HqlExprArray args;
  1194. args.append(*LINK(represents));
  1195. // unwindChildren(args, expr);
  1196. OwnedHqlExpr ret = createValue(no_loopcounter, expr->getType(), args);
  1197. //Yuk: Wrap this in an alias to ensure it is evaluated at the correct place.
  1198. //there has to be a better way..... We could...
  1199. //a) strictly defined when it can be evaluated - e.g., ctx->defines(graph) && (!parentctx || !parentctx->definesGraph)
  1200. //b) set a flag in the expression to indicate forced evaluation (even worse than the alias)
  1201. //c) add the code to evaluate no_loopcounter inside evaluateInContext
  1202. return createAlias(ret, internalAttrExpr);
  1203. }
  1204. else
  1205. {
  1206. counterResult.setown(createCounterAsGraphResult(counter, represents, 0));
  1207. return LINK(counterResult);
  1208. }
  1209. }
  1210. break;
  1211. case no_rowsetindex:
  1212. {
  1213. IHqlExpression * rowset = expr->queryChild(0);
  1214. if (rowset->getOperator() != no_rowset)
  1215. break;
  1216. IHqlExpression * rows = rowset->queryChild(0);
  1217. if (rows->queryChild(1) != rowsid)
  1218. break;
  1219. HqlExprArray args;
  1220. args.append(*LINK(rows->queryChild(0)->queryRecord()));
  1221. args.append(*LINK(represents));
  1222. args.append(*transform(expr->queryChild(1)));
  1223. return createDataset(no_getgraphloopresult, args);
  1224. }
  1225. case no_rowset:
  1226. {
  1227. IHqlExpression * rows = expr->queryChild(0);
  1228. if (rows->queryChild(1) != rowsid)
  1229. break;
  1230. HqlExprArray args;
  1231. args.append(*LINK(rows->queryChild(0)->queryRecord()));
  1232. args.append(*LINK(represents));
  1233. return createValue(no_getgraphloopresultset, expr->getType(), args);
  1234. }
  1235. }
  1236. return NewHqlTransformer::createTransformed(expr);
  1237. }
  1238. inline IHqlExpression * queryCounterResult() { return counterResult; }
  1239. protected:
  1240. IHqlExpression * rowsid;
  1241. IHqlExpression * represents;
  1242. IHqlExpression * counter;
  1243. OwnedHqlExpr counterResult;
  1244. bool isParallel;
  1245. };
  1246. unique_id_t ChildGraphBuilder::buildGraphLoopBody(BuildCtx & ctx, bool isParallel)
  1247. {
  1248. BuildCtx subctx(ctx);
  1249. subctx.addGroup();
  1250. OwnedHqlExpr query = createActionList(results);
  1251. translator.traceExpression("Before Loop resource", query);
  1252. OwnedHqlExpr resourced = translator.getResourcedChildGraph(ctx, query, represents, numResults, no_loop);
  1253. translator.traceExpression("After Loop resource", resourced);
  1254. //Add a flag to indicate multi instance
  1255. if (isParallel)
  1256. {
  1257. HqlExprArray args;
  1258. unwindChildren(args, resourced);
  1259. args.append(*createAttribute(multiInstanceAtom));
  1260. args.append(*createAttribute(delayedAtom));
  1261. resourced.setown(resourced->clone(args));
  1262. }
  1263. bool isGlobalThorLoop = translator.targetThor() && !translator.insideChildQuery(ctx);
  1264. Owned<ParentExtract> extractBuilder = isGlobalThorLoop ? translator.createExtractBuilder(ctx, PETloop, represents, GraphRemote, false)
  1265. : translator.createExtractBuilder(ctx, PETloop, represents, resourced, false);
  1266. createBuilderAlias(subctx, extractBuilder);
  1267. translator.beginExtract(ctx, extractBuilder);
  1268. translator.doBuildThorSubGraph(ctx, resourced, SubGraphChild, id, represents);
  1269. translator.endExtract(ctx, extractBuilder);
  1270. return id;
  1271. }
  1272. unique_id_t ChildGraphBuilder::buildRemoteGraph(BuildCtx & ctx)
  1273. {
  1274. BuildCtx subctx(ctx);
  1275. subctx.addGroup();
  1276. OwnedHqlExpr query = createActionList(results);
  1277. OwnedHqlExpr resourced = translator.getResourcedChildGraph(ctx, query, represents, numResults, no_allnodes);
  1278. Owned<ParentExtract> extractBuilder = translator.createExtractBuilder(ctx, PETremote, represents, GraphRemote, false);
  1279. createBuilderAlias(subctx, extractBuilder);
  1280. translator.beginExtract(ctx, extractBuilder);
  1281. translator.doBuildThorSubGraph(ctx, resourced, SubGraphChild, id, represents);
  1282. translator.endExtract(ctx, extractBuilder);
  1283. return id;
  1284. }
  1285. void HqlCppTranslator::buildChildGraph(BuildCtx & ctx, IHqlExpression * expr)
  1286. {
  1287. IHqlExpression * represents= expr->queryChild(0);
  1288. OwnedHqlExpr resultsExpr = createAttribute(resultsAtom, LINK(represents));
  1289. //Shouldn't really happen, but if this graph has already benn called just use the results
  1290. if (ctx.queryMatchExpr(resultsExpr))
  1291. return;
  1292. ChildGraphBuilder graphBuilder(*this, expr);
  1293. graphBuilder.generateGraph(ctx);
  1294. }
  1295. void HqlCppTranslator::beginExtract(BuildCtx & ctx, ParentExtract * extractBuilder)
  1296. {
  1297. ctx.associate(*extractBuilder);
  1298. }
  1299. void HqlCppTranslator::endExtract(BuildCtx & ctx, ParentExtract * extractBuilder)
  1300. {
  1301. extractBuilder->endUseExtract(ctx);
  1302. ctx.removeAssociation(extractBuilder);
  1303. }
  1304. void HqlCppTranslator::buildAssignChildDataset(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  1305. {
  1306. switch (expr->getOperator())
  1307. {
  1308. case no_call:
  1309. case no_externalcall:
  1310. case no_libraryinput:
  1311. buildDatasetAssign(ctx, target, expr);
  1312. return;
  1313. }
  1314. OwnedHqlExpr call;
  1315. {
  1316. ChildGraphExprBuilder builder(0);
  1317. call.setown(builder.addDataset(expr));
  1318. OwnedHqlExpr subquery = builder.getGraph();
  1319. buildStmt(ctx, subquery);
  1320. }
  1321. buildExprAssign(ctx, target, call);
  1322. }
  1323. IHqlExpression * HqlCppTranslator::getResourcedChildGraph(BuildCtx & ctx, IHqlExpression * expr, IHqlExpression * graphIdExpr, unsigned numResults, node_operator graphKind)
  1324. {
  1325. if (options.paranoidCheckNormalized || options.paranoidCheckDependencies)
  1326. DBGLOG("Before resourcing a child graph");
  1327. LinkedHqlExpr resourced = expr;
  1328. checkNormalized(ctx, resourced);
  1329. unsigned csfFlags = CSFindex|options.optimizeDiskFlag;
  1330. switch (targetClusterType)
  1331. {
  1332. case HThorCluster:
  1333. csfFlags |= CSFcompoundSpill;
  1334. break;
  1335. case ThorCluster:
  1336. case ThorLCRCluster:
  1337. //Don't compound spills inside a child query because it can cause non remote projects to become remote
  1338. //And we'll also probably be using the roxie code to implement
  1339. break;
  1340. case RoxieCluster:
  1341. break;
  1342. }
  1343. {
  1344. unsigned time = msTick();
  1345. CompoundSourceTransformer transformer(*this, CSFpreload|csfFlags);
  1346. resourced.setown(transformer.process(resourced));
  1347. checkNormalized(ctx, resourced);
  1348. DEBUG_TIMER("EclServer: tree transform: optimize disk read", msTick()-time);
  1349. }
  1350. if (options.optimizeChildGraph)
  1351. {
  1352. unsigned time = msTick();
  1353. traceExpression("BeforeOptimizeSub", resourced);
  1354. resourced.setown(optimizeHqlExpression(resourced, getOptimizeFlags()|HOOcompoundproject));
  1355. traceExpression("AfterOptimizeSub", resourced);
  1356. DEBUG_TIMER("EclServer: optimize graph", msTick()-time);
  1357. }
  1358. traceExpression("BeforeResourcing Child", resourced);
  1359. cycle_t time = msTick();
  1360. HqlExprCopyArray activeRows;
  1361. gatherActiveCursors(ctx, activeRows);
  1362. if (graphKind == no_loop)
  1363. {
  1364. bool insideChild = insideChildQuery(ctx);
  1365. resourced.setown(resourceLoopGraph(*this, activeRows, resourced, targetClusterType, graphIdExpr, &numResults, insideChild));
  1366. }
  1367. else
  1368. resourced.setown(resourceNewChildGraph(*this, activeRows, resourced, targetClusterType, graphIdExpr, &numResults));
  1369. DEBUG_TIMER("EclServer: resource graph", msTick()-time);
  1370. checkNormalized(ctx, resourced);
  1371. traceExpression("AfterResourcing Child", resourced);
  1372. //Convert queries on preloaded into compound activities - before resourcing so keyed gets done correctly
  1373. // Second attempt to spot compound disk reads - this time of spill files. Since resourcing has removed
  1374. // any sharing we don't need to bother about sharing.
  1375. {
  1376. unsigned time = msTick();
  1377. CompoundSourceTransformer transformer(*this, csfFlags);
  1378. resourced.setown(transformer.process(resourced));
  1379. DEBUG_TIMER("EclServer: tree transform: optimize disk read", msTick()-time);
  1380. }
  1381. //Now call the optimizer again - the main purpose is to move projects over limits and into compound index/disk reads
  1382. if (options.optimizeChildGraph)
  1383. {
  1384. unsigned time = msTick();
  1385. traceExpression("BeforeOptimize2", resourced);
  1386. resourced.setown(optimizeHqlExpression(resourced, getOptimizeFlags()|HOOcompoundproject));
  1387. traceExpression("AfterOptimize2", resourced);
  1388. DEBUG_TIMER("EclServer: optimize graph", msTick()-time);
  1389. }
  1390. if (numResults == 0) numResults++;
  1391. HqlExprArray children;
  1392. resourced->unwindList(children, no_actionlist);
  1393. children.append(*createAttribute(numResultsAtom, getSizetConstant(numResults)));
  1394. children.append(*LINK(graphIdExpr));
  1395. resourced.setown(createValue(no_subgraph, makeVoidType(), children));
  1396. if (options.paranoidCheckNormalized || options.paranoidCheckDependencies)
  1397. DBGLOG("After resourcing a child graph");
  1398. return resourced.getClear();
  1399. }
  1400. void HqlCppTranslator::buildChildDataset(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt)
  1401. {
  1402. if (expr->isPure() && ctx.getMatchExpr(expr, tgt))
  1403. return;
  1404. LoopInvariantHelper helper;
  1405. BuildCtx bestctx(ctx);
  1406. if (options.optimizeLoopInvariant)
  1407. helper.getBestContext(bestctx, expr);
  1408. CHqlBoundTarget temp;
  1409. //MORE: Should have similar code to buildTempExpr()
  1410. createTempFor(bestctx, expr, temp);
  1411. buildAssignChildDataset(bestctx, temp, expr);
  1412. tgt.setFromTarget(temp);
  1413. if (expr->isPure())
  1414. bestctx.associateExpr(expr, tgt);
  1415. }
  1416. unique_id_t HqlCppTranslator::buildGraphLoopSubgraph(BuildCtx & ctx, IHqlExpression * dataset, IHqlExpression * selSeq, IHqlExpression * rowsid, IHqlExpression * body, IHqlExpression * counter, bool multiInstance)
  1417. {
  1418. ChildGraphExprBuilder graphBuilder(0);
  1419. OwnedHqlExpr transformedBody;
  1420. OwnedHqlExpr counterResult;
  1421. IHqlExpression * graphid = graphBuilder.queryRepresents();
  1422. {
  1423. const bool isParallel = multiInstance;
  1424. GraphLoopReplacer replacer(rowsid, graphid, counter, isParallel);
  1425. transformedBody.setown(replacer.transformRoot(body));
  1426. counterResult.set(replacer.queryCounterResult());
  1427. }
  1428. if (counterResult)
  1429. graphBuilder.addInput();
  1430. OwnedHqlExpr result = createValue(no_setgraphloopresult, makeVoidType(), LINK(transformedBody), LINK(graphid));
  1431. graphBuilder.addAction(result);
  1432. OwnedHqlExpr subquery = graphBuilder.getGraph();
  1433. ChildGraphBuilder builder(*this, subquery);
  1434. return builder.buildGraphLoopBody(ctx, multiInstance);
  1435. }
  1436. unique_id_t HqlCppTranslator::buildRemoteSubgraph(BuildCtx & ctx, IHqlExpression * dataset)
  1437. {
  1438. ChildGraphExprBuilder graphBuilder(0);
  1439. if (dataset->isAction())
  1440. {
  1441. graphBuilder.addAction(dataset);
  1442. }
  1443. else
  1444. {
  1445. OwnedHqlExpr ignoredResult = graphBuilder.addDataset(dataset);
  1446. }
  1447. OwnedHqlExpr subquery = graphBuilder.getGraph();
  1448. ChildGraphBuilder builder(*this, subquery);
  1449. return builder.buildRemoteGraph(ctx);
  1450. }
  1451. //---------------------------------------------------------------------------
  1452. // Functions to check whether a dataset can be evaluated inline or not.
  1453. //MORE: These should probably be split out into an hqlinline.cpp
  1454. bool HqlCppTranslator::canIterateInline(BuildCtx * ctx, IHqlExpression * expr)
  1455. {
  1456. return (isInlineOk() && ::canIterateInline(ctx, expr));
  1457. }
  1458. bool HqlCppTranslator::canAssignInline(BuildCtx * ctx, IHqlExpression * expr)
  1459. {
  1460. if (!isInlineOk())
  1461. return false;
  1462. return options.allowInlineSpill ? ::canProcessInline(ctx, expr) : ::canAssignInline(ctx, expr);
  1463. }
  1464. bool HqlCppTranslator::canEvaluateInline(BuildCtx * ctx, IHqlExpression * expr)
  1465. {
  1466. if (!isInlineOk())
  1467. return false;
  1468. return options.allowInlineSpill ? ::canProcessInline(ctx, expr) : ::canEvaluateInline(ctx, expr);
  1469. }
  1470. bool HqlCppTranslator::canProcessInline(BuildCtx * ctx, IHqlExpression * expr)
  1471. {
  1472. if (!isInlineOk())
  1473. return false;
  1474. return ::canProcessInline(ctx, expr);
  1475. }
  1476. bool HqlCppTranslator::isInlineOk()
  1477. {
  1478. if (!activeGraphCtx)
  1479. return true;
  1480. return true;
  1481. }
  1482. IHqlExpression * HqlCppTranslator::buildSpillChildDataset(BuildCtx & ctx, IHqlExpression * expr)
  1483. {
  1484. CHqlBoundExpr bound;
  1485. buildChildDataset(ctx, expr, bound);
  1486. return bound.getTranslatedExpr();
  1487. }
  1488. IHqlExpression * HqlCppTranslator::forceInlineAssignDataset(BuildCtx & ctx, IHqlExpression * expr)
  1489. {
  1490. loop
  1491. {
  1492. CHqlBoundExpr bound;
  1493. if (expr->isPure() && ctx.getMatchExpr(expr, bound))
  1494. return bound.getTranslatedExpr();
  1495. if (canProcessInline(&ctx, expr) || (expr->getOperator() == no_translated))
  1496. return LINK(expr);
  1497. switch (expr->getOperator())
  1498. {
  1499. case no_compound:
  1500. buildStmt(ctx, expr->queryChild(0));
  1501. expr = expr->queryChild(1);
  1502. break;
  1503. default:
  1504. return buildSpillChildDataset(ctx, expr);
  1505. }
  1506. }
  1507. }
  1508. //---------------------------------------------------------------------------
  1509. // Dataset temp creation
  1510. IHqlExpression * createGetResultFromWorkunitDataset(IHqlExpression * expr)
  1511. {
  1512. IHqlExpression * name = queryPropertyChild(expr, nameAtom, 0);
  1513. if (name)
  1514. name = createExprAttribute(namedAtom, LINK(name));
  1515. assertex(expr->isDataset());
  1516. return createDataset(no_getresult, LINK(expr->queryRecord()), createComma(LINK(expr->queryProperty(sequenceAtom)), name));
  1517. }
  1518. void HqlCppTranslator::buildAssignSerializedDataset(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  1519. {
  1520. OwnedITypeInfo serializedType = getSerializedForm(expr->queryType());
  1521. assertex(recordTypesMatch(target.queryType(), serializedType));
  1522. HqlExprArray args;
  1523. args.append(*createRowSerializer(ctx, expr->queryRecord(), serializerAtom));
  1524. args.append(*LINK(expr));
  1525. OwnedHqlExpr call = bindFunctionCall(rowset2DatasetXAtom, args);
  1526. buildExprAssign(ctx, target, call);
  1527. }
  1528. void HqlCppTranslator::buildSerializedDataset(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt)
  1529. {
  1530. CHqlBoundTarget target;
  1531. OwnedITypeInfo serializedType = getSerializedForm(expr->queryType());
  1532. createTempFor(ctx, serializedType, target, typemod_none, FormatBlockedDataset);
  1533. buildAssignSerializedDataset(ctx, target, expr);
  1534. tgt.setFromTarget(target);
  1535. }
  1536. void HqlCppTranslator::buildAssignLinkedDataset(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  1537. {
  1538. OwnedITypeInfo serializedType = getSerializedForm(target.queryType());
  1539. assertex(recordTypesMatch(serializedType, expr->queryType()));
  1540. IHqlExpression * record = ::queryRecord(target.queryType());
  1541. HqlExprArray args;
  1542. args.append(*createRowSerializer(ctx, record, deserializerAtom));
  1543. args.append(*LINK(expr));
  1544. OwnedHqlExpr call = bindFunctionCall(isGrouped(expr) ? groupedDataset2RowsetXAtom : dataset2RowsetXAtom, args, target.queryType());
  1545. buildExprAssign(ctx, target, call);
  1546. }
  1547. void HqlCppTranslator::buildLinkedDataset(BuildCtx & ctx, ITypeInfo * type, IHqlExpression * expr, CHqlBoundExpr & tgt)
  1548. {
  1549. CHqlBoundTarget target;
  1550. createTempFor(ctx, type, target, typemod_none, FormatLinkedDataset);
  1551. if (hasLinkedRow(expr->queryType()))
  1552. buildDatasetAssign(ctx, target, expr);
  1553. else
  1554. buildAssignLinkedDataset(ctx, target, expr);
  1555. tgt.setFromTarget(target);
  1556. }
  1557. void HqlCppTranslator::ensureDatasetFormat(BuildCtx & ctx, ITypeInfo * type, CHqlBoundExpr & tgt, ExpressionFormat format)
  1558. {
  1559. switch (format)
  1560. {
  1561. case FormatBlockedDataset:
  1562. if (isArrayRowset(tgt.queryType()))
  1563. {
  1564. OwnedHqlExpr deserializedExpr = tgt.getTranslatedExpr();
  1565. LinkedHqlExpr savedCount = tgt.count;
  1566. buildSerializedDataset(ctx, deserializedExpr, tgt);
  1567. if (savedCount && !isFixedWidthDataset(deserializedExpr))
  1568. tgt.count.set(savedCount);
  1569. return;
  1570. }
  1571. break;
  1572. case FormatLinkedDataset:
  1573. if (!hasLinkCountedModifier(tgt.queryType()))
  1574. {
  1575. OwnedHqlExpr serializedExpr = tgt.getTranslatedExpr();
  1576. buildLinkedDataset(ctx, type, serializedExpr, tgt);
  1577. return;
  1578. }
  1579. break;
  1580. case FormatArrayDataset:
  1581. if (!isArrayRowset(tgt.queryType()))
  1582. {
  1583. OwnedHqlExpr serializedExpr = tgt.getTranslatedExpr();
  1584. buildLinkedDataset(ctx, type, serializedExpr, tgt);
  1585. return;
  1586. }
  1587. break;
  1588. }
  1589. }
  1590. void HqlCppTranslator::buildDataset(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt, ExpressionFormat format)
  1591. {
  1592. doBuildDataset(ctx, expr, tgt, format);
  1593. ensureDatasetFormat(ctx, expr->queryType(), tgt, format);
  1594. }
  1595. void HqlCppTranslator::doBuildDataset(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt, ExpressionFormat format)
  1596. {
  1597. if (expr->isPure() && ctx.getMatchExpr(expr, tgt))
  1598. return;
  1599. /*
  1600. OwnedHqlExpr transformed = normalizeAnyDatasetAliases(expr);
  1601. if (transformed && (transformed != expr))
  1602. {
  1603. doBuildDataset(ctx, transformed, tgt, format);
  1604. ctx.associateExpr(expr, tgt);
  1605. return;
  1606. }
  1607. */
  1608. node_operator op = expr->getOperator();
  1609. switch (op)
  1610. {
  1611. case no_dataset_alias:
  1612. if (!expr->hasProperty(_normalized_Atom))
  1613. {
  1614. OwnedHqlExpr uniqueChild = normalizeDatasetAlias(expr);
  1615. doBuildDataset(ctx, uniqueChild, tgt, format);
  1616. }
  1617. else
  1618. doBuildDataset(ctx, expr->queryChild(0), tgt, format);
  1619. return;
  1620. case no_alias:
  1621. doBuildExprAlias(ctx, expr, &tgt);
  1622. return;
  1623. case no_owned_ds:
  1624. buildTempExpr(ctx, expr, tgt);
  1625. return;
  1626. case no_null:
  1627. {
  1628. tgt.count.setown(getSizetConstant(0));
  1629. tgt.length.setown(getSizetConstant(0));
  1630. IHqlExpression * record = expr->queryRecord();
  1631. ITypeInfo * type = makeTableType(makeRowType(record->getType()), NULL, NULL, NULL);
  1632. if ((format == FormatLinkedDataset) || (format == FormatArrayDataset))
  1633. type = makeAttributeModifier(type, getLinkCountedAttr());
  1634. tgt.expr.setown(createValue(no_nullptr, makeReferenceModifier(type)));
  1635. return;
  1636. }
  1637. case no_translated:
  1638. expandTranslated(expr, tgt);
  1639. return;
  1640. case no_select:
  1641. {
  1642. if (isMultiLevelDatasetSelector(expr, false))
  1643. break;
  1644. Owned<IReferenceSelector> selected = buildReference(ctx, expr);
  1645. selected->get(ctx, tgt);
  1646. return;
  1647. }
  1648. case no_libraryinput:
  1649. if (!buildExprInCorrectContext(ctx, expr, tgt, false))
  1650. throwUnexpected();
  1651. return;
  1652. case no_call:
  1653. case no_externalcall:
  1654. if (!hasStreamedModifier(expr->queryType()))
  1655. {
  1656. buildTempExpr(ctx, expr, tgt);
  1657. return;
  1658. }
  1659. break;
  1660. case no_newaggregate:
  1661. if (canAssignInline(&ctx, expr))
  1662. {
  1663. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  1664. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  1665. Owned<IReferenceSelector> createdRef = createReferenceSelector(rowBuilder);
  1666. BuildCtx subctx(ctx);
  1667. subctx.addGroup();
  1668. doBuildRowAssignAggregate(subctx, createdRef, expr);
  1669. finalizeTempRow(ctx, tempRow, rowBuilder);
  1670. convertBoundRowToDataset(ctx, tgt, tempRow, format);
  1671. return;
  1672. }
  1673. break;
  1674. case no_id2blob:
  1675. doBuildExprIdToBlob(ctx, expr, tgt);
  1676. return;
  1677. case no_rows:
  1678. {
  1679. if (!buildExprInCorrectContext(ctx, expr, tgt, false))
  1680. throwError(HQLERR_RowsUsedOutsideContext);
  1681. return;
  1682. }
  1683. case no_limit:
  1684. if (expr->hasProperty(skipAtom) || expr->hasProperty(onFailAtom))
  1685. break;
  1686. doBuildDatasetLimit(ctx, expr, tgt, format);
  1687. return;
  1688. case no_compound_childread:
  1689. case no_compound_childnormalize:
  1690. case no_compound_childaggregate:
  1691. case no_compound_selectnew:
  1692. case no_compound_inline:
  1693. case no_distributed:
  1694. case no_preservemeta:
  1695. case no_sorted:
  1696. case no_nofold:
  1697. case no_nohoist:
  1698. case no_section:
  1699. case no_sectioninput:
  1700. buildDataset(ctx, expr->queryChild(0), tgt, format);
  1701. return;
  1702. case no_forcegraph:
  1703. #ifdef _DEBUG
  1704. throwUnexpected();
  1705. #endif
  1706. buildDataset(ctx, expr->queryChild(0), tgt, format);
  1707. return;
  1708. case no_getgraphresult:
  1709. doBuildExprGetGraphResult(ctx, expr, tgt, format);
  1710. return;
  1711. case no_getresult:
  1712. case no_workunit_dataset:
  1713. if (!isGrouped(expr))
  1714. {
  1715. doBuildExprGetResult(ctx, expr, tgt);
  1716. return;
  1717. }
  1718. break;
  1719. case no_skip:
  1720. {
  1721. buildStmt(ctx, expr);
  1722. OwnedHqlExpr null = createNullExpr(expr);
  1723. buildDataset(ctx, null, tgt, format);
  1724. return;
  1725. }
  1726. case no_serialize:
  1727. if (isDummySerializeDeserialize(expr))
  1728. doBuildDataset(ctx, expr->queryChild(0)->queryChild(0), tgt, format);
  1729. else
  1730. buildSerializedDataset(ctx, expr->queryChild(0), tgt);
  1731. return;
  1732. case no_deserialize:
  1733. if (isDummySerializeDeserialize(expr))
  1734. doBuildDataset(ctx, expr->queryChild(0)->queryChild(0), tgt, format);
  1735. else
  1736. buildLinkedDataset(ctx, expr->queryType(), expr->queryChild(0), tgt);
  1737. return;
  1738. case no_datasetfromrow:
  1739. {
  1740. IHqlExpression * row = expr->queryChild(0);
  1741. if (isAlwaysActiveRow(row) && (format == FormatNatural))
  1742. {
  1743. Owned<IReferenceSelector> selector = buildActiveRow(ctx, row);
  1744. BuildCtx groupctx(ctx);
  1745. groupctx.addGroup();
  1746. BoundRow * bound = bindSelectorAsRootRow(groupctx, selector, row);
  1747. convertBoundRowToDataset(groupctx, tgt, bound, format);
  1748. tgt.count.setown(getSizetConstant(1));
  1749. ctx.associateExpr(expr, tgt);
  1750. return;
  1751. }
  1752. break;
  1753. }
  1754. case no_inlinetable:
  1755. case no_inlinedictionary:
  1756. if (doBuildDatasetInlineTable(ctx, expr, tgt, format))
  1757. return;
  1758. break;
  1759. case no_compound:
  1760. {
  1761. buildStmt(ctx, expr->queryChild(0));
  1762. buildDataset(ctx, expr->queryChild(1), tgt, format);
  1763. return;
  1764. }
  1765. }
  1766. bool singleRow = hasSingleRow(expr);
  1767. bool useTempRow = singleRow && canAssignInline(&ctx, expr) && (format != FormatLinkedDataset) && (format != FormatArrayDataset);
  1768. //Conditional row assignment if variable length causes offset to be recalculated outside of the if()
  1769. //if (useTempRow && (op == no_if) && isVariableSizeRecord(expr->queryRecord()))
  1770. // useTempRow = false;
  1771. if (useTempRow)
  1772. {
  1773. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  1774. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  1775. Owned<IHqlCppDatasetBuilder> builder = createSingleRowTempDatasetBuilder(expr->queryRecord(), rowBuilder);
  1776. builder->buildDeclare(ctx);
  1777. buildDatasetAssign(ctx, builder, expr);
  1778. //builder->buildFinish(ctx, tempTarget);
  1779. finalizeTempRow(ctx, tempRow, rowBuilder);
  1780. convertBoundRowToDataset(ctx, tgt, tempRow, format);
  1781. }
  1782. else
  1783. {
  1784. if (!canAssignInline(&ctx, expr))
  1785. {
  1786. CHqlBoundTarget tempTarget;
  1787. createTempFor(ctx, expr->queryType(), tempTarget, typemod_none, format);
  1788. buildDatasetAssign(ctx, tempTarget, expr);
  1789. tgt.setFromTarget(tempTarget);
  1790. //buildTempExpr(ctx, expr, tgt); // can't use this because it causes recursion on no_selectnth
  1791. }
  1792. else
  1793. {
  1794. Owned<IHqlCppDatasetBuilder> builder;
  1795. IHqlExpression * record = expr->queryRecord();
  1796. OwnedHqlExpr serializedRecord = getSerializedForm(record);
  1797. if (format == FormatNatural)
  1798. {
  1799. if (record != serializedRecord)
  1800. ensureContextAvailable(ctx);
  1801. if (!ctx.queryMatchExpr(codeContextMarkerExpr))
  1802. {
  1803. if (record != serializedRecord)
  1804. throwError(HQLERR_LinkedDatasetNoContext);
  1805. format = FormatBlockedDataset;
  1806. }
  1807. else if ((record == serializedRecord) && !options.tempDatasetsUseLinkedRows)
  1808. {
  1809. format = FormatBlockedDataset;
  1810. }
  1811. else
  1812. {
  1813. format = FormatLinkedDataset;
  1814. }
  1815. }
  1816. else if (record != serializedRecord)
  1817. format = FormatLinkedDataset; // Have to serialize it later - otherwise it won't be compatible
  1818. if (format == FormatLinkedDataset || format == FormatArrayDataset)
  1819. {
  1820. IHqlExpression * choosenLimit = NULL;
  1821. if ((op == no_choosen) && !isChooseNAllLimit(expr->queryChild(1)) && !queryRealChild(expr, 2))
  1822. {
  1823. choosenLimit = expr->queryChild(1);
  1824. expr = expr->queryChild(0);
  1825. }
  1826. //MORE: Extract limit and choosen and pass as parameters
  1827. builder.setown(createLinkedDatasetBuilder(record, choosenLimit));
  1828. }
  1829. else if ((op == no_choosen) && !isChooseNAllLimit(expr->queryChild(1)) && !queryRealChild(expr, 2))
  1830. {
  1831. //Build a limited builder - it is likely to be just as efficient, and often much more e.g., choosen(a+b, n)
  1832. builder.setown(createChoosenDatasetBuilder(serializedRecord, expr->queryChild(1)));
  1833. expr = expr->queryChild(0);
  1834. }
  1835. else
  1836. builder.setown(createBlockedDatasetBuilder(serializedRecord));
  1837. builder->buildDeclare(ctx);
  1838. buildDatasetAssign(ctx, builder, expr);
  1839. builder->buildFinish(ctx, tgt);
  1840. }
  1841. }
  1842. if (singleRow)
  1843. tgt.count.setown(getSizetConstant(1));
  1844. else if (op == no_inlinetable)
  1845. {
  1846. IHqlExpression * transforms = expr->queryChild(0);
  1847. if (!transformListContainsSkip(transforms))
  1848. tgt.count.setown(getSizetConstant(transforms->numChildren()));
  1849. }
  1850. ctx.associateExpr(expr, tgt);
  1851. }
  1852. //---------------------------------------------------------------------------
  1853. // Dataset assignment - to temp
  1854. void HqlCppTranslator::buildDatasetAssign(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  1855. {
  1856. node_operator op = expr->getOperator();
  1857. switch (op)
  1858. {
  1859. case no_call:
  1860. case no_externalcall:
  1861. if (!hasStreamedModifier(expr->queryType()))
  1862. {
  1863. doBuildCall(ctx, &target, expr, NULL);
  1864. return;
  1865. }
  1866. break;
  1867. case no_getgraphresult:
  1868. doBuildAssignGetGraphResult(ctx, target, expr);
  1869. return;
  1870. case no_workunit_dataset:
  1871. case no_getresult:
  1872. buildExprAssign(ctx, target, expr);
  1873. return;
  1874. case no_null:
  1875. {
  1876. CHqlBoundExpr bound;
  1877. buildDataset(ctx, expr, bound, isArrayRowset(expr->queryType()) ? FormatLinkedDataset : FormatBlockedDataset);
  1878. OwnedHqlExpr translated = bound.getTranslatedExpr();
  1879. if (target.count) ctx.addAssign(target.count, bound.count);
  1880. if (target.length) ctx.addAssign(target.length, bound.length);
  1881. ctx.addAssign(target.expr, bound.expr);
  1882. return;
  1883. }
  1884. case no_inlinedictionary:
  1885. case no_inlinetable:
  1886. {
  1887. //This will typically generate a loop. If few items then it is more efficient to expand the assigns/clones out.
  1888. if (expr->queryChild(0)->numChildren() > INLINE_TABLE_EXPAND_LIMIT)
  1889. {
  1890. CHqlBoundExpr bound;
  1891. if (doBuildDatasetInlineTable(ctx, expr, bound, FormatNatural))
  1892. {
  1893. OwnedHqlExpr translated = bound.getTranslatedExpr();
  1894. buildDatasetAssign(ctx, target, translated);
  1895. return;
  1896. }
  1897. }
  1898. break;
  1899. }
  1900. case no_alias:
  1901. {
  1902. CHqlBoundExpr bound;
  1903. buildDataset(ctx, expr, bound, FormatNatural);
  1904. OwnedHqlExpr translated = bound.getTranslatedExpr();
  1905. buildDatasetAssign(ctx, target, translated);
  1906. return;
  1907. }
  1908. case no_owned_ds:
  1909. {
  1910. ITypeInfo * targetType = target.queryType();
  1911. if (hasLinkCountedModifier(targetType) && hasWrapperModifier(targetType))
  1912. {
  1913. CHqlBoundExpr bound;
  1914. buildDataset(ctx, expr->queryChild(0), bound, FormatLinkedDataset);
  1915. OwnedHqlExpr compound = createValue(no_complex, bound.expr->getType(), LINK(bound.count), LINK(bound.expr));
  1916. ctx.addAssign(target.expr, compound);
  1917. return;
  1918. }
  1919. break;
  1920. }
  1921. case no_compound:
  1922. {
  1923. buildStmt(ctx, expr->queryChild(0));
  1924. buildDatasetAssign(ctx, target, expr->queryChild(1));
  1925. return;
  1926. }
  1927. case no_compound_childread:
  1928. case no_compound_childnormalize:
  1929. case no_compound_childaggregate:
  1930. case no_compound_selectnew:
  1931. case no_compound_inline:
  1932. case no_distributed:
  1933. case no_preservemeta:
  1934. case no_sorted:
  1935. case no_nofold:
  1936. case no_nohoist:
  1937. case no_section:
  1938. case no_sectioninput:
  1939. buildDatasetAssign(ctx, target, expr->queryChild(0));
  1940. return;
  1941. case no_serialize:
  1942. if (isDummySerializeDeserialize(expr))
  1943. buildDatasetAssign(ctx, target, expr->queryChild(0)->queryChild(0));
  1944. else
  1945. buildAssignSerializedDataset(ctx, target, expr->queryChild(0));
  1946. return;
  1947. case no_deserialize:
  1948. if (isDummySerializeDeserialize(expr))
  1949. buildDatasetAssign(ctx, target, expr->queryChild(0)->queryChild(0));
  1950. else
  1951. buildAssignLinkedDataset(ctx, target, expr->queryChild(0));
  1952. return;
  1953. case no_select:
  1954. {
  1955. bool isNew;
  1956. IHqlExpression * ds = querySelectorDataset(expr, isNew);
  1957. if (!isNew || ds->isDatarow())
  1958. {
  1959. Owned<IReferenceSelector> selected = buildReference(ctx, expr);
  1960. selected->assignTo(ctx, target);
  1961. return;
  1962. }
  1963. break;
  1964. }
  1965. case no_typetransfer:
  1966. {
  1967. IHqlExpression * child = expr->queryChild(0);
  1968. if (expr->isDataset() && child->isDataset())
  1969. {
  1970. //Special case no-op type transfers and assignment from a call returning unknown-type dataset
  1971. if (!recordTypesMatch(expr, child) &&
  1972. !(child->getOperator() == no_externalcall && recordTypesMatch(child, queryNullRecord())))
  1973. {
  1974. CHqlBoundExpr bound;
  1975. buildDataset(ctx, child, bound, FormatNatural);
  1976. ITypeInfo * newType = cloneModifiers(bound.expr->queryType(), expr->queryType());
  1977. bound.expr.setown(createValue(no_typetransfer, newType, LINK(bound.expr)));
  1978. OwnedHqlExpr translated = bound.getTranslatedExpr();
  1979. buildDatasetAssign(ctx, target, translated);
  1980. }
  1981. else
  1982. buildDatasetAssign(ctx, target, child);
  1983. return;
  1984. }
  1985. break;
  1986. }
  1987. }
  1988. if (!canAssignInline(&ctx, expr) && (op != no_translated))
  1989. {
  1990. buildAssignChildDataset(ctx, target, expr);
  1991. return;
  1992. }
  1993. ITypeInfo * to = target.queryType();
  1994. ITypeInfo * exprType = expr->queryType();
  1995. bool targetOutOfLine = isArrayRowset(to);
  1996. switch (op)
  1997. {
  1998. case no_limit:
  1999. assertex(!expr->hasProperty(skipAtom) && !expr->hasProperty(onFailAtom));
  2000. //Do the limit check as a post test.
  2001. //It means we may read more records than we need to, but the code is inline, and the code is generally much better.
  2002. if (target.count)
  2003. {
  2004. buildDatasetAssign(ctx, target, expr->queryChild(0));
  2005. CHqlBoundExpr bound;
  2006. bound.setFromTarget(target);
  2007. doBuildCheckDatasetLimit(ctx, expr, bound);
  2008. return;
  2009. }
  2010. break;
  2011. case no_translated:
  2012. {
  2013. bool sourceOutOfLine = isArrayRowset(exprType);
  2014. if (sourceOutOfLine != targetOutOfLine)
  2015. {
  2016. OwnedITypeInfo serializedSourceType = getSerializedForm(exprType);
  2017. OwnedITypeInfo serializedTargetType = getSerializedForm(to);
  2018. if (queryUnqualifiedType(serializedSourceType) == queryUnqualifiedType(serializedTargetType))
  2019. {
  2020. if (targetOutOfLine)
  2021. {
  2022. buildAssignLinkedDataset(ctx, target, expr);
  2023. }
  2024. else
  2025. {
  2026. buildAssignSerializedDataset(ctx, target, expr);
  2027. }
  2028. return;
  2029. }
  2030. }
  2031. break;
  2032. }
  2033. }
  2034. if (recordTypesMatch(to, exprType))
  2035. {
  2036. switch (op)
  2037. {
  2038. case no_rows:
  2039. {
  2040. CHqlBoundExpr bound;
  2041. buildDataset(ctx, expr, bound, FormatLinkedDataset);
  2042. OwnedHqlExpr translated = bound.getTranslatedExpr();
  2043. buildDatasetAssign(ctx, target, translated);
  2044. return;
  2045. }
  2046. case no_select:
  2047. {
  2048. bool isNew;
  2049. IHqlExpression * ds = querySelectorDataset(expr, isNew);
  2050. if (isNew && !ds->isDatarow())
  2051. break;
  2052. }
  2053. //fall through
  2054. case no_translated:
  2055. case no_null:
  2056. case no_id2blob:
  2057. {
  2058. _ATOM func = NULL;
  2059. if (!isArrayRowset(to))
  2060. {
  2061. if (!isArrayRowset(exprType))
  2062. func = dataset2DatasetXAtom;
  2063. }
  2064. else if (hasLinkCountedModifier(to))
  2065. {
  2066. if (hasLinkCountedModifier(exprType))
  2067. {
  2068. CHqlBoundExpr bound;
  2069. buildDataset(ctx, expr, bound, FormatLinkedDataset);
  2070. assertex(bound.count && bound.expr);
  2071. if (hasWrapperModifier(to))
  2072. {
  2073. //assigns to count and rows members
  2074. StringBuffer s;
  2075. generateExprCpp(s, target.expr);
  2076. s.append(".set(");
  2077. generateExprCpp(s, bound.count);
  2078. s.append(",");
  2079. generateExprCpp(s, bound.expr);
  2080. s.append(");");
  2081. ctx.addQuoted(s);
  2082. }
  2083. else
  2084. {
  2085. ctx.addAssign(target.count, bound.count);
  2086. HqlExprArray args;
  2087. args.append(*LINK(bound.expr));
  2088. OwnedHqlExpr call = bindTranslatedFunctionCall(linkRowsetAtom, args);
  2089. ctx.addAssign(target.expr, call);
  2090. }
  2091. return;
  2092. }
  2093. }
  2094. if (func)
  2095. {
  2096. HqlExprArray args;
  2097. args.append(*LINK(expr));
  2098. OwnedHqlExpr call = bindFunctionCall(func, args);
  2099. buildExprAssign(ctx, target, call);
  2100. return;
  2101. }
  2102. }
  2103. }
  2104. }
  2105. IHqlExpression * record = ::queryRecord(to);
  2106. Owned<IHqlCppDatasetBuilder> builder;
  2107. if (targetOutOfLine)
  2108. {
  2109. if (target.queryType()->getTypeCode() == type_dictionary)
  2110. {
  2111. builder.setown(createLinkedDictionaryBuilder(record));
  2112. }
  2113. else
  2114. {
  2115. IHqlExpression * choosenLimit = NULL;
  2116. if ((op == no_choosen) && !isChooseNAllLimit(expr->queryChild(1)) && !queryRealChild(expr, 2))
  2117. {
  2118. choosenLimit = expr->queryChild(1);
  2119. expr = expr->queryChild(0);
  2120. }
  2121. builder.setown(createLinkedDatasetBuilder(record, choosenLimit));
  2122. }
  2123. }
  2124. else
  2125. builder.setown(createBlockedDatasetBuilder(record));
  2126. builder->buildDeclare(ctx);
  2127. buildDatasetAssign(ctx, builder, expr);
  2128. builder->buildFinish(ctx, target);
  2129. }
  2130. //---------------------------------------------------------------------------
  2131. void HqlCppTranslator::doBuildCheckDatasetLimit(BuildCtx & ctx, IHqlExpression * expr, const CHqlBoundExpr & bound)
  2132. {
  2133. IHqlExpression * record = expr->queryRecord();
  2134. IHqlExpression * limit = expr->queryChild(1);
  2135. OwnedHqlExpr test;
  2136. if (!bound.count && bound.length && isFixedRecordSize(record))
  2137. {
  2138. OwnedHqlExpr size = bound.length->queryValue() ? LINK(bound.length) : createTranslated(bound.length);
  2139. OwnedHqlExpr maxSize = createValue(no_mul, LINK(sizetType), ensureExprType(limit, sizetType), getSizetConstant(getFixedRecordSize(record)));
  2140. test.setown(createBoolExpr(no_gt, ensureExprType(size, sizetType), LINK(maxSize)));
  2141. }
  2142. else
  2143. {
  2144. OwnedHqlExpr count = getBoundCount(bound);
  2145. OwnedHqlExpr translatedCount = count->queryValue() ? LINK(count) : createTranslated(count);
  2146. test.setown(createBoolExpr(no_gt, ensureExprType(translatedCount, sizetType), ensureExprType(limit, sizetType)));
  2147. }
  2148. OwnedHqlExpr folded = foldHqlExpression(test);
  2149. LinkedHqlExpr fail = queryRealChild(expr, 2);
  2150. if (folded->queryValue())
  2151. {
  2152. if (!folded->queryValue()->getBoolValue())
  2153. return;
  2154. StringBuffer failMessageText;
  2155. if (fail)
  2156. {
  2157. OwnedHqlExpr failMessage = getFailMessage(fail, true);
  2158. if (failMessage && failMessage->queryValue())
  2159. failMessage->queryValue()->getStringValue(failMessageText);
  2160. }
  2161. if (failMessageText.length())
  2162. WARNING1(HQLWRN_LimitAlwaysExceededX, failMessageText.str());
  2163. else
  2164. WARNING(HQLWRN_LimitAlwaysExceeded);
  2165. }
  2166. if (!fail)
  2167. fail.setown(createFailAction("Limit exceeded", limit, NULL, queryCurrentActivityId(ctx)));
  2168. BuildCtx subctx(ctx);
  2169. buildFilter(subctx, folded);
  2170. buildStmt(subctx, fail);
  2171. }
  2172. void HqlCppTranslator::doBuildDatasetLimit(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt, ExpressionFormat format)
  2173. {
  2174. buildDataset(ctx, expr->queryChild(0), tgt, format);
  2175. doBuildCheckDatasetLimit(ctx, expr, tgt);
  2176. }
  2177. bool HqlCppTranslator::doBuildDatasetInlineTable(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt, ExpressionFormat format)
  2178. {
  2179. if (!options.generateStaticInlineTables)
  2180. return false;
  2181. IHqlExpression * transforms = expr->queryChild(0);
  2182. IHqlExpression * record = expr->queryRecord();
  2183. if (transforms->numChildren() == 0)
  2184. {
  2185. OwnedHqlExpr null = createDataset(no_null, LINK(record));
  2186. buildDataset(ctx, null, tgt, format);
  2187. return true;
  2188. }
  2189. BuildCtx declareCtx(*code, literalAtom);
  2190. //Remove unique id when checking for constant datasets already generated
  2191. OwnedHqlExpr exprNoUnique = removeProperty(expr, _uid_Atom);
  2192. if (declareCtx.getMatchExpr(exprNoUnique, tgt))
  2193. return true;
  2194. HqlExprArray rows;
  2195. unsigned maxRows = transforms->numChildren();
  2196. unsigned row;
  2197. for (row = 0; row < maxRows; row++)
  2198. {
  2199. OwnedHqlExpr constRow = createConstantRowExpr(transforms->queryChild(row));
  2200. if (!constRow || !canGenerateStringInline(constRow->queryType()->getSize()))
  2201. return false;
  2202. rows.append(*constRow.getClear());
  2203. }
  2204. HqlExprArray boundRows;
  2205. ForEachItemIn(i, rows)
  2206. {
  2207. CHqlBoundExpr bound;
  2208. buildConstRow(record, &rows.item(i), bound);
  2209. boundRows.append(*bound.expr.getClear());
  2210. }
  2211. Owned<ITypeInfo> rowType = makeConstantModifier(makeReferenceModifier(makeRowType(LINK(queryRecordType(expr->queryType())))));
  2212. OwnedHqlExpr values = createValue(no_list, makeSetType(LINK(rowType)), boundRows);
  2213. Owned<ITypeInfo> tableType = makeConstantModifier(makeArrayType(LINK(rowType), maxRows));
  2214. OwnedITypeInfo rowsType = makeOutOfLineModifier(makeTableType(LINK(rowType), NULL, NULL, NULL));
  2215. OwnedHqlExpr table = declareCtx.getTempDeclare(tableType, values);
  2216. if (options.spanMultipleCpp)
  2217. {
  2218. BuildCtx protoctx(*code, mainprototypesAtom);
  2219. protoctx.addDeclareExternal(table);
  2220. }
  2221. tgt.count.setown(getSizetConstant(maxRows));
  2222. tgt.expr.setown(createValue(no_typetransfer, LINK(rowsType), LINK(table)));
  2223. declareCtx.associateExpr(exprNoUnique, tgt);
  2224. return true;
  2225. }
  2226. //---------------------------------------------------------------------------
  2227. // Dataset creation via builder
  2228. void HqlCppTranslator::buildDatasetAssignTempTable(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * expr)
  2229. {
  2230. OwnedHqlExpr values = normalizeListCasts(expr->queryChild(0));
  2231. if (values->getOperator() == no_null)
  2232. return;
  2233. IHqlExpression * record = expr->queryChild(1);
  2234. OwnedHqlExpr rowsExpr;
  2235. if (values->queryType()->getTypeCode() == type_set)
  2236. {
  2237. if ((values->getOperator() == no_list) && !values->isConstant())
  2238. {
  2239. ForEachChild(i, values)
  2240. {
  2241. BuildCtx loopctx(ctx);
  2242. BoundRow * targetRow = target->buildCreateRow(loopctx);
  2243. OwnedHqlExpr targetField = createSelectExpr(LINK(targetRow->querySelector()), LINK(record->queryChild(0)));
  2244. buildAssign(loopctx, targetField, values->queryChild(i));
  2245. target->finishRow(loopctx, targetRow);
  2246. }
  2247. }
  2248. else
  2249. {
  2250. Owned<IHqlCppSetCursor> cursor = createSetSelector(ctx, values);
  2251. BuildCtx loopctx(ctx);
  2252. CHqlBoundExpr boundCurElement;
  2253. cursor->buildIterateLoop(loopctx, boundCurElement, false);
  2254. BoundRow * targetRow = target->buildCreateRow(loopctx);
  2255. OwnedHqlExpr targetField = createSelectExpr(LINK(targetRow->querySelector()), LINK(record->queryChild(0)));
  2256. OwnedHqlExpr value = boundCurElement.getTranslatedExpr();
  2257. buildAssign(loopctx, targetField, value);
  2258. target->finishRow(loopctx, targetRow);
  2259. }
  2260. }
  2261. else
  2262. {
  2263. BuildCtx subctx(ctx);
  2264. BoundRow * targetRow = target->buildCreateRow(subctx);
  2265. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2266. buildRowAssign(subctx, targetRef, values);
  2267. target->finishRow(subctx, targetRow);
  2268. }
  2269. }
  2270. void HqlCppTranslator::buildDatasetAssignInlineTable(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * expr)
  2271. {
  2272. IHqlExpression * transforms = expr->queryChild(0);
  2273. if (transforms->numChildren() == 0)
  2274. return;
  2275. unsigned maxRows = transforms->numChildren();
  2276. unsigned row;
  2277. const bool copyConstantRows = true;//getFieldCount(expr->queryRecord()) > 2;
  2278. for (row = 0; row < maxRows; row++)
  2279. {
  2280. IHqlExpression * transform = transforms->queryChild(row);
  2281. OwnedHqlExpr rowValue = createRow(no_createrow, LINK(transform));
  2282. BuildCtx subctx(ctx);
  2283. CHqlBoundExpr bound;
  2284. //Work in progress. Check if there are several fields - otherwise not worth it.s
  2285. if (doBuildRowConstantTransform(transform, bound))
  2286. {
  2287. BoundRow * row = bindConstantRow(subctx, rowValue, bound);
  2288. if (target->buildLinkRow(subctx, row))
  2289. continue;
  2290. }
  2291. BoundRow * targetRow = target->buildCreateRow(subctx);
  2292. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2293. buildRowAssign(subctx, targetRef, rowValue);
  2294. target->finishRow(subctx, targetRow);
  2295. }
  2296. }
  2297. class InlineDatasetSkipCallback : public CInterface, implements IHqlCodeCallback
  2298. {
  2299. public:
  2300. IMPLEMENT_IINTERFACE
  2301. virtual void buildCode(HqlCppTranslator & translator, BuildCtx & ctx)
  2302. {
  2303. ctx.addContinue();
  2304. }
  2305. };
  2306. void HqlCppTranslator::buildDatasetAssignProject(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * expr)
  2307. {
  2308. BuildCtx iterctx(ctx);
  2309. IHqlExpression * ds = expr->queryChild(0);
  2310. IHqlExpression * counter = queryPropertyChild(expr, _countProject_Atom, 0);
  2311. OwnedHqlExpr counterVar;
  2312. if (counter)
  2313. {
  2314. counterVar.setown(iterctx.getTempDeclare(unsignedType, queryZero()));
  2315. }
  2316. bool containsSkip = transformContainsSkip(expr->queryChild(1));
  2317. BoundRow * sourceCursor = buildDatasetIterate(iterctx, ds, containsSkip);
  2318. if (counter)
  2319. {
  2320. iterctx.associateExpr(counter, counterVar);
  2321. OwnedHqlExpr inc = createValue(no_postinc, LINK(unsignedType), LINK(counterVar));
  2322. iterctx.addExpr(inc);
  2323. }
  2324. if (sourceCursor)
  2325. {
  2326. BoundRow * targetRow = target->buildCreateRow(iterctx);
  2327. HqlExprAssociation * skipAssociation = NULL;
  2328. if (containsSkip)
  2329. {
  2330. OwnedHqlExpr callback = createUnknown(no_unknown, makeVoidType(), NULL, new InlineDatasetSkipCallback);
  2331. skipAssociation = ctx.associateExpr(skipActionMarker, callback);
  2332. }
  2333. Owned<IReferenceSelector> targetRef = buildActiveRow(iterctx, targetRow->querySelector());
  2334. switch (expr->getOperator())
  2335. {
  2336. case no_hqlproject:
  2337. doBuildRowAssignProject(iterctx, targetRef, expr);
  2338. break;
  2339. case no_newusertable:
  2340. doBuildRowAssignUserTable(iterctx, targetRef, expr);
  2341. break;
  2342. }
  2343. ctx.removeAssociation(skipAssociation);
  2344. target->finishRow(iterctx, targetRow);
  2345. }
  2346. }
  2347. void HqlCppTranslator::buildDatasetAssignJoin(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * expr)
  2348. {
  2349. IHqlExpression * left = expr->queryChild(0);
  2350. IHqlExpression * right = expr->queryChild(1);
  2351. IHqlExpression * cond = expr->queryChild(2);
  2352. IHqlExpression * selSeq = querySelSeq(expr);
  2353. bool leftOuter = expr->hasProperty(leftonlyAtom) || expr->hasProperty(leftouterAtom);
  2354. CHqlBoundExpr nullRhs;
  2355. if (leftOuter)
  2356. buildDefaultRow(ctx, right, nullRhs);
  2357. BuildCtx leftIterCtx(ctx);
  2358. BoundRow * leftCursor = buildDatasetIterate(leftIterCtx, left, false);
  2359. bindTableCursor(leftIterCtx, left, leftCursor->queryBound(), no_left, selSeq);
  2360. OwnedHqlExpr matchedAnyVar;
  2361. if (leftOuter)
  2362. matchedAnyVar.setown(leftIterCtx.getTempDeclare(queryBoolType(), queryBoolExpr(false)));
  2363. BuildCtx rightIterCtx(leftIterCtx);
  2364. BoundRow * rightCursor = buildDatasetIterate(rightIterCtx, right, false);
  2365. bindTableCursor(rightIterCtx, right, rightCursor->queryBound(), no_right, selSeq);
  2366. OwnedHqlExpr cseCond = options.spotCSE ? spotScalarCSE(cond) : LINK(cond);
  2367. buildFilter(rightIterCtx, cseCond);
  2368. if (!expr->hasProperty(leftonlyAtom))
  2369. {
  2370. BoundRow * targetRow = target->buildCreateRow(rightIterCtx);
  2371. Owned<IReferenceSelector> targetRef = buildActiveRow(rightIterCtx, targetRow->querySelector());
  2372. OwnedHqlExpr rowValue = createRow(no_createrow, LINK(expr->queryChild(3)));
  2373. buildRowAssign(rightIterCtx, targetRef, rowValue);
  2374. target->finishRow(rightIterCtx, targetRow);
  2375. }
  2376. if (matchedAnyVar)
  2377. {
  2378. buildAssignToTemp(rightIterCtx, matchedAnyVar, queryBoolExpr(true));
  2379. OwnedHqlExpr test = getInverse(matchedAnyVar);
  2380. leftIterCtx.addFilter(test);
  2381. OwnedHqlExpr defaultRowPtr = getPointer(nullRhs.expr);
  2382. bindTableCursor(leftIterCtx, right, defaultRowPtr, no_right, selSeq);
  2383. BoundRow * targetRow = target->buildCreateRow(leftIterCtx);
  2384. Owned<IReferenceSelector> targetRef = buildActiveRow(leftIterCtx, targetRow->querySelector());
  2385. OwnedHqlExpr rowValue = createRow(no_createrow, LINK(expr->queryChild(3)));
  2386. buildRowAssign(leftIterCtx, targetRef, rowValue);
  2387. target->finishRow(leftIterCtx, targetRow);
  2388. }
  2389. }
  2390. void HqlCppTranslator::buildDatasetAssignAggregate(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * expr)
  2391. {
  2392. BuildCtx subctx(ctx);
  2393. subctx.addGroup();
  2394. BoundRow * targetRow = target->buildCreateRow(subctx);
  2395. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2396. doBuildRowAssignAggregate(subctx, targetRef, expr);
  2397. target->finishRow(subctx, targetRow);
  2398. }
  2399. void HqlCppTranslator::buildDatasetAssign(BuildCtx & ctx, IHqlCppDatasetBuilder * target, IHqlExpression * _expr)
  2400. {
  2401. OwnedHqlExpr expr = forceInlineAssignDataset(ctx, _expr);
  2402. bool isRowAssign = false;
  2403. BuildCtx subctx(ctx);
  2404. switch (expr->getOperator())
  2405. {
  2406. case no_addfiles:
  2407. buildDatasetAssign(subctx, target, expr->queryChild(0));
  2408. buildDatasetAssign(subctx, target, expr->queryChild(1));
  2409. return;
  2410. case no_temptable:
  2411. buildDatasetAssignTempTable(subctx, target, expr);
  2412. //MORE: Create rows and assign each one in turn. Could possibly be done with a different dataset selector
  2413. return;
  2414. case no_inlinedictionary:
  2415. case no_inlinetable:
  2416. buildDatasetAssignInlineTable(subctx, target, expr);
  2417. return;
  2418. case no_xmlproject:
  2419. buildDatasetAssignXmlProject(subctx, target, expr);
  2420. return;
  2421. case no_datasetfromrow:
  2422. {
  2423. isRowAssign = true;
  2424. expr.set(expr->queryChild(0));
  2425. break;
  2426. }
  2427. case no_if:
  2428. {
  2429. CHqlBoundExpr bound;
  2430. buildExpr(subctx, expr->queryChild(0), bound);
  2431. IHqlStmt * filter = subctx.addFilter(bound.expr);
  2432. buildDatasetAssign(subctx, target, expr->queryChild(1));
  2433. IHqlExpression * elseExpr = expr->queryChild(2);
  2434. if (elseExpr && elseExpr->getOperator() != no_null)
  2435. {
  2436. subctx.selectElse(filter);
  2437. buildDatasetAssign(subctx, target, elseExpr);
  2438. }
  2439. }
  2440. return;
  2441. case no_null:
  2442. return;
  2443. case no_activetable:
  2444. case no_temprow:
  2445. case no_createrow:
  2446. case no_projectrow:
  2447. case no_typetransfer:
  2448. isRowAssign = true;
  2449. break;
  2450. case no_newaggregate:
  2451. buildDatasetAssignAggregate(subctx, target, expr);
  2452. return;
  2453. case no_hqlproject:
  2454. case no_newusertable:
  2455. buildDatasetAssignProject(subctx, target, expr);
  2456. return;
  2457. case no_compound_childread:
  2458. case no_compound_childnormalize:
  2459. case no_compound_childaggregate:
  2460. case no_compound_selectnew:
  2461. case no_compound_inline:
  2462. case no_distributed:
  2463. case no_preservemeta:
  2464. case no_sorted:
  2465. case no_nofold:
  2466. case no_nohoist:
  2467. case no_section:
  2468. case no_sectioninput:
  2469. buildDatasetAssign(subctx, target, expr->queryChild(0));
  2470. return;
  2471. case no_alias_scope:
  2472. // expandAliasScope(subctx, expr);
  2473. buildDatasetAssign(subctx, target, expr->queryChild(0));
  2474. return;
  2475. case no_filter:
  2476. {
  2477. //We want to evaluate invariant conditions outside of the loop, rather than inside the dataset assignment
  2478. //Currently the test is whether the expression is independent of any tables. Better would be to
  2479. //see if the test was dependent on any of the datasets introduced by expr->queryChild(0).
  2480. HqlExprArray conds;
  2481. unwindFilterConditions(conds, expr);
  2482. IHqlExpression * ds = expr->queryChild(0);
  2483. #if 0
  2484. HqlExprCopyArray selectors;
  2485. loop
  2486. {
  2487. selectors.append(*ds->queryNormalizedSelector());
  2488. IHqlExpression * root = queryRoot(expr);
  2489. if (!root || root->getOperator() != no_select)
  2490. break;
  2491. bool isNew;
  2492. ds = querySelectorDataset(root, isNew);
  2493. if (!isNew)
  2494. break;
  2495. }
  2496. #endif
  2497. unsigned max = conds.ordinality();
  2498. unsigned i = 0;
  2499. bool optimized = false;
  2500. while (i < max)
  2501. {
  2502. IHqlExpression & cur = conds.item(i);
  2503. #if 0
  2504. bool overlap = false;
  2505. ForEachItemIn(j, selectors)
  2506. {
  2507. if (containsSelector(&cur, &selectors.item(j)))
  2508. {
  2509. overlap = true;
  2510. break;
  2511. }
  2512. }
  2513. #else
  2514. bool overlap = containsSelector(&cur, ds->queryNormalizedSelector());
  2515. #endif
  2516. if (!overlap)
  2517. {
  2518. buildFilter(subctx, &cur);
  2519. conds.remove(i);
  2520. optimized = true;
  2521. max--;
  2522. }
  2523. else
  2524. i++;
  2525. }
  2526. if (max == 0)
  2527. {
  2528. buildDatasetAssign(subctx, target, ds);
  2529. return;
  2530. }
  2531. if (optimized)
  2532. {
  2533. conds.add(*LINK(ds), 0);
  2534. expr.setown(expr->clone(conds));
  2535. }
  2536. break;
  2537. }
  2538. case no_join:
  2539. buildDatasetAssignJoin(subctx, target, expr);
  2540. return;
  2541. }
  2542. ITypeInfo * type = expr->queryType();
  2543. if (type)
  2544. {
  2545. switch (type->getTypeCode())
  2546. {
  2547. case type_record: // e.g. dataset.recordfield
  2548. case type_row:
  2549. isRowAssign = true;
  2550. break;
  2551. }
  2552. }
  2553. if (isRowAssign)
  2554. {
  2555. bool done = false;
  2556. subctx.addGroup();
  2557. //Some code primarily here to improve the generated code for productions inside parse statements for text parsing.
  2558. //see if we can replace a memcpy of the child record with a link...
  2559. if (!target->isRestricted())
  2560. {
  2561. switch (expr->getOperator())
  2562. {
  2563. case no_matchattr:
  2564. case no_left:
  2565. case no_right:
  2566. {
  2567. //Only try for really simple cases...
  2568. Owned<IReferenceSelector> sourceRef = buildNewRow(subctx, expr);
  2569. Owned<BoundRow> sourceRow = sourceRef->getRow(subctx);
  2570. if (!target->buildLinkRow(subctx, sourceRow))
  2571. {
  2572. BoundRow * targetRow = target->buildCreateRow(subctx);
  2573. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2574. buildRowAssign(subctx, targetRef, sourceRef);
  2575. target->finishRow(subctx, targetRow);
  2576. }
  2577. done = true;
  2578. break;
  2579. }
  2580. }
  2581. }
  2582. if (!done)
  2583. {
  2584. BoundRow * match = static_cast<BoundRow *>(ctx.queryAssociation(expr, AssocRow, NULL));
  2585. if (match && target->buildLinkRow(subctx, match))
  2586. done = true;
  2587. }
  2588. if (!done)
  2589. {
  2590. BoundRow * targetRow = target->buildCreateRow(subctx);
  2591. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2592. buildRowAssign(subctx, targetRef, expr);
  2593. target->finishRow(subctx, targetRow);
  2594. }
  2595. }
  2596. else
  2597. {
  2598. if (!target->buildAppendRows(subctx, expr))
  2599. {
  2600. BoundRow * sourceRow = buildDatasetIterate(subctx, expr, false);
  2601. if (sourceRow)
  2602. {
  2603. if (!target->buildLinkRow(subctx, sourceRow))
  2604. {
  2605. BoundRow * targetRow = target->buildCreateRow(subctx);
  2606. Owned<IReferenceSelector> targetRef = buildActiveRow(subctx, targetRow->querySelector());
  2607. Owned<IReferenceSelector> sourceRef = buildActiveRow(subctx, sourceRow->querySelector());
  2608. buildRowAssign(subctx, targetRef, sourceRef);
  2609. target->finishRow(subctx, targetRow);
  2610. }
  2611. }
  2612. }
  2613. }
  2614. }
  2615. //---------------------------------------------------------------------------
  2616. // Dataset iteration
  2617. BoundRow * HqlCppTranslator::buildDatasetIterateSelectN(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2618. {
  2619. OwnedHqlExpr counter = ctx.getTempDeclare(sizetType, NULL);
  2620. buildAssignToTemp(ctx, counter, expr->queryChild(1));
  2621. BoundRow * cursor = buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2622. if (cursor)
  2623. {
  2624. OwnedHqlExpr dec = createValue(no_predec, counter->getType(), LINK(counter));
  2625. OwnedHqlExpr test = createBoolExpr(no_eq, LINK(dec), getSizetConstant(0));
  2626. ctx.addFilter(test);
  2627. }
  2628. return cursor;
  2629. }
  2630. BoundRow * HqlCppTranslator::buildDatasetIterateChoosen(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2631. {
  2632. OwnedHqlExpr counter = ctx.getTempDeclare(sizetType, queryZero());
  2633. CHqlBoundExpr boundLow, boundHigh;
  2634. OwnedHqlExpr foldedHigh = foldHqlExpression(expr->queryChild(1));
  2635. if (!isChooseNAllLimit(foldedHigh))
  2636. buildSimpleExpr(ctx, foldedHigh, boundHigh);
  2637. if (queryRealChild(expr, 2))
  2638. {
  2639. OwnedHqlExpr foldedLow = foldHqlExpression(expr->queryChild(2));
  2640. OwnedHqlExpr low = adjustValue(foldedLow, -1);
  2641. buildSimpleExpr(ctx, low, boundLow);
  2642. if (boundHigh.expr)
  2643. boundHigh.expr.setown(createValue(no_add, LINK(boundHigh.queryType()), LINK(boundHigh.expr), LINK(boundLow.expr)));
  2644. }
  2645. BoundRow * cursor = buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2646. if (cursor)
  2647. {
  2648. OwnedHqlExpr inc = createValue(no_postinc, counter->getType(), LINK(counter));
  2649. ctx.addExpr(inc);
  2650. OwnedHqlExpr cond;
  2651. if (boundLow.expr)
  2652. extendConditionOwn(cond, no_and, createBoolExpr(no_gt, LINK(counter), LINK(boundLow.expr)));
  2653. if (boundHigh.expr)
  2654. extendConditionOwn(cond, no_and, createBoolExpr(no_le, LINK(counter), LINK(boundHigh.expr)));
  2655. if (cond)
  2656. ctx.addFilter(cond);
  2657. }
  2658. return cursor;
  2659. }
  2660. BoundRow * HqlCppTranslator::buildDatasetIterateLimit(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2661. {
  2662. OwnedHqlExpr counter = ctx.getTempDeclare(sizetType, queryZero());
  2663. CHqlBoundExpr boundHigh;
  2664. OwnedHqlExpr foldedHigh = foldHqlExpression(expr->queryChild(1));
  2665. buildSimpleExpr(ctx, foldedHigh, boundHigh);
  2666. BoundRow * cursor = buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2667. if (cursor)
  2668. {
  2669. OwnedHqlExpr inc = createValue(no_preinc, counter->getType(), LINK(counter));
  2670. OwnedHqlExpr cond = createBoolExpr(no_gt, LINK(inc), LINK(boundHigh.expr));
  2671. BuildCtx subctx(ctx);
  2672. subctx.addFilter(cond);
  2673. LinkedHqlExpr fail = expr->queryChild(2);
  2674. if (!fail || fail->isAttribute())
  2675. fail.setown(createFailAction("Limit exceeded", foldedHigh, NULL, queryCurrentActivityId(ctx)));
  2676. buildStmt(subctx, fail);
  2677. }
  2678. return cursor;
  2679. }
  2680. BoundRow * HqlCppTranslator::buildDatasetIterateProject(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2681. {
  2682. IHqlExpression * dataset = expr->queryChild(0);
  2683. OwnedHqlExpr counterVar;
  2684. IHqlExpression * counter = queryPropertyChild(expr, _countProject_Atom, 0);
  2685. if (counter)
  2686. {
  2687. counterVar.setown(ctx.getTempDeclare(unsignedType, queryZero()));
  2688. }
  2689. bool containsSkip = transformContainsSkip(expr->queryChild(1));
  2690. if (containsSkip)
  2691. needToBreak = true;
  2692. buildDatasetIterate(ctx, dataset, needToBreak);
  2693. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  2694. if (counter)
  2695. {
  2696. ctx.associateExpr(counter, counterVar);
  2697. OwnedHqlExpr inc = createValue(no_postinc, LINK(unsignedType), LINK(counterVar));
  2698. ctx.addExpr(inc);
  2699. }
  2700. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  2701. OwnedHqlExpr leftSelect = createSelector(no_left, dataset, querySelSeq(expr));
  2702. OwnedHqlExpr transform = replaceSelector(expr->queryChild(1), leftSelect, dataset->queryNormalizedSelector());
  2703. HqlExprAssociation * skipAssociation = NULL;
  2704. if (containsSkip)
  2705. {
  2706. OwnedHqlExpr callback = createUnknown(no_unknown, makeVoidType(), NULL, new InlineDatasetSkipCallback);
  2707. skipAssociation = ctx.associateExpr(skipActionMarker, callback);
  2708. }
  2709. doTransform(ctx, transform, rowBuilder);
  2710. ctx.removeAssociation(skipAssociation); //remove it in case keeping hold of it causes issues.
  2711. finalizeTempRow(ctx, tempRow, rowBuilder);
  2712. return bindTableCursor(ctx, expr, tempRow->queryBound());
  2713. }
  2714. BoundRow * HqlCppTranslator::buildDatasetIterateUserTable(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2715. {
  2716. IHqlExpression * dataset = expr->queryChild(0);
  2717. buildDatasetIterate(ctx, dataset, needToBreak);
  2718. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  2719. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  2720. doTransform(ctx, expr->queryChild(2), rowBuilder);
  2721. finalizeTempRow(ctx, tempRow, rowBuilder);
  2722. return bindTableCursor(ctx, expr, tempRow->queryBound());
  2723. }
  2724. BoundRow * HqlCppTranslator::buildDatasetIterateSpecialTempTable(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2725. {
  2726. IHqlExpression * values = expr->queryChild(0);
  2727. bool requiresTempRow = false;
  2728. ITypeInfo * type = values->queryType()->queryChildType();
  2729. switch (type->getTypeCode())
  2730. {
  2731. case type_alien:
  2732. case type_bitfield:
  2733. requiresTempRow = true;
  2734. break;
  2735. default:
  2736. if (type->getSize() == UNKNOWN_LENGTH)
  2737. requiresTempRow = true;
  2738. break;
  2739. }
  2740. Owned<IHqlCppSetCursor> cursor = createSetSelector(ctx, values);
  2741. CHqlBoundExpr boundCurElement;
  2742. cursor->buildIterateLoop(ctx, boundCurElement, false);
  2743. OwnedHqlExpr address = getPointer(boundCurElement.expr);
  2744. if (requiresTempRow)
  2745. {
  2746. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  2747. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  2748. IHqlExpression * record = expr->queryRecord();
  2749. OwnedHqlExpr target = createSelectExpr(LINK(rowBuilder->querySelector()), LINK(record->queryChild(0)));
  2750. OwnedHqlExpr curValue = boundCurElement.getTranslatedExpr();
  2751. buildAssign(ctx, target, curValue);
  2752. finalizeTempRow(ctx, tempRow, rowBuilder);
  2753. return bindTableCursor(ctx, expr, tempRow->queryBound());
  2754. }
  2755. else
  2756. {
  2757. assertex(!boundCurElement.length && !boundCurElement.count);
  2758. address.setown(createValue(no_implicitcast, makeRowReferenceType(expr), LINK(address)));
  2759. return bindTableCursor(ctx, expr, address);
  2760. }
  2761. }
  2762. BoundRow * HqlCppTranslator::buildDatasetIterateStreamedCall(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2763. {
  2764. CHqlBoundExpr bound;
  2765. doBuildExprCall(ctx, expr, bound);
  2766. ITypeInfo * exprType = expr->queryType();
  2767. Owned<ITypeInfo> wrappedType = makeWrapperModifier(LINK(exprType));
  2768. OwnedHqlExpr temp = ctx.getTempDeclare(wrappedType, bound.expr);
  2769. ctx.addLoop(NULL, NULL, false);
  2770. Owned<ITypeInfo> wrappedRowType = makeWrapperModifier(LINK(queryRowType(exprType)));
  2771. OwnedHqlExpr tempRow = ctx.getTempDeclare(wrappedRowType, NULL);
  2772. StringBuffer s;
  2773. generateExprCpp(s, tempRow).append(".setown(");
  2774. generateExprCpp(s, temp).append("->nextRow());");
  2775. ctx.addQuoted(s);
  2776. s.clear().append("if (!");generateExprCpp(s, tempRow).append(".getbytes()) break;");
  2777. ctx.addQuoted(s);
  2778. return bindTableCursor(ctx, expr, tempRow);
  2779. }
  2780. BoundRow * HqlCppTranslator::buildDatasetIterate(BuildCtx & ctx, IHqlExpression * expr, bool needToBreak)
  2781. {
  2782. if (!canProcessInline(&ctx, expr))
  2783. {
  2784. Owned<IHqlCppDatasetCursor> cursor = createDatasetSelector(ctx, expr);
  2785. return cursor->buildIterateLoop(ctx, needToBreak);
  2786. }
  2787. switch (expr->getOperator())
  2788. {
  2789. case no_dataset_alias:
  2790. if (!expr->hasProperty(_normalized_Atom))
  2791. {
  2792. OwnedHqlExpr uniqueChild = normalizeDatasetAlias(expr);
  2793. BoundRow * childCursor = buildDatasetIterate(ctx, uniqueChild, needToBreak);
  2794. return rebindTableCursor(ctx, expr, childCursor, no_none, NULL);
  2795. }
  2796. else
  2797. {
  2798. throwUnexpected();
  2799. //The following would only be triggered for a splitter (not yet generated), and that would require
  2800. //disambiguation when that was built.
  2801. BoundRow * childCursor = buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2802. return rebindTableCursor(ctx, expr, childCursor, no_none, NULL);
  2803. }
  2804. case no_null:
  2805. buildFilter(ctx, queryBoolExpr(false));
  2806. return NULL;
  2807. case no_filter:
  2808. {
  2809. IHqlExpression * dataset = expr->queryChild(0);
  2810. #ifdef _OPTIMZE_INLINE_FILTERS_
  2811. //Good code, but messes up accidental cse in some createSegmentMonitor calls.
  2812. HqlExprAttr invariant;
  2813. OwnedHqlExpr cond = extractFilterConditions(invariant, expr, dataset);
  2814. if (invariant)
  2815. buildFilter(ctx, invariant);
  2816. //MORE: if (canAssignInline(ctx, ds) && !canIterateInline(ctx, ds)) break;
  2817. BoundRow * cursor = buildDatasetIterate(ctx, dataset, needToBreak);
  2818. if (cond)
  2819. buildFilter(ctx, cond);
  2820. #else
  2821. //MORE: if (canAssignInline(ctx, ds) && !canIterateInline(ctx, ds)) break;
  2822. BoundRow * cursor = buildDatasetIterate(ctx, dataset, needToBreak);
  2823. unsigned max = expr->numChildren();
  2824. for (unsigned i=1; i < max; i++)
  2825. buildFilter(ctx, expr->queryChild(i));
  2826. #endif
  2827. return cursor;
  2828. }
  2829. case no_id2blob:
  2830. case no_select:
  2831. case no_translated:
  2832. {
  2833. Owned<IHqlCppDatasetCursor> cursor = createDatasetSelector(ctx, expr);
  2834. return cursor->buildIterateLoop(ctx, needToBreak);
  2835. }
  2836. case no_choosen:
  2837. return buildDatasetIterateChoosen(ctx, expr, needToBreak);
  2838. case no_limit:
  2839. return buildDatasetIterateLimit(ctx, expr, needToBreak);
  2840. case no_index:
  2841. case no_selectnth:
  2842. return buildDatasetIterateSelectN(ctx, expr, needToBreak);
  2843. case no_hqlproject:
  2844. return buildDatasetIterateProject(ctx, expr, needToBreak);
  2845. case no_newusertable:
  2846. return buildDatasetIterateUserTable(ctx, expr, needToBreak);
  2847. case no_compound_childread:
  2848. case no_compound_childnormalize:
  2849. case no_compound_childaggregate:
  2850. case no_compound_selectnew:
  2851. case no_compound_inline:
  2852. case no_distributed:
  2853. case no_preservemeta:
  2854. case no_sorted:
  2855. case no_nofold:
  2856. case no_nohoist:
  2857. return buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2858. case no_sectioninput:
  2859. case no_section:
  2860. {
  2861. BoundRow * row = buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2862. #ifdef _DEBUG
  2863. StringBuffer s;
  2864. if (expr->getOperator() == no_section)
  2865. s.append("//---- section ");
  2866. else
  2867. s.append("//---- end section ");
  2868. getStringValue(s, expr->queryChild(1), "<?>").append(" ");
  2869. getStringValue(s, expr->queryChild(2)).append("----");
  2870. ctx.addQuoted(s);
  2871. #endif
  2872. return row;
  2873. }
  2874. case no_alias_scope:
  2875. // expandAliasScope(ctx, expr);
  2876. return buildDatasetIterate(ctx, expr->queryChild(0), needToBreak);
  2877. case no_temptable:
  2878. {
  2879. IHqlExpression * values = expr->queryChild(0);
  2880. if (values->queryType()->getTypeCode() == type_set)
  2881. {
  2882. if (values->getOperator() == no_alias)
  2883. values = values->queryChild(0);
  2884. bool special = false;
  2885. switch (values->getOperator())
  2886. {
  2887. case no_getresult:
  2888. case no_null:
  2889. special = true;
  2890. break;
  2891. }
  2892. if (special)
  2893. return buildDatasetIterateSpecialTempTable(ctx, expr, needToBreak);
  2894. }
  2895. break;
  2896. }
  2897. case no_call:
  2898. case no_externalcall:
  2899. if (hasStreamedModifier(expr->queryType()))
  2900. return buildDatasetIterateStreamedCall(ctx, expr, needToBreak);
  2901. break;
  2902. #if 0
  2903. //Following should improve the code, but I'm not sure how to correctly convert a referenceSelector to a boundExpr (since it may be with an existing row)
  2904. case no_datasetfromrow:
  2905. if (!needToBreak)
  2906. {
  2907. BoundRow * row = buildNewRow(ctx, expr->queryChild(0));
  2908. bindTableCursor(ctx, expr, tempRow->queryBound(), no_none, NULL);
  2909. return row;
  2910. }
  2911. break;
  2912. #endif
  2913. }
  2914. Owned<IHqlCppDatasetCursor> cursor = createDatasetSelector(ctx, expr);
  2915. return cursor->buildIterateLoop(ctx, needToBreak);
  2916. }
  2917. //---------------------------------------------------------------------------
  2918. // Row Assignment
  2919. void HqlCppTranslator::buildCompoundAssign(BuildCtx & ctx, IHqlExpression * left, IReferenceSelector * leftSelector, IHqlExpression * rightScope, IHqlExpression * rightSelector)
  2920. {
  2921. switch (left->getOperator())
  2922. {
  2923. case no_ifblock:
  2924. {
  2925. BuildCtx subctx(ctx);
  2926. OwnedHqlExpr test = replaceSelector(left->queryChild(0), querySelfReference(), leftSelector->queryExpr());
  2927. buildFilter(subctx, test);
  2928. buildCompoundAssign(subctx, left->queryChild(1), leftSelector, rightScope, rightSelector);
  2929. //This calculates the size of the previous block. It means that subsequent uses of the
  2930. //offsets are cached - even if they are inside another ifblock().
  2931. CHqlBoundExpr bound;
  2932. IHqlExpression * mapexpr = createSelectExpr(LINK(leftSelector->queryExpr()), LINK(left));
  2933. OwnedHqlExpr size = createValue(no_sizeof, makeIntType(4,false), mapexpr);
  2934. buildCachedExpr(ctx, size, bound);
  2935. }
  2936. break;
  2937. case no_field:
  2938. {
  2939. Owned<IReferenceSelector> selectedLeft = leftSelector->select(ctx, left);
  2940. OwnedHqlExpr selectedRight;
  2941. IHqlExpression * leftRecord = ::queryRecord(leftSelector->queryType());
  2942. if (!rightScope || (leftRecord==rightScope))
  2943. {
  2944. selectedRight.setown(createSelectExpr(LINK(rightSelector), LINK(left)));
  2945. }
  2946. else
  2947. {
  2948. IHqlSimpleScope * scope = rightScope->querySimpleScope();
  2949. IHqlExpression * resolved = scope->lookupSymbol(left->queryName());
  2950. assertex(resolved);
  2951. selectedRight.setown(createSelectExpr(LINK(rightSelector), resolved));
  2952. }
  2953. if (left->queryType()->getTypeCode() == type_row)
  2954. buildCompoundAssign(ctx, left->queryRecord(), selectedLeft, selectedRight->queryRecord(), selectedRight);
  2955. else
  2956. selectedLeft->set(ctx, selectedRight);
  2957. break;
  2958. }
  2959. break;
  2960. case no_record:
  2961. {
  2962. ForEachChild(i, left)
  2963. buildCompoundAssign(ctx, left->queryChild(i), leftSelector, rightScope, rightSelector);
  2964. break;
  2965. }
  2966. case no_attr:
  2967. case no_attr_expr:
  2968. case no_attr_link:
  2969. break;
  2970. default:
  2971. UNIMPLEMENTED;
  2972. }
  2973. }
  2974. void HqlCppTranslator::buildCompoundAssign(BuildCtx & ctx, IHqlExpression * left, IHqlExpression * right)
  2975. {
  2976. //MORE: May need to resolve lhs and rhs in scope to find out the record type.
  2977. // buildCompoundAssign(ctx, column->queryRecord(), selector->queryExpr(), value->queryRecord(), value);
  2978. UNIMPLEMENTED;
  2979. }
  2980. void HqlCppTranslator::doBuildRowAssignAggregateClear(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  2981. {
  2982. IHqlExpression * transform = expr->queryChild(2);
  2983. unsigned numAggregates = transform->numChildren();
  2984. unsigned idx;
  2985. OwnedHqlExpr self = getSelf(expr);
  2986. for (idx = 0; idx < numAggregates; idx++)
  2987. {
  2988. IHqlExpression * cur = transform->queryChild(idx);
  2989. Owned<IReferenceSelector> curTarget = createSelfSelect(ctx, target, cur->queryChild(0), self);
  2990. IHqlExpression * src = cur->queryChild(1);
  2991. switch (src->getOperator())
  2992. {
  2993. case no_countgroup:
  2994. case no_maxgroup:
  2995. case no_mingroup:
  2996. case no_sumgroup:
  2997. case no_existsgroup:
  2998. curTarget->buildClear(ctx, 0);
  2999. break;
  3000. default:
  3001. if (src->isConstant())
  3002. curTarget->set(ctx, src);
  3003. else
  3004. curTarget->buildClear(ctx, 0);
  3005. break;
  3006. }
  3007. }
  3008. }
  3009. void HqlCppTranslator::doBuildRowAssignAggregateNext(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr, bool isSingleExists, IHqlExpression * alreadyDoneExpr)
  3010. {
  3011. IHqlExpression * transform = expr->queryChild(2);
  3012. unsigned numAggregates = transform->numChildren();
  3013. OwnedHqlExpr self = getSelf(expr);
  3014. OwnedHqlExpr notAlreadyDone = alreadyDoneExpr ? getInverse(alreadyDoneExpr) : NULL;
  3015. bool isVariableOffset = false;
  3016. for (unsigned idx = 0; idx < numAggregates; idx++)
  3017. {
  3018. IHqlExpression * cur = transform->queryChild(idx);
  3019. if (cur->isAttribute())
  3020. continue;
  3021. IHqlExpression * targetSelect = cur->queryChild(0);
  3022. Owned<IReferenceSelector> curTarget = createSelfSelect(ctx, target, targetSelect, self);
  3023. IHqlExpression * src = cur->queryChild(1);
  3024. IHqlExpression * arg = src->queryChild(0);
  3025. IHqlExpression * cond = src->queryChild(1);
  3026. BuildCtx condctx(ctx);
  3027. node_operator srcOp = src->getOperator();
  3028. switch (srcOp)
  3029. {
  3030. case no_countgroup:
  3031. {
  3032. assertex(!(arg && isVariableOffset));
  3033. if (arg)
  3034. buildFilter(condctx, arg);
  3035. OwnedHqlExpr one = getSizetConstant(1);
  3036. if (isVariableOffset)
  3037. {
  3038. IHqlStmt * ifStmt = condctx.addFilter(notAlreadyDone);
  3039. curTarget->set(condctx, one);
  3040. condctx.selectElse(ifStmt);
  3041. }
  3042. buildIncrementAssign(condctx, curTarget, one);
  3043. }
  3044. break;
  3045. case no_sumgroup:
  3046. {
  3047. assertex(!(cond && isVariableOffset));
  3048. if (cond)
  3049. buildFilter(condctx, cond);
  3050. if (isVariableOffset)
  3051. {
  3052. IHqlStmt * ifStmt = condctx.addFilter(notAlreadyDone);
  3053. curTarget->set(condctx, arg);
  3054. condctx.selectElse(ifStmt);
  3055. }
  3056. buildIncrementAssign(condctx, curTarget, arg);
  3057. }
  3058. break;
  3059. case no_maxgroup:
  3060. case no_mingroup:
  3061. {
  3062. node_operator op = (srcOp == no_maxgroup) ? no_gt : no_lt;
  3063. assertex(!cond);
  3064. OwnedHqlExpr castArg = ensureExprType(arg, targetSelect->queryType()); // cast to correct type, assume it can fit in the target type.
  3065. OwnedHqlExpr temp = buildSimplifyExpr(condctx, castArg);
  3066. OwnedHqlExpr compare = createBoolExpr (op, LINK(temp), LINK(curTarget->queryExpr()));
  3067. if (notAlreadyDone)
  3068. compare.setown(createBoolExpr(no_or, LINK(notAlreadyDone), compare.getClear()));
  3069. buildFilter(condctx, compare);
  3070. curTarget->set(condctx, temp);
  3071. }
  3072. break;
  3073. case no_existsgroup:
  3074. assertex(!(arg && isVariableOffset));
  3075. if (arg)
  3076. buildFilter(condctx, arg);
  3077. curTarget->set(condctx, queryBoolExpr(true));
  3078. if (isSingleExists)
  3079. condctx.addBreak();
  3080. break;
  3081. default:
  3082. if (!src->isConstant() || isVariableOffset)
  3083. {
  3084. condctx.addFilter(notAlreadyDone);
  3085. curTarget->set(condctx, src);
  3086. }
  3087. break;
  3088. }
  3089. if (targetSelect->queryType()->getSize() == UNKNOWN_LENGTH)
  3090. isVariableOffset = true;
  3091. }
  3092. if (alreadyDoneExpr)
  3093. buildAssignToTemp(ctx, alreadyDoneExpr, queryBoolExpr(true));
  3094. }
  3095. void HqlCppTranslator::doBuildRowAssignAggregate(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3096. {
  3097. IHqlExpression * dataset = expr->queryChild(0);
  3098. IHqlExpression * transform = expr->queryChild(2);
  3099. unsigned numAggregates = transform->numChildren();
  3100. if (isKeyedCountAggregate(expr))
  3101. throwError(HQLERR_KeyedCountNonKeyable);
  3102. bool needGuard = false;
  3103. bool isSingleExists = true;
  3104. unsigned idx;
  3105. for (idx = 0; idx < numAggregates; idx++)
  3106. {
  3107. IHqlExpression * cur = transform->queryChild(idx);
  3108. if (cur->isAttribute())
  3109. continue;
  3110. IHqlExpression * tgt = cur->queryChild(0);
  3111. IHqlExpression * src = cur->queryChild(1);
  3112. switch (src->getOperator())
  3113. {
  3114. case no_countgroup:
  3115. case no_sumgroup:
  3116. isSingleExists = false;
  3117. break;
  3118. case no_existsgroup:
  3119. break;
  3120. case no_mingroup:
  3121. isSingleExists = false;
  3122. needGuard = true;
  3123. break;
  3124. case no_maxgroup:
  3125. isSingleExists = false;
  3126. if (!isNullValueMinimumValue(src->queryType()))
  3127. needGuard = true;
  3128. break;
  3129. default:
  3130. isSingleExists = false;
  3131. if (!src->isConstant())
  3132. needGuard = true;
  3133. break;
  3134. }
  3135. if ((tgt->queryType()->getSize() == UNKNOWN_LENGTH) && (idx+1 != numAggregates))
  3136. needGuard = true;
  3137. }
  3138. OwnedHqlExpr guard;
  3139. if (needGuard)
  3140. {
  3141. Owned<ITypeInfo> boolType = makeBoolType();
  3142. guard.setown(ctx.getTempDeclare(boolType, queryBoolExpr(false)));
  3143. }
  3144. doBuildRowAssignAggregateClear(ctx, target, expr);
  3145. BuildCtx condctx(ctx);
  3146. BoundRow * cursor = buildDatasetIterate(condctx, dataset, isSingleExists);
  3147. doBuildRowAssignAggregateNext(condctx, target, expr, isSingleExists, guard);
  3148. }
  3149. void HqlCppTranslator::doBuildRowAssignProject(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3150. {
  3151. IHqlExpression * dataset = expr->queryChild(0);
  3152. IHqlExpression * counter = queryPropertyChild(expr, _countProject_Atom, 0);
  3153. if (counter && !ctx.queryMatchExpr(counter))
  3154. throwError(HQLERR_CounterNotFound);
  3155. BuildCtx subctx(ctx);
  3156. assertex(target->isRoot());
  3157. IHqlExpression * selSeq = querySelSeq(expr);
  3158. OwnedHqlExpr leftSelect = createSelector(no_left, dataset, selSeq);
  3159. OwnedHqlExpr activeDataset = ensureActiveRow(dataset->queryNormalizedSelector());
  3160. OwnedHqlExpr transform = queryNewReplaceSelector(expr->queryChild(1), leftSelect, activeDataset);
  3161. Owned<BoundRow> selfCursor;
  3162. if (!transform)
  3163. {
  3164. subctx.addGroup();
  3165. selfCursor.set(bindSelectorAsSelf(subctx, target, expr));
  3166. //Mapping may potentially be ambiguous, so do things correctly (see hqlsource for details)
  3167. BoundRow * prevCursor = resolveSelectorDataset(subctx, dataset->queryNormalizedSelector());
  3168. transform.set(expr->queryChild(1));
  3169. bindTableCursor(subctx, dataset, prevCursor->queryBound(), no_left, selSeq);
  3170. }
  3171. else
  3172. {
  3173. //Not introducing any new left rows => no problem assigning to target selector
  3174. selfCursor.setown(target->getRow(subctx));
  3175. }
  3176. doTransform(subctx, transform, selfCursor);
  3177. }
  3178. void HqlCppTranslator::doBuildRowAssignCreateRow(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3179. {
  3180. IHqlExpression * transform = expr->queryChild(0);
  3181. if (transform->isConstant())
  3182. {
  3183. #ifdef USE_CONSTANT_ROW_FOR_ASSIGN
  3184. //Generally not worthwhile - unless maybe it has a large number of fields....
  3185. CHqlBoundExpr bound;
  3186. if (doBuildRowConstantTransform(transform, bound))
  3187. {
  3188. OwnedHqlExpr raw = bound.getTranslatedExpr();
  3189. buildRowAssign(ctx, target, raw);
  3190. return;
  3191. }
  3192. #endif
  3193. }
  3194. Owned<BoundRow> selfCursor = target->getRow(ctx);
  3195. doTransform(ctx, transform, selfCursor);
  3196. }
  3197. void HqlCppTranslator::doBuildRowAssignNullRow(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3198. {
  3199. target->buildClear(ctx, 0);
  3200. }
  3201. void HqlCppTranslator::doBuildRowAssignProjectRow(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3202. {
  3203. IHqlExpression * srcRow = expr->queryChild(0);
  3204. IHqlExpression * transform = expr->queryChild(1);
  3205. Owned<IReferenceSelector> source = buildNewRow(ctx, srcRow);
  3206. BuildCtx subctx(ctx);
  3207. OwnedHqlExpr leftSelect = createSelector(no_left, srcRow, querySelSeq(expr));
  3208. OwnedHqlExpr newTransform = replaceSelector(transform, leftSelect, srcRow);
  3209. Owned<BoundRow> selfCursor = target->getRow(subctx);
  3210. doTransform(subctx, newTransform, selfCursor);
  3211. }
  3212. void HqlCppTranslator::doBuildRowAssignSerializeRow(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3213. {
  3214. IHqlExpression * srcRow = expr->queryChild(0);
  3215. Owned<IReferenceSelector> source = buildNewRow(ctx, srcRow);
  3216. BuildCtx subctx(ctx);
  3217. Owned<BoundRow> leftCursor = source->getRow(subctx);
  3218. BoundRow * selfCursor = bindSelectorAsSelf(subctx, target, expr);
  3219. IHqlExpression * unserializedRecord = srcRow->queryRecord();
  3220. //If builder isn't provided then the target must be a fixed size record that doesn't require serialization
  3221. //Therefore this should never be called.
  3222. assertex(selfCursor->queryBuilder());
  3223. {
  3224. HqlExprArray args;
  3225. args.append(*createRowSerializer(ctx, unserializedRecord, serializerAtom));
  3226. args.append(*LINK(srcRow));
  3227. Owned<ITypeInfo> type = makeTransformType(expr->queryRecord()->getType());
  3228. OwnedHqlExpr call = bindFunctionCall(rtlSerializeToBuilderAtom, args, type);
  3229. doTransform(subctx, call, selfCursor);
  3230. //MORE: This doesn't associated the returned size with the target if assigned to a child field.
  3231. //very unusual code, so not too concerned.
  3232. }
  3233. subctx.removeAssociation(selfCursor);
  3234. }
  3235. void HqlCppTranslator::doBuildRowAssignUserTable(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3236. {
  3237. Owned<BoundRow> selfCursor = target->getRow(ctx);
  3238. doTransform(ctx, expr->queryChild(2), selfCursor);
  3239. }
  3240. void HqlCppTranslator::buildRowAssign(BuildCtx & ctx, IReferenceSelector * target, IHqlExpression * expr)
  3241. {
  3242. switch (expr->getOperator())
  3243. {
  3244. case no_temprow:
  3245. throwUnexpected();
  3246. case no_projectrow:
  3247. doBuildRowAssignProjectRow(ctx, target, expr);
  3248. return;
  3249. case no_createrow:
  3250. doBuildRowAssignCreateRow(ctx, target, expr);
  3251. return;
  3252. case no_null:
  3253. doBuildRowAssignNullRow(ctx, target, expr);
  3254. return;
  3255. case no_serialize:
  3256. if (isDummySerializeDeserialize(expr))
  3257. buildRowAssign(ctx, target, expr->queryChild(0)->queryChild(0));
  3258. else
  3259. doBuildRowAssignSerializeRow(ctx, target, expr);
  3260. return;
  3261. case no_if:
  3262. {
  3263. OwnedHqlExpr foldedCond = foldHqlExpression(expr->queryChild(0));
  3264. if (foldedCond->queryValue())
  3265. {
  3266. unsigned branch = (foldedCond->queryValue()->getBoolValue()) ? 1 : 2;
  3267. buildRowAssign(ctx, target, expr->queryChild(branch));
  3268. return;
  3269. }
  3270. //Assigning a variable size record can mean that references to self need recalculating outside of the condition,
  3271. //producing poor code.
  3272. if (!isVariableSizeRecord(expr->queryRecord()))
  3273. {
  3274. BuildCtx condctx(ctx);
  3275. IHqlStmt * cond = buildFilterViaExpr(condctx, foldedCond);
  3276. buildRowAssign(condctx, target, expr->queryChild(1));
  3277. condctx.selectElse(cond);
  3278. buildRowAssign(condctx, target, expr->queryChild(2));
  3279. return;
  3280. }
  3281. }
  3282. break;
  3283. /*
  3284. case no_externalcall:
  3285. //MORE: Should assign directly to the target, but may not be very easy....
  3286. if (target->isBinary() && queryRecord(source->queryType()) == expr->queryRecord())
  3287. {
  3288. CHqlBoundExpr address;
  3289. target->buildAddress(ctx, address);
  3290. row.setown(createValue(no_typetransfer, makeReferenceModifier(LINK(selector->queryType())), LINK(address.expr)));
  3291. doBuildCall(ctx, &target, expr, NULL);
  3292. }
  3293. break;
  3294. */
  3295. case no_comma:
  3296. case no_compound:
  3297. buildStmt(ctx, expr->queryChild(0));
  3298. buildRowAssign(ctx, target, expr->queryChild(1));
  3299. return;
  3300. }
  3301. Owned<IReferenceSelector> src = buildNewRow(ctx, expr);
  3302. buildRowAssign(ctx, target, src);
  3303. }
  3304. void HqlCppTranslator::buildRowAssign(BuildCtx & ctx, IReferenceSelector * target, IReferenceSelector * source)
  3305. {
  3306. BuildCtx subctx(ctx);
  3307. IHqlExpression * sourceRecord = ::queryRecord(source->queryType());
  3308. IHqlExpression * targetRecord = ::queryRecord(target->queryType());
  3309. //if record structures are identical, then we must just be able to block copy the information across.
  3310. bool useMemcpy = (sourceRecord == targetRecord) && source->isBinary() && !source->isConditional() &&
  3311. !recordRequiresSerialization(sourceRecord);
  3312. if (useMemcpy)
  3313. {
  3314. if (source->queryRootRow()->isConditional())
  3315. {
  3316. IHqlStmt * ifStmt = subctx.addFilter(source->queryRootRow()->queryBound());
  3317. target->setRow(subctx, source);
  3318. subctx.selectElse(ifStmt);
  3319. target->buildClear(subctx, 0);
  3320. }
  3321. else
  3322. target->setRow(subctx, source);
  3323. }
  3324. else
  3325. buildCompoundAssign(subctx, targetRecord, target, sourceRecord, source->queryExpr());
  3326. }
  3327. //---------------------------------------------------------------------------
  3328. // Dataset selection
  3329. IReferenceSelector * HqlCppTranslator::doBuildRowSelectTop(BuildCtx & ctx, IHqlExpression * expr)
  3330. {
  3331. //create a temporary
  3332. Owned<ITypeInfo> rowType = makeReferenceModifier(expr->getType());
  3333. OwnedHqlExpr rowExpr = ctx.getTempDeclare(rowType, NULL);
  3334. Owned<BoundRow> row = createBoundRow(expr, rowExpr);
  3335. ctx.associate(*row); // associate here because it is compared inside the loop
  3336. CHqlBoundExpr boundCleared;
  3337. #ifdef CREATE_DEAULT_ROW_IF_NULL
  3338. buildDefaultRow(ctx, expr, boundCleared);
  3339. #else
  3340. buildNullRow(ctx, expr, boundCleared);
  3341. #endif
  3342. OwnedHqlExpr defaultRowPtr = getPointer(boundCleared.expr);
  3343. //Declare row for final level, iterate the appropriate number of times, and then assign and break.
  3344. BuildCtx initctx(ctx);
  3345. initctx.addGroup(); // add a group to allow a temporary to be declared later.
  3346. initctx.addAssign(rowExpr, defaultRowPtr);
  3347. HqlExprAssociation * savedMarker = ctx.associateExpr(queryConditionalRowMarker(), rowExpr);
  3348. BuildCtx iterctx(ctx);
  3349. IHqlExpression * sort = expr->queryChild(0);
  3350. IHqlExpression * dataset = sort->queryChild(0);
  3351. IHqlExpression * sortList = sort->queryChild(1);
  3352. BoundRow * chooseCursor = buildDatasetIterate(iterctx, dataset, false);
  3353. //if (!best) { best=row; } else { if (next < best) best = row; } Must short-circuit the test of best
  3354. OwnedHqlExpr testBest = createBoolExpr(no_eq, LINK(rowExpr), LINK(defaultRowPtr));
  3355. IHqlStmt * ifStmt = iterctx.addFilter(testBest);
  3356. {
  3357. OwnedHqlExpr source = ensureIteratedRowIsLive(initctx, ctx, iterctx, chooseCursor, dataset, expr);
  3358. OwnedHqlExpr castLeftRow = createValue(no_implicitcast, LINK(rowType), LINK(source));//chooseCursor->queryBound()));
  3359. iterctx.addAssign(rowExpr, castLeftRow);
  3360. }
  3361. iterctx.selectElse(ifStmt);
  3362. CHqlBoundExpr bound;
  3363. buildOrderedCompare(iterctx, dataset, sortList, bound, dataset, expr);
  3364. OwnedHqlExpr compare = createBoolExpr(no_lt, LINK(bound.expr), getZero());
  3365. iterctx.addFilter(compare);
  3366. {
  3367. OwnedHqlExpr source = ensureIteratedRowIsLive(initctx, ctx, iterctx, chooseCursor, dataset, expr);
  3368. OwnedHqlExpr castLeftRow = createValue(no_implicitcast, LINK(rowType), LINK(source));//chooseCursor->queryBound()));
  3369. iterctx.addAssign(rowExpr, castLeftRow);
  3370. }
  3371. ctx.removeAssociation(savedMarker);
  3372. //Set conditional later on, so test in main loop is explicit
  3373. #ifndef CREATE_DEAULT_ROW_IF_NULL
  3374. row->setConditional(true);
  3375. #endif
  3376. return createReferenceSelector(row);
  3377. }
  3378. BoundRow * HqlCppTranslator::buildOptimizeSelectFirstRow(BuildCtx & ctx, IHqlExpression * expr)
  3379. {
  3380. BoundRow * parentRow = NULL;
  3381. node_operator op = expr->getOperator();
  3382. switch (op)
  3383. {
  3384. case no_compound_childaggregate:
  3385. return buildOptimizeSelectFirstRow(ctx, expr->queryChild(0));
  3386. case no_hqlproject:
  3387. case no_newusertable:
  3388. {
  3389. parentRow = buildOptimizeSelectFirstRow(ctx, expr->queryChild(0));
  3390. if (!parentRow)
  3391. return NULL;
  3392. }
  3393. //fall through
  3394. case no_newaggregate:
  3395. {
  3396. Owned<BoundRow> tempRow = declareTempAnonRow(ctx, ctx, expr);
  3397. Owned<BoundRow> rowBuilder = createRowBuilder(ctx, tempRow);
  3398. Owned<IReferenceSelector> createdRef = createReferenceSelector(rowBuilder);
  3399. BuildCtx subctx(ctx);
  3400. subctx.addGroup();
  3401. if (parentRow)
  3402. {
  3403. if (op == no_hqlproject)
  3404. bindTableCursor(ctx, expr->queryChild(0), tempRow->queryBound(), no_left, querySelSeq(expr));
  3405. else
  3406. bindTableCursor(ctx, expr->queryChild(0), tempRow->queryBound(), no_none, NULL);
  3407. }
  3408. doBuildRowAssignAggregate(subctx, createdRef, expr);
  3409. finalizeTempRow(ctx, tempRow, rowBuilder);
  3410. return tempRow;
  3411. }
  3412. //
  3413. default:
  3414. return NULL;
  3415. }
  3416. }
  3417. void HqlCppTranslator::convertBoundDatasetToFirstRow(IHqlExpression * expr, CHqlBoundExpr & bound)
  3418. {
  3419. Owned<ITypeInfo> type = makeReferenceModifier(expr->getType());
  3420. ITypeInfo * boundType = bound.queryType();
  3421. if (isArrayRowset(boundType))
  3422. {
  3423. Linked<ITypeInfo> rowType = queryUnqualifiedType(queryRowType(boundType));
  3424. rowType.setown(makeReferenceModifier(LINK(rowType)));
  3425. if (hasLinkedRow(boundType))
  3426. type.setown(setLinkCountedAttr(type, true));
  3427. bound.expr.setown(createValue(no_deref, LINK(rowType), LINK(bound.expr)));
  3428. }
  3429. else if (bound.queryType()->isReference())
  3430. bound.expr.setown(createValue(no_typetransfer, LINK(type), bound.expr.getClear()));
  3431. else
  3432. bound.expr.setown(createValue(no_implicitcast, LINK(type), bound.expr.getClear()));
  3433. }
  3434. void HqlCppTranslator::convertBoundRowToDataset(BuildCtx & ctx, CHqlBoundExpr & bound, const BoundRow * row, ExpressionFormat preferredFormat)
  3435. {
  3436. IHqlExpression * boundRow = row->queryBound();
  3437. IHqlExpression * record = row->queryDataset()->queryRecord();
  3438. Owned<ITypeInfo> type = makeTableType(makeRowType(LINK(record->queryType())), NULL, NULL, NULL);
  3439. Owned<ITypeInfo> refType = makeReferenceModifier(LINK(type));
  3440. if (hasLinkCountedModifier(boundRow->queryType()) && (preferredFormat != FormatBlockedDataset))
  3441. {
  3442. OwnedHqlExpr curActivityId = getCurrentActivityId(ctx);
  3443. StringBuffer allocatorName;
  3444. ensureRowAllocator(allocatorName, ctx, record, curActivityId);
  3445. OwnedHqlExpr src = getPointer(boundRow);
  3446. //We can't just take the address of the link counted row, have to create a temporary dataset
  3447. //could be fixed once the link counting is tracked on rows and datasets
  3448. CHqlBoundTarget target;
  3449. createTempFor(ctx, type, target, typemod_none, FormatLinkedDataset);
  3450. StringBuffer s;
  3451. generateExprCpp(s, target.expr).append(".setRow(").append(allocatorName).append(",");
  3452. generateExprCpp(s, src).append(");");
  3453. ctx.addQuoted(s);
  3454. bound.setFromTarget(target);
  3455. }
  3456. else if (boundRow->queryType()->isReference())
  3457. bound.expr.setown(createValue(no_typetransfer, LINK(refType), LINK(boundRow)));
  3458. else
  3459. bound.expr.setown(createValue(no_implicitcast, LINK(refType), LINK(boundRow)));
  3460. bound.count.setown(getSizetConstant(1));
  3461. }
  3462. IHqlExpression * HqlCppTranslator::ensureIteratedRowIsLive(BuildCtx & initctx, BuildCtx & searchctx, BuildCtx & iterctx, BoundRow * row, IHqlExpression * dataset, IHqlExpression * rowExpr)
  3463. {
  3464. //The problem is that we are iterating through the rows in a dataset, and we need to access the "best" row outside of the loop
  3465. //However subsequent iterations of the loop might invalidate the current row, so we need to ensure the best row is retained.
  3466. //There should really be a better way, but it isn't too bad with link counted rows.
  3467. bool needToPreserve = false;
  3468. IHqlExpression * ds = dataset;
  3469. while (ds && !needToPreserve)
  3470. {
  3471. if (ds->isDatarow())
  3472. {
  3473. if (initctx.queryAssociation(ds, AssocRow, NULL))
  3474. break;
  3475. }
  3476. else
  3477. {
  3478. if (initctx.queryMatchExpr(ds))
  3479. break;
  3480. }
  3481. switch (ds->getOperator())
  3482. {
  3483. case no_filter:
  3484. case no_grouped:
  3485. case no_stepped:
  3486. case no_sorted:
  3487. case no_distributed:
  3488. case no_preservemeta:
  3489. case no_choosen:
  3490. case no_selectnth: // can occur as the lhs of no_select
  3491. case no_compound_childnormalize:
  3492. case no_compound_selectnew:
  3493. case no_compound_childread:
  3494. case no_dataset_alias:
  3495. ds = ds->queryChild(0);
  3496. break;
  3497. case no_select:
  3498. if (ds->isDataset())
  3499. {
  3500. if (ds->hasProperty(newAtom))
  3501. {
  3502. ds = ds->queryChild(0);
  3503. //don't walk complexds[1].childDataset<new> since the [1] will be generated as a temporary
  3504. // if (!ds->isDataset() && (ds->getOperator() != no_select))
  3505. // ds = NULL;
  3506. }
  3507. else
  3508. ds = NULL;
  3509. }
  3510. else
  3511. {
  3512. //ds.x.y, always walk to to ds.x
  3513. ds = ds->queryChild(0);
  3514. }
  3515. break;
  3516. case no_rows:
  3517. case no_call:
  3518. case no_externalcall:
  3519. case no_getresult:
  3520. case no_getgraphresult:
  3521. case no_alias:
  3522. ds = NULL;
  3523. break;
  3524. default:
  3525. needToPreserve = true;
  3526. break;
  3527. }
  3528. }
  3529. OwnedHqlExpr source = getPointer(row->queryBound());
  3530. if (!needToPreserve || initctx.hasAssociation(*row, false))
  3531. return source.getClear();
  3532. //If link counted, then declare a member that is a link counted row to ensure this row remains linked.
  3533. ITypeInfo * rowType = rowExpr->queryType();
  3534. if (hasLinkCountedModifier(row->queryBound()))
  3535. {
  3536. CHqlBoundTarget saveTarget;
  3537. createTempFor(initctx, rowType, saveTarget, typemod_wrapper, FormatLinkedDataset);
  3538. StringBuffer s;
  3539. generateExprCpp(s, saveTarget.expr);
  3540. s.append(".set(");
  3541. generateExprCpp(s, source).append(");");
  3542. iterctx.addQuoted(s);
  3543. return getPointer(saveTarget.expr);
  3544. }
  3545. BuildCtx childctx(iterctx);
  3546. childctx.addGroup();
  3547. Owned<BoundRow> tempRow = declareTempRow(childctx, childctx, rowExpr);
  3548. Owned<BoundRow> rowBuilder = createRowBuilder(childctx, tempRow);
  3549. OwnedHqlExpr sourceSelector = ensureActiveRow(row->querySelector());
  3550. buildAssign(childctx, rowBuilder->querySelector(), sourceSelector);
  3551. finalizeTempRow(childctx, tempRow, rowBuilder);
  3552. return getPointer(tempRow->queryBound());
  3553. }
  3554. IReferenceSelector * HqlCppTranslator::buildDatasetIndexViaIterator(BuildCtx & ctx, IHqlExpression * expr)
  3555. {
  3556. OwnedHqlExpr dataset = normalizeAnyDatasetAliases(querySkipDatasetMeta(expr->queryChild(0)));
  3557. IHqlExpression * index = expr->queryChild(1);
  3558. IHqlExpression * childDataset = dataset;
  3559. switch (dataset->getOperator())
  3560. {
  3561. case no_hqlproject:
  3562. //optimize selectnth(project(rows, t), n) to projectrow(selectnth(rows, n), t)
  3563. IHqlExpression * transform = dataset->queryChild(1);
  3564. if (!containsSkip(transform) && !expr->hasProperty(_countProject_Atom))
  3565. childDataset = dataset->queryChild(0);
  3566. break;
  3567. }
  3568. //create a temporary
  3569. //Following works because rows are created as temporaries in the class, so still in scope outside the iterate loop.
  3570. //Not a strictly correct assumption - e.g., if ever done in the main process() code.
  3571. Owned<ITypeInfo> rowType = makeReferenceModifier(expr->getType());
  3572. OwnedHqlExpr rowExpr = ctx.getTempDeclare(rowType, NULL);
  3573. Owned<BoundRow> row = createBoundRow(expr, rowExpr);
  3574. CHqlBoundExpr boundCleared;
  3575. #ifdef CREATE_DEAULT_ROW_IF_NULL
  3576. buildDefaultRow(ctx, expr, boundCleared);
  3577. #else
  3578. buildNullRow(ctx, expr, boundCleared);
  3579. row->setConditional(true);
  3580. #endif
  3581. OwnedHqlExpr defaultRowPtr = getPointer(boundCleared.expr);
  3582. //Declare row for final level, iterate the appropriate number of times, and then assign and break.
  3583. BuildCtx initctx(ctx);
  3584. initctx.addGroup(); // add a group to allow a temporary to be declared later.
  3585. initctx.addAssign(rowExpr, defaultRowPtr);
  3586. HqlExprAssociation * savedMarker = ctx.associateExpr(queryConditionalRowMarker(), rowExpr);
  3587. BuildCtx iterctx(ctx);
  3588. bool done = false;
  3589. if (childDataset->getOperator() == no_rows) //hasOutOfLineModifier(dataset->queryType()))
  3590. {
  3591. CHqlBoundExpr boundDs;
  3592. buildDataset(ctx, childDataset, boundDs, FormatNatural);
  3593. if (boundDs.count && isArrayRowset(boundDs.expr->queryType()))
  3594. {
  3595. OwnedHqlExpr castIndex = ensureExprType(index, unsignedType);
  3596. OwnedHqlExpr adjustedIndex = adjustValue(castIndex, -1);
  3597. CHqlBoundExpr boundIndex;
  3598. buildExpr(ctx, adjustedIndex, boundIndex);
  3599. OwnedHqlExpr count = getBoundCount(boundDs); // could be serialized, so can't assume bound.count is set
  3600. OwnedHqlExpr test = createValue(no_gt, makeBoolType(), LINK(count), LINK(boundIndex.expr));
  3601. iterctx.addFilter(test);
  3602. if (dataset != childDataset)
  3603. {
  3604. Owned<ITypeInfo> datasetRowType = makeRowReferenceType(boundDs);
  3605. OwnedHqlExpr selectedRow = createValue(no_index, LINK(datasetRowType), LINK(boundDs.expr), LINK(boundIndex.expr));
  3606. OwnedHqlExpr projected = createRow(no_projectrow, createTranslated(selectedRow), createComma(LINK(dataset->queryChild(1)), LINK(querySelSeq(dataset))));
  3607. Owned<IReferenceSelector> newRow = buildNewRow(iterctx, projected);
  3608. OwnedHqlExpr newPtr = getPointer(newRow->queryRootRow()->queryBound());
  3609. iterctx.addAssign(rowExpr, newPtr);
  3610. }
  3611. else
  3612. {
  3613. OwnedHqlExpr selectedRow = createValue(no_index, LINK(rowType), LINK(boundDs.expr), LINK(boundIndex.expr));
  3614. iterctx.addAssign(rowExpr, selectedRow);
  3615. }
  3616. done = true;
  3617. }
  3618. }
  3619. if (!done)
  3620. {
  3621. BoundRow * chooseCursor;
  3622. //If choosing the first element, then no need to maintain a counter...
  3623. IValue * indexValue = index->queryValue();
  3624. if (indexValue && (indexValue->getIntValue() == 1))
  3625. chooseCursor = buildDatasetIterate(iterctx, dataset, true);
  3626. else
  3627. chooseCursor = buildDatasetIterate(iterctx, expr, true);
  3628. if (chooseCursor)
  3629. {
  3630. OwnedHqlExpr source = getPointer(chooseCursor->queryBound());
  3631. //MORE: Need casts because cursor may be (probably are) constant, but temporary isn't
  3632. //should somehow fnd out by looking at the cursors.
  3633. OwnedHqlExpr castLeftRow = createValue(no_implicitcast, LINK(rowType), LINK(source));
  3634. iterctx.addAssign(rowExpr, castLeftRow);
  3635. iterctx.addBreak();
  3636. }
  3637. }
  3638. ctx.removeAssociation(savedMarker);
  3639. ctx.associate(*row);
  3640. return createReferenceSelector(row);
  3641. }
  3642. IReferenceSelector * HqlCppTranslator::buildDatasetIndex(BuildCtx & ctx, IHqlExpression * expr)
  3643. {
  3644. HqlExprAssociation * match = ctx.queryAssociation(expr, AssocRow, NULL);
  3645. if (match)
  3646. return createReferenceSelector(static_cast<BoundRow *>(match));
  3647. #if 0
  3648. //Causes some queries (ncf10) to run out of memory, so disable for the moment.
  3649. OwnedHqlExpr optimized = optimizeHqlExpression(expr, getOptimizeFlags()|HOOcompoundproject);
  3650. if (optimized != expr)
  3651. return buildNewRow(ctx, optimized);
  3652. #endif
  3653. OwnedHqlExpr dataset = normalizeAnyDatasetAliases(expr->queryChild(0));
  3654. //Special cases:
  3655. //i) selecting row [1] from something that only has a single row
  3656. //ii) selecting row [n] from something that can be iterated.
  3657. //iii) row[1] from something sorted that can be iterated.
  3658. BoundRow * row = NULL;
  3659. if (isTrivialSelectN(expr))
  3660. {
  3661. BoundRow * row = NULL;
  3662. #if 0
  3663. //This could be a good idea - but again it can mess up cse since dataset never gets bound.
  3664. //Could enable if I implement cse on datasets within transforms.
  3665. // if (canIterateInline(&ctx, dataset))
  3666. // row = buildOptimizeSelectFirstRow(ctx, dataset);
  3667. #endif
  3668. if (!row)
  3669. {
  3670. CHqlBoundExpr bound;
  3671. buildDataset(ctx, dataset, bound, FormatNatural);
  3672. convertBoundDatasetToFirstRow(expr, bound);
  3673. row = bindRow(ctx, expr, bound.expr);
  3674. }
  3675. return createReferenceSelector(row);
  3676. }
  3677. else if (canIterateInline(&ctx, dataset))
  3678. {
  3679. //MORE? Following doesn't work for implicit normalize which iterates multiple levels
  3680. bool specialCase = false;
  3681. dataset.set(querySkipDatasetMeta(dataset));
  3682. switch (dataset->getOperator())
  3683. {
  3684. case no_select:
  3685. specialCase = !isMultiLevelDatasetSelector(expr, false);
  3686. break;
  3687. case no_if:
  3688. case no_inlinedictionary:
  3689. case no_inlinetable:
  3690. case no_join:
  3691. //Always creates a temporary, so don't use an iterator
  3692. specialCase = true;
  3693. break;
  3694. default:
  3695. specialCase = alwaysEvaluatesToBound(dataset);
  3696. break;
  3697. }
  3698. if (!specialCase)
  3699. return buildDatasetIndexViaIterator(ctx, expr);
  3700. }
  3701. else if (isSelectSortedTop(expr) && canIterateInline(&ctx, dataset->queryChild(0)))
  3702. {
  3703. return doBuildRowSelectTop(ctx, expr);
  3704. }
  3705. //MORE: Is this a good idea???
  3706. else if (!canProcessInline(&ctx, expr))
  3707. {
  3708. CHqlBoundExpr bound;
  3709. OwnedHqlExpr dsExpr = expr->isDatarow() ? createDatasetFromRow(LINK(expr)) : LINK(expr);
  3710. buildDataset(ctx, dsExpr, bound, FormatNatural);
  3711. convertBoundDatasetToFirstRow(expr, bound);
  3712. row = bindRow(ctx, expr, bound.expr);
  3713. }
  3714. if (!row)
  3715. {
  3716. Owned<IHqlCppDatasetCursor> cursor = createDatasetSelector(ctx, dataset);
  3717. row = cursor->buildSelect(ctx, expr);
  3718. if (!row)
  3719. {
  3720. CHqlBoundExpr boundCleared;
  3721. buildDefaultRow(ctx, dataset, boundCleared);
  3722. OwnedHqlExpr defaultRowPtr = getPointer(boundCleared.expr);
  3723. row = bindRow(ctx, expr, defaultRowPtr);
  3724. }
  3725. }
  3726. return createReferenceSelector(row);
  3727. }
  3728. IReferenceSelector * HqlCppTranslator::buildDatasetSelectMap(BuildCtx & ctx, IHqlExpression * expr)
  3729. {
  3730. HqlExprAssociation * match = ctx.queryAssociation(expr, AssocRow, NULL);
  3731. if (match)
  3732. return createReferenceSelector(static_cast<BoundRow *>(match));
  3733. OwnedHqlExpr dataset = normalizeAnyDatasetAliases(expr->queryChild(0));
  3734. // Create a row to pass to the lookup function
  3735. BoundRow * row = NULL;
  3736. if (!canProcessInline(&ctx, expr))
  3737. {
  3738. CHqlBoundExpr bound;
  3739. OwnedHqlExpr dsExpr = expr->isDatarow() ? createDatasetFromRow(LINK(expr)) : LINK(expr);
  3740. buildDataset(ctx, dsExpr, bound, FormatNatural);
  3741. convertBoundDatasetToFirstRow(expr, bound);
  3742. row = bindRow(ctx, expr, bound.expr);
  3743. }
  3744. if (!row)
  3745. {
  3746. Owned<IHqlCppDatasetCursor> cursor = createDatasetSelector(ctx, dataset);
  3747. row = cursor->buildSelect(ctx, expr);
  3748. if (!row)
  3749. {
  3750. CHqlBoundExpr boundCleared;
  3751. buildDefaultRow(ctx, dataset, boundCleared);
  3752. OwnedHqlExpr defaultRowPtr = getPointer(boundCleared.expr);
  3753. row = bindRow(ctx, expr, defaultRowPtr);
  3754. }
  3755. }
  3756. return createReferenceSelector(row);
  3757. }
  3758. //---------------------------------------------------------------------------
  3759. IHqlExpression * HqlCppTranslator::buildGetLocalResult(BuildCtx & ctx, IHqlExpression * expr, bool preferLinkedRows)
  3760. {
  3761. IHqlExpression * graphId = expr->queryChild(1);
  3762. IHqlExpression * resultNum = expr->queryChild(2);
  3763. Linked<ITypeInfo> exprType = queryUnqualifiedType(expr->queryType());
  3764. if (preferLinkedRows && !hasLinkCountedModifier(exprType))
  3765. exprType.setown(makeAttributeModifier(LINK(exprType), getLinkCountedAttr()));
  3766. if (expr->hasProperty(externalAtom))
  3767. {
  3768. IHqlExpression * resultInstance = queryPropertyChild(expr, externalAtom, 0);
  3769. HqlExprAssociation * matchedResults = ctx.queryMatchExpr(resultInstance);
  3770. if (!matchedResults)
  3771. {
  3772. //Very unusual - a result is required from a child query, but that child query is actually in
  3773. //the parent/grandparent. We need to evaluate in the parent instead.
  3774. CHqlBoundExpr match;
  3775. if (!buildExprInCorrectContext(ctx, expr, match, false))
  3776. throwUnexpected();
  3777. return match.getTranslatedExpr();
  3778. }
  3779. HqlExprArray args;
  3780. args.append(*LINK(matchedResults->queryExpr()));
  3781. args.append(*LINK(resultNum));
  3782. if (preferLinkedRows)
  3783. return bindFunctionCall(getChildQueryLinkedResultAtom, args, exprType);
  3784. return bindFunctionCall(getChildQueryResultAtom, args, exprType);
  3785. }
  3786. assertex(activeActivities.ordinality());
  3787. queryAddResultDependancy(activeActivities.tos(), graphId, resultNum);
  3788. SubGraphInfo * activeSubgraph = queryActiveSubGraph(ctx);
  3789. assertex(activeSubgraph && graphId == activeSubgraph->graphTag);
  3790. unique_id_t id = activeSubgraph->graphId;
  3791. EvalContext * instance = queryEvalContext(ctx);
  3792. OwnedHqlExpr retInstanceExpr;
  3793. if (instance && !insideOnCreate(ctx))
  3794. retInstanceExpr.setown(instance->createGraphLookup(id, false));
  3795. else
  3796. retInstanceExpr.setown(doCreateGraphLookup(ctx, ctx, id, "this", true));
  3797. HqlExprArray args;
  3798. args.append(*LINK(retInstanceExpr));
  3799. args.append(*LINK(resultNum));
  3800. if (preferLinkedRows)
  3801. return bindFunctionCall(getLocalLinkedResultAtom, args, exprType);
  3802. return bindFunctionCall(getLocalResultAtom, args, exprType);
  3803. }
  3804. void HqlCppTranslator::doBuildAssignGetGraphResult(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  3805. {
  3806. if (expr->hasProperty(_streaming_Atom))
  3807. throwError(HQLERR_LoopTooComplexForParallel);
  3808. bool isTargetLinkCounted = hasLinkCountedModifier(target.queryType());
  3809. if (expr->hasProperty(externalAtom))
  3810. {
  3811. OwnedHqlExpr call = buildGetLocalResult(ctx, expr, isTargetLinkCounted);
  3812. buildExprAssign(ctx, target, call);
  3813. return;
  3814. }
  3815. if (!isCurrentActiveGraph(ctx, expr->queryChild(1)))
  3816. {
  3817. CHqlBoundExpr match;
  3818. if (!buildExprInCorrectContext(ctx, expr, match, false))
  3819. throwError(HQLERR_GraphContextNotFound);
  3820. assign(ctx, target, match);
  3821. return;
  3822. }
  3823. OwnedHqlExpr call = buildGetLocalResult(ctx, expr, isTargetLinkCounted);
  3824. buildExprAssign(ctx, target, call);
  3825. }
  3826. void HqlCppTranslator::doBuildExprGetGraphResult(BuildCtx & ctx, IHqlExpression * expr, CHqlBoundExpr & tgt, ExpressionFormat format)
  3827. {
  3828. if (!expr->hasProperty(externalAtom))
  3829. {
  3830. doBuildAliasValue(ctx, expr, tgt);
  3831. return;
  3832. if (!isCurrentActiveGraph(ctx, expr->queryChild(1)))
  3833. {
  3834. if (!buildExprInCorrectContext(ctx, expr, tgt, false))
  3835. throwError(HQLERR_GraphContextNotFound);
  3836. return;
  3837. }
  3838. }
  3839. bool useLinkCounted = recordRequiresSerialization(expr->queryRecord()) || options.tempDatasetsUseLinkedRows;
  3840. OwnedHqlExpr call = buildGetLocalResult(ctx, expr, useLinkCounted);
  3841. switch (expr->queryType()->getTypeCode())
  3842. {
  3843. case type_row:
  3844. throwUnexpected();
  3845. case type_dictionary:
  3846. case type_table:
  3847. case type_groupedtable:
  3848. buildTempExpr(ctx, call, tgt);
  3849. break;
  3850. default:
  3851. buildExpr(ctx, call, tgt);
  3852. break;
  3853. }
  3854. }
  3855. ABoundActivity * HqlCppTranslator::doBuildActivityGetGraphResult(BuildCtx & ctx, IHqlExpression * expr)
  3856. {
  3857. IHqlExpression * graphId = expr->queryChild(1);
  3858. IHqlExpression * resultNum = expr->queryChild(2);
  3859. ThorActivityKind activityKind = (expr->hasProperty(_streaming_Atom) ? TAKlocalstreamread : TAKlocalresultread);
  3860. bool useImplementationClass = options.minimizeActivityClasses && (resultNum->getOperator() == no_constant);
  3861. Owned<ActivityInstance> instance = new ActivityInstance(*this, ctx, activityKind, expr, "LocalResultRead");
  3862. if (useImplementationClass)
  3863. instance->setImplementationClass(newLocalResultReadArgAtom);
  3864. if (expr->hasProperty(_loop_Atom))
  3865. {
  3866. if (isCurrentActiveGraph(ctx, graphId))
  3867. instance->graphLabel.set("Begin Loop");
  3868. else
  3869. instance->graphLabel.set("Outer Loop Input");
  3870. }
  3871. buildActivityFramework(instance);
  3872. buildInstancePrefix(instance);
  3873. if (!useImplementationClass)
  3874. doBuildUnsignedFunction(instance->classctx, "querySequence", resultNum);
  3875. else
  3876. instance->addConstructorParameter(resultNum);
  3877. addGraphIdAttribute(instance, ctx, graphId);
  3878. buildInstanceSuffix(instance);
  3879. queryAddResultDependancy(*instance->queryBoundActivity(), graphId, resultNum);
  3880. return instance->getBoundActivity();
  3881. }
  3882. ABoundActivity * HqlCppTranslator::doBuildActivitySetGraphResult(BuildCtx & ctx, IHqlExpression * expr, bool isRoot)
  3883. {
  3884. IHqlExpression * dataset = expr->queryChild(0);
  3885. IHqlExpression * graphId = expr->queryChild(1);
  3886. IHqlExpression * resultNum = expr->queryChild(2);
  3887. bool isSpill = expr->hasProperty(_spill_Atom);
  3888. ABoundActivity * parentActivity = activeActivities.ordinality() ? &activeActivities.tos() : NULL;
  3889. Owned<ABoundActivity> boundDataset = buildCachedActivity(ctx, dataset);
  3890. bool useImplementationClass = options.minimizeActivityClasses;
  3891. Owned<ActivityInstance> instance;
  3892. if (expr->getOperator() == no_spillgraphresult)
  3893. {
  3894. instance.setown(new ActivityInstance(*this, ctx, TAKlocalresultspill, expr, "LocalResultSpill"));
  3895. }
  3896. else
  3897. {
  3898. instance.setown(new ActivityInstance(*this, ctx, TAKlocalresultwrite, expr, "LocalResultWrite"));
  3899. }
  3900. if (useImplementationClass)
  3901. instance->setImplementationClass(newLocalResultSpillArgAtom);
  3902. if (expr->hasProperty(_loop_Atom))
  3903. instance->graphLabel.set("End Loop");
  3904. buildActivityFramework(instance, isRoot && !isSpill);
  3905. buildInstancePrefix(instance);
  3906. if (!useImplementationClass)
  3907. {
  3908. doBuildUnsignedFunction(instance->classctx, "querySequence", resultNum);
  3909. doBuildBoolFunction(instance->classctx, "usedOutsideGraph", !isSpill);
  3910. }
  3911. else
  3912. {
  3913. instance->addConstructorParameter(resultNum);
  3914. instance->addConstructorParameter(queryBoolExpr(!isSpill));
  3915. }
  3916. if (parentActivity && !insideRemoteGraph(ctx) && !isSpill)
  3917. {
  3918. const char * relationship;
  3919. if (expr->hasProperty(_loop_Atom))
  3920. relationship = "Body";
  3921. else if (insideRemoteGraph(ctx))
  3922. relationship = "Remote";
  3923. else
  3924. relationship = "Child";
  3925. addDependency(ctx, instance->queryBoundActivity(), parentActivity, childAtom, relationship);
  3926. }
  3927. instance->addAttributeBool("_isSpill", isSpill);
  3928. if (targetRoxie())
  3929. addGraphIdAttribute(instance, ctx, graphId);
  3930. buildInstanceSuffix(instance);
  3931. buildConnectInputOutput(ctx, instance, boundDataset, 0, 0);
  3932. associateRemoteResult(*instance, graphId, resultNum);
  3933. return instance->getBoundActivity();
  3934. }
  3935. ABoundActivity * HqlCppTranslator::doBuildActivityReturnResult(BuildCtx & ctx, IHqlExpression * expr, bool isRoot)
  3936. {
  3937. IHqlExpression * dataset = expr->queryChild(0);
  3938. ABoundActivity * parentActivity = activeActivities.ordinality() ? &activeActivities.tos() : NULL;
  3939. Owned<ABoundActivity> boundDataset = buildCachedActivity(ctx, dataset);
  3940. ThorActivityKind kind;
  3941. const char * helper;
  3942. if (dataset->isDataset())
  3943. {
  3944. kind = TAKdatasetresult;
  3945. helper = "DatasetResult";
  3946. }
  3947. else
  3948. {
  3949. kind = TAKrowresult;
  3950. helper = "RowResult";
  3951. }
  3952. Owned<ActivityInstance> instance = new ActivityInstance(*this, ctx, kind, expr, helper);
  3953. buildActivityFramework(instance, isRoot);
  3954. buildInstancePrefix(instance);
  3955. if (parentActivity && !insideRemoteGraph(ctx))
  3956. addDependency(ctx, instance->queryBoundActivity(), parentActivity, childAtom, "Child");
  3957. buildInstanceSuffix(instance);
  3958. buildConnectInputOutput(ctx, instance, boundDataset, 0, 0);
  3959. return instance->getBoundActivity();
  3960. }
  3961. void HqlCppTranslator::doBuildAssignLoopCounter(BuildCtx & ctx, const CHqlBoundTarget & target, IHqlExpression * expr)
  3962. {
  3963. if (!isCurrentActiveGraph(ctx, expr->queryChild(0)))
  3964. {
  3965. CHqlBoundExpr match;
  3966. if (!buildExprInCorrectContext(ctx, expr, match, false))
  3967. throwError(HQLERR_GraphContextNotFound);
  3968. assign(ctx, target, match);
  3969. return;
  3970. }
  3971. HqlExprArray args;
  3972. OwnedHqlExpr call = bindFunctionCall(getGraphLoopCounterAtom, args);
  3973. buildExprAssign(ctx, target, call);
  3974. }
  3975. //---------------------------------------------------------------------------
  3976. ABoundActivity * HqlCppTranslator::doBuildActivityGetGraphLoopResult(BuildCtx & ctx, IHqlExpression * expr)
  3977. {
  3978. IHqlExpression * graphId = expr->queryChild(1);
  3979. IHqlExpression * resultNum = expr->queryChild(2);
  3980. Owned<ActivityInstance> instance = new ActivityInstance(*this, ctx, TAKgraphloopresultread, expr, "GraphLoopResultRead");
  3981. buildActivityFramework(instance);
  3982. buildInstancePrefix(instance);
  3983. doBuildUnsignedFunction(instance->startctx, "querySequence", resultNum);
  3984. addGraphIdAttribute(instance, ctx, graphId);
  3985. buildInstanceSuffix(instance);
  3986. return instance->getBoundActivity();
  3987. }
  3988. ABoundActivity * HqlCppTranslator::doBuildActivitySetGraphLoopResult(BuildCtx & ctx, IHqlExpression * expr)
  3989. {
  3990. IHqlExpression * dataset = expr->queryChild(0);
  3991. IHqlExpression * graphId = expr->queryChild(1);
  3992. bool isSpill = expr->hasProperty(_spill_Atom);
  3993. ABoundActivity * parentActivity = activeActivities.ordinality() ? &activeActivities.tos() : NULL;
  3994. Owned<ABoundActivity> boundDataset = buildCachedActivity(ctx, dataset);
  3995. bool useImplementationClass = options.minimizeActivityClasses;
  3996. Owned<ActivityInstance> instance = new ActivityInstance(*this, ctx, TAKgraphloopresultwrite, expr, "GraphLoopResultWrite");
  3997. if (useImplementationClass)
  3998. instance->setImplementationClass(newGraphLoopResultWriteArgAtom);
  3999. buildActivityFramework(instance, true);
  4000. buildInstancePrefix(instance);
  4001. if (parentActivity && !insideRemoteGraph(ctx) && !isSpill)
  4002. addDependency(ctx, instance->queryBoundActivity(), parentActivity, childAtom, "Body");
  4003. if (targetRoxie())
  4004. addGraphIdAttribute(instance, ctx, graphId);
  4005. buildInstanceSuffix(instance);
  4006. buildConnectInputOutput(ctx, instance, boundDataset, 0, 0);
  4007. return instance->getBoundActivity();
  4008. }
  4009. //---------------------------------------------------------------------------
  4010. static IHqlExpression * queryResultExpr(IHqlExpression * expr)
  4011. {
  4012. loop
  4013. {
  4014. switch (expr->getOperator())
  4015. {
  4016. case no_compound:
  4017. expr = expr->queryChild(1);
  4018. break;
  4019. case no_subgraph:
  4020. expr = expr->queryChild(0);
  4021. break;
  4022. case no_returnresult:
  4023. return expr;
  4024. default:
  4025. throwUnexpectedOp(expr->getOperator());
  4026. }
  4027. }
  4028. }
  4029. ABoundActivity * HqlCppTranslator::doBuildActivityForceLocal(BuildCtx & ctx, IHqlExpression * expr)
  4030. {
  4031. IHqlExpression * child = expr->queryChild(0);
  4032. if (targetHThor() || (targetThor() && !insideChildQuery(ctx)))
  4033. {
  4034. WARNING(HQLWRN_LocalHasNoEffect);
  4035. return buildCachedActivity(ctx, child);
  4036. }
  4037. OwnedHqlExpr result = createValue(no_returnresult, makeVoidType(), LINK(child));
  4038. OwnedHqlExpr remote = resourceThorGraph(*this, result, RoxieCluster, 1, NULL);
  4039. unique_id_t localId = doBuildThorSubGraph(ctx, remote, SubGraphRemote);
  4040. Owned<ActivityInstance> instance = new ActivityInstance(*this, ctx, TAKlocalgraph, expr, "Null");
  4041. buildActivityFramework(instance);
  4042. buildInstancePrefix(instance);
  4043. instance->addAttributeInt("_subgraph", localId);
  4044. ActivityAssociation * match = static_cast<ActivityAssociation *>(ctx.queryAssociation(queryResultExpr(remote), AssocActivity, NULL));
  4045. assertex(match);
  4046. addDependency(ctx, match->activity, instance->queryBoundActivity(), childAtom);
  4047. buildInstanceSuffix(instance);
  4048. return instance->getBoundActivity();
  4049. }
  4050. void HqlCppTranslator::doBuildStmtApply(BuildCtx & ctx, IHqlExpression * expr)
  4051. {
  4052. IHqlExpression * dataset = expr->queryChild(0);
  4053. IHqlExpression * start = expr->queryProperty(beforeAtom);
  4054. IHqlExpression * end = expr->queryProperty(afterAtom);
  4055. if (start)
  4056. buildStmt(ctx, start->queryChild(0));
  4057. BuildCtx condctx(ctx);
  4058. BoundRow * cursor = buildDatasetIterate(condctx, dataset, false);
  4059. unsigned max = expr->numChildren();
  4060. for (unsigned i=1; i < max; i++)
  4061. {
  4062. IHqlExpression * cur = expr->queryChild(i);
  4063. if (!cur->isAttribute())
  4064. buildStmt(condctx, cur);
  4065. }
  4066. if (end)
  4067. buildStmt(ctx, end->queryChild(0));
  4068. }