client.h 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576
  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. /// @brief Get root client outside of all transactions.
  141. virtual IClientPtr GetParentClient() = 0;
  142. };
  143. ////////////////////////////////////////////////////////////////////////////////
  144. /// @brief Interface representing a master transaction.
  145. ///
  146. /// @see [YT doc](https://ytsaurus.tech/docs/en/user-guide/storage/transactions.html#master_transactions)
  147. class ITransaction
  148. : virtual public IClientBase
  149. {
  150. public:
  151. /// Get id of transaction.
  152. virtual const TTransactionId& GetId() const = 0;
  153. ///
  154. /// @brief Try to lock given path.
  155. ///
  156. /// Lock will be held until transaction is commited/aborted or @ref NYT::ITransaction::Unlock method is called.
  157. /// Lock modes:
  158. /// - `LM_EXCLUSIVE`: if exclusive lock is taken no other transaction can take exclusive or shared lock.
  159. /// - `LM_SHARED`: if shared lock is taken other transactions can take shared lock but not exclusive.
  160. /// - `LM_SNAPSHOT`: snapshot lock always succeeds, when snapshot lock is taken current transaction snapshots object.
  161. /// It will not see changes that occurred to it in other transactions.
  162. ///
  163. /// Exclusive/shared lock can be waitable or not.
  164. /// If nonwaitable lock cannot be taken exception is thrown.
  165. /// If waitable lock cannot be taken it is created in pending state and client can wait until it actually taken.
  166. /// Check @ref NYT::TLockOptions::Waitable and @ref NYT::ILock::GetAcquiredFuture for more details.
  167. ///
  168. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#lock)
  169. virtual ILockPtr Lock(
  170. const TYPath& path,
  171. ELockMode mode,
  172. const TLockOptions& options = TLockOptions()) = 0;
  173. ///
  174. /// @brief Remove all the locks (including pending ones) for this transaction from a Cypress node at `path`.
  175. ///
  176. /// If the locked version of the node differs from the original one,
  177. /// an error will be thrown.
  178. ///
  179. /// Command is successful even if the node has no locks.
  180. /// Only explicit (created by @ref NYT::ITransaction::Lock) locks are removed.
  181. ///
  182. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unlock)
  183. virtual void Unlock(
  184. const TYPath& path,
  185. const TUnlockOptions& options = TUnlockOptions()) = 0;
  186. ///
  187. /// @brief Commit transaction.
  188. ///
  189. /// All changes that are made by transactions become visible globally or to parent transaction.
  190. ///
  191. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#commit_tx)
  192. virtual void Commit() = 0;
  193. ///
  194. /// @brief Abort transaction.
  195. ///
  196. /// All changes made by current transaction are lost.
  197. ///
  198. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#abort_tx)
  199. virtual void Abort() = 0;
  200. /// @brief Explicitly ping transaction.
  201. ///
  202. /// User usually does not need this method (as transactions are pinged automatically,
  203. /// see @ref NYT::TStartTransactionOptions::AutoPingable).
  204. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#ping_tx)
  205. virtual void Ping() = 0;
  206. ///
  207. /// @brief Detach transaction.
  208. ///
  209. /// Stop any activities connected with it: pinging, aborting on crashes etc.
  210. /// Forget about the transaction totally.
  211. virtual void Detach();
  212. };
  213. ////////////////////////////////////////////////////////////////////////////////
  214. /// Interface containing non-transactional commands.
  215. class IClient
  216. : virtual public IClientBase
  217. {
  218. public:
  219. ///
  220. /// @brief Attach to existing master transaction.
  221. ///
  222. /// Returned object WILL NOT:
  223. /// - ping transaction automatically (unless @ref NYT::TAttachTransactionOptions::AutoPing is set)
  224. /// - abort it on program termination (unless @ref NYT::TAttachTransactionOptions::AbortOnTermination is set).
  225. /// Otherwise returned object is similar to the object returned by @ref NYT::IClientBase::StartTransaction.
  226. /// and it can see all the changes made inside the transaction.
  227. [[nodiscard]] virtual ITransactionPtr AttachTransaction(
  228. const TTransactionId& transactionId,
  229. const TAttachTransactionOptions& options = TAttachTransactionOptions()) = 0;
  230. ///
  231. /// @brief Mount dynamic table.
  232. ///
  233. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#mount_table)
  234. virtual void MountTable(
  235. const TYPath& path,
  236. const TMountTableOptions& options = TMountTableOptions()) = 0;
  237. ///
  238. /// @brief Unmount dynamic table.
  239. ///
  240. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unmount_table)
  241. virtual void UnmountTable(
  242. const TYPath& path,
  243. const TUnmountTableOptions& options = TUnmountTableOptions()) = 0;
  244. ///
  245. /// @brief Remount dynamic table.
  246. ///
  247. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#remount_table)
  248. virtual void RemountTable(
  249. const TYPath& path,
  250. const TRemountTableOptions& options = TRemountTableOptions()) = 0;
  251. ///
  252. /// @brief Switch dynamic table from `mounted' into `frozen' state.
  253. ///
  254. /// When table is in frozen state all its data is flushed to disk and writes are disabled.
  255. ///
  256. /// @note this function launches the process of switching, but doesn't wait until switching is accomplished.
  257. /// Waiting has to be performed by user.
  258. ///
  259. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#freeze_table)
  260. virtual void FreezeTable(
  261. const TYPath& path,
  262. const TFreezeTableOptions& options = TFreezeTableOptions()) = 0;
  263. ///
  264. /// @brief Switch dynamic table from `frozen` into `mounted` state.
  265. ///
  266. /// @note this function launches the process of switching, but doesn't wait until switching is accomplished.
  267. /// Waiting has to be performed by user.
  268. ///
  269. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#unfreeze_table)
  270. virtual void UnfreezeTable(
  271. const TYPath& path,
  272. const TUnfreezeTableOptions& options = TUnfreezeTableOptions()) = 0;
  273. ///
  274. /// @brief Reshard dynamic table (break it into tablets) by given pivot keys.
  275. ///
  276. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#reshard_table)
  277. virtual void ReshardTable(
  278. const TYPath& path,
  279. const TVector<TKey>& pivotKeys,
  280. const TReshardTableOptions& options = TReshardTableOptions()) = 0;
  281. ///
  282. /// @brief Reshard dynamic table, breaking it into given number of tablets.
  283. ///
  284. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#reshard_table)
  285. virtual void ReshardTable(
  286. const TYPath& path,
  287. i64 tabletCount,
  288. const TReshardTableOptions& options = TReshardTableOptions()) = 0;
  289. ///
  290. /// @brief Insert rows into dynamic table.
  291. ///
  292. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#insert_rows)
  293. virtual void InsertRows(
  294. const TYPath& path,
  295. const TNode::TListType& rows,
  296. const TInsertRowsOptions& options = TInsertRowsOptions()) = 0;
  297. ///
  298. /// @brief Delete rows from dynamic table.
  299. ///
  300. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#delete_rows)
  301. virtual void DeleteRows(
  302. const TYPath& path,
  303. const TNode::TListType& keys,
  304. const TDeleteRowsOptions& options = TDeleteRowsOptions()) = 0;
  305. ///
  306. /// @brief Trim rows from the beginning of ordered dynamic table.
  307. ///
  308. /// Asynchronously removes `rowCount` rows from the beginning of ordered dynamic table.
  309. /// Numeration of remaining rows *does not change*, e.g. after `trim(10)` and `trim(20)`
  310. /// you get in total `20` deleted rows.
  311. ///
  312. /// @param path Path to ordered dynamic table.
  313. /// @param tabletIndex Which tablet to trim.
  314. /// @param rowCount How many trimmed rows will be in the table after command.
  315. /// @param options Optional parameters.
  316. ///
  317. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#trim_rows)
  318. virtual void TrimRows(
  319. const TYPath& path,
  320. i64 tabletIndex,
  321. i64 rowCount,
  322. const TTrimRowsOptions& options = TTrimRowsOptions()) = 0;
  323. ///
  324. /// @brief Lookup rows with given keys from dynamic table.
  325. ///
  326. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#lookup-rows)
  327. virtual TNode::TListType LookupRows(
  328. const TYPath& path,
  329. const TNode::TListType& keys,
  330. const TLookupRowsOptions& options = TLookupRowsOptions()) = 0;
  331. ///
  332. /// @brief Select rows from dynamic table, using [SQL dialect](https://ytsaurus.tech/docs/en//description/dynamic_tables/dyn_query_language.html).
  333. ///
  334. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#select-rows)
  335. virtual TNode::TListType SelectRows(
  336. const TString& query,
  337. const TSelectRowsOptions& options = TSelectRowsOptions()) = 0;
  338. ///
  339. /// @brief Change properties of table replica.
  340. ///
  341. /// Allows to enable/disable replica and/or change its mode.
  342. ///
  343. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#alter_table_replica)
  344. virtual void AlterTableReplica(
  345. const TReplicaId& replicaId,
  346. const TAlterTableReplicaOptions& alterTableReplicaOptions) = 0;
  347. ///
  348. /// @brief Generate a monotonously increasing master timestamp.
  349. ///
  350. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#generate_timestamp)
  351. virtual ui64 GenerateTimestamp() = 0;
  352. /// Return YT username of current client.
  353. virtual TAuthorizationInfo WhoAmI() = 0;
  354. ///
  355. /// @brief Get operation attributes.
  356. ///
  357. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_operation)
  358. virtual TOperationAttributes GetOperation(
  359. const TOperationId& operationId,
  360. const TGetOperationOptions& options = TGetOperationOptions()) = 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 TString& alias,
  367. const TGetOperationOptions& options = TGetOperationOptions()) = 0;
  368. ///
  369. /// @brief List operations satisfying given filters.
  370. ///
  371. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#list_operations)
  372. virtual TListOperationsResult ListOperations(
  373. const TListOperationsOptions& options = TListOperationsOptions()) = 0;
  374. ///
  375. /// @brief Update operation runtime parameters.
  376. ///
  377. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#update_operation_parameters)
  378. virtual void UpdateOperationParameters(
  379. const TOperationId& operationId,
  380. const TUpdateOperationParametersOptions& options) = 0;
  381. ///
  382. /// @brief Get job attributes.
  383. ///
  384. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job)
  385. virtual TJobAttributes GetJob(
  386. const TOperationId& operationId,
  387. const TJobId& jobId,
  388. const TGetJobOptions& options = TGetJobOptions()) = 0;
  389. ///
  390. /// List attributes of jobs satisfying given filters.
  391. ///
  392. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#list_jobs)
  393. virtual TListJobsResult ListJobs(
  394. const TOperationId& operationId,
  395. const TListJobsOptions& options = TListJobsOptions()) = 0;
  396. ///
  397. /// @brief Get the input of a running or failed job.
  398. ///
  399. /// @ref NYT::TErrorResponse exception is thrown if job is missing.
  400. ///
  401. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_input)
  402. virtual IFileReaderPtr GetJobInput(
  403. const TJobId& jobId,
  404. const TGetJobInputOptions& options = TGetJobInputOptions()) = 0;
  405. ///
  406. /// @brief Get fail context of a failed job.
  407. ///
  408. /// @ref NYT::TErrorResponse exception is thrown if it is missing.
  409. ///
  410. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_fail_context)
  411. virtual IFileReaderPtr GetJobFailContext(
  412. const TOperationId& operationId,
  413. const TJobId& jobId,
  414. const TGetJobFailContextOptions& options = TGetJobFailContextOptions()) = 0;
  415. ///
  416. /// @brief Get stderr of a running or failed job.
  417. ///
  418. /// @ref NYT::TErrorResponse exception is thrown if it is missing.
  419. ///
  420. /// @note YT doesn't store all job stderrs
  421. ///
  422. /// @note If job stderr exceeds few megabytes YT will store only head and tail of stderr.
  423. ///
  424. /// @see Description of `max_stderr_size` spec option [here](https://ytsaurus.tech/docs/en/user-guide/data-processing/operations/operations-options).
  425. ///
  426. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#get_job_stderr)
  427. virtual IFileReaderPtr GetJobStderr(
  428. const TOperationId& operationId,
  429. const TJobId& jobId,
  430. const TGetJobStderrOptions& options = TGetJobStderrOptions()) = 0;
  431. ///
  432. /// @brief Create one or several rbtorrents for files in a blob table.
  433. ///
  434. /// If specified, one torrent is created for each value of `KeyColumns` option.
  435. /// Otherwise, a single torrent with all files of a table is created.
  436. ///
  437. /// @return list of nodes, each node has two fields
  438. /// * `key`: list of key columns values. Empty if `KeyColumns` is not specified.
  439. /// * `rbtorrent`: rbtorrent string (with `rbtorrent:` prefix)
  440. ///
  441. /// @see [More info.](https://docs.yandex-team.ru/docs/yt/description/storage/blobtables#sky_share)
  442. virtual TNode::TListType SkyShareTable(
  443. const std::vector<TYPath>& tablePaths,
  444. const TSkyShareTableOptions& options) = 0;
  445. ///
  446. /// @brief Check if `user` has `permission` to access a Cypress node at `path`.
  447. ///
  448. /// For tables access to columns specified in `options.Columns_` can be checked
  449. /// (@see [the doc](https://ytsaurus.tech/docs/en/user-guide/storage/columnar-acl)).
  450. ///
  451. /// If access is denied (the returned result has `.Action == ESecurityAction::Deny`)
  452. /// because of a `deny` rule, the "denying" object name and id
  453. /// and "denied" subject name an id may be returned.
  454. ///
  455. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#check_permission)
  456. virtual TCheckPermissionResponse CheckPermission(
  457. const TString& user,
  458. EPermission permission,
  459. const TYPath& path,
  460. const TCheckPermissionOptions& options = TCheckPermissionOptions()) = 0;
  461. /// @brief Get information about tablet
  462. /// @see NYT::TTabletInfo
  463. virtual TVector<TTabletInfo> GetTabletInfos(
  464. const TYPath& path,
  465. const TVector<int>& tabletIndexes,
  466. const TGetTabletInfosOptions& options = TGetTabletInfosOptions()) = 0;
  467. ///
  468. /// @brief Suspend operation.
  469. ///
  470. /// Jobs will be aborted.
  471. ///
  472. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#suspend_operation)
  473. virtual void SuspendOperation(
  474. const TOperationId& operationId,
  475. const TSuspendOperationOptions& options = TSuspendOperationOptions()) = 0;
  476. /// @brief Resume previously suspended operation.
  477. ///
  478. /// @see [YT doc](https://ytsaurus.tech/docs/en/api/commands.html#resume_operation)
  479. virtual void ResumeOperation(
  480. const TOperationId& operationId,
  481. const TResumeOperationOptions& options = TResumeOperationOptions()) = 0;
  482. ///
  483. /// @brief Synchronously terminates all client's background activities
  484. ///
  485. /// e.g. no callbacks will be executed after the function is completed
  486. ///
  487. /// @note It is safe to call Shutdown multiple times
  488. ///
  489. /// @note @ref NYT::TApiUsageError will be thrown if any client's method is called after shutdown
  490. ///
  491. virtual void Shutdown() = 0;
  492. };
  493. /// Create a client for particular MapReduce cluster.
  494. IClientPtr CreateClient(
  495. const TString& serverName,
  496. const TCreateClientOptions& options = TCreateClientOptions());
  497. /// Create a client for mapreduce cluster specified in `YT_PROXY` environment variable.
  498. IClientPtr CreateClientFromEnv(
  499. const TCreateClientOptions& options = TCreateClientOptions());
  500. ////////////////////////////////////////////////////////////////////////////////
  501. } // namespace NYT