blob: 12b2939ffaddf792aaab5ed5f73b8813f2e88274 (
plain) (
blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
|
#include "retryful_writer.h"
#include "retry_heavy_write_request.h"
#include <yt/cpp/mapreduce/http/requests.h>
#include <yt/cpp/mapreduce/interface/errors.h>
#include <yt/cpp/mapreduce/interface/finish_or_die.h>
#include <yt/cpp/mapreduce/interface/logging/yt_log.h>
#include <util/generic/size_literals.h>
namespace NYT {
////////////////////////////////////////////////////////////////////////////////
TRetryfulWriter::~TRetryfulWriter()
{
NDetail::FinishOrDie(this, "TRetryfulWriter");
}
void TRetryfulWriter::CheckWriterState()
{
switch (WriterState_) {
case Ok:
break;
case Completed:
ythrow TApiUsageError() << "Cannot use table writer that is finished";
case Error:
ythrow TApiUsageError() << "Cannot use table writer that finished with error";
}
}
void TRetryfulWriter::NotifyRowEnd()
{
CheckWriterState();
if (Buffer_.Size() >= BufferSize_) {
FlushBuffer(false);
}
}
void TRetryfulWriter::DoWrite(const void* buf, size_t len)
{
CheckWriterState();
while (Buffer_.Size() + len > Buffer_.Capacity()) {
Buffer_.Reserve(Buffer_.Capacity() * 2);
}
Buffer_.Append(static_cast<const char*>(buf), len);
}
void TRetryfulWriter::DoFinish()
{
if (WriterState_ != Ok) {
return;
}
FlushBuffer(true);
if (Started_) {
FilledBuffers_.Stop();
Thread_.Join();
}
if (Exception_) {
WriterState_ = Error;
std::rethrow_exception(Exception_);
}
if (WriteTransaction_) {
WriteTransaction_->Commit();
}
WriterState_ = Completed;
}
void TRetryfulWriter::FlushBuffer(bool lastBlock)
{
if (!Started_) {
if (lastBlock) {
try {
Send(Buffer_);
} catch (...) {
WriterState_ = Error;
throw;
}
return;
} else {
Started_ = true;
Thread_.Start();
}
}
auto emptyBuffer = EmptyBuffers_.Pop();
if (!emptyBuffer) {
WriterState_ = Error;
std::rethrow_exception(Exception_);
}
FilledBuffers_.Push(std::move(Buffer_));
Buffer_ = std::move(emptyBuffer.GetRef());
}
void TRetryfulWriter::Send(const TBuffer& buffer)
{
THttpHeader header("PUT", Command_);
header.SetInputFormat(Format_);
header.MergeParameters(Parameters_);
auto streamMaker = [&buffer] () {
return MakeHolder<TBufferInput>(buffer);
};
auto transactionId = (WriteTransaction_ ? WriteTransaction_->GetId() : ParentTransactionId_);
RetryHeavyWriteRequest(ClientRetryPolicy_, TransactionPinger_, Context_, transactionId, header, streamMaker);
Parameters_ = SecondaryParameters_; // all blocks except the first one are appended
}
void TRetryfulWriter::SendThread()
{
while (auto maybeBuffer = FilledBuffers_.Pop()) {
auto& buffer = maybeBuffer.GetRef();
try {
Send(buffer);
} catch (const std::exception&) {
Exception_ = std::current_exception();
EmptyBuffers_.Stop();
break;
}
buffer.Clear();
EmptyBuffers_.Push(std::move(buffer));
}
}
void* TRetryfulWriter::SendThread(void* opaque)
{
static_cast<TRetryfulWriter*>(opaque)->SendThread();
return nullptr;
}
void TRetryfulWriter::Abort()
{
if (Started_) {
FilledBuffers_.Stop();
Thread_.Join();
}
if (WriteTransaction_) {
WriteTransaction_->Abort();
}
WriterState_ = Completed;
}
size_t TRetryfulWriter::GetBufferSize(const TMaybe<TWriterOptions>& writerOptions)
{
auto retryBlockSize = TMaybe<size_t>();
if (writerOptions) {
if (writerOptions->RetryBlockSize_) {
retryBlockSize = *writerOptions->RetryBlockSize_;
} else if (writerOptions->DesiredChunkSize_) {
retryBlockSize = *writerOptions->DesiredChunkSize_;
}
}
return retryBlockSize.GetOrElse(64_MB);
}
////////////////////////////////////////////////////////////////////////////////
} // namespace NYT
|