-
Notifications
You must be signed in to change notification settings - Fork 409
/
MultiplexInputStream.h
232 lines (199 loc) · 6.67 KB
/
MultiplexInputStream.h
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
227
228
229
230
231
232
// Copyright 2023 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <memory>
#include <queue>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
} // namespace ErrorCodes
class MultiPartitionStreamPool
{
public:
MultiPartitionStreamPool() = default;
void addPartitionStreams(const BlockInputStreams & cur_streams)
{
if (cur_streams.empty())
return;
std::unique_lock lk(mu);
streams_queue_by_partition.push_back(std::make_shared<std::queue<std::shared_ptr<IBlockInputStream>>>());
for (const auto & stream : cur_streams)
streams_queue_by_partition.back()->push(stream);
added_streams.insert(added_streams.end(), cur_streams.begin(), cur_streams.end());
}
std::shared_ptr<IBlockInputStream> pickOne()
{
std::unique_lock lk(mu);
if (streams_queue_by_partition.empty())
return nullptr;
if (streams_queue_id >= static_cast<int>(streams_queue_by_partition.size()))
streams_queue_id = 0;
auto & q = *streams_queue_by_partition[streams_queue_id];
std::shared_ptr<IBlockInputStream> ret = nullptr;
assert(!q.empty());
ret = q.front();
q.pop();
if (q.empty())
streams_queue_id = removeQueue(streams_queue_id);
else
streams_queue_id = nextQueueId(streams_queue_id);
return ret;
}
int exportAddedStreams(BlockInputStreams & ret_streams)
{
std::unique_lock lk(mu);
for (auto & stream : added_streams)
ret_streams.push_back(stream);
return added_streams.size();
}
int addedStreamsCnt()
{
std::unique_lock lk(mu);
return added_streams.size();
}
private:
int removeQueue(int queue_id)
{
streams_queue_by_partition[queue_id] = nullptr;
if (queue_id != static_cast<int>(streams_queue_by_partition.size()) - 1)
{
swap(streams_queue_by_partition[queue_id], streams_queue_by_partition.back());
streams_queue_by_partition.pop_back();
return queue_id;
}
else
{
streams_queue_by_partition.pop_back();
return 0;
}
}
int nextQueueId(int queue_id) const
{
if (queue_id + 1 < static_cast<int>(streams_queue_by_partition.size()))
return queue_id + 1;
else
return 0;
}
static void swap(
std::shared_ptr<std::queue<std::shared_ptr<IBlockInputStream>>> & a,
std::shared_ptr<std::queue<std::shared_ptr<IBlockInputStream>>> & b)
{
a.swap(b);
}
std::vector<std::shared_ptr<std::queue<std::shared_ptr<IBlockInputStream>>>> streams_queue_by_partition;
std::vector<std::shared_ptr<IBlockInputStream>> added_streams;
int streams_queue_id = 0;
std::mutex mu;
};
class MultiplexInputStream final : public IProfilingBlockInputStream
{
private:
static constexpr auto NAME = "Multiplex";
public:
MultiplexInputStream(std::shared_ptr<MultiPartitionStreamPool> & shared_pool, const String & req_id)
: log(Logger::get(req_id))
, shared_pool(shared_pool)
{
shared_pool->exportAddedStreams(children);
size_t num_children = children.size();
if (num_children > 1)
{
Block header = children.at(0)->getHeader();
for (size_t i = 1; i < num_children; ++i)
assertBlocksHaveEqualStructure(children[i]->getHeader(), header, "MULTIPLEX");
}
}
String getName() const override { return NAME; }
~MultiplexInputStream() override
{
try
{
if (!all_read)
cancel(false);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
}
/** Different from the default implementation by trying to stop all sources,
* skipping failed by execution.
*/
void cancel(bool kill) override
{
if (kill)
is_killed = true;
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
if (cur_stream)
{
if (auto * child = dynamic_cast<IProfilingBlockInputStream *>(&*cur_stream))
{
child->cancel(kill);
}
}
}
Block getHeader() const override { return children.at(0)->getHeader(); }
protected:
/// Do nothing, to make the preparation when underlying InputStream is picked from the pool
void readPrefix() override {}
/** The following options are possible:
* 1. `readImpl` function is called until it returns an empty block.
* Then `readSuffix` function is called and then destructor.
* 2. `readImpl` function is called. At some point, `cancel` function is called perhaps from another thread.
* Then `readSuffix` function is called and then destructor.
* 3. At any time, the object can be destroyed (destructor called).
*/
Block readImpl() override
{
if (all_read)
return {};
Block ret;
while (!cur_stream || !(ret = cur_stream->read()))
{
if (cur_stream)
cur_stream->readSuffix(); // release old inputstream
cur_stream = shared_pool->pickOne();
if (!cur_stream)
{ // shared_pool is empty
all_read = true;
return {};
}
cur_stream->readPrefix();
}
return ret;
}
/// Called either after everything is read, or after cancel.
void readSuffix() override
{
if (!all_read && !is_cancelled)
throw Exception("readSuffix called before all data is read", ErrorCodes::LOGICAL_ERROR);
if (cur_stream)
{
cur_stream->readSuffix();
cur_stream = nullptr;
}
}
private:
LoggerPtr log;
std::shared_ptr<MultiPartitionStreamPool> shared_pool;
std::shared_ptr<IBlockInputStream> cur_stream;
bool all_read = false;
};
} // namespace DB