forked from apache/arrow
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathasync_generator.h
More file actions
2031 lines (1866 loc) · 74.8 KB
/
Copy pathasync_generator.h
File metadata and controls
2031 lines (1866 loc) · 74.8 KB
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
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
#pragma once
#include <atomic>
#include <cassert>
#include <cstring>
#include <deque>
#include <limits>
#include <optional>
#include <queue>
#include "arrow/util/async_generator_fwd.h"
#include "arrow/util/async_util.h"
#include "arrow/util/functional.h"
#include "arrow/util/future.h"
#include "arrow/util/io_util.h"
#include "arrow/util/iterator.h"
#include "arrow/util/mutex.h"
#include "arrow/util/queue.h"
#include "arrow/util/thread_pool.h"
namespace arrow {
// The methods in this file create, modify, and utilize AsyncGenerator which is an
// iterator of futures. This allows an asynchronous source (like file input) to be run
// through a pipeline in the same way that iterators can be used to create pipelined
// workflows.
//
// In order to support pipeline parallelism we introduce the concept of asynchronous
// reentrancy. This is different than synchronous reentrancy. With synchronous code a
// function is reentrant if the function can be called again while a previous call to that
// function is still running. Unless otherwise specified none of these generators are
// synchronously reentrant. Care should be taken to avoid calling them in such a way (and
// the utilities Visit/Collect/Await take care to do this).
//
// Asynchronous reentrancy on the other hand means the function is called again before the
// future returned by the function is marked finished (but after the call to get the
// future returns). Some of these generators are async-reentrant while others (e.g.
// those that depend on ordered processing like decompression) are not. Read the MakeXYZ
// function comments to determine which generators support async reentrancy.
//
// Note: Generators that are not asynchronously reentrant can still support readahead
// (\see MakeSerialReadaheadGenerator).
//
// Readahead operators, and some other operators, may introduce queueing. Any operators
// that introduce buffering should detail the amount of buffering they introduce in their
// MakeXYZ function comments.
//
// A generator should always be fully consumed before it is destroyed.
// A generator should not mark a future complete with an error status or a terminal value
// until all outstanding futures have completed. Generators that spawn multiple
// concurrent futures may need to hold onto an error while other concurrent futures wrap
// up.
template <typename T>
struct IterationTraits<AsyncGenerator<T>> {
/// \brief by default when iterating through a sequence of AsyncGenerator<T>,
/// an empty function indicates the end of iteration.
static AsyncGenerator<T> End() { return AsyncGenerator<T>(); }
static bool IsEnd(const AsyncGenerator<T>& val) { return !val; }
};
template <typename T>
Future<T> AsyncGeneratorEnd() {
return Future<T>::MakeFinished(IterationTraits<T>::End());
}
/// returning a future that completes when all have been visited
template <typename T, typename Visitor>
Future<> VisitAsyncGenerator(AsyncGenerator<T> generator, Visitor visitor) {
struct LoopBody {
struct Callback {
Result<ControlFlow<>> operator()(const T& next) {
if (IsIterationEnd(next)) {
return Break();
} else {
auto visited = visitor(next);
if (visited.ok()) {
return Continue();
} else {
return visited;
}
}
}
Visitor visitor;
};
Future<ControlFlow<>> operator()() {
Callback callback{visitor};
auto next = generator();
return next.Then(std::move(callback));
}
AsyncGenerator<T> generator;
Visitor visitor;
};
return Loop(LoopBody{std::move(generator), std::move(visitor)});
}
/// \brief Wait for an async generator to complete, discarding results.
template <typename T>
Future<> DiscardAllFromAsyncGenerator(AsyncGenerator<T> generator) {
std::function<Status(T)> visitor = [](const T&) { return Status::OK(); };
return VisitAsyncGenerator(generator, visitor);
}
/// \brief Collect the results of an async generator into a vector
template <typename T>
Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
auto vec = std::make_shared<std::vector<T>>();
auto loop_body = [generator = std::move(generator),
vec = std::move(vec)]() -> Future<ControlFlow<std::vector<T>>> {
auto next = generator();
return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
if (IsIterationEnd(result)) {
return Break(*vec);
} else {
vec->push_back(result);
return Continue();
}
});
};
return Loop(std::move(loop_body));
}
/// \see MakeMappedGenerator
template <typename T, typename V>
class MappingGenerator {
public:
MappingGenerator(AsyncGenerator<T> source, std::function<Future<V>(const T&)> map)
: state_(std::make_shared<State>(std::move(source), std::move(map))) {}
Future<V> operator()() {
auto future = Future<V>::Make();
bool should_trigger;
{
auto guard = state_->mutex.Lock();
if (state_->finished) {
return AsyncGeneratorEnd<V>();
}
should_trigger = state_->waiting_jobs.empty();
state_->waiting_jobs.push_back(future);
}
if (should_trigger) {
state_->source().AddCallback(Callback{state_});
}
return future;
}
private:
struct State {
State(AsyncGenerator<T> source, std::function<Future<V>(const T&)> map)
: source(std::move(source)),
map(std::move(map)),
waiting_jobs(),
mutex(),
finished(false) {}
void Purge() {
// This might be called by an original callback (if the source iterator fails or
// ends) or by a mapped callback (if the map function fails or ends prematurely).
// Either way it should only be called once and after finished is set so there is no
// need to guard access to `waiting_jobs`.
while (!waiting_jobs.empty()) {
waiting_jobs.front().MarkFinished(IterationTraits<V>::End());
waiting_jobs.pop_front();
}
}
AsyncGenerator<T> source;
std::function<Future<V>(const T&)> map;
std::deque<Future<V>> waiting_jobs;
util::Mutex mutex;
bool finished;
};
struct Callback;
struct MappedCallback {
void operator()(const Result<V>& maybe_next) {
bool end = !maybe_next.ok() || IsIterationEnd(*maybe_next);
bool should_purge = false;
if (end) {
{
auto guard = state->mutex.Lock();
should_purge = !state->finished;
state->finished = true;
}
}
sink.MarkFinished(maybe_next);
if (should_purge) {
state->Purge();
}
}
std::shared_ptr<State> state;
Future<V> sink;
};
struct Callback {
void operator()(const Result<T>& maybe_next) {
Future<V> sink;
bool end = !maybe_next.ok() || IsIterationEnd(*maybe_next);
bool should_purge = false;
bool should_trigger;
{
auto guard = state->mutex.Lock();
// A MappedCallback may have purged or be purging the queue;
// we shouldn't do anything here.
if (state->finished) return;
if (end) {
should_purge = !state->finished;
state->finished = true;
}
sink = state->waiting_jobs.front();
state->waiting_jobs.pop_front();
should_trigger = !end && !state->waiting_jobs.empty();
}
if (should_purge) {
state->Purge();
}
if (should_trigger) {
state->source().AddCallback(Callback{state});
}
if (maybe_next.ok()) {
const T& val = maybe_next.ValueUnsafe();
if (IsIterationEnd(val)) {
sink.MarkFinished(IterationTraits<V>::End());
} else {
Future<V> mapped_fut = state->map(val);
mapped_fut.AddCallback(MappedCallback{std::move(state), std::move(sink)});
}
} else {
sink.MarkFinished(maybe_next.status());
}
}
std::shared_ptr<State> state;
};
std::shared_ptr<State> state_;
};
/// \brief Create a generator that will apply the map function to each element of
/// source. The map function is not called on the end token.
///
/// Note: This function makes a copy of `map` for each item
/// Note: Errors returned from the `map` function will be propagated
///
/// If the source generator is async-reentrant then this generator will be also
template <typename T, typename MapFn,
typename Mapped = detail::result_of_t<MapFn(const T&)>,
typename V = typename EnsureFuture<Mapped>::type::ValueType>
AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFn map) {
auto map_callback = [map = std::move(map)](const T& val) mutable -> Future<V> {
return ToFuture(map(val));
};
return MappingGenerator<T, V>(std::move(source_generator), std::move(map_callback));
}
/// \brief Create a generator that will apply the map function to
/// each element of source. The map function is not called on the end
/// token. The result of the map function should be another
/// generator; all these generators will then be flattened to produce
/// a single stream of items.
///
/// Note: This function makes a copy of `map` for each item
/// Note: Errors returned from the `map` function will be propagated
///
/// If the source generator is async-reentrant then this generator will be also
template <typename T, typename MapFn,
typename Mapped = detail::result_of_t<MapFn(const T&)>,
typename V = typename EnsureFuture<Mapped>::type::ValueType>
AsyncGenerator<T> MakeFlatMappedGenerator(AsyncGenerator<T> source_generator, MapFn map) {
return MakeConcatenatedGenerator(
MakeMappedGenerator(std::move(source_generator), std::move(map)));
}
/// \see MakeSequencingGenerator
template <typename T, typename ComesAfter, typename IsNext>
class SequencingGenerator {
public:
SequencingGenerator(AsyncGenerator<T> source, ComesAfter compare, IsNext is_next,
T initial_value)
: state_(std::make_shared<State>(std::move(source), std::move(compare),
std::move(is_next), std::move(initial_value))) {}
Future<T> operator()() {
{
auto guard = state_->mutex.Lock();
// We can send a result immediately if the top of the queue is either an
// error or the next item
if (!state_->queue.empty() &&
(!state_->queue.top().ok() ||
state_->is_next(state_->previous_value, *state_->queue.top()))) {
auto result = std::move(state_->queue.top());
if (result.ok()) {
state_->previous_value = *result;
}
state_->queue.pop();
return Future<T>::MakeFinished(result);
}
if (state_->finished) {
return AsyncGeneratorEnd<T>();
}
// The next item is not in the queue so we will need to wait
auto new_waiting_fut = Future<T>::Make();
state_->waiting_future = new_waiting_fut;
guard.Unlock();
state_->source().AddCallback(Callback{state_});
return new_waiting_fut;
}
}
private:
struct WrappedComesAfter {
bool operator()(const Result<T>& left, const Result<T>& right) {
if (!left.ok() || !right.ok()) {
// Should never happen
return false;
}
return compare(*left, *right);
}
ComesAfter compare;
};
struct State {
State(AsyncGenerator<T> source, ComesAfter compare, IsNext is_next, T initial_value)
: source(std::move(source)),
is_next(std::move(is_next)),
previous_value(std::move(initial_value)),
waiting_future(),
queue(WrappedComesAfter{compare}),
finished(false),
mutex() {}
AsyncGenerator<T> source;
IsNext is_next;
T previous_value;
Future<T> waiting_future;
std::priority_queue<Result<T>, std::vector<Result<T>>, WrappedComesAfter> queue;
bool finished;
util::Mutex mutex;
};
class Callback {
public:
explicit Callback(std::shared_ptr<State> state) : state_(std::move(state)) {}
void operator()(const Result<T> result) {
Future<T> to_deliver;
bool finished;
{
auto guard = state_->mutex.Lock();
bool ready_to_deliver = false;
if (!result.ok()) {
// Clear any cached results
while (!state_->queue.empty()) {
state_->queue.pop();
}
ready_to_deliver = true;
state_->finished = true;
} else if (IsIterationEnd<T>(result.ValueUnsafe())) {
ready_to_deliver = state_->queue.empty();
state_->finished = true;
} else {
ready_to_deliver = state_->is_next(state_->previous_value, *result);
}
if (ready_to_deliver && state_->waiting_future.is_valid()) {
to_deliver = state_->waiting_future;
if (result.ok()) {
state_->previous_value = *result;
}
} else {
state_->queue.push(result);
}
// Capture state_->finished so we can access it outside the mutex
finished = state_->finished;
}
// Must deliver result outside of the mutex
if (to_deliver.is_valid()) {
to_deliver.MarkFinished(result);
} else {
// Otherwise, if we didn't get the next item (or a terminal item), we
// need to keep looking
if (!finished) {
state_->source().AddCallback(Callback{state_});
}
}
}
private:
const std::shared_ptr<State> state_;
};
const std::shared_ptr<State> state_;
};
/// \brief Buffer an AsyncGenerator to return values in sequence order ComesAfter
/// and IsNext determine the sequence order.
///
/// ComesAfter should be a BinaryPredicate that only returns true if a comes after b
///
/// IsNext should be a BinaryPredicate that returns true, given `a` and `b`, only if
/// `b` follows immediately after `a`. It should return true given `initial_value` and
/// `b` if `b` is the first item in the sequence.
///
/// This operator will queue unboundedly while waiting for the next item. It is intended
/// for jittery sources that might scatter an ordered sequence. It is NOT intended to
/// sort. Using it to try and sort could result in excessive RAM usage. This generator
/// will queue up to N blocks where N is the max "out of order"ness of the source.
///
/// For example, if the source is 1,6,2,5,4,3 it will queue 3 blocks because 3 is 3
/// blocks beyond where it belongs.
///
/// This generator is not async-reentrant but it consists only of a simple log(n)
/// insertion into a priority queue.
template <typename T, typename ComesAfter, typename IsNext>
AsyncGenerator<T> MakeSequencingGenerator(AsyncGenerator<T> source_generator,
ComesAfter compare, IsNext is_next,
T initial_value) {
return SequencingGenerator<T, ComesAfter, IsNext>(
std::move(source_generator), std::move(compare), std::move(is_next),
std::move(initial_value));
}
/// \see MakeTransformedGenerator
template <typename T, typename V>
class TransformingGenerator {
// The transforming generator state will be referenced as an async generator but will
// also be referenced via callback to various futures. If the async generator owner
// moves it around we need the state to be consistent for future callbacks.
struct TransformingGeneratorState
: std::enable_shared_from_this<TransformingGeneratorState> {
TransformingGeneratorState(AsyncGenerator<T> generator, Transformer<T, V> transformer)
: generator_(std::move(generator)),
transformer_(std::move(transformer)),
last_value_(),
finished_() {}
Future<V> operator()() {
while (true) {
auto maybe_next_result = Pump();
if (!maybe_next_result.ok()) {
return Future<V>::MakeFinished(maybe_next_result.status());
}
auto maybe_next = std::move(maybe_next_result).ValueUnsafe();
if (maybe_next.has_value()) {
return Future<V>::MakeFinished(*std::move(maybe_next));
}
auto next_fut = generator_();
// If finished already, process results immediately inside the loop to avoid
// stack overflow
if (next_fut.is_finished()) {
auto next_result = next_fut.result();
if (next_result.ok()) {
last_value_ = *next_result;
} else {
return Future<V>::MakeFinished(next_result.status());
}
// Otherwise, if not finished immediately, add callback to process results
} else {
auto self = this->shared_from_this();
return next_fut.Then([self](const T& next_result) {
self->last_value_ = next_result;
return (*self)();
});
}
}
}
// See comment on TransformingIterator::Pump
Result<std::optional<V>> Pump() {
if (!finished_ && last_value_.has_value()) {
ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
if (next.ReadyForNext()) {
if (IsIterationEnd(*last_value_)) {
finished_ = true;
}
last_value_.reset();
}
if (next.Finished()) {
finished_ = true;
}
if (next.HasValue()) {
return next.Value();
}
}
if (finished_) {
return IterationTraits<V>::End();
}
return std::nullopt;
}
AsyncGenerator<T> generator_;
Transformer<T, V> transformer_;
std::optional<T> last_value_;
bool finished_;
};
public:
explicit TransformingGenerator(AsyncGenerator<T> generator,
Transformer<T, V> transformer)
: state_(std::make_shared<TransformingGeneratorState>(std::move(generator),
std::move(transformer))) {}
Future<V> operator()() { return (*state_)(); }
protected:
std::shared_ptr<TransformingGeneratorState> state_;
};
/// \brief Transform an async generator using a transformer function returning a new
/// AsyncGenerator
///
/// The transform function here behaves exactly the same as the transform function in
/// MakeTransformedIterator and you can safely use the same transform function to
/// transform both synchronous and asynchronous streams.
///
/// This generator is not async-reentrant
///
/// This generator may queue up to 1 instance of T but will not delay
template <typename T, typename V>
AsyncGenerator<V> MakeTransformedGenerator(AsyncGenerator<T> generator,
Transformer<T, V> transformer) {
return TransformingGenerator<T, V>(generator, transformer);
}
/// \see MakeSerialReadaheadGenerator
template <typename T>
class SerialReadaheadGenerator {
public:
SerialReadaheadGenerator(AsyncGenerator<T> source_generator, int max_readahead)
: state_(std::make_shared<State>(std::move(source_generator), max_readahead)) {}
Future<T> operator()() {
if (state_->first_) {
// Lazy generator, need to wait for the first ask to prime the pump
state_->first_ = false;
auto next = state_->source_();
return next.Then(Callback{state_}, ErrCallback{state_});
}
// This generator is not async-reentrant. We won't be called until the last
// future finished so we know there is something in the queue
auto finished = state_->finished_.load();
if (finished && state_->readahead_queue_.IsEmpty()) {
return AsyncGeneratorEnd<T>();
}
std::shared_ptr<Future<T>> next;
if (!state_->readahead_queue_.Read(next)) {
return Status::UnknownError("Could not read from readahead_queue");
}
auto last_available = state_->spaces_available_.fetch_add(1);
if (last_available == 0 && !finished) {
// Reader idled out, we need to restart it
ARROW_RETURN_NOT_OK(state_->Pump(state_));
}
return *next;
}
private:
struct State {
State(AsyncGenerator<T> source, int max_readahead)
: first_(true),
source_(std::move(source)),
finished_(false),
// There is one extra "space" for the in-flight request
spaces_available_(max_readahead + 1),
// The SPSC queue has size-1 "usable" slots so we need to overallocate 1
readahead_queue_(max_readahead + 1) {}
Status Pump(const std::shared_ptr<State>& self) {
// Can't do readahead_queue.write(source().Then(...)) because then the
// callback might run immediately and add itself to the queue before this gets added
// to the queue messing up the order.
auto next_slot = std::make_shared<Future<T>>();
auto written = readahead_queue_.Write(next_slot);
if (!written) {
return Status::UnknownError("Could not write to readahead_queue");
}
// If this Pump is being called from a callback it is possible for the source to
// poll and read from the queue between the Write and this spot where we fill the
// value in. However, it is not possible for the future to read this value we are
// writing. That is because this callback (the callback for future X) must be
// finished before future X is marked complete and this source is not pulled
// reentrantly so it will not poll for future X+1 until this callback has completed.
*next_slot = source_().Then(Callback{self}, ErrCallback{self});
return Status::OK();
}
// Only accessed by the consumer end
bool first_;
// Accessed by both threads
AsyncGenerator<T> source_;
std::atomic<bool> finished_;
// The queue has a size but it is not atomic. We keep track of how many spaces are
// left in the queue here so we know if we've just written the last value and we need
// to stop reading ahead or if we've just read from a full queue and we need to
// restart reading ahead
std::atomic<uint32_t> spaces_available_;
// Needs to be a queue of shared_ptr and not Future because we set the value of the
// future after we add it to the queue
util::SpscQueue<std::shared_ptr<Future<T>>> readahead_queue_;
};
struct Callback {
Result<T> operator()(const T& next) {
if (IsIterationEnd(next)) {
state_->finished_.store(true);
return next;
}
auto last_available = state_->spaces_available_.fetch_sub(1);
if (last_available > 1) {
ARROW_RETURN_NOT_OK(state_->Pump(state_));
}
return next;
}
std::shared_ptr<State> state_;
};
struct ErrCallback {
Result<T> operator()(const Status& st) {
state_->finished_.store(true);
return st;
}
std::shared_ptr<State> state_;
};
std::shared_ptr<State> state_;
};
/// \see MakeFromFuture
template <typename T>
class FutureFirstGenerator {
public:
explicit FutureFirstGenerator(Future<AsyncGenerator<T>> future)
: state_(std::make_shared<State>(std::move(future))) {}
Future<T> operator()() {
if (state_->source_) {
return state_->source_();
} else {
auto state = state_;
return state_->future_.Then([state](const AsyncGenerator<T>& source) {
state->source_ = source;
return state->source_();
});
}
}
private:
struct State {
explicit State(Future<AsyncGenerator<T>> future) : future_(future), source_() {}
Future<AsyncGenerator<T>> future_;
AsyncGenerator<T> source_;
};
std::shared_ptr<State> state_;
};
/// \brief Transform a Future<AsyncGenerator<T>> into an AsyncGenerator<T>
/// that waits for the future to complete as part of the first item.
///
/// This generator is not async-reentrant (even if the generator yielded by future is)
///
/// This generator does not queue
template <typename T>
AsyncGenerator<T> MakeFromFuture(Future<AsyncGenerator<T>> future) {
return FutureFirstGenerator<T>(std::move(future));
}
/// \brief Create a generator that will pull from the source into a queue. Unlike
/// MakeReadaheadGenerator this will not pull reentrantly from the source.
///
/// The source generator does not need to be async-reentrant
///
/// This generator is not async-reentrant (even if the source is)
///
/// This generator may queue up to max_readahead additional instances of T
template <typename T>
AsyncGenerator<T> MakeSerialReadaheadGenerator(AsyncGenerator<T> source_generator,
int max_readahead) {
return SerialReadaheadGenerator<T>(std::move(source_generator), max_readahead);
}
/// \brief Create a generator that immediately pulls from the source
///
/// Typical generators do not pull from their source until they themselves
/// are pulled. This generator does not follow that convention and will call
/// generator() once before it returns. The returned generator will otherwise
/// mirror the source.
///
/// This generator forwards aysnc-reentrant pressure to the source
/// This generator buffers one item (the first result) until it is delivered.
template <typename T>
AsyncGenerator<T> MakeAutoStartingGenerator(AsyncGenerator<T> generator) {
struct AutostartGenerator {
Future<T> operator()() {
if (first_future->is_valid()) {
Future<T> result = *first_future;
*first_future = Future<T>();
return result;
}
return source();
}
std::shared_ptr<Future<T>> first_future;
AsyncGenerator<T> source;
};
std::shared_ptr<Future<T>> first_future = std::make_shared<Future<T>>(generator());
return AutostartGenerator{std::move(first_future), std::move(generator)};
}
/// \see MakeReadaheadGenerator
template <typename T>
class ReadaheadGenerator {
public:
ReadaheadGenerator(AsyncGenerator<T> source_generator, int max_readahead)
: state_(std::make_shared<State>(std::move(source_generator), max_readahead)) {}
Future<T> AddMarkFinishedContinuation(Future<T> fut) {
auto state = state_;
return fut.Then(
[state](const T& result) -> Future<T> {
state->MarkFinishedIfDone(result);
if (state->finished.load()) {
if (state->num_running.fetch_sub(1) == 1) {
state->final_future.MarkFinished();
}
} else {
state->num_running.fetch_sub(1);
}
return result;
},
[state](const Status& err) -> Future<T> {
// If there is an error we need to make sure all running
// tasks finish before we return the error.
state->finished.store(true);
if (state->num_running.fetch_sub(1) == 1) {
state->final_future.MarkFinished();
}
return state->final_future.Then([err]() -> Result<T> { return err; });
});
}
Future<T> operator()() {
if (state_->readahead_queue.empty()) {
// This is the first request, let's pump the underlying queue
state_->num_running.store(state_->max_readahead);
for (int i = 0; i < state_->max_readahead; i++) {
auto next = state_->source_generator();
auto next_after_check = AddMarkFinishedContinuation(std::move(next));
state_->readahead_queue.push(std::move(next_after_check));
}
}
// Pop one and add one
auto result = state_->readahead_queue.front();
state_->readahead_queue.pop();
if (state_->finished.load()) {
state_->readahead_queue.push(AsyncGeneratorEnd<T>());
} else {
state_->num_running.fetch_add(1);
auto back_of_queue = state_->source_generator();
auto back_of_queue_after_check =
AddMarkFinishedContinuation(std::move(back_of_queue));
state_->readahead_queue.push(std::move(back_of_queue_after_check));
}
return result;
}
private:
struct State {
State(AsyncGenerator<T> source_generator, int max_readahead)
: source_generator(std::move(source_generator)), max_readahead(max_readahead) {}
void MarkFinishedIfDone(const T& next_result) {
if (IsIterationEnd(next_result)) {
finished.store(true);
}
}
AsyncGenerator<T> source_generator;
int max_readahead;
Future<> final_future = Future<>::Make();
std::atomic<int> num_running{0};
std::atomic<bool> finished{false};
std::queue<Future<T>> readahead_queue;
};
std::shared_ptr<State> state_;
};
/// \brief A generator where the producer pushes items on a queue.
///
/// No back-pressure is applied, so this generator is mostly useful when
/// producing the values is neither CPU- nor memory-expensive (e.g. fetching
/// filesystem metadata).
///
/// This generator is not async-reentrant.
template <typename T>
class PushGenerator {
struct State {
State() {}
util::Mutex mutex;
std::deque<Result<T>> result_q;
std::optional<Future<T>> consumer_fut;
bool finished = false;
};
public:
/// Producer API for PushGenerator
class Producer {
public:
explicit Producer(const std::shared_ptr<State>& state) : weak_state_(state) {}
/// \brief Push a value on the queue
///
/// True is returned if the value was pushed, false if the generator is
/// already closed or destroyed. If the latter, it is recommended to stop
/// producing any further values.
bool Push(Result<T> result) {
auto state = weak_state_.lock();
if (!state) {
// Generator was destroyed
return false;
}
auto lock = state->mutex.Lock();
if (state->finished) {
// Closed early
return false;
}
if (state->consumer_fut.has_value()) {
auto fut = std::move(state->consumer_fut.value());
state->consumer_fut.reset();
lock.Unlock(); // unlock before potentially invoking a callback
fut.MarkFinished(std::move(result));
} else {
state->result_q.push_back(std::move(result));
}
return true;
}
/// \brief Tell the consumer we have finished producing
///
/// It is allowed to call this and later call Push() again ("early close").
/// In this case, calls to Push() after the queue is closed are silently
/// ignored. This can help implementing non-trivial cancellation cases.
///
/// True is returned on success, false if the generator is already closed
/// or destroyed.
bool Close() {
auto state = weak_state_.lock();
if (!state) {
// Generator was destroyed
return false;
}
auto lock = state->mutex.Lock();
if (state->finished) {
// Already closed
return false;
}
state->finished = true;
if (state->consumer_fut.has_value()) {
auto fut = std::move(state->consumer_fut.value());
state->consumer_fut.reset();
lock.Unlock(); // unlock before potentially invoking a callback
fut.MarkFinished(IterationTraits<T>::End());
}
return true;
}
/// Return whether the generator was closed or destroyed.
bool is_closed() const {
auto state = weak_state_.lock();
if (!state) {
// Generator was destroyed
return true;
}
auto lock = state->mutex.Lock();
return state->finished;
}
private:
const std::weak_ptr<State> weak_state_;
};
PushGenerator() : state_(std::make_shared<State>()) {}
/// Read an item from the queue
Future<T> operator()() const {
auto lock = state_->mutex.Lock();
assert(!state_->consumer_fut.has_value()); // Non-reentrant
if (!state_->result_q.empty()) {
auto fut = Future<T>::MakeFinished(std::move(state_->result_q.front()));
state_->result_q.pop_front();
return fut;
}
if (state_->finished) {
return AsyncGeneratorEnd<T>();
}
auto fut = Future<T>::Make();
state_->consumer_fut = fut;
return fut;
}
/// \brief Return producer-side interface
///
/// The returned object must be used by the producer to push values on the queue.
/// Only a single Producer object should be instantiated.
Producer producer() { return Producer{state_}; }
private:
const std::shared_ptr<State> state_;
};
/// \brief Create a generator that pulls reentrantly from a source
/// This generator will pull reentrantly from a source, ensuring that max_readahead
/// requests are active at any given time.
///
/// The source generator must be async-reentrant
///
/// This generator itself is async-reentrant.
///
/// This generator may queue up to max_readahead instances of T
template <typename T>
AsyncGenerator<T> MakeReadaheadGenerator(AsyncGenerator<T> source_generator,
int max_readahead) {
return ReadaheadGenerator<T>(std::move(source_generator), max_readahead);
}
/// \brief Creates a generator that will yield finished futures from a vector
///
/// This generator is async-reentrant
template <typename T>
AsyncGenerator<T> MakeVectorGenerator(std::vector<T> vec) {
struct State {
explicit State(std::vector<T> vec_) : vec(std::move(vec_)), vec_idx(0) {}
std::vector<T> vec;
std::atomic<std::size_t> vec_idx;
};
auto state = std::make_shared<State>(std::move(vec));
return [state]() {
auto idx = state->vec_idx.fetch_add(1);
if (idx >= state->vec.size()) {
// Eagerly return memory
state->vec.clear();
return AsyncGeneratorEnd<T>();
}
return Future<T>::MakeFinished(state->vec[idx]);
};
}
/// \see MakeMergedGenerator
template <typename T>
class MergedGenerator {
// Note, the implementation of this class is quite complex at the moment (PRs to
// simplify are always welcome)
//
// Terminology is borrowed from rxjs. This is a pull based implementation of the
// mergeAll operator. The "outer subscription" refers to the async
// generator that the caller provided when creating this. The outer subscription
// yields generators.
//
// Each of these generators is then subscribed to (up to max_subscriptions) and these
// are referred to as "inner subscriptions".
//
// As soon as we start we try and establish `max_subscriptions` inner subscriptions. For
// each inner subscription we will cache up to 1 value. This means we may have more
// values than we have been asked for. In our example, if a caller asks for one record
// batch we will start scanning `max_subscriptions` different files. For each file we
// will only queue up to 1 batch (so a separate readahead is needed on the file if batch