operation.cpp 102 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003
  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 "transaction.h"
  7. #include "prepare_operation.h"
  8. #include "retry_heavy_write_request.h"
  9. #include "skiff.h"
  10. #include "structured_table_formats.h"
  11. #include "yt_poller.h"
  12. #include <yt/cpp/mapreduce/common/helpers.h>
  13. #include <yt/cpp/mapreduce/common/retry_lib.h>
  14. #include <yt/cpp/mapreduce/common/wait_proxy.h>
  15. #include <yt/cpp/mapreduce/interface/config.h>
  16. #include <yt/cpp/mapreduce/interface/errors.h>
  17. #include <yt/cpp/mapreduce/interface/fluent.h>
  18. #include <yt/cpp/mapreduce/interface/format.h>
  19. #include <yt/cpp/mapreduce/interface/job_statistics.h>
  20. #include <yt/cpp/mapreduce/interface/protobuf_format.h>
  21. #include <yt/cpp/mapreduce/interface/logging/yt_log.h>
  22. #include <yt/cpp/mapreduce/interface/logging/yt_log.h>
  23. #include <yt/cpp/mapreduce/http/requests.h>
  24. #include <yt/cpp/mapreduce/http/retry_request.h>
  25. #include <yt/cpp/mapreduce/io/job_reader.h>
  26. #include <yt/cpp/mapreduce/io/job_writer.h>
  27. #include <yt/cpp/mapreduce/io/yamr_table_reader.h>
  28. #include <yt/cpp/mapreduce/io/yamr_table_writer.h>
  29. #include <yt/cpp/mapreduce/io/node_table_reader.h>
  30. #include <yt/cpp/mapreduce/io/node_table_writer.h>
  31. #include <yt/cpp/mapreduce/io/proto_table_reader.h>
  32. #include <yt/cpp/mapreduce/io/proto_table_writer.h>
  33. #include <yt/cpp/mapreduce/io/proto_helpers.h>
  34. #include <yt/cpp/mapreduce/io/skiff_table_reader.h>
  35. #include <yt/cpp/mapreduce/raw_client/raw_batch_request.h>
  36. #include <yt/cpp/mapreduce/raw_client/raw_requests.h>
  37. #include <library/cpp/yson/node/serialize.h>
  38. #include <util/generic/hash_set.h>
  39. #include <util/string/builder.h>
  40. #include <util/string/cast.h>
  41. #include <util/system/thread.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. }
  668. template <typename T>
  669. void BuildCommonOperationPart(const TConfigPtr& config, const TOperationSpecBase<T>& baseSpec, const TOperationOptions& options, TFluentMap fluent)
  670. {
  671. const TProcessState* properties = TProcessState::Get();
  672. TString pool = config->Pool;
  673. if (baseSpec.Pool_) {
  674. pool = *baseSpec.Pool_;
  675. }
  676. fluent
  677. .Item("started_by")
  678. .BeginMap()
  679. .Item("hostname").Value(properties->FqdnHostName)
  680. .Item("pid").Value(properties->Pid)
  681. .Item("user").Value(properties->UserName)
  682. .Item("command").List(properties->CensoredCommandLine)
  683. .Item("wrapper_version").Value(properties->ClientVersion)
  684. .EndMap()
  685. .DoIf(!pool.empty(), [&] (TFluentMap fluentMap) {
  686. fluentMap.Item("pool").Value(pool);
  687. })
  688. .DoIf(baseSpec.Weight_.Defined(), [&] (TFluentMap fluentMap) {
  689. fluentMap.Item("weight").Value(*baseSpec.Weight_);
  690. })
  691. .DoIf(baseSpec.TimeLimit_.Defined(), [&] (TFluentMap fluentMap) {
  692. fluentMap.Item("time_limit").Value(baseSpec.TimeLimit_->MilliSeconds());
  693. })
  694. .DoIf(baseSpec.PoolTrees().Defined(), [&] (TFluentMap fluentMap) {
  695. TNode poolTreesSpec = TNode::CreateList();
  696. for (const auto& tree : *baseSpec.PoolTrees()) {
  697. poolTreesSpec.Add(tree);
  698. }
  699. fluentMap.Item("pool_trees").Value(poolTreesSpec);
  700. })
  701. .DoIf(baseSpec.ResourceLimits().Defined(), [&] (TFluentMap fluentMap) {
  702. auto resourceLimitsSpec = BuildSchedulerResourcesSpec(*baseSpec.ResourceLimits());
  703. if (!resourceLimitsSpec.IsUndefined()) {
  704. fluentMap.Item("resource_limits").Value(std::move(resourceLimitsSpec));
  705. }
  706. })
  707. .DoIf(options.SecureVault_.Defined(), [&] (TFluentMap fluentMap) {
  708. Y_ENSURE(options.SecureVault_->IsMap(),
  709. "SecureVault must be a map node, got " << options.SecureVault_->GetType());
  710. fluentMap.Item("secure_vault").Value(*options.SecureVault_);
  711. })
  712. .DoIf(baseSpec.Title_.Defined(), [&] (TFluentMap fluentMap) {
  713. fluentMap.Item("title").Value(*baseSpec.Title_);
  714. });
  715. }
  716. template <typename TSpec>
  717. void BuildCommonUserOperationPart(const TSpec& baseSpec, TNode* spec)
  718. {
  719. if (baseSpec.MaxFailedJobCount_.Defined()) {
  720. (*spec)["max_failed_job_count"] = *baseSpec.MaxFailedJobCount_;
  721. }
  722. if (baseSpec.FailOnJobRestart_.Defined()) {
  723. (*spec)["fail_on_job_restart"] = *baseSpec.FailOnJobRestart_;
  724. }
  725. if (baseSpec.StderrTablePath_.Defined()) {
  726. (*spec)["stderr_table_path"] = *baseSpec.StderrTablePath_;
  727. }
  728. if (baseSpec.CoreTablePath_.Defined()) {
  729. (*spec)["core_table_path"] = *baseSpec.CoreTablePath_;
  730. }
  731. if (baseSpec.WaitingJobTimeout_.Defined()) {
  732. (*spec)["waiting_job_timeout"] = baseSpec.WaitingJobTimeout_->MilliSeconds();
  733. }
  734. }
  735. template <typename TSpec>
  736. void BuildJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  737. {
  738. if (spec.JobCount_.Defined()) {
  739. (*nodeSpec)["job_count"] = *spec.JobCount_;
  740. }
  741. if (spec.DataSizePerJob_.Defined()) {
  742. (*nodeSpec)["data_size_per_job"] = *spec.DataSizePerJob_;
  743. }
  744. }
  745. template <typename TSpec>
  746. void BuildPartitionCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  747. {
  748. if (spec.PartitionCount_.Defined()) {
  749. (*nodeSpec)["partition_count"] = *spec.PartitionCount_;
  750. }
  751. if (spec.PartitionDataSize_.Defined()) {
  752. (*nodeSpec)["partition_data_size"] = *spec.PartitionDataSize_;
  753. }
  754. }
  755. template <typename TSpec>
  756. void BuildDataSizePerSortJobPart(const TSpec& spec, TNode* nodeSpec)
  757. {
  758. if (spec.DataSizePerSortJob_.Defined()) {
  759. (*nodeSpec)["data_size_per_sort_job"] = *spec.DataSizePerSortJob_;
  760. }
  761. }
  762. template <typename TSpec>
  763. void BuildPartitionJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  764. {
  765. if (spec.PartitionJobCount_.Defined()) {
  766. (*nodeSpec)["partition_job_count"] = *spec.PartitionJobCount_;
  767. }
  768. if (spec.DataSizePerPartitionJob_.Defined()) {
  769. (*nodeSpec)["data_size_per_partition_job"] = *spec.DataSizePerPartitionJob_;
  770. }
  771. }
  772. template <typename TSpec>
  773. void BuildMapJobCountOperationPart(const TSpec& spec, TNode* nodeSpec)
  774. {
  775. if (spec.MapJobCount_.Defined()) {
  776. (*nodeSpec)["map_job_count"] = *spec.MapJobCount_;
  777. }
  778. if (spec.DataSizePerMapJob_.Defined()) {
  779. (*nodeSpec)["data_size_per_map_job"] = *spec.DataSizePerMapJob_;
  780. }
  781. }
  782. template <typename TSpec>
  783. void BuildIntermediateDataPart(const TSpec& spec, TNode* nodeSpec)
  784. {
  785. if (spec.IntermediateDataAccount_.Defined()) {
  786. (*nodeSpec)["intermediate_data_account"] = *spec.IntermediateDataAccount_;
  787. }
  788. if (spec.IntermediateDataReplicationFactor_.Defined()) {
  789. (*nodeSpec)["intermediate_data_replication_factor"] = *spec.IntermediateDataReplicationFactor_;
  790. }
  791. }
  792. ////////////////////////////////////////////////////////////////////////////////
  793. TNode MergeSpec(TNode dst, TNode spec, const TOperationOptions& options)
  794. {
  795. MergeNodes(dst["spec"], spec);
  796. if (options.Spec_) {
  797. MergeNodes(dst["spec"], *options.Spec_);
  798. }
  799. return dst;
  800. }
  801. template <typename TSpec>
  802. void CreateDebugOutputTables(const TSpec& spec, const TOperationPreparer& preparer)
  803. {
  804. if (spec.StderrTablePath_.Defined()) {
  805. NYT::NDetail::Create(
  806. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  807. preparer.GetContext(),
  808. TTransactionId(),
  809. *spec.StderrTablePath_,
  810. NT_TABLE,
  811. TCreateOptions()
  812. .IgnoreExisting(true)
  813. .Recursive(true));
  814. }
  815. if (spec.CoreTablePath_.Defined()) {
  816. NYT::NDetail::Create(
  817. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  818. preparer.GetContext(),
  819. TTransactionId(),
  820. *spec.CoreTablePath_,
  821. NT_TABLE,
  822. TCreateOptions()
  823. .IgnoreExisting(true)
  824. .Recursive(true));
  825. }
  826. }
  827. void CreateOutputTable(
  828. const TOperationPreparer& preparer,
  829. const TRichYPath& path)
  830. {
  831. Y_ENSURE(path.Path_, "Output table is not set");
  832. Create(
  833. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  834. preparer.GetContext(), preparer.GetTransactionId(), path.Path_, NT_TABLE,
  835. TCreateOptions()
  836. .IgnoreExisting(true)
  837. .Recursive(true));
  838. }
  839. void CreateOutputTables(
  840. const TOperationPreparer& preparer,
  841. const TVector<TRichYPath>& paths)
  842. {
  843. for (auto& path : paths) {
  844. CreateOutputTable(preparer, path);
  845. }
  846. }
  847. void CheckInputTablesExist(
  848. const TOperationPreparer& preparer,
  849. const TVector<TRichYPath>& paths)
  850. {
  851. Y_ENSURE(!paths.empty(), "Input tables are not set");
  852. for (auto& path : paths) {
  853. auto curTransactionId = path.TransactionId_.GetOrElse(preparer.GetTransactionId());
  854. Y_ENSURE_EX(
  855. Exists(
  856. preparer.GetClientRetryPolicy()->CreatePolicyForGenericRequest(),
  857. preparer.GetContext(),
  858. curTransactionId,
  859. path.Path_),
  860. TApiUsageError() << "Input table '" << path.Path_ << "' doesn't exist");
  861. }
  862. }
  863. void LogJob(const TOperationId& opId, const IJob* job, const char* type)
  864. {
  865. if (job) {
  866. YT_LOG_INFO("Operation %v; %v = %v",
  867. opId,
  868. type,
  869. TJobFactory::Get()->GetJobName(job));
  870. }
  871. }
  872. void LogYPaths(const TOperationId& opId, const TVector<TRichYPath>& paths, const char* type)
  873. {
  874. for (size_t i = 0; i < paths.size(); ++i) {
  875. YT_LOG_INFO("Operation %v; %v[%v] = %v",
  876. opId,
  877. type,
  878. i,
  879. paths[i].Path_);
  880. }
  881. }
  882. void LogYPath(const TOperationId& opId, const TRichYPath& path, const char* type)
  883. {
  884. YT_LOG_INFO("Operation %v; %v = %v",
  885. opId,
  886. type,
  887. path.Path_);
  888. }
  889. TString AddModeToTitleIfDebug(const TString& title) {
  890. #ifndef NDEBUG
  891. return title + " (debug build)";
  892. #else
  893. return title;
  894. #endif
  895. }
  896. } // namespace
  897. ////////////////////////////////////////////////////////////////////////////////
  898. template <typename T>
  899. void DoExecuteMap(
  900. const TOperationPtr& operation,
  901. const TOperationPreparerPtr& preparer,
  902. const TSimpleOperationIo& operationIo,
  903. TMapOperationSpecBase<T> spec,
  904. const IJobPtr& mapper,
  905. const TOperationOptions& options)
  906. {
  907. if (options.CreateDebugOutputTables_) {
  908. CreateDebugOutputTables(spec, *preparer);
  909. }
  910. if (options.CreateOutputTables_) {
  911. CheckInputTablesExist(*preparer, operationIo.Inputs);
  912. CreateOutputTables(*preparer, operationIo.Outputs);
  913. }
  914. TJobPreparer map(
  915. *preparer,
  916. spec.MapperSpec_,
  917. *mapper,
  918. operationIo.Outputs.size(),
  919. operationIo.JobFiles,
  920. options);
  921. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(map.GetClassName()));
  922. TNode specNode = BuildYsonNodeFluently()
  923. .BeginMap().Item("spec").BeginMap()
  924. .Item("mapper").DoMap([&] (TFluentMap fluent) {
  925. BuildUserJobFluently(
  926. map,
  927. operationIo.InputFormat,
  928. operationIo.OutputFormat,
  929. fluent);
  930. })
  931. .DoIf(spec.AutoMerge_.Defined(), [&] (TFluentMap fluent) {
  932. auto autoMergeSpec = BuildAutoMergeSpec(*spec.AutoMerge_);
  933. if (!autoMergeSpec.IsUndefined()) {
  934. fluent.Item("auto_merge").Value(std::move(autoMergeSpec));
  935. }
  936. })
  937. .Item("input_table_paths").List(operationIo.Inputs)
  938. .Item("output_table_paths").List(operationIo.Outputs)
  939. .DoIf(spec.Ordered_.Defined(), [&] (TFluentMap fluent) {
  940. fluent.Item("ordered").Value(spec.Ordered_.GetRef());
  941. })
  942. .Do(std::bind(BuildCommonOperationPart<T>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  943. .EndMap().EndMap();
  944. specNode["spec"]["job_io"]["control_attributes"]["enable_row_index"] = TNode(true);
  945. specNode["spec"]["job_io"]["control_attributes"]["enable_range_index"] = TNode(true);
  946. if (!preparer->GetContext().Config->TableWriter.Empty()) {
  947. specNode["spec"]["job_io"]["table_writer"] = preparer->GetContext().Config->TableWriter;
  948. }
  949. BuildCommonUserOperationPart(spec, &specNode["spec"]);
  950. BuildJobCountOperationPart(spec, &specNode["spec"]);
  951. auto startOperation = [
  952. operation=operation.Get(),
  953. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  954. preparer,
  955. operationIo,
  956. mapper
  957. ] () {
  958. auto operationId = preparer->StartOperation(operation, "map", spec);
  959. LogJob(operationId, mapper.Get(), "mapper");
  960. LogYPaths(operationId, operationIo.Inputs, "input");
  961. LogYPaths(operationId, operationIo.Outputs, "output");
  962. return operationId;
  963. };
  964. operation->SetDelayedStartFunction(std::move(startOperation));
  965. }
  966. void ExecuteMap(
  967. const TOperationPtr& operation,
  968. const TOperationPreparerPtr& preparer,
  969. const TMapOperationSpec& spec,
  970. const ::TIntrusivePtr<IStructuredJob>& mapper,
  971. const TOperationOptions& options)
  972. {
  973. YT_LOG_DEBUG("Starting map operation (PreparationId: %v)",
  974. preparer->GetPreparationId());
  975. auto operationIo = CreateSimpleOperationIo(*mapper, *preparer, spec, options, /* allowSkiff = */ true);
  976. DoExecuteMap(
  977. operation,
  978. preparer,
  979. operationIo,
  980. spec,
  981. mapper,
  982. options);
  983. }
  984. void ExecuteRawMap(
  985. const TOperationPtr& operation,
  986. const TOperationPreparerPtr& preparer,
  987. const TRawMapOperationSpec& spec,
  988. const ::TIntrusivePtr<IRawJob>& mapper,
  989. const TOperationOptions& options)
  990. {
  991. YT_LOG_DEBUG("Starting raw map operation (PreparationId: %v)",
  992. preparer->GetPreparationId());
  993. auto operationIo = CreateSimpleOperationIo(*mapper, *preparer, spec);
  994. DoExecuteMap(
  995. operation,
  996. preparer,
  997. operationIo,
  998. spec,
  999. mapper,
  1000. options);
  1001. }
  1002. ////////////////////////////////////////////////////////////////////////////////
  1003. template <typename T>
  1004. void DoExecuteReduce(
  1005. const TOperationPtr& operation,
  1006. const TOperationPreparerPtr& preparer,
  1007. const TSimpleOperationIo& operationIo,
  1008. TReduceOperationSpecBase<T> spec,
  1009. const IJobPtr& reducer,
  1010. const TOperationOptions& options)
  1011. {
  1012. if (options.CreateDebugOutputTables_) {
  1013. CreateDebugOutputTables(spec, *preparer);
  1014. }
  1015. if (options.CreateOutputTables_) {
  1016. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1017. CreateOutputTables(*preparer, operationIo.Outputs);
  1018. }
  1019. TJobPreparer reduce(
  1020. *preparer,
  1021. spec.ReducerSpec_,
  1022. *reducer,
  1023. operationIo.Outputs.size(),
  1024. operationIo.JobFiles,
  1025. options);
  1026. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(reduce.GetClassName()));
  1027. TNode specNode = BuildYsonNodeFluently()
  1028. .BeginMap().Item("spec").BeginMap()
  1029. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1030. BuildUserJobFluently(
  1031. reduce,
  1032. operationIo.InputFormat,
  1033. operationIo.OutputFormat,
  1034. fluent);
  1035. })
  1036. .DoIf(!spec.SortBy_.Parts_.empty(), [&] (TFluentMap fluent) {
  1037. fluent.Item("sort_by").Value(spec.SortBy_);
  1038. })
  1039. .Item("reduce_by").Value(spec.ReduceBy_)
  1040. .DoIf(spec.JoinBy_.Defined(), [&] (TFluentMap fluent) {
  1041. fluent.Item("join_by").Value(spec.JoinBy_.GetRef());
  1042. })
  1043. .DoIf(spec.EnableKeyGuarantee_.Defined(), [&] (TFluentMap fluent) {
  1044. fluent.Item("enable_key_guarantee").Value(spec.EnableKeyGuarantee_.GetRef());
  1045. })
  1046. .Item("input_table_paths").List(operationIo.Inputs)
  1047. .Item("output_table_paths").List(operationIo.Outputs)
  1048. .Item("job_io").BeginMap()
  1049. .Item("control_attributes").BeginMap()
  1050. .Item("enable_key_switch").Value(true)
  1051. .Item("enable_row_index").Value(true)
  1052. .Item("enable_range_index").Value(true)
  1053. .EndMap()
  1054. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1055. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1056. })
  1057. .EndMap()
  1058. .DoIf(spec.AutoMerge_.Defined(), [&] (TFluentMap fluent) {
  1059. fluent.Item("auto_merge").Value(BuildAutoMergeSpec(*spec.AutoMerge_));
  1060. })
  1061. .Do(std::bind(BuildCommonOperationPart<T>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1062. .EndMap().EndMap();
  1063. BuildCommonUserOperationPart(spec, &specNode["spec"]);
  1064. BuildJobCountOperationPart(spec, &specNode["spec"]);
  1065. auto startOperation = [
  1066. operation=operation.Get(),
  1067. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1068. preparer,
  1069. operationIo,
  1070. reducer
  1071. ] () {
  1072. auto operationId = preparer->StartOperation(operation, "reduce", spec);
  1073. LogJob(operationId, reducer.Get(), "reducer");
  1074. LogYPaths(operationId, operationIo.Inputs, "input");
  1075. LogYPaths(operationId, operationIo.Outputs, "output");
  1076. return operationId;
  1077. };
  1078. operation->SetDelayedStartFunction(std::move(startOperation));
  1079. }
  1080. void ExecuteReduce(
  1081. const TOperationPtr& operation,
  1082. const TOperationPreparerPtr& preparer,
  1083. const TReduceOperationSpec& spec,
  1084. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1085. const TOperationOptions& options)
  1086. {
  1087. YT_LOG_DEBUG("Starting reduce operation (PreparationId: %v)",
  1088. preparer->GetPreparationId());
  1089. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec, options, /* allowSkiff = */ false);
  1090. DoExecuteReduce(
  1091. operation,
  1092. preparer,
  1093. operationIo,
  1094. spec,
  1095. reducer,
  1096. options);
  1097. }
  1098. void ExecuteRawReduce(
  1099. const TOperationPtr& operation,
  1100. const TOperationPreparerPtr& preparer,
  1101. const TRawReduceOperationSpec& spec,
  1102. const ::TIntrusivePtr<IRawJob>& reducer,
  1103. const TOperationOptions& options)
  1104. {
  1105. YT_LOG_DEBUG("Starting raw reduce operation (PreparationId: %v)",
  1106. preparer->GetPreparationId());
  1107. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec);
  1108. DoExecuteReduce(
  1109. operation,
  1110. preparer,
  1111. operationIo,
  1112. spec,
  1113. reducer,
  1114. options);
  1115. }
  1116. ////////////////////////////////////////////////////////////////////////////////
  1117. template <typename T>
  1118. void DoExecuteJoinReduce(
  1119. const TOperationPtr& operation,
  1120. const TOperationPreparerPtr& preparer,
  1121. const TSimpleOperationIo& operationIo,
  1122. TJoinReduceOperationSpecBase<T> spec,
  1123. const IJobPtr& reducer,
  1124. const TOperationOptions& options)
  1125. {
  1126. if (options.CreateDebugOutputTables_) {
  1127. CreateDebugOutputTables(spec, *preparer);
  1128. }
  1129. if (options.CreateOutputTables_) {
  1130. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1131. CreateOutputTables(*preparer, operationIo.Outputs);
  1132. }
  1133. TJobPreparer reduce(
  1134. *preparer,
  1135. spec.ReducerSpec_,
  1136. *reducer,
  1137. operationIo.Outputs.size(),
  1138. operationIo.JobFiles,
  1139. options);
  1140. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(reduce.GetClassName()));
  1141. TNode specNode = BuildYsonNodeFluently()
  1142. .BeginMap().Item("spec").BeginMap()
  1143. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1144. BuildUserJobFluently(
  1145. reduce,
  1146. operationIo.InputFormat,
  1147. operationIo.OutputFormat,
  1148. fluent);
  1149. })
  1150. .Item("join_by").Value(spec.JoinBy_)
  1151. .Item("input_table_paths").List(operationIo.Inputs)
  1152. .Item("output_table_paths").List(operationIo.Outputs)
  1153. .Item("job_io").BeginMap()
  1154. .Item("control_attributes").BeginMap()
  1155. .Item("enable_key_switch").Value(true)
  1156. .Item("enable_row_index").Value(true)
  1157. .Item("enable_range_index").Value(true)
  1158. .EndMap()
  1159. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1160. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1161. })
  1162. .EndMap()
  1163. .Do(std::bind(BuildCommonOperationPart<T>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1164. .EndMap().EndMap();
  1165. BuildCommonUserOperationPart(spec, &specNode["spec"]);
  1166. BuildJobCountOperationPart(spec, &specNode["spec"]);
  1167. auto startOperation = [
  1168. operation=operation.Get(),
  1169. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1170. preparer,
  1171. reducer,
  1172. operationIo
  1173. ] () {
  1174. auto operationId = preparer->StartOperation(operation, "join_reduce", spec);
  1175. LogJob(operationId, reducer.Get(), "reducer");
  1176. LogYPaths(operationId, operationIo.Inputs, "input");
  1177. LogYPaths(operationId, operationIo.Outputs, "output");
  1178. return operationId;
  1179. };
  1180. operation->SetDelayedStartFunction(std::move(startOperation));
  1181. }
  1182. void ExecuteJoinReduce(
  1183. const TOperationPtr& operation,
  1184. const TOperationPreparerPtr& preparer,
  1185. const TJoinReduceOperationSpec& spec,
  1186. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1187. const TOperationOptions& options)
  1188. {
  1189. YT_LOG_DEBUG("Starting join reduce operation (PreparationId: %v)",
  1190. preparer->GetPreparationId());
  1191. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec, options, /* allowSkiff = */ false);
  1192. return DoExecuteJoinReduce(
  1193. operation,
  1194. preparer,
  1195. operationIo,
  1196. spec,
  1197. reducer,
  1198. options);
  1199. }
  1200. void ExecuteRawJoinReduce(
  1201. const TOperationPtr& operation,
  1202. const TOperationPreparerPtr& preparer,
  1203. const TRawJoinReduceOperationSpec& spec,
  1204. const ::TIntrusivePtr<IRawJob>& reducer,
  1205. const TOperationOptions& options)
  1206. {
  1207. YT_LOG_DEBUG("Starting raw join reduce operation (PreparationId: %v)",
  1208. preparer->GetPreparationId());
  1209. auto operationIo = CreateSimpleOperationIo(*reducer, *preparer, spec);
  1210. return DoExecuteJoinReduce(
  1211. operation,
  1212. preparer,
  1213. operationIo,
  1214. spec,
  1215. reducer,
  1216. options);
  1217. }
  1218. ////////////////////////////////////////////////////////////////////////////////
  1219. template <typename T>
  1220. void DoExecuteMapReduce(
  1221. const TOperationPtr& operation,
  1222. const TOperationPreparerPtr& preparer,
  1223. const TMapReduceOperationIo& operationIo,
  1224. TMapReduceOperationSpecBase<T> spec,
  1225. const IJobPtr& mapper,
  1226. const IJobPtr& reduceCombiner,
  1227. const IJobPtr& reducer,
  1228. const TOperationOptions& options)
  1229. {
  1230. TVector<TRichYPath> allOutputs;
  1231. allOutputs.insert(allOutputs.end(), operationIo.MapOutputs.begin(), operationIo.MapOutputs.end());
  1232. allOutputs.insert(allOutputs.end(), operationIo.Outputs.begin(), operationIo.Outputs.end());
  1233. if (options.CreateDebugOutputTables_) {
  1234. CreateDebugOutputTables(spec, *preparer);
  1235. }
  1236. if (options.CreateOutputTables_) {
  1237. CheckInputTablesExist(*preparer, operationIo.Inputs);
  1238. CreateOutputTables(*preparer, allOutputs);
  1239. }
  1240. TSortColumns sortBy = spec.SortBy_;
  1241. TSortColumns reduceBy = spec.ReduceBy_;
  1242. if (sortBy.Parts_.empty()) {
  1243. sortBy = reduceBy;
  1244. }
  1245. const bool hasMapper = mapper != nullptr;
  1246. const bool hasCombiner = reduceCombiner != nullptr;
  1247. TVector<TRichYPath> files;
  1248. TJobPreparer reduce(
  1249. *preparer,
  1250. spec.ReducerSpec_,
  1251. *reducer,
  1252. operationIo.Outputs.size(),
  1253. operationIo.ReducerJobFiles,
  1254. options);
  1255. TString title;
  1256. TNode specNode = BuildYsonNodeFluently()
  1257. .BeginMap().Item("spec").BeginMap()
  1258. .DoIf(hasMapper, [&] (TFluentMap fluent) {
  1259. TJobPreparer map(
  1260. *preparer,
  1261. spec.MapperSpec_,
  1262. *mapper,
  1263. 1 + operationIo.MapOutputs.size(),
  1264. operationIo.MapperJobFiles,
  1265. options);
  1266. fluent.Item("mapper").DoMap([&] (TFluentMap fluent) {
  1267. BuildUserJobFluently(
  1268. std::cref(map),
  1269. *operationIo.MapperInputFormat,
  1270. *operationIo.MapperOutputFormat,
  1271. fluent);
  1272. });
  1273. title = "mapper:" + map.GetClassName() + " ";
  1274. })
  1275. .DoIf(hasCombiner, [&] (TFluentMap fluent) {
  1276. TJobPreparer combine(
  1277. *preparer,
  1278. spec.ReduceCombinerSpec_,
  1279. *reduceCombiner,
  1280. size_t(1),
  1281. operationIo.ReduceCombinerJobFiles,
  1282. options);
  1283. fluent.Item("reduce_combiner").DoMap([&] (TFluentMap fluent) {
  1284. BuildUserJobFluently(
  1285. combine,
  1286. *operationIo.ReduceCombinerInputFormat,
  1287. *operationIo.ReduceCombinerOutputFormat,
  1288. fluent);
  1289. });
  1290. title += "combiner:" + combine.GetClassName() + " ";
  1291. })
  1292. .Item("reducer").DoMap([&] (TFluentMap fluent) {
  1293. BuildUserJobFluently(
  1294. reduce,
  1295. operationIo.ReducerInputFormat,
  1296. operationIo.ReducerOutputFormat,
  1297. fluent);
  1298. })
  1299. .Item("sort_by").Value(sortBy)
  1300. .Item("reduce_by").Value(reduceBy)
  1301. .Item("input_table_paths").List(operationIo.Inputs)
  1302. .Item("output_table_paths").List(allOutputs)
  1303. .Item("mapper_output_table_count").Value(operationIo.MapOutputs.size())
  1304. .DoIf(spec.ForceReduceCombiners_.Defined(), [&] (TFluentMap fluent) {
  1305. fluent.Item("force_reduce_combiners").Value(*spec.ForceReduceCombiners_);
  1306. })
  1307. .Item("map_job_io").BeginMap()
  1308. .Item("control_attributes").BeginMap()
  1309. .Item("enable_row_index").Value(true)
  1310. .Item("enable_range_index").Value(true)
  1311. .EndMap()
  1312. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1313. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1314. })
  1315. .EndMap()
  1316. .Item("sort_job_io").BeginMap()
  1317. .Item("control_attributes").BeginMap()
  1318. .Item("enable_key_switch").Value(true)
  1319. .EndMap()
  1320. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1321. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1322. })
  1323. .EndMap()
  1324. .Item("reduce_job_io").BeginMap()
  1325. .Item("control_attributes").BeginMap()
  1326. .Item("enable_key_switch").Value(true)
  1327. .EndMap()
  1328. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&] (TFluentMap fluent) {
  1329. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1330. })
  1331. .EndMap()
  1332. .Do([&] (TFluentMap) {
  1333. spec.Title_ = spec.Title_.GetOrElse(AddModeToTitleIfDebug(title + "reducer:" + reduce.GetClassName()));
  1334. })
  1335. .Do(std::bind(BuildCommonOperationPart<T>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1336. .EndMap().EndMap();
  1337. if (spec.Ordered_) {
  1338. specNode["spec"]["ordered"] = *spec.Ordered_;
  1339. }
  1340. BuildCommonUserOperationPart(spec, &specNode["spec"]);
  1341. BuildMapJobCountOperationPart(spec, &specNode["spec"]);
  1342. BuildPartitionCountOperationPart(spec, &specNode["spec"]);
  1343. BuildIntermediateDataPart(spec, &specNode["spec"]);
  1344. BuildDataSizePerSortJobPart(spec, &specNode["spec"]);
  1345. auto startOperation = [
  1346. operation=operation.Get(),
  1347. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1348. preparer,
  1349. mapper,
  1350. reduceCombiner,
  1351. reducer,
  1352. inputs=operationIo.Inputs,
  1353. allOutputs
  1354. ] () {
  1355. auto operationId = preparer->StartOperation(operation, "map_reduce", spec);
  1356. LogJob(operationId, mapper.Get(), "mapper");
  1357. LogJob(operationId, reduceCombiner.Get(), "reduce_combiner");
  1358. LogJob(operationId, reducer.Get(), "reducer");
  1359. LogYPaths(operationId, inputs, "input");
  1360. LogYPaths(operationId, allOutputs, "output");
  1361. return operationId;
  1362. };
  1363. operation->SetDelayedStartFunction(std::move(startOperation));
  1364. }
  1365. void ExecuteMapReduce(
  1366. const TOperationPtr& operation,
  1367. const TOperationPreparerPtr& preparer,
  1368. const TMapReduceOperationSpec& spec_,
  1369. const ::TIntrusivePtr<IStructuredJob>& mapper,
  1370. const ::TIntrusivePtr<IStructuredJob>& reduceCombiner,
  1371. const ::TIntrusivePtr<IStructuredJob>& reducer,
  1372. const TOperationOptions& options)
  1373. {
  1374. YT_LOG_DEBUG("Starting map-reduce operation (PreparationId: %v)",
  1375. preparer->GetPreparationId());
  1376. TMapReduceOperationSpec spec = spec_;
  1377. TMapReduceOperationIo operationIo;
  1378. auto structuredInputs = CanonizeStructuredTableList(preparer->GetContext(), spec.GetStructuredInputs());
  1379. auto structuredMapOutputs = CanonizeStructuredTableList(preparer->GetContext(), spec.GetStructuredMapOutputs());
  1380. auto structuredOutputs = CanonizeStructuredTableList(preparer->GetContext(), spec.GetStructuredOutputs());
  1381. const bool inferOutputSchema = options.InferOutputSchema_.GetOrElse(preparer->GetContext().Config->InferTableSchema);
  1382. TVector<TTableSchema> currentInferenceResult;
  1383. auto fixSpec = [&] (const TFormat& format) {
  1384. if (format.IsYamredDsv()) {
  1385. spec.SortBy_.Parts_.clear();
  1386. spec.ReduceBy_.Parts_.clear();
  1387. const TYamredDsvAttributes attributes = format.GetYamredDsvAttributes();
  1388. for (auto& column : attributes.KeyColumnNames) {
  1389. spec.SortBy_.Parts_.push_back(column);
  1390. spec.ReduceBy_.Parts_.push_back(column);
  1391. }
  1392. for (const auto& column : attributes.SubkeyColumnNames) {
  1393. spec.SortBy_.Parts_.push_back(column);
  1394. }
  1395. }
  1396. };
  1397. VerifyHasElements(structuredInputs, "inputs");
  1398. TFormatBuilder formatBuilder(
  1399. preparer->GetClientRetryPolicy(),
  1400. preparer->GetContext(),
  1401. preparer->GetTransactionId(),
  1402. options);
  1403. if (mapper) {
  1404. auto mapperOutputDescription =
  1405. spec.GetIntermediateMapOutputDescription()
  1406. .GetOrElse(TUnspecifiedTableStructure());
  1407. TStructuredJobTableList mapperOutput = {
  1408. TStructuredJobTable::Intermediate(mapperOutputDescription),
  1409. };
  1410. for (const auto& table : structuredMapOutputs) {
  1411. mapperOutput.push_back(TStructuredJobTable{table.Description, table.RichYPath});
  1412. }
  1413. auto hints = spec.MapperFormatHints_;
  1414. auto mapperInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1415. *mapper,
  1416. TOperationPreparationContext(
  1417. structuredInputs,
  1418. mapperOutput,
  1419. preparer->GetContext(),
  1420. preparer->GetClientRetryPolicy(),
  1421. preparer->GetTransactionId()),
  1422. &structuredInputs,
  1423. /* outputs */ nullptr,
  1424. hints);
  1425. auto nodeReaderFormat = NodeReaderFormatFromHintAndGlobalConfig(spec.MapperFormatHints_);
  1426. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1427. *mapper,
  1428. EIODirection::Input,
  1429. structuredInputs,
  1430. hints.InputFormatHints_,
  1431. nodeReaderFormat,
  1432. /* allowFormatFromTableAttribute */ true);
  1433. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1434. *mapper,
  1435. EIODirection::Output,
  1436. mapperOutput,
  1437. hints.OutputFormatHints_,
  1438. ENodeReaderFormat::Yson,
  1439. /* allowFormatFromTableAttribute */ false);
  1440. operationIo.MapperJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1441. operationIo.MapperInputFormat = inputFormat;
  1442. operationIo.MapperOutputFormat = outputFormat;
  1443. Y_ABORT_UNLESS(mapperInferenceResult.size() >= 1);
  1444. currentInferenceResult = TVector<TTableSchema>{mapperInferenceResult[0]};
  1445. // The first output as it corresponds to the intermediate data.
  1446. TVector<TTableSchema> additionalOutputsInferenceResult(mapperInferenceResult.begin() + 1, mapperInferenceResult.end());
  1447. operationIo.MapOutputs = GetPathList(
  1448. structuredMapOutputs,
  1449. additionalOutputsInferenceResult,
  1450. inferOutputSchema);
  1451. }
  1452. if (reduceCombiner) {
  1453. const bool isFirstStep = !mapper;
  1454. TStructuredJobTableList inputs;
  1455. if (isFirstStep) {
  1456. inputs = structuredInputs;
  1457. } else {
  1458. auto reduceCombinerIntermediateInput =
  1459. spec.GetIntermediateReduceCombinerInputDescription()
  1460. .GetOrElse(TUnspecifiedTableStructure());
  1461. inputs = {
  1462. TStructuredJobTable::Intermediate(reduceCombinerIntermediateInput),
  1463. };
  1464. }
  1465. auto reduceCombinerOutputDescription = spec.GetIntermediateReduceCombinerOutputDescription()
  1466. .GetOrElse(TUnspecifiedTableStructure());
  1467. TStructuredJobTableList outputs = {
  1468. TStructuredJobTable::Intermediate(reduceCombinerOutputDescription),
  1469. };
  1470. auto hints = spec.ReduceCombinerFormatHints_;
  1471. if (isFirstStep) {
  1472. currentInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1473. *reduceCombiner,
  1474. TOperationPreparationContext(
  1475. inputs,
  1476. outputs,
  1477. preparer->GetContext(),
  1478. preparer->GetClientRetryPolicy(),
  1479. preparer->GetTransactionId()),
  1480. &inputs,
  1481. /* outputs */ nullptr,
  1482. hints);
  1483. } else {
  1484. currentInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1485. *reduceCombiner,
  1486. TSpeculativeOperationPreparationContext(
  1487. currentInferenceResult,
  1488. inputs,
  1489. outputs),
  1490. /* inputs */ nullptr,
  1491. /* outputs */ nullptr,
  1492. hints);
  1493. }
  1494. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1495. *reduceCombiner,
  1496. EIODirection::Input,
  1497. inputs,
  1498. hints.InputFormatHints_,
  1499. ENodeReaderFormat::Yson,
  1500. /* allowFormatFromTableAttribute = */ isFirstStep);
  1501. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1502. *reduceCombiner,
  1503. EIODirection::Output,
  1504. outputs,
  1505. hints.OutputFormatHints_,
  1506. ENodeReaderFormat::Yson,
  1507. /* allowFormatFromTableAttribute = */ false);
  1508. operationIo.ReduceCombinerJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1509. operationIo.ReduceCombinerInputFormat = inputFormat;
  1510. operationIo.ReduceCombinerOutputFormat = outputFormat;
  1511. if (isFirstStep) {
  1512. fixSpec(*operationIo.ReduceCombinerInputFormat);
  1513. }
  1514. }
  1515. const bool isFirstStep = (!mapper && !reduceCombiner);
  1516. TStructuredJobTableList reducerInputs;
  1517. if (isFirstStep) {
  1518. reducerInputs = structuredInputs;
  1519. } else {
  1520. auto reducerInputDescription =
  1521. spec.GetIntermediateReducerInputDescription()
  1522. .GetOrElse(TUnspecifiedTableStructure());
  1523. reducerInputs = {
  1524. TStructuredJobTable::Intermediate(reducerInputDescription),
  1525. };
  1526. }
  1527. auto hints = spec.ReducerFormatHints_;
  1528. TVector<TTableSchema> reducerInferenceResult;
  1529. if (isFirstStep) {
  1530. reducerInferenceResult = PrepareOperation(
  1531. *reducer,
  1532. TOperationPreparationContext(
  1533. structuredInputs,
  1534. structuredOutputs,
  1535. preparer->GetContext(),
  1536. preparer->GetClientRetryPolicy(),
  1537. preparer->GetTransactionId()),
  1538. &structuredInputs,
  1539. &structuredOutputs,
  1540. hints);
  1541. } else {
  1542. reducerInferenceResult = PrepareOperation<TStructuredJobTableList>(
  1543. *reducer,
  1544. TSpeculativeOperationPreparationContext(
  1545. currentInferenceResult,
  1546. reducerInputs,
  1547. structuredOutputs),
  1548. /* inputs */ nullptr,
  1549. &structuredOutputs,
  1550. hints);
  1551. }
  1552. auto [inputFormat, inputFormatConfig] = formatBuilder.CreateFormat(
  1553. *reducer,
  1554. EIODirection::Input,
  1555. reducerInputs,
  1556. hints.InputFormatHints_,
  1557. ENodeReaderFormat::Yson,
  1558. /* allowFormatFromTableAttribute = */ isFirstStep);
  1559. auto [outputFormat, outputFormatConfig] = formatBuilder.CreateFormat(
  1560. *reducer,
  1561. EIODirection::Output,
  1562. ToStructuredJobTableList(spec.GetStructuredOutputs()),
  1563. hints.OutputFormatHints_,
  1564. ENodeReaderFormat::Yson,
  1565. /* allowFormatFromTableAttribute = */ false);
  1566. operationIo.ReducerJobFiles = CreateFormatConfig(inputFormatConfig, outputFormatConfig);
  1567. operationIo.ReducerInputFormat = inputFormat;
  1568. operationIo.ReducerOutputFormat = outputFormat;
  1569. if (isFirstStep) {
  1570. fixSpec(operationIo.ReducerInputFormat);
  1571. }
  1572. operationIo.Inputs = GetPathList(
  1573. ApplyProtobufColumnFilters(
  1574. structuredInputs,
  1575. *preparer,
  1576. GetColumnsUsedInOperation(spec),
  1577. options),
  1578. /* jobSchemaInferenceResult */ Nothing(),
  1579. /* inferSchema */ false);
  1580. operationIo.Outputs = GetPathList(
  1581. structuredOutputs,
  1582. reducerInferenceResult,
  1583. inferOutputSchema);
  1584. VerifyHasElements(operationIo.Outputs, "outputs");
  1585. return DoExecuteMapReduce(
  1586. operation,
  1587. preparer,
  1588. operationIo,
  1589. spec,
  1590. mapper,
  1591. reduceCombiner,
  1592. reducer,
  1593. options);
  1594. }
  1595. void ExecuteRawMapReduce(
  1596. const TOperationPtr& operation,
  1597. const TOperationPreparerPtr& preparer,
  1598. const TRawMapReduceOperationSpec& spec,
  1599. const ::TIntrusivePtr<IRawJob>& mapper,
  1600. const ::TIntrusivePtr<IRawJob>& reduceCombiner,
  1601. const ::TIntrusivePtr<IRawJob>& reducer,
  1602. const TOperationOptions& options)
  1603. {
  1604. YT_LOG_DEBUG("Starting raw map-reduce operation (PreparationId: %v)",
  1605. preparer->GetPreparationId());
  1606. TMapReduceOperationIo operationIo;
  1607. operationIo.Inputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.GetInputs());
  1608. operationIo.MapOutputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.GetMapOutputs());
  1609. operationIo.Outputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.GetOutputs());
  1610. VerifyHasElements(operationIo.Inputs, "inputs");
  1611. VerifyHasElements(operationIo.Outputs, "outputs");
  1612. auto getFormatOrDefault = [&] (const TMaybe<TFormat>& maybeFormat, const TMaybe<TFormat> stageDefaultFormat, const char* formatName) {
  1613. if (maybeFormat) {
  1614. return *maybeFormat;
  1615. } else if (stageDefaultFormat) {
  1616. return *stageDefaultFormat;
  1617. } else {
  1618. ythrow TApiUsageError() << "Cannot derive " << formatName;
  1619. }
  1620. };
  1621. if (mapper) {
  1622. operationIo.MapperInputFormat = getFormatOrDefault(spec.MapperInputFormat_, spec.MapperFormat_, "mapper input format");
  1623. operationIo.MapperOutputFormat = getFormatOrDefault(spec.MapperOutputFormat_, spec.MapperFormat_, "mapper output format");
  1624. }
  1625. if (reduceCombiner) {
  1626. operationIo.ReduceCombinerInputFormat = getFormatOrDefault(spec.ReduceCombinerInputFormat_, spec.ReduceCombinerFormat_, "reduce combiner input format");
  1627. operationIo.ReduceCombinerOutputFormat = getFormatOrDefault(spec.ReduceCombinerOutputFormat_, spec.ReduceCombinerFormat_, "reduce combiner output format");
  1628. }
  1629. operationIo.ReducerInputFormat = getFormatOrDefault(spec.ReducerInputFormat_, spec.ReducerFormat_, "reducer input format");
  1630. operationIo.ReducerOutputFormat = getFormatOrDefault(spec.ReducerOutputFormat_, spec.ReducerFormat_, "reducer output format");
  1631. return DoExecuteMapReduce(
  1632. operation,
  1633. preparer,
  1634. operationIo,
  1635. spec,
  1636. mapper,
  1637. reduceCombiner,
  1638. reducer,
  1639. options);
  1640. }
  1641. void ExecuteSort(
  1642. const TOperationPtr& operation,
  1643. const TOperationPreparerPtr& preparer,
  1644. const TSortOperationSpec& spec,
  1645. const TOperationOptions& options)
  1646. {
  1647. YT_LOG_DEBUG("Starting sort operation (PreparationId: %v)",
  1648. preparer->GetPreparationId());
  1649. auto inputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.Inputs_);
  1650. auto output = CanonizeYPath(nullptr, preparer->GetContext(), spec.Output_);
  1651. if (options.CreateOutputTables_) {
  1652. CheckInputTablesExist(*preparer, inputs);
  1653. CreateOutputTable(*preparer, output);
  1654. }
  1655. TNode specNode = BuildYsonNodeFluently()
  1656. .BeginMap().Item("spec").BeginMap()
  1657. .Item("input_table_paths").List(inputs)
  1658. .Item("output_table_path").Value(output)
  1659. .Item("sort_by").Value(spec.SortBy_)
  1660. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1661. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1662. })
  1663. .Do(std::bind(BuildCommonOperationPart<TSortOperationSpec>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1664. .EndMap().EndMap();
  1665. BuildPartitionCountOperationPart(spec, &specNode["spec"]);
  1666. BuildPartitionJobCountOperationPart(spec, &specNode["spec"]);
  1667. BuildIntermediateDataPart(spec, &specNode["spec"]);
  1668. auto startOperation = [
  1669. operation=operation.Get(),
  1670. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1671. preparer,
  1672. inputs,
  1673. output
  1674. ] () {
  1675. auto operationId = preparer->StartOperation(operation, "sort", spec);
  1676. LogYPaths(operationId, inputs, "input");
  1677. LogYPath(operationId, output, "output");
  1678. return operationId;
  1679. };
  1680. operation->SetDelayedStartFunction(std::move(startOperation));
  1681. }
  1682. void ExecuteMerge(
  1683. const TOperationPtr& operation,
  1684. const TOperationPreparerPtr& preparer,
  1685. const TMergeOperationSpec& spec,
  1686. const TOperationOptions& options)
  1687. {
  1688. YT_LOG_DEBUG("Starting merge operation (PreparationId: %v)",
  1689. preparer->GetPreparationId());
  1690. auto inputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.Inputs_);
  1691. auto output = CanonizeYPath(nullptr, preparer->GetContext(), spec.Output_);
  1692. if (options.CreateOutputTables_) {
  1693. CheckInputTablesExist(*preparer, inputs);
  1694. CreateOutputTable(*preparer, output);
  1695. }
  1696. TNode specNode = BuildYsonNodeFluently()
  1697. .BeginMap().Item("spec").BeginMap()
  1698. .Item("input_table_paths").List(inputs)
  1699. .Item("output_table_path").Value(output)
  1700. .Item("mode").Value(ToString(spec.Mode_))
  1701. .Item("combine_chunks").Value(spec.CombineChunks_)
  1702. .Item("force_transform").Value(spec.ForceTransform_)
  1703. .Item("merge_by").Value(spec.MergeBy_)
  1704. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1705. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1706. })
  1707. .Do(std::bind(BuildCommonOperationPart<TMergeOperationSpec>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1708. .EndMap().EndMap();
  1709. BuildJobCountOperationPart(spec, &specNode["spec"]);
  1710. auto startOperation = [
  1711. operation=operation.Get(),
  1712. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1713. preparer,
  1714. inputs,
  1715. output
  1716. ] () {
  1717. auto operationId = preparer->StartOperation(operation, "merge", spec);
  1718. LogYPaths(operationId, inputs, "input");
  1719. LogYPath(operationId, output, "output");
  1720. return operationId;
  1721. };
  1722. operation->SetDelayedStartFunction(std::move(startOperation));
  1723. }
  1724. void ExecuteErase(
  1725. const TOperationPtr& operation,
  1726. const TOperationPreparerPtr& preparer,
  1727. const TEraseOperationSpec& spec,
  1728. const TOperationOptions& options)
  1729. {
  1730. YT_LOG_DEBUG("Starting erase operation (PreparationId: %v)",
  1731. preparer->GetPreparationId());
  1732. auto tablePath = CanonizeYPath(nullptr, preparer->GetContext(), spec.TablePath_);
  1733. TNode specNode = BuildYsonNodeFluently()
  1734. .BeginMap().Item("spec").BeginMap()
  1735. .Item("table_path").Value(tablePath)
  1736. .Item("combine_chunks").Value(spec.CombineChunks_)
  1737. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1738. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1739. })
  1740. .Do(std::bind(BuildCommonOperationPart<TEraseOperationSpec>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1741. .EndMap().EndMap();
  1742. auto startOperation = [
  1743. operation=operation.Get(),
  1744. spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options),
  1745. preparer,
  1746. tablePath
  1747. ] () {
  1748. auto operationId = preparer->StartOperation(operation, "erase", spec);
  1749. LogYPath(operationId, tablePath, "table_path");
  1750. return operationId;
  1751. };
  1752. operation->SetDelayedStartFunction(std::move(startOperation));
  1753. }
  1754. void ExecuteRemoteCopy(
  1755. const TOperationPtr& operation,
  1756. const TOperationPreparerPtr& preparer,
  1757. const TRemoteCopyOperationSpec& spec,
  1758. const TOperationOptions& options)
  1759. {
  1760. YT_LOG_DEBUG("Starting remote copy operation (PreparationId: %v)",
  1761. preparer->GetPreparationId());
  1762. auto inputs = CanonizeYPaths(/* retryPolicy */ nullptr, preparer->GetContext(), spec.Inputs_);
  1763. auto output = CanonizeYPath(nullptr, preparer->GetContext(), spec.Output_);
  1764. if (options.CreateOutputTables_) {
  1765. CreateOutputTable(*preparer, output);
  1766. }
  1767. Y_ENSURE_EX(!spec.ClusterName_.empty(), TApiUsageError() << "ClusterName parameter is required");
  1768. TNode specNode = BuildYsonNodeFluently()
  1769. .BeginMap().Item("spec").BeginMap()
  1770. .Item("cluster_name").Value(spec.ClusterName_)
  1771. .Item("input_table_paths").List(inputs)
  1772. .Item("output_table_path").Value(output)
  1773. .DoIf(spec.NetworkName_.Defined(), [&] (TFluentMap fluent) {
  1774. fluent.Item("network_name").Value(*spec.NetworkName_);
  1775. })
  1776. .DoIf(spec.SchemaInferenceMode_.Defined(), [&] (TFluentMap fluent) {
  1777. fluent.Item("schema_inference_mode").Value(ToString(*spec.SchemaInferenceMode_));
  1778. })
  1779. .Item("copy_attributes").Value(spec.CopyAttributes_)
  1780. .DoIf(!spec.AttributeKeys_.empty(), [&] (TFluentMap fluent) {
  1781. Y_ENSURE_EX(spec.CopyAttributes_, TApiUsageError() <<
  1782. "Specifying nonempty AttributeKeys in RemoteCopy "
  1783. "doesn't make sense without CopyAttributes == true");
  1784. fluent.Item("attribute_keys").List(spec.AttributeKeys_);
  1785. })
  1786. .Do(std::bind(BuildCommonOperationPart<TRemoteCopyOperationSpec>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1787. .EndMap().EndMap();
  1788. auto startOperation = [
  1789. operation=operation.Get(),
  1790. spec=MergeSpec(specNode, preparer->GetContext().Config->Spec, options),
  1791. preparer,
  1792. inputs,
  1793. output
  1794. ] () {
  1795. auto operationId = preparer->StartOperation(operation, "remote_copy", spec);
  1796. LogYPaths(operationId, inputs, "input");
  1797. LogYPath(operationId, output, "output");
  1798. return operationId;
  1799. };
  1800. operation->SetDelayedStartFunction(std::move(startOperation));
  1801. }
  1802. void ExecuteVanilla(
  1803. const TOperationPtr& operation,
  1804. const TOperationPreparerPtr& preparer,
  1805. const TVanillaOperationSpec& spec,
  1806. const TOperationOptions& options)
  1807. {
  1808. YT_LOG_DEBUG("Starting vanilla operation (PreparationId: %v)",
  1809. preparer->GetPreparationId());
  1810. auto addTask = [&](TFluentMap fluent, const TVanillaTask& task) {
  1811. Y_ABORT_UNLESS(task.Job_.Get());
  1812. if (std::holds_alternative<TVoidStructuredRowStream>(task.Job_->GetOutputRowStreamDescription())) {
  1813. Y_ENSURE_EX(task.Outputs_.empty(),
  1814. TApiUsageError() << "Vanilla task with void IVanillaJob doesn't expect output tables");
  1815. TJobPreparer jobPreparer(
  1816. *preparer,
  1817. task.Spec_,
  1818. *task.Job_,
  1819. /* outputTableCount */ 0,
  1820. /* smallFileList */ {},
  1821. options);
  1822. fluent
  1823. .Item(task.Name_).BeginMap()
  1824. .Item("job_count").Value(task.JobCount_)
  1825. .DoIf(task.NetworkProject_.Defined(), [&](TFluentMap fluent) {
  1826. fluent.Item("network_project").Value(*task.NetworkProject_);
  1827. })
  1828. .Do([&] (TFluentMap fluent) {
  1829. BuildUserJobFluently(
  1830. std::cref(jobPreparer),
  1831. /* inputFormat */ Nothing(),
  1832. /* outputFormat */ Nothing(),
  1833. fluent);
  1834. })
  1835. .EndMap();
  1836. } else {
  1837. auto operationIo = CreateSimpleOperationIo(
  1838. *task.Job_,
  1839. *preparer,
  1840. task,
  1841. options,
  1842. false);
  1843. Y_ENSURE_EX(operationIo.Outputs.size() > 0,
  1844. TApiUsageError() << "Vanilla task with IVanillaJob that has table writer expects output tables");
  1845. if (options.CreateOutputTables_) {
  1846. CreateOutputTables(*preparer, operationIo.Outputs);
  1847. }
  1848. TJobPreparer jobPreparer(
  1849. *preparer,
  1850. task.Spec_,
  1851. *task.Job_,
  1852. operationIo.Outputs.size(),
  1853. operationIo.JobFiles,
  1854. options);
  1855. fluent
  1856. .Item(task.Name_).BeginMap()
  1857. .Item("job_count").Value(task.JobCount_)
  1858. .DoIf(task.NetworkProject_.Defined(), [&](TFluentMap fluent) {
  1859. fluent.Item("network_project").Value(*task.NetworkProject_);
  1860. })
  1861. .Do([&] (TFluentMap fluent) {
  1862. BuildUserJobFluently(
  1863. std::cref(jobPreparer),
  1864. /* inputFormat */ Nothing(),
  1865. operationIo.OutputFormat,
  1866. fluent);
  1867. })
  1868. .Item("output_table_paths").List(operationIo.Outputs)
  1869. .Item("job_io").BeginMap()
  1870. .DoIf(!preparer->GetContext().Config->TableWriter.Empty(), [&](TFluentMap fluent) {
  1871. fluent.Item("table_writer").Value(preparer->GetContext().Config->TableWriter);
  1872. })
  1873. .Item("control_attributes").BeginMap()
  1874. .Item("enable_row_index").Value(TNode(true))
  1875. .Item("enable_range_index").Value(TNode(true))
  1876. .EndMap()
  1877. .EndMap()
  1878. .EndMap();
  1879. }
  1880. };
  1881. if (options.CreateDebugOutputTables_) {
  1882. CreateDebugOutputTables(spec, *preparer);
  1883. }
  1884. TNode specNode = BuildYsonNodeFluently()
  1885. .BeginMap().Item("spec").BeginMap()
  1886. .Item("tasks").DoMapFor(spec.Tasks_, addTask)
  1887. .Do(std::bind(BuildCommonOperationPart<TVanillaOperationSpec>, preparer->GetContext().Config, spec, options, std::placeholders::_1))
  1888. .EndMap().EndMap();
  1889. BuildCommonUserOperationPart(spec, &specNode["spec"]);
  1890. auto startOperation = [operation=operation.Get(), spec=MergeSpec(std::move(specNode), preparer->GetContext().Config->Spec, options), preparer] () {
  1891. auto operationId = preparer->StartOperation(operation, "vanilla", spec, /* useStartOperationRequest */ true);
  1892. return operationId;
  1893. };
  1894. operation->SetDelayedStartFunction(std::move(startOperation));
  1895. }
  1896. ////////////////////////////////////////////////////////////////////////////////
  1897. class TOperation::TOperationImpl
  1898. : public TThrRefBase
  1899. {
  1900. public:
  1901. TOperationImpl(
  1902. IClientRetryPolicyPtr clientRetryPolicy,
  1903. TClientContext context,
  1904. const TMaybe<TOperationId>& operationId = {})
  1905. : ClientRetryPolicy_(clientRetryPolicy)
  1906. , Context_(std::move(context))
  1907. , Id_(operationId)
  1908. , PreparedPromise_(::NThreading::NewPromise<void>())
  1909. , StartedPromise_(::NThreading::NewPromise<void>())
  1910. {
  1911. if (Id_) {
  1912. PreparedPromise_.SetValue();
  1913. StartedPromise_.SetValue();
  1914. } else {
  1915. PreparedPromise_.GetFuture().Subscribe([this_=::TIntrusivePtr(this)] (const ::NThreading::TFuture<void>& preparedResult) {
  1916. try {
  1917. preparedResult.GetValue();
  1918. } catch (...) {
  1919. this_->StartedPromise_.SetException(std::current_exception());
  1920. return;
  1921. }
  1922. });
  1923. }
  1924. }
  1925. const TOperationId& GetId() const;
  1926. TString GetWebInterfaceUrl() const;
  1927. void OnPrepared();
  1928. void SetDelayedStartFunction(std::function<TOperationId()> start);
  1929. void Start();
  1930. bool IsStarted() const;
  1931. void OnPreparationException(std::exception_ptr e);
  1932. TString GetStatus();
  1933. void OnStatusUpdated(const TString& newStatus);
  1934. ::NThreading::TFuture<void> GetPreparedFuture();
  1935. ::NThreading::TFuture<void> GetStartedFuture();
  1936. ::NThreading::TFuture<void> Watch(TClientPtr client);
  1937. EOperationBriefState GetBriefState();
  1938. TMaybe<TYtError> GetError();
  1939. TJobStatistics GetJobStatistics();
  1940. TMaybe<TOperationBriefProgress> GetBriefProgress();
  1941. void AbortOperation();
  1942. void CompleteOperation();
  1943. void SuspendOperation(const TSuspendOperationOptions& options);
  1944. void ResumeOperation(const TResumeOperationOptions& options);
  1945. TOperationAttributes GetAttributes(const TGetOperationOptions& options);
  1946. void UpdateParameters(const TUpdateOperationParametersOptions& options);
  1947. TJobAttributes GetJob(const TJobId& jobId, const TGetJobOptions& options);
  1948. TListJobsResult ListJobs(const TListJobsOptions& options);
  1949. void AsyncFinishOperation(TOperationAttributes operationAttributes);
  1950. void FinishWithException(std::exception_ptr exception);
  1951. void UpdateBriefProgress(TMaybe<TOperationBriefProgress> briefProgress);
  1952. void AnalyzeUnrecognizedSpec(TNode unrecognizedSpec);
  1953. const TClientContext& GetContext() const;
  1954. private:
  1955. void OnStarted(const TOperationId& operationId);
  1956. void UpdateAttributesAndCall(bool needJobStatistics, std::function<void(const TOperationAttributes&)> func);
  1957. void SyncFinishOperationImpl(const TOperationAttributes&);
  1958. static void* SyncFinishOperationProc(void* );
  1959. void ValidateOperationStarted() const;
  1960. private:
  1961. IClientRetryPolicyPtr ClientRetryPolicy_;
  1962. const TClientContext Context_;
  1963. TMaybe<TOperationId> Id_;
  1964. TMutex Lock_;
  1965. ::NThreading::TPromise<void> PreparedPromise_;
  1966. ::NThreading::TPromise<void> StartedPromise_;
  1967. TMaybe<::NThreading::TPromise<void>> CompletePromise_;
  1968. std::function<TOperationId()> DelayedStartFunction_;
  1969. TString Status_;
  1970. TOperationAttributes Attributes_;
  1971. };
  1972. ////////////////////////////////////////////////////////////////////////////////
  1973. class TOperationPollerItem
  1974. : public IYtPollerItem
  1975. {
  1976. public:
  1977. TOperationPollerItem(::TIntrusivePtr<TOperation::TOperationImpl> operationImpl)
  1978. : OperationImpl_(std::move(operationImpl))
  1979. { }
  1980. void PrepareRequest(TRawBatchRequest* batchRequest) override
  1981. {
  1982. auto filter = TOperationAttributeFilter()
  1983. .Add(EOperationAttribute::State)
  1984. .Add(EOperationAttribute::BriefProgress)
  1985. .Add(EOperationAttribute::Result);
  1986. if (!UnrecognizedSpecAnalyzed_) {
  1987. filter.Add(EOperationAttribute::UnrecognizedSpec);
  1988. }
  1989. OperationState_ = batchRequest->GetOperation(
  1990. OperationImpl_->GetId(),
  1991. TGetOperationOptions().AttributeFilter(filter));
  1992. }
  1993. EStatus OnRequestExecuted() override
  1994. {
  1995. try {
  1996. const auto& attributes = OperationState_.GetValue();
  1997. if (!UnrecognizedSpecAnalyzed_ && !attributes.UnrecognizedSpec.Empty()) {
  1998. OperationImpl_->AnalyzeUnrecognizedSpec(*attributes.UnrecognizedSpec);
  1999. UnrecognizedSpecAnalyzed_ = true;
  2000. }
  2001. Y_ABORT_UNLESS(attributes.BriefState,
  2002. "get_operation for operation %s has not returned \"state\" field",
  2003. GetGuidAsString(OperationImpl_->GetId()).Data());
  2004. if (*attributes.BriefState != EOperationBriefState::InProgress) {
  2005. OperationImpl_->AsyncFinishOperation(attributes);
  2006. return PollBreak;
  2007. } else {
  2008. OperationImpl_->UpdateBriefProgress(attributes.BriefProgress);
  2009. }
  2010. } catch (const TErrorResponse& e) {
  2011. if (!IsRetriable(e)) {
  2012. OperationImpl_->FinishWithException(std::current_exception());
  2013. return PollBreak;
  2014. }
  2015. } catch (const std::exception& e) {
  2016. OperationImpl_->FinishWithException(std::current_exception());
  2017. return PollBreak;
  2018. }
  2019. return PollContinue;
  2020. }
  2021. void OnItemDiscarded() override {
  2022. OperationImpl_->FinishWithException(std::make_exception_ptr(yexception() << "Operation cancelled"));
  2023. }
  2024. private:
  2025. ::TIntrusivePtr<TOperation::TOperationImpl> OperationImpl_;
  2026. ::NThreading::TFuture<TOperationAttributes> OperationState_;
  2027. bool UnrecognizedSpecAnalyzed_ = false;
  2028. };
  2029. ////////////////////////////////////////////////////////////////////////////////
  2030. const TOperationId& TOperation::TOperationImpl::GetId() const
  2031. {
  2032. ValidateOperationStarted();
  2033. return *Id_;
  2034. }
  2035. TString TOperation::TOperationImpl::GetWebInterfaceUrl() const
  2036. {
  2037. ValidateOperationStarted();
  2038. return GetOperationWebInterfaceUrl(Context_.ServerName, *Id_);
  2039. }
  2040. void TOperation::TOperationImpl::OnPrepared()
  2041. {
  2042. Y_ABORT_UNLESS(!PreparedPromise_.HasException() && !PreparedPromise_.HasValue());
  2043. PreparedPromise_.SetValue();
  2044. }
  2045. void TOperation::TOperationImpl::SetDelayedStartFunction(std::function<TOperationId()> start)
  2046. {
  2047. DelayedStartFunction_ = std::move(start);
  2048. }
  2049. void TOperation::TOperationImpl::Start()
  2050. {
  2051. {
  2052. auto guard = Guard(Lock_);
  2053. if (Id_) {
  2054. ythrow TApiUsageError() << "Start() should not be called on running operations";
  2055. }
  2056. }
  2057. GetPreparedFuture().GetValueSync();
  2058. std::function<TOperationId()> startStuff;
  2059. {
  2060. auto guard = Guard(Lock_);
  2061. startStuff.swap(DelayedStartFunction_);
  2062. }
  2063. if (!startStuff) {
  2064. ythrow TApiUsageError() << "Seems that Start() was called multiple times. If not, contact yt@";
  2065. }
  2066. TOperationId operationId;
  2067. try {
  2068. operationId = startStuff();
  2069. } catch (...) {
  2070. auto exception = std::current_exception();
  2071. StartedPromise_.SetException(exception);
  2072. std::rethrow_exception(exception);
  2073. }
  2074. OnStarted(operationId);
  2075. }
  2076. bool TOperation::TOperationImpl::IsStarted() const {
  2077. auto guard = Guard(Lock_);
  2078. return bool(Id_);
  2079. }
  2080. void TOperation::TOperationImpl::OnPreparationException(std::exception_ptr e)
  2081. {
  2082. Y_ABORT_UNLESS(!PreparedPromise_.HasValue() && !PreparedPromise_.HasException());
  2083. PreparedPromise_.SetException(e);
  2084. }
  2085. TString TOperation::TOperationImpl::GetStatus()
  2086. {
  2087. {
  2088. auto guard = Guard(Lock_);
  2089. if (!Id_) {
  2090. return Status_;
  2091. }
  2092. }
  2093. TMaybe<TString> state;
  2094. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2095. state = attributes.State;
  2096. });
  2097. return "On YT cluster: " + state.GetOrElse("undefined state");
  2098. }
  2099. void TOperation::TOperationImpl::OnStatusUpdated(const TString& newStatus)
  2100. {
  2101. auto guard = Guard(Lock_);
  2102. Status_ = newStatus;
  2103. }
  2104. ::NThreading::TFuture<void> TOperation::TOperationImpl::GetPreparedFuture()
  2105. {
  2106. return PreparedPromise_.GetFuture();
  2107. }
  2108. ::NThreading::TFuture<void> TOperation::TOperationImpl::GetStartedFuture()
  2109. {
  2110. return StartedPromise_.GetFuture();
  2111. }
  2112. ::NThreading::TFuture<void> TOperation::TOperationImpl::Watch(TClientPtr client)
  2113. {
  2114. {
  2115. auto guard = Guard(Lock_);
  2116. if (CompletePromise_) {
  2117. return *CompletePromise_;
  2118. }
  2119. CompletePromise_ = ::NThreading::NewPromise<void>();
  2120. }
  2121. GetStartedFuture().Subscribe([
  2122. this_=::TIntrusivePtr(this),
  2123. client=std::move(client)
  2124. ] (const ::NThreading::TFuture<void>& startedResult) {
  2125. try {
  2126. startedResult.GetValue();
  2127. } catch (...) {
  2128. this_->CompletePromise_->SetException(std::current_exception());
  2129. return;
  2130. }
  2131. client->GetYtPoller().Watch(::MakeIntrusive<TOperationPollerItem>(this_));
  2132. auto operationId = this_->GetId();
  2133. auto registry = TAbortableRegistry::Get();
  2134. registry->Add(
  2135. operationId,
  2136. ::MakeIntrusive<TOperationAbortable>(this_->ClientRetryPolicy_, this_->Context_, operationId));
  2137. // We have to own an IntrusivePtr to registry to prevent use-after-free
  2138. auto removeOperation = [registry, operationId] (const ::NThreading::TFuture<void>&) {
  2139. registry->Remove(operationId);
  2140. };
  2141. this_->CompletePromise_->GetFuture().Subscribe(removeOperation);
  2142. });
  2143. return *CompletePromise_;
  2144. }
  2145. EOperationBriefState TOperation::TOperationImpl::GetBriefState()
  2146. {
  2147. ValidateOperationStarted();
  2148. EOperationBriefState result = EOperationBriefState::InProgress;
  2149. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2150. Y_ABORT_UNLESS(attributes.BriefState,
  2151. "get_operation for operation %s has not returned \"state\" field",
  2152. GetGuidAsString(*Id_).Data());
  2153. result = *attributes.BriefState;
  2154. });
  2155. return result;
  2156. }
  2157. TMaybe<TYtError> TOperation::TOperationImpl::GetError()
  2158. {
  2159. ValidateOperationStarted();
  2160. TMaybe<TYtError> result;
  2161. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2162. Y_ABORT_UNLESS(attributes.Result);
  2163. result = attributes.Result->Error;
  2164. });
  2165. return result;
  2166. }
  2167. TJobStatistics TOperation::TOperationImpl::GetJobStatistics()
  2168. {
  2169. ValidateOperationStarted();
  2170. TJobStatistics result;
  2171. UpdateAttributesAndCall(true, [&] (const TOperationAttributes& attributes) {
  2172. if (attributes.Progress) {
  2173. result = attributes.Progress->JobStatistics;
  2174. }
  2175. });
  2176. return result;
  2177. }
  2178. TMaybe<TOperationBriefProgress> TOperation::TOperationImpl::GetBriefProgress()
  2179. {
  2180. ValidateOperationStarted();
  2181. {
  2182. auto g = Guard(Lock_);
  2183. if (CompletePromise_.Defined()) {
  2184. // Poller do this job for us
  2185. return Attributes_.BriefProgress;
  2186. }
  2187. }
  2188. TMaybe<TOperationBriefProgress> result;
  2189. UpdateAttributesAndCall(false, [&] (const TOperationAttributes& attributes) {
  2190. result = attributes.BriefProgress;
  2191. });
  2192. return result;
  2193. }
  2194. void TOperation::TOperationImpl::UpdateBriefProgress(TMaybe<TOperationBriefProgress> briefProgress)
  2195. {
  2196. auto g = Guard(Lock_);
  2197. Attributes_.BriefProgress = std::move(briefProgress);
  2198. }
  2199. void TOperation::TOperationImpl::AnalyzeUnrecognizedSpec(TNode unrecognizedSpec)
  2200. {
  2201. static const TVector<TVector<TString>> knownUnrecognizedSpecFieldPaths = {
  2202. {"mapper", "class_name"},
  2203. {"reducer", "class_name"},
  2204. {"reduce_combiner", "class_name"},
  2205. };
  2206. auto removeByPath = [] (TNode& node, auto pathBegin, auto pathEnd, auto& removeByPath) {
  2207. if (pathBegin == pathEnd) {
  2208. return;
  2209. }
  2210. if (!node.IsMap()) {
  2211. return;
  2212. }
  2213. auto* child = node.AsMap().FindPtr(*pathBegin);
  2214. if (!child) {
  2215. return;
  2216. }
  2217. removeByPath(*child, std::next(pathBegin), pathEnd, removeByPath);
  2218. if (std::next(pathBegin) == pathEnd || (child->IsMap() && child->Empty())) {
  2219. node.AsMap().erase(*pathBegin);
  2220. }
  2221. };
  2222. Y_ABORT_UNLESS(unrecognizedSpec.IsMap());
  2223. for (const auto& knownFieldPath : knownUnrecognizedSpecFieldPaths) {
  2224. Y_ABORT_UNLESS(!knownFieldPath.empty());
  2225. removeByPath(unrecognizedSpec, knownFieldPath.cbegin(), knownFieldPath.cend(), removeByPath);
  2226. }
  2227. if (!unrecognizedSpec.Empty()) {
  2228. YT_LOG_INFO(
  2229. "WARNING! Unrecognized spec for operation %s is not empty "
  2230. "(fields added by the YT API library are excluded): %s",
  2231. GetGuidAsString(*Id_).Data(),
  2232. NodeToYsonString(unrecognizedSpec).Data());
  2233. }
  2234. }
  2235. void TOperation::TOperationImpl::OnStarted(const TOperationId& operationId)
  2236. {
  2237. auto guard = Guard(Lock_);
  2238. Y_ABORT_UNLESS(!Id_,
  2239. "OnStarted() called with operationId = %s for operation with id %s",
  2240. GetGuidAsString(operationId).Data(),
  2241. GetGuidAsString(*Id_).Data());
  2242. Id_ = operationId;
  2243. Y_ABORT_UNLESS(!StartedPromise_.HasValue() && !StartedPromise_.HasException());
  2244. StartedPromise_.SetValue();
  2245. }
  2246. void TOperation::TOperationImpl::UpdateAttributesAndCall(bool needJobStatistics, std::function<void(const TOperationAttributes&)> func)
  2247. {
  2248. {
  2249. auto g = Guard(Lock_);
  2250. if (Attributes_.BriefState
  2251. && *Attributes_.BriefState != EOperationBriefState::InProgress
  2252. && (!needJobStatistics || Attributes_.Progress))
  2253. {
  2254. func(Attributes_);
  2255. return;
  2256. }
  2257. }
  2258. TOperationAttributes attributes = NDetail::GetOperation(
  2259. ClientRetryPolicy_->CreatePolicyForGenericRequest(),
  2260. Context_,
  2261. *Id_,
  2262. TGetOperationOptions().AttributeFilter(TOperationAttributeFilter()
  2263. .Add(EOperationAttribute::Result)
  2264. .Add(EOperationAttribute::Progress)
  2265. .Add(EOperationAttribute::State)
  2266. .Add(EOperationAttribute::BriefProgress)));
  2267. func(attributes);
  2268. Y_ENSURE(attributes.BriefState);
  2269. if (*attributes.BriefState != EOperationBriefState::InProgress) {
  2270. auto g = Guard(Lock_);
  2271. Attributes_ = std::move(attributes);
  2272. }
  2273. }
  2274. void TOperation::TOperationImpl::FinishWithException(std::exception_ptr e)
  2275. {
  2276. CompletePromise_->SetException(std::move(e));
  2277. }
  2278. void TOperation::TOperationImpl::AbortOperation()
  2279. {
  2280. ValidateOperationStarted();
  2281. NYT::NDetail::AbortOperation(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_);
  2282. }
  2283. void TOperation::TOperationImpl::CompleteOperation()
  2284. {
  2285. ValidateOperationStarted();
  2286. NYT::NDetail::CompleteOperation(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_);
  2287. }
  2288. void TOperation::TOperationImpl::SuspendOperation(const TSuspendOperationOptions& options)
  2289. {
  2290. ValidateOperationStarted();
  2291. NYT::NDetail::SuspendOperation(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, options);
  2292. }
  2293. void TOperation::TOperationImpl::ResumeOperation(const TResumeOperationOptions& options)
  2294. {
  2295. ValidateOperationStarted();
  2296. NYT::NDetail::ResumeOperation(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, options);
  2297. }
  2298. TOperationAttributes TOperation::TOperationImpl::GetAttributes(const TGetOperationOptions& options)
  2299. {
  2300. ValidateOperationStarted();
  2301. return NYT::NDetail::GetOperation(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, options);
  2302. }
  2303. void TOperation::TOperationImpl::UpdateParameters(const TUpdateOperationParametersOptions& options)
  2304. {
  2305. ValidateOperationStarted();
  2306. return NYT::NDetail::UpdateOperationParameters(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, options);
  2307. }
  2308. TJobAttributes TOperation::TOperationImpl::GetJob(const TJobId& jobId, const TGetJobOptions& options)
  2309. {
  2310. ValidateOperationStarted();
  2311. return NYT::NDetail::GetJob(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, jobId, options);
  2312. }
  2313. TListJobsResult TOperation::TOperationImpl::ListJobs(const TListJobsOptions& options)
  2314. {
  2315. ValidateOperationStarted();
  2316. return NYT::NDetail::ListJobs(ClientRetryPolicy_->CreatePolicyForGenericRequest(), Context_, *Id_, options);
  2317. }
  2318. struct TAsyncFinishOperationsArgs
  2319. {
  2320. ::TIntrusivePtr<TOperation::TOperationImpl> OperationImpl;
  2321. TOperationAttributes OperationAttributes;
  2322. };
  2323. void TOperation::TOperationImpl::AsyncFinishOperation(TOperationAttributes operationAttributes)
  2324. {
  2325. auto args = new TAsyncFinishOperationsArgs;
  2326. args->OperationImpl = this;
  2327. args->OperationAttributes = std::move(operationAttributes);
  2328. TThread thread(TThread::TParams(&TOperation::TOperationImpl::SyncFinishOperationProc, args).SetName("finish operation"));
  2329. thread.Start();
  2330. thread.Detach();
  2331. }
  2332. void* TOperation::TOperationImpl::SyncFinishOperationProc(void* pArgs)
  2333. {
  2334. THolder<TAsyncFinishOperationsArgs> args(static_cast<TAsyncFinishOperationsArgs*>(pArgs));
  2335. args->OperationImpl->SyncFinishOperationImpl(args->OperationAttributes);
  2336. return nullptr;
  2337. }
  2338. void TOperation::TOperationImpl::SyncFinishOperationImpl(const TOperationAttributes& attributes)
  2339. {
  2340. {
  2341. auto guard = Guard(Lock_);
  2342. Y_ABORT_UNLESS(Id_);
  2343. }
  2344. Y_ABORT_UNLESS(attributes.BriefState,
  2345. "get_operation for operation %s has not returned \"state\" field",
  2346. GetGuidAsString(*Id_).Data());
  2347. Y_ABORT_UNLESS(*attributes.BriefState != EOperationBriefState::InProgress);
  2348. {
  2349. try {
  2350. // `attributes' that came from poller don't have JobStatistics
  2351. // so we call `GetJobStatistics' in order to get it from server
  2352. // and cache inside object.
  2353. GetJobStatistics();
  2354. } catch (const TErrorResponse& ) {
  2355. // But if for any reason we failed to get attributes
  2356. // we complete operation using what we have.
  2357. auto g = Guard(Lock_);
  2358. Attributes_ = attributes;
  2359. }
  2360. }
  2361. if (*attributes.BriefState == EOperationBriefState::Completed) {
  2362. CompletePromise_->SetValue();
  2363. } else if (*attributes.BriefState == EOperationBriefState::Aborted || *attributes.BriefState == EOperationBriefState::Failed) {
  2364. Y_ABORT_UNLESS(attributes.Result && attributes.Result->Error);
  2365. const auto& error = *attributes.Result->Error;
  2366. YT_LOG_ERROR("Operation %v is `%v' with error: %v",
  2367. *Id_,
  2368. ToString(*attributes.BriefState),
  2369. error.FullDescription());
  2370. TString additionalExceptionText;
  2371. TVector<TFailedJobInfo> failedJobStderrInfo;
  2372. if (*attributes.BriefState == EOperationBriefState::Failed) {
  2373. try {
  2374. failedJobStderrInfo = NYT::NDetail::GetFailedJobInfo(ClientRetryPolicy_, Context_, *Id_, TGetFailedJobInfoOptions());
  2375. } catch (const std::exception& e) {
  2376. additionalExceptionText = "Cannot get job stderrs: ";
  2377. additionalExceptionText += e.what();
  2378. }
  2379. }
  2380. CompletePromise_->SetException(
  2381. std::make_exception_ptr(
  2382. TOperationFailedError(
  2383. *attributes.BriefState == EOperationBriefState::Failed
  2384. ? TOperationFailedError::Failed
  2385. : TOperationFailedError::Aborted,
  2386. *Id_,
  2387. error,
  2388. failedJobStderrInfo) << additionalExceptionText));
  2389. }
  2390. }
  2391. void TOperation::TOperationImpl::ValidateOperationStarted() const
  2392. {
  2393. auto guard = Guard(Lock_);
  2394. if (!Id_) {
  2395. ythrow TApiUsageError() << "Operation is not started";
  2396. }
  2397. }
  2398. const TClientContext& TOperation::TOperationImpl::GetContext() const
  2399. {
  2400. return Context_;
  2401. }
  2402. ////////////////////////////////////////////////////////////////////////////////
  2403. TOperation::TOperation(TClientPtr client)
  2404. : Client_(std::move(client))
  2405. , Impl_(::MakeIntrusive<TOperationImpl>(Client_->GetRetryPolicy(), Client_->GetContext()))
  2406. {
  2407. }
  2408. TOperation::TOperation(TOperationId id, TClientPtr client)
  2409. : Client_(std::move(client))
  2410. , Impl_(::MakeIntrusive<TOperationImpl>(Client_->GetRetryPolicy(), Client_->GetContext(), id))
  2411. {
  2412. }
  2413. const TOperationId& TOperation::GetId() const
  2414. {
  2415. return Impl_->GetId();
  2416. }
  2417. TString TOperation::GetWebInterfaceUrl() const
  2418. {
  2419. return Impl_->GetWebInterfaceUrl();
  2420. }
  2421. void TOperation::OnPrepared()
  2422. {
  2423. Impl_->OnPrepared();
  2424. }
  2425. void TOperation::SetDelayedStartFunction(std::function<TOperationId()> start)
  2426. {
  2427. Impl_->SetDelayedStartFunction(std::move(start));
  2428. }
  2429. void TOperation::Start()
  2430. {
  2431. Impl_->Start();
  2432. }
  2433. bool TOperation::IsStarted() const
  2434. {
  2435. return Impl_->IsStarted();
  2436. }
  2437. void TOperation::OnPreparationException(std::exception_ptr e)
  2438. {
  2439. Impl_->OnPreparationException(std::move(e));
  2440. }
  2441. TString TOperation::GetStatus() const
  2442. {
  2443. return Impl_->GetStatus();
  2444. }
  2445. void TOperation::OnStatusUpdated(const TString& newStatus)
  2446. {
  2447. Impl_->OnStatusUpdated(newStatus);
  2448. }
  2449. ::NThreading::TFuture<void> TOperation::GetPreparedFuture()
  2450. {
  2451. return Impl_->GetPreparedFuture();
  2452. }
  2453. ::NThreading::TFuture<void> TOperation::GetStartedFuture()
  2454. {
  2455. return Impl_->GetStartedFuture();
  2456. }
  2457. ::NThreading::TFuture<void> TOperation::Watch()
  2458. {
  2459. return Impl_->Watch(Client_);
  2460. }
  2461. TVector<TFailedJobInfo> TOperation::GetFailedJobInfo(const TGetFailedJobInfoOptions& options)
  2462. {
  2463. return NYT::NDetail::GetFailedJobInfo(Client_->GetRetryPolicy(), Client_->GetContext(), GetId(), options);
  2464. }
  2465. EOperationBriefState TOperation::GetBriefState()
  2466. {
  2467. return Impl_->GetBriefState();
  2468. }
  2469. TMaybe<TYtError> TOperation::GetError()
  2470. {
  2471. return Impl_->GetError();
  2472. }
  2473. TJobStatistics TOperation::GetJobStatistics()
  2474. {
  2475. return Impl_->GetJobStatistics();
  2476. }
  2477. TMaybe<TOperationBriefProgress> TOperation::GetBriefProgress()
  2478. {
  2479. return Impl_->GetBriefProgress();
  2480. }
  2481. void TOperation::AbortOperation()
  2482. {
  2483. Impl_->AbortOperation();
  2484. }
  2485. void TOperation::CompleteOperation()
  2486. {
  2487. Impl_->CompleteOperation();
  2488. }
  2489. void TOperation::SuspendOperation(const TSuspendOperationOptions& options)
  2490. {
  2491. Impl_->SuspendOperation(options);
  2492. }
  2493. void TOperation::ResumeOperation(const TResumeOperationOptions& options)
  2494. {
  2495. Impl_->ResumeOperation(options);
  2496. }
  2497. TOperationAttributes TOperation::GetAttributes(const TGetOperationOptions& options)
  2498. {
  2499. return Impl_->GetAttributes(options);
  2500. }
  2501. void TOperation::UpdateParameters(const TUpdateOperationParametersOptions& options)
  2502. {
  2503. Impl_->UpdateParameters(options);
  2504. }
  2505. TJobAttributes TOperation::GetJob(const TJobId& jobId, const TGetJobOptions& options)
  2506. {
  2507. return Impl_->GetJob(jobId, options);
  2508. }
  2509. TListJobsResult TOperation::ListJobs(const TListJobsOptions& options)
  2510. {
  2511. return Impl_->ListJobs(options);
  2512. }
  2513. ////////////////////////////////////////////////////////////////////////////////
  2514. struct TAsyncPrepareAndStartOperationArgs
  2515. {
  2516. std::function<void()> PrepareAndStart;
  2517. };
  2518. void* SyncPrepareAndStartOperation(void* pArgs)
  2519. {
  2520. THolder<TAsyncPrepareAndStartOperationArgs> args(static_cast<TAsyncPrepareAndStartOperationArgs*>(pArgs));
  2521. args->PrepareAndStart();
  2522. return nullptr;
  2523. }
  2524. ::TIntrusivePtr<TOperation> ProcessOperation(
  2525. NYT::NDetail::TClientPtr client,
  2526. std::function<void()> prepare,
  2527. ::TIntrusivePtr<TOperation> operation,
  2528. const TOperationOptions& options)
  2529. {
  2530. auto prepareAndStart = [prepare = std::move(prepare), operation, mode = options.StartOperationMode_] () {
  2531. try {
  2532. prepare();
  2533. operation->OnPrepared();
  2534. } catch (...) {
  2535. operation->OnPreparationException(std::current_exception());
  2536. }
  2537. if (mode >= TOperationOptions::EStartOperationMode::AsyncStart) {
  2538. try {
  2539. operation->Start();
  2540. } catch (...) { }
  2541. }
  2542. };
  2543. if (options.StartOperationMode_ >= TOperationOptions::EStartOperationMode::SyncStart) {
  2544. prepareAndStart();
  2545. WaitIfRequired(operation, client, options);
  2546. } else {
  2547. auto args = new TAsyncPrepareAndStartOperationArgs;
  2548. args->PrepareAndStart = std::move(prepareAndStart);
  2549. TThread thread(TThread::TParams(SyncPrepareAndStartOperation, args).SetName("prepare and start operation"));
  2550. thread.Start();
  2551. thread.Detach();
  2552. }
  2553. return operation;
  2554. }
  2555. void WaitIfRequired(const TOperationPtr& operation, const TClientPtr& client, const TOperationOptions& options)
  2556. {
  2557. auto retryPolicy = client->GetRetryPolicy();
  2558. auto context = client->GetContext();
  2559. if (options.StartOperationMode_ >= TOperationOptions::EStartOperationMode::SyncStart) {
  2560. operation->GetStartedFuture().GetValueSync();
  2561. }
  2562. if (options.StartOperationMode_ == TOperationOptions::EStartOperationMode::SyncWait) {
  2563. auto finishedFuture = operation->Watch();
  2564. TWaitProxy::Get()->WaitFuture(finishedFuture);
  2565. finishedFuture.GetValue();
  2566. if (context.Config->WriteStderrSuccessfulJobs) {
  2567. auto stderrs = GetJobsStderr(retryPolicy, context, operation->GetId());
  2568. for (const auto& jobStderr : stderrs) {
  2569. if (!jobStderr.empty()) {
  2570. Cerr << jobStderr << '\n';
  2571. }
  2572. }
  2573. }
  2574. }
  2575. }
  2576. ////////////////////////////////////////////////////////////////////////////////
  2577. void ResetUseClientProtobuf(const char* methodName)
  2578. {
  2579. Cerr << "WARNING! OPTION `TConfig::UseClientProtobuf' IS RESET TO `true'; "
  2580. << "IT CAN DETERIORATE YOUR CODE PERFORMANCE!!! DON'T USE DEPRECATED METHOD `"
  2581. << "TOperationIOSpec::" << methodName << "' TO AVOID THIS RESET" << Endl;
  2582. // Give users some time to contemplate about usage of deprecated functions.
  2583. Cerr << "Sleeping for 5 seconds..." << Endl;
  2584. Sleep(TDuration::Seconds(5));
  2585. TConfig::Get()->UseClientProtobuf = true;
  2586. }
  2587. } // namespace NDetail
  2588. ////////////////////////////////////////////////////////////////////////////////
  2589. ::TIntrusivePtr<INodeReaderImpl> CreateJobNodeReader(TRawTableReaderPtr rawTableReader)
  2590. {
  2591. if (auto schema = NDetail::GetJobInputSkiffSchema()) {
  2592. return new NDetail::TSkiffTableReader(rawTableReader, schema);
  2593. } else {
  2594. return new TNodeTableReader(rawTableReader);
  2595. }
  2596. }
  2597. ::TIntrusivePtr<IYaMRReaderImpl> CreateJobYaMRReader(TRawTableReaderPtr rawTableReader)
  2598. {
  2599. return new TYaMRTableReader(rawTableReader);
  2600. }
  2601. ::TIntrusivePtr<IProtoReaderImpl> CreateJobProtoReader(TRawTableReaderPtr rawTableReader)
  2602. {
  2603. if (TConfig::Get()->UseClientProtobuf) {
  2604. return new TProtoTableReader(
  2605. rawTableReader,
  2606. GetJobInputDescriptors());
  2607. } else {
  2608. return new TLenvalProtoTableReader(
  2609. rawTableReader,
  2610. GetJobInputDescriptors());
  2611. }
  2612. }
  2613. ::TIntrusivePtr<INodeWriterImpl> CreateJobNodeWriter(THolder<IProxyOutput> rawJobWriter)
  2614. {
  2615. return new TNodeTableWriter(std::move(rawJobWriter));
  2616. }
  2617. ::TIntrusivePtr<IYaMRWriterImpl> CreateJobYaMRWriter(THolder<IProxyOutput> rawJobWriter)
  2618. {
  2619. return new TYaMRTableWriter(std::move(rawJobWriter));
  2620. }
  2621. ::TIntrusivePtr<IProtoWriterImpl> CreateJobProtoWriter(THolder<IProxyOutput> rawJobWriter)
  2622. {
  2623. if (TConfig::Get()->UseClientProtobuf) {
  2624. return new TProtoTableWriter(
  2625. std::move(rawJobWriter),
  2626. GetJobOutputDescriptors());
  2627. } else {
  2628. return new TLenvalProtoTableWriter(
  2629. std::move(rawJobWriter),
  2630. GetJobOutputDescriptors());
  2631. }
  2632. }
  2633. ////////////////////////////////////////////////////////////////////////////////
  2634. } // namespace NYT