operation.cpp 107 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000200120022003200420052006200720082009201020112012201320142015201620172018201920202021202220232024202520262027202820292030203120322033203420352036203720382039204020412042204320442045204620472048204920502051205220532054205520562057205820592060206120622063206420652066206720682069207020712072207320742075207620772078207920802081208220832084208520862087208820892090209120922093209420952096209720982099210021012102210321042105210621072108210921102111211221132114211521162117211821192120212121222123212421252126212721282129213021312132213321342135213621372138213921402141214221432144214521462147214821492150215121522153215421552156215721582159216021612162216321642165216621672168216921702171217221732174217521762177217821792180218121822183218421852186218721882189219021912192219321942195219621972198219922002201220222032204220522062207220822092210221122122213221422152216221722182219222022212222222322242225222622272228222922302231223222332234223522362237223822392240224122422243224422452246224722482249225022512252225322542255225622572258225922602261226222632264226522662267226822692270227122722273227422752276227722782279228022812282228322842285228622872288228922902291229222932294229522962297229822992300230123022303230423052306230723082309231023112312231323142315231623172318231923202321232223232324232523262327232823292330233123322333233423352336233723382339234023412342234323442345234623472348234923502351235223532354235523562357235823592360236123622363236423652366236723682369237023712372237323742375237623772378237923802381238223832384238523862387238823892390239123922393239423952396239723982399240024012402240324042405240624072408240924102411241224132414241524162417241824192420242124222423242424252426242724282429243024312432243324342435243624372438243924402441244224432444244524462447244824492450245124522453245424552456245724582459246024612462246324642465246624672468246924702471247224732474247524762477247824792480248124822483248424852486248724882489249024912492249324942495249624972498249925002501250225032504250525062507250825092510251125122513251425152516251725182519252025212522252325242525252625272528252925302531253225332534253525362537253825392540254125422543254425452546254725482549255025512552255325542555255625572558255925602561256225632564256525662567256825692570257125722573257425752576257725782579258025812582258325842585258625872588258925902591259225932594259525962597259825992600260126022603260426052606260726082609261026112612261326142615261626172618261926202621262226232624262526262627262826292630263126322633263426352636263726382639264026412642264326442645264626472648264926502651265226532654265526562657265826592660266126622663266426652666266726682669267026712672267326742675267626772678267926802681268226832684268526862687268826892690269126922693269426952696269726982699270027012702270327042705270627072708270927102711271227132714271527162717271827192720272127222723272427252726272727282729273027312732273327342735273627372738273927402741274227432744274527462747274827492750275127522753275427552756275727582759276027612762276327642765276627672768276927702771277227732774277527762777277827792780278127822783278427852786278727882789279027912792279327942795279627972798279928002801280228032804280528062807280828092810281128122813281428152816281728182819282028212822282328242825282628272828282928302831283228332834283528362837283828392840284128422843284428452846284728482849285028512852285328542855285628572858285928602861286228632864286528662867286828692870287128722873287428752876287728782879288028812882288328842885288628872888288928902891289228932894289528962897289828992900290129022903290429052906290729082909291029112912291329142915291629172918291929202921292229232924292529262927292829292930293129322933293429352936293729382939294029412942294329442945294629472948294929502951295229532954295529562957295829592960296129622963296429652966296729682969297029712972297329742975297629772978297929802981298229832984298529862987298829892990299129922993299429952996299729982999300030013002300330043005300630073008300930103011301230133014301530163017301830193020302130223023302430253026302730283029303030313032303330343035303630373038303930403041304230433044304530463047304830493050305130523053305430553056305730583059306030613062306330643065306630673068306930703071307230733074307530763077307830793080308130823083308430853086308730883089309030913092309330943095309630973098309931003101310231033104310531063107310831093110311131123113311431153116311731183119312031213122312331243125312631273128312931303131313231333134313531363137313831393140314131423143314431453146314731483149315031513152315331543155315631573158315931603161
  1. #include "operation.h"
  2. #include "abortable_registry.h"
  3. #include "client.h"
  4. #include "operation_helpers.h"
  5. #include "operation_tracker.h"
  6. #include "prepare_operation.h"
  7. #include "skiff.h"
  8. #include "structured_table_formats.h"
  9. #include "yt_poller.h"
  10. #include <yt/cpp/mapreduce/common/helpers.h>
  11. #include <yt/cpp/mapreduce/common/retry_lib.h>
  12. #include <yt/cpp/mapreduce/common/retry_request.h>
  13. #include <yt/cpp/mapreduce/common/wait_proxy.h>
  14. #include <yt/cpp/mapreduce/interface/config.h>
  15. #include <yt/cpp/mapreduce/interface/errors.h>
  16. #include <yt/cpp/mapreduce/interface/fluent.h>
  17. #include <yt/cpp/mapreduce/interface/format.h>
  18. #include <yt/cpp/mapreduce/interface/job_statistics.h>
  19. #include <yt/cpp/mapreduce/interface/protobuf_format.h>
  20. #include <yt/cpp/mapreduce/interface/logging/yt_log.h>
  21. #include <yt/cpp/mapreduce/io/job_reader.h>
  22. #include <yt/cpp/mapreduce/io/job_writer.h>
  23. #include <yt/cpp/mapreduce/io/yamr_table_reader.h>
  24. #include <yt/cpp/mapreduce/io/yamr_table_writer.h>
  25. #include <yt/cpp/mapreduce/io/node_table_reader.h>
  26. #include <yt/cpp/mapreduce/io/node_table_writer.h>
  27. #include <yt/cpp/mapreduce/io/proto_table_reader.h>
  28. #include <yt/cpp/mapreduce/io/proto_table_writer.h>
  29. #include <yt/cpp/mapreduce/io/proto_helpers.h>
  30. #include <yt/cpp/mapreduce/io/skiff_table_reader.h>
  31. #include <yt/cpp/mapreduce/http_client/raw_requests.h>
  32. #include <library/cpp/yson/node/serialize.h>
  33. #include <util/generic/hash_set.h>
  34. #include <util/string/builder.h>
  35. #include <util/string/cast.h>
  36. #include <util/system/thread.h>
  37. #include <util/system/env.h>
  38. #include <util/system/fs.h>
  39. namespace NYT {
  40. namespace NDetail {
  41. using ::ToString;
  42. ////////////////////////////////////////////////////////////////////////////////
  43. static const ui64 DefaultExrtaTmpfsSize = 1024LL * 1024LL;
  44. ////////////////////////////////////////////////////////////////////////////////
  45. namespace {
  46. ////////////////////////////////////////////////////////////////////////////////
  47. struct TMapReduceOperationIo
  48. {
  49. TVector<TRichYPath> Inputs;
  50. TVector<TRichYPath> MapOutputs;
  51. TVector<TRichYPath> Outputs;
  52. TMaybe<TFormat> MapperInputFormat;
  53. TMaybe<TFormat> MapperOutputFormat;
  54. TMaybe<TFormat> ReduceCombinerInputFormat;
  55. TMaybe<TFormat> ReduceCombinerOutputFormat;
  56. TFormat ReducerInputFormat = TFormat::YsonBinary();
  57. TFormat ReducerOutputFormat = TFormat::YsonBinary();
  58. TVector<TSmallJobFile> MapperJobFiles;
  59. TVector<TSmallJobFile> ReduceCombinerJobFiles;
  60. TVector<TSmallJobFile> ReducerJobFiles;
  61. };
  62. template <typename T>
  63. void VerifyHasElements(const TVector<T>& paths, TStringBuf name)
  64. {
  65. if (paths.empty()) {
  66. ythrow TApiUsageError() << "no " << name << " table is specified";
  67. }
  68. }
  69. ////////////////////////////////////////////////////////////////////////////////
  70. TVector<TSmallJobFile> CreateFormatConfig(
  71. TMaybe<TSmallJobFile> inputConfig,
  72. const TMaybe<TSmallJobFile>& outputConfig)
  73. {
  74. TVector<TSmallJobFile> result;
  75. if (inputConfig) {
  76. result.push_back(std::move(*inputConfig));
  77. }
  78. if (outputConfig) {
  79. result.push_back(std::move(*outputConfig));
  80. }
  81. return result;
  82. }
  83. template <typename T>
  84. ENodeReaderFormat NodeReaderFormatFromHintAndGlobalConfig(const TUserJobFormatHintsBase<T>& formatHints)
  85. {
  86. auto result = TConfig::Get()->NodeReaderFormat;
  87. if (formatHints.InputFormatHints_ && formatHints.InputFormatHints_->SkipNullValuesForTNode_) {
  88. Y_ENSURE_EX(
  89. result != ENodeReaderFormat::Skiff,
  90. TApiUsageError() << "skiff format doesn't support SkipNullValuesForTNode format hint");
  91. result = ENodeReaderFormat::Yson;
  92. }
  93. return result;
  94. }
  95. template <class TSpec>
  96. const TVector<TStructuredTablePath>& GetStructuredInputs(const TSpec& spec)
  97. {
  98. if constexpr (std::is_same_v<TSpec, TVanillaTask>) {
  99. static const TVector<TStructuredTablePath> empty;
  100. return empty;
  101. } else {
  102. return spec.GetStructuredInputs();
  103. }
  104. }
  105. template <class TSpec>
  106. const TVector<TStructuredTablePath>& GetStructuredOutputs(const TSpec& spec)
  107. {
  108. return spec.GetStructuredOutputs();
  109. }
  110. template <class TSpec>
  111. const TMaybe<TFormatHints>& GetInputFormatHints(const TSpec& spec)
  112. {
  113. if constexpr (std::is_same_v<TSpec, TVanillaTask>) {
  114. static const TMaybe<TFormatHints> empty = Nothing();
  115. return empty;
  116. } else {
  117. return spec.InputFormatHints_;
  118. }
  119. }
  120. template <class TSpec>
  121. const TMaybe<TFormatHints>& GetOutputFormatHints(const TSpec& spec)
  122. {
  123. return spec.OutputFormatHints_;
  124. }
  125. template <class TSpec>
  126. ENodeReaderFormat GetNodeReaderFormat(const TSpec& spec, bool allowSkiff)
  127. {
  128. if constexpr (std::is_same<TSpec, TVanillaTask>::value) {
  129. return ENodeReaderFormat::Yson;
  130. } else {
  131. return allowSkiff
  132. ? NodeReaderFormatFromHintAndGlobalConfig(spec)
  133. : ENodeReaderFormat::Yson;
  134. }
  135. }
  136. static void SortColumnsToNames(const TSortColumns& sortColumns, THashSet<TString>* result)
  137. {
  138. auto names = sortColumns.GetNames();
  139. result->insert(names.begin(), names.end());
  140. }
  141. static THashSet<TString> SortColumnsToNames(const TSortColumns& sortColumns)
  142. {
  143. THashSet<TString> columnNames;
  144. SortColumnsToNames(sortColumns, &columnNames);
  145. return columnNames;
  146. }
  147. THashSet<TString> GetColumnsUsedInOperation(const TJoinReduceOperationSpec& spec)
  148. {
  149. return SortColumnsToNames(spec.JoinBy_);
  150. }
  151. THashSet<TString> GetColumnsUsedInOperation(const TReduceOperationSpec& spec) {
  152. auto result = SortColumnsToNames(spec.SortBy_);
  153. SortColumnsToNames(spec.ReduceBy_, &result);
  154. if (spec.JoinBy_) {
  155. SortColumnsToNames(*spec.JoinBy_, &result);
  156. }
  157. return result;
  158. }
  159. THashSet<TString> GetColumnsUsedInOperation(const TMapReduceOperationSpec& spec)
  160. {
  161. auto result = SortColumnsToNames(spec.SortBy_);
  162. SortColumnsToNames(spec.ReduceBy_, &result);
  163. return result;
  164. }
  165. THashSet<TString> GetColumnsUsedInOperation(const TMapOperationSpec&)
  166. {
  167. return THashSet<TString>();
  168. }
  169. THashSet<TString> GetColumnsUsedInOperation(const TVanillaTask&)
  170. {
  171. return THashSet<TString>();
  172. }
  173. TStructuredJobTableList ApplyProtobufColumnFilters(
  174. const TStructuredJobTableList& tableList,
  175. const TOperationPreparer& preparer,
  176. const THashSet<TString>& columnsUsedInOperations,
  177. const TOperationOptions& options)
  178. {
  179. bool hasInputQuery = options.Spec_.Defined() && options.Spec_->IsMap() && options.Spec_->HasKey("input_query");
  180. if (hasInputQuery) {
  181. return tableList;
  182. }
  183. TVector<TRichYPath> tableListPaths;
  184. for (const auto& table: tableList) {
  185. Y_ABORT_UNLESS(table.RichYPath, "Cannot get path to apply column filters");
  186. tableListPaths.emplace_back(*table.RichYPath);
  187. }
  188. auto isDynamic = NRawClient::RemoteClustersBatchTransform(
  189. preparer.GetClient()->GetRawClient(),
  190. preparer.GetContext(),
  191. tableListPaths,
  192. [&] (IRawBatchRequestPtr batch, const auto& table) {
  193. // In case of external cluster, we can't use the current transaction
  194. // since it is unknown for the external cluster.
  195. // Hence, we should take a global transaction.
  196. if (table.Cluster_ && !table.Cluster_->empty()) {
  197. return batch->Get(TTransactionId(), table.Path_ + "/@dynamic", TGetOptions());
  198. }
  199. return batch->Get(preparer.GetTransactionId(), table.Path_ + "/@dynamic", TGetOptions());
  200. });
  201. auto newTableList = tableList;
  202. for (size_t tableIndex = 0; tableIndex < tableList.size(); ++tableIndex) {
  203. if (isDynamic[tableIndex].AsBool()) {
  204. continue;
  205. }
  206. auto& table = newTableList[tableIndex];
  207. Y_ABORT_UNLESS(table.RichYPath);
  208. if (table.RichYPath->Columns_) {
  209. continue;
  210. }
  211. if (!std::holds_alternative<TProtobufTableStructure>(table.Description)) {
  212. continue;
  213. }
  214. const auto& descriptor = std::get<TProtobufTableStructure>(table.Description).Descriptor;
  215. if (!descriptor) {
  216. continue;
  217. }
  218. auto fromDescriptor = NDetail::InferColumnFilter(*descriptor);
  219. if (!fromDescriptor) {
  220. continue;
  221. }
  222. THashSet<TString> columns(fromDescriptor->begin(), fromDescriptor->end());
  223. columns.insert(columnsUsedInOperations.begin(), columnsUsedInOperations.end());
  224. table.RichYPath->Columns(TVector<TString>(columns.begin(), columns.end()));
  225. }
  226. return newTableList;
  227. }
  228. template <class TSpec>
  229. TSimpleOperationIo CreateSimpleOperationIo(
  230. const IStructuredJob& structuredJob,
  231. const TOperationPreparer& preparer,
  232. const TSpec& spec,
  233. const TOperationOptions& options,
  234. bool allowSkiff)
  235. {
  236. if (!std::holds_alternative<TVoidStructuredRowStream>(structuredJob.GetInputRowStreamDescription())) {
  237. VerifyHasElements(GetStructuredInputs(spec), "input");
  238. }
  239. TUserJobFormatHints hints;
  240. hints.InputFormatHints_ = GetInputFormatHints(spec);
  241. hints.OutputFormatHints_ = GetOutputFormatHints(spec);
  242. ENodeReaderFormat nodeReaderFormat = GetNodeReaderFormat(spec, allowSkiff);
  243. return CreateSimpleOperationIoHelper(
  244. structuredJob,
  245. preparer,
  246. options,
  247. CanonizeStructuredTableList(preparer.GetClient()->GetRawClient(), GetStructuredInputs(spec)),
  248. CanonizeStructuredTableList(preparer.GetClient()->GetRawClient(), GetStructuredOutputs(spec)),
  249. hints,
  250. nodeReaderFormat,
  251. GetColumnsUsedInOperation(spec));
  252. }
  253. template <class T>
  254. TSimpleOperationIo CreateSimpleOperationIo(
  255. const IJob& job,
  256. const TOperationPreparer& preparer,
  257. const TSimpleRawOperationIoSpec<T>& spec)
  258. {
  259. auto getFormatOrDefault = [&] (const TMaybe<TFormat>& maybeFormat, const char* formatName) {
  260. if (maybeFormat) {
  261. return *maybeFormat;
  262. } else if (spec.Format_) {
  263. return *spec.Format_;
  264. } else {
  265. ythrow TApiUsageError() << "Neither " << formatName << "format nor default format is specified for raw operation";
  266. }
  267. };
  268. auto inputs = NRawClient::CanonizeYPaths(preparer.GetClient()->GetRawClient(), spec.GetInputs());
  269. auto outputs = NRawClient::CanonizeYPaths(preparer.GetClient()->GetRawClient(), spec.GetOutputs());
  270. VerifyHasElements(inputs, "input");
  271. VerifyHasElements(outputs, "output");
  272. TUserJobFormatHints hints;
  273. auto outputSchemas = PrepareOperation(
  274. job,
  275. TOperationPreparationContext(
  276. inputs,
  277. outputs,
  278. preparer.GetClient()->GetRawClient(),
  279. preparer.GetClientRetryPolicy(),
  280. preparer.GetTransactionId()),
  281. &inputs,
  282. &outputs,
  283. hints);
  284. Y_ABORT_UNLESS(outputs.size() == outputSchemas.size());
  285. for (int i = 0; i < static_cast<int>(outputs.size()); ++i) {
  286. if (!outputs[i].Schema_ && !outputSchemas[i].Columns().empty()) {
  287. outputs[i].Schema_ = outputSchemas[i];
  288. }
  289. }
  290. return TSimpleOperationIo {
  291. inputs,
  292. outputs,
  293. getFormatOrDefault(spec.InputFormat_, "input"),
  294. getFormatOrDefault(spec.OutputFormat_, "output"),
  295. TVector<TSmallJobFile>{},
  296. };
  297. }
  298. ////////////////////////////////////////////////////////////////////////////////
  299. TString GetJobStderrWithRetriesAndIgnoreErrors(
  300. const IRequestRetryPolicyPtr& retryPolicy,
  301. const IRawClientPtr& rawClient,
  302. const TOperationId& operationId,
  303. const TJobId& jobId,
  304. const size_t stderrTailSize,
  305. const TGetJobStderrOptions& options = {})
  306. {
  307. TString jobStderr;
  308. try {
  309. jobStderr = RequestWithRetry<TString>(
  310. retryPolicy,
  311. [&rawClient, &operationId, &jobId, &options] (TMutationId /*mutationId*/) {
  312. return rawClient->GetJobStderr(operationId, jobId, options)->ReadAll();
  313. });
  314. } catch (const TErrorResponse& e) {
  315. YT_LOG_ERROR("Cannot get job stderr (OperationId: %v, JobId: %v, Error: %v)",
  316. operationId,
  317. jobId,
  318. e.what());
  319. }
  320. if (jobStderr.size() > stderrTailSize) {
  321. jobStderr = jobStderr.substr(jobStderr.size() - stderrTailSize, stderrTailSize);
  322. }
  323. return jobStderr;
  324. }
  325. TVector<TFailedJobInfo> GetFailedJobInfo(
  326. const IClientRetryPolicyPtr& clientRetryPolicy,
  327. const IRawClientPtr& rawClient,
  328. const TOperationId& operationId,
  329. const TGetFailedJobInfoOptions& options)
  330. {
  331. const auto listJobsResult = RequestWithRetry<TListJobsResult>(
  332. clientRetryPolicy->CreatePolicyForGenericRequest(),
  333. [&rawClient, &operationId, &options] (TMutationId /*mutationId*/) {
  334. return rawClient->ListJobs(
  335. operationId,
  336. TListJobsOptions()
  337. .State(EJobState::Failed)
  338. .Limit(options.MaxJobCount_));
  339. });
  340. const auto stderrTailSize = options.StderrTailSize_;
  341. TVector<TFailedJobInfo> result;
  342. for (const auto& job : listJobsResult.Jobs) {
  343. auto& info = result.emplace_back();
  344. Y_ENSURE(job.Id);
  345. info.JobId = *job.Id;
  346. info.Error = job.Error.GetOrElse(TYtError(TString("unknown error")));
  347. if (job.StderrSize.GetOrElse(0) != 0) {
  348. // There are cases when due to bad luck we cannot read stderr even if
  349. // list_jobs reports that stderr_size > 0.
  350. //
  351. // Such errors don't have special error code
  352. // so we ignore all errors and try our luck on other jobs.
  353. info.Stderr = GetJobStderrWithRetriesAndIgnoreErrors(
  354. clientRetryPolicy->CreatePolicyForGenericRequest(),
  355. rawClient,
  356. operationId,
  357. *job.Id,
  358. stderrTailSize);
  359. }
  360. }
  361. return result;
  362. }
  363. struct TGetJobsStderrOptions
  364. {
  365. using TSelf = TGetJobsStderrOptions;
  366. // How many jobs to download. Which jobs will be chosen is undefined.
  367. FLUENT_FIELD_DEFAULT(ui64, MaxJobCount, 10);
  368. // How much of stderr should be downloaded.
  369. FLUENT_FIELD_DEFAULT(ui64, StderrTailSize, 64 * 1024);
  370. };
  371. static TVector<TString> GetJobsStderr(
  372. const IClientRetryPolicyPtr& clientRetryPolicy,
  373. const IRawClientPtr& rawClient,
  374. const TOperationId& operationId,
  375. const TGetJobsStderrOptions& options = {})
  376. {
  377. const auto listJobsResult = RequestWithRetry<TListJobsResult>(
  378. clientRetryPolicy->CreatePolicyForGenericRequest(),
  379. [&rawClient, &operationId, &options] (TMutationId /*mutationId*/) {
  380. return rawClient->ListJobs(
  381. operationId,
  382. TListJobsOptions()
  383. .Limit(options.MaxJobCount_)
  384. .WithStderr(true));
  385. });
  386. const auto stderrTailSize = options.StderrTailSize_;
  387. TVector<TString> result;
  388. for (const auto& job : listJobsResult.Jobs) {
  389. result.push_back(
  390. // There are cases when due to bad luck we cannot read stderr even if
  391. // list_jobs reports that stderr_size > 0.
  392. //
  393. // Such errors don't have special error code
  394. // so we ignore all errors and try our luck on other jobs.
  395. GetJobStderrWithRetriesAndIgnoreErrors(
  396. clientRetryPolicy->CreatePolicyForGenericRequest(),
  397. rawClient,
  398. operationId,
  399. *job.Id,
  400. stderrTailSize)
  401. );
  402. }
  403. return result;
  404. }
  405. int CountIntermediateTables(const TStructuredJobTableList& tables)
  406. {
  407. int result = 0;
  408. for (const auto& table : tables) {
  409. if (table.RichYPath) {
  410. break;
  411. }
  412. ++result;
  413. }
  414. return result;
  415. }
  416. ////////////////////////////////////////////////////////////////////////////////
  417. } // namespace
  418. ////////////////////////////////////////////////////////////////////////////////
  419. TSimpleOperationIo CreateSimpleOperationIoHelper(
  420. const IStructuredJob& structuredJob,
  421. const TOperationPreparer& preparer,
  422. const TOperationOptions& options,
  423. TStructuredJobTableList structuredInputs,
  424. TStructuredJobTableList structuredOutputs,
  425. TUserJobFormatHints hints,
  426. ENodeReaderFormat nodeReaderFormat,
  427. const THashSet<TString>& columnsUsedInOperations)
  428. {
  429. auto intermediateInputTableCount = CountIntermediateTables(structuredInputs);
  430. auto intermediateOutputTableCount = CountIntermediateTables(structuredOutputs);
  431. auto jobSchemaInferenceResult = PrepareOperation(
  432. structuredJob,
  433. TOperationPreparationContext(
  434. structuredInputs,
  435. structuredOutputs,
  436. preparer.GetClient()->GetRawClient(),
  437. preparer.GetClientRetryPolicy(),
  438. preparer.GetTransactionId()),
  439. &structuredInputs,
  440. &structuredOutputs,
  441. hints);
  442. TVector<TSmallJobFile> formatConfigList;
  443. TFormatBuilder formatBuilder(
  444. preparer.GetClient()->GetRawClient(),
  445. preparer.GetClientRetryPolicy(),
  446. preparer.GetContext(),
  447. preparer.GetTransactionId(),
  448. options);
  449. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  450. structuredJob,
  451. EIODirection::Input,
  452. structuredInputs,
  453. hints.InputFormatHints_,
  454. nodeReaderFormat,
  455. /* allowFormatFromTableAttribute = */ true);
  456. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  457. structuredJob,
  458. EIODirection::Output,
  459. structuredOutputs,
  460. hints.OutputFormatHints_,
  461. ENodeReaderFormat::Yson,
  462. /* allowFormatFromTableAttribute = */ false);
  463. const bool inferOutputSchema = options.InferOutputSchema_.GetOrElse(preparer.GetContext().Config->InferTableSchema);
  464. auto outputPaths = GetPathList(
  465. TStructuredJobTableList(structuredOutputs.begin() + intermediateOutputTableCount, structuredOutputs.end()),
  466. TVector<TTableSchema>(jobSchemaInferenceResult.begin() + intermediateOutputTableCount, jobSchemaInferenceResult.end()),
  467. inferOutputSchema);
  468. auto inputPaths = GetPathList(
  469. ApplyProtobufColumnFilters(
  470. TStructuredJobTableList(structuredInputs.begin() + intermediateInputTableCount, structuredInputs.end()),
  471. preparer,
  472. columnsUsedInOperations,
  473. options),
  474. /*schemaInferenceResult*/ Nothing(),
  475. /*inferSchema*/ false);
  476. return TSimpleOperationIo {
  477. inputPaths,
  478. outputPaths,
  479. inputFormat,
  480. outputFormat,
  481. CreateFormatConfig(inputFormatConfig, outputFormatConfig)
  482. };
  483. }
  484. EOperationBriefState CheckOperation(
  485. const IRawClientPtr& rawClient,
  486. const IClientRetryPolicyPtr& clientRetryPolicy,
  487. const TOperationId& operationId)
  488. {
  489. auto attributes = RequestWithRetry<TOperationAttributes>(
  490. clientRetryPolicy->CreatePolicyForGenericRequest(),
  491. [&rawClient, &operationId] (TMutationId /*mutationId*/) {
  492. return rawClient->GetOperation(
  493. operationId,
  494. TGetOperationOptions().AttributeFilter(TOperationAttributeFilter()
  495. .Add(EOperationAttribute::State)
  496. .Add(EOperationAttribute::Result)));
  497. });
  498. Y_ABORT_UNLESS(attributes.BriefState,
  499. "get_operation for operation %s has not returned \"state\" field",
  500. GetGuidAsString(operationId).data());
  501. if (*attributes.BriefState == EOperationBriefState::Completed) {
  502. return EOperationBriefState::Completed;
  503. } else if (*attributes.BriefState == EOperationBriefState::Aborted || *attributes.BriefState == EOperationBriefState::Failed) {
  504. YT_LOG_ERROR("Operation %v %v (%v)",
  505. operationId,
  506. ToString(*attributes.BriefState),
  507. ToString(TOperationExecutionTimeTracker::Get()->Finish(operationId)));
  508. auto failedJobInfoList = GetFailedJobInfo(
  509. clientRetryPolicy,
  510. rawClient,
  511. operationId,
  512. TGetFailedJobInfoOptions());
  513. Y_ABORT_UNLESS(attributes.Result && attributes.Result->Error);
  514. ythrow TOperationFailedError(
  515. *attributes.BriefState == EOperationBriefState::Aborted
  516. ? TOperationFailedError::Aborted
  517. : TOperationFailedError::Failed,
  518. operationId,
  519. *attributes.Result->Error,
  520. failedJobInfoList);
  521. }
  522. return EOperationBriefState::InProgress;
  523. }
  524. void WaitForOperation(
  525. const IClientRetryPolicyPtr& clientRetryPolicy,
  526. const IRawClientPtr& rawClient,
  527. const TClientContext& context,
  528. const TOperationId& operationId)
  529. {
  530. const TDuration checkOperationStateInterval =
  531. UseLocalModeOptimization(rawClient, context, clientRetryPolicy)
  532. ? Min(TDuration::MilliSeconds(100), context.Config->OperationTrackerPollPeriod)
  533. : context.Config->OperationTrackerPollPeriod;
  534. while (true) {
  535. auto status = CheckOperation(rawClient, clientRetryPolicy, operationId);
  536. if (status == EOperationBriefState::Completed) {
  537. YT_LOG_INFO("Operation %v completed (%v)",
  538. operationId,
  539. TOperationExecutionTimeTracker::Get()->Finish(operationId));
  540. break;
  541. }
  542. TWaitProxy::Get()->Sleep(checkOperationStateInterval);
  543. }
  544. }
  545. ////////////////////////////////////////////////////////////////////////////////
  546. namespace {
  547. TNode BuildAutoMergeSpec(const TAutoMergeSpec& options)
  548. {
  549. TNode result;
  550. if (options.Mode_) {
  551. result["mode"] = ToString(*options.Mode_);
  552. }
  553. if (options.MaxIntermediateChunkCount_) {
  554. result["max_intermediate_chunk_count"] = *options.MaxIntermediateChunkCount_;
  555. }
  556. if (options.ChunkCountPerMergeJob_) {
  557. result["chunk_count_per_merge_job"] = *options.ChunkCountPerMergeJob_;
  558. }
  559. if (options.ChunkSizeThreshold_) {
  560. result["chunk_size_threshold"] = *options.ChunkSizeThreshold_;
  561. }
  562. return result;
  563. }
  564. [[maybe_unused]] TNode BuildJobProfilerSpec(const TJobProfilerSpec& profilerSpec)
  565. {
  566. TNode result;
  567. if (profilerSpec.ProfilingBinary_) {
  568. result["binary"] = ToString(*profilerSpec.ProfilingBinary_);
  569. }
  570. if (profilerSpec.ProfilerType_) {
  571. result["type"] = ToString(*profilerSpec.ProfilerType_);
  572. }
  573. if (profilerSpec.ProfilingProbability_) {
  574. result["profiling_probability"] = *profilerSpec.ProfilingProbability_;
  575. }
  576. if (profilerSpec.SamplingFrequency_) {
  577. result["sampling_frequency"] = *profilerSpec.SamplingFrequency_;
  578. }
  579. return result;
  580. }
  581. // Returns undefined node if resources doesn't contain any meaningful field
  582. TNode BuildSchedulerResourcesSpec(const TSchedulerResources& resources)
  583. {
  584. TNode result;
  585. if (resources.UserSlots().Defined()) {
  586. result["user_slots"] = *resources.UserSlots();
  587. }
  588. if (resources.Cpu().Defined()) {
  589. result["cpu"] = *resources.Cpu();
  590. }
  591. if (resources.Memory().Defined()) {
  592. result["memory"] = *resources.Memory();
  593. }
  594. return result;
  595. }
  596. void BuildUserJobFluently(
  597. const TJobPreparer& preparer,
  598. const TMaybe<TFormat>& inputFormat,
  599. const TMaybe<TFormat>& outputFormat,
  600. TFluentMap fluent)
  601. {
  602. const auto& userJobSpec = preparer.GetSpec();
  603. TMaybe<i64> memoryLimit = userJobSpec.MemoryLimit_;
  604. TMaybe<double> cpuLimit = userJobSpec.CpuLimit_;
  605. TMaybe<ui16> portCount = userJobSpec.PortCount_;
  606. // Use 1MB extra tmpfs size by default, it helps to detect job sandbox as tmp directory
  607. // for standard python libraries. See YTADMINREQ-14505 for more details.
  608. auto tmpfsSize = preparer.GetSpec().ExtraTmpfsSize_.GetOrElse(DefaultExrtaTmpfsSize);
  609. if (preparer.ShouldMountSandbox()) {
  610. tmpfsSize += preparer.GetTotalFileSize();
  611. if (tmpfsSize == 0) {
  612. // This can be a case for example when it is local mode and we don't upload binary.
  613. // NOTE: YT doesn't like zero tmpfs size.
  614. tmpfsSize = RoundUpFileSize(1);
  615. }
  616. memoryLimit = memoryLimit.GetOrElse(512ll << 20) + tmpfsSize;
  617. }
  618. fluent
  619. .Item("file_paths").List(preparer.GetFiles())
  620. .DoIf(!preparer.GetLayers().empty(), [&] (TFluentMap fluentMap) {
  621. fluentMap.Item("layer_paths").List(preparer.GetLayers());
  622. })
  623. .DoIf(userJobSpec.DockerImage_.Defined(), [&] (TFluentMap fluentMap) {
  624. fluentMap.Item("docker_image").Value(*userJobSpec.DockerImage_);
  625. })
  626. .Item("command").Value(preparer.GetCommand())
  627. .Item("class_name").Value(preparer.GetClassName())
  628. .DoIf(!userJobSpec.Environment_.empty(), [&] (TFluentMap fluentMap) {
  629. TNode environment;
  630. for (const auto& item : userJobSpec.Environment_) {
  631. environment[item.first] = item.second;
  632. }
  633. fluentMap.Item("environment").Value(environment);
  634. })
  635. .DoIf(userJobSpec.DiskSpaceLimit_.Defined(), [&] (TFluentMap fluentMap) {
  636. fluentMap.Item("disk_space_limit").Value(*userJobSpec.DiskSpaceLimit_);
  637. })
  638. .DoIf(inputFormat.Defined(), [&] (TFluentMap fluentMap) {
  639. fluentMap.Item("input_format").Value(inputFormat->Config);
  640. })
  641. .DoIf(outputFormat.Defined(), [&] (TFluentMap fluentMap) {
  642. fluentMap.Item("output_format").Value(outputFormat->Config);
  643. })
  644. .DoIf(memoryLimit.Defined(), [&] (TFluentMap fluentMap) {
  645. fluentMap.Item("memory_limit").Value(*memoryLimit);
  646. })
  647. .DoIf(userJobSpec.MemoryReserveFactor_.Defined(), [&] (TFluentMap fluentMap) {
  648. fluentMap.Item("memory_reserve_factor").Value(*userJobSpec.MemoryReserveFactor_);
  649. })
  650. .DoIf(cpuLimit.Defined(), [&] (TFluentMap fluentMap) {
  651. fluentMap.Item("cpu_limit").Value(*cpuLimit);
  652. })
  653. .DoIf(portCount.Defined(), [&] (TFluentMap fluentMap) {
  654. fluentMap.Item("port_count").Value(*portCount);
  655. })
  656. .DoIf(userJobSpec.JobTimeLimit_.Defined(), [&] (TFluentMap fluentMap) {
  657. fluentMap.Item("job_time_limit").Value(userJobSpec.JobTimeLimit_->MilliSeconds());
  658. })
  659. .DoIf(userJobSpec.DiskRequest_.Defined(), [&] (TFluentMap fluentMap) {
  660. const auto& diskRequest = *userJobSpec.DiskRequest_;
  661. TNode diskRequestNode = TNode::CreateMap();
  662. if (diskRequest.DiskSpace_.Defined()) {
  663. diskRequestNode["disk_space"] = *diskRequest.DiskSpace_;
  664. }
  665. if (diskRequest.InodeCount_.Defined()) {
  666. diskRequestNode["inode_count"] = *diskRequest.InodeCount_;
  667. }
  668. if (diskRequest.Account_.Defined()) {
  669. diskRequestNode["account"] = *diskRequest.Account_;
  670. }
  671. if (diskRequest.MediumName_.Defined()) {
  672. diskRequestNode["medium_name"] = *diskRequest.MediumName_;
  673. }
  674. fluentMap.Item("disk_request").Value(diskRequestNode);
  675. })
  676. .DoIf(userJobSpec.NetworkProject_.Defined(), [&] (TFluentMap fluentMap) {
  677. fluentMap.Item("network_project").Value(*userJobSpec.NetworkProject_);
  678. })
  679. .DoIf(preparer.ShouldMountSandbox(), [&] (TFluentMap fluentMap) {
  680. fluentMap.Item("tmpfs_path").Value(".");
  681. fluentMap.Item("tmpfs_size").Value(tmpfsSize);
  682. fluentMap.Item("copy_files").Value(true);
  683. })
  684. .Item("profilers")
  685. .BeginList()
  686. .DoFor(userJobSpec.JobProfilers_, [&] (TFluentList list, const auto& jobProfiler) {
  687. list.Item().Value(BuildJobProfilerSpec(jobProfiler));
  688. })
  689. .EndList()
  690. .Item("start_queue_consumer_registration_manager").Value(false)
  691. .Item("redirect_stdout_to_stderr").Value(preparer.ShouldRedirectStdoutToStderr());
  692. }
  693. struct TNirvanaContext
  694. {
  695. TNode BlockUrl;
  696. TNode Annotations;
  697. };
  698. // Try to detect if we are inside nirvana operation and reat nirvana job context.
  699. // Items of TNirvanaContext might be Undefined, if we are not inside nirvana context (or if nirvana context is unexpected)
  700. TNirvanaContext GetNirvanaContext()
  701. {
  702. static const auto filePath = TString("/slot/sandbox/j/job_context.json");
  703. auto nvYtOperationId = GetEnv("NV_YT_OPERATION_ID");
  704. if (nvYtOperationId.empty()) {
  705. return {};
  706. }
  707. if (!NFs::Exists(filePath)) {
  708. return {};
  709. }
  710. NJson::TJsonValue json;
  711. try {
  712. auto inf = TIFStream(filePath);
  713. json = NJson::ReadJsonTree(&inf, /*throwOnError*/ true);
  714. } catch (const std::exception& ex) {
  715. YT_LOG_ERROR("Failed to load nirvana job context: %v", ex.what());
  716. return {};
  717. }
  718. TNirvanaContext result;
  719. const auto* url = json.GetValueByPath("meta.blockURL");
  720. if (url && url->IsString()) {
  721. result.BlockUrl = url->GetString();
  722. result.BlockUrl.Attributes()["_type_tag"] = "url";
  723. }
  724. const auto* annotations = json.GetValueByPath("meta.annotations");
  725. if (annotations && annotations->IsMap()) {
  726. result.Annotations = NodeFromJsonValue(*annotations);
  727. }
  728. return result;
  729. }
  730. template <typename T>
  731. void BuildCommonOperationPart(
  732. const TConfigPtr& config,
  733. const TOperationSpecBase<T>& baseSpec,
  734. const TOperationOptions& options,
  735. TNode* specNode)
  736. {
  737. const TProcessState* properties = TProcessState::Get();
  738. auto& startedBySpec = (*specNode)["started_by"];
  739. startedBySpec["hostname"] = properties->FqdnHostName,
  740. startedBySpec["pid"] = properties->Pid;
  741. startedBySpec["user"] = properties->UserName;
  742. startedBySpec["wrapper_version"] = properties->ClientVersion;
  743. startedBySpec["binary"] = properties->BinaryPath;
  744. startedBySpec["binary_name"] = properties->BinaryName;
  745. auto nirvanaContext = GetNirvanaContext();
  746. if (!nirvanaContext.BlockUrl.IsUndefined()) {
  747. startedBySpec["nirvana_block_url"] = nirvanaContext.BlockUrl;
  748. }
  749. if (!nirvanaContext.Annotations.IsUndefined()) {
  750. MergeNodes((*specNode)["annotations"], nirvanaContext.Annotations);
  751. }
  752. TString pool;
  753. if (baseSpec.Pool_) {
  754. pool = *baseSpec.Pool_;
  755. } else {
  756. pool = config->Pool;
  757. }
  758. if (!pool.empty()) {
  759. (*specNode)["pool"] = pool;
  760. }
  761. if (baseSpec.Weight_.Defined()) {
  762. (*specNode)["weight"] = *baseSpec.Weight_;
  763. }
  764. if (baseSpec.TimeLimit_.Defined()) {
  765. (*specNode)["time_limit"] = baseSpec.TimeLimit_->MilliSeconds();
  766. }
  767. if (baseSpec.PoolTrees().Defined()) {
  768. TNode poolTreesSpec = TNode::CreateList();
  769. for (const auto& tree : *baseSpec.PoolTrees()) {
  770. poolTreesSpec.Add(tree);
  771. }
  772. (*specNode)["pool_trees"] = std::move(poolTreesSpec);
  773. }
  774. if (baseSpec.ResourceLimits().Defined()) {
  775. auto resourceLimitsSpec = BuildSchedulerResourcesSpec(*baseSpec.ResourceLimits());
  776. if (!resourceLimitsSpec.IsUndefined()) {
  777. (*specNode)["resource_limits"] = std::move(resourceLimitsSpec);
  778. }
  779. }
  780. if (options.SecureVault_.Defined()) {
  781. Y_ENSURE(options.SecureVault_->IsMap(),
  782. "SecureVault must be a map node, got " << options.SecureVault_->GetType());
  783. (*specNode)["secure_vault"] = *options.SecureVault_;
  784. }
  785. if (baseSpec.Title_.Defined()) {
  786. (*specNode)["title"] = *baseSpec.Title_;
  787. }
  788. if (baseSpec.MaxFailedJobCount_.Defined()) {
  789. (*specNode)["max_failed_job_count"] = *baseSpec.MaxFailedJobCount_;
  790. }
  791. if (baseSpec.Description_.Defined()) {
  792. (*specNode)["description"] = *baseSpec.Description_;
  793. }
  794. if (baseSpec.Annotations_.Defined()) {
  795. (*specNode)["annotations"] = *baseSpec.Annotations_;
  796. }
  797. }
  798. template <typename TSpec>
  799. void BuildCommonUserOperationPart(const TSpec& baseSpec, TNode* spec)
  800. {
  801. if (baseSpec.FailOnJobRestart_.Defined()) {
  802. (*spec)["fail_on_job_restart"] = *baseSpec.FailOnJobRestart_;
  803. }
  804. if (baseSpec.StderrTablePath_.Defined()) {
  805. (*spec)["stderr_table_path"] = *baseSpec.StderrTablePath_;
  806. }
  807. if (baseSpec.CoreTablePath_.Defined()) {
  808. (*spec)["core_table_path"] = *baseSpec.CoreTablePath_;
  809. }
  810. if (baseSpec.WaitingJobTimeout_.Defined()) {
  811. (*spec)["waiting_job_timeout"] = baseSpec.WaitingJobTimeout_->MilliSeconds();
  812. }
  813. }
  814. template <typename TSpec>
  815. void BuildJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  816. {
  817. if (spec.JobCount_.Defined()) {
  818. (*nodeSpec)["job_count"] = *spec.JobCount_;
  819. }
  820. if (spec.DataSizePerJob_.Defined()) {
  821. (*nodeSpec)["data_size_per_job"] = *spec.DataSizePerJob_;
  822. }
  823. }
  824. template <typename TSpec>
  825. void BuildPartitionCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  826. {
  827. if (spec.PartitionCount_.Defined()) {
  828. (*nodeSpec)["partition_count"] = *spec.PartitionCount_;
  829. }
  830. if (spec.PartitionDataSize_.Defined()) {
  831. (*nodeSpec)["partition_data_size"] = *spec.PartitionDataSize_;
  832. }
  833. }
  834. template <typename TSpec>
  835. void BuildDataSizePerSortJobPart(const TSpec& spec, TNode* nodeSpec)
  836. {
  837. if (spec.DataSizePerSortJob_.Defined()) {
  838. (*nodeSpec)["data_size_per_sort_job"] = *spec.DataSizePerSortJob_;
  839. }
  840. }
  841. template <typename TSpec>
  842. void BuildPartitionJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  843. {
  844. if (spec.PartitionJobCount_.Defined()) {
  845. (*nodeSpec)["partition_job_count"] = *spec.PartitionJobCount_;
  846. }
  847. if (spec.DataSizePerPartitionJob_.Defined()) {
  848. (*nodeSpec)["data_size_per_partition_job"] = *spec.DataSizePerPartitionJob_;
  849. }
  850. }
  851. template <typename TSpec>
  852. void BuildMapJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  853. {
  854. if (spec.MapJobCount_.Defined()) {
  855. (*nodeSpec)["map_job_count"] = *spec.MapJobCount_;
  856. }
  857. if (spec.DataSizePerMapJob_.Defined()) {
  858. (*nodeSpec)["data_size_per_map_job"] = *spec.DataSizePerMapJob_;
  859. }
  860. }
  861. template <typename TSpec>
  862. void BuildIntermediateDataPart(const TSpec& spec, TNode* nodeSpec)
  863. {
  864. if (spec.IntermediateDataAccount_.Defined()) {
  865. (*nodeSpec)["intermediate_data_account"] = *spec.IntermediateDataAccount_;
  866. }
  867. if (spec.IntermediateDataReplicationFactor_.Defined()) {
  868. (*nodeSpec)["intermediate_data_replication_factor"] = *spec.IntermediateDataReplicationFactor_;
  869. }
  870. }
  871. ////////////////////////////////////////////////////////////////////////////////
  872. TNode MergeSpec(TNode dst, TNode spec, const TOperationOptions& options)
  873. {
  874. MergeNodes(dst, spec);
  875. if (options.Spec_) {
  876. MergeNodes(dst, *options.Spec_);
  877. }
  878. return dst;
  879. }
  880. template <typename TSpec>
  881. void CreateDebugOutputTables(const TSpec& spec, const TOperationPreparer& preparer)
  882. {
  883. if (spec.StderrTablePath_.Defined()) {
  884. RequestWithRetry<void>(
  885. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  886. [&spec, &preparer] (TMutationId& mutationId) {
  887. preparer.GetClient()->GetRawClient()->Create(
  888. mutationId,
  889. TTransactionId(),
  890. *spec.StderrTablePath_,
  891. NT_TABLE,
  892. TCreateOptions()
  893. .IgnoreExisting(true)
  894. .Recursive(true));
  895. });
  896. }
  897. if (spec.CoreTablePath_.Defined()) {
  898. RequestWithRetry<void>(
  899. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  900. [&spec, &preparer] (TMutationId& mutationId) {
  901. preparer.GetClient()->GetRawClient()->Create(
  902. mutationId,
  903. TTransactionId(),
  904. *spec.CoreTablePath_,
  905. NT_TABLE,
  906. TCreateOptions()
  907. .IgnoreExisting(true)
  908. .Recursive(true));
  909. });
  910. }
  911. }
  912. void CreateOutputTable(
  913. const TOperationPreparer& preparer,
  914. const TRichYPath& path)
  915. {
  916. Y_ENSURE(path.Path_, "Output table is not set");
  917. if (!path.Create_.Defined()) {
  918. // If `create` attribute is defined
  919. RequestWithRetry<void>(
  920. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  921. [&preparer, &path] (TMutationId& mutationId) {
  922. preparer.GetClient()->GetRawClient()->Create(
  923. mutationId,
  924. preparer.GetTransactionId(),
  925. path.Path_,
  926. NT_TABLE,
  927. TCreateOptions()
  928. .IgnoreExisting(true)
  929. .Recursive(true));
  930. });
  931. }
  932. }
  933. void CreateOutputTables(
  934. const TOperationPreparer& preparer,
  935. const TVector<TRichYPath>& paths)
  936. {
  937. for (auto& path : paths) {
  938. CreateOutputTable(preparer, path);
  939. }
  940. }
  941. void CheckInputTablesExist(
  942. const TOperationPreparer& preparer,
  943. const TVector<TRichYPath>& paths)
  944. {
  945. Y_ENSURE(!paths.empty(), "Input tables are not set");
  946. for (auto& path : paths) {
  947. auto curTransactionId = path.TransactionId_.GetOrElse(preparer.GetTransactionId());
  948. auto exists = RequestWithRetry<bool>(
  949. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  950. [&preparer, &curTransactionId, &path] (TMutationId /*mutationId*/) {
  951. return preparer.GetClient()->GetRawClient()->Exists(
  952. curTransactionId,
  953. path.Path_);
  954. });
  955. Y_ENSURE_EX(
  956. path.Cluster_.Defined() || exists,
  957. TApiUsageError() << "Input table '" << path.Path_ << "' doesn't exist");
  958. }
  959. }
  960. void LogJob(const TOperationId& opId, const IJob* job, const char* type)
  961. {
  962. if (job) {
  963. YT_LOG_INFO("Operation %v; %v = %v",
  964. opId,
  965. type,
  966. TJobFactory::Get()->GetJobName(job));
  967. }
  968. }
  969. void LogYPaths(const TOperationId& opId, const TVector<TRichYPath>& paths, const char* type)
  970. {
  971. for (size_t i = 0; i < paths.size(); ++i) {
  972. YT_LOG_INFO("Operation %v; %v[%v] = %v",
  973. opId,
  974. type,
  975. i,
  976. paths[i].Path_);
  977. }
  978. }
  979. void LogYPath(const TOperationId& opId, const TRichYPath& path, const char* type)
  980. {
  981. YT_LOG_INFO("Operation %v; %v = %v",
  982. opId,
  983. type,
  984. path.Path_);
  985. }
  986. TString AddModeToTitleIfDebug(const TString& title) {
  987. #ifndef NDEBUG
  988. return title + " (debug build)";
  989. #else
  990. return title;
  991. #endif
  992. }
  993. } // namespace
  994. ////////////////////////////////////////////////////////////////////////////////
  995. template <typename T>
  996. void DoExecuteMap(
  997. const TOperationPtr& operation,
  998. const TOperationPreparerPtr& preparer,
  999. const TSimpleOperationIo& operationIo,
  1000. TMapOperationSpecBase<T> spec,
  1001. const IJobPtr& mapper,
  1002. const TOperationOptions& options)
  1003. {
  1004. if (options.CreateDebugOutputTables_) {
  1005. CreateDebugOutputTables(spec, *preparer);
  1006. }
  1007. if (options.CreateOutputTables_) {
  1008. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1009. CreateOutputTables(*preparer, operationIo.Outputs);
  1010. }
  1011. TJobPreparer map(
  1012. *preparer,
  1013. spec.MapperSpec_,
  1014. *mapper,
  1015. operationIo.Outputs.size(),
  1016. operationIo.JobFiles,
  1017. options);
  1018. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(map.GetClassName()));
  1019. TNode specNode = BuildYsonNodeFluently()
  1020. .BeginMap()
  1021. .Item("mapper").DoMap([&] (TFluentMap fluent) {
  1022. BuildUserJobFluently(
  1023. map,
  1024. operationIo.InputFormat,
  1025. operationIo.OutputFormat,
  1026. fluent);
  1027. })
  1028. .DoIf(spec.AutoMerge_.Defined(), [&] (TFluentMap fluent) {
  1029. auto autoMergeSpec = BuildAutoMergeSpec(*spec.AutoMerge_);
  1030. if (!autoMergeSpec.IsUndefined()) {
  1031. fluent.Item("auto_merge").Value(std::move(autoMergeSpec));
  1032. }
  1033. })
  1034. .Item("input_table_paths").List(operationIo.Inputs)
  1035. .Item("output_table_paths").List(operationIo.Outputs)
  1036. .DoIf(spec.Ordered_.Defined(), [&] (TFluentMap fluent) {
  1037. fluent.Item("ordered").Value(spec.Ordered_.GetRef());
  1038. })
  1039. .EndMap();
  1040. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1041. specNode["job_io"]["control_attributes"]["enable_row_index"] = TNode(true);
  1042. specNode["job_io"]["control_attributes"]["enable_range_index"] = TNode(true);
  1043. if (!preparer->GetContext().Config->TableWriter.Empty()) {
  1044. specNode["job_io"]["table_writer"] = preparer->GetContext().Config->TableWriter;
  1045. }
  1046. BuildCommonUserOperationPart(spec, &specNode);
  1047. BuildJobCountOperationPart(spec, &specNode);
  1048. auto startOperation = [
  1049. operation=operation.Get(),
  1050. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1051. preparer,
  1052. operationIo,
  1053. mapper
  1054. ] () {
  1055. auto operationId = preparer->StartOperation(operation, EOperationType::Map, spec);
  1056. LogJob(operationId, mapper.Get(), "mapper");
  1057. LogYPaths(operationId, operationIo.Inputs, "input");
  1058. LogYPaths(operationId, operationIo.Outputs, "output");
  1059. return operationId;
  1060. };
  1061. operation->SetDelayedStartFunction(std::move(startOperation));
  1062. }
  1063. void ExecuteMap(
  1064. const TOperationPtr& operation,
  1065. const TOperationPreparerPtr& preparer,
  1066. const TMapOperationSpec& spec,
  1067. const ::TIntrusivePtr<IStructuredJob>& mapper,
  1068. const TOperationOptions& options)
  1069. {
  1070. YT_LOG_DEBUG("Starting map operation (PreparationId: %v)",
  1071. preparer->GetPreparationId());
  1072. auto operationIo = CreateSimpleOperationIo(*mapper, *preparer, spec, options, /* allowSkiff = */ true);
  1073. DoExecuteMap(
  1074. operation,
  1075. preparer,
  1076. operationIo,
  1077. spec,
  1078. mapper,
  1079. options);
  1080. }
  1081. void ExecuteRawMap(
  1082. const TOperationPtr& operation,
  1083. const TOperationPreparerPtr& preparer,
  1084. const TRawMapOperationSpec& spec,
  1085. const ::TIntrusivePtr<IRawJob>& mapper,
  1086. const TOperationOptions& options)
  1087. {
  1088. YT_LOG_DEBUG("Starting raw map operation (PreparationId: %v)",
  1089. preparer->GetPreparationId());
  1090. auto operationIo = CreateSimpleOperationIo(*mapper, *preparer, spec);
  1091. DoExecuteMap(
  1092. operation,
  1093. preparer,
  1094. operationIo,
  1095. spec,
  1096. mapper,
  1097. options);
  1098. }
  1099. ////////////////////////////////////////////////////////////////////////////////
  1100. template <typename T>
  1101. void DoExecuteReduce(
  1102. const TOperationPtr& operation,
  1103. const TOperationPreparerPtr& preparer,
  1104. const TSimpleOperationIo& operationIo,
  1105. TReduceOperationSpecBase<T> spec,
  1106. const IJobPtr& reducer,
  1107. const TOperationOptions& options)
  1108. {
  1109. if (options.CreateDebugOutputTables_) {
  1110. CreateDebugOutputTables(spec, *preparer);
  1111. }
  1112. if (options.CreateOutputTables_) {
  1113. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1114. CreateOutputTables(*preparer, operationIo.Outputs);
  1115. }
  1116. TJobPreparer reduce(
  1117. *preparer,
  1118. spec.ReducerSpec_,
  1119. *reducer,
  1120. operationIo.Outputs.size(),
  1121. operationIo.JobFiles,
  1122. options);
  1123. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(reduce.GetClassName()));
  1124. TNode specNode = BuildYsonNodeFluently()
  1125. .BeginMap()
  1126. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1127. BuildUserJobFluently(
  1128. reduce,
  1129. operationIo.InputFormat,
  1130. operationIo.OutputFormat,
  1131. fluent);
  1132. })
  1133. .DoIf(!spec.SortBy_.Parts_.empty(), [&] (TFluentMap fluent) {
  1134. fluent.Item("sort_by").Value(spec.SortBy_);
  1135. })
  1136. .Item("reduce_by").Value(spec.ReduceBy_)
  1137. .DoIf(spec.JoinBy_.Defined(), [&] (TFluentMap fluent) {
  1138. fluent.Item("join_by").Value(spec.JoinBy_.GetRef());
  1139. })
  1140. .DoIf(spec.EnableKeyGuarantee_.Defined(), [&] (TFluentMap fluent) {
  1141. fluent.Item("enable_key_guarantee").Value(spec.EnableKeyGuarantee_.GetRef());
  1142. })
  1143. .Item("input_table_paths").List(operationIo.Inputs)
  1144. .Item("output_table_paths").List(operationIo.Outputs)
  1145. .Item("job_io").BeginMap()
  1146. .Item("control_attributes").BeginMap()
  1147. .Item("enable_key_switch").Value(true)
  1148. .Item("enable_row_index").Value(true)
  1149. .Item("enable_range_index").Value(true)
  1150. .EndMap()
  1151. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1152. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1153. })
  1154. .EndMap()
  1155. .DoIf(spec.AutoMerge_.Defined(), [&] (TFluentMap fluent) {
  1156. fluent.Item("auto_merge").Value(BuildAutoMergeSpec(*spec.AutoMerge_));
  1157. })
  1158. .EndMap();
  1159. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1160. BuildCommonUserOperationPart(spec, &specNode);
  1161. BuildJobCountOperationPart(spec, &specNode);
  1162. auto startOperation = [
  1163. operation=operation.Get(),
  1164. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1165. preparer,
  1166. operationIo,
  1167. reducer
  1168. ] () {
  1169. auto operationId = preparer->StartOperation(operation, EOperationType::Reduce, spec);
  1170. LogJob(operationId, reducer.Get(), "reducer");
  1171. LogYPaths(operationId, operationIo.Inputs, "input");
  1172. LogYPaths(operationId, operationIo.Outputs, "output");
  1173. return operationId;
  1174. };
  1175. operation->SetDelayedStartFunction(std::move(startOperation));
  1176. }
  1177. void ExecuteReduce(
  1178. const TOperationPtr& operation,
  1179. const TOperationPreparerPtr& preparer,
  1180. const TReduceOperationSpec& spec,
  1181. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1182. const TOperationOptions& options)
  1183. {
  1184. YT_LOG_DEBUG("Starting reduce operation (PreparationId: %v)",
  1185. preparer->GetPreparationId());
  1186. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec, options, /* allowSkiff = */ false);
  1187. DoExecuteReduce(
  1188. operation,
  1189. preparer,
  1190. operationIo,
  1191. spec,
  1192. reducer,
  1193. options);
  1194. }
  1195. void ExecuteRawReduce(
  1196. const TOperationPtr& operation,
  1197. const TOperationPreparerPtr& preparer,
  1198. const TRawReduceOperationSpec& spec,
  1199. const ::TIntrusivePtr<IRawJob>& reducer,
  1200. const TOperationOptions& options)
  1201. {
  1202. YT_LOG_DEBUG("Starting raw reduce operation (PreparationId: %v)",
  1203. preparer->GetPreparationId());
  1204. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec);
  1205. DoExecuteReduce(
  1206. operation,
  1207. preparer,
  1208. operationIo,
  1209. spec,
  1210. reducer,
  1211. options);
  1212. }
  1213. ////////////////////////////////////////////////////////////////////////////////
  1214. template <typename T>
  1215. void DoExecuteJoinReduce(
  1216. const TOperationPtr& operation,
  1217. const TOperationPreparerPtr& preparer,
  1218. const TSimpleOperationIo& operationIo,
  1219. TJoinReduceOperationSpecBase<T> spec,
  1220. const IJobPtr& reducer,
  1221. const TOperationOptions& options)
  1222. {
  1223. if (options.CreateDebugOutputTables_) {
  1224. CreateDebugOutputTables(spec, *preparer);
  1225. }
  1226. if (options.CreateOutputTables_) {
  1227. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1228. CreateOutputTables(*preparer, operationIo.Outputs);
  1229. }
  1230. TJobPreparer reduce(
  1231. *preparer,
  1232. spec.ReducerSpec_,
  1233. *reducer,
  1234. operationIo.Outputs.size(),
  1235. operationIo.JobFiles,
  1236. options);
  1237. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(reduce.GetClassName()));
  1238. TNode specNode = BuildYsonNodeFluently()
  1239. .BeginMap()
  1240. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1241. BuildUserJobFluently(
  1242. reduce,
  1243. operationIo.InputFormat,
  1244. operationIo.OutputFormat,
  1245. fluent);
  1246. })
  1247. .Item("join_by").Value(spec.JoinBy_)
  1248. .Item("input_table_paths").List(operationIo.Inputs)
  1249. .Item("output_table_paths").List(operationIo.Outputs)
  1250. .Item("job_io").BeginMap()
  1251. .Item("control_attributes").BeginMap()
  1252. .Item("enable_key_switch").Value(true)
  1253. .Item("enable_row_index").Value(true)
  1254. .Item("enable_range_index").Value(true)
  1255. .EndMap()
  1256. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1257. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1258. })
  1259. .EndMap()
  1260. .EndMap();
  1261. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1262. BuildCommonUserOperationPart(spec, &specNode);
  1263. BuildJobCountOperationPart(spec, &specNode);
  1264. auto startOperation = [
  1265. operation=operation.Get(),
  1266. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1267. preparer,
  1268. reducer,
  1269. operationIo
  1270. ] () {
  1271. auto operationId = preparer->StartOperation(operation, EOperationType::JoinReduce, spec);
  1272. LogJob(operationId, reducer.Get(), "reducer");
  1273. LogYPaths(operationId, operationIo.Inputs, "input");
  1274. LogYPaths(operationId, operationIo.Outputs, "output");
  1275. return operationId;
  1276. };
  1277. operation->SetDelayedStartFunction(std::move(startOperation));
  1278. }
  1279. void ExecuteJoinReduce(
  1280. const TOperationPtr& operation,
  1281. const TOperationPreparerPtr& preparer,
  1282. const TJoinReduceOperationSpec& spec,
  1283. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1284. const TOperationOptions& options)
  1285. {
  1286. YT_LOG_DEBUG("Starting join reduce operation (PreparationId: %v)",
  1287. preparer->GetPreparationId());
  1288. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec, options, /* allowSkiff = */ false);
  1289. return DoExecuteJoinReduce(
  1290. operation,
  1291. preparer,
  1292. operationIo,
  1293. spec,
  1294. reducer,
  1295. options);
  1296. }
  1297. void ExecuteRawJoinReduce(
  1298. const TOperationPtr& operation,
  1299. const TOperationPreparerPtr& preparer,
  1300. const TRawJoinReduceOperationSpec& spec,
  1301. const ::TIntrusivePtr<IRawJob>& reducer,
  1302. const TOperationOptions& options)
  1303. {
  1304. YT_LOG_DEBUG("Starting raw join reduce operation (PreparationId: %v)",
  1305. preparer->GetPreparationId());
  1306. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec);
  1307. return DoExecuteJoinReduce(
  1308. operation,
  1309. preparer,
  1310. operationIo,
  1311. spec,
  1312. reducer,
  1313. options);
  1314. }
  1315. ////////////////////////////////////////////////////////////////////////////////
  1316. template <typename T>
  1317. void DoExecuteMapReduce(
  1318. const TOperationPtr& operation,
  1319. const TOperationPreparerPtr& preparer,
  1320. const TMapReduceOperationIo& operationIo,
  1321. TMapReduceOperationSpecBase<T> spec,
  1322. const IJobPtr& mapper,
  1323. const IJobPtr& reduceCombiner,
  1324. const IJobPtr& reducer,
  1325. const TOperationOptions& options)
  1326. {
  1327. TVector<TRichYPath> allOutputs;
  1328. allOutputs.insert(allOutputs.end(), operationIo.MapOutputs.begin(), operationIo.MapOutputs.end());
  1329. allOutputs.insert(allOutputs.end(), operationIo.Outputs.begin(), operationIo.Outputs.end());
  1330. if (options.CreateDebugOutputTables_) {
  1331. CreateDebugOutputTables(spec, *preparer);
  1332. }
  1333. if (options.CreateOutputTables_) {
  1334. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1335. CreateOutputTables(*preparer, allOutputs);
  1336. }
  1337. TSortColumns sortBy = spec.SortBy_;
  1338. TSortColumns reduceBy = spec.ReduceBy_;
  1339. if (sortBy.Parts_.empty()) {
  1340. sortBy = reduceBy;
  1341. }
  1342. const bool hasMapper = mapper != nullptr;
  1343. const bool hasCombiner = reduceCombiner != nullptr;
  1344. TVector<TRichYPath> files;
  1345. TJobPreparer reduce(
  1346. *preparer,
  1347. spec.ReducerSpec_,
  1348. *reducer,
  1349. operationIo.Outputs.size(),
  1350. operationIo.ReducerJobFiles,
  1351. options);
  1352. TString title;
  1353. TNode specNode = BuildYsonNodeFluently()
  1354. .BeginMap()
  1355. .DoIf(hasMapper, [&] (TFluentMap fluent) {
  1356. TJobPreparer map(
  1357. *preparer,
  1358. spec.MapperSpec_,
  1359. *mapper,
  1360. 1 + operationIo.MapOutputs.size(),
  1361. operationIo.MapperJobFiles,
  1362. options);
  1363. fluent.Item("mapper").DoMap([&] (TFluentMap fluent) {
  1364. BuildUserJobFluently(
  1365. std::cref(map),
  1366. *operationIo.MapperInputFormat,
  1367. *operationIo.MapperOutputFormat,
  1368. fluent);
  1369. });
  1370. title = "mapper:" + map.GetClassName() + " ";
  1371. })
  1372. .DoIf(hasCombiner, [&] (TFluentMap fluent) {
  1373. TJobPreparer combine(
  1374. *preparer,
  1375. spec.ReduceCombinerSpec_,
  1376. *reduceCombiner,
  1377. size_t(1),
  1378. operationIo.ReduceCombinerJobFiles,
  1379. options);
  1380. fluent.Item("reduce_combiner").DoMap([&] (TFluentMap fluent) {
  1381. BuildUserJobFluently(
  1382. combine,
  1383. *operationIo.ReduceCombinerInputFormat,
  1384. *operationIo.ReduceCombinerOutputFormat,
  1385. fluent);
  1386. });
  1387. title += "combiner:" + combine.GetClassName() + " ";
  1388. })
  1389. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1390. BuildUserJobFluently(
  1391. reduce,
  1392. operationIo.ReducerInputFormat,
  1393. operationIo.ReducerOutputFormat,
  1394. fluent);
  1395. })
  1396. .Item("sort_by").Value(sortBy)
  1397. .Item("reduce_by").Value(reduceBy)
  1398. .Item("input_table_paths").List(operationIo.Inputs)
  1399. .Item("output_table_paths").List(allOutputs)
  1400. .Item("mapper_output_table_count").Value(operationIo.MapOutputs.size())
  1401. .DoIf(spec.ForceReduceCombiners_.Defined(), [&] (TFluentMap fluent) {
  1402. fluent.Item("force_reduce_combiners").Value(*spec.ForceReduceCombiners_);
  1403. })
  1404. .Item("map_job_io").BeginMap()
  1405. .Item("control_attributes").BeginMap()
  1406. .Item("enable_row_index").Value(true)
  1407. .Item("enable_range_index").Value(true)
  1408. .EndMap()
  1409. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1410. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1411. })
  1412. .EndMap()
  1413. .Item("sort_job_io").BeginMap()
  1414. .Item("control_attributes").BeginMap()
  1415. .Item("enable_key_switch").Value(true)
  1416. .EndMap()
  1417. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1418. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1419. })
  1420. .EndMap()
  1421. .Item("reduce_job_io").BeginMap()
  1422. .Item("control_attributes").BeginMap()
  1423. .Item("enable_key_switch").Value(true)
  1424. .EndMap()
  1425. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1426. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1427. })
  1428. .EndMap()
  1429. .Do([&] (TFluentMap) {
  1430. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(title + "reducer:" + reduce.GetClassName()));
  1431. })
  1432. .EndMap();
  1433. if (spec.Ordered_) {
  1434. specNode["ordered"] = *spec.Ordered_;
  1435. }
  1436. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1437. BuildCommonUserOperationPart(spec, &specNode);
  1438. BuildMapJobCountOperationPart(spec, &specNode);
  1439. BuildPartitionCountOperationPart(spec, &specNode);
  1440. BuildIntermediateDataPart(spec, &specNode);
  1441. BuildDataSizePerSortJobPart(spec, &specNode);
  1442. auto startOperation = [
  1443. operation=operation.Get(),
  1444. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1445. preparer,
  1446. mapper,
  1447. reduceCombiner,
  1448. reducer,
  1449. inputs=operationIo.Inputs,
  1450. allOutputs
  1451. ] () {
  1452. auto operationId = preparer->StartOperation(operation, EOperationType::MapReduce, spec);
  1453. LogJob(operationId, mapper.Get(), "mapper");
  1454. LogJob(operationId, reduceCombiner.Get(), "reduce_combiner");
  1455. LogJob(operationId, reducer.Get(), "reducer");
  1456. LogYPaths(operationId, inputs, "input");
  1457. LogYPaths(operationId, allOutputs, "output");
  1458. return operationId;
  1459. };
  1460. operation->SetDelayedStartFunction(std::move(startOperation));
  1461. }
  1462. void ExecuteMapReduce(
  1463. const TOperationPtr& operation,
  1464. const TOperationPreparerPtr& preparer,
  1465. const TMapReduceOperationSpec& spec_,
  1466. const ::TIntrusivePtr<IStructuredJob>& mapper,
  1467. const ::TIntrusivePtr<IStructuredJob>& reduceCombiner,
  1468. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1469. const TOperationOptions& options)
  1470. {
  1471. YT_LOG_DEBUG("Starting map-reduce operation (PreparationId: %v)",
  1472. preparer->GetPreparationId());
  1473. TMapReduceOperationSpec spec = spec_;
  1474. TMapReduceOperationIo operationIo;
  1475. auto structuredInputs = CanonizeStructuredTableList(preparer->GetClient()->GetRawClient(), spec.GetStructuredInputs());
  1476. auto structuredMapOutputs = CanonizeStructuredTableList(preparer->GetClient()->GetRawClient(), spec.GetStructuredMapOutputs());
  1477. auto structuredOutputs = CanonizeStructuredTableList(preparer->GetClient()->GetRawClient(), spec.GetStructuredOutputs());
  1478. const bool inferOutputSchema = options.InferOutputSchema_.GetOrElse(preparer->GetContext().Config->InferTableSchema);
  1479. TVector<TTableSchema> currentInferenceResult;
  1480. auto fixSpec = [&] (const TFormat& format) {
  1481. if (format.IsYamredDsv()) {
  1482. spec.SortBy_.Parts_.clear();
  1483. spec.ReduceBy_.Parts_.clear();
  1484. const TYamredDsvAttributes attributes = format.GetYamredDsvAttributes();
  1485. for (auto& column : attributes.KeyColumnNames) {
  1486. spec.SortBy_.Parts_.push_back(column);
  1487. spec.ReduceBy_.Parts_.push_back(column);
  1488. }
  1489. for (const auto& column : attributes.SubkeyColumnNames) {
  1490. spec.SortBy_.Parts_.push_back(column);
  1491. }
  1492. }
  1493. };
  1494. VerifyHasElements(structuredInputs, "inputs");
  1495. TFormatBuilder formatBuilder(
  1496. preparer->GetClient()->GetRawClient(),
  1497. preparer->GetClientRetryPolicy(),
  1498. preparer->GetContext(),
  1499. preparer->GetTransactionId(),
  1500. options);
  1501. if (mapper) {
  1502. auto mapperOutputDescription =
  1503. spec.GetIntermediateMapOutputDescription()
  1504. .GetOrElse(TUnspecifiedTableStructure());
  1505. TStructuredJobTableList mapperOutput = {
  1506. TStructuredJobTable::Intermediate(mapperOutputDescription),
  1507. };
  1508. for (const auto& table : structuredMapOutputs) {
  1509. mapperOutput.push_back(TStructuredJobTable{table.Description, table.RichYPath});
  1510. }
  1511. auto hints = spec.MapperFormatHints_;
  1512. auto mapperInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1513. *mapper,
  1514. TOperationPreparationContext(
  1515. structuredInputs,
  1516. mapperOutput,
  1517. preparer->GetClient()->GetRawClient(),
  1518. preparer->GetClientRetryPolicy(),
  1519. preparer->GetTransactionId()),
  1520. &structuredInputs,
  1521. /* outputs */ nullptr,
  1522. hints);
  1523. auto nodeReaderFormat = NodeReaderFormatFromHintAndGlobalConfig(spec.MapperFormatHints_);
  1524. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1525. *mapper,
  1526. EIODirection::Input,
  1527. structuredInputs,
  1528. hints.InputFormatHints_,
  1529. nodeReaderFormat,
  1530. /* allowFormatFromTableAttribute */ true);
  1531. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1532. *mapper,
  1533. EIODirection::Output,
  1534. mapperOutput,
  1535. hints.OutputFormatHints_,
  1536. ENodeReaderFormat::Yson,
  1537. /* allowFormatFromTableAttribute */ false);
  1538. operationIo.MapperJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1539. operationIo.MapperInputFormat = inputFormat;
  1540. operationIo.MapperOutputFormat = outputFormat;
  1541. Y_ABORT_UNLESS(mapperInferenceResult.size() >= 1);
  1542. currentInferenceResult = TVector<TTableSchema>{mapperInferenceResult[0]};
  1543. // The first output as it corresponds to the intermediate data.
  1544. TVector<TTableSchema> additionalOutputsInferenceResult(mapperInferenceResult.begin() + 1, mapperInferenceResult.end());
  1545. operationIo.MapOutputs = GetPathList(
  1546. structuredMapOutputs,
  1547. additionalOutputsInferenceResult,
  1548. inferOutputSchema);
  1549. }
  1550. if (reduceCombiner) {
  1551. const bool isFirstStep = !mapper;
  1552. TStructuredJobTableList inputs;
  1553. if (isFirstStep) {
  1554. inputs = structuredInputs;
  1555. } else {
  1556. auto reduceCombinerIntermediateInput =
  1557. spec.GetIntermediateReduceCombinerInputDescription()
  1558. .GetOrElse(TUnspecifiedTableStructure());
  1559. inputs = {
  1560. TStructuredJobTable::Intermediate(reduceCombinerIntermediateInput),
  1561. };
  1562. }
  1563. auto reduceCombinerOutputDescription = spec.GetIntermediateReduceCombinerOutputDescription()
  1564. .GetOrElse(TUnspecifiedTableStructure());
  1565. TStructuredJobTableList outputs = {
  1566. TStructuredJobTable::Intermediate(reduceCombinerOutputDescription),
  1567. };
  1568. auto hints = spec.ReduceCombinerFormatHints_;
  1569. if (isFirstStep) {
  1570. currentInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1571. *reduceCombiner,
  1572. TOperationPreparationContext(
  1573. inputs,
  1574. outputs,
  1575. preparer->GetClient()->GetRawClient(),
  1576. preparer->GetClientRetryPolicy(),
  1577. preparer->GetTransactionId()),
  1578. &inputs,
  1579. /* outputs */ nullptr,
  1580. hints);
  1581. } else {
  1582. currentInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1583. *reduceCombiner,
  1584. TSpeculativeOperationPreparationContext(
  1585. currentInferenceResult,
  1586. inputs,
  1587. outputs),
  1588. /* inputs */ nullptr,
  1589. /* outputs */ nullptr,
  1590. hints);
  1591. }
  1592. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1593. *reduceCombiner,
  1594. EIODirection::Input,
  1595. inputs,
  1596. hints.InputFormatHints_,
  1597. ENodeReaderFormat::Yson,
  1598. /* allowFormatFromTableAttribute = */ isFirstStep);
  1599. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1600. *reduceCombiner,
  1601. EIODirection::Output,
  1602. outputs,
  1603. hints.OutputFormatHints_,
  1604. ENodeReaderFormat::Yson,
  1605. /* allowFormatFromTableAttribute = */ false);
  1606. operationIo.ReduceCombinerJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1607. operationIo.ReduceCombinerInputFormat = inputFormat;
  1608. operationIo.ReduceCombinerOutputFormat = outputFormat;
  1609. if (isFirstStep) {
  1610. fixSpec(*operationIo.ReduceCombinerInputFormat);
  1611. }
  1612. }
  1613. const bool isFirstStep = (!mapper && !reduceCombiner);
  1614. TStructuredJobTableList reducerInputs;
  1615. if (isFirstStep) {
  1616. reducerInputs = structuredInputs;
  1617. } else {
  1618. auto reducerInputDescription =
  1619. spec.GetIntermediateReducerInputDescription()
  1620. .GetOrElse(TUnspecifiedTableStructure());
  1621. reducerInputs = {
  1622. TStructuredJobTable::Intermediate(reducerInputDescription),
  1623. };
  1624. }
  1625. auto hints = spec.ReducerFormatHints_;
  1626. TVector<TTableSchema> reducerInferenceResult;
  1627. if (isFirstStep) {
  1628. reducerInferenceResult = PrepareOperation(
  1629. *reducer,
  1630. TOperationPreparationContext(
  1631. structuredInputs,
  1632. structuredOutputs,
  1633. preparer->GetClient()->GetRawClient(),
  1634. preparer->GetClientRetryPolicy(),
  1635. preparer->GetTransactionId()),
  1636. &structuredInputs,
  1637. &structuredOutputs,
  1638. hints);
  1639. } else {
  1640. reducerInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1641. *reducer,
  1642. TSpeculativeOperationPreparationContext(
  1643. currentInferenceResult,
  1644. reducerInputs,
  1645. structuredOutputs),
  1646. /* inputs */ nullptr,
  1647. &structuredOutputs,
  1648. hints);
  1649. }
  1650. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1651. *reducer,
  1652. EIODirection::Input,
  1653. reducerInputs,
  1654. hints.InputFormatHints_,
  1655. ENodeReaderFormat::Yson,
  1656. /* allowFormatFromTableAttribute = */ isFirstStep);
  1657. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1658. *reducer,
  1659. EIODirection::Output,
  1660. ToStructuredJobTableList(spec.GetStructuredOutputs()),
  1661. hints.OutputFormatHints_,
  1662. ENodeReaderFormat::Yson,
  1663. /* allowFormatFromTableAttribute = */ false);
  1664. operationIo.ReducerJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1665. operationIo.ReducerInputFormat = inputFormat;
  1666. operationIo.ReducerOutputFormat = outputFormat;
  1667. if (isFirstStep) {
  1668. fixSpec(operationIo.ReducerInputFormat);
  1669. }
  1670. operationIo.Inputs = GetPathList(
  1671. ApplyProtobufColumnFilters(
  1672. structuredInputs,
  1673. *preparer,
  1674. GetColumnsUsedInOperation(spec),
  1675. options),
  1676. /* jobSchemaInferenceResult */ Nothing(),
  1677. /* inferSchema */ false);
  1678. operationIo.Outputs = GetPathList(
  1679. structuredOutputs,
  1680. reducerInferenceResult,
  1681. inferOutputSchema);
  1682. VerifyHasElements(operationIo.Outputs, "outputs");
  1683. return DoExecuteMapReduce(
  1684. operation,
  1685. preparer,
  1686. operationIo,
  1687. spec,
  1688. mapper,
  1689. reduceCombiner,
  1690. reducer,
  1691. options);
  1692. }
  1693. void ExecuteRawMapReduce(
  1694. const TOperationPtr& operation,
  1695. const TOperationPreparerPtr& preparer,
  1696. const TRawMapReduceOperationSpec& spec,
  1697. const ::TIntrusivePtr<IRawJob>& mapper,
  1698. const ::TIntrusivePtr<IRawJob>& reduceCombiner,
  1699. const ::TIntrusivePtr<IRawJob>& reducer,
  1700. const TOperationOptions& options)
  1701. {
  1702. YT_LOG_DEBUG("Starting raw map-reduce operation (PreparationId: %v)",
  1703. preparer->GetPreparationId());
  1704. TMapReduceOperationIo operationIo;
  1705. operationIo.Inputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.GetInputs());
  1706. operationIo.MapOutputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.GetMapOutputs());
  1707. operationIo.Outputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.GetOutputs());
  1708. VerifyHasElements(operationIo.Inputs, "inputs");
  1709. VerifyHasElements(operationIo.Outputs, "outputs");
  1710. auto getFormatOrDefault = [&] (const TMaybe<TFormat>& maybeFormat, const TMaybe<TFormat> stageDefaultFormat, const char* formatName) {
  1711. if (maybeFormat) {
  1712. return *maybeFormat;
  1713. } else if (stageDefaultFormat) {
  1714. return *stageDefaultFormat;
  1715. } else {
  1716. ythrow TApiUsageError() << "Cannot derive " << formatName;
  1717. }
  1718. };
  1719. if (mapper) {
  1720. operationIo.MapperInputFormat = getFormatOrDefault(spec.MapperInputFormat_, spec.MapperFormat_, "mapper input format");
  1721. operationIo.MapperOutputFormat = getFormatOrDefault(spec.MapperOutputFormat_, spec.MapperFormat_, "mapper output format");
  1722. }
  1723. if (reduceCombiner) {
  1724. operationIo.ReduceCombinerInputFormat = getFormatOrDefault(spec.ReduceCombinerInputFormat_, spec.ReduceCombinerFormat_, "reduce combiner input format");
  1725. operationIo.ReduceCombinerOutputFormat = getFormatOrDefault(spec.ReduceCombinerOutputFormat_, spec.ReduceCombinerFormat_, "reduce combiner output format");
  1726. }
  1727. operationIo.ReducerInputFormat = getFormatOrDefault(spec.ReducerInputFormat_, spec.ReducerFormat_, "reducer input format");
  1728. operationIo.ReducerOutputFormat = getFormatOrDefault(spec.ReducerOutputFormat_, spec.ReducerFormat_, "reducer output format");
  1729. return DoExecuteMapReduce(
  1730. operation,
  1731. preparer,
  1732. operationIo,
  1733. spec,
  1734. mapper,
  1735. reduceCombiner,
  1736. reducer,
  1737. options);
  1738. }
  1739. void ExecuteSort(
  1740. const TOperationPtr& operation,
  1741. const TOperationPreparerPtr& preparer,
  1742. const TSortOperationSpec& spec,
  1743. const TOperationOptions& options)
  1744. {
  1745. YT_LOG_DEBUG("Starting sort operation (PreparationId: %v)",
  1746. preparer->GetPreparationId());
  1747. auto inputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.Inputs_);
  1748. auto output = NRawClient::CanonizeYPath(preparer->GetClient()->GetRawClient(), spec.Output_);
  1749. if (options.CreateOutputTables_) {
  1750. CheckInputTablesExist(*preparer, inputs);
  1751. CreateOutputTable(*preparer, output);
  1752. }
  1753. TNode specNode = BuildYsonNodeFluently()
  1754. .BeginMap()
  1755. .Item("input_table_paths").List(inputs)
  1756. .Item("output_table_path").Value(output)
  1757. .Item("sort_by").Value(spec.SortBy_)
  1758. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1759. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1760. })
  1761. .EndMap();
  1762. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1763. BuildPartitionCountOperationPart(spec, &specNode);
  1764. BuildPartitionJobCountOperationPart(spec, &specNode);
  1765. BuildIntermediateDataPart(spec, &specNode);
  1766. auto startOperation = [
  1767. operation=operation.Get(),
  1768. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1769. preparer,
  1770. inputs,
  1771. output
  1772. ] () {
  1773. auto operationId = preparer->StartOperation(operation, EOperationType::Sort, spec);
  1774. LogYPaths(operationId, inputs, "input");
  1775. LogYPath(operationId, output, "output");
  1776. return operationId;
  1777. };
  1778. operation->SetDelayedStartFunction(std::move(startOperation));
  1779. }
  1780. void ExecuteMerge(
  1781. const TOperationPtr& operation,
  1782. const TOperationPreparerPtr& preparer,
  1783. const TMergeOperationSpec& spec,
  1784. const TOperationOptions& options)
  1785. {
  1786. YT_LOG_DEBUG("Starting merge operation (PreparationId: %v)",
  1787. preparer->GetPreparationId());
  1788. auto inputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.Inputs_);
  1789. auto output = NRawClient::CanonizeYPath(preparer->GetClient()->GetRawClient(), spec.Output_);
  1790. if (options.CreateOutputTables_) {
  1791. CheckInputTablesExist(*preparer, inputs);
  1792. CreateOutputTable(*preparer, output);
  1793. }
  1794. TNode specNode = BuildYsonNodeFluently()
  1795. .BeginMap()
  1796. .Item("input_table_paths").List(inputs)
  1797. .Item("output_table_path").Value(output)
  1798. .Item("mode").Value(ToString(spec.Mode_))
  1799. .Item("combine_chunks").Value(spec.CombineChunks_)
  1800. .Item("force_transform").Value(spec.ForceTransform_)
  1801. .Item("merge_by").Value(spec.MergeBy_)
  1802. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1803. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1804. })
  1805. .EndMap();
  1806. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1807. BuildJobCountOperationPart(spec, &specNode);
  1808. auto startOperation = [
  1809. operation=operation.Get(),
  1810. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1811. preparer,
  1812. inputs,
  1813. output
  1814. ] () {
  1815. auto operationId = preparer->StartOperation(operation, EOperationType::Merge, spec);
  1816. LogYPaths(operationId, inputs, "input");
  1817. LogYPath(operationId, output, "output");
  1818. return operationId;
  1819. };
  1820. operation->SetDelayedStartFunction(std::move(startOperation));
  1821. }
  1822. void ExecuteErase(
  1823. const TOperationPtr& operation,
  1824. const TOperationPreparerPtr& preparer,
  1825. const TEraseOperationSpec& spec,
  1826. const TOperationOptions& options)
  1827. {
  1828. YT_LOG_DEBUG("Starting erase operation (PreparationId: %v)",
  1829. preparer->GetPreparationId());
  1830. auto tablePath = NRawClient::CanonizeYPath(preparer->GetClient()->GetRawClient(), spec.TablePath_);
  1831. TNode specNode = BuildYsonNodeFluently()
  1832. .BeginMap()
  1833. .Item("table_path").Value(tablePath)
  1834. .Item("combine_chunks").Value(spec.CombineChunks_)
  1835. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1836. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1837. })
  1838. .EndMap();
  1839. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1840. auto startOperation = [
  1841. operation=operation.Get(),
  1842. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1843. preparer,
  1844. tablePath
  1845. ] () {
  1846. auto operationId = preparer->StartOperation(operation, EOperationType::Erase, spec);
  1847. LogYPath(operationId, tablePath, "table_path");
  1848. return operationId;
  1849. };
  1850. operation->SetDelayedStartFunction(std::move(startOperation));
  1851. }
  1852. void ExecuteRemoteCopy(
  1853. const TOperationPtr& operation,
  1854. const TOperationPreparerPtr& preparer,
  1855. const TRemoteCopyOperationSpec& spec,
  1856. const TOperationOptions& options)
  1857. {
  1858. YT_LOG_DEBUG("Starting remote copy operation (PreparationId: %v)",
  1859. preparer->GetPreparationId());
  1860. auto inputs = NRawClient::CanonizeYPaths(preparer->GetClient()->GetRawClient(), spec.Inputs_);
  1861. auto output = NRawClient::CanonizeYPath(preparer->GetClient()->GetRawClient(), spec.Output_);
  1862. if (options.CreateOutputTables_) {
  1863. CreateOutputTable(*preparer, output);
  1864. }
  1865. Y_ENSURE_EX(!spec.ClusterName_.empty(), TApiUsageError() << "ClusterName parameter is required");
  1866. TNode specNode = BuildYsonNodeFluently()
  1867. .BeginMap()
  1868. .Item("cluster_name").Value(spec.ClusterName_)
  1869. .Item("input_table_paths").List(inputs)
  1870. .Item("output_table_path").Value(output)
  1871. .DoIf(spec.NetworkName_.Defined(), [&] (TFluentMap fluent) {
  1872. fluent.Item("network_name").Value(*spec.NetworkName_);
  1873. })
  1874. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1875. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1876. })
  1877. .Item("copy_attributes").Value(spec.CopyAttributes_)
  1878. .DoIf(!spec.AttributeKeys_.empty(), [&] (TFluentMap fluent) {
  1879. Y_ENSURE_EX(spec.CopyAttributes_, TApiUsageError() <<
  1880. "Specifying nonempty AttributeKeys in RemoteCopy "
  1881. "doesn't make sense without CopyAttributes == true");
  1882. fluent.Item("attribute_keys").List(spec.AttributeKeys_);
  1883. })
  1884. .EndMap();
  1885. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1886. auto startOperation = [
  1887. operation=operation.Get(),
  1888. spec=MergeSpec(specNode, preparer->GetContext().Config->Spec, options),
  1889. preparer,
  1890. inputs,
  1891. output
  1892. ] () {
  1893. auto operationId = preparer->StartOperation(operation, EOperationType::RemoteCopy, spec);
  1894. LogYPaths(operationId, inputs, "input");
  1895. LogYPath(operationId, output, "output");
  1896. return operationId;
  1897. };
  1898. operation->SetDelayedStartFunction(std::move(startOperation));
  1899. }
  1900. void ExecuteVanilla(
  1901. const TOperationPtr& operation,
  1902. const TOperationPreparerPtr& preparer,
  1903. const TVanillaOperationSpec& spec,
  1904. const TOperationOptions& options)
  1905. {
  1906. YT_LOG_DEBUG("Starting vanilla operation (PreparationId: %v)",
  1907. preparer->GetPreparationId());
  1908. auto addTask = [&](TFluentMap fluent, const TVanillaTask& task) {
  1909. Y_ABORT_UNLESS(task.Job_.Get());
  1910. if (std::holds_alternative<TVoidStructuredRowStream>(task.Job_->GetOutputRowStreamDescription())) {
  1911. Y_ENSURE_EX(task.Outputs_.empty(),
  1912. TApiUsageError() << "Vanilla task with void IVanillaJob doesn't expect output tables");
  1913. TJobPreparer jobPreparer(
  1914. *preparer,
  1915. task.Spec_,
  1916. *task.Job_,
  1917. /* outputTableCount */ 0,
  1918. /* smallFileList */ {},
  1919. options);
  1920. fluent
  1921. .Item(task.Name_).BeginMap()
  1922. .Item("job_count").Value(task.JobCount_)
  1923. .DoIf(task.NetworkProject_.Defined(), [&](TFluentMap fluent) {
  1924. fluent.Item("network_project").Value(*task.NetworkProject_);
  1925. })
  1926. .Do([&] (TFluentMap fluent) {
  1927. BuildUserJobFluently(
  1928. std::cref(jobPreparer),
  1929. /* inputFormat */ Nothing(),
  1930. /* outputFormat */ Nothing(),
  1931. fluent);
  1932. })
  1933. .EndMap();
  1934. } else {
  1935. auto operationIo = CreateSimpleOperationIo(
  1936. *task.Job_,
  1937. *preparer,
  1938. task,
  1939. options,
  1940. false);
  1941. Y_ENSURE_EX(operationIo.Outputs.size() > 0,
  1942. TApiUsageError() << "Vanilla task with IVanillaJob that has table writer expects output tables");
  1943. if (options.CreateOutputTables_) {
  1944. CreateOutputTables(*preparer, operationIo.Outputs);
  1945. }
  1946. TJobPreparer jobPreparer(
  1947. *preparer,
  1948. task.Spec_,
  1949. *task.Job_,
  1950. operationIo.Outputs.size(),
  1951. operationIo.JobFiles,
  1952. options);
  1953. fluent
  1954. .Item(task.Name_).BeginMap()
  1955. .Item("job_count").Value(task.JobCount_)
  1956. .DoIf(task.NetworkProject_.Defined(), [&](TFluentMap fluent) {
  1957. fluent.Item("network_project").Value(*task.NetworkProject_);
  1958. })
  1959. .Do([&] (TFluentMap fluent) {
  1960. BuildUserJobFluently(
  1961. std::cref(jobPreparer),
  1962. /* inputFormat */ Nothing(),
  1963. operationIo.OutputFormat,
  1964. fluent);
  1965. })
  1966. .Item("output_table_paths").List(operationIo.Outputs)
  1967. .Item("job_io").BeginMap()
  1968. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&](TFluentMap fluent) {
  1969. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1970. })
  1971. .Item("control_attributes").BeginMap()
  1972. .Item("enable_row_index").Value(TNode(true))
  1973. .Item("enable_range_index").Value(TNode(true))
  1974. .EndMap()
  1975. .EndMap()
  1976. .EndMap();
  1977. }
  1978. };
  1979. if (options.CreateDebugOutputTables_) {
  1980. CreateDebugOutputTables(spec, *preparer);
  1981. }
  1982. TNode specNode = BuildYsonNodeFluently()
  1983. .BeginMap()
  1984. .Item("tasks").DoMapFor(spec.Tasks_, addTask)
  1985. .EndMap();
  1986. BuildCommonOperationPart(preparer->GetContext().Config, spec, options, &specNode);
  1987. BuildCommonUserOperationPart(spec, &specNode);
  1988. auto startOperation = [operation=operation.Get(), spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options), preparer] () {
  1989. auto operationId = preparer->StartOperation(operation, EOperationType::Vanilla, spec);
  1990. return operationId;
  1991. };
  1992. operation->SetDelayedStartFunction(std::move(startOperation));
  1993. }
  1994. ////////////////////////////////////////////////////////////////////////////////
  1995. class TOperation::TOperationImpl
  1996. : public TThrRefBase
  1997. {
  1998. public:
  1999. TOperationImpl(
  2000. IRawClientPtr rawClient,
  2001. IClientRetryPolicyPtr clientRetryPolicy,
  2002. TClientContext context,
  2003. const TMaybe<TOperationId>& operationId = {})
  2004. : RawClient_(std::move(rawClient))
  2005. , Context_(std::move(context))
  2006. , ClientRetryPolicy_(clientRetryPolicy)
  2007. , Id_(operationId)
  2008. , PreparedPromise_(::NThreading::NewPromise<void>())
  2009. , StartedPromise_(::NThreading::NewPromise<void>())
  2010. {
  2011. if (Id_) {
  2012. PreparedPromise_.SetValue();
  2013. StartedPromise_.SetValue();
  2014. } else {
  2015. PreparedPromise_.GetFuture().Subscribe([this_=::TIntrusivePtr(this)] (const ::NThreading::TFuture<void>& preparedResult) {
  2016. try {
  2017. preparedResult.GetValue();
  2018. } catch (...) {
  2019. this_->StartedPromise_.SetException(std::current_exception());
  2020. return;
  2021. }
  2022. });
  2023. }
  2024. }
  2025. const TOperationId& GetId() const;
  2026. TString GetWebInterfaceUrl() const;
  2027. void OnPrepared();
  2028. void SetDelayedStartFunction(std::function<TOperationId()> start);
  2029. void Start();
  2030. bool IsStarted() const;
  2031. void OnPreparationException(std::exception_ptr e);
  2032. TString GetStatus();
  2033. void OnStatusUpdated(const TString& newStatus);
  2034. ::NThreading::TFuture<void> GetPreparedFuture();
  2035. ::NThreading::TFuture<void> GetStartedFuture();
  2036. ::NThreading::TFuture<void> Watch(TClientPtr client);
  2037. EOperationBriefState GetBriefState();
  2038. TMaybe<TYtError> GetError();
  2039. TJobStatistics GetJobStatistics();
  2040. TMaybe<TOperationBriefProgress> GetBriefProgress();
  2041. void AbortOperation();
  2042. void CompleteOperation();
  2043. void SuspendOperation(const TSuspendOperationOptions& options);
  2044. void ResumeOperation(const TResumeOperationOptions& options);
  2045. TOperationAttributes GetAttributes(const TGetOperationOptions& options);
  2046. void UpdateParameters(const TUpdateOperationParametersOptions& options);
  2047. TJobAttributes GetJob(const TJobId& jobId, const TGetJobOptions& options);
  2048. TListJobsResult ListJobs(const TListJobsOptions& options);
  2049. void AsyncFinishOperation(TOperationAttributes operationAttributes);
  2050. void FinishWithException(std::exception_ptr exception);
  2051. void UpdateBriefProgress(TMaybe<TOperationBriefProgress> briefProgress);
  2052. void AnalyzeUnrecognizedSpec(TNode unrecognizedSpec);
  2053. const TClientContext& GetContext() const;
  2054. private:
  2055. void OnStarted(const TOperationId& operationId);
  2056. void UpdateAttributesAndCall(bool needJobStatistics, std::function<void(const TOperationAttributes&)> func);
  2057. void SyncFinishOperationImpl(const TOperationAttributes&);
  2058. static void* SyncFinishOperationProc(void* );
  2059. void ValidateOperationStarted() const;
  2060. private:
  2061. const IRawClientPtr RawClient_;
  2062. const TClientContext Context_;
  2063. IClientRetryPolicyPtr ClientRetryPolicy_;
  2064. TMaybe<TOperationId> Id_;
  2065. TMutex Lock_;
  2066. ::NThreading::TPromise<void> PreparedPromise_;
  2067. ::NThreading::TPromise<void> StartedPromise_;
  2068. TMaybe<::NThreading::TPromise<void>> CompletePromise_;
  2069. std::function<TOperationId()> DelayedStartFunction_;
  2070. TString Status_;
  2071. TOperationAttributes Attributes_;
  2072. };
  2073. ////////////////////////////////////////////////////////////////////////////////
  2074. class TOperationPollerItem
  2075. : public IYtPollerItem
  2076. {
  2077. public:
  2078. TOperationPollerItem(::TIntrusivePtr<TOperation::TOperationImpl> operationImpl)
  2079. : OperationImpl_(std::move(operationImpl))
  2080. { }
  2081. void PrepareRequest(IRawBatchRequest* batchRequest) override
  2082. {
  2083. auto filter = TOperationAttributeFilter()
  2084. .Add(EOperationAttribute::State)
  2085. .Add(EOperationAttribute::BriefProgress)
  2086. .Add(EOperationAttribute::Result);
  2087. if (!UnrecognizedSpecAnalyzed_) {
  2088. filter.Add(EOperationAttribute::UnrecognizedSpec);
  2089. }
  2090. OperationState_ = batchRequest->GetOperation(
  2091. OperationImpl_->GetId(),
  2092. TGetOperationOptions().AttributeFilter(filter));
  2093. }
  2094. EStatus OnRequestExecuted() override
  2095. {
  2096. try {
  2097. const auto& attributes = OperationState_.GetValue();
  2098. if (!UnrecognizedSpecAnalyzed_ && !attributes.UnrecognizedSpec.Empty()) {
  2099. OperationImpl_->AnalyzeUnrecognizedSpec(*attributes.UnrecognizedSpec);
  2100. UnrecognizedSpecAnalyzed_ = true;
  2101. }
  2102. Y_ABORT_UNLESS(attributes.BriefState,
  2103. "get_operation for operation %s has not returned \"state\" field",
  2104. GetGuidAsString(OperationImpl_->GetId()).data());
  2105. if (*attributes.BriefState != EOperationBriefState::InProgress) {
  2106. OperationImpl_->AsyncFinishOperation(attributes);
  2107. return PollBreak;
  2108. } else {
  2109. OperationImpl_->UpdateBriefProgress(attributes.BriefProgress);
  2110. }
  2111. } catch (const TErrorResponse& e) {
  2112. if (!IsRetriable(e)) {
  2113. OperationImpl_->FinishWithException(std::current_exception());
  2114. return PollBreak;
  2115. }
  2116. } catch (const std::exception& e) {
  2117. OperationImpl_->FinishWithException(std::current_exception());
  2118. return PollBreak;
  2119. }
  2120. return PollContinue;
  2121. }
  2122. void OnItemDiscarded() override {
  2123. OperationImpl_->FinishWithException(std::make_exception_ptr(yexception() << "Operation cancelled"));
  2124. }
  2125. private:
  2126. ::TIntrusivePtr<TOperation::TOperationImpl> OperationImpl_;
  2127. ::NThreading::TFuture<TOperationAttributes> OperationState_;
  2128. bool UnrecognizedSpecAnalyzed_ = false;
  2129. };
  2130. ////////////////////////////////////////////////////////////////////////////////
  2131. const TOperationId& TOperation::TOperationImpl::GetId() const
  2132. {
  2133. ValidateOperationStarted();
  2134. return *Id_;
  2135. }
  2136. TString TOperation::TOperationImpl::GetWebInterfaceUrl() const
  2137. {
  2138. ValidateOperationStarted();
  2139. return GetOperationWebInterfaceUrl(Context_.ServerName, *Id_);
  2140. }
  2141. void TOperation::TOperationImpl::OnPrepared()
  2142. {
  2143. Y_ABORT_IF(PreparedPromise_.HasException());
  2144. Y_ABORT_IF(PreparedPromise_.HasValue());
  2145. PreparedPromise_.SetValue();
  2146. }
  2147. void TOperation::TOperationImpl::SetDelayedStartFunction(std::function<TOperationId()> start)
  2148. {
  2149. DelayedStartFunction_ = std::move(start);
  2150. }
  2151. void TOperation::TOperationImpl::Start()
  2152. {
  2153. {
  2154. auto guard = Guard(Lock_);
  2155. if (Id_) {
  2156. ythrow TApiUsageError() << "Start() should not be called on running operations";
  2157. }
  2158. }
  2159. GetPreparedFuture().GetValueSync();
  2160. std::function<TOperationId()> startStuff;
  2161. {
  2162. auto guard = Guard(Lock_);
  2163. startStuff.swap(DelayedStartFunction_);
  2164. }
  2165. if (!startStuff) {
  2166. ythrow TApiUsageError() << "Seems that Start() was called multiple times. If not, contact yt@";
  2167. }
  2168. TOperationId operationId;
  2169. try {
  2170. operationId = startStuff();
  2171. } catch (...) {
  2172. auto exception = std::current_exception();
  2173. StartedPromise_.SetException(exception);
  2174. std::rethrow_exception(exception);
  2175. }
  2176. OnStarted(operationId);
  2177. }
  2178. bool TOperation::TOperationImpl::IsStarted() const {
  2179. auto guard = Guard(Lock_);
  2180. return bool(Id_);
  2181. }
  2182. void TOperation::TOperationImpl::OnPreparationException(std::exception_ptr e)
  2183. {
  2184. Y_ABORT_IF(PreparedPromise_.HasValue());
  2185. Y_ABORT_IF(PreparedPromise_.HasException());
  2186. PreparedPromise_.SetException(e);
  2187. }
  2188. TString TOperation::TOperationImpl::GetStatus()
  2189. {
  2190. {
  2191. auto guard = Guard(Lock_);
  2192. if (!Id_) {
  2193. return Status_;
  2194. }
  2195. }
  2196. TMaybe<TString> state;
  2197. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2198. state = attributes.State;
  2199. });
  2200. return "On YT cluster: " + state.GetOrElse("undefined state");
  2201. }
  2202. void TOperation::TOperationImpl::OnStatusUpdated(const TString& newStatus)
  2203. {
  2204. auto guard = Guard(Lock_);
  2205. Status_ = newStatus;
  2206. }
  2207. ::NThreading::TFuture<void> TOperation::TOperationImpl::GetPreparedFuture()
  2208. {
  2209. return PreparedPromise_.GetFuture();
  2210. }
  2211. ::NThreading::TFuture<void> TOperation::TOperationImpl::GetStartedFuture()
  2212. {
  2213. return StartedPromise_.GetFuture();
  2214. }
  2215. ::NThreading::TFuture<void> TOperation::TOperationImpl::Watch(TClientPtr client)
  2216. {
  2217. {
  2218. auto guard = Guard(Lock_);
  2219. if (CompletePromise_) {
  2220. return *CompletePromise_;
  2221. }
  2222. CompletePromise_ = ::NThreading::NewPromise<void>();
  2223. }
  2224. GetStartedFuture().Subscribe([
  2225. this_=::TIntrusivePtr(this),
  2226. client=std::move(client)
  2227. ] (const ::NThreading::TFuture<void>& startedResult) {
  2228. try {
  2229. startedResult.GetValue();
  2230. } catch (...) {
  2231. this_->CompletePromise_->SetException(std::current_exception());
  2232. return;
  2233. }
  2234. client->GetYtPoller().Watch(::MakeIntrusive<TOperationPollerItem>(this_));
  2235. auto operationId = this_->GetId();
  2236. auto registry = TAbortableRegistry::Get();
  2237. registry->Add(
  2238. operationId,
  2239. ::MakeIntrusive<TOperationAbortable>(this_->RawClient_, this_->ClientRetryPolicy_, operationId));
  2240. // We have to own an IntrusivePtr to registry to prevent use-after-free
  2241. auto removeOperation = [registry, operationId] (const ::NThreading::TFuture<void>&) {
  2242. registry->Remove(operationId);
  2243. };
  2244. this_->CompletePromise_->GetFuture().Subscribe(removeOperation);
  2245. });
  2246. return *CompletePromise_;
  2247. }
  2248. EOperationBriefState TOperation::TOperationImpl::GetBriefState()
  2249. {
  2250. ValidateOperationStarted();
  2251. EOperationBriefState result = EOperationBriefState::InProgress;
  2252. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2253. Y_ABORT_UNLESS(attributes.BriefState,
  2254. "get_operation for operation %s has not returned \"state\" field",
  2255. GetGuidAsString(*Id_).data());
  2256. result = *attributes.BriefState;
  2257. });
  2258. return result;
  2259. }
  2260. TMaybe<TYtError> TOperation::TOperationImpl::GetError()
  2261. {
  2262. ValidateOperationStarted();
  2263. TMaybe<TYtError> result;
  2264. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2265. Y_ABORT_UNLESS(attributes.Result);
  2266. result = attributes.Result->Error;
  2267. });
  2268. return result;
  2269. }
  2270. TJobStatistics TOperation::TOperationImpl::GetJobStatistics()
  2271. {
  2272. ValidateOperationStarted();
  2273. TJobStatistics result;
  2274. UpdateAttributesAndCall(true, [&] (const TOperationAttributes& attributes) {
  2275. if (attributes.Progress) {
  2276. result = attributes.Progress->JobStatistics;
  2277. }
  2278. });
  2279. return result;
  2280. }
  2281. TMaybe<TOperationBriefProgress> TOperation::TOperationImpl::GetBriefProgress()
  2282. {
  2283. ValidateOperationStarted();
  2284. {
  2285. auto g = Guard(Lock_);
  2286. if (CompletePromise_.Defined()) {
  2287. // Poller do this job for us
  2288. return Attributes_.BriefProgress;
  2289. }
  2290. }
  2291. TMaybe<TOperationBriefProgress> result;
  2292. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2293. result = attributes.BriefProgress;
  2294. });
  2295. return result;
  2296. }
  2297. void TOperation::TOperationImpl::UpdateBriefProgress(TMaybe<TOperationBriefProgress> briefProgress)
  2298. {
  2299. auto g = Guard(Lock_);
  2300. Attributes_.BriefProgress = std::move(briefProgress);
  2301. }
  2302. void TOperation::TOperationImpl::AnalyzeUnrecognizedSpec(TNode unrecognizedSpec)
  2303. {
  2304. static const TVector<TVector<TString>> knownUnrecognizedSpecFieldPaths = {
  2305. {"mapper", "class_name"},
  2306. {"reducer", "class_name"},
  2307. {"reduce_combiner", "class_name"},
  2308. };
  2309. auto removeByPath = [] (TNode& node, auto pathBegin, auto pathEnd, auto& removeByPath) {
  2310. if (pathBegin == pathEnd) {
  2311. return;
  2312. }
  2313. if (!node.IsMap()) {
  2314. return;
  2315. }
  2316. auto* child = node.AsMap().FindPtr(*pathBegin);
  2317. if (!child) {
  2318. return;
  2319. }
  2320. removeByPath(*child, std::next(pathBegin), pathEnd, removeByPath);
  2321. if (std::next(pathBegin) == pathEnd || (child->IsMap() && child->Empty())) {
  2322. node.AsMap().erase(*pathBegin);
  2323. }
  2324. };
  2325. Y_ABORT_UNLESS(unrecognizedSpec.IsMap());
  2326. for (const auto& knownFieldPath : knownUnrecognizedSpecFieldPaths) {
  2327. Y_ABORT_UNLESS(!knownFieldPath.empty());
  2328. removeByPath(unrecognizedSpec, knownFieldPath.cbegin(), knownFieldPath.cend(), removeByPath);
  2329. }
  2330. if (!unrecognizedSpec.Empty()) {
  2331. YT_LOG_INFO(
  2332. "WARNING! Unrecognized spec for operation %s is not empty "
  2333. "(fields added by the YT API library are excluded): %s",
  2334. GetGuidAsString(*Id_).data(),
  2335. NodeToYsonString(unrecognizedSpec).data());
  2336. }
  2337. }
  2338. void TOperation::TOperationImpl::OnStarted(const TOperationId& operationId)
  2339. {
  2340. auto guard = Guard(Lock_);
  2341. Y_ABORT_UNLESS(!Id_,
  2342. "OnStarted() called with operationId = %s for operation with id %s",
  2343. GetGuidAsString(operationId).data(),
  2344. GetGuidAsString(*Id_).data());
  2345. Id_ = operationId;
  2346. Y_ABORT_UNLESS(!StartedPromise_.HasValue() && !StartedPromise_.HasException());
  2347. StartedPromise_.SetValue();
  2348. }
  2349. void TOperation::TOperationImpl::UpdateAttributesAndCall(
  2350. bool needJobStatistics,
  2351. std::function<void(const TOperationAttributes&)> func)
  2352. {
  2353. {
  2354. auto g = Guard(Lock_);
  2355. if (Attributes_.BriefState
  2356. && *Attributes_.BriefState != EOperationBriefState::InProgress
  2357. && (!needJobStatistics || Attributes_.Progress))
  2358. {
  2359. func(Attributes_);
  2360. return;
  2361. }
  2362. }
  2363. auto attributes = RequestWithRetry<TOperationAttributes>(
  2364. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2365. [this] (TMutationId /*mutationId*/) {
  2366. return RawClient_->GetOperation(
  2367. *Id_,
  2368. TGetOperationOptions().AttributeFilter(TOperationAttributeFilter()
  2369. .Add(EOperationAttribute::Result)
  2370. .Add(EOperationAttribute::Progress)
  2371. .Add(EOperationAttribute::State)
  2372. .Add(EOperationAttribute::BriefProgress)));
  2373. });
  2374. func(attributes);
  2375. Y_ENSURE(attributes.BriefState);
  2376. if (*attributes.BriefState != EOperationBriefState::InProgress) {
  2377. auto g = Guard(Lock_);
  2378. Attributes_ = std::move(attributes);
  2379. }
  2380. }
  2381. void TOperation::TOperationImpl::FinishWithException(std::exception_ptr e)
  2382. {
  2383. CompletePromise_->SetException(std::move(e));
  2384. }
  2385. void TOperation::TOperationImpl::AbortOperation()
  2386. {
  2387. ValidateOperationStarted();
  2388. RequestWithRetry<void>(
  2389. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2390. [this] (TMutationId& mutationId) {
  2391. RawClient_->AbortOperation(mutationId, *Id_);
  2392. });
  2393. }
  2394. void TOperation::TOperationImpl::CompleteOperation()
  2395. {
  2396. ValidateOperationStarted();
  2397. RequestWithRetry<void>(
  2398. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2399. [this] (TMutationId& mutationId) {
  2400. RawClient_->CompleteOperation(mutationId, *Id_);
  2401. });
  2402. }
  2403. void TOperation::TOperationImpl::SuspendOperation(const TSuspendOperationOptions& options)
  2404. {
  2405. ValidateOperationStarted();
  2406. RequestWithRetry<void>(
  2407. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2408. [this, &options] (TMutationId& mutationId) {
  2409. RawClient_->SuspendOperation(mutationId, *Id_, options);
  2410. });
  2411. }
  2412. void TOperation::TOperationImpl::ResumeOperation(const TResumeOperationOptions& options)
  2413. {
  2414. ValidateOperationStarted();
  2415. RequestWithRetry<void>(
  2416. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2417. [this, &options] (TMutationId& mutationId) {
  2418. RawClient_->ResumeOperation(mutationId, *Id_, options);
  2419. });
  2420. }
  2421. TOperationAttributes TOperation::TOperationImpl::GetAttributes(const TGetOperationOptions& options)
  2422. {
  2423. ValidateOperationStarted();
  2424. return RequestWithRetry<TOperationAttributes>(
  2425. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2426. [this, &options] (TMutationId /*mutationId*/) {
  2427. return RawClient_->GetOperation(*Id_, options);
  2428. });
  2429. }
  2430. void TOperation::TOperationImpl::UpdateParameters(const TUpdateOperationParametersOptions& options)
  2431. {
  2432. ValidateOperationStarted();
  2433. RequestWithRetry<void>(
  2434. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2435. [this, &options] (TMutationId /*mutationId*/) {
  2436. RawClient_->UpdateOperationParameters(*Id_, options);
  2437. });
  2438. }
  2439. TJobAttributes TOperation::TOperationImpl::GetJob(const TJobId& jobId, const TGetJobOptions& options)
  2440. {
  2441. ValidateOperationStarted();
  2442. auto result = RequestWithRetry<NYson::TYsonString>(
  2443. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2444. [this, &jobId, &options] (TMutationId /*mutationId*/) {
  2445. return RawClient_->GetJob(*Id_, jobId, options);
  2446. });
  2447. return NRawClient::ParseJobAttributes(NodeFromYsonString(result.AsStringBuf()));
  2448. }
  2449. TListJobsResult TOperation::TOperationImpl::ListJobs(const TListJobsOptions& options)
  2450. {
  2451. ValidateOperationStarted();
  2452. return RequestWithRetry<TListJobsResult>(
  2453. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2454. [this, &options] (TMutationId /*mutationId*/) {
  2455. return RawClient_->ListJobs(*Id_, options);
  2456. });
  2457. }
  2458. struct TAsyncFinishOperationsArgs
  2459. {
  2460. ::TIntrusivePtr<TOperation::TOperationImpl> OperationImpl;
  2461. TOperationAttributes OperationAttributes;
  2462. };
  2463. void TOperation::TOperationImpl::AsyncFinishOperation(TOperationAttributes operationAttributes)
  2464. {
  2465. auto args = new TAsyncFinishOperationsArgs;
  2466. args->OperationImpl = this;
  2467. args->OperationAttributes = std::move(operationAttributes);
  2468. TThread thread(TThread::TParams(&TOperation::TOperationImpl::SyncFinishOperationProc, args).SetName("finish operation"));
  2469. thread.Start();
  2470. thread.Detach();
  2471. }
  2472. void* TOperation::TOperationImpl::SyncFinishOperationProc(void* pArgs)
  2473. {
  2474. std::unique_ptr<TAsyncFinishOperationsArgs> args(static_cast<TAsyncFinishOperationsArgs*>(pArgs));
  2475. args->OperationImpl->SyncFinishOperationImpl(args->OperationAttributes);
  2476. return nullptr;
  2477. }
  2478. void TOperation::TOperationImpl::SyncFinishOperationImpl(const TOperationAttributes& attributes)
  2479. {
  2480. {
  2481. auto guard = Guard(Lock_);
  2482. Y_ABORT_UNLESS(Id_);
  2483. }
  2484. Y_ABORT_UNLESS(attributes.BriefState,
  2485. "get_operation for operation %s has not returned \"state\" field",
  2486. GetGuidAsString(*Id_).data());
  2487. Y_ABORT_UNLESS(*attributes.BriefState != EOperationBriefState::InProgress);
  2488. {
  2489. try {
  2490. // `attributes' that came from poller don't have JobStatistics
  2491. // so we call `GetJobStatistics' in order to get it from server
  2492. // and cache inside object.
  2493. GetJobStatistics();
  2494. } catch (const TErrorResponse& ) {
  2495. // But if for any reason we failed to get attributes
  2496. // we complete operation using what we have.
  2497. auto g = Guard(Lock_);
  2498. Attributes_ = attributes;
  2499. }
  2500. }
  2501. if (*attributes.BriefState == EOperationBriefState::Completed) {
  2502. CompletePromise_->SetValue();
  2503. } else if (*attributes.BriefState == EOperationBriefState::Aborted || *attributes.BriefState == EOperationBriefState::Failed) {
  2504. Y_ABORT_UNLESS(attributes.Result && attributes.Result->Error);
  2505. const auto& error = *attributes.Result->Error;
  2506. YT_LOG_ERROR("Operation %v is `%v' with error: %v",
  2507. *Id_,
  2508. ToString(*attributes.BriefState),
  2509. error.FullDescription());
  2510. TString additionalExceptionText;
  2511. TVector<TFailedJobInfo> failedJobStderrInfo;
  2512. if (*attributes.BriefState == EOperationBriefState::Failed) {
  2513. try {
  2514. failedJobStderrInfo = NYT::NDetail::GetFailedJobInfo(ClientRetryPolicy_, RawClient_, *Id_, TGetFailedJobInfoOptions());
  2515. } catch (const std::exception& e) {
  2516. additionalExceptionText = "Cannot get job stderrs: ";
  2517. additionalExceptionText += e.what();
  2518. }
  2519. }
  2520. CompletePromise_->SetException(
  2521. std::make_exception_ptr(
  2522. TOperationFailedError(
  2523. *attributes.BriefState == EOperationBriefState::Failed
  2524. ? TOperationFailedError::Failed
  2525. : TOperationFailedError::Aborted,
  2526. *Id_,
  2527. error,
  2528. failedJobStderrInfo) << additionalExceptionText));
  2529. }
  2530. }
  2531. void TOperation::TOperationImpl::ValidateOperationStarted() const
  2532. {
  2533. auto guard = Guard(Lock_);
  2534. if (!Id_) {
  2535. ythrow TApiUsageError() << "Operation is not started";
  2536. }
  2537. }
  2538. const TClientContext& TOperation::TOperationImpl::GetContext() const
  2539. {
  2540. return Context_;
  2541. }
  2542. ////////////////////////////////////////////////////////////////////////////////
  2543. TOperation::TOperation(TClientPtr client)
  2544. : Client_(std::move(client))
  2545. , Impl_(::MakeIntrusive<TOperationImpl>(
  2546. Client_->GetRawClient(),
  2547. Client_->GetRetryPolicy(),
  2548. Client_->GetContext()))
  2549. {
  2550. }
  2551. TOperation::TOperation(TOperationId id, TClientPtr client)
  2552. : Client_(std::move(client))
  2553. , Impl_(::MakeIntrusive<TOperationImpl>(
  2554. Client_->GetRawClient(),
  2555. Client_->GetRetryPolicy(),
  2556. Client_->GetContext(), id))
  2557. {
  2558. }
  2559. const TOperationId& TOperation::GetId() const
  2560. {
  2561. return Impl_->GetId();
  2562. }
  2563. TString TOperation::GetWebInterfaceUrl() const
  2564. {
  2565. return Impl_->GetWebInterfaceUrl();
  2566. }
  2567. void TOperation::OnPrepared()
  2568. {
  2569. Impl_->OnPrepared();
  2570. }
  2571. void TOperation::SetDelayedStartFunction(std::function<TOperationId()> start)
  2572. {
  2573. Impl_->SetDelayedStartFunction(std::move(start));
  2574. }
  2575. void TOperation::Start()
  2576. {
  2577. Impl_->Start();
  2578. }
  2579. bool TOperation::IsStarted() const
  2580. {
  2581. return Impl_->IsStarted();
  2582. }
  2583. void TOperation::OnPreparationException(std::exception_ptr e)
  2584. {
  2585. Impl_->OnPreparationException(std::move(e));
  2586. }
  2587. TString TOperation::GetStatus() const
  2588. {
  2589. return Impl_->GetStatus();
  2590. }
  2591. void TOperation::OnStatusUpdated(const TString& newStatus)
  2592. {
  2593. Impl_->OnStatusUpdated(newStatus);
  2594. }
  2595. ::NThreading::TFuture<void> TOperation::GetPreparedFuture()
  2596. {
  2597. return Impl_->GetPreparedFuture();
  2598. }
  2599. ::NThreading::TFuture<void> TOperation::GetStartedFuture()
  2600. {
  2601. return Impl_->GetStartedFuture();
  2602. }
  2603. ::NThreading::TFuture<void> TOperation::Watch()
  2604. {
  2605. return Impl_->Watch(Client_);
  2606. }
  2607. TVector<TFailedJobInfo> TOperation::GetFailedJobInfo(const TGetFailedJobInfoOptions& options)
  2608. {
  2609. return NYT::NDetail::GetFailedJobInfo(Client_->GetRetryPolicy(), Client_->GetRawClient(), GetId(), options);
  2610. }
  2611. EOperationBriefState TOperation::GetBriefState()
  2612. {
  2613. return Impl_->GetBriefState();
  2614. }
  2615. TMaybe<TYtError> TOperation::GetError()
  2616. {
  2617. return Impl_->GetError();
  2618. }
  2619. TJobStatistics TOperation::GetJobStatistics()
  2620. {
  2621. return Impl_->GetJobStatistics();
  2622. }
  2623. TMaybe<TOperationBriefProgress> TOperation::GetBriefProgress()
  2624. {
  2625. return Impl_->GetBriefProgress();
  2626. }
  2627. void TOperation::AbortOperation()
  2628. {
  2629. Impl_->AbortOperation();
  2630. }
  2631. void TOperation::CompleteOperation()
  2632. {
  2633. Impl_->CompleteOperation();
  2634. }
  2635. void TOperation::SuspendOperation(const TSuspendOperationOptions& options)
  2636. {
  2637. Impl_->SuspendOperation(options);
  2638. }
  2639. void TOperation::ResumeOperation(const TResumeOperationOptions& options)
  2640. {
  2641. Impl_->ResumeOperation(options);
  2642. }
  2643. TOperationAttributes TOperation::GetAttributes(const TGetOperationOptions& options)
  2644. {
  2645. return Impl_->GetAttributes(options);
  2646. }
  2647. void TOperation::UpdateParameters(const TUpdateOperationParametersOptions& options)
  2648. {
  2649. Impl_->UpdateParameters(options);
  2650. }
  2651. TJobAttributes TOperation::GetJob(const TJobId& jobId, const TGetJobOptions& options)
  2652. {
  2653. return Impl_->GetJob(jobId, options);
  2654. }
  2655. TListJobsResult TOperation::ListJobs(const TListJobsOptions& options)
  2656. {
  2657. return Impl_->ListJobs(options);
  2658. }
  2659. ////////////////////////////////////////////////////////////////////////////////
  2660. struct TAsyncPrepareAndStartOperationArgs
  2661. {
  2662. std::function<void()> PrepareAndStart;
  2663. };
  2664. void* SyncPrepareAndStartOperation(void* pArgs)
  2665. {
  2666. std::unique_ptr<TAsyncPrepareAndStartOperationArgs> args(static_cast<TAsyncPrepareAndStartOperationArgs*>(pArgs));
  2667. args->PrepareAndStart();
  2668. return nullptr;
  2669. }
  2670. ::TIntrusivePtr<TOperation> ProcessOperation(
  2671. NYT::NDetail::TClientPtr client,
  2672. std::function<void()> prepare,
  2673. ::TIntrusivePtr<TOperation> operation,
  2674. const TOperationOptions& options)
  2675. {
  2676. auto prepareAndStart = [prepare = std::move(prepare), operation, mode = options.StartOperationMode_] () {
  2677. try {
  2678. prepare();
  2679. operation->OnPrepared();
  2680. } catch (const std::exception& ex) {
  2681. YT_LOG_INFO("Operation preparation failed: %v", ex.what());
  2682. operation->OnPreparationException(std::current_exception());
  2683. }
  2684. if (mode >= TOperationOptions::EStartOperationMode::AsyncStart) {
  2685. try {
  2686. operation->Start();
  2687. } catch (...) { }
  2688. }
  2689. };
  2690. if (options.StartOperationMode_ >= TOperationOptions::EStartOperationMode::SyncStart) {
  2691. prepareAndStart();
  2692. WaitIfRequired(operation, client, options);
  2693. } else {
  2694. auto args = new TAsyncPrepareAndStartOperationArgs;
  2695. args->PrepareAndStart = std::move(prepareAndStart);
  2696. TThread thread(TThread::TParams(SyncPrepareAndStartOperation, args).SetName("prepare and start operation"));
  2697. thread.Start();
  2698. thread.Detach();
  2699. }
  2700. return operation;
  2701. }
  2702. void WaitIfRequired(const TOperationPtr& operation, const TClientPtr& client, const TOperationOptions& options)
  2703. {
  2704. auto retryPolicy = client->GetRetryPolicy();
  2705. auto context = client->GetContext();
  2706. if (options.StartOperationMode_ >= TOperationOptions::EStartOperationMode::SyncStart) {
  2707. operation->GetStartedFuture().GetValueSync();
  2708. }
  2709. if (options.StartOperationMode_ == TOperationOptions::EStartOperationMode::SyncWait) {
  2710. auto finishedFuture = operation->Watch();
  2711. TWaitProxy::Get()->WaitFuture(finishedFuture);
  2712. finishedFuture.GetValue();
  2713. if (context.Config->WriteStderrSuccessfulJobs) {
  2714. auto stderrs = GetJobsStderr(retryPolicy, client->GetRawClient(), operation->GetId());
  2715. for (const auto& jobStderr : stderrs) {
  2716. if (!jobStderr.empty()) {
  2717. Cerr << jobStderr << '\n';
  2718. }
  2719. }
  2720. }
  2721. }
  2722. }
  2723. ////////////////////////////////////////////////////////////////////////////////
  2724. void ResetUseClientProtobuf(const char* methodName)
  2725. {
  2726. Cerr << "WARNING! OPTION `TConfig::UseClientProtobuf' IS RESET TO `true'; "
  2727. << "IT CAN DETERIORATE YOUR CODE PERFORMANCE!!! DON'T USE DEPRECATED METHOD `"
  2728. << "TOperationIOSpec::" << methodName << "' TO AVOID THIS RESET" << Endl;
  2729. // Give users some time to contemplate about usage of deprecated functions.
  2730. Cerr << "Sleeping for 5 seconds..." << Endl;
  2731. Sleep(TDuration::Seconds(5));
  2732. TConfig::Get()->UseClientProtobuf = true;
  2733. }
  2734. } // namespace NDetail
  2735. ////////////////////////////////////////////////////////////////////////////////
  2736. ::TIntrusivePtr<INodeReaderImpl> CreateJobNodeReader(TRawTableReaderPtr rawTableReader)
  2737. {
  2738. if (auto schema = NDetail::GetJobInputSkiffSchema()) {
  2739. return new NDetail::TSkiffTableReader(rawTableReader, schema);
  2740. } else {
  2741. return new TNodeTableReader(rawTableReader);
  2742. }
  2743. }
  2744. ::TIntrusivePtr<IYaMRReaderImpl> CreateJobYaMRReader(TRawTableReaderPtr rawTableReader)
  2745. {
  2746. return new TYaMRTableReader(rawTableReader);
  2747. }
  2748. ::TIntrusivePtr<IProtoReaderImpl> CreateJobProtoReader(TRawTableReaderPtr rawTableReader)
  2749. {
  2750. if (TConfig::Get()->UseClientProtobuf) {
  2751. return new TProtoTableReader(
  2752. rawTableReader,
  2753. GetJobInputDescriptors());
  2754. } else {
  2755. return new TLenvalProtoTableReader(
  2756. rawTableReader,
  2757. GetJobInputDescriptors());
  2758. }
  2759. }
  2760. ::TIntrusivePtr<INodeWriterImpl> CreateJobNodeWriter(THolder<IProxyOutput> rawJobWriter)
  2761. {
  2762. return new TNodeTableWriter(std::move(rawJobWriter));
  2763. }
  2764. ::TIntrusivePtr<IYaMRWriterImpl> CreateJobYaMRWriter(THolder<IProxyOutput> rawJobWriter)
  2765. {
  2766. return new TYaMRTableWriter(std::move(rawJobWriter));
  2767. }
  2768. ::TIntrusivePtr<IProtoWriterImpl> CreateJobProtoWriter(THolder<IProxyOutput> rawJobWriter)
  2769. {
  2770. if (TConfig::Get()->UseClientProtobuf) {
  2771. return new TProtoTableWriter(
  2772. std::move(rawJobWriter),
  2773. GetJobOutputDescriptors());
  2774. } else {
  2775. return new TLenvalProtoTableWriter(
  2776. std::move(rawJobWriter),
  2777. GetJobOutputDescriptors());
  2778. }
  2779. }
  2780. ////////////////////////////////////////////////////////////////////////////////
  2781. } // namespace NYT