retryful_writer_v2.cpp 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410
  1. #include "retryful_writer_v2.h"
  2. #include <util/generic/scope.h>
  3. #include <yt/cpp/mapreduce/client/retry_heavy_write_request.h>
  4. #include <yt/cpp/mapreduce/client/transaction.h>
  5. #include <yt/cpp/mapreduce/client/transaction_pinger.h>
  6. #include <yt/cpp/mapreduce/common/fwd.h>
  7. #include <yt/cpp/mapreduce/common/helpers.h>
  8. #include <yt/cpp/mapreduce/common/retry_lib.h>
  9. #include <yt/cpp/mapreduce/common/retry_request.h>
  10. #include <yt/cpp/mapreduce/common/wait_proxy.h>
  11. #include <yt/cpp/mapreduce/http/context.h>
  12. #include <yt/cpp/mapreduce/http/helpers.h>
  13. #include <yt/cpp/mapreduce/http/http.h>
  14. #include <yt/cpp/mapreduce/interface/raw_client.h>
  15. #include <util/system/condvar.h>
  16. #include <queue>
  17. namespace NYT::NPrivate {
  18. ////////////////////////////////////////////////////////////////////////////////
  19. class TRetryfulWriterV2::TSentBuffer
  20. {
  21. public:
  22. TSentBuffer() = default;
  23. TSentBuffer(const TSentBuffer& ) = delete;
  24. std::pair<std::shared_ptr<std::string>, ssize_t> Snapshot() const
  25. {
  26. return {Buffer_, Size_};
  27. }
  28. void Clear()
  29. {
  30. // This method can be called only if no other object is holding snapshot.
  31. Y_ABORT_IF(Buffer_.use_count() != 1);
  32. Size_ = 0;
  33. }
  34. ssize_t Size() const
  35. {
  36. return Size_;
  37. }
  38. void Append(const void* data, ssize_t size)
  39. {
  40. auto newSize = Size_ + size;
  41. if (newSize < Capacity_) {
  42. memcpy(Buffer_->data() + Size_, data, size);
  43. } else {
  44. // Closest power of 2 exceeding new size
  45. auto newCapacity = 1 << (MostSignificantBit(newSize) + 1);
  46. newCapacity = Max<ssize_t>(64, newCapacity);
  47. auto newBuffer = std::make_shared<std::string>();
  48. newBuffer->resize(newCapacity);
  49. memcpy(newBuffer->data(), Buffer_->data(), Size_);
  50. memcpy(newBuffer->data() + Size_, data, size);
  51. Buffer_ = newBuffer;
  52. Capacity_ = newCapacity;
  53. }
  54. Size_ = newSize;
  55. }
  56. private:
  57. std::shared_ptr<std::string> Buffer_ = std::make_shared<std::string>();
  58. ssize_t Size_ = 0;
  59. ssize_t Capacity_ = 0;
  60. };
  61. ////////////////////////////////////////////////////////////////////////////////
  62. class TRetryfulWriterV2::TSender
  63. {
  64. public:
  65. TSender(TRichYPath path, THeavyRequestRetrier::TParameters parameters)
  66. : SenderThread_(
  67. [this, path=std::move(path), parameters=std::move(parameters)] {
  68. ThreadMain(std::move(path), parameters);
  69. })
  70. {
  71. SenderThread_.SetCurrentThreadName("retryful-writer-v2-sender");
  72. SenderThread_.Start();
  73. }
  74. ~TSender()
  75. {
  76. Abort();
  77. SenderThread_.Join();
  78. }
  79. bool IsRunning() const
  80. {
  81. auto g = Guard(Lock_);
  82. return State_.load() == EState::Running && !Error_;
  83. }
  84. void Abort()
  85. {
  86. auto g = Guard(Lock_);
  87. SetFinishedState(EState::Aborted, g);
  88. }
  89. void Finish()
  90. {
  91. {
  92. auto g = Guard(Lock_);
  93. SetFinishedState(EState::Completed, g);
  94. }
  95. SenderThread_.Join();
  96. CheckNoError();
  97. Y_ABORT_UNLESS(TaskIdQueue_.empty());
  98. Y_ABORT_UNLESS(TaskMap_.empty());
  99. }
  100. // Return future that is complete once upload of this buffer is successfully complete
  101. std::pair<NThreading::TFuture<void>, int> StartBlock()
  102. {
  103. auto g = Guard(Lock_);
  104. CheckNoError();
  105. auto taskId = NextTaskId_++;
  106. const auto& [it, inserted] = TaskMap_.emplace(taskId, TWriteTask{});
  107. Y_ABORT_IF(!inserted);
  108. TaskIdQueue_.push(taskId);
  109. HaveMoreData_.Signal();
  110. it->second.SendingComplete = NThreading::NewPromise();
  111. return {it->second.SendingComplete, taskId};
  112. }
  113. void UpdateBlock(int taskId, const TSentBuffer& buffer, bool complete)
  114. {
  115. auto snapshot = buffer.Snapshot();
  116. {
  117. auto g = Guard(Lock_);
  118. CheckNoError();
  119. auto it = TaskMap_.find(taskId);
  120. Y_ABORT_IF(it == TaskMap_.end());
  121. auto& writeTask = it->second;
  122. writeTask.Data = std::move(snapshot.first);
  123. writeTask.Size = snapshot.second;
  124. writeTask.BufferComplete = complete;
  125. if (TaskIdQueue_.empty() || TaskIdQueue_.back() != taskId) {
  126. TaskIdQueue_.push(taskId);
  127. }
  128. HaveMoreData_.Signal();
  129. }
  130. }
  131. private:
  132. enum class EState;
  133. private:
  134. void CheckNoError()
  135. {
  136. if (Error_) {
  137. std::rethrow_exception(Error_);
  138. }
  139. }
  140. void SetFinishedState(EState state, TGuard<TMutex>&)
  141. {
  142. if (State_ == EState::Running) {
  143. State_ = state;
  144. }
  145. HaveMoreData_.Signal();
  146. }
  147. void ThreadMain(TRichYPath path, const THeavyRequestRetrier::TParameters& parameters)
  148. {
  149. std::unique_ptr<THeavyRequestRetrier> retrier;
  150. auto firstRequestParameters = parameters;
  151. auto restRequestParameters = parameters;
  152. {
  153. TNode firstPath = PathToNode(path);
  154. firstRequestParameters.Header.MergeParameters(TNode()("path", firstPath), /*overwrite*/ true);
  155. TNode restPath = PathToNode(TRichYPath(path.Path_).Append(true));
  156. restRequestParameters.Header.MergeParameters(TNode()("path", restPath), /*overwrite*/ true);
  157. }
  158. const auto* currentParameters = &firstRequestParameters;
  159. while (true) {
  160. int taskId = 0;
  161. TWriteTask task;
  162. {
  163. auto g = Guard(Lock_);
  164. while (State_ == EState::Running && TaskIdQueue_.empty()) {
  165. HaveMoreData_.Wait(Lock_);
  166. }
  167. if (
  168. State_ == EState::Aborted ||
  169. State_ == EState::Completed && TaskIdQueue_.empty()
  170. ) {
  171. break;
  172. }
  173. taskId = TaskIdQueue_.front();
  174. TaskIdQueue_.pop();
  175. if (auto it = TaskMap_.find(taskId); it != TaskMap_.end()) {
  176. task = it->second;
  177. } else {
  178. Y_ABORT();
  179. }
  180. }
  181. try {
  182. if (!retrier) {
  183. retrier = std::make_unique<THeavyRequestRetrier>(*currentParameters);
  184. }
  185. retrier->Update([task=task] {
  186. return std::make_unique<TMemoryInput>(task.Data->data(), task.Size);
  187. });
  188. if (task.BufferComplete) {
  189. retrier->Finish();
  190. retrier.reset();
  191. }
  192. } catch (const std::exception& ex) {
  193. task.SendingComplete.SetException(std::current_exception());
  194. auto g = Guard(Lock_);
  195. Error_ = std::current_exception();
  196. return;
  197. }
  198. if (task.BufferComplete) {
  199. retrier.reset();
  200. task.SendingComplete.SetValue();
  201. currentParameters = &restRequestParameters;
  202. auto g = Guard(Lock_);
  203. auto erased = TaskMap_.erase(taskId);
  204. Y_ABORT_UNLESS(erased == 1);
  205. }
  206. }
  207. if (State_ == EState::Completed) {
  208. auto g = Guard(Lock_);
  209. Y_ABORT_UNLESS(TaskIdQueue_.empty());
  210. Y_ABORT_UNLESS(TaskMap_.empty());
  211. }
  212. }
  213. private:
  214. struct TWriteTask
  215. {
  216. NThreading::TPromise<void> SendingComplete;
  217. std::shared_ptr<std::string> Data = std::make_shared<std::string>();
  218. ssize_t Size = 0;
  219. bool BufferComplete = false;
  220. };
  221. TMutex Lock_;
  222. TCondVar HaveMoreData_;
  223. TThread SenderThread_;
  224. THashMap<int, TWriteTask> TaskMap_;
  225. std::queue<int> TaskIdQueue_;
  226. std::exception_ptr Error_;
  227. enum class EState {
  228. Running,
  229. Completed,
  230. Aborted,
  231. };
  232. std::atomic<EState> State_ = EState::Running;
  233. int NextTaskId_ = 0;
  234. };
  235. ////////////////////////////////////////////////////////////////////////////////
  236. struct TRetryfulWriterV2::TSendTask
  237. {
  238. TSentBuffer Buffer;
  239. NThreading::TFuture<void> SentFuture = NThreading::MakeFuture();
  240. int TaskId = 0;
  241. };
  242. ////////////////////////////////////////////////////////////////////////////////
  243. TRetryfulWriterV2::TRetryfulWriterV2(
  244. const IRawClientPtr& rawClient,
  245. IClientRetryPolicyPtr clientRetryPolicy,
  246. ITransactionPingerPtr transactionPinger,
  247. const TClientContext& context,
  248. const TTransactionId& parentId,
  249. const TString& command,
  250. const TMaybe<TFormat>& format,
  251. const TRichYPath& path,
  252. const TNode& serializedWriterOptions,
  253. ssize_t bufferSize,
  254. bool createTransaction)
  255. : BufferSize_(bufferSize)
  256. , Current_(std::make_unique<TSendTask>())
  257. , Previous_(std::make_unique<TSendTask>())
  258. {
  259. THttpHeader httpHeader("PUT", command);
  260. httpHeader.SetInputFormat(format);
  261. httpHeader.MergeParameters(serializedWriterOptions);
  262. if (createTransaction) {
  263. WriteTransaction_ = std::make_unique<TPingableTransaction>(
  264. rawClient,
  265. clientRetryPolicy,
  266. context,
  267. parentId,
  268. transactionPinger->GetChildTxPinger(),
  269. TStartTransactionOptions()
  270. );
  271. auto append = path.Append_.GetOrElse(false);
  272. auto lockMode = (append ? LM_SHARED : LM_EXCLUSIVE);
  273. NDetail::RequestWithRetry<void>(
  274. clientRetryPolicy->CreatePolicyForGenericRequest(),
  275. [this, &rawClient, &path, &lockMode] (TMutationId& mutationId) {
  276. rawClient->Lock(mutationId, WriteTransaction_->GetId(), path.Path_, lockMode);
  277. });
  278. }
  279. THeavyRequestRetrier::TParameters parameters = {
  280. .RawClientPtr = rawClient,
  281. .ClientRetryPolicy = clientRetryPolicy,
  282. .TransactionPinger = transactionPinger,
  283. .Context = context,
  284. .TransactionId = WriteTransaction_ ? WriteTransaction_->GetId() : parentId,
  285. .Header = std::move(httpHeader),
  286. };
  287. Sender_ = std::make_unique<TSender>(path, parameters);
  288. DoStartBatch();
  289. }
  290. void TRetryfulWriterV2::Abort()
  291. {
  292. auto sender = std::move(Sender_);
  293. auto writeTransaction = std::move(WriteTransaction_);
  294. if (sender) {
  295. sender->Abort();
  296. if (writeTransaction) {
  297. writeTransaction->Abort();
  298. }
  299. }
  300. }
  301. size_t TRetryfulWriterV2::GetBufferMemoryUsage() const
  302. {
  303. return BufferSize_ * 4;
  304. }
  305. void TRetryfulWriterV2::DoFinish()
  306. {
  307. auto sender = std::move(Sender_);
  308. auto writeTransaction = std::move(WriteTransaction_);
  309. if (sender && sender->IsRunning()) {
  310. sender->UpdateBlock(Current_->TaskId, Current_->Buffer, true);
  311. sender->Finish();
  312. if (writeTransaction) {
  313. writeTransaction->Commit();
  314. }
  315. }
  316. }
  317. void TRetryfulWriterV2::DoStartBatch()
  318. {
  319. Previous_->SentFuture.Wait();
  320. std::swap(Previous_, Current_);
  321. auto&& [future, taskId] = Sender_->StartBlock();
  322. Current_->SentFuture = future;
  323. Current_->TaskId = taskId;
  324. Current_->Buffer.Clear();
  325. NextSizeToSend_ = SendStep_;
  326. }
  327. void TRetryfulWriterV2::DoWrite(const void* buf, size_t len)
  328. {
  329. Current_->Buffer.Append(buf, len);
  330. auto currentSize = Current_->Buffer.Size();
  331. if (currentSize >= NextSizeToSend_) {
  332. Sender_->UpdateBlock(Current_->TaskId, Current_->Buffer, false);
  333. NextSizeToSend_ = currentSize + SendStep_;
  334. }
  335. }
  336. void TRetryfulWriterV2::NotifyRowEnd()
  337. {
  338. if (Current_->Buffer.Size() >= BufferSize_) {
  339. Sender_->UpdateBlock(Current_->TaskId, Current_->Buffer, true);
  340. DoStartBatch();
  341. }
  342. }
  343. ////////////////////////////////////////////////////////////////////////////////
  344. } // namespace NYT::NPrivate