client.h 22 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598
  1. #pragma once
  2. ///
  3. /// @file yt/cpp/mapreduce/interface/client.h
  4. ///
  5. /// Main header of the C++ YT Wrapper.
  6. ///
  7. /// @mainpage C++ library for working with YT
  8. ///
  9. /// This library provides possibilities to work with YT as a [MapReduce](https://en.wikipedia.org/wiki/MapReduce) system. It allows:
  10. /// - to read/write tables and files
  11. /// - to run operations
  12. /// - to work with transactions.
  13. ///
  14. /// This library provides only basic functions for working with dynamic tables.
  15. /// To access full powers of YT dynamic tables one should use
  16. /// [yt/client](https://github.com/ytsaurus/ytsaurus/tree/main/yt/yt/client) library.
  17. ///
  18. /// Entry points to this library:
  19. /// - @ref NYT::Initialize() initialization function for this library;
  20. /// - @ref NYT::IClient main interface to work with YT cluster;
  21. /// - @ref NYT::CreateClient() function that creates client for particular cluster;
  22. /// - @ref NYT::IOperationClient ancestor of @ref NYT::IClient containing the set of methods to run operations.
  23. ///
  24. /// Tutorial on how to use this library can be found [here](https://ytsaurus.tech/docs/en/api/c/description).
  25. #include "fwd.h"
  26. #include "client_method_options.h"
  27. #include "constants.h"
  28. #include "batch_request.h"
  29. #include "cypress.h"
  30. #include "init.h"
  31. #include "io.h"
  32. #include "node.h"
  33. #include "operation.h"
  34. #include <library/cpp/threading/future/future.h>
  35. #include <util/datetime/base.h>
  36. #include <util/generic/maybe.h>
  37. #include <util/system/compiler.h>
  38. /// Main namespace of YT client
  39. namespace NYT {
  40. ////////////////////////////////////////////////////////////////////////////////
  41. /// OAuth info (returned by @ref NYT::IClient::WhoAmI).
  42. struct TAuthorizationInfo
  43. {
  44. /// User's login.
  45. TString Login;
  46. /// Realm.
  47. TString Realm;
  48. };
  49. ////////////////////////////////////////////////////////////////////////////////
  50. /// @brief Part of @ref NYT::TCheckPermissionResponse.
  51. ///
  52. /// In case when 'Action == ESecurityAction::Deny' because of a 'deny' rule,
  53. /// the "denying" object name and id and "denied" subject name an id may be returned.
  54. struct TCheckPermissionResult
  55. {
  56. /// Was the access granted or not.
  57. ESecurityAction Action;
  58. /// Id of the object whose ACL's "deny" rule forbids the access.
  59. TMaybe<TGUID> ObjectId;
  60. ///
  61. /// @brief Name of the object whose ACL's "deny" rule forbids the access.
  62. ///
  63. /// Example is "node //tmp/x/y".
  64. TMaybe<TString> ObjectName;
  65. /// Id of the subject for whom the access was denied by a "deny" rule.
  66. TMaybe<TGUID> SubjectId;
  67. /// Name of the subject for whom the access was denied by a "deny" rule.
  68. TMaybe<TString> SubjectName;
  69. };
  70. /// @brief Result of @ref NYT::IClient::CheckPermission command.
  71. ///
  72. /// The base part of the response corresponds to the check result for the node itself.
  73. /// `Columns` vector contains check results for the columns (in the same order as in the request).
  74. struct TCheckPermissionResponse
  75. : public TCheckPermissionResult
  76. {
  77. /// @brief Results for the table columns access permissions.
  78. ///
  79. /// @see [Columnar ACL doc](https://ytsaurus.tech/docs/en/user-guide/storage/columnar-acl)
  80. TVector<TCheckPermissionResult> Columns;
  81. };
  82. ////////////////////////////////////////////////////////////////////////////////
  83. /// @brief Interface representing a lock obtained from @ref NYT::ITransaction::Lock.
  84. ///
  85. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#start_tx)
  86. class ILock
  87. : public TThrRefBase
  88. {
  89. public:
  90. virtual ~ILock() = default;
  91. /// Get cypress node id of lock itself.
  92. virtual const TLockId& GetId() const = 0;
  93. /// Get cypress node id of locked object.
  94. virtual TNodeId GetLockedNodeId() const = 0;
  95. ///
  96. /// @brief Get future that will be set once lock is in "acquired" state.
  97. ///
  98. /// Note that future might contain exception if some error occurred
  99. /// e.g. lock transaction was aborted.
  100. virtual const ::NThreading::TFuture<void>& GetAcquiredFuture() const = 0;
  101. ///
  102. /// @brief Wait until lock is in "acquired" state.
  103. ///
  104. /// Throws exception if timeout exceeded or some error occurred
  105. /// e.g. lock transaction was aborted.
  106. void Wait(TDuration timeout = TDuration::Max());
  107. };
  108. ////////////////////////////////////////////////////////////////////////////////
  109. /// @brief Base class for @ref NYT::IClient and @ref NYT::ITransaction.
  110. ///
  111. /// This class contains transactional commands.
  112. class IClientBase
  113. : public TThrRefBase
  114. , public ICypressClient
  115. , public IIOClient
  116. , public IOperationClient
  117. {
  118. public:
  119. ///
  120. /// @brief Start a [transaction] (https://ytsaurus.tech/docs/en/user-guide/storage/transactions.html#master_transactions).
  121. ///
  122. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#start_tx)
  123. [[nodiscard]] virtual ITransactionPtr StartTransaction(
  124. const TStartTransactionOptions& options = TStartTransactionOptions()) = 0;
  125. ///
  126. /// @brief Change properties of table.
  127. ///
  128. /// Allows to:
  129. /// - switch table between dynamic/static mode
  130. /// - or change table schema
  131. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#alter_table)
  132. virtual void AlterTable(
  133. const TYPath& path,
  134. const TAlterTableOptions& options = TAlterTableOptions()) = 0;
  135. ///
  136. /// @brief Create batch request object that allows to execute several light requests in parallel.
  137. ///
  138. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#execute_batch)
  139. virtual TBatchRequestPtr CreateBatchRequest() = 0;
  140. ///
  141. /// @brief Get root client.
  142. ///
  143. /// @param ignoreGlobalTx root client could be created already attached to some global transaction, @ref NYT::TConfig::GlobalTx.
  144. /// when ignoreGlobalTx = false original client (attached to GlobalTx) is returned
  145. /// when ignoreGlobalTx = true, returned client is not attached to any transaction.
  146. ///
  147. /// TODO: rename to GetRootClient()
  148. virtual IClientPtr GetParentClient(bool ignoreGlobalTx = false) = 0;
  149. };
  150. ////////////////////////////////////////////////////////////////////////////////
  151. /// @brief Interface representing a master transaction.
  152. ///
  153. /// @see [YT doc](https://ytsaurus.tech/docs/en/user-guide/storage/transactions.html#master_transactions)
  154. class ITransaction
  155. : virtual public IClientBase
  156. {
  157. public:
  158. /// Get id of transaction.
  159. virtual const TTransactionId& GetId() const = 0;
  160. ///
  161. /// @brief Try to lock given path.
  162. ///
  163. /// Lock will be held until transaction is committed/aborted or @ref NYT::ITransaction::Unlock method is called.
  164. /// Lock modes:
  165. /// - `LM_EXCLUSIVE`: if exclusive lock is taken no other transaction can take exclusive or shared lock.
  166. /// - `LM_SHARED`: if shared lock is taken other transactions can take shared lock but not exclusive.
  167. /// - `LM_SNAPSHOT`: snapshot lock always succeeds, when snapshot lock is taken current transaction snapshots object.
  168. /// It will not see changes that occurred to it in other transactions.
  169. ///
  170. /// Exclusive/shared lock can be waitable or not.
  171. /// If nonwaitable lock cannot be taken exception is thrown.
  172. /// If waitable lock cannot be taken it is created in pending state and client can wait until it actually taken.
  173. /// Check @ref NYT::TLockOptions::Waitable and @ref NYT::ILock::GetAcquiredFuture for more details.
  174. ///
  175. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#lock)
  176. virtual ILockPtr Lock(
  177. const TYPath& path,
  178. ELockMode mode,
  179. const TLockOptions& options = TLockOptions()) = 0;
  180. ///
  181. /// @brief Remove all the locks (including pending ones) for this transaction from a Cypress node at `path`.
  182. ///
  183. /// If the locked version of the node differs from the original one,
  184. /// an error will be thrown.
  185. ///
  186. /// Command is successful even if the node has no locks.
  187. /// Only explicit (created by @ref NYT::ITransaction::Lock) locks are removed.
  188. ///
  189. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unlock)
  190. virtual void Unlock(
  191. const TYPath& path,
  192. const TUnlockOptions& options = TUnlockOptions()) = 0;
  193. ///
  194. /// @brief Commit transaction.
  195. ///
  196. /// All changes that are made by transactions become visible globally or to parent transaction.
  197. ///
  198. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#commit_tx)
  199. virtual void Commit() = 0;
  200. ///
  201. /// @brief Abort transaction.
  202. ///
  203. /// All changes made by current transaction are lost.
  204. ///
  205. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#abort_tx)
  206. virtual void Abort() = 0;
  207. /// @brief Explicitly ping transaction.
  208. ///
  209. /// User usually does not need this method (as transactions are pinged automatically,
  210. /// see @ref NYT::TStartTransactionOptions::AutoPingable).
  211. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#ping_tx)
  212. virtual void Ping() = 0;
  213. ///
  214. /// @brief Detach transaction.
  215. ///
  216. /// Stop any activities connected with it: pinging, aborting on crashes etc.
  217. /// Forget about the transaction totally.
  218. virtual void Detach();
  219. };
  220. ////////////////////////////////////////////////////////////////////////////////
  221. /// Interface containing non-transactional commands.
  222. class IClient
  223. : virtual public IClientBase
  224. {
  225. public:
  226. ///
  227. /// @brief Attach to existing master transaction.
  228. ///
  229. /// Returned object WILL NOT:
  230. /// - ping transaction automatically (unless @ref NYT::TAttachTransactionOptions::AutoPing is set)
  231. /// - abort it on program termination (unless @ref NYT::TAttachTransactionOptions::AbortOnTermination is set).
  232. /// Otherwise returned object is similar to the object returned by @ref NYT::IClientBase::StartTransaction.
  233. /// and it can see all the changes made inside the transaction.
  234. [[nodiscard]] virtual ITransactionPtr AttachTransaction(
  235. const TTransactionId& transactionId,
  236. const TAttachTransactionOptions& options = TAttachTransactionOptions()) = 0;
  237. ///
  238. /// @brief Mount dynamic table.
  239. ///
  240. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#mount_table)
  241. virtual void MountTable(
  242. const TYPath& path,
  243. const TMountTableOptions& options = TMountTableOptions()) = 0;
  244. ///
  245. /// @brief Unmount dynamic table.
  246. ///
  247. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unmount_table)
  248. virtual void UnmountTable(
  249. const TYPath& path,
  250. const TUnmountTableOptions& options = TUnmountTableOptions()) = 0;
  251. ///
  252. /// @brief Remount dynamic table.
  253. ///
  254. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#remount_table)
  255. virtual void RemountTable(
  256. const TYPath& path,
  257. const TRemountTableOptions& options = TRemountTableOptions()) = 0;
  258. ///
  259. /// @brief Switch dynamic table from `mounted' into `frozen' state.
  260. ///
  261. /// When table is in frozen state all its data is flushed to disk and writes are disabled.
  262. ///
  263. /// @note this function launches the process of switching, but doesn't wait until switching is accomplished.
  264. /// Waiting has to be performed by user.
  265. ///
  266. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#freeze_table)
  267. virtual void FreezeTable(
  268. const TYPath& path,
  269. const TFreezeTableOptions& options = TFreezeTableOptions()) = 0;
  270. ///
  271. /// @brief Switch dynamic table from `frozen` into `mounted` state.
  272. ///
  273. /// @note this function launches the process of switching, but doesn't wait until switching is accomplished.
  274. /// Waiting has to be performed by user.
  275. ///
  276. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unfreeze_table)
  277. virtual void UnfreezeTable(
  278. const TYPath& path,
  279. const TUnfreezeTableOptions& options = TUnfreezeTableOptions()) = 0;
  280. ///
  281. /// @brief Reshard dynamic table (break it into tablets) by given pivot keys.
  282. ///
  283. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#reshard_table)
  284. virtual void ReshardTable(
  285. const TYPath& path,
  286. const TVector<TKey>& pivotKeys,
  287. const TReshardTableOptions& options = TReshardTableOptions()) = 0;
  288. ///
  289. /// @brief Reshard dynamic table, breaking it into given number of tablets.
  290. ///
  291. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#reshard_table)
  292. virtual void ReshardTable(
  293. const TYPath& path,
  294. i64 tabletCount,
  295. const TReshardTableOptions& options = TReshardTableOptions()) = 0;
  296. ///
  297. /// @brief Insert rows into dynamic table.
  298. ///
  299. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#insert_rows)
  300. virtual void InsertRows(
  301. const TYPath& path,
  302. const TNode::TListType& rows,
  303. const TInsertRowsOptions& options = TInsertRowsOptions()) = 0;
  304. ///
  305. /// @brief Delete rows from dynamic table.
  306. ///
  307. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#delete_rows)
  308. virtual void DeleteRows(
  309. const TYPath& path,
  310. const TNode::TListType& keys,
  311. const TDeleteRowsOptions& options = TDeleteRowsOptions()) = 0;
  312. ///
  313. /// @brief Trim rows from the beginning of ordered dynamic table.
  314. ///
  315. /// Asynchronously removes `rowCount` rows from the beginning of ordered dynamic table.
  316. /// Numeration of remaining rows *does not change*, e.g. after `trim(10)` and `trim(20)`
  317. /// you get in total `20` deleted rows.
  318. ///
  319. /// @param path Path to ordered dynamic table.
  320. /// @param tabletIndex Which tablet to trim.
  321. /// @param rowCount How many trimmed rows will be in the table after command.
  322. /// @param options Optional parameters.
  323. ///
  324. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#trim_rows)
  325. virtual void TrimRows(
  326. const TYPath& path,
  327. i64 tabletIndex,
  328. i64 rowCount,
  329. const TTrimRowsOptions& options = TTrimRowsOptions()) = 0;
  330. ///
  331. /// @brief Lookup rows with given keys from dynamic table.
  332. ///
  333. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#lookup-rows)
  334. virtual TNode::TListType LookupRows(
  335. const TYPath& path,
  336. const TNode::TListType& keys,
  337. const TLookupRowsOptions& options = TLookupRowsOptions()) = 0;
  338. ///
  339. /// @brief Select rows from dynamic table, using [SQL dialect](https://ytsaurus.tech/docs/en//description/dynamic_tables/dyn_query_language.html).
  340. ///
  341. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#select-rows)
  342. virtual TNode::TListType SelectRows(
  343. const TString& query,
  344. const TSelectRowsOptions& options = TSelectRowsOptions()) = 0;
  345. ///
  346. /// @brief Change properties of table replica.
  347. ///
  348. /// Allows to enable/disable replica and/or change its mode.
  349. ///
  350. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#alter_table_replica)
  351. virtual void AlterTableReplica(
  352. const TReplicaId& replicaId,
  353. const TAlterTableReplicaOptions& alterTableReplicaOptions) = 0;
  354. ///
  355. /// @brief Generate a monotonously increasing master timestamp.
  356. ///
  357. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#generate_timestamp)
  358. virtual ui64 GenerateTimestamp() = 0;
  359. /// Return YT username of current client.
  360. virtual TAuthorizationInfo WhoAmI() = 0;
  361. ///
  362. /// @brief Get operation attributes.
  363. ///
  364. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_operation)
  365. virtual TOperationAttributes GetOperation(
  366. const TOperationId& operationId,
  367. const TGetOperationOptions& options = TGetOperationOptions()) = 0;
  368. ///
  369. /// @brief Get operation attributes.
  370. ///
  371. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_operation)
  372. virtual TOperationAttributes GetOperation(
  373. const TString& alias,
  374. const TGetOperationOptions& options = TGetOperationOptions()) = 0;
  375. ///
  376. /// @brief List operations satisfying given filters.
  377. ///
  378. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#list_operations)
  379. virtual TListOperationsResult ListOperations(
  380. const TListOperationsOptions& options = TListOperationsOptions()) = 0;
  381. ///
  382. /// @brief Update operation runtime parameters.
  383. ///
  384. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#update_operation_parameters)
  385. virtual void UpdateOperationParameters(
  386. const TOperationId& operationId,
  387. const TUpdateOperationParametersOptions& options) = 0;
  388. ///
  389. /// @brief Get job attributes.
  390. ///
  391. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job)
  392. virtual TJobAttributes GetJob(
  393. const TOperationId& operationId,
  394. const TJobId& jobId,
  395. const TGetJobOptions& options = TGetJobOptions()) = 0;
  396. ///
  397. /// List attributes of jobs satisfying given filters.
  398. ///
  399. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#list_jobs)
  400. virtual TListJobsResult ListJobs(
  401. const TOperationId& operationId,
  402. const TListJobsOptions& options = TListJobsOptions()) = 0;
  403. ///
  404. /// @brief Get the input of a running or failed job.
  405. ///
  406. /// @ref NYT::TErrorResponse exception is thrown if job is missing.
  407. ///
  408. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_input)
  409. virtual IFileReaderPtr GetJobInput(
  410. const TJobId& jobId,
  411. const TGetJobInputOptions& options = TGetJobInputOptions()) = 0;
  412. ///
  413. /// @brief Get fail context of a failed job.
  414. ///
  415. /// @ref NYT::TErrorResponse exception is thrown if it is missing.
  416. ///
  417. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_fail_context)
  418. virtual IFileReaderPtr GetJobFailContext(
  419. const TOperationId& operationId,
  420. const TJobId& jobId,
  421. const TGetJobFailContextOptions& options = TGetJobFailContextOptions()) = 0;
  422. ///
  423. /// @brief Get stderr of a running or failed job.
  424. ///
  425. /// @ref NYT::TErrorResponse exception is thrown if it is missing.
  426. ///
  427. /// @note YT doesn't store all job stderrs
  428. ///
  429. /// @note If job stderr exceeds few megabytes YT will store only head and tail of stderr.
  430. ///
  431. /// @see Description of `max_stderr_size` spec option [here](https://ytsaurus.tech/docs/en/user-guide/data-processing/operations/operations-options).
  432. ///
  433. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_stderr)
  434. virtual IFileReaderPtr GetJobStderr(
  435. const TOperationId& operationId,
  436. const TJobId& jobId,
  437. const TGetJobStderrOptions& options = TGetJobStderrOptions()) = 0;
  438. ///
  439. /// @brief Get trace of a job.
  440. ///
  441. /// @ref NYT::TErrorResponse exception is thrown if it is missing.
  442. ///
  443. /// @note YT doesn't store all job traces.
  444. ///
  445. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_trace)
  446. virtual std::vector<TJobTraceEvent> GetJobTrace(
  447. const TOperationId& operationId,
  448. const TGetJobTraceOptions& options = TGetJobTraceOptions()) = 0;
  449. ///
  450. /// @brief Create one or several rbtorrents for files in a blob table.
  451. ///
  452. /// If specified, one torrent is created for each value of `KeyColumns` option.
  453. /// Otherwise, a single torrent with all files of a table is created.
  454. ///
  455. /// @return list of nodes, each node has two fields
  456. /// * `key`: list of key columns values. Empty if `KeyColumns` is not specified.
  457. /// * `rbtorrent`: rbtorrent string (with `rbtorrent:` prefix)
  458. ///
  459. /// @see [More info.](https://docs.yandex-team.ru/docs/yt/description/storage/blobtables#sky_share)
  460. virtual TNode::TListType SkyShareTable(
  461. const std::vector<TYPath>& tablePaths,
  462. const TSkyShareTableOptions& options) = 0;
  463. ///
  464. /// @brief Check if `user` has `permission` to access a Cypress node at `path`.
  465. ///
  466. /// For tables access to columns specified in `options.Columns_` can be checked
  467. /// (@see [the doc](https://ytsaurus.tech/docs/en/user-guide/storage/columnar-acl)).
  468. ///
  469. /// If access is denied (the returned result has `.Action == ESecurityAction::Deny`)
  470. /// because of a `deny` rule, the "denying" object name and id
  471. /// and "denied" subject name an id may be returned.
  472. ///
  473. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#check_permission)
  474. virtual TCheckPermissionResponse CheckPermission(
  475. const TString& user,
  476. EPermission permission,
  477. const TYPath& path,
  478. const TCheckPermissionOptions& options = TCheckPermissionOptions()) = 0;
  479. /// @brief Get information about tablet
  480. /// @see NYT::TTabletInfo
  481. virtual TVector<TTabletInfo> GetTabletInfos(
  482. const TYPath& path,
  483. const TVector<int>& tabletIndexes,
  484. const TGetTabletInfosOptions& options = TGetTabletInfosOptions()) = 0;
  485. ///
  486. /// @brief Suspend operation.
  487. ///
  488. /// Jobs will be aborted.
  489. ///
  490. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#suspend_operation)
  491. virtual void SuspendOperation(
  492. const TOperationId& operationId,
  493. const TSuspendOperationOptions& options = TSuspendOperationOptions()) = 0;
  494. /// @brief Resume previously suspended operation.
  495. ///
  496. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#resume_operation)
  497. virtual void ResumeOperation(
  498. const TOperationId& operationId,
  499. const TResumeOperationOptions& options = TResumeOperationOptions()) = 0;
  500. ///
  501. /// @brief Synchronously terminates all client's background activities
  502. ///
  503. /// e.g. no callbacks will be executed after the function is completed
  504. ///
  505. /// @note It is safe to call Shutdown multiple times
  506. ///
  507. /// @note @ref NYT::TApiUsageError will be thrown if any client's method is called after shutdown
  508. ///
  509. virtual void Shutdown() = 0;
  510. };
  511. /// Create a rpc client for particular cluster.
  512. IClientPtr CreateRpcClient(
  513. const TString& serverName,
  514. const TCreateClientOptions& options = {});
  515. /// Create a client for particular MapReduce cluster.
  516. IClientPtr CreateClient(
  517. const TString& serverName,
  518. const TCreateClientOptions& options = {});
  519. /// Create a client for mapreduce cluster specified in `YT_PROXY` environment variable.
  520. IClientPtr CreateClientFromEnv(
  521. const TCreateClientOptions& options = {});
  522. ////////////////////////////////////////////////////////////////////////////////
  523. } // namespace NYT