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
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
|
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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.
#include "contrib/libs/apache/arrow_next/cpp/src/arrow/util/future.h"
#include <algorithm>
#include <atomic>
#include <chrono>
#include <condition_variable>
#include <mutex>
#include <numeric>
#include "contrib/libs/apache/arrow_next/cpp/src/arrow/util/checked_cast.h"
#include "contrib/libs/apache/arrow_next/src/arrow/util/config.h"
#include "contrib/libs/apache/arrow_next/cpp/src/arrow/util/logging.h"
#include "contrib/libs/apache/arrow_next/cpp/src/arrow/util/thread_pool.h"
#include "contrib/libs/apache/arrow_next/cpp/src/arrow/util/tracing_internal.h"
namespace arrow20 {
using internal::checked_cast;
class ConcreteFutureImpl : public FutureImpl {
public:
void DoMarkFinished() { DoMarkFinishedOrFailed(FutureState::SUCCESS); }
void DoMarkFailed() { DoMarkFinishedOrFailed(FutureState::FAILURE); }
void CheckOptions(const CallbackOptions& opts) {
if (opts.should_schedule != ShouldSchedule::Never) {
DCHECK_NE(opts.executor, nullptr)
<< "An executor must be specified when adding a callback that might schedule";
}
}
void AddCallback(Callback callback, CallbackOptions opts) {
CheckOptions(opts);
std::unique_lock<std::mutex> lock(mutex_);
#ifdef ARROW_WITH_OPENTELEMETRY
callback = [func = std::move(callback),
active_span = ::arrow20::internal::tracing::GetTracer()->GetCurrentSpan()](
const FutureImpl& impl) mutable {
auto scope = ::arrow20::internal::tracing::GetTracer()->WithActiveSpan(active_span);
std::move(func)(impl);
};
#endif
CallbackRecord callback_record{std::move(callback), opts};
if (IsFutureFinished(state_)) {
lock.unlock();
RunOrScheduleCallback(shared_from_this(), std::move(callback_record),
/*in_add_callback=*/true);
} else {
callbacks_.push_back(std::move(callback_record));
}
}
bool TryAddCallback(const std::function<Callback()>& callback_factory,
CallbackOptions opts) {
CheckOptions(opts);
std::unique_lock<std::mutex> lock(mutex_);
if (IsFutureFinished(state_)) {
return false;
} else {
callbacks_.push_back({callback_factory(), opts});
return true;
}
}
static bool ShouldScheduleCallback(const CallbackRecord& callback_record,
bool in_add_callback) {
switch (callback_record.options.should_schedule) {
case ShouldSchedule::Never:
return false;
case ShouldSchedule::Always:
return true;
case ShouldSchedule::IfUnfinished:
return !in_add_callback;
case ShouldSchedule::IfDifferentExecutor:
return !(callback_record.options.executor->IsCurrentExecutor());
default:
DCHECK(false) << "Unrecognized ShouldSchedule option";
return false;
}
}
static void RunOrScheduleCallback(const std::shared_ptr<FutureImpl>& self,
CallbackRecord&& callback_record,
bool in_add_callback) {
if (ShouldScheduleCallback(callback_record, in_add_callback)) {
// Need to keep `this` alive until the callback has a chance to be scheduled.
auto task = [self, callback = std::move(callback_record.callback)]() mutable {
return std::move(callback)(*self);
};
DCHECK_OK(callback_record.options.executor->Spawn(std::move(task)));
} else {
std::move(callback_record.callback)(*self);
}
}
void DoMarkFinishedOrFailed(FutureState state) {
std::vector<CallbackRecord> callbacks;
std::shared_ptr<FutureImpl> self;
{
std::unique_lock<std::mutex> lock(mutex_);
#ifdef ARROW_WITH_OPENTELEMETRY
if (this->span_) {
util::tracing::Span& span = *span_;
END_SPAN(span);
}
#endif
DCHECK(!IsFutureFinished(state_)) << "Future already marked finished";
if (!callbacks_.empty()) {
callbacks = std::move(callbacks_);
auto self_inner = shared_from_this();
self = std::move(self_inner);
}
state_ = state;
// We need to notify while holding the lock. This notify often triggers
// waiters to delete the future and it is not safe to delete a cv_ while
// it is performing a notify_all
cv_.notify_all();
}
if (callbacks.empty()) return;
// run callbacks, lock not needed since the future is finished by this
// point so nothing else can modify the callbacks list and it is safe
// to iterate.
//
// In fact, it is important not to hold the locks because the callback
// may be slow or do its own locking on other resources
for (auto& callback_record : callbacks) {
RunOrScheduleCallback(self, std::move(callback_record), /*in_add_callback=*/false);
}
}
void DoWait() {
#ifdef ARROW_ENABLE_THREADING
std::unique_lock<std::mutex> lock(mutex_);
cv_.wait(lock, [this] { return IsFutureFinished(state_); });
#else
auto last_processed_time = std::chrono::steady_clock::now();
while (true) {
if (IsFutureFinished(state_)) {
return;
}
if (arrow20::internal::SerialExecutor::RunTasksOnAllExecutors() == false) {
auto this_time = std::chrono::steady_clock::now();
if (this_time - last_processed_time < std::chrono::seconds(10)) {
ARROW_LOG(WARNING) << "Waiting for future, but no executors have had any tasks "
"pending for last 10 seconds";
last_processed_time = std::chrono::steady_clock::now();
}
}
}
#endif
}
bool DoWait(double seconds) {
#ifdef ARROW_ENABLE_THREADING
std::unique_lock<std::mutex> lock(mutex_);
cv_.wait_for(lock, std::chrono::duration<double>(seconds),
[this] { return IsFutureFinished(state_); });
return IsFutureFinished(state_);
#else
auto start = std::chrono::steady_clock::now();
auto fsec = std::chrono::duration<double>(seconds);
while (std::chrono::steady_clock::now() - start < fsec) {
// run one task then check time
if (IsFutureFinished(state_)) {
return true;
}
arrow20::internal::SerialExecutor::RunTasksOnAllExecutors();
}
return IsFutureFinished(state_);
#endif
}
std::mutex mutex_;
std::condition_variable cv_;
};
namespace {
ConcreteFutureImpl* GetConcreteFuture(FutureImpl* future) {
return checked_cast<ConcreteFutureImpl*>(future);
}
} // namespace
std::unique_ptr<FutureImpl> FutureImpl::Make() {
return std::make_unique<ConcreteFutureImpl>();
}
std::unique_ptr<FutureImpl> FutureImpl::MakeFinished(FutureState state) {
std::unique_ptr<ConcreteFutureImpl> ptr(new ConcreteFutureImpl());
ptr->state_ = state;
return ptr;
}
FutureImpl::FutureImpl() : state_(FutureState::PENDING) {}
void FutureImpl::Wait() { GetConcreteFuture(this)->DoWait(); }
bool FutureImpl::Wait(double seconds) { return GetConcreteFuture(this)->DoWait(seconds); }
void FutureImpl::MarkFinished() { GetConcreteFuture(this)->DoMarkFinished(); }
void FutureImpl::MarkFailed() { GetConcreteFuture(this)->DoMarkFailed(); }
void FutureImpl::AddCallback(Callback callback, CallbackOptions opts) {
GetConcreteFuture(this)->AddCallback(std::move(callback), opts);
}
bool FutureImpl::TryAddCallback(const std::function<Callback()>& callback_factory,
CallbackOptions opts) {
return GetConcreteFuture(this)->TryAddCallback(callback_factory, opts);
}
Future<> AllComplete(const std::vector<Future<>>& futures) {
struct State {
explicit State(int64_t n_futures) : mutex(), n_remaining(n_futures) {}
std::mutex mutex;
std::atomic<size_t> n_remaining;
};
if (futures.empty()) {
return Future<>::MakeFinished();
}
auto state = std::make_shared<State>(futures.size());
auto out = Future<>::Make();
for (const auto& future : futures) {
future.AddCallback([state, out](const Status& status) mutable {
if (!status.ok()) {
std::unique_lock<std::mutex> lock(state->mutex);
if (!out.is_finished()) {
out.MarkFinished(status);
}
return;
}
if (state->n_remaining.fetch_sub(1) != 1) return;
out.MarkFinished();
});
}
return out;
}
Future<> AllFinished(const std::vector<Future<>>& futures) {
return All(futures).Then([](const std::vector<Result<internal::Empty>>& results) {
for (const auto& res : results) {
if (!res.ok()) {
return res.status();
}
}
return Status::OK();
});
}
} // namespace arrow20
|