Skip to content

Commit c783db6

Browse files
westonpacebkietz
authored andcommitted
ARROW-12004: [C++] Result<detail::Empty> is annoying
Per the JIRA `Future<>::AddCallback` callbacks receive a `Status`. `Future<T>::AddCallback` callbacks receive a `Result<T>` `Future<>::Then` callbacks receive nothing `Future<T>::Then` callbacks receive `const T&` To achieve this I had to explicitly specialize the empty `Future` but I introduced `FutureBase` to reduce the amount of duplicated code. `detail::Empty` is still around (although it got renamed to `internal::Empty` as a side effect of moving into `functional.h`). It could even be removed if one wanted to create a specialized `FutureImpl` but that doesn't seem to be needed at the moment. Closes apache#10205 from westonpace/feature/ARROW-12004--c-result-detail-empty-is-annoying Authored-by: Weston Pace <weston.pace@gmail.com> Signed-off-by: Benjamin Kietzman <bengilgit@gmail.com>
1 parent e4952e4 commit c783db6

19 files changed

Lines changed: 405 additions & 219 deletions

cpp/src/arrow/compute/exec/plan_test.cc

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -274,9 +274,7 @@ static Result<RecordBatchGenerator> MakeSlowRecordBatchGenerator(
274274
std::move(gen), [](const std::shared_ptr<RecordBatch>& batch) {
275275
auto fut = Future<std::shared_ptr<RecordBatch>>::Make();
276276
SleepABitAsync().AddCallback(
277-
[fut, batch](const Result<::arrow::detail::Empty>&) mutable {
278-
fut.MarkFinished(batch);
279-
});
277+
[fut, batch](const Status& status) mutable { fut.MarkFinished(batch); });
280278
return fut;
281279
});
282280
// Adding readahead implicitly adds parallelism by pulling reentrantly from

cpp/src/arrow/csv/reader.cc

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -759,8 +759,7 @@ class SerialStreamingReader : public BaseStreamingReader,
759759
}
760760
return Status::OK();
761761
})
762-
.Then([self](const ::arrow::detail::Empty& st)
763-
-> Result<std::shared_ptr<RecordBatch>> {
762+
.Then([self]() -> Result<std::shared_ptr<RecordBatch>> {
764763
return self->DecodeBatchAndUpdateSchema();
765764
});
766765
}
@@ -788,14 +787,14 @@ class SerialStreamingReader : public BaseStreamingReader,
788787
}
789788
auto self = shared_from_this();
790789
if (!block_generator_) {
791-
return SetupReader(self).Then([self](const Result<::arrow::detail::Empty>& res)
792-
-> Future<std::shared_ptr<RecordBatch>> {
793-
if (!res.ok()) {
794-
self->eof_ = true;
795-
return res.status();
796-
}
797-
return self->ReadNextSkippingEmpty(self);
798-
});
790+
return SetupReader(self).Then(
791+
[self]() -> Future<std::shared_ptr<RecordBatch>> {
792+
return self->ReadNextSkippingEmpty(self);
793+
},
794+
[self](const Status& err) -> Result<std::shared_ptr<RecordBatch>> {
795+
self->eof_ = true;
796+
return err;
797+
});
799798
} else {
800799
return self->ReadNextSkippingEmpty(self);
801800
}
@@ -925,7 +924,7 @@ class AsyncThreadedTableReader
925924
internal::TaskGroup::MakeThreaded(cpu_executor_, io_context_.stop_token());
926925

