thgraph.cpp 106 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282
  1. /*##############################################################################
  2. HPCC SYSTEMS software Copyright (C) 2012 HPCC Systems®.
  3. Licensed under the Apache License, Version 2.0 (the "License");
  4. you may not use this file except in compliance with the License.
  5. You may obtain a copy of the License at
  6. http://www.apache.org/licenses/LICENSE-2.0
  7. Unless required by applicable law or agreed to in writing, software
  8. distributed under the License is distributed on an "AS IS" BASIS,
  9. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. See the License for the specific language governing permissions and
  11. limitations under the License.
  12. ############################################################################## */
  13. #include "thgraph.hpp"
  14. #include "jptree.hpp"
  15. #include "commonext.hpp"
  16. #include "dasess.hpp"
  17. #include "jhtree.hpp"
  18. #include "thcodectx.hpp"
  19. #include "thbuf.hpp"
  20. #include "thormisc.hpp"
  21. #include "thbufdef.hpp"
  22. #include "thmem.hpp"
  23. #include "rtlformat.hpp"
  24. #include "thorsoapcall.hpp"
  25. PointerArray createFuncs;
  26. void registerCreateFunc(CreateFunc func)
  27. {
  28. createFuncs.append((void *)func);
  29. }
  30. ///////////////////////////////////
  31. //////
  32. /////
  33. class CThorGraphResult : implements IThorResult, implements IRowWriter, public CInterface
  34. {
  35. CActivityBase &activity;
  36. rowcount_t rowStreamCount;
  37. IOutputMetaData *meta;
  38. Owned<IRowWriterMultiReader> rowBuffer;
  39. IThorRowInterfaces *rowIf;
  40. IEngineRowAllocator *allocator;
  41. bool stopped, readers;
  42. ThorGraphResultType resultType;
  43. void init()
  44. {
  45. stopped = readers = false;
  46. allocator = rowIf->queryRowAllocator();
  47. meta = allocator->queryOutputMeta();
  48. rowStreamCount = 0;
  49. }
  50. class CStreamWriter : implements IRowWriterMultiReader, public CSimpleInterface
  51. {
  52. CThorGraphResult &owner;
  53. CThorExpandingRowArray rows;
  54. public:
  55. IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
  56. CStreamWriter(CThorGraphResult &_owner, EmptyRowSemantics emptyRowSemantics) : owner(_owner), rows(owner.activity, owner.rowIf, emptyRowSemantics)
  57. {
  58. }
  59. //IRowWriterMultiReader
  60. virtual void putRow(const void *row)
  61. {
  62. rows.append(row);
  63. }
  64. virtual void flush() { }
  65. virtual IRowStream *getReader()
  66. {
  67. return rows.createRowStream(0, (rowidx_t)-1, false);
  68. }
  69. };
  70. public:
  71. IMPLEMENT_IINTERFACE;
  72. CThorGraphResult(CActivityBase &_activity, IThorRowInterfaces *_rowIf, ThorGraphResultType _resultType, unsigned spillPriority) : activity(_activity), rowIf(_rowIf), resultType(_resultType)
  73. {
  74. init();
  75. EmptyRowSemantics emptyRowSemantics;
  76. if (isGrouped())
  77. emptyRowSemantics = ers_eogonly;
  78. else if (isSparse())
  79. emptyRowSemantics = ers_allow;
  80. else
  81. emptyRowSemantics = ers_forbidden;
  82. if (SPILL_PRIORITY_DISABLE == spillPriority)
  83. rowBuffer.setown(new CStreamWriter(*this, emptyRowSemantics));
  84. else
  85. rowBuffer.setown(createOverflowableBuffer(activity, rowIf, emptyRowSemantics, true));
  86. }
  87. // IRowWriter
  88. virtual void putRow(const void *row)
  89. {
  90. assertex(!readers);
  91. ++rowStreamCount;
  92. rowBuffer->putRow(row);
  93. }
  94. virtual void flush() { }
  95. virtual offset_t getPosition() { UNIMPLEMENTED; return 0; }
  96. // IThorResult
  97. virtual IRowWriter *getWriter()
  98. {
  99. return LINK(this);
  100. }
  101. virtual void setResultStream(IRowWriterMultiReader *stream, rowcount_t count)
  102. {
  103. assertex(!readers);
  104. rowBuffer.setown(stream);
  105. rowStreamCount = count;
  106. }
  107. virtual IRowStream *getRowStream()
  108. {
  109. readers = true;
  110. return rowBuffer->getReader();
  111. }
  112. virtual IThorRowInterfaces *queryRowInterfaces() { return rowIf; }
  113. virtual CActivityBase *queryActivity() { return &activity; }
  114. virtual bool isDistributed() const { return resultType & thorgraphresult_distributed; }
  115. virtual bool isSparse() const { return resultType & thorgraphresult_sparse; }
  116. virtual bool isGrouped() const { return resultType & thorgraphresult_grouped; }
  117. virtual void serialize(MemoryBuffer &mb)
  118. {
  119. Owned<IRowStream> stream = getRowStream();
  120. bool grouped = meta->isGrouped();
  121. if (grouped)
  122. {
  123. OwnedConstThorRow prev = stream->nextRow();
  124. if (prev)
  125. {
  126. bool eog;
  127. for (;;)
  128. {
  129. eog = false;
  130. OwnedConstThorRow next = stream->nextRow();
  131. if (!next)
  132. eog = true;
  133. size32_t sz = meta->getRecordSize(prev);
  134. mb.append(sz, prev.get());
  135. mb.append(eog);
  136. if (!next)
  137. {
  138. next.setown(stream->nextRow());
  139. if (!next)
  140. break;
  141. }
  142. prev.set(next);
  143. }
  144. }
  145. }
  146. else
  147. {
  148. for (;;)
  149. {
  150. OwnedConstThorRow row = stream->nextRow();
  151. if (!row)
  152. break;
  153. size32_t sz = meta->getRecordSize(row);
  154. mb.append(sz, row.get());
  155. }
  156. }
  157. }
  158. virtual void getResult(size32_t &len, void * & data)
  159. {
  160. MemoryBuffer mb;
  161. serialize(mb);
  162. len = mb.length();
  163. data = mb.detach();
  164. }
  165. virtual void getLinkedResult(unsigned &countResult, const byte * * & result) override
  166. {
  167. assertex(rowStreamCount==((unsigned)rowStreamCount)); // catch, just in case
  168. Owned<IRowStream> stream = getRowStream();
  169. countResult = 0;
  170. OwnedConstThorRow _rowset = allocator->createRowset((unsigned)rowStreamCount);
  171. const void **rowset = (const void **)_rowset.get();
  172. while (countResult < rowStreamCount)
  173. {
  174. OwnedConstThorRow row = stream->nextRow();
  175. rowset[countResult++] = row.getClear();
  176. }
  177. result = (const byte **)_rowset.getClear();
  178. }
  179. virtual const void * getLinkedRowResult()
  180. {
  181. assertex(rowStreamCount==1); // catch, just in case
  182. Owned<IRowStream> stream = getRowStream();
  183. return stream->nextRow();
  184. }
  185. };
  186. /////
  187. IThorResult *CThorGraphResults::createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, ThorGraphResultType resultType, unsigned spillPriority)
  188. {
  189. Owned<IThorResult> result = ::createResult(activity, rowIf, resultType, spillPriority);
  190. setResult(id, result);
  191. return result;
  192. }
  193. /////
  194. IThorResult *createResult(CActivityBase &activity, IThorRowInterfaces *rowIf, ThorGraphResultType resultType, unsigned spillPriority)
  195. {
  196. return new CThorGraphResult(activity, rowIf, resultType, spillPriority);
  197. }
  198. /////
  199. class CThorBoundLoopGraph : implements IThorBoundLoopGraph, public CInterface
  200. {
  201. CGraphBase *graph;
  202. activity_id activityId;
  203. Linked<IOutputMetaData> resultMeta;
  204. Owned<IOutputMetaData> counterMeta, loopAgainMeta;
  205. Owned<IThorRowInterfaces> resultRowIf, countRowIf, loopAgainRowIf;
  206. public:
  207. IMPLEMENT_IINTERFACE;
  208. CThorBoundLoopGraph(CGraphBase *_graph, IOutputMetaData * _resultMeta, unsigned _activityId) : graph(_graph), resultMeta(_resultMeta), activityId(_activityId)
  209. {
  210. counterMeta.setown(createFixedSizeMetaData(sizeof(thor_loop_counter_t)));
  211. loopAgainMeta.setown(createFixedSizeMetaData(sizeof(bool)));
  212. }
  213. virtual void prepareCounterResult(CActivityBase &activity, IThorGraphResults *results, unsigned loopCounter, unsigned pos)
  214. {
  215. if (!countRowIf)
  216. countRowIf.setown(activity.createRowInterfaces(counterMeta));
  217. RtlDynamicRowBuilder counterRow(countRowIf->queryRowAllocator());
  218. thor_loop_counter_t * res = (thor_loop_counter_t *)counterRow.ensureCapacity(sizeof(thor_loop_counter_t),NULL);
  219. *res = loopCounter;
  220. OwnedConstThorRow counterRowFinal = counterRow.finalizeRowClear(sizeof(thor_loop_counter_t));
  221. IThorResult *counterResult = results->createResult(activity, pos, countRowIf, thorgraphresult_nul, SPILL_PRIORITY_DISABLE);
  222. Owned<IRowWriter> counterResultWriter = counterResult->getWriter();
  223. counterResultWriter->putRow(counterRowFinal.getClear());
  224. graph->setLoopCounter(loopCounter);
  225. }
  226. virtual void prepareLoopAgainResult(CActivityBase &activity, IThorGraphResults *results, unsigned pos)
  227. {
  228. if (!loopAgainRowIf)
  229. loopAgainRowIf.setown(activity.createRowInterfaces(loopAgainMeta));
  230. activity.queryGraph().createResult(activity, pos, results, loopAgainRowIf, activity.queryGraph().isLocalChild() ? thorgraphresult_nul : thorgraphresult_distributed, SPILL_PRIORITY_DISABLE);
  231. }
  232. virtual void prepareLoopResults(CActivityBase &activity, IThorGraphResults *results)
  233. {
  234. if (!resultRowIf)
  235. resultRowIf.setown(activity.createRowInterfaces(resultMeta));
  236. ThorGraphResultType resultType = activity.queryGraph().isLocalChild() ? thorgraphresult_nul : thorgraphresult_distributed;
  237. IThorResult *loopResult = activity.queryGraph().createResult(activity, 0, results, resultRowIf, resultType); // loop output
  238. IThorResult *inputResult = activity.queryGraph().createResult(activity, 1, results, resultRowIf, resultType); // loop input
  239. }
  240. virtual CGraphBase *queryGraph() { return graph; }
  241. };
  242. IThorBoundLoopGraph *createBoundLoopGraph(CGraphBase *graph, IOutputMetaData *resultMeta, unsigned activityId)
  243. {
  244. return new CThorBoundLoopGraph(graph, resultMeta, activityId);
  245. }
  246. ///////////////////////////////////
  247. bool isDiskInput(ThorActivityKind kind)
  248. {
  249. switch (kind)
  250. {
  251. case TAKcsvread:
  252. case TAKxmlread:
  253. case TAKjsonread:
  254. case TAKdiskread:
  255. case TAKdisknormalize:
  256. case TAKdiskaggregate:
  257. case TAKdiskcount:
  258. case TAKdiskgroupaggregate:
  259. case TAKindexread:
  260. case TAKindexcount:
  261. case TAKindexnormalize:
  262. case TAKindexaggregate:
  263. case TAKindexgroupaggregate:
  264. case TAKindexgroupexists:
  265. case TAKindexgroupcount:
  266. case TAKspillread:
  267. return true;
  268. default:
  269. return false;
  270. }
  271. }
  272. void CIOConnection::connect(unsigned which, CActivityBase *destActivity)
  273. {
  274. destActivity->setInput(which, activity->queryActivity(), index);
  275. }
  276. ///////////////////////////////////
  277. CGraphElementBase *createGraphElement(IPropertyTree &node, CGraphBase &owner, CGraphBase *resultsGraph)
  278. {
  279. CGraphElementBase *container = NULL;
  280. ForEachItemIn(m, createFuncs)
  281. {
  282. CreateFunc func = (CreateFunc)createFuncs.item(m);
  283. container = func(node, owner, resultsGraph);
  284. if (container) break;
  285. }
  286. if (NULL == container)
  287. {
  288. ThorActivityKind tak = (ThorActivityKind)node.getPropInt("att[@name=\"_kind\"]/@value", TAKnone);
  289. throw MakeStringException(TE_UnsupportedActivityKind, "Unsupported activity kind: %s", activityKindStr(tak));
  290. }
  291. container->setResultsGraph(resultsGraph);
  292. return container;
  293. }
  294. CGraphElementBase::CGraphElementBase(CGraphBase &_owner, IPropertyTree &_xgmml) : owner(&_owner)
  295. {
  296. xgmml.setown(createPTreeFromIPT(&_xgmml));
  297. eclText.set(xgmml->queryProp("att[@name=\"ecl\"]/@value"));
  298. id = xgmml->getPropInt("@id", 0);
  299. kind = (ThorActivityKind)xgmml->getPropInt("att[@name=\"_kind\"]/@value", TAKnone);
  300. sink = isActivitySink(kind);
  301. bool coLocal = xgmml->getPropBool("att[@name=\"coLocal\"]/@value", false);
  302. isLocalData = xgmml->getPropBool("att[@name=\"local\"]/@value", false); // local execute + local data access only
  303. isLocal = isLocalData || coLocal; // local execute
  304. isGrouped = xgmml->getPropBool("att[@name=\"grouped\"]/@value", false);
  305. resultsGraph = NULL;
  306. ownerId = xgmml->getPropInt("att[@name=\"_parentActivity\"]/@value", 0);
  307. onCreateCalled = prepared = haveCreateCtx = nullAct = false;
  308. onlyUpdateIfChanged = xgmml->getPropBool("att[@name=\"_updateIfChanged\"]/@value", false);
  309. isCodeSigned = isActivityCodeSigned(_xgmml);
  310. StringBuffer helperName("fAc");
  311. xgmml->getProp("@id", helperName);
  312. helperFactory = (EclHelperFactory) queryJob().queryDllEntry().getEntry(helperName.str());
  313. if (!helperFactory)
  314. throw makeOsExceptionV(GetLastError(), "Failed to load helper factory method: %s (dll handle = %p)", helperName.str(), queryJob().queryDllEntry().getInstance());
  315. alreadyUpdated = false;
  316. whichBranch = (unsigned)-1;
  317. log = true;
  318. sentActInitData.setown(createThreadSafeBitSet());
  319. maxCores = queryXGMML().getPropInt("hint[@name=\"max_cores\"]/@value", 0);
  320. if (0 == maxCores)
  321. maxCores = queryJob().queryMaxDefaultActivityCores();
  322. baseHelper.setown(helperFactory());
  323. }
  324. CGraphElementBase::~CGraphElementBase()
  325. {
  326. activity.clear();
  327. baseHelper.clear(); // clear before dll is unloaded
  328. }
  329. CJobBase &CGraphElementBase::queryJob() const
  330. {
  331. return owner->queryJob();
  332. }
  333. CJobChannel &CGraphElementBase::queryJobChannel() const
  334. {
  335. return owner->queryJobChannel();
  336. }
  337. IGraphTempHandler *CGraphElementBase::queryTempHandler(bool assert) const
  338. {
  339. if (resultsGraph)
  340. return resultsGraph->queryTempHandler(assert);
  341. else
  342. return queryJob().queryTempHandler(assert);
  343. }
  344. void CGraphElementBase::releaseIOs()
  345. {
  346. loopGraph.clear();
  347. if (activity)
  348. activity->releaseIOs();
  349. connectedInputs.kill();
  350. inputs.kill();
  351. outputs.kill();
  352. activity.clear();
  353. }
  354. void CGraphElementBase::addDependsOn(CGraphBase *graph, int controlId)
  355. {
  356. ForEachItemIn(i, dependsOn)
  357. {
  358. if (dependsOn.item(i).graph == graph)
  359. return;
  360. }
  361. dependsOn.append(*new CGraphDependency(graph, controlId));
  362. }
  363. StringBuffer &CGraphElementBase::getOpt(const char *prop, StringBuffer &out) const
  364. {
  365. VStringBuffer path("hint[@name=\"%s\"]/@value", prop);
  366. if (!queryXGMML().getProp(path.toLowerCase().str(), out))
  367. queryJob().getOpt(prop, out);
  368. return out;
  369. }
  370. bool CGraphElementBase::getOptBool(const char *prop, bool defVal) const
  371. {
  372. bool def = queryJob().getOptBool(prop, defVal);
  373. VStringBuffer path("hint[@name=\"%s\"]/@value", prop);
  374. return queryXGMML().getPropBool(path.toLowerCase().str(), def);
  375. }
  376. int CGraphElementBase::getOptInt(const char *prop, int defVal) const
  377. {
  378. int def = queryJob().getOptInt(prop, defVal);
  379. VStringBuffer path("hint[@name=\"%s\"]/@value", prop);
  380. return queryXGMML().getPropInt(path.toLowerCase().str(), def);
  381. }
  382. __int64 CGraphElementBase::getOptInt64(const char *prop, __int64 defVal) const
  383. {
  384. __int64 def = queryJob().getOptInt64(prop, defVal);
  385. VStringBuffer path("hint[@name=\"%s\"]/@value", prop);
  386. return queryXGMML().getPropInt64(path.toLowerCase().str(), def);
  387. }
  388. IThorGraphDependencyIterator *CGraphElementBase::getDependsIterator() const
  389. {
  390. return new ArrayIIteratorOf<const CGraphDependencyArray, CGraphDependency, IThorGraphDependencyIterator>(dependsOn);
  391. }
  392. void CGraphElementBase::reset()
  393. {
  394. alreadyUpdated = false;
  395. if (activity)
  396. activity->reset();
  397. }
  398. void CGraphElementBase::ActPrintLog(const char *format, ...)
  399. {
  400. va_list args;
  401. va_start(args, format);
  402. ::ActPrintLogArgs(this, thorlog_null, MCdebugProgress, format, args);
  403. va_end(args);
  404. }
  405. void CGraphElementBase::ActPrintLog(IException *e, const char *format, ...)
  406. {
  407. va_list args;
  408. va_start(args, format);
  409. ::ActPrintLogArgs(this, e, thorlog_all, MCexception(e), format, args);
  410. va_end(args);
  411. }
  412. void CGraphElementBase::ActPrintLog(IException *e)
  413. {
  414. ActPrintLog(e, "%s", "");
  415. }
  416. void CGraphElementBase::abort(IException *e)
  417. {
  418. CActivityBase *activity = queryActivity();
  419. if (activity)
  420. activity->abort();
  421. }
  422. void CGraphElementBase::doconnect()
  423. {
  424. ForEachItemIn(i, connectedInputs)
  425. {
  426. CIOConnection *io = connectedInputs.item(i);
  427. if (io)
  428. io->connect(i, queryActivity());
  429. }
  430. }
  431. void CGraphElementBase::clearConnections()
  432. {
  433. connectedInputs.kill();
  434. connectedOutputs.kill();
  435. if (activity)
  436. activity->clearConnections();
  437. }
  438. void CGraphElementBase::addInput(unsigned input, CGraphElementBase *inputAct, unsigned inputOutIdx)
  439. {
  440. while (inputs.ordinality()<=input) inputs.append(NULL);
  441. inputs.replace(new COwningSimpleIOConnection(LINK(inputAct), inputOutIdx), input);
  442. while (inputAct->outputs.ordinality()<=inputOutIdx) inputAct->outputs.append(NULL);
  443. inputAct->outputs.replace(new CIOConnection(this, input), inputOutIdx);
  444. }
  445. void CGraphElementBase::connectInput(unsigned input, CGraphElementBase *inputAct, unsigned inputOutIdx)
  446. {
  447. ActPrintLog("CONNECTING (id=%" ACTPF "d, idx=%d) to (id=%" ACTPF "d, idx=%d)", inputAct->queryId(), inputOutIdx, queryId(), input);
  448. while (connectedInputs.ordinality()<=input) connectedInputs.append(NULL);
  449. connectedInputs.replace(new COwningSimpleIOConnection(LINK(inputAct), inputOutIdx), input);
  450. while (inputAct->connectedOutputs.ordinality()<=inputOutIdx) inputAct->connectedOutputs.append(NULL);
  451. inputAct->connectedOutputs.replace(new CIOConnection(this, input), inputOutIdx);
  452. }
  453. void CGraphElementBase::serializeCreateContext(MemoryBuffer &mb)
  454. {
  455. if (!onCreateCalled) return;
  456. DelayedSizeMarker sizeMark(mb);
  457. queryHelper()->serializeCreateContext(mb);
  458. sizeMark.write();
  459. if (isSink())
  460. mb.append(alreadyUpdated);
  461. }
  462. void CGraphElementBase::deserializeCreateContext(MemoryBuffer &mb)
  463. {
  464. size32_t createCtxLen;
  465. mb.read(createCtxLen);
  466. createCtxMb.clear().append(createCtxLen, mb.readDirect(createCtxLen));
  467. haveCreateCtx = true;
  468. if (isSink())
  469. mb.read(alreadyUpdated);
  470. }
  471. void CGraphElementBase::serializeStartContext(MemoryBuffer &mb)
  472. {
  473. DelayedSizeMarker sizeMark(mb);
  474. queryHelper()->serializeStartContext(mb);
  475. sizeMark.write();
  476. }
  477. void CGraphElementBase::onCreate()
  478. {
  479. CriticalBlock b(crit);
  480. if (onCreateCalled)
  481. return;
  482. onCreateCalled = true;
  483. if (!nullAct)
  484. {
  485. CGraphElementBase *ownerActivity = owner->queryOwner() ? owner->queryOwner()->queryElement(ownerId) : NULL;
  486. if (ownerActivity)
  487. {
  488. ownerActivity->onCreate(); // ensure owner created, might not be if this is child query inside another child query.
  489. baseHelper->onCreate(queryCodeContext(), ownerActivity->queryHelper(), haveCreateCtx?&createCtxMb:NULL);
  490. }
  491. else
  492. baseHelper->onCreate(queryCodeContext(), NULL, haveCreateCtx?&createCtxMb:NULL);
  493. if (isLoopActivity(*this))
  494. {
  495. unsigned loopId = queryXGMML().getPropInt("att[@name=\"_loopid\"]/@value");
  496. Owned<CGraphStub> stub = owner->getChildGraph(loopId);
  497. Owned<IThorBoundLoopGraph> boundLoopGraph = createBoundLoopGraph(&stub->queryOriginalGraph(), baseHelper->queryOutputMeta(), queryId());
  498. setBoundGraph(boundLoopGraph);
  499. }
  500. }
  501. }
  502. void CGraphElementBase::onStart(size32_t parentExtractSz, const byte *parentExtract, MemoryBuffer *startCtx)
  503. {
  504. if (nullAct)
  505. return;
  506. CriticalBlock b(crit);
  507. baseHelper->onStart(parentExtract, startCtx);
  508. }
  509. bool CGraphElementBase::executeDependencies(size32_t parentExtractSz, const byte *parentExtract, int controlId, bool async)
  510. {
  511. Owned<IThorGraphDependencyIterator> deps = getDependsIterator();
  512. ForEach(*deps)
  513. {
  514. CGraphDependency &dep = deps->query();
  515. if (dep.controlId == controlId)
  516. dep.graph->execute(parentExtractSz, parentExtract, true, async);
  517. if (owner->queryJob().queryAborted() || owner->queryAborted()) return false;
  518. }
  519. return true;
  520. }
  521. bool CGraphElementBase::prepareContext(size32_t parentExtractSz, const byte *parentExtract, bool checkDependencies, bool shortCircuit, bool async, bool connectOnly)
  522. {
  523. try
  524. {
  525. bool create = true;
  526. if (connectOnly)
  527. {
  528. if (prepared)
  529. return true;
  530. ForEachItemIn(i, inputs)
  531. {
  532. if (!queryInput(i)->prepareContext(parentExtractSz, parentExtract, false, false, async, true))
  533. return false;
  534. }
  535. }
  536. else
  537. {
  538. bool _shortCircuit = shortCircuit;
  539. Owned<IThorGraphDependencyIterator> deps = getDependsIterator();
  540. bool depsDone = true;
  541. ForEach(*deps)
  542. {
  543. CGraphDependency &dep = deps->query();
  544. if (0 == dep.controlId && NotFound == owner->dependentSubGraphs.find(*dep.graph))
  545. {
  546. owner->dependentSubGraphs.append(*dep.graph);
  547. if (!dep.graph->isComplete())
  548. depsDone = false;
  549. }
  550. }
  551. if (depsDone) _shortCircuit = false;
  552. if (!depsDone && checkDependencies)
  553. {
  554. if (!executeDependencies(parentExtractSz, parentExtract, 0, async))
  555. return false;
  556. }
  557. whichBranch = (unsigned)-1;
  558. switch (getKind())
  559. {
  560. case TAKindexwrite:
  561. case TAKdiskwrite:
  562. case TAKcsvwrite:
  563. case TAKxmlwrite:
  564. case TAKjsonwrite:
  565. case TAKspillwrite:
  566. if (_shortCircuit) return true;
  567. onCreate();
  568. alreadyUpdated = checkUpdate();
  569. if (alreadyUpdated)
  570. return false;
  571. break;
  572. case TAKchildif:
  573. case TAKif:
  574. case TAKifaction:
  575. {
  576. if (_shortCircuit) return true;
  577. onCreate();
  578. onStart(parentExtractSz, parentExtract);
  579. IHThorIfArg *helper = (IHThorIfArg *)baseHelper.get();
  580. whichBranch = helper->getCondition() ? 0 : 1; // True argument precedes false...
  581. /* NB - The executeDependencies code below is only needed if actionLinkInNewGraph=true, which is no longer the default
  582. * It should be removed, once we are positive there are no issues with in-line conditional actions
  583. */
  584. if (TAKifaction == getKind())
  585. {
  586. if (!executeDependencies(parentExtractSz, parentExtract, whichBranch+1, async)) //NB whenId 1 based
  587. return false;
  588. create = false;
  589. }
  590. break;
  591. }
  592. case TAKchildcase:
  593. case TAKcase:
  594. {
  595. if (_shortCircuit) return true;
  596. onCreate();
  597. onStart(parentExtractSz, parentExtract);
  598. IHThorCaseArg *helper = (IHThorCaseArg *)baseHelper.get();
  599. whichBranch = helper->getBranch();
  600. if (whichBranch >= inputs.ordinality())
  601. whichBranch = inputs.ordinality()-1;
  602. break;
  603. }
  604. case TAKfilter:
  605. case TAKfiltergroup:
  606. case TAKfilterproject:
  607. {
  608. if (_shortCircuit) return true;
  609. onCreate();
  610. onStart(parentExtractSz, parentExtract);
  611. switch (getKind())
  612. {
  613. case TAKfilter:
  614. whichBranch = ((IHThorFilterArg *)baseHelper.get())->canMatchAny() ? 0 : 1;
  615. break;
  616. case TAKfiltergroup:
  617. whichBranch = ((IHThorFilterGroupArg *)baseHelper.get())->canMatchAny() ? 0 : 1;
  618. break;
  619. case TAKfilterproject:
  620. whichBranch = ((IHThorFilterProjectArg *)baseHelper.get())->canMatchAny() ? 0 : 1;
  621. break;
  622. }
  623. break;
  624. }
  625. case TAKsequential:
  626. case TAKparallel:
  627. {
  628. /* NB - The executeDependencies code below is only needed if actionLinkInNewGraph=true, which is no longer the default
  629. * It should be removed, once we are positive there are no issues with in-line sequential/parallel activities
  630. */
  631. for (unsigned s=1; s<=dependsOn.ordinality(); s++)
  632. executeDependencies(parentExtractSz, parentExtract, s, async);
  633. create = false;
  634. break;
  635. }
  636. case TAKwhen_dataset:
  637. case TAKwhen_action:
  638. {
  639. if (!executeDependencies(parentExtractSz, parentExtract, WhenBeforeId, async))
  640. return false;
  641. if (!executeDependencies(parentExtractSz, parentExtract, WhenParallelId, async))
  642. return false;
  643. break;
  644. }
  645. }
  646. if (checkDependencies && ((unsigned)-1 != whichBranch))
  647. {
  648. if (inputs.queryItem(whichBranch))
  649. {
  650. if (!queryInput(whichBranch)->prepareContext(parentExtractSz, parentExtract, true, false, async, connectOnly))
  651. return false;
  652. }
  653. ForEachItemIn(i, inputs)
  654. {
  655. if (i != whichBranch)
  656. {
  657. if (!queryInput(i)->prepareContext(parentExtractSz, parentExtract, false, false, async, true))
  658. return false;
  659. }
  660. }
  661. }
  662. else
  663. {
  664. ForEachItemIn(i, inputs)
  665. {
  666. if (!queryInput(i)->prepareContext(parentExtractSz, parentExtract, checkDependencies, false, async, connectOnly))
  667. return false;
  668. }
  669. }
  670. }
  671. if (create)
  672. {
  673. if (prepared) // no need to recreate
  674. return true;
  675. prepared = true;
  676. ForEachItemIn(i2, inputs)
  677. {
  678. CIOConnection *inputIO = inputs.item(i2);
  679. connectInput(i2, inputIO->activity, inputIO->index);
  680. }
  681. createActivity();
  682. }
  683. return true;
  684. }
  685. catch (IException *_e)
  686. {
  687. IThorException *e = QUERYINTERFACE(_e, IThorException);
  688. if (e)
  689. {
  690. if (!e->queryActivityId())
  691. setExceptionActivityInfo(*this, e);
  692. }
  693. else
  694. {
  695. e = MakeActivityException(this, _e);
  696. _e->Release();
  697. }
  698. throw e;
  699. }
  700. }
  701. void CGraphElementBase::preStart(size32_t parentExtractSz, const byte *parentExtract)
  702. {
  703. activity->preStart(parentExtractSz, parentExtract);
  704. }
  705. void CGraphElementBase::createActivity()
  706. {
  707. CriticalBlock b(crit);
  708. if (activity)
  709. return;
  710. activity.setown(factory());
  711. if (isSink())
  712. owner->addActiveSink(*this);
  713. }
  714. ICodeContext *CGraphElementBase::queryCodeContext()
  715. {
  716. return queryOwner().queryCodeContext();
  717. }
  718. /////
  719. // JCSMORE loop - probably need better way to check if any act in graph is global(meaning needs some synchronization between slaves in activity execution)
  720. bool isGlobalActivity(CGraphElementBase &container)
  721. {
  722. switch (container.getKind())
  723. {
  724. // always global, but only co-ordinate init/done
  725. case TAKcsvwrite:
  726. case TAKxmlwrite:
  727. case TAKjsonwrite:
  728. case TAKindexwrite:
  729. case TAKkeydiff:
  730. case TAKkeypatch:
  731. case TAKdictionaryworkunitwrite:
  732. return true;
  733. case TAKdiskwrite:
  734. {
  735. Owned<IHThorDiskWriteArg> helper = (IHThorDiskWriteArg *)container.helperFactory();
  736. unsigned flags = helper->getFlags();
  737. return (0 == (TDXtemporary & flags)); // global if not temporary
  738. }
  739. case TAKspillwrite:
  740. case TAKspill:
  741. return false;
  742. case TAKcsvread:
  743. {
  744. Owned<IHThorCsvReadArg> helper = (IHThorCsvReadArg *)container.helperFactory();
  745. // if header lines, then [may] need to co-ordinate across slaves
  746. if (container.queryOwner().queryOwner() && (!container.queryOwner().isGlobal())) // I am in a child query
  747. return false;
  748. return helper->queryCsvParameters()->queryHeaderLen() > 0;
  749. }
  750. // dependent on child acts?
  751. case TAKlooprow:
  752. case TAKloopcount:
  753. case TAKgraphloop:
  754. case TAKparallelgraphloop:
  755. case TAKloopdataset:
  756. case TAKexternalsink:
  757. case TAKexternalsource:
  758. case TAKexternalprocess:
  759. return false;
  760. // dependent on local/grouped
  761. case TAKkeyeddistribute:
  762. case TAKhashdistribute:
  763. case TAKhashdistributemerge:
  764. case TAKnwaydistribute:
  765. case TAKworkunitwrite:
  766. case TAKdistribution:
  767. case TAKpartition:
  768. case TAKdiskaggregate:
  769. case TAKdiskcount:
  770. case TAKdiskgroupaggregate:
  771. case TAKindexaggregate:
  772. case TAKindexcount:
  773. case TAKindexgroupaggregate:
  774. case TAKindexgroupexists:
  775. case TAKindexgroupcount:
  776. case TAKremoteresult:
  777. case TAKcountproject:
  778. case TAKcreaterowlimit:
  779. case TAKskiplimit:
  780. case TAKlimit:
  781. case TAKsort:
  782. case TAKdedup:
  783. case TAKjoin:
  784. case TAKselfjoin:
  785. case TAKhashjoin:
  786. case TAKsmartjoin:
  787. case TAKkeyeddenormalize:
  788. case TAKhashdenormalize:
  789. case TAKdenormalize:
  790. case TAKlookupdenormalize: //GH->JCS why are these here, and join not?
  791. case TAKalldenormalize:
  792. case TAKsmartdenormalize:
  793. case TAKdenormalizegroup:
  794. case TAKhashdenormalizegroup:
  795. case TAKlookupdenormalizegroup:
  796. case TAKkeyeddenormalizegroup:
  797. case TAKalldenormalizegroup:
  798. case TAKsmartdenormalizegroup:
  799. case TAKaggregate:
  800. case TAKexistsaggregate:
  801. case TAKcountaggregate:
  802. case TAKhashaggregate:
  803. case TAKhashdedup:
  804. case TAKrollup:
  805. case TAKiterate:
  806. case TAKselectn:
  807. case TAKfirstn:
  808. case TAKenth:
  809. case TAKsample:
  810. case TAKgroup:
  811. case TAKchoosesets:
  812. case TAKchoosesetsenth:
  813. case TAKchoosesetslast:
  814. case TAKtopn:
  815. case TAKprocess:
  816. case TAKchildcount:
  817. case TAKwhen_dataset:
  818. case TAKwhen_action:
  819. case TAKnonempty:
  820. case TAKifaction:
  821. if (!container.queryLocalOrGrouped())
  822. return true;
  823. break;
  824. case TAKkeyedjoin:
  825. case TAKalljoin:
  826. case TAKlookupjoin:
  827. if (!container.queryLocal())
  828. return true;
  829. // always local
  830. case TAKfilter:
  831. case TAKfilterproject:
  832. case TAKfiltergroup:
  833. case TAKsplit:
  834. case TAKpipewrite:
  835. case TAKdegroup:
  836. case TAKproject:
  837. case TAKprefetchproject:
  838. case TAKprefetchcountproject:
  839. case TAKnormalize:
  840. case TAKnormalizechild:
  841. case TAKnormalizelinkedchild:
  842. case TAKpipethrough:
  843. case TAKif:
  844. case TAKchildif:
  845. case TAKchildcase:
  846. case TAKcase:
  847. case TAKparse:
  848. case TAKpiperead:
  849. case TAKxmlparse:
  850. case TAKjoinlight:
  851. case TAKselfjoinlight:
  852. case TAKdiskread:
  853. case TAKdisknormalize:
  854. case TAKchildaggregate:
  855. case TAKchildgroupaggregate:
  856. case TAKchildthroughnormalize:
  857. case TAKchildnormalize:
  858. case TAKspillread:
  859. case TAKindexread:
  860. case TAKindexnormalize:
  861. case TAKxmlread:
  862. case TAKjsonread:
  863. case TAKdiskexists:
  864. case TAKindexexists:
  865. case TAKchildexists:
  866. case TAKthroughaggregate:
  867. case TAKmerge:
  868. case TAKfunnel:
  869. case TAKregroup:
  870. case TAKcombine:
  871. case TAKrollupgroup:
  872. case TAKcombinegroup:
  873. case TAKsoap_rowdataset:
  874. case TAKhttp_rowdataset:
  875. case TAKsoap_rowaction:
  876. case TAKsoap_datasetdataset:
  877. case TAKsoap_datasetaction:
  878. case TAKlinkedrawiterator:
  879. case TAKchilditerator:
  880. case TAKstreamediterator:
  881. case TAKworkunitread:
  882. case TAKchilddataset:
  883. case TAKinlinetable:
  884. case TAKnull:
  885. case TAKemptyaction:
  886. case TAKlocalresultread:
  887. case TAKlocalresultwrite:
  888. case TAKdictionaryresultwrite:
  889. case TAKgraphloopresultread:
  890. case TAKgraphloopresultwrite:
  891. case TAKnwaygraphloopresultread:
  892. case TAKapply:
  893. case TAKsideeffect:
  894. case TAKsimpleaction:
  895. case TAKsorted:
  896. case TAKdistributed:
  897. case TAKtrace:
  898. break;
  899. case TAKnwayjoin:
  900. case TAKnwaymerge:
  901. case TAKnwaymergejoin:
  902. case TAKnwayinput:
  903. case TAKnwayselect:
  904. return false; // JCSMORE - I think and/or have to be for now
  905. // undefined
  906. case TAKdatasetresult:
  907. case TAKrowresult:
  908. case TAKremotegraph:
  909. case TAKlibrarycall:
  910. default:
  911. return true; // if in doubt
  912. }
  913. return false;
  914. }
  915. bool isLoopActivity(CGraphElementBase &container)
  916. {
  917. switch (container.getKind())
  918. {
  919. case TAKlooprow:
  920. case TAKloopcount:
  921. case TAKloopdataset:
  922. case TAKgraphloop:
  923. case TAKparallelgraphloop:
  924. return true;
  925. }
  926. return false;
  927. }
  928. static void getGlobalDeps(CGraphBase &graph, CICopyArrayOf<CGraphDependency> &deps)
  929. {
  930. Owned<IThorActivityIterator> iter = graph.getIterator();
  931. ForEach(*iter)
  932. {
  933. CGraphElementBase &elem = iter->query();
  934. Owned<IThorGraphDependencyIterator> dependIterator = elem.getDependsIterator();
  935. ForEach(*dependIterator)
  936. {
  937. CGraphDependency &dependency = dependIterator->query();
  938. if (dependency.graph->isGlobal() && NULL==dependency.graph->queryOwner())
  939. deps.append(dependency);
  940. getGlobalDeps(*dependency.graph, deps);
  941. }
  942. }
  943. }
  944. static void noteDependency(CGraphElementBase *targetActivity, CGraphElementBase *sourceActivity, CGraphBase *targetGraph, CGraphBase *sourceGraph, unsigned controlId)
  945. {
  946. targetActivity->addDependsOn(sourceGraph, controlId);
  947. // NB: record dependency in source graph, serialized to slaves, used to decided if should run dependency sinks or not
  948. Owned<IPropertyTree> dependencyFor = createPTree();
  949. dependencyFor->setPropInt("@id", sourceActivity->queryId());
  950. dependencyFor->setPropInt("@graphId", targetGraph->queryGraphId());
  951. if (controlId)
  952. dependencyFor->setPropInt("@conditionalId", controlId);
  953. sourceGraph->queryXGMML().addPropTree("Dependency", dependencyFor.getClear());
  954. }
  955. static void addDependencies(IPropertyTree *xgmml, bool failIfMissing, CGraphTableCopy &graphs)
  956. {
  957. CGraphArrayCopy dependentchildGraphs;
  958. CGraphElementArrayCopy targetActivities, sourceActivities;
  959. Owned<IPropertyTreeIterator> iter = xgmml->getElements("edge");
  960. ForEach(*iter)
  961. {
  962. IPropertyTree &edge = iter->query();
  963. graph_id sourceGid = edge.getPropInt("@source");
  964. graph_id targetGid = edge.getPropInt("@target");
  965. Owned<CGraphBase> source = LINK(graphs.find(sourceGid));
  966. Owned<CGraphBase> target = LINK(graphs.find(targetGid));
  967. if (!source || !target)
  968. {
  969. if (failIfMissing)
  970. throwUnexpected();
  971. else
  972. continue; // expected if assigning dependencies in slaves
  973. }
  974. CGraphElementBase *targetActivity = (CGraphElementBase *)target->queryElement(edge.getPropInt("att[@name=\"_targetActivity\"]/@value"));
  975. CGraphElementBase *sourceActivity = (CGraphElementBase *)source->queryElement(edge.getPropInt("att[@name=\"_sourceActivity\"]/@value"));
  976. if (!edge.getPropBool("att[@name=\"_childGraph\"]/@value"))
  977. {
  978. if (TAKlocalresultwrite == sourceActivity->getKind() && (TAKlocalresultread != targetActivity->getKind()))
  979. {
  980. if (source->isLoopSubGraph())
  981. source->setGlobal(true);
  982. }
  983. }
  984. int controlId = 0;
  985. if (edge.getPropBool("att[@name=\"_dependsOn\"]/@value", false))
  986. {
  987. if (!edge.getPropBool("att[@name=\"_childGraph\"]/@value", false)) // JCSMORE - not sure if necess. roxie seem to do.
  988. controlId = edge.getPropInt("att[@name=\"_when\"]/@value", 0);
  989. CGraphBase &sourceGraph = sourceActivity->queryOwner();
  990. unsigned sourceGraphContext = sourceGraph.queryParentActivityId();
  991. CGraphBase *targetGraph = NULL;
  992. unsigned targetGraphContext = -1;
  993. for (;;)
  994. {
  995. targetGraph = &targetActivity->queryOwner();
  996. targetGraphContext = targetGraph->queryParentActivityId();
  997. if (sourceGraphContext == targetGraphContext)
  998. break;
  999. targetActivity = targetGraph->queryElement(targetGraphContext);
  1000. }
  1001. assertex(targetActivity && sourceActivity);
  1002. noteDependency(targetActivity, sourceActivity, target, source, controlId);
  1003. }
  1004. else if (edge.getPropBool("att[@name=\"_conditionSource\"]/@value", false))
  1005. { /* Ignore it */ }
  1006. else if (edge.getPropBool("att[@name=\"_childGraph\"]/@value", false))
  1007. {
  1008. // NB: any dependencies of the child acts. are dependencies of this act.
  1009. dependentchildGraphs.append(*source);
  1010. targetActivities.append(*targetActivity);
  1011. sourceActivities.append(*sourceActivity);
  1012. }
  1013. else
  1014. {
  1015. if (!edge.getPropBool("att[@name=\"_childGraph\"]/@value", false)) // JCSMORE - not sure if necess. roxie seem to do.
  1016. controlId = edge.getPropInt("att[@name=\"_when\"]/@value", 0);
  1017. noteDependency(targetActivity, sourceActivity, target, source, controlId);
  1018. }
  1019. }
  1020. ForEachItemIn(c, dependentchildGraphs)
  1021. {
  1022. CGraphBase &childGraph = dependentchildGraphs.item(c);
  1023. CGraphElementBase &targetActivity = targetActivities.item(c);
  1024. CGraphElementBase &sourceActivity = sourceActivities.item(c);
  1025. if (!childGraph.isGlobal())
  1026. {
  1027. CICopyArrayOf<CGraphDependency> globalChildGraphDeps;
  1028. getGlobalDeps(childGraph, globalChildGraphDeps);
  1029. ForEachItemIn(gcd, globalChildGraphDeps)
  1030. {
  1031. CGraphDependency &globalDep = globalChildGraphDeps.item(gcd);
  1032. noteDependency(&targetActivity, &sourceActivity, globalDep.graph, &childGraph, globalDep.controlId);
  1033. }
  1034. }
  1035. }
  1036. SuperHashIteratorOf<CGraphBase> allIter(graphs);
  1037. ForEach(allIter)
  1038. {
  1039. CGraphBase &subGraph = allIter.query();
  1040. if (subGraph.queryOwner() && subGraph.queryParentActivityId())
  1041. {
  1042. CGraphElementBase *parentElement = subGraph.queryOwner()->queryElement(subGraph.queryParentActivityId());
  1043. if (isLoopActivity(*parentElement))
  1044. {
  1045. if (!parentElement->queryOwner().isLocalChild() && !subGraph.isLocalOnly())
  1046. subGraph.setGlobal(true);
  1047. }
  1048. }
  1049. }
  1050. }
  1051. void traceMemUsage()
  1052. {
  1053. StringBuffer memStatsStr;
  1054. roxiemem::memstats(memStatsStr);
  1055. PROGLOG("Roxiemem stats: %s", memStatsStr.str());
  1056. memsize_t heapUsage = getMapInfo("heap");
  1057. if (heapUsage) // if 0, assumed to be unavailable
  1058. {
  1059. memsize_t rmtotal = roxiemem::getTotalMemoryLimit();
  1060. PROGLOG("Heap usage (excluding Roxiemem) : %" I64F "d bytes", (unsigned __int64)(heapUsage-rmtotal));
  1061. }
  1062. }
  1063. /////
  1064. CGraphBase::CGraphBase(CJobChannel &_jobChannel) : jobChannel(_jobChannel), job(_jobChannel.queryJob()), progressUpdated(false)
  1065. {
  1066. xgmml = NULL;
  1067. parent = owner = graphResultsContainer = NULL;
  1068. complete = false;
  1069. parentActivityId = 0;
  1070. connected = started = graphDone = aborted = false;
  1071. startBarrier = waitBarrier = doneBarrier = NULL;
  1072. mpTag = waitBarrierTag = startBarrierTag = doneBarrierTag = TAG_NULL;
  1073. executeReplyTag = TAG_NULL;
  1074. parentExtractSz = 0;
  1075. counter = 0; // loop/graph counter, will be set by loop/graph activity if needed
  1076. loopBodySubgraph = false;
  1077. }
  1078. CGraphBase::~CGraphBase()
  1079. {
  1080. clean();
  1081. }
  1082. CGraphBase *CGraphBase::cloneGraph()
  1083. {
  1084. Owned<CGraphBase> subGraph = queryJobChannel().createGraph();
  1085. CGraphTableCopy newGraphs;
  1086. subGraph->createFromXGMML(node, owner, parent, graphResultsContainer, newGraphs);
  1087. addDependencies(queryJob().queryXGMML(), false, newGraphs);
  1088. return subGraph.getClear();
  1089. }
  1090. void CGraphBase::init()
  1091. {
  1092. bool log = queryJob().queryForceLogging(queryGraphId(), (NULL == queryOwner()) || isGlobal());
  1093. setLogging(log);
  1094. }
  1095. void CGraphBase::clean()
  1096. {
  1097. ::Release(startBarrier);
  1098. ::Release(waitBarrier);
  1099. ::Release(doneBarrier);
  1100. localResults.clear();
  1101. graphLoopResults.clear();
  1102. childGraphsTable.releaseAll();
  1103. disconnectActivities();
  1104. containers.releaseAll();
  1105. sinks.kill();
  1106. activeSinks.kill();
  1107. }
  1108. void CGraphBase::serializeCreateContexts(MemoryBuffer &mb)
  1109. {
  1110. DelayedSizeMarker sizeMark(mb);
  1111. Owned<IThorActivityIterator> iter = getIterator();
  1112. ForEach (*iter)
  1113. {
  1114. CGraphElementBase &element = iter->query();
  1115. if (element.isOnCreated())
  1116. {
  1117. mb.append(element.queryId());
  1118. element.serializeCreateContext(mb);
  1119. }
  1120. }
  1121. mb.append((activity_id)0);
  1122. sizeMark.write();
  1123. }
  1124. void CGraphBase::deserializeCreateContexts(MemoryBuffer &mb)
  1125. {
  1126. activity_id id;
  1127. for (;;)
  1128. {
  1129. mb.read(id);
  1130. if (0 == id) break;
  1131. CGraphElementBase *element = queryElement(id);
  1132. assertex(element);
  1133. element->deserializeCreateContext(mb);
  1134. }
  1135. }
  1136. void CGraphBase::reset()
  1137. {
  1138. setCompleteEx(false);
  1139. clearProgressUpdated();
  1140. graphCancelHandler.reset();
  1141. if (0 == containers.count())
  1142. {
  1143. Owned<IThorGraphIterator> iter = getChildGraphIterator();
  1144. ForEach(*iter)
  1145. iter->query().reset();
  1146. }
  1147. else
  1148. {
  1149. Owned<IThorActivityIterator> iter = getIterator();
  1150. ForEach(*iter)
  1151. {
  1152. CGraphElementBase &element = iter->query();
  1153. element.reset();
  1154. }
  1155. dependentSubGraphs.kill();
  1156. }
  1157. if (!queryOwner())
  1158. clearNodeStats();
  1159. }
  1160. void CGraphBase::addChildGraph(CGraphStub *stub)
  1161. {
  1162. CriticalBlock b(crit);
  1163. childGraphsTable.replace(*LINK(stub));
  1164. if (sequential)
  1165. orderedChildGraphs.append(*stub);
  1166. }
  1167. IThorGraphStubIterator *CGraphBase::getChildStubIterator() const
  1168. {
  1169. CriticalBlock b(crit);
  1170. class CIter : private SuperHashIteratorOf<CGraphStub>, public CSimpleInterfaceOf<IThorGraphStubIterator>
  1171. {
  1172. typedef SuperHashIteratorOf<CGraphStub> PARENT;
  1173. public:
  1174. CIter(const CChildGraphTable &table) : PARENT(table) { }
  1175. // IIterator
  1176. virtual bool first() { return PARENT::first(); }
  1177. virtual bool next() { return PARENT::next(); }
  1178. virtual bool isValid() { return PARENT::isValid(); }
  1179. virtual CGraphStub &query() { return PARENT::query(); }
  1180. };
  1181. return new CIter(childGraphsTable);
  1182. }
  1183. IThorGraphIterator *CGraphBase::getChildGraphIterator() const
  1184. {
  1185. CriticalBlock b(crit);
  1186. class CIter : public CSimpleInterfaceOf<IThorGraphIterator>
  1187. {
  1188. Owned<IThorGraphStubIterator> iter;
  1189. public:
  1190. CIter(IThorGraphStubIterator *_iter) : iter(_iter)
  1191. {
  1192. }
  1193. // IIterator
  1194. virtual bool first() { return iter->first(); }
  1195. virtual bool next() { return iter->next(); }
  1196. virtual bool isValid() { return iter->isValid(); }
  1197. virtual CGraphBase &query()
  1198. {
  1199. CGraphStub &stub = iter->query();
  1200. return stub.queryOriginalGraph();
  1201. }
  1202. };
  1203. return new CIter(getChildStubIterator());
  1204. }
  1205. bool CGraphBase::fireException(IException *e)
  1206. {
  1207. return queryJobChannel().fireException(e);
  1208. }
  1209. bool CGraphBase::preStart(size32_t parentExtractSz, const byte *parentExtract)
  1210. {
  1211. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1212. ForEach(*iter)
  1213. {
  1214. CGraphElementBase &element = iter->query();
  1215. element.preStart(parentExtractSz, parentExtract);
  1216. }
  1217. return true;
  1218. }
  1219. void CGraphBase::executeSubGraph(size32_t parentExtractSz, const byte *parentExtract)
  1220. {
  1221. CriticalBlock b(executeCrit);
  1222. if (job.queryPausing())
  1223. return;
  1224. Owned<IException> exception;
  1225. try
  1226. {
  1227. if (!queryOwner())
  1228. {
  1229. StringBuffer s;
  1230. toXML(&queryXGMML(), s, 2);
  1231. GraphPrintLog("Running graph [%s] : %s", isGlobal()?"global":"local", s.str());
  1232. }
  1233. if (localResults)
  1234. localResults->clear();
  1235. doExecute(parentExtractSz, parentExtract, false);
  1236. }
  1237. catch (IException *e)
  1238. {
  1239. GraphPrintLog(e);
  1240. exception.setown(e);
  1241. }
  1242. if (!queryOwner())
  1243. {
  1244. GraphPrintLog("Graph Done");
  1245. StringBuffer memStr;
  1246. getSystemTraceInfo(memStr, PerfMonStandard | PerfMonExtended);
  1247. GraphPrintLog("%s", memStr.str());
  1248. }
  1249. if (exception)
  1250. throw exception.getClear();
  1251. }
  1252. void CGraphBase::onCreate()
  1253. {
  1254. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1255. ForEach(*iter)
  1256. {
  1257. CGraphElementBase &element = iter->query();
  1258. element.onCreate();
  1259. }
  1260. }
  1261. void CGraphBase::execute(size32_t _parentExtractSz, const byte *parentExtract, bool checkDependencies, bool async)
  1262. {
  1263. if (isComplete())
  1264. return;
  1265. if (async)
  1266. queryJobChannel().startGraph(*this, checkDependencies, _parentExtractSz, parentExtract); // may block if enough running
  1267. else
  1268. {
  1269. if (!prepare(_parentExtractSz, parentExtract, checkDependencies, false, false))
  1270. {
  1271. setComplete();
  1272. return;
  1273. }
  1274. executeSubGraph(_parentExtractSz, parentExtract);
  1275. }
  1276. }
  1277. void CGraphBase::doExecute(size32_t parentExtractSz, const byte *parentExtract, bool checkDependencies)
  1278. {
  1279. if (isComplete()) return;
  1280. if (queryAborted())
  1281. {
  1282. if (abortException)
  1283. throw abortException.getLink();
  1284. throw MakeGraphException(this, 0, "subgraph aborted");
  1285. }
  1286. GraphPrintLog("Processing graph");
  1287. Owned<IException> exception;
  1288. try
  1289. {
  1290. if (started)
  1291. reset();
  1292. else
  1293. started = true;
  1294. ++numExecuted;
  1295. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1296. ForEach(*iter)
  1297. {
  1298. CGraphElementBase &element = iter->query();
  1299. element.onStart(parentExtractSz, parentExtract);
  1300. element.initActivity();
  1301. }
  1302. initialized = true;
  1303. if (!preStart(parentExtractSz, parentExtract)) return;
  1304. start();
  1305. if (!wait(aborted?MEDIUMTIMEOUT:INFINITE)) // can't wait indefinitely, query may have aborted and stall, but prudent to wait a short time for underlying graphs to unwind.
  1306. GraphPrintLogEx(this, thorlog_null, MCuserWarning, "Graph wait cancelled, aborted=%s", aborted?"true":"false");
  1307. else
  1308. graphDone = true;
  1309. }
  1310. catch (IException *e)
  1311. {
  1312. GraphPrintLog(e);
  1313. exception.setown(e);
  1314. }
  1315. try
  1316. {
  1317. if (!exception && abortException)
  1318. exception.setown(abortException.getClear());
  1319. if (exception)
  1320. {
  1321. if (NULL == owner || isGlobal())
  1322. waitBarrier->cancel(exception);
  1323. if (!queryOwner())
  1324. {
  1325. StringBuffer str;
  1326. Owned<IThorException> e = MakeGraphException(this, exception->errorCode(), "%s", exception->errorMessage(str).str());
  1327. e->setAction(tea_abort);
  1328. fireException(e);
  1329. }
  1330. }
  1331. }
  1332. catch (IException *e)
  1333. {
  1334. GraphPrintLog(e, "during abort()");
  1335. e->Release();
  1336. }
  1337. try
  1338. {
  1339. done();
  1340. if (doneBarrier)
  1341. doneBarrier->wait(false);
  1342. }
  1343. catch (IException *e)
  1344. {
  1345. GraphPrintLog(e);
  1346. if (!exception.get())
  1347. exception.setown(e);
  1348. else
  1349. e->Release();
  1350. }
  1351. end();
  1352. if (exception)
  1353. throw exception.getClear();
  1354. if (!queryAborted())
  1355. setComplete();
  1356. }
  1357. bool CGraphBase::prepare(size32_t parentExtractSz, const byte *parentExtract, bool checkDependencies, bool shortCircuit, bool async)
  1358. {
  1359. if (isComplete()) return false;
  1360. bool needToExecute = false;
  1361. ForEachItemIn(s, sinks)
  1362. {
  1363. CGraphElementBase &sink = sinks.item(s);
  1364. if (sink.prepareContext(parentExtractSz, parentExtract, checkDependencies, shortCircuit, async, false))
  1365. needToExecute = true;
  1366. }
  1367. onCreate();
  1368. return needToExecute;
  1369. }
  1370. void CGraphBase::done()
  1371. {
  1372. if (aborted) return; // activity done methods only called on success
  1373. if (isLocalChild()) // CQ master activities are created on demand, call done() on any created
  1374. {
  1375. Owned<IThorActivityIterator> iter = getIterator();
  1376. ForEach(*iter)
  1377. {
  1378. CGraphElementBase &element = iter->query();
  1379. if (element.queryActivity())
  1380. element.queryActivity()->done();
  1381. }
  1382. }
  1383. else
  1384. {
  1385. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1386. ForEach (*iter)
  1387. {
  1388. CGraphElementBase &element = iter->query();
  1389. element.queryActivity()->done();
  1390. }
  1391. }
  1392. Owned<IThorGraphIterator> childIter = getChildGraphIterator();
  1393. ForEach(*childIter)
  1394. childIter->query().done();
  1395. }
  1396. unsigned CGraphBase::queryJobChannelNumber() const
  1397. {
  1398. return queryJobChannel().queryChannel();
  1399. }
  1400. IMPServer &CGraphBase::queryMPServer() const
  1401. {
  1402. return jobChannel.queryMPServer();
  1403. }
  1404. bool CGraphBase::syncInitData()
  1405. {
  1406. if (loopBodySubgraph)
  1407. {
  1408. CGraphElementBase *parentElement = queryOwner() ? queryOwner()->queryElement(queryParentActivityId()) : nullptr;
  1409. assertex(parentElement);
  1410. return parentElement->queryLoopGraph()->queryGraph()->isGlobal();
  1411. }
  1412. else
  1413. return !isLocalChild();
  1414. }
  1415. void CGraphBase::end()
  1416. {
  1417. // always called, any final action clear up
  1418. Owned<IThorActivityIterator> iter = getIterator();
  1419. ForEach(*iter)
  1420. {
  1421. CGraphElementBase &element = iter->query();
  1422. try
  1423. {
  1424. if (element.queryActivity())
  1425. element.queryActivity()->kill();
  1426. }
  1427. catch (IException *e)
  1428. {
  1429. Owned<IException> e2 = MakeActivityException(element.queryActivity(), e, "Error calling kill()");
  1430. GraphPrintLog(e2);
  1431. e->Release();
  1432. }
  1433. }
  1434. }
  1435. class CGraphTraverseIteratorBase : implements IThorActivityIterator, public CInterface
  1436. {
  1437. protected:
  1438. CGraphBase &graph;
  1439. Linked<CGraphElementBase> cur;
  1440. CIArrayOf<CGraphElementBase> others;
  1441. CGraphElementArrayCopy covered;
  1442. CGraphElementBase *popNext()
  1443. {
  1444. if (!others.ordinality())
  1445. {
  1446. cur.clear();
  1447. return NULL;
  1448. }
  1449. cur.setown(&others.popGet());
  1450. return cur;
  1451. }
  1452. void setNext(bool branchOnConditional)
  1453. {
  1454. if (branchOnConditional && ((unsigned)-1) != cur->whichBranch)
  1455. {
  1456. CIOConnection *io = cur->connectedInputs.queryItem(cur->whichBranch);
  1457. if (io)
  1458. cur.set(io->activity);
  1459. else
  1460. cur.clear();
  1461. }
  1462. else
  1463. {
  1464. CIOConnectionArray &inputs = cur->connectedInputs;
  1465. cur.clear();
  1466. unsigned n = inputs.ordinality();
  1467. bool first = true;
  1468. for (unsigned i=0; i<n; i++)
  1469. {
  1470. CIOConnection *io = inputs.queryItem(i);
  1471. if (io)
  1472. {
  1473. if (first)
  1474. {
  1475. first = false;
  1476. cur.set(io->activity);
  1477. }
  1478. else
  1479. others.append(*LINK(io->activity));
  1480. }
  1481. }
  1482. }
  1483. if (!cur)
  1484. {
  1485. if (!popNext())
  1486. return;
  1487. }
  1488. // check haven't been here before
  1489. for (;;)
  1490. {
  1491. if (cur->getOutputs() < 2)
  1492. break;
  1493. else if (NotFound == covered.find(*cur))
  1494. {
  1495. if (!cur->alreadyUpdated)
  1496. {
  1497. covered.append(*cur);
  1498. break;
  1499. }
  1500. }
  1501. if (!popNext())
  1502. return;
  1503. }
  1504. }
  1505. public:
  1506. IMPLEMENT_IINTERFACE;
  1507. CGraphTraverseIteratorBase(CGraphBase &_graph) : graph(_graph)
  1508. {
  1509. }
  1510. virtual bool first()
  1511. {
  1512. covered.kill();
  1513. others.kill();
  1514. cur.clear();
  1515. Owned<IThorActivityIterator> sinkIter = graph.getSinkIterator();
  1516. if (!sinkIter->first())
  1517. return false;
  1518. for (;;)
  1519. {
  1520. cur.set(& sinkIter->query());
  1521. if (!cur->alreadyUpdated)
  1522. break;
  1523. if (!sinkIter->next())
  1524. return false;
  1525. }
  1526. while (sinkIter->next())
  1527. others.append(sinkIter->get());
  1528. return true;
  1529. }
  1530. virtual bool isValid() { return NULL != cur.get(); }
  1531. virtual CGraphElementBase & query() { return *cur; }
  1532. CGraphElementBase & get() { return *LINK(cur); }
  1533. };
  1534. class CGraphTraverseConnectedIterator : public CGraphTraverseIteratorBase
  1535. {
  1536. bool branchOnConditional;
  1537. public:
  1538. CGraphTraverseConnectedIterator(CGraphBase &graph, bool _branchOnConditional) : CGraphTraverseIteratorBase(graph), branchOnConditional(_branchOnConditional) { }
  1539. virtual bool next()
  1540. {
  1541. setNext(branchOnConditional);
  1542. return NULL!=cur.get();
  1543. }
  1544. };
  1545. IThorActivityIterator *CGraphBase::getConnectedIterator(bool branchOnConditional)
  1546. {
  1547. return new CGraphTraverseConnectedIterator(*this, branchOnConditional);
  1548. }
  1549. bool CGraphBase::wait(unsigned timeout)
  1550. {
  1551. CTimeMon tm(timeout);
  1552. unsigned remaining = timeout;
  1553. class CWaitException
  1554. {
  1555. CGraphBase *graph;
  1556. Owned<IException> exception;
  1557. public:
  1558. CWaitException(CGraphBase *_graph) : graph(_graph) { }
  1559. IException *get() { return exception; }
  1560. void set(IException *e)
  1561. {
  1562. if (!exception)
  1563. exception.setown(e);
  1564. else
  1565. e->Release();
  1566. }
  1567. void throwException()
  1568. {
  1569. if (exception)
  1570. throw exception.getClear();
  1571. throw MakeGraphException(graph, 0, "Timed out waiting for graph to end");
  1572. }
  1573. } waitException(this);
  1574. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1575. ForEach (*iter)
  1576. {
  1577. CGraphElementBase &element = iter->query();
  1578. CActivityBase *activity = element.queryActivity();
  1579. if (INFINITE != timeout && tm.timedout(&remaining))
  1580. waitException.throwException();
  1581. try
  1582. {
  1583. if (!activity->wait(remaining))
  1584. waitException.throwException();
  1585. }
  1586. catch (IException *e)
  1587. {
  1588. waitException.set(e); // will discard if already set
  1589. if (timeout == INFINITE)
  1590. {
  1591. unsigned e = tm.elapsed();
  1592. if (e >= MEDIUMTIMEOUT)
  1593. waitException.throwException();
  1594. timeout = MEDIUMTIMEOUT-e;
  1595. tm.reset(timeout);
  1596. }
  1597. }
  1598. }
  1599. if (waitException.get())
  1600. waitException.throwException();
  1601. // synchronize all slaves to end of graphs
  1602. if (NULL == owner || isGlobal())
  1603. {
  1604. if (INFINITE != timeout && tm.timedout(&remaining))
  1605. waitException.throwException();
  1606. if (!waitBarrier->wait(true, remaining))
  1607. return false;
  1608. }
  1609. return true;
  1610. }
  1611. void CGraphBase::abort(IException *e)
  1612. {
  1613. if (aborted)
  1614. return;
  1615. {
  1616. CriticalBlock cb(crit);
  1617. abortException.set(e);
  1618. aborted = true;
  1619. graphCancelHandler.cancel(0);
  1620. if (0 == containers.count())
  1621. {
  1622. Owned<IThorGraphStubIterator> iter = getChildStubIterator();
  1623. ForEach(*iter)
  1624. {
  1625. CGraphStub &graph = iter->query();
  1626. graph.abort(e);
  1627. }
  1628. }
  1629. }
  1630. if (started && !graphDone)
  1631. {
  1632. Owned<IThorActivityIterator> iter = getConnectedIterator();
  1633. ForEach (*iter)
  1634. {
  1635. iter->query().abort(e); // JCSMORE - could do in parallel, they can take some time to timeout
  1636. }
  1637. if (startBarrier)
  1638. startBarrier->cancel(e);
  1639. if (waitBarrier)
  1640. waitBarrier->cancel(e);
  1641. if (doneBarrier)
  1642. doneBarrier->cancel(e);
  1643. }
  1644. }
  1645. void CGraphBase::GraphPrintLog(const char *format, ...)
  1646. {
  1647. va_list args;
  1648. va_start(args, format);
  1649. ::GraphPrintLogArgs(this, thorlog_null, MCdebugProgress, format, args);
  1650. va_end(args);
  1651. }
  1652. void CGraphBase::GraphPrintLog(IException *e, const char *format, ...)
  1653. {
  1654. va_list args;
  1655. va_start(args, format);
  1656. ::GraphPrintLogArgs(this, e, thorlog_null, MCdebugProgress, format, args);
  1657. va_end(args);
  1658. }
  1659. void CGraphBase::GraphPrintLog(IException *e)
  1660. {
  1661. GraphPrintLog(e, "%s", "");
  1662. }
  1663. void CGraphBase::setLogging(bool tf)
  1664. {
  1665. Owned<IThorActivityIterator> iter = getIterator();
  1666. ForEach(*iter)
  1667. iter->query().setLogging(tf);
  1668. }
  1669. void CGraphBase::createFromXGMML(IPropertyTree *_node, CGraphBase *_owner, CGraphBase *_parent, CGraphBase *resultsGraph, CGraphTableCopy &newGraphs)
  1670. {
  1671. class CChildParallelFactory : public CGraphStub
  1672. {
  1673. Linked<CGraphBase> originalChildGraph;
  1674. CriticalSection crit;
  1675. CIArrayOf<CGraphBase> stack;
  1676. CIArrayOf<CGraphBase> active;
  1677. bool originalAvailable = true;
  1678. CGraphBase *getGraph()
  1679. {
  1680. Owned<CGraphBase> childGraph;
  1681. {
  1682. CriticalBlock b(crit);
  1683. if (originalAvailable)
  1684. {
  1685. originalAvailable = false;
  1686. active.append(*originalChildGraph.getLink());
  1687. return originalChildGraph.getLink();
  1688. }
  1689. if (stack.length())
  1690. childGraph.setown(&stack.popGet());
  1691. }
  1692. if (!childGraph)
  1693. childGraph.setown(originalChildGraph->cloneGraph());
  1694. if (originalChildGraph->queryAborted())
  1695. throw MakeGraphException(originalChildGraph, 0, "Job aborted");
  1696. {
  1697. CriticalBlock b(crit);
  1698. active.append(*childGraph.getLink());
  1699. }
  1700. return childGraph.getClear();
  1701. }
  1702. void pushGraph(CGraphBase *childGraph)
  1703. {
  1704. CriticalBlock b(crit);
  1705. verifyex(active.zap(*childGraph));
  1706. if (childGraph == originalChildGraph)
  1707. originalAvailable = true;
  1708. else
  1709. stack.append(*LINK(childGraph));
  1710. }
  1711. public:
  1712. CChildParallelFactory(CGraphBase *_originalChildGraph) : originalChildGraph(_originalChildGraph)
  1713. {
  1714. graphId = originalChildGraph->queryGraphId();
  1715. }
  1716. virtual CGraphBase &queryOriginalGraph() override { return *originalChildGraph; }
  1717. virtual void abort(IException *e) override
  1718. {
  1719. for (;;)
  1720. {
  1721. Owned<CGraphBase> activeChildGraph;
  1722. {
  1723. CriticalBlock b(crit);
  1724. activeChildGraph.setown(&active.popGet());
  1725. if (!activeChildGraph)
  1726. break;
  1727. }
  1728. activeChildGraph->abort(e);
  1729. }
  1730. }
  1731. virtual bool serializeStats(MemoryBuffer &mb) override
  1732. {
  1733. // JCSMORE - need to merge other instances
  1734. return originalChildGraph->serializeStats(mb);
  1735. }
  1736. virtual IEclGraphResults * evaluate(unsigned parentExtractSz, const byte * parentExtract) override
  1737. {
  1738. Owned<CGraphBase> childGraph = getGraph();
  1739. Owned<IEclGraphResults> results = childGraph->evaluate(parentExtractSz, parentExtract);
  1740. pushGraph(childGraph);
  1741. return results.getClear();
  1742. }
  1743. };
  1744. owner = _owner;
  1745. parent = _parent?_parent:owner;
  1746. node.setown(createPTreeFromIPT(_node));
  1747. xgmml = node->queryPropTree("att/graph");
  1748. sink = xgmml->getPropBool("att[@name=\"rootGraph\"]/@value", false);
  1749. sequential = xgmml->getPropBool("@sequential");
  1750. graphId = node->getPropInt("@id");
  1751. global = false;
  1752. localOnly = -1; // unset
  1753. parentActivityId = node->getPropInt("att[@name=\"_parentActivity\"]/@value", 0);
  1754. graphResultsContainer = resultsGraph;
  1755. CGraphBase *graphContainer = this;
  1756. if (resultsGraph)
  1757. graphContainer = resultsGraph; // JCSMORE is this right?
  1758. graphCodeContext.setContext(this, graphContainer, (ICodeContextExt *)&jobChannel.queryCodeContext());
  1759. unsigned numResults = xgmml->getPropInt("att[@name=\"_numResults\"]/@value", 0);
  1760. if (numResults)
  1761. {
  1762. localResults.setown(createThorGraphResults(numResults));
  1763. resultsGraph = this;
  1764. // JCSMORE - it might more sense if this temp handler was owned by parent act., which may finish(get stopped) earlier than the owning graph
  1765. tmpHandler.setown(queryJob().createTempHandler(false));
  1766. }
  1767. localChild = false;
  1768. if (owner && parentActivityId)
  1769. {
  1770. CGraphElementBase *parentElement = owner->queryElement(parentActivityId);
  1771. if (isLoopActivity(*parentElement))
  1772. {
  1773. localChild = parentElement->queryOwner().isLocalChild();
  1774. unsigned loopId = parentElement->queryXGMML().getPropInt("att[@name=\"_loopid\"]/@value");
  1775. if ((graphId == loopId) || (owner->queryGraphId() == loopId))
  1776. loopBodySubgraph = true;
  1777. else
  1778. localChild = true;
  1779. }
  1780. else
  1781. localChild = true;
  1782. }
  1783. Owned<IPropertyTreeIterator> nodes = xgmml->getElements("node");
  1784. ForEach(*nodes)
  1785. {
  1786. IPropertyTree &e = nodes->query();
  1787. ThorActivityKind kind = (ThorActivityKind) e.getPropInt("att[@name=\"_kind\"]/@value");
  1788. if (TAKsubgraph == kind)
  1789. {
  1790. Owned<CGraphBase> subGraph = queryJobChannel().createGraph();
  1791. subGraph->createFromXGMML(&e, this, parent, resultsGraph, newGraphs);
  1792. activity_id subGraphParentActivityId = e.getPropInt("att[@name=\"_parentActivity\"]/@value", 0);
  1793. if (subGraphParentActivityId) // JCS - not sure if ever false
  1794. {
  1795. Owned<CGraphStub> stub = new CChildParallelFactory(subGraph);
  1796. addChildGraph(stub);
  1797. }
  1798. else
  1799. addChildGraph(subGraph);
  1800. if (!global)
  1801. global = subGraph->isGlobal();
  1802. newGraphs.replace(*subGraph);
  1803. }
  1804. else
  1805. {
  1806. if (localChild && !e.getPropBool("att[@name=\"coLocal\"]/@value", false))
  1807. {
  1808. IPropertyTree *att = createPTree("att");
  1809. att->setProp("@name", "coLocal");
  1810. att->setPropBool("@value", true);
  1811. e.addPropTree("att", att);
  1812. }
  1813. CGraphElementBase *act = createGraphElement(e, *this, resultsGraph);
  1814. addActivity(act);
  1815. if (!global)
  1816. global = isGlobalActivity(*act);
  1817. }
  1818. }
  1819. Owned<IPropertyTreeIterator> edges = xgmml->getElements("edge");
  1820. ForEach(*edges)
  1821. {
  1822. IPropertyTree &edge = edges->query();
  1823. //Ignore edges that represent dependencies from parent activities to child activities.
  1824. if (edge.getPropBool("att[@name=\"_childGraph\"]/@value", false))
  1825. continue;
  1826. unsigned sourceOutput = edge.getPropInt("att[@name=\"_sourceIndex\"]/@value", 0);
  1827. unsigned targetInput = edge.getPropInt("att[@name=\"_targetIndex\"]/@value", 0);
  1828. CGraphElementBase *source = queryElement(edge.getPropInt("@source"));
  1829. CGraphElementBase *target = queryElement(edge.getPropInt("@target"));
  1830. target->addInput(targetInput, source, sourceOutput);
  1831. }
  1832. Owned<IThorActivityIterator> iter = getIterator();
  1833. ForEach(*iter)
  1834. {
  1835. CGraphElementBase &element = iter->query();
  1836. if (0 == element.getOutputs())
  1837. {
  1838. /* JCSMORE - Making some outputs conditional, will require:
  1839. * a) Pass through information as to which dependent graph causes this graph (and this sink) to execute)
  1840. * b) Allow the subgraph to re-executed by other dependent subgraphs and avoid re-executing completed sinks
  1841. * c) Keep common points (splitters) around (preferably in memory), re-execution of graph will need them
  1842. */
  1843. sinks.append(*LINK(&element));
  1844. }
  1845. }
  1846. init();
  1847. }
  1848. void CGraphBase::executeChildGraphs(size32_t parentExtractSz, const byte *parentExtract)
  1849. {
  1850. if (sequential)
  1851. {
  1852. // JCSMORE - would need to re-think how this is done if these sibling child queries could be executed in parallel
  1853. ForEachItemIn(o, orderedChildGraphs)
  1854. {
  1855. CGraphBase &graph = orderedChildGraphs.item(o).queryOriginalGraph();
  1856. if (graph.isSink())
  1857. graph.execute(parentExtractSz, parentExtract, true, false);
  1858. }
  1859. }
  1860. else
  1861. {
  1862. Owned<IThorGraphIterator> iter = getChildGraphIterator();
  1863. ForEach(*iter)
  1864. {
  1865. CGraphBase &graph = iter->query();
  1866. if (graph.isSink())
  1867. graph.execute(parentExtractSz, parentExtract, true, false);
  1868. }
  1869. }
  1870. }
  1871. void CGraphBase::doExecuteChild(size32_t parentExtractSz, const byte *parentExtract)
  1872. {
  1873. reset();
  1874. if (0 == containers.count())
  1875. executeChildGraphs(parentExtractSz, parentExtract);
  1876. else
  1877. execute(parentExtractSz, parentExtract, false, false);
  1878. IGraphTempHandler *tempHandler = queryTempHandler(false);
  1879. if (tempHandler)
  1880. tempHandler->clearTemps();
  1881. }
  1882. void CGraphBase::executeChild(size32_t & retSize, void * &ret, size32_t parentExtractSz, const byte *parentExtract)
  1883. {
  1884. reset();
  1885. doExecute(parentExtractSz, parentExtract, false);
  1886. UNIMPLEMENTED;
  1887. /*
  1888. ForEachItemIn(idx1, elements)
  1889. {
  1890. EclGraphElement & cur = elements.item(idx1);
  1891. if (cur.isResult)
  1892. {
  1893. cur.extractResult(retSize, ret);
  1894. return;
  1895. }
  1896. }
  1897. */
  1898. throwUnexpected();
  1899. }
  1900. void CGraphBase::setResults(IThorGraphResults *results) // used by master only
  1901. {
  1902. localResults.set(results);
  1903. }
  1904. void CGraphBase::executeChild(size32_t parentExtractSz, const byte *parentExtract, IThorGraphResults *results, IThorGraphResults *_graphLoopResults)
  1905. {
  1906. localResults.set(results);
  1907. graphLoopResults.set(_graphLoopResults);
  1908. doExecuteChild(parentExtractSz, parentExtract);
  1909. graphLoopResults.clear();
  1910. localResults.clear();
  1911. }
  1912. StringBuffer &getGlobals(CGraphBase &graph, StringBuffer &str)
  1913. {
  1914. bool first = true;
  1915. Owned<IThorActivityIterator> iter = graph.getIterator();
  1916. ForEach(*iter)
  1917. {
  1918. CGraphElementBase &e = iter->query();
  1919. if (isGlobalActivity(e))
  1920. {
  1921. if (first)
  1922. str.append("Graph(").append(graph.queryGraphId()).append("): [");
  1923. else
  1924. str.append(", ");
  1925. first = false;
  1926. ThorActivityKind kind = e.getKind();
  1927. str.append(activityKindStr(kind));
  1928. str.append("(").append(e.queryId()).append(")");
  1929. }
  1930. }
  1931. if (!first)
  1932. str.append("]");
  1933. Owned<IThorGraphIterator> childIter = graph.getChildGraphIterator();
  1934. ForEach(*childIter)
  1935. {
  1936. CGraphBase &childGraph = childIter->query();
  1937. getGlobals(childGraph, str);
  1938. }
  1939. return str;
  1940. }
  1941. void CGraphBase::executeChild(size32_t parentExtractSz, const byte *parentExtract)
  1942. {
  1943. assertex(localResults);
  1944. localResults->clear();
  1945. if (isGlobal()) // any slave
  1946. {
  1947. StringBuffer str("Global acts = ");
  1948. getGlobals(*this, str);
  1949. throw MakeGraphException(this, 0, "Global child graph? : %s", str.str());
  1950. }
  1951. doExecuteChild(parentExtractSz, parentExtract);
  1952. }
  1953. IThorResult *CGraphBase::getResult(unsigned id, bool distributed)
  1954. {
  1955. return localResults->getResult(id, distributed);
  1956. }
  1957. IThorResult *CGraphBase::getGraphLoopResult(unsigned id, bool distributed)
  1958. {
  1959. return graphLoopResults->getResult(id, distributed);
  1960. }
  1961. IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IThorGraphResults *results, IThorRowInterfaces *rowIf, ThorGraphResultType resultType, unsigned spillPriority)
  1962. {
  1963. return results->createResult(activity, id, rowIf, resultType, spillPriority);
  1964. }
  1965. IThorResult *CGraphBase::createResult(CActivityBase &activity, unsigned id, IThorRowInterfaces *rowIf, ThorGraphResultType resultType, unsigned spillPriority)
  1966. {
  1967. return localResults->createResult(activity, id, rowIf, resultType, spillPriority);
  1968. }
  1969. IThorResult *CGraphBase::createGraphLoopResult(CActivityBase &activity, IThorRowInterfaces *rowIf, ThorGraphResultType resultType, unsigned spillPriority)
  1970. {
  1971. return graphLoopResults->createResult(activity, rowIf, resultType, spillPriority);
  1972. }
  1973. // IEclGraphResults
  1974. void CGraphBase::getDictionaryResult(unsigned & count, const byte * * & ret, unsigned id)
  1975. {
  1976. Owned<IThorResult> result = getResult(id, true); // will get collated distributed result
  1977. result->getLinkedResult(count, ret);
  1978. }
  1979. void CGraphBase::getLinkedResult(unsigned & count, const byte * * & ret, unsigned id)
  1980. {
  1981. Owned<IThorResult> result = getResult(id, true); // will get collated distributed result
  1982. result->getLinkedResult(count, ret);
  1983. }
  1984. const void * CGraphBase::getLinkedRowResult(unsigned id)
  1985. {
  1986. Owned<IThorResult> result = getResult(id, true); // will get collated distributed result
  1987. return result->getLinkedRowResult();
  1988. }
  1989. // IThorChildGraph impl.
  1990. IEclGraphResults *CGraphBase::evaluate(unsigned _parentExtractSz, const byte *parentExtract)
  1991. {
  1992. CriticalBlock block(evaluateCrit);
  1993. localResults.setown(createThorGraphResults(xgmml->getPropInt("att[@name=\"_numResults\"]/@value", 0)));
  1994. parentExtractSz = _parentExtractSz;
  1995. executeChild(parentExtractSz, parentExtract);
  1996. return localResults.getClear();
  1997. }
  1998. static bool isLocalOnly(const CGraphElementBase &activity);
  1999. static bool isLocalOnly(const CGraphBase &graph) // checks all dependencies, if something needs to be global, whole body is forced to be execution sync.
  2000. {
  2001. if (0 == graph.activityCount())
  2002. {
  2003. Owned<IThorGraphIterator> iter = graph.getChildGraphIterator();
  2004. ForEach(*iter)
  2005. {
  2006. CGraphBase &childGraph = iter->query();
  2007. if (childGraph.isSink())
  2008. {
  2009. if (!isLocalOnly(childGraph))
  2010. return false;
  2011. }
  2012. }
  2013. }
  2014. else
  2015. {
  2016. if (graph.isGlobal())
  2017. return false;
  2018. Owned<IThorActivityIterator> sinkIter = graph.getAllSinkIterator();
  2019. ForEach(*sinkIter)
  2020. {
  2021. CGraphElementBase &sink = sinkIter->query();
  2022. if (!isLocalOnly(sink))
  2023. return false;
  2024. }
  2025. }
  2026. return true;
  2027. }
  2028. static bool isLocalOnly(const CGraphElementBase &activity)
  2029. {
  2030. Owned<IThorGraphDependencyIterator> deps = activity.getDependsIterator();
  2031. ForEach(*deps)
  2032. {
  2033. if (!isLocalOnly(*(deps->query().graph)))
  2034. return false;
  2035. }
  2036. StringBuffer match("edge[@target=\"");
  2037. match.append(activity.queryId()).append("\"]");
  2038. Owned<IPropertyTreeIterator> inputs = activity.queryOwner().queryXGMML().getElements(match.str());
  2039. ForEach(*inputs)
  2040. {
  2041. IPropertyTree &edge = inputs->query();
  2042. //Ignore edges that represent dependencies from parent activities to child activities.
  2043. if (edge.getPropBool("att[@name=\"_childGraph\"]/@value", false))
  2044. continue;
  2045. CGraphElementBase *sourceAct = activity.queryOwner().queryElement(edge.getPropInt("@source"));
  2046. if (!isLocalOnly(*sourceAct))
  2047. return false;
  2048. }
  2049. return true;
  2050. }
  2051. bool CGraphBase::isLocalOnly() const // checks all dependencies, if something needs to be global, whole body is forced to be execution sync.
  2052. {
  2053. if (-1 == localOnly)
  2054. localOnly = (int)::isLocalOnly(*this);
  2055. return 1==localOnly;
  2056. }
  2057. IThorGraphResults *CGraphBase::createThorGraphResults(unsigned num)
  2058. {
  2059. return new CThorGraphResults(num);
  2060. }
  2061. ////
  2062. void CGraphTempHandler::registerFile(const char *name, graph_id graphId, unsigned usageCount, bool temp, WUFileKind fileKind, StringArray *clusters)
  2063. {
  2064. assertex(temp);
  2065. LOG(MCdebugProgress, thorJob, "registerTmpFile name=%s, usageCount=%d", name, usageCount);
  2066. CriticalBlock b(crit);
  2067. if (tmpFiles.find(name))
  2068. throw MakeThorException(TE_FileAlreadyUsedAsTempFile, "File already used as temp file (%s)", name);
  2069. tmpFiles.replace(* new CFileUsageEntry(name, graphId, fileKind, usageCount));
  2070. }
  2071. void CGraphTempHandler::deregisterFile(const char *name, bool kept)
  2072. {
  2073. LOG(MCdebugProgress, thorJob, "deregisterTmpFile name=%s", name);
  2074. CriticalBlock b(crit);
  2075. CFileUsageEntry *fileUsage = tmpFiles.find(name);
  2076. if (!fileUsage)
  2077. {
  2078. if (errorOnMissing)
  2079. throw MakeThorException(TE_FileNotFound, "File not found (%s) deregistering tmp file", name);
  2080. return;
  2081. }
  2082. if (0 == fileUsage->queryUsage()) // marked 'not to be deleted' until workunit complete.
  2083. return;
  2084. else if (1 == fileUsage->queryUsage())
  2085. {
  2086. tmpFiles.remove(name);
  2087. try
  2088. {
  2089. if (!removeTemp(name))
  2090. LOG(MCwarning, unknownJob, "Failed to delete tmp file : %s (not found)", name);
  2091. }
  2092. catch (IException *e) { StringBuffer s("Failed to delete tmp file : "); FLLOG(MCwarning, thorJob, e, s.append(name).str()); }
  2093. }
  2094. else
  2095. fileUsage->decUsage();
  2096. }
  2097. void CGraphTempHandler::clearTemps()
  2098. {
  2099. CriticalBlock b(crit);
  2100. Owned<IFileUsageIterator> iter = getIterator();
  2101. ForEach(*iter)
  2102. {
  2103. CFileUsageEntry &entry = iter->query();
  2104. const char *tmpname = entry.queryName();
  2105. try
  2106. {
  2107. if (!removeTemp(tmpname))
  2108. LOG(MCwarning, thorJob, "Failed to delete tmp file : %s (not found)", tmpname);
  2109. }
  2110. catch (IException *e) { StringBuffer s("Failed to delete tmp file : "); FLLOG(MCwarning, thorJob, e, s.append(tmpname).str()); }
  2111. }
  2112. iter.clear();
  2113. tmpFiles.kill();
  2114. }
  2115. /////
  2116. class CGraphExecutor;
  2117. class CGraphExecutorGraphInfo : public CInterface
  2118. {
  2119. public:
  2120. CGraphExecutorGraphInfo(CGraphExecutor &_executor, CGraphBase *_subGraph, IGraphCallback &_callback, const byte *parentExtract, size32_t parentExtractSz) : executor(_executor), subGraph(_subGraph), callback(_callback)
  2121. {
  2122. parentExtractMb.append(parentExtractSz, parentExtract);
  2123. }
  2124. CGraphExecutor &executor;
  2125. IGraphCallback &callback;
  2126. Linked<CGraphBase> subGraph;
  2127. MemoryBuffer parentExtractMb;
  2128. };
  2129. class CGraphExecutor : implements IGraphExecutor, public CInterface
  2130. {
  2131. CJobChannel &jobChannel;
  2132. CJobBase &job;
  2133. CIArrayOf<CGraphExecutorGraphInfo> stack, running, toRun;
  2134. UnsignedArray seen;
  2135. bool stopped;
  2136. unsigned limit;
  2137. unsigned waitOnRunning;
  2138. CriticalSection crit;
  2139. Semaphore runningSem;
  2140. Owned<IThreadPool> graphPool;
  2141. class CGraphExecutorFactory : implements IThreadFactory, public CInterface
  2142. {
  2143. CGraphExecutor &executor;
  2144. public:
  2145. IMPLEMENT_IINTERFACE;
  2146. CGraphExecutorFactory(CGraphExecutor &_executor) : executor(_executor) { }
  2147. // IThreadFactory
  2148. virtual IPooledThread *createNew()
  2149. {
  2150. class CGraphExecutorThread : implements IPooledThread, public CInterface
  2151. {
  2152. Owned<CGraphExecutorGraphInfo> graphInfo;
  2153. public:
  2154. IMPLEMENT_IINTERFACE;
  2155. CGraphExecutorThread()
  2156. {
  2157. }
  2158. virtual void init(void *startInfo) override
  2159. {
  2160. graphInfo.setown((CGraphExecutorGraphInfo *)startInfo);
  2161. }
  2162. virtual void threadmain() override
  2163. {
  2164. for (;;)
  2165. {
  2166. Linked<CGraphBase> graph = graphInfo->subGraph;
  2167. Owned<IException> e;
  2168. try
  2169. {
  2170. PROGLOG("CGraphExecutor: Running graph, graphId=%" GIDPF "d", graph->queryGraphId());
  2171. graphInfo->callback.runSubgraph(*graph, graphInfo->parentExtractMb.length(), (const byte *)graphInfo->parentExtractMb.toByteArray());
  2172. }
  2173. catch (IException *_e)
  2174. {
  2175. e.setown(_e);
  2176. }
  2177. Owned<CGraphExecutorGraphInfo> nextGraphInfo;
  2178. try
  2179. {
  2180. nextGraphInfo.setown(graphInfo->executor.graphDone(*graphInfo, e));
  2181. }
  2182. catch (IException *e)
  2183. {
  2184. GraphPrintLog(graph, e, "graphDone");
  2185. e->Release();
  2186. }
  2187. graphInfo.clear(); // NB: at this point the graph will be destroyed
  2188. if (e)
  2189. throw e.getClear();
  2190. if (!nextGraphInfo)
  2191. return;
  2192. graphInfo.setown(nextGraphInfo.getClear());
  2193. }
  2194. }
  2195. virtual bool canReuse() const override { return true; }
  2196. virtual bool stop() override { return true; }
  2197. };
  2198. return new CGraphExecutorThread();
  2199. }
  2200. } *factory;
  2201. CGraphExecutorGraphInfo *findRunning(graph_id gid)
  2202. {
  2203. ForEachItemIn(r, running)
  2204. {
  2205. CGraphExecutorGraphInfo *graphInfo = &running.item(r);
  2206. if (gid == graphInfo->subGraph->queryGraphId())
  2207. return graphInfo;
  2208. }
  2209. return NULL;
  2210. }
  2211. public:
  2212. IMPLEMENT_IINTERFACE;
  2213. CGraphExecutor(CJobChannel &_jobChannel) : jobChannel(_jobChannel), job(_jobChannel.queryJob())
  2214. {
  2215. limit = (unsigned)job.getWorkUnitValueInt("concurrentSubGraphs", globals->getPropInt("@concurrentSubGraphs", 1));
  2216. PROGLOG("CGraphExecutor: limit = %d", limit);
  2217. waitOnRunning = 0;
  2218. stopped = false;
  2219. factory = new CGraphExecutorFactory(*this);
  2220. graphPool.setown(createThreadPool("CGraphExecutor pool", factory, &jobChannel, limit));
  2221. }
  2222. ~CGraphExecutor()
  2223. {
  2224. stopped = true;
  2225. graphPool->joinAll();
  2226. factory->Release();
  2227. }
  2228. CGraphExecutorGraphInfo *graphDone(CGraphExecutorGraphInfo &doneGraphInfo, IException *e)
  2229. {
  2230. CriticalBlock b(crit);
  2231. running.zap(doneGraphInfo);
  2232. if (waitOnRunning)
  2233. {
  2234. runningSem.signal(waitOnRunning);
  2235. waitOnRunning = 0;
  2236. }
  2237. if (e || job.queryAborted())
  2238. {
  2239. stopped = true;
  2240. stack.kill();
  2241. return NULL;
  2242. }
  2243. if (job.queryPausing())
  2244. stack.kill();
  2245. else if (stack.ordinality())
  2246. {
  2247. CICopyArrayOf<CGraphExecutorGraphInfo> toMove;
  2248. ForEachItemIn(s, stack)
  2249. {
  2250. bool dependenciesDone = true;
  2251. CGraphExecutorGraphInfo &graphInfo = stack.item(s);
  2252. ForEachItemIn (d, graphInfo.subGraph->dependentSubGraphs)
  2253. {
  2254. CGraphBase &subGraph = graphInfo.subGraph->dependentSubGraphs.item(d);
  2255. if (!subGraph.isComplete())
  2256. {
  2257. dependenciesDone = false;
  2258. break;
  2259. }
  2260. }
  2261. if (dependenciesDone)
  2262. {
  2263. graphInfo.subGraph->dependentSubGraphs.kill();
  2264. graphInfo.subGraph->prepare(graphInfo.parentExtractMb.length(), (const byte *)graphInfo.parentExtractMb.toByteArray(), true, true, true); // now existing deps done, maybe more to prepare
  2265. ForEachItemIn (d, graphInfo.subGraph->dependentSubGraphs)
  2266. {
  2267. CGraphBase &subGraph = graphInfo.subGraph->dependentSubGraphs.item(d);
  2268. if (!subGraph.isComplete())
  2269. {
  2270. dependenciesDone = false;
  2271. break;
  2272. }
  2273. }
  2274. if (dependenciesDone)
  2275. {
  2276. graphInfo.subGraph->dependentSubGraphs.kill(); // none to track anymore
  2277. toMove.append(graphInfo);
  2278. }
  2279. }
  2280. }
  2281. ForEachItemIn(m, toMove)
  2282. {
  2283. Linked<CGraphExecutorGraphInfo> graphInfo = &toMove.item(m);
  2284. stack.zap(*graphInfo);
  2285. toRun.add(*graphInfo.getClear(), 0);
  2286. }
  2287. }
  2288. job.markWuDirty();
  2289. PROGLOG("CGraphExecutor running=%d, waitingToRun=%d, dependentsWaiting=%d", running.ordinality(), toRun.ordinality(), stack.ordinality());
  2290. while (toRun.ordinality())
  2291. {
  2292. if (job.queryPausing())
  2293. return NULL;
  2294. Linked<CGraphExecutorGraphInfo> nextGraphInfo = &toRun.item(0);
  2295. toRun.remove(0);
  2296. if (!nextGraphInfo->subGraph->isComplete() && (NULL == findRunning(nextGraphInfo->subGraph->queryGraphId())))
  2297. {
  2298. running.append(*nextGraphInfo.getLink());
  2299. return nextGraphInfo.getClear();
  2300. }
  2301. }
  2302. return NULL;
  2303. }
  2304. // IGraphExecutor
  2305. virtual void add(CGraphBase *subGraph, IGraphCallback &callback, bool checkDependencies, size32_t parentExtractSz, const byte *parentExtract)
  2306. {
  2307. bool alreadyRunning;
  2308. {
  2309. CriticalBlock b(crit);
  2310. if (job.queryPausing())
  2311. return;
  2312. if (subGraph->isComplete())
  2313. return;
  2314. alreadyRunning = NULL != findRunning(subGraph->queryGraphId());
  2315. if (alreadyRunning)
  2316. ++waitOnRunning;
  2317. }
  2318. if (alreadyRunning)
  2319. {
  2320. for (;;)
  2321. {
  2322. PROGLOG("Waiting on subgraph %" GIDPF "d", subGraph->queryGraphId());
  2323. if (runningSem.wait(MEDIUMTIMEOUT) || job.queryAborted() || job.queryPausing())
  2324. break;
  2325. }
  2326. return;
  2327. }
  2328. else
  2329. {
  2330. CriticalBlock b(crit);
  2331. if (seen.contains(subGraph->queryGraphId()))
  2332. return; // already queued;
  2333. seen.append(subGraph->queryGraphId());
  2334. }
  2335. if (!subGraph->prepare(parentExtractSz, parentExtract, checkDependencies, true, true))
  2336. {
  2337. subGraph->setComplete();
  2338. return;
  2339. }
  2340. if (subGraph->dependentSubGraphs.ordinality())
  2341. {
  2342. bool dependenciesDone = true;
  2343. ForEachItemIn (d, subGraph->dependentSubGraphs)
  2344. {
  2345. CGraphBase &graph = subGraph->dependentSubGraphs.item(d);
  2346. if (!graph.isComplete())
  2347. {
  2348. dependenciesDone = false;
  2349. break;
  2350. }
  2351. }
  2352. if (dependenciesDone)
  2353. subGraph->dependentSubGraphs.kill(); // none to track anymore
  2354. }
  2355. Owned<CGraphExecutorGraphInfo> graphInfo = new CGraphExecutorGraphInfo(*this, subGraph, callback, parentExtract, parentExtractSz);
  2356. CriticalBlock b(crit);
  2357. if (0 == subGraph->dependentSubGraphs.ordinality())
  2358. {
  2359. if (running.ordinality()<limit)
  2360. {
  2361. running.append(*LINK(graphInfo));
  2362. PROGLOG("Add: Launching graph thread for graphId=%" GIDPF "d", subGraph->queryGraphId());
  2363. graphPool->start(graphInfo.getClear());
  2364. }
  2365. else
  2366. stack.add(*graphInfo.getClear(), 0); // push to front, no dependency, free to run next.
  2367. }
  2368. else
  2369. stack.append(*graphInfo.getClear()); // as dependencies finish, may move up the list
  2370. }
  2371. virtual IThreadPool &queryGraphPool() { return *graphPool; }
  2372. virtual void wait()
  2373. {
  2374. PROGLOG("CGraphExecutor exiting, waiting on graph pool");
  2375. graphPool->joinAll();
  2376. PROGLOG("CGraphExecutor graphPool finished");
  2377. }
  2378. };
  2379. ////
  2380. // IContextLogger
  2381. class CThorContextLogger : implements IContextLogger, public CSimpleInterface
  2382. {
  2383. CJobBase &job;
  2384. unsigned traceLevel;
  2385. StringAttr globalIdHeader;
  2386. StringAttr callerIdHeader;
  2387. StringAttr globalId;
  2388. StringAttr callerId;
  2389. StringBuffer localId;
  2390. public:
  2391. IMPLEMENT_IINTERFACE_USING(CSimpleInterface);
  2392. CThorContextLogger(CJobBase &_job) : job(_job)
  2393. {
  2394. traceLevel = 1;
  2395. if (globals->hasProp("@httpGlobalIdHeader"))
  2396. setHttpIdHeaders(globals->queryProp("@httpGlobalIdHeader"), globals->queryProp("@httpCallerIdHeader"));
  2397. }
  2398. virtual void CTXLOGva(const char *format, va_list args) const __attribute__((format(printf,2,0)))
  2399. {
  2400. StringBuffer ss;
  2401. ss.valist_appendf(format, args);
  2402. LOG(MCdebugProgress, thorJob, "%s", ss.str());
  2403. }
  2404. virtual void logOperatorExceptionVA(IException *E, const char *file, unsigned line, const char *format, va_list args) const __attribute__((format(printf,5,0)))
  2405. {
  2406. StringBuffer ss;
  2407. ss.append("ERROR");
  2408. if (E)
  2409. ss.append(": ").append(E->errorCode());
  2410. if (file)
  2411. ss.appendf(": %s(%d) ", file, line);
  2412. if (E)
  2413. E->errorMessage(ss.append(": "));
  2414. if (format)
  2415. ss.append(": ").valist_appendf(format, args);
  2416. LOG(MCoperatorProgress, thorJob, "%s", ss.str());
  2417. }
  2418. virtual void noteStatistic(StatisticKind kind, unsigned __int64 value) const
  2419. {
  2420. }
  2421. virtual void mergeStats(const CRuntimeStatisticCollection &from) const
  2422. {
  2423. }
  2424. virtual unsigned queryTraceLevel() const
  2425. {
  2426. return traceLevel;
  2427. }
  2428. virtual void setGlobalId(const char *id, SocketEndpoint &ep, unsigned pid) override
  2429. {
  2430. globalId.set(id);
  2431. appendGloballyUniqueId(localId.clear());
  2432. }
  2433. virtual void setCallerId(const char *id) override
  2434. {
  2435. callerId.set(id);
  2436. }
  2437. virtual const char *queryGlobalId() const
  2438. {
  2439. return globalId.get();
  2440. }
  2441. virtual const char *queryLocalId() const
  2442. {
  2443. return localId.str();
  2444. }
  2445. virtual const char *queryCallerId() const override
  2446. {
  2447. return callerId.str();
  2448. }
  2449. virtual void setHttpIdHeaders(const char *global, const char *caller)
  2450. {
  2451. if (global && *global)
  2452. globalIdHeader.set(global);
  2453. if (caller && *caller)
  2454. callerIdHeader.set(caller);
  2455. }
  2456. virtual const char *queryGlobalIdHttpHeader() const
  2457. {
  2458. return globalIdHeader.str();
  2459. }
  2460. virtual const char *queryCallerIdHttpHeader() const
  2461. {
  2462. return callerIdHeader.str();
  2463. }
  2464. };
  2465. ////
  2466. CJobBase::CJobBase(ILoadedDllEntry *_querySo, const char *_graphName) : querySo(_querySo), graphName(_graphName)
  2467. {
  2468. maxDiskUsage = diskUsage = 0;
  2469. dirty = true;
  2470. aborted = false;
  2471. globalMemoryMB = globals->getPropInt("@globalMemorySize"); // in MB
  2472. channelsPerSlave = globals->getPropInt("@channelsPerSlave", 1);
  2473. numChannels = channelsPerSlave;
  2474. pluginMap = new SafePluginMap(&pluginCtx, true);
  2475. // JCSMORE - Will pass down at job creation time...
  2476. jobGroup.set(&::queryClusterGroup());
  2477. slaveGroup.setown(jobGroup->remove(0));
  2478. nodeGroup.set(&queryNodeGroup());
  2479. myNodeRank = nodeGroup->rank(::queryMyNode());
  2480. unsigned channelsPerSlave = globals->getPropInt("@channelsPerSlave", 1);
  2481. jobChannelSlaveNumbers.allocateN(channelsPerSlave, true); // filled when channels are added.
  2482. jobSlaveChannelNum.allocateN(querySlaves()); // filled when channels are added.
  2483. for (unsigned s=0; s<querySlaves(); s++)
  2484. jobSlaveChannelNum[s] = NotFound;
  2485. StringBuffer wuXML;
  2486. if (!getEmbeddedWorkUnitXML(querySo, wuXML))
  2487. throw MakeStringException(0, "Failed to locate workunit info in query : %s", querySo->queryName());
  2488. Owned<ILocalWorkUnit> localWU = createLocalWorkUnit(wuXML);
  2489. Owned<IConstWUGraph> graph = localWU->getGraph(graphName);
  2490. graphXGMML.setown(graph->getXGMMLTree(false));
  2491. if (!graphXGMML)
  2492. throwUnexpected();
  2493. }
  2494. void CJobBase::init()
  2495. {
  2496. StringBuffer tmp;
  2497. tmp.append(wuid);
  2498. tmp.append(graphName);
  2499. key.set(tmp.str());
  2500. StringBuffer user;
  2501. extractFromWorkunitDAToken(token.str(), nullptr, &user, nullptr);
  2502. userDesc = createUserDescriptor();
  2503. userDesc->set(user.str(), token.str());//use workunit token as password
  2504. forceLogGraphIdMin = (graph_id)getWorkUnitValueInt("forceLogGraphIdMin", 0);
  2505. forceLogGraphIdMax = (graph_id)getWorkUnitValueInt("forceLogGraphIdMax", 0);
  2506. logctx.setown(new CThorContextLogger(*this));
  2507. // global setting default on, can be overridden by #option
  2508. timeActivities = 0 != getWorkUnitValueInt("timeActivities", globals->getPropBool("@timeActivities", true));
  2509. maxActivityCores = (unsigned)getWorkUnitValueInt("maxActivityCores", 0); // NB: 0 means system decides
  2510. if (0 == maxActivityCores)
  2511. maxActivityCores = getAffinityCpus();
  2512. pausing = false;
  2513. resumed = false;
  2514. crcChecking = 0 != getWorkUnitValueInt("THOR_ROWCRC", globals->getPropBool("@THOR_ROWCRC", false));
  2515. usePackedAllocator = 0 != getWorkUnitValueInt("THOR_PACKEDALLOCATOR", globals->getPropBool("@THOR_PACKEDALLOCATOR", true));
  2516. memorySpillAtPercentage = (unsigned)getWorkUnitValueInt("memorySpillAt", globals->getPropInt("@memorySpillAt", 80));
  2517. sharedMemoryLimitPercentage = (unsigned)getWorkUnitValueInt("globalMemoryLimitPC", globals->getPropInt("@sharedMemoryLimit", 90));
  2518. sharedMemoryMB = globalMemoryMB*sharedMemoryLimitPercentage/100;
  2519. failOnLeaks = getOptBool("failOnLeaks");
  2520. maxLfnBlockTimeMins = getOptInt(THOROPT_MAXLFN_BLOCKTIME_MINS, DEFAULT_MAXLFN_BLOCKTIME_MINS);
  2521. soapTraceLevel = getOptInt("soapTraceLevel", 1);
  2522. PROGLOG("Global memory size = %d MB, shared memory = %d%%, memory spill at = %d%%", globalMemoryMB, sharedMemoryLimitPercentage, memorySpillAtPercentage);
  2523. StringBuffer tracing("maxActivityCores = ");
  2524. if (maxActivityCores)
  2525. tracing.append(maxActivityCores);
  2526. else
  2527. tracing.append("[unbound]");
  2528. PROGLOG("%s", tracing.str());
  2529. }
  2530. void CJobBase::beforeDispose()
  2531. {
  2532. endJob();
  2533. }
  2534. CJobChannel &CJobBase::queryJobChannel(unsigned c) const
  2535. {
  2536. return jobChannels.item(c);
  2537. }
  2538. CActivityBase &CJobBase::queryChannelActivity(unsigned c, graph_id gid, activity_id id) const
  2539. {
  2540. CJobChannel &channel = queryJobChannel(c);
  2541. Owned<CGraphBase> graph = channel.getGraph(gid);
  2542. dbgassertex(graph);
  2543. CGraphElementBase *container = graph->queryElement(id);
  2544. dbgassertex(container);
  2545. return *container->queryActivity();
  2546. }
  2547. void CJobBase::startJob()
  2548. {
  2549. LOG(MCdebugProgress, thorJob, "New Graph started : %s", graphName.get());
  2550. ClearTempDirs();
  2551. perfmonhook.setown(createThorMemStatsPerfMonHook(*this, getOptInt(THOROPT_MAX_KERNLOG, 3)));
  2552. setPerformanceMonitorHook(perfmonhook);
  2553. PrintMemoryStatusLog();
  2554. logDiskSpace();
  2555. unsigned keyNodeCacheMB = (unsigned)getWorkUnitValueInt("keyNodeCacheMB", DEFAULT_KEYNODECACHEMB * queryJobChannels());
  2556. unsigned keyLeafCacheMB = (unsigned)getWorkUnitValueInt("keyLeafCacheMB", DEFAULT_KEYLEAFCACHEMB * queryJobChannels());
  2557. unsigned keyBlobCacheMB = (unsigned)getWorkUnitValueInt("keyBlobCacheMB", DEFAULT_KEYBLOBCACHEMB * queryJobChannels());
  2558. setNodeCacheMem(keyNodeCacheMB * 0x100000);
  2559. setLeafCacheMem(keyLeafCacheMB * 0x100000);
  2560. setBlobCacheMem(keyBlobCacheMB * 0x100000);
  2561. PROGLOG("Key node caching setting: node=%u MB, leaf=%u MB, blob=%u MB", keyNodeCacheMB, keyLeafCacheMB, keyBlobCacheMB);
  2562. unsigned keyFileCacheLimit = (unsigned)getWorkUnitValueInt("keyFileCacheLimit", 0);
  2563. if (!keyFileCacheLimit)
  2564. keyFileCacheLimit = (querySlaves()+1)*2;
  2565. setKeyIndexCacheSize(keyFileCacheLimit);
  2566. PROGLOG("Key file cache size set to: %d", keyFileCacheLimit);
  2567. if (getOptBool("dumpStacks")) // mainly as an example of printAllStacks() usage
  2568. {
  2569. StringBuffer output;
  2570. if (getAllStacks(output))
  2571. {
  2572. IERRLOG("%s", output.str());
  2573. }
  2574. else
  2575. IWARNLOG("Failed to capture process stacks: %s", output.str());
  2576. }
  2577. }
  2578. void CJobBase::endJob()
  2579. {
  2580. if (jobEnded)
  2581. return;
  2582. jobEnded = true;
  2583. setPerformanceMonitorHook(nullptr);
  2584. LOG(MCdebugProgress, thorJob, "Job ended : %s", graphName.get());
  2585. clearKeyStoreCache(true);
  2586. PrintMemoryStatusLog();
  2587. Owned<IMultiException> exceptions;
  2588. ForEachItemIn(c, jobChannels)
  2589. {
  2590. try
  2591. {
  2592. jobChannels.item(c).clean();
  2593. }
  2594. catch (IException *e)
  2595. {
  2596. if (!exceptions)
  2597. exceptions.setown(makeMultiException());
  2598. exceptions->append(*LINK(e));
  2599. }
  2600. }
  2601. try
  2602. {
  2603. jobChannels.kill(); // avoiding circular references. Kill before other CJobBase components are destroyed that channels reference.
  2604. ::Release(userDesc);
  2605. callThreadTerminationHooks(); // must call any installed thread termination functions, before unloading plugins
  2606. ::Release(pluginMap);
  2607. traceMemUsage();
  2608. if (numChannels > 1) // if only 1 - then channel allocator is same as sharedAllocator, leaks will be reported by the single channel
  2609. checkAndReportLeaks(sharedAllocator->queryRowManager());
  2610. }
  2611. catch (IException *e)
  2612. {
  2613. if (!exceptions)
  2614. exceptions.setown(makeMultiException());
  2615. exceptions->append(*LINK(e));
  2616. }
  2617. if (exceptions && exceptions->ordinality())
  2618. throw exceptions.getClear();
  2619. }
  2620. void CJobBase::checkAndReportLeaks(roxiemem::IRowManager *rowManager)
  2621. {
  2622. if (!failOnLeaks) // NB: leaks reported by row manager destructor anyway
  2623. return;
  2624. if (rowManager->allocated())
  2625. {
  2626. rowManager->reportLeaks();
  2627. throw MakeThorException(TE_RowLeaksDetected, "Row leaks detected");
  2628. }
  2629. }
  2630. bool CJobBase::queryForceLogging(graph_id graphId, bool def) const
  2631. {
  2632. // JCSMORE, could add comma separated range, e.g. 1-5,10-12
  2633. if ((graphId >= forceLogGraphIdMin) && (graphId <= forceLogGraphIdMax))
  2634. return true;
  2635. return def;
  2636. }
  2637. void CJobBase::addSubGraph(IPropertyTree &xgmml)
  2638. {
  2639. CriticalBlock b(crit);
  2640. for (unsigned c=0; c<queryJobChannels(); c++)
  2641. {
  2642. CJobChannel &jobChannel = queryJobChannel(c);
  2643. Owned<CGraphBase> subGraph = jobChannel.createGraph();
  2644. subGraph->createFromXGMML(&xgmml, NULL, NULL, NULL, jobChannel.queryAllGraphs());
  2645. jobChannel.addSubGraph(*subGraph.getClear());
  2646. }
  2647. }
  2648. void CJobBase::addDependencies(IPropertyTree *xgmml, bool failIfMissing)
  2649. {
  2650. for (unsigned c=0; c<queryJobChannels(); c++)
  2651. {
  2652. CJobChannel &jobChannel = queryJobChannel(c);
  2653. jobChannel.addDependencies(xgmml, failIfMissing);
  2654. }
  2655. }
  2656. bool CJobBase::queryUseCheckpoints() const
  2657. {
  2658. return globals->getPropBool("@checkPointRecovery") || 0 != getWorkUnitValueInt("checkPointRecovery", 0);
  2659. }
  2660. void CJobBase::abort(IException *e)
  2661. {
  2662. aborted = true;
  2663. for (unsigned c=0; c<queryJobChannels(); c++)
  2664. {
  2665. CJobChannel &jobChannel = queryJobChannel(c);
  2666. jobChannel.abort(e);
  2667. }
  2668. }
  2669. void CJobBase::increase(offset_t usage, const char *key)
  2670. {
  2671. diskUsage += usage;
  2672. if (diskUsage > maxDiskUsage) maxDiskUsage = diskUsage;
  2673. }
  2674. void CJobBase::decrease(offset_t usage, const char *key)
  2675. {
  2676. diskUsage -= usage;
  2677. }
  2678. // these getX methods for property in workunit settings, then global setting, defaulting to provided 'dft' if not present
  2679. StringBuffer &CJobBase::getOpt(const char *opt, StringBuffer &out)
  2680. {
  2681. if (!opt || !*opt)
  2682. return out; // probably error
  2683. VStringBuffer gOpt("Debug/@%s", opt);
  2684. getWorkUnitValue(opt, out);
  2685. if (0 == out.length())
  2686. globals->getProp(gOpt, out);
  2687. return out;
  2688. }
  2689. bool CJobBase::getOptBool(const char *opt, bool dft)
  2690. {
  2691. if (!opt || !*opt)
  2692. return dft; // probably error
  2693. VStringBuffer gOpt("Debug/@%s", opt);
  2694. return getWorkUnitValueBool(opt, globals->getPropBool(gOpt, dft));
  2695. }
  2696. int CJobBase::getOptInt(const char *opt, int dft)
  2697. {
  2698. if (!opt || !*opt)
  2699. return dft; // probably error
  2700. VStringBuffer gOpt("Debug/@%s", opt);
  2701. return (int)getWorkUnitValueInt(opt, globals->getPropInt(gOpt, dft));
  2702. }
  2703. __int64 CJobBase::getOptInt64(const char *opt, __int64 dft)
  2704. {
  2705. if (!opt || !*opt)
  2706. return dft; // probably error
  2707. VStringBuffer gOpt("Debug/@%s", opt);
  2708. return getWorkUnitValueInt(opt, globals->getPropInt64(gOpt, dft));
  2709. }
  2710. IThorAllocator *CJobBase::getThorAllocator(unsigned channel)
  2711. {
  2712. return sharedAllocator.getLink();
  2713. }
  2714. /// CJobChannel
  2715. CJobChannel::CJobChannel(CJobBase &_job, IMPServer *_mpServer, unsigned _channel)
  2716. : job(_job), mpServer(_mpServer), channel(_channel)
  2717. {
  2718. aborted = false;
  2719. thorAllocator.setown(job.getThorAllocator(channel));
  2720. jobComm.setown(mpServer->createCommunicator(&job.queryJobGroup()));
  2721. myrank = job.queryJobGroup().rank(queryMyNode());
  2722. graphExecutor.setown(new CGraphExecutor(*this));
  2723. }
  2724. CJobChannel::~CJobChannel()
  2725. {
  2726. if (!cleaned)
  2727. clean();
  2728. }
  2729. INode *CJobChannel::queryMyNode()
  2730. {
  2731. return mpServer->queryMyNode();
  2732. }
  2733. void CJobChannel::wait()
  2734. {
  2735. if (graphExecutor)
  2736. graphExecutor->wait();
  2737. }
  2738. ICodeContext &CJobChannel::queryCodeContext() const
  2739. {
  2740. return *codeCtx;
  2741. }
  2742. ICodeContext &CJobChannel::querySharedMemCodeContext() const
  2743. {
  2744. return *sharedMemCodeCtx;
  2745. }
  2746. mptag_t CJobChannel::deserializeMPTag(MemoryBuffer &mb)
  2747. {
  2748. mptag_t tag;
  2749. deserializeMPtag(mb, tag);
  2750. if (TAG_NULL != tag)
  2751. {
  2752. PROGLOG("deserializeMPTag: tag = %d", (int)tag);
  2753. jobComm->flush(tag);
  2754. }
  2755. return tag;
  2756. }
  2757. IEngineRowAllocator *CJobChannel::getRowAllocator(IOutputMetaData * meta, activity_id activityId, roxiemem::RoxieHeapFlags flags) const
  2758. {
  2759. return thorAllocator->getRowAllocator(meta, activityId, flags);
  2760. }
  2761. roxiemem::IRowManager *CJobChannel::queryRowManager() const
  2762. {
  2763. return thorAllocator->queryRowManager();
  2764. }
  2765. void CJobChannel::addDependencies(IPropertyTree *xgmml, bool failIfMissing)
  2766. {
  2767. ::addDependencies(xgmml, failIfMissing, allGraphs);
  2768. }
  2769. IThorGraphIterator *CJobChannel::getSubGraphs()
  2770. {
  2771. CriticalBlock b(crit);
  2772. return new CGraphTableIterator(subGraphs);
  2773. }
  2774. void CJobChannel::clean()
  2775. {
  2776. if (cleaned)
  2777. return;
  2778. cleaned = true;
  2779. wait();
  2780. queryRowManager()->reportMemoryUsage(false);
  2781. PROGLOG("CJobBase resetting memory manager");
  2782. if (graphExecutor)
  2783. {
  2784. graphExecutor->queryGraphPool().stopAll();
  2785. graphExecutor.clear();
  2786. }
  2787. subGraphs.kill();
  2788. job.checkAndReportLeaks(thorAllocator->queryRowManager());
  2789. thorAllocator.clear();
  2790. codeCtx.clear();
  2791. }
  2792. void CJobChannel::startGraph(CGraphBase &graph, bool checkDependencies, size32_t parentExtractSize, const byte *parentExtract)
  2793. {
  2794. graphExecutor->add(&graph, *this, checkDependencies, parentExtractSize, parentExtract);
  2795. }
  2796. IThorResult *CJobChannel::getOwnedResult(graph_id gid, activity_id ownerId, unsigned resultId)
  2797. {
  2798. Owned<CGraphBase> graph = getGraph(gid);
  2799. if (!graph)
  2800. {
  2801. Owned<IThorException> e = MakeThorException(0, "getOwnedResult: graph not found");
  2802. e->setGraphInfo(queryJob().queryGraphName(), gid);
  2803. throw e.getClear();
  2804. }
  2805. Owned<IThorResult> result;
  2806. if (ownerId)
  2807. {
  2808. CGraphElementBase *container = graph->queryElement(ownerId);
  2809. assertex(container);
  2810. CActivityBase *activity = container->queryActivity();
  2811. IThorGraphResults *results = activity->queryResults();
  2812. if (!results)
  2813. throw MakeGraphException(graph, 0, "GraphGetResult: no results created (requesting: %d)", resultId);
  2814. result.setown(activity->queryResults()->getResult(resultId));
  2815. }
  2816. else
  2817. result.setown(graph->getResult(resultId));
  2818. if (!result)
  2819. throw MakeGraphException(graph, 0, "GraphGetResult: result not found: %d", resultId);
  2820. return result.getClear();
  2821. }
  2822. void CJobChannel::abort(IException *e)
  2823. {
  2824. aborted = true;
  2825. Owned<IThorGraphIterator> iter = getSubGraphs();
  2826. ForEach (*iter)
  2827. {
  2828. CGraphBase &graph = iter->query();
  2829. graph.abort(e);
  2830. }
  2831. }
  2832. // IGraphCallback
  2833. void CJobChannel::runSubgraph(CGraphBase &graph, size32_t parentExtractSz, const byte *parentExtract)
  2834. {
  2835. graph.executeSubGraph(parentExtractSz, parentExtract);
  2836. }
  2837. static IThorResource *iThorResource = NULL;
  2838. void setIThorResource(IThorResource &r)
  2839. {
  2840. iThorResource = &r;
  2841. }
  2842. IThorResource &queryThor()
  2843. {
  2844. return *iThorResource;
  2845. }
  2846. //
  2847. //
  2848. //
  2849. //
  2850. CActivityBase::CActivityBase(CGraphElementBase *_container) : container(*_container), timeActivities(_container->queryJob().queryTimeActivities())
  2851. {
  2852. mpTag = TAG_NULL;
  2853. abortSoon = receiving = cancelledReceive = initialized = reInit = false;
  2854. baseHelper.set(container.queryHelper());
  2855. parentExtractSz = 0;
  2856. parentExtract = NULL;
  2857. defaultRoxieMemHeapFlags = (roxiemem::RoxieHeapFlags)container.getOptInt("heapflags", defaultHeapFlags);
  2858. if (container.queryJob().queryUsePackedAllocators())
  2859. defaultRoxieMemHeapFlags = (roxiemem::RoxieHeapFlags)(defaultRoxieMemHeapFlags | roxiemem::RHFpacked);
  2860. }
  2861. CActivityBase::~CActivityBase()
  2862. {
  2863. }
  2864. void CActivityBase::abort()
  2865. {
  2866. if (!abortSoon) ActPrintLog("Abort condition set");
  2867. abortSoon = true;
  2868. }
  2869. void CActivityBase::kill()
  2870. {
  2871. ownedResults.clear();
  2872. }
  2873. bool CActivityBase::appendRowXml(StringBuffer & target, IOutputMetaData & meta, const void * row) const
  2874. {
  2875. if (!meta.hasXML())
  2876. {
  2877. target.append("<xml-unavailable/>");
  2878. return false;
  2879. }
  2880. try
  2881. {
  2882. CommonXmlWriter xmlWrite(XWFnoindent);
  2883. meta.toXML((byte *) row, xmlWrite);
  2884. target.append(xmlWrite.str());
  2885. return true;
  2886. }
  2887. catch (IException * e)
  2888. {
  2889. e->Release();
  2890. target.append("<invalid-row/>");
  2891. return false;
  2892. }
  2893. }
  2894. void CActivityBase::logRow(const char * prefix, IOutputMetaData & meta, const void * row) const
  2895. {
  2896. bool blindLogging = false; // MORE: should check a workunit/global option
  2897. if (meta.hasXML() && !blindLogging)
  2898. {
  2899. StringBuffer xml;
  2900. appendRowXml(xml, meta, row);
  2901. ActPrintLog("%s: %s", prefix, xml.str());
  2902. }
  2903. }
  2904. void CActivityBase::ActPrintLog(const char *format, ...) const
  2905. {
  2906. va_list args;
  2907. va_start(args, format);
  2908. ::ActPrintLogArgs(&queryContainer(), thorlog_null, MCdebugProgress, format, args);
  2909. va_end(args);
  2910. }
  2911. void CActivityBase::ActPrintLog(IException *e, const char *format, ...) const
  2912. {
  2913. va_list args;
  2914. va_start(args, format);
  2915. ::ActPrintLogArgs(&queryContainer(), e, thorlog_all, MCexception(e), format, args);
  2916. va_end(args);
  2917. }
  2918. void CActivityBase::ActPrintLog(IException *e) const
  2919. {
  2920. ActPrintLog(e, "%s", "");
  2921. }
  2922. IThorRowInterfaces * CActivityBase::createRowInterfaces(IOutputMetaData * meta, byte seq)
  2923. {
  2924. activity_id id = createCompoundActSeqId(queryId(), seq);
  2925. return createThorRowInterfaces(queryRowManager(), meta, id, queryHeapFlags(), queryCodeContext());
  2926. }
  2927. IThorRowInterfaces * CActivityBase::createRowInterfaces(IOutputMetaData * meta, roxiemem::RoxieHeapFlags heapFlags, byte seq)
  2928. {
  2929. activity_id id = createCompoundActSeqId(queryId(), seq);
  2930. return createThorRowInterfaces(queryRowManager(), meta, id, heapFlags, queryCodeContext());
  2931. }
  2932. bool CActivityBase::fireException(IException *e)
  2933. {
  2934. Owned<IThorException> _te;
  2935. IThorException *te = QUERYINTERFACE(e, IThorException);
  2936. if (te)
  2937. {
  2938. if (!te->queryActivityId())
  2939. setExceptionActivityInfo(container, te);
  2940. }
  2941. else
  2942. {
  2943. te = MakeActivityException(this, e);
  2944. te->setAudience(e->errorAudience());
  2945. _te.setown(te);
  2946. }
  2947. return container.queryOwner().fireException(te);
  2948. }
  2949. void CActivityBase::processAndThrowOwnedException(IException * _e)
  2950. {
  2951. IThorException *e = QUERYINTERFACE(_e, IThorException);
  2952. if (e)
  2953. {
  2954. if (!e->queryActivityId())
  2955. setExceptionActivityInfo(container, e);
  2956. }
  2957. else
  2958. {
  2959. e = MakeActivityException(this, _e);
  2960. _e->Release();
  2961. }
  2962. throw e;
  2963. }
  2964. IEngineRowAllocator * CActivityBase::queryRowAllocator()
  2965. {
  2966. if (CABallocatorlock.lock()) {
  2967. if (!rowAllocator)
  2968. {
  2969. roxiemem::RoxieHeapFlags heapFlags = queryHeapFlags();
  2970. rowAllocator.setown(getRowAllocator(queryRowMetaData(), heapFlags));
  2971. }
  2972. CABallocatorlock.unlock();
  2973. }
  2974. return rowAllocator;
  2975. }
  2976. IOutputRowSerializer * CActivityBase::queryRowSerializer()
  2977. {
  2978. if (CABserializerlock.lock()) {
  2979. if (!rowSerializer)
  2980. rowSerializer.setown(queryRowMetaData()->createDiskSerializer(queryCodeContext(),queryId()));
  2981. CABserializerlock.unlock();
  2982. }
  2983. return rowSerializer;
  2984. }
  2985. IOutputRowDeserializer * CActivityBase::queryRowDeserializer()
  2986. {
  2987. if (CABdeserializerlock.lock()) {
  2988. if (!rowDeserializer)
  2989. rowDeserializer.setown(queryRowMetaData()->createDiskDeserializer(queryCodeContext(),queryId()));
  2990. CABdeserializerlock.unlock();
  2991. }
  2992. return rowDeserializer;
  2993. }
  2994. IThorRowInterfaces *CActivityBase::getRowInterfaces()
  2995. {
  2996. // create an independent instance, to avoid circular link dependency problems
  2997. return createThorRowInterfaces(queryRowManager(), queryRowMetaData(), container.queryId(), queryHeapFlags(), queryCodeContext());
  2998. }
  2999. IEngineRowAllocator *CActivityBase::getRowAllocator(IOutputMetaData * meta, roxiemem::RoxieHeapFlags flags, byte seq) const
  3000. {
  3001. activity_id actId = createCompoundActSeqId(queryId(), seq);
  3002. return queryJobChannel().getRowAllocator(meta, actId, flags);
  3003. }
  3004. bool CActivityBase::receiveMsg(ICommunicator &comm, CMessageBuffer &mb, const rank_t rank, const mptag_t mpTag, rank_t *sender, unsigned timeout)
  3005. {
  3006. BooleanOnOff onOff(receiving);
  3007. CTimeMon t(timeout);
  3008. unsigned remaining = timeout;
  3009. // check 'cancelledReceive' every 10 secs
  3010. while (!cancelledReceive && ((MP_WAIT_FOREVER==timeout) || !t.timedout(&remaining)))
  3011. {
  3012. if (comm.recv(mb, rank, mpTag, sender, remaining>10000?10000:remaining))
  3013. return true;
  3014. }
  3015. return false;
  3016. }
  3017. bool CActivityBase::receiveMsg(CMessageBuffer &mb, const rank_t rank, const mptag_t mpTag, rank_t *sender, unsigned timeout)
  3018. {
  3019. return receiveMsg(queryJobChannel().queryJobComm(), mb, rank, mpTag, sender, timeout);
  3020. }
  3021. void CActivityBase::cancelReceiveMsg(ICommunicator &comm, const rank_t rank, const mptag_t mpTag)
  3022. {
  3023. cancelledReceive = true;
  3024. if (receiving)
  3025. comm.cancel(rank, mpTag);
  3026. }
  3027. void CActivityBase::cancelReceiveMsg(const rank_t rank, const mptag_t mpTag)
  3028. {
  3029. cancelReceiveMsg(queryJobChannel().queryJobComm(), rank, mpTag);
  3030. }