operation.cpp 106 KB

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