operation.cpp 103 KB

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