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
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
|
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/ISource.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/ReadProgressCallback.h>
#include <Common/ThreadPool.h>
#include <Common/setThreadName.h>
#include <Common/scope_guard_safe.h>
#include <Common/CurrentThread.h>
#include <Poco/Event.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class PushingAsyncSource : public ISource
{
public:
explicit PushingAsyncSource(const Block & header)
: ISource(header)
{}
String getName() const override { return "PushingAsyncSource"; }
bool setData(Chunk chunk)
{
std::unique_lock lock(mutex);
condvar.wait(lock, [this] { return !has_data || is_finished; });
if (is_finished)
return false;
data.swap(chunk);
has_data = true;
condvar.notify_one();
return true;
}
void finish()
{
std::unique_lock lock(mutex);
is_finished = true;
condvar.notify_all();
}
protected:
Chunk generate() override
{
std::unique_lock lock(mutex);
condvar.wait(lock, [this] { return has_data || is_finished; });
Chunk res;
res.swap(data);
has_data = false;
condvar.notify_one();
return res;
}
private:
Chunk data;
bool has_data = false;
bool is_finished = false;
std::mutex mutex;
std::condition_variable condvar;
};
struct PushingAsyncPipelineExecutor::Data
{
PipelineExecutorPtr executor;
std::exception_ptr exception;
PushingAsyncSource * source = nullptr;
std::atomic_bool is_finished = false;
std::atomic_bool has_exception = false;
ThreadFromGlobalPool thread;
Poco::Event finish_event;
~Data()
{
if (thread.joinable())
thread.join();
}
void rethrowExceptionIfHas()
{
if (has_exception)
{
has_exception = false;
std::rethrow_exception(exception);
}
}
};
static void threadFunction(
PushingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads, bool concurrency_control)
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
setThreadName("QueryPushPipeEx");
try
{
if (thread_group)
CurrentThread::attachToGroup(thread_group);
data.executor->execute(num_threads, concurrency_control);
}
catch (...)
{
data.exception = std::current_exception();
data.has_exception = true;
/// Finish source in case of exception. Otherwise thread.join() may hung.
if (data.source)
data.source->finish();
}
data.is_finished = true;
data.finish_event.set();
}
PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
{
if (!pipeline.pushing())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PushingPipelineExecutor must be pushing");
pushing_source = std::make_shared<PushingAsyncSource>(pipeline.input->getHeader());
connect(pushing_source->getPort(), *pipeline.input);
pipeline.processors->emplace_back(pushing_source);
}
PushingAsyncPipelineExecutor::~PushingAsyncPipelineExecutor()
{
/// It must be finalized explicitly. Otherwise we cancel it assuming it's due to an exception.
chassert(finished || std::uncaught_exceptions() || std::current_exception());
try
{
cancel();
}
catch (...)
{
tryLogCurrentException("PushingAsyncPipelineExecutor");
}
}
const Block & PushingAsyncPipelineExecutor::getHeader() const
{
return pushing_source->getPort().getHeader();
}
void PushingAsyncPipelineExecutor::start()
{
if (started)
return;
started = true;
data = std::make_unique<Data>();
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
data->executor->setReadProgressCallback(pipeline.getReadProgressCallback());
data->source = pushing_source.get();
auto func = [&, thread_group = CurrentThread::getGroup()]()
{
threadFunction(*data, thread_group, pipeline.getNumThreads(), pipeline.getConcurrencyControl());
};
data->thread = ThreadFromGlobalPool(std::move(func));
}
void PushingAsyncPipelineExecutor::push(Chunk chunk)
{
if (!started)
start();
bool is_pushed = pushing_source->setData(std::move(chunk));
data->rethrowExceptionIfHas();
if (!is_pushed)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Pipeline for PushingAsyncPipelineExecutor was finished before all data was inserted");
}
void PushingAsyncPipelineExecutor::push(Block block)
{
push(Chunk(block.getColumns(), block.rows()));
}
void PushingAsyncPipelineExecutor::finish()
{
if (finished)
return;
finished = true;
pushing_source->finish();
/// Join thread here to wait for possible exception.
if (data && data->thread.joinable())
data->thread.join();
/// Rethrow exception to not swallow it in destructor.
if (data)
data->rethrowExceptionIfHas();
}
void PushingAsyncPipelineExecutor::cancel()
{
/// Cancel execution if it wasn't finished.
if (data && !data->is_finished && data->executor)
data->executor->cancel();
finish();
}
}
|