927926
auto self = shared_from_this();
928-
return ProcessFirstBuffer().Then([self](std::shared_ptr<Buffer> first_buffer) {
927+
return ProcessFirstBuffer().Then([self](const std::shared_ptr<Buffer>& first_buffer) {
929928
auto block_generator = ThreadedBlockReader::MakeAsyncIterator(
930929
self->buffer_generator_, MakeChunker(self->parse_options_),
931930
std::move(first_buffer));
@@ -950,12 +949,12 @@ class AsyncThreadedTableReader
950949
};
951950

952951
return VisitAsyncGenerator(std::move(block_generator), block_visitor)
953-
.Then([self](...) -> Future<> {
952+
.Then([self]() -> Future<> {
954953
// By this point we've added all top level tasks so it is safe to call
955954
// FinishAsync
956955
return self->task_group_->FinishAsync();
957956
})
958-
.Then([self](...) -> Result<std::shared_ptr<Table>> {
957+
.Then([self]() -> Result<std::shared_ptr<Table>> {
959958
// Finish conversion, create schema and table
960959
return self->MakeTable();
961960
});

cpp/src/arrow/dataset/file_base.cc

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -508,11 +508,11 @@ Status WriteInternal(const ScanOptions& scan_options, WriteState& state,
508508
[&](std::shared_ptr<RecordBatch> batch) {
509509
return WriteNextBatch(state, scan_task->fragment(), std::move(batch));
510510
};
511-
return internal::SerialExecutor::RunInSerialExecutor<detail::Empty>(
512-
[&](internal::Executor* executor) {
513-
return scan_task->SafeVisit(executor, visitor);
514-
})
515-
.status();
511+
return internal::RunSynchronously<Future<>>(
512+
[&](internal::Executor* executor) {
513+
return scan_task->SafeVisit(executor, visitor);
514+
},
515+
/*use_threads=*/false);
516516
});
517517
}
518518
return task_group->Finish();

cpp/src/arrow/dataset/scanner.cc

Lines changed: 7 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -698,11 +698,10 @@ struct AsyncTableAssemblyState {
698698
};
699699

700700
Status AsyncScanner::Scan(std::function<Status(TaggedRecordBatch)> visitor) {
701-
return internal::RunSynchronouslyVoid(
702-
[this, &visitor](Executor* executor) {
703-
return VisitBatchesAsync(visitor, executor);
704-
},
705-
scan_options_->use_threads);
701+
auto top_level_task = [this, &visitor](Executor* executor) {
702+
return VisitBatchesAsync(visitor, executor);
703+
};
704+
return internal::RunSynchronously<Future<>>(top_level_task, scan_options_->use_threads);
706705
}
707706

708707
Future<> AsyncScanner::VisitBatchesAsync(std::function<Status(TaggedRecordBatch)> visitor,
@@ -729,10 +728,9 @@ Future<std::shared_ptr<Table>> AsyncScanner::ToTableAsync(
729728
auto table_building_gen = MakeMappedGenerator<EnumeratedRecordBatch>(
730729
positioned_batch_gen, table_building_task);
731730

732-
return DiscardAllFromAsyncGenerator(table_building_gen)
733-
.Then([state, scan_options](const detail::Empty&) {
734-
return Table::FromRecordBatches(scan_options->projected_schema, state->Finish());
735-
});
731+
return DiscardAllFromAsyncGenerator(table_building_gen).Then([state, scan_options]() {
732+
return Table::FromRecordBatches(scan_options->projected_schema, state->Finish());
733+
});
736734
}
737735

738736
Result<int64_t> AsyncScanner::CountRows() {

cpp/src/arrow/filesystem/s3fs.cc

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1595,8 +1595,7 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
15951595

15961596
TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
15971597
handle_results, handle_error, handle_recursion)
1598-
.AddCallback([collector, producer,
1599-
self](const Result<::arrow::detail::Empty>& res) mutable {
1598+
.AddCallback([collector, producer, self](const Status& status) mutable {
16001599
auto st = collector->Finish(self.get());
16011600
if (!st.ok()) {
16021601
producer.Push(st);
@@ -1645,11 +1644,7 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
16451644
struct DeleteCallback {
16461645
const std::string bucket;
16471646

1648-
Status operator()(const Result<S3Model::DeleteObjectsOutcome>& result) {
1649-
if (!result.ok()) {
1650-
return result.status();
1651-
}
1652-
const auto& outcome = *result;
1647+
Status operator()(const S3Model::DeleteObjectsOutcome& outcome) {
16531648
if (!outcome.IsSuccess()) {
16541649
return ErrorToStatus(outcome.GetError());
16551650
}
@@ -1754,7 +1749,10 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
17541749
Future<std::vector<std::string>> ListBucketsAsync(io::IOContext ctx) {
17551750
auto self = shared_from_this();
17561751
return DeferNotOk(SubmitIO(ctx, [self]() { return self->client_->ListBuckets(); }))
1757-
.Then(Impl::ProcessListBuckets);
1752+
// TODO(ARROW-12655) Change to Then(Impl::ProcessListBuckets)
1753+
.Then([](const Aws::S3::Model::ListBucketsOutcome& outcome) {
1754+
return Impl::ProcessListBuckets(outcome);
1755+
});
17581756
}
17591757

17601758
Result<std::shared_ptr<ObjectInputFile>> OpenInputFile(const std::string& s,

cpp/src/arrow/io/memory_test.cc

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -779,8 +779,8 @@ TEST(RangeReadCache, Concurrency) {
779779
ASSERT_OK(cache.Cache(ranges));
780780
std::vector<Future<std::shared_ptr<Buffer>>> futures;
781781
for (const auto& range : ranges) {
782-
futures.push_back(cache.WaitFor({range}).Then(
783-
[&cache, range](const detail::Empty&) { return cache.Read(range); }));
782+
futures.push_back(
783+
cache.WaitFor({range}).Then([&cache, range]() { return cache.Read(range); }));
784784
}
785785
for (auto fut : futures) {
786786
ASSERT_FINISHES_OK(fut);

cpp/src/arrow/testing/gtest_util.cc

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -637,7 +637,7 @@ Future<> SleepAsync(double seconds) {
637637
auto out = Future<>::Make();
638638
std::thread([out, seconds]() mutable {
639639
SleepFor(seconds);
640-
out.MarkFinished(Status::OK());
640+
out.MarkFinished();
641641
}).detach();
642642
return out;
643643
}
@@ -646,7 +646,7 @@ Future<> SleepABitAsync() {
646646
auto out = Future<>::Make();
647647
std::thread([out]() mutable {
648648
SleepABit();
649-
out.MarkFinished(Status::OK());
649+
out.MarkFinished();
650650
}).detach();
651651
return out;
652652
}

cpp/src/arrow/type_fwd.h

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -37,10 +37,10 @@ class Result;
3737

3838
class Status;
3939

40-
namespace detail {
40+
namespace internal {
4141
struct Empty;
42-
}
43-
template <typename T = detail::Empty>
42+
} // namespace internal
43+
template <typename T = internal::Empty>
4444
class Future;
4545

4646
namespace util {

cpp/src/arrow/util/async_generator.h

Lines changed: 22 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -78,9 +78,9 @@ Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
7878
std::function<Status(T)> visitor) {
7979
struct LoopBody {
8080
struct Callback {
81-
Result<ControlFlow<detail::Empty>> operator()(const T& result) {
81+
Result<ControlFlow<>> operator()(const T& result) {
8282
if (IsIterationEnd(result)) {
83-
return Break(detail::Empty());
83+
return Break();
8484
} else {
8585
auto visited = visitor(result);
8686
if (visited.ok()) {
@@ -94,7 +94,7 @@ Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
9494
std::function<Status(T)> visitor;
9595
};
9696

97-
Future<ControlFlow<detail::Empty>> operator()() {
97+
Future<ControlFlow<>> operator()() {
9898
Callback callback{visitor};
9999
auto next = generator();
100100
return next.Then(std::move(callback));
@@ -484,13 +484,9 @@ class TransformingGenerator {
484484
// Otherwise, if not finished immediately, add callback to process results
485485
} else {
486486
auto self = this->shared_from_this();
487-
return next_fut.Then([self](const Result<T>& next_result) {
488-
if (next_result.ok()) {
489-
self->last_value_ = *next_result;
490-
return (*self)();
491-
} else {
492-
return Future<V>::MakeFinished(next_result.status());
493-
}
487+
return next_fut.Then([self](const T& next_result) {
488+
self->last_value_ = next_result;
489+
return (*self)();
494490
});
495491
}
496492
}
@@ -565,7 +561,7 @@ class SerialReadaheadGenerator {
565561
// Lazy generator, need to wait for the first ask to prime the pump
566562
state_->first_ = false;
567563
auto next = state_->source_();
568-
return next.Then(Callback{state_});
564+
return next.Then(Callback{state_}, ErrCallback{state_});
569565
}
570566

571567
// This generator is not async-reentrant. We won't be called until the last
@@ -600,7 +596,7 @@ class SerialReadaheadGenerator {
600596
readahead_queue_(max_readahead + 1) {}
601597

602598
Status Pump(const std::shared_ptr<State>& self) {
603-
// Can't do readahead_queue.write(source().Then(Callback{self})) because then the
599+
// Can't do readahead_queue.write(source().Then(...)) because then the
604600
// callback might run immediately and add itself to the queue before this gets added
605601
// to the queue messing up the order.
606602
auto next_slot = std::make_shared<Future<T>>();
@@ -614,7 +610,7 @@ class SerialReadaheadGenerator {
614610
// writing. That is because this callback (the callback for future X) must be
615611
// finished before future X is marked complete and this source is not pulled
616612
// reentrantly so it will not poll for future X+1 until this callback has completed.
617-
*next_slot = source_().Then(Callback{self});
613+
*next_slot = source_().Then(Callback{self}, ErrCallback{self});
618614
return Status::OK();
619615
}
620616

@@ -634,21 +630,25 @@ class SerialReadaheadGenerator {
634630
};
635631

636632
struct Callback {
637-
Result<T> operator()(const Result<T>& maybe_next) {
638-
if (!maybe_next.ok()) {
639-
state_->finished_.store(true);
640-
return maybe_next;
641-
}
642-
const auto& next = *maybe_next;
633+
Result<T> operator()(const T& next) {
643634
if (IsIterationEnd(next)) {
644635
state_->finished_.store(true);
645-
return maybe_next;
636+
return next;
646637
}
647638
auto last_available = state_->spaces_available_.fetch_sub(1);
648639
if (last_available > 1) {
649640
ARROW_RETURN_NOT_OK(state_->Pump(state_));
650641
}
651-
return maybe_next;
642+
return next;
643+
}
644+
645+
std::shared_ptr<State> state_;
646+
};
647+
648+
struct ErrCallback {
649+
Result<T> operator()(const Status& st) {
650+
state_->finished_.store(true);
651+
return st;
652652
}
653653

654654
std::shared_ptr<State> state_;
@@ -1308,7 +1308,7 @@ class BackgroundGenerator {
13081308
// If the task is still cleaning up we need to wait for it to finish before
13091309
// restarting. We also want to block the consumer until we've restarted the
13101310
// reader to avoid multiple restarts
1311-
return task_finished.Then([state, next](...) {
1311+
return task_finished.Then([state, next]() {
13121312
// This may appear dangerous (recursive mutex) but we should be guaranteed the
13131313
// outer guard has been released by this point. We know...
13141314
// * task_finished is not already finished (it would be invalid in that case)

cpp/src/arrow/util/async_generator_test.cc

Lines changed: 8 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -52,8 +52,7 @@ AsyncGenerator<T> FailsAt(AsyncGenerator<T> src, int failing_index) {
5252
template <typename T>
5353
AsyncGenerator<T> SlowdownABit(AsyncGenerator<T> source) {
5454
return MakeMappedGenerator<T, T>(std::move(source), [](const T& res) -> Future<T> {
55-
return SleepABitAsync().Then(
56-
[res](const Result<detail::Empty>& empty) { return res; });
55+
return SleepABitAsync().Then([res]() { return res; });
5756
});
5857
}
5958

@@ -164,7 +163,7 @@ class ReentrantChecker {
164163
std::atomic<bool> valid;
165164
};
166165
struct Callback {
167-
Future<T> operator()(const Result<T>& result) {
166+
Future<T> operator()(const T& result) {
168167
state_->generated_unfinished_future.store(false);
169168
return result;
170169
}
@@ -362,9 +361,7 @@ TEST(TestAsyncUtil, MapAsync) {
362361
std::vector<TestInt> input = {1, 2, 3};
363362
auto generator = AsyncVectorIt(input);
364363
std::function<Future<TestStr>(const TestInt&)> mapper = [](const TestInt& in) {
365-
return SleepAsync(1e-3).Then([in](const Result<detail::Empty>& empty) {
366-
return TestStr(std::to_string(in.value));
367-
});
364+
return SleepAsync(1e-3).Then([in]() { return TestStr(std::to_string(in.value)); });
368365
};
369366
auto mapped = MakeMappedGenerator(std::move(generator), mapper);
370367
std::vector<TestStr> expected{"1", "2", "3"};
@@ -383,7 +380,7 @@ TEST(TestAsyncUtil, MapReentrant) {
383380
Future<> can_proceed = Future<>::Make();
384381
std::function<Future<TestStr>(const TestInt&)> mapper = [&](const TestInt& in) {
385382
map_tasks_running.fetch_add(1);
386-
return can_proceed.Then([in](...) { return TestStr(std::to_string(in.value)); });
383+
return can_proceed.Then([in]() { return TestStr(std::to_string(in.value)); });
387384
};
388385
auto mapped = MakeMappedGenerator(std::move(source), mapper);
389386

@@ -469,7 +466,7 @@ TEST_P(FromFutureFixture, Basic) {
469466
auto source = Future<std::vector<TestInt>>::MakeFinished(RangeVector(3));
470467
if (IsSlow()) {
471468
source = SleepABitAsync().Then(
472-
[](...) -> Result<std::vector<TestInt>> { return RangeVector(3); });
469+
[]() -> Result<std::vector<TestInt>> { return RangeVector(3); });
473470
}
474471
auto slow = IsSlow();
475472
auto to_gen = source.Then([slow](const std::vector<TestInt>& vec) {
@@ -651,7 +648,7 @@ TEST(TestAsyncUtil, MakeTransferredGenerator) {
651648
MakeTransferredGenerator<TestInt>(std::move(slow_generator), thread_pool.get());
652649

653650
auto current_thread_id = std::this_thread::get_id();
654-
auto fut = transferred().Then([&current_thread_id](const Result<TestInt>& result) {
651+
auto fut = transferred().Then([&current_thread_id](const TestInt&) {
655652
ASSERT_NE(current_thread_id, std::this_thread::get_id());
656653
});
657654

@@ -1009,8 +1006,8 @@ TEST(TestAsyncUtil, SerialReadaheadStressFailing) {
10091006
AsyncGenerator<TestInt> it = BackgroundAsyncVectorIt(RangeVector(NITEMS));
10101007
AsyncGenerator<TestInt> fails_at_ten = [&it]() {
10111008
auto next = it();
1012-
return next.Then([](const Result<TestInt>& item) -> Result<TestInt> {
1013-
if (item->value >= 10) {
1009+
return next.Then([](const TestInt& item) -> Result<TestInt> {
1010+
if (item.value >= 10) {
10141011
return Status::Invalid("XYZ");
10151012
} else {
10161013
return item;

0 commit comments

Comments
 (0)