forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 1
/
Copy pathtask_manager.cc
1591 lines (1428 loc) · 62.2 KB
/
task_manager.cc
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
// Copyright 2017 The Ray Authors.
//
// 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.
#include "ray/core_worker/task_manager.h"
#include <algorithm>
#include <memory>
#include <string>
#include <unordered_map>
#include <utility>
#include <vector>
#include "absl/strings/match.h"
#include "ray/common/buffer.h"
#include "ray/common/common_protocol.h"
#include "ray/core_worker/actor_manager.h"
#include "ray/gcs/pb_util.h"
#include "ray/util/exponential_backoff.h"
#include "ray/util/util.h"
namespace ray {
namespace core {
// Start throttling task failure logs once we hit this threshold.
constexpr int64_t kTaskFailureThrottlingThreshold = 50;
// Throttle task failure logs to once this interval.
constexpr int64_t kTaskFailureLoggingFrequencyMillis = 5000;
absl::flat_hash_set<ObjectID> ObjectRefStream::GetItemsUnconsumed() const {
absl::flat_hash_set<ObjectID> result;
for (int64_t index = 0; index <= max_index_seen_; index++) {
const auto &object_id = GetObjectRefAtIndex(index);
if (refs_written_to_stream_.find(object_id) == refs_written_to_stream_.end()) {
continue;
}
if (index >= next_index_) {
result.emplace(object_id);
}
}
if (end_of_stream_index_ != -1) {
// End of stream index is never consumed by a caller
// so we should add it here.
const auto &object_id = GetObjectRefAtIndex(end_of_stream_index_);
result.emplace(object_id);
}
// Temporarily owned refs are not consumed.
for (const auto &object_id : temporarily_owned_refs_) {
result.emplace(object_id);
}
return result;
}
std::vector<ObjectID> ObjectRefStream::PopUnconsumedItems() {
// Get all unconsumed refs.
std::vector<ObjectID> unconsumed_ids;
for (int64_t index = 0; index <= max_index_seen_; index++) {
const auto &object_id = GetObjectRefAtIndex(index);
auto it = refs_written_to_stream_.find(object_id);
if (it == refs_written_to_stream_.end()) {
continue;
}
if (index >= next_index_) {
unconsumed_ids.push_back(object_id);
refs_written_to_stream_.erase(it);
}
}
if (end_of_stream_index_ != -1) {
// End of stream index is never consumed by a caller
// so we should add it here.
const auto &object_id = GetObjectRefAtIndex(end_of_stream_index_);
unconsumed_ids.push_back(object_id);
}
// Temporarily owned refs are not consumed.
for (const auto &object_id : temporarily_owned_refs_) {
unconsumed_ids.push_back(object_id);
}
temporarily_owned_refs_.clear();
return unconsumed_ids;
}
bool ObjectRefStream::IsObjectConsumed(int64_t item_index) const {
return item_index < next_index_;
}
Status ObjectRefStream::TryReadNextItem(ObjectID *object_id_out) {
*object_id_out = GetObjectRefAtIndex(next_index_);
if (IsFinished()) {
// next_index_ cannot be bigger than end_of_stream_index_.
RAY_CHECK(next_index_ == end_of_stream_index_);
RAY_LOG(DEBUG) << "ObjectRefStream of an id " << generator_id_
<< " has no more objects.";
return Status::ObjectRefEndOfStream("");
}
auto it = refs_written_to_stream_.find(*object_id_out);
if (it != refs_written_to_stream_.end()) {
total_num_object_consumed_ += 1;
next_index_ += 1;
RAY_LOG_EVERY_MS(DEBUG, 10000) << "Get the next object id " << *object_id_out
<< " generator id: " << generator_id_;
} else {
// If the current index hasn't been written, return nothing.
// The caller is supposed to retry.
RAY_LOG_EVERY_MS(DEBUG, 10000)
<< "Object not available. Current index: " << next_index_
<< " end_of_stream_index_: " << end_of_stream_index_
<< " generator id: " << generator_id_;
*object_id_out = ObjectID::Nil();
}
return Status::OK();
}
bool ObjectRefStream::IsFinished() const {
bool is_eof_set = end_of_stream_index_ != -1;
return is_eof_set && next_index_ >= end_of_stream_index_;
}
std::pair<ObjectID, bool> ObjectRefStream::PeekNextItem() {
const auto &object_id = GetObjectRefAtIndex(next_index_);
if (refs_written_to_stream_.find(object_id) == refs_written_to_stream_.end()) {
return {object_id, false};
} else {
return {object_id, true};
}
}
bool ObjectRefStream::TemporarilyInsertToStreamIfNeeded(const ObjectID &object_id) {
// Write to a stream if the object ID is not consumed yet.
if (refs_written_to_stream_.find(object_id) == refs_written_to_stream_.end()) {
temporarily_owned_refs_.insert(object_id);
return true;
}
return false;
}
bool ObjectRefStream::InsertToStream(const ObjectID &object_id, int64_t item_index) {
RAY_CHECK_EQ(object_id, GetObjectRefAtIndex(item_index));
if (end_of_stream_index_ != -1 && item_index >= end_of_stream_index_) {
RAY_CHECK(next_index_ <= end_of_stream_index_);
// Ignore the index after the end of the stream index.
// It can happen if the stream is marked as ended
// and a new item is written. E.g., Report RPC sent ->
// worker crashes -> worker crash detected (task failed)
// -> report RPC received.
return false;
}
if (item_index < next_index_) {
// Index is already used. Don't write it to the stream.
return false;
}
if (temporarily_owned_refs_.find(object_id) != temporarily_owned_refs_.end()) {
temporarily_owned_refs_.erase(object_id);
}
auto [_, inserted] = refs_written_to_stream_.emplace(object_id);
if (!inserted) {
return false;
}
max_index_seen_ = std::max(max_index_seen_, item_index);
total_num_object_written_ += 1;
return true;
}
void ObjectRefStream::MarkEndOfStream(int64_t item_index,
ObjectID *object_id_in_last_index) {
if (end_of_stream_index_ != -1) {
return;
}
// ObjectRefStream should guarantee that next_index_ will always have an
// object value, to avoid hanging the caller the next time it tries to read
// the stream.
//
// NOTE: If the task returns a nondeterministic number of values, the second
// try may return fewer values than the first try. If the first try fails
// mid-execution, then on a successful second try, when we mark the end of
// the stream here, any extra unconsumed returns from the first try will be
// dropped.
end_of_stream_index_ = std::max(next_index_, item_index);
auto end_of_stream_id = GetObjectRefAtIndex(end_of_stream_index_);
*object_id_in_last_index = end_of_stream_id;
}
ObjectID ObjectRefStream::GetObjectRefAtIndex(int64_t generator_index) const {
RAY_CHECK_LT(generator_index, RayConfig::instance().max_num_generator_returns());
// Index 1 is reserved for the first task return from a generator task itself.
return ObjectID::FromIndex(generator_task_id_, 2 + generator_index);
}
std::vector<rpc::ObjectReference> TaskManager::AddPendingTask(
const rpc::Address &caller_address,
const TaskSpecification &spec,
const std::string &call_site,
int max_retries) {
int32_t max_oom_retries =
(max_retries != 0) ? RayConfig::instance().task_oom_retries() : 0;
RAY_LOG(DEBUG) << "Adding pending task " << spec.TaskId() << " with " << max_retries
<< " retries, " << max_oom_retries << " oom retries";
// Add references for the dependencies to the task.
std::vector<ObjectID> task_deps;
for (size_t i = 0; i < spec.NumArgs(); i++) {
if (spec.ArgByRef(i)) {
task_deps.push_back(spec.ArgId(i));
RAY_LOG(DEBUG) << "Adding arg ID " << spec.ArgId(i);
} else {
const auto &inlined_refs = spec.ArgInlinedRefs(i);
for (const auto &inlined_ref : inlined_refs) {
const auto inlined_id = ObjectID::FromBinary(inlined_ref.object_id());
task_deps.push_back(inlined_id);
RAY_LOG(DEBUG) << "Adding inlined ID " << inlined_id;
}
}
}
if (spec.IsActorTask()) {
const auto actor_creation_return_id = spec.ActorCreationDummyObjectId();
task_deps.push_back(actor_creation_return_id);
}
// Add new owned objects for the return values of the task.
size_t num_returns = spec.NumReturns();
std::vector<rpc::ObjectReference> returned_refs;
returned_refs.reserve(num_returns);
std::vector<ObjectID> return_ids;
return_ids.reserve(num_returns);
for (size_t i = 0; i < num_returns; i++) {
auto return_id = spec.ReturnId(i);
if (!spec.IsActorCreationTask()) {
bool is_reconstructable = max_retries != 0;
// We pass an empty vector for inner IDs because we do not know the return
// value of the task yet. If the task returns an ID(s), the worker will
// publish the WaitForRefRemoved message that we are now a borrower for
// the inner IDs. Note that this message can be received *before* the
// PushTaskReply.
// NOTE(swang): We increment the local ref count to ensure that the
// object is considered in scope before we return the ObjectRef to the
// language frontend. Note that the language bindings should set
// skip_adding_local_ref=True to avoid double referencing the object.
reference_counter_.AddOwnedObject(return_id,
/*contained_ids=*/{},
caller_address,
call_site,
-1,
is_reconstructable,
/*add_local_ref=*/true);
}
return_ids.push_back(return_id);
rpc::ObjectReference ref;
ref.set_object_id(spec.ReturnId(i).Binary());
ref.mutable_owner_address()->CopyFrom(caller_address);
ref.set_call_site(call_site);
returned_refs.push_back(std::move(ref));
}
reference_counter_.UpdateSubmittedTaskReferences(return_ids, task_deps);
// If it is a generator task, create an object ref stream.
// The language frontend is responsible for calling DeleteObjectRefStream.
if (spec.IsStreamingGenerator()) {
const auto generator_id = spec.ReturnId(0);
RAY_LOG(DEBUG) << "Create an object ref stream of an id " << generator_id;
absl::MutexLock lock(&object_ref_stream_ops_mu_);
auto inserted =
object_ref_streams_.emplace(generator_id, ObjectRefStream(generator_id));
ref_stream_execution_signal_callbacks_.emplace(
generator_id, std::vector<ExecutionSignalCallback>());
RAY_CHECK(inserted.second);
}
{
absl::MutexLock lock(&mu_);
auto inserted = submissible_tasks_.try_emplace(
spec.TaskId(), spec, max_retries, num_returns, task_counter_, max_oom_retries);
RAY_CHECK(inserted.second);
num_pending_tasks_++;
}
RAY_UNUSED(task_event_buffer_.RecordTaskStatusEventIfNeeded(
spec.TaskId(),
spec.JobId(),
spec.AttemptNumber(),
spec,
rpc::TaskStatus::PENDING_ARGS_AVAIL,
/* include_task_info */ true));
return returned_refs;
}
bool TaskManager::ResubmitTask(const TaskID &task_id, std::vector<ObjectID> *task_deps) {
RAY_CHECK(task_deps->empty());
TaskSpecification spec;
bool resubmit = false;
{
absl::MutexLock lock(&mu_);
auto it = submissible_tasks_.find(task_id);
if (it == submissible_tasks_.end()) {
// This can happen when the task has already been
// retried up to its max attempts.
return false;
}
if (!it->second.IsPending()) {
resubmit = true;
MarkTaskRetryOnResubmit(it->second);
num_pending_tasks_++;
// The task is pending again, so it's no longer counted as lineage. If
// the task finishes and we still need the spec, we'll add the task back
// to the footprint sum.
total_lineage_footprint_bytes_ -= it->second.lineage_footprint_bytes;
it->second.lineage_footprint_bytes = 0;
if (it->second.num_retries_left > 0) {
it->second.num_retries_left--;
} else {
RAY_CHECK(it->second.num_retries_left == -1);
}
spec = it->second.spec;
}
}
if (!resubmit) {
return true;
}
task_deps->reserve(spec.NumArgs());
for (size_t i = 0; i < spec.NumArgs(); i++) {
if (spec.ArgByRef(i)) {
task_deps->emplace_back(spec.ArgId(i));
} else {
const auto &inlined_refs = spec.ArgInlinedRefs(i);
for (const auto &inlined_ref : inlined_refs) {
task_deps->emplace_back(ObjectID::FromBinary(inlined_ref.object_id()));
}
}
}
reference_counter_.UpdateResubmittedTaskReferences(*task_deps);
for (const auto &task_dep : *task_deps) {
bool was_freed = reference_counter_.TryMarkFreedObjectInUseAgain(task_dep);
if (was_freed) {
RAY_LOG(DEBUG) << "Dependency " << task_dep << " of task " << task_id
<< " was freed";
// We do not keep around copies for objects that were freed, but now that
// they're needed for recovery, we need to generate and pin a new copy.
// Delete the old in-memory marker that indicated that the object was
// freed. Now workers that attempt to get the object will be able to get
// the reconstructed value.
in_memory_store_.Delete({task_dep});
}
}
if (spec.IsActorTask()) {
const auto actor_creation_return_id = spec.ActorCreationDummyObjectId();
reference_counter_.UpdateResubmittedTaskReferences({actor_creation_return_id});
}
RAY_LOG(INFO) << "Resubmitting task that produced lost plasma object, attempt #"
<< spec.AttemptNumber() << ": " << spec.DebugString();
// We should actually detect if the actor for this task is dead, but let's just assume
// it's not for now.
retry_task_callback_(spec, /*object_recovery*/ true, /*delay_ms*/ 0);
return true;
}
void TaskManager::DrainAndShutdown(std::function<void()> shutdown) {
bool has_pending_tasks = false;
{
absl::MutexLock lock(&mu_);
if (num_pending_tasks_ > 0) {
has_pending_tasks = true;
RAY_LOG(WARNING)
<< "This worker is still managing " << submissible_tasks_.size()
<< " in flight tasks, waiting for them to finish before shutting down.";
shutdown_hook_ = std::move(shutdown);
}
}
// Do not hold the lock when calling callbacks.
if (!has_pending_tasks) {
shutdown();
}
}
bool TaskManager::IsTaskSubmissible(const TaskID &task_id) const {
absl::MutexLock lock(&mu_);
return submissible_tasks_.contains(task_id);
}
bool TaskManager::IsTaskPending(const TaskID &task_id) const {
absl::MutexLock lock(&mu_);
const auto it = submissible_tasks_.find(task_id);
if (it == submissible_tasks_.end()) {
return false;
}
return it->second.IsPending();
}
bool TaskManager::IsTaskWaitingForExecution(const TaskID &task_id) const {
absl::MutexLock lock(&mu_);
const auto it = submissible_tasks_.find(task_id);
if (it == submissible_tasks_.end()) {
return false;
}
return it->second.IsWaitingForExecution();
}
size_t TaskManager::NumSubmissibleTasks() const {
absl::MutexLock lock(&mu_);
return submissible_tasks_.size();
}
size_t TaskManager::NumPendingTasks() const {
absl::MutexLock lock(&mu_);
return num_pending_tasks_;
}
bool TaskManager::HandleTaskReturn(const ObjectID &object_id,
const rpc::ReturnObject &return_object,
const NodeID &worker_raylet_id,
bool store_in_plasma) {
bool direct_return = false;
reference_counter_.UpdateObjectSize(object_id, return_object.size());
RAY_LOG(DEBUG) << "Task return object " << object_id << " has size "
<< return_object.size();
const auto nested_refs =
VectorFromProtobuf<rpc::ObjectReference>(return_object.nested_inlined_refs());
if (return_object.in_plasma()) {
// NOTE(swang): We need to add the location of the object before marking
// it as local in the in-memory store so that the data locality policy
// will choose the right raylet for any queued dependent tasks.
reference_counter_.UpdateObjectPinnedAtRaylet(object_id, worker_raylet_id);
// Mark it as in plasma with a dummy object.
RAY_CHECK(
in_memory_store_.Put(RayObject(rpc::ErrorType::OBJECT_IN_PLASMA), object_id));
} else {
// NOTE(swang): If a direct object was promoted to plasma, then we do not
// record the node ID that it was pinned at, which means that we will not
// be able to reconstruct it if the plasma object copy is lost. However,
// this is okay because the pinned copy is on the local node, so we will
// fate-share with the object if the local node fails.
std::shared_ptr<LocalMemoryBuffer> data_buffer;
if (!return_object.data().empty()) {
data_buffer = std::make_shared<LocalMemoryBuffer>(
const_cast<uint8_t *>(
reinterpret_cast<const uint8_t *>(return_object.data().data())),
return_object.data().size());
}
std::shared_ptr<LocalMemoryBuffer> metadata_buffer;
if (!return_object.metadata().empty()) {
metadata_buffer = std::make_shared<LocalMemoryBuffer>(
const_cast<uint8_t *>(
reinterpret_cast<const uint8_t *>(return_object.metadata().data())),
return_object.metadata().size());
}
RayObject object(data_buffer, metadata_buffer, nested_refs);
if (store_in_plasma) {
put_in_local_plasma_callback_(object, object_id);
} else {
direct_return = in_memory_store_.Put(object, object_id);
}
}
rpc::Address owner_address;
if (reference_counter_.GetOwner(object_id, &owner_address) && !nested_refs.empty()) {
std::vector<ObjectID> nested_ids;
nested_ids.reserve(nested_refs.size());
for (const auto &nested_ref : nested_refs) {
nested_ids.emplace_back(ObjectRefToId(nested_ref));
}
reference_counter_.AddNestedObjectIds(object_id, nested_ids, owner_address);
}
return direct_return;
}
bool TaskManager::TryDelObjectRefStream(const ObjectID &generator_id) {
absl::MutexLock lock(&object_ref_stream_ops_mu_);
bool can_gc_lineage = TryDelObjectRefStreamInternal(generator_id);
if (!can_gc_lineage) {
RAY_LOG(DEBUG) << "Generator " << generator_id
<< " still has lineage in scope, try again later";
return false;
}
RAY_LOG(DEBUG) << "Deleting object ref stream of an id " << generator_id;
object_ref_streams_.erase(generator_id);
return true;
}
Status TaskManager::TryReadObjectRefStream(const ObjectID &generator_id,
ObjectID *object_id_out) {
auto backpressure_threshold = 0;
{
absl::MutexLock lock(&mu_);
auto it = submissible_tasks_.find(generator_id.TaskId());
if (it != submissible_tasks_.end()) {
backpressure_threshold = it->second.spec.GeneratorBackpressureNumObjects();
}
}
absl::MutexLock lock(&object_ref_stream_ops_mu_);
RAY_CHECK(object_id_out != nullptr);
auto stream_it = object_ref_streams_.find(generator_id);
RAY_CHECK(stream_it != object_ref_streams_.end())
<< "TryReadObjectRefStream API can be used only when the stream has been "
"created "
"and not removed.";
auto status = stream_it->second.TryReadNextItem(object_id_out);
/// If you could read the next item, signal the executor to resume
/// if necessary.
if (status.ok()) {
auto total_generated = stream_it->second.TotalNumObjectWritten();
auto total_consumed = stream_it->second.TotalNumObjectConsumed();
auto total_unconsumed = total_generated - total_consumed;
if (backpressure_threshold != -1 && total_unconsumed < backpressure_threshold) {
auto it = ref_stream_execution_signal_callbacks_.find(generator_id);
if (it != ref_stream_execution_signal_callbacks_.end()) {
for (const auto &execution_signal : it->second) {
RAY_LOG(DEBUG) << "The task for a stream " << generator_id
<< " should resume. total_generated: " << total_generated
<< ". total_consumed: " << total_consumed
<< ". threshold: " << backpressure_threshold;
execution_signal(Status::OK(), total_consumed);
}
it->second.clear();
}
}
}
return status;
}
bool TaskManager::StreamingGeneratorIsFinished(const ObjectID &generator_id) const {
absl::MutexLock lock(&object_ref_stream_ops_mu_);
auto stream_it = object_ref_streams_.find(generator_id);
RAY_CHECK(stream_it != object_ref_streams_.end())
<< "IsFinished API can be used only when the stream has been "
"created "
"and not removed.";
return stream_it->second.IsFinished();
}
bool TaskManager::TryDelObjectRefStreamInternal(const ObjectID &generator_id) {
// Call execution signal callbacks to ensure that the executor does not block
// after the generator goes out of scope at the caller.
auto signal_it = ref_stream_execution_signal_callbacks_.find(generator_id);
if (signal_it != ref_stream_execution_signal_callbacks_.end()) {
RAY_LOG(DEBUG) << "Deleting execution signal callbacks for generator "
<< generator_id;
for (const auto &execution_signal : signal_it->second) {
execution_signal(Status::NotFound("Stream is deleted."), -1);
}
// We may still receive more generator return reports in the future, if the
// generator task is still running or is retried. They will get the
// callback immediately because we deleted this entry.
ref_stream_execution_signal_callbacks_.erase(signal_it);
}
auto stream_it = object_ref_streams_.find(generator_id);
if (stream_it == object_ref_streams_.end()) {
return true;
}
// Remove any unconsumed refs from the stream metadata in-memory store.
auto unconsumed_ids = stream_it->second.PopUnconsumedItems();
std::vector<ObjectID> deleted;
reference_counter_.TryReleaseLocalRefs(unconsumed_ids, &deleted);
in_memory_store_.Delete(deleted);
int64_t num_objects_generated = stream_it->second.EofIndex();
if (num_objects_generated == -1) {
RAY_LOG(DEBUG) << "Skip streaming generator deletion, EOF not written yet";
// Generator task has not finished yet. Wait for EoF to be marked before
// deleting.
return false;
}
bool can_gc_lineage = reference_counter_.CheckGeneratorRefsLineageOutOfScope(
generator_id, num_objects_generated);
return can_gc_lineage;
}
std::pair<ObjectID, bool> TaskManager::PeekObjectRefStream(const ObjectID &generator_id) {
ObjectID next_object_id;
absl::MutexLock lock(&object_ref_stream_ops_mu_);
auto stream_it = object_ref_streams_.find(generator_id);
RAY_CHECK(stream_it != object_ref_streams_.end())
<< "PeekObjectRefStream API can be used only when the stream has been "
"created and not removed.";
const auto &result = stream_it->second.PeekNextItem();
// Temporarily own the ref since the corresponding reference is probably
// not reported yet.
TemporarilyOwnGeneratorReturnRefIfNeededInternal(result.first /*=object_id*/,
generator_id);
return result;
}
bool TaskManager::ObjectRefStreamExists(const ObjectID &generator_id) {
absl::MutexLock lock(&object_ref_stream_ops_mu_);
auto it = object_ref_streams_.find(generator_id);
return it != object_ref_streams_.end();
}
void TaskManager::MarkEndOfStream(const ObjectID &generator_id,
int64_t end_of_stream_index) {
absl::MutexLock lock(&object_ref_stream_ops_mu_);
ObjectID last_object_id;
auto stream_it = object_ref_streams_.find(generator_id);
if (stream_it == object_ref_streams_.end()) {
// Stream has been already deleted. Do not handle it.
return;
}
stream_it->second.MarkEndOfStream(end_of_stream_index, &last_object_id);
if (!last_object_id.IsNil()) {
RAY_LOG(DEBUG) << "Write EoF to the object ref stream. Index: "
<< stream_it->second.EofIndex()
<< ". Last object id: " << last_object_id;
reference_counter_.OwnDynamicStreamingTaskReturnRef(last_object_id, generator_id);
RayObject error(rpc::ErrorType::END_OF_STREAMING_GENERATOR);
// Put a dummy object at the end of the stream. We don't need to check if
// the object should be stored in plasma because the end of the stream is a
// fake ObjectRef that should never be read by the application.
in_memory_store_.Put(error, last_object_id);
}
}
bool TaskManager::HandleReportGeneratorItemReturns(
const rpc::ReportGeneratorItemReturnsRequest &request,
const ExecutionSignalCallback &execution_signal_callback) {
const auto &generator_id = ObjectID::FromBinary(request.generator_id());
const auto &task_id = generator_id.TaskId();
int64_t item_index = request.item_index();
uint64_t attempt_number = request.attempt_number();
// Every generated object has the same task id.
RAY_LOG(DEBUG) << "Received an intermediate result of index " << item_index
<< " generator_id: " << generator_id;
auto backpressure_threshold = -1;
{
absl::MutexLock lock(&mu_);
auto it = submissible_tasks_.find(task_id);
if (it != submissible_tasks_.end()) {
backpressure_threshold = it->second.spec.GeneratorBackpressureNumObjects();
if (it->second.spec.AttemptNumber() > attempt_number) {
// Generator task reports can arrive at any time. If the first attempt
// fails, we may receive a report from the first executor after the
// second attempt has started. In this case, we should ignore the first
// attempt.
execution_signal_callback(
Status::NotFound("Stale object reports from the previous attempt."), -1);
return false;
}
}
}
// NOTE: If it is the first execution (e.g., CompletePendingTask has never been called),
// it is always empty.
const auto store_in_plasma_ids = GetTaskReturnObjectsToStoreInPlasma(task_id);
absl::MutexLock lock(&object_ref_stream_ops_mu_);
auto stream_it = object_ref_streams_.find(generator_id);
if (stream_it == object_ref_streams_.end()) {
// Stream has been already deleted. Do not handle it.
execution_signal_callback(Status::NotFound("Stream is already deleted"), -1);
return false;
}
// TODO(sang): Support the regular return values as well.
size_t num_objects_written = 0;
for (const auto &return_object : request.dynamic_return_objects()) {
const auto object_id = ObjectID::FromBinary(return_object.object_id());
RAY_LOG(DEBUG) << "Write an object " << object_id
<< " to the object ref stream of id " << generator_id;
auto index_not_used_yet = stream_it->second.InsertToStream(object_id, item_index);
// If the ref was written to a stream, we should also
// own the dynamically generated task return.
// NOTE: If we call this method while holding a lock, it can deadlock.
if (index_not_used_yet) {
reference_counter_.OwnDynamicStreamingTaskReturnRef(object_id, generator_id);
num_objects_written += 1;
}
// When an object is reported, the object is ready to be fetched.
reference_counter_.UpdateObjectPendingCreation(object_id, false);
HandleTaskReturn(object_id,
return_object,
NodeID::FromBinary(request.worker_addr().raylet_id()),
/*store_in_plasma=*/store_in_plasma_ids.contains(object_id));
}
// Handle backpressure if needed.
auto total_generated = stream_it->second.TotalNumObjectWritten();
auto total_consumed = stream_it->second.TotalNumObjectConsumed();
if (stream_it->second.IsObjectConsumed(item_index)) {
execution_signal_callback(Status::OK(), total_consumed);
return false;
}
// Otherwise, follow the regular backpressure logic.
// NOTE, here we check `item_index - last_consumed_index >= backpressure_threshold`,
// instead of the number of unconsumed items, because we may receive the
// `HandleReportGeneratorItemReturns` requests out of order.
if (backpressure_threshold != -1 &&
(item_index - stream_it->second.LastConsumedIndex()) >= backpressure_threshold) {
RAY_LOG(DEBUG) << "Stream " << generator_id
<< " is backpressured. total_generated: " << total_generated
<< ". total_consumed: " << total_consumed
<< ". threshold: " << backpressure_threshold;
auto signal_it = ref_stream_execution_signal_callbacks_.find(generator_id);
if (signal_it == ref_stream_execution_signal_callbacks_.end()) {
execution_signal_callback(Status::NotFound("Stream is deleted."), -1);
} else {
signal_it->second.push_back(execution_signal_callback);
}
} else {
// No need to backpressure.
execution_signal_callback(Status::OK(), total_consumed);
}
return num_objects_written != 0;
}
bool TaskManager::TemporarilyOwnGeneratorReturnRefIfNeeded(const ObjectID &object_id,
const ObjectID &generator_id) {
absl::MutexLock lock(&object_ref_stream_ops_mu_);
return TemporarilyOwnGeneratorReturnRefIfNeededInternal(object_id, generator_id);
}
bool TaskManager::TemporarilyOwnGeneratorReturnRefIfNeededInternal(
const ObjectID &object_id, const ObjectID &generator_id) {
bool inserted_to_stream = false;
auto stream_it = object_ref_streams_.find(generator_id);
if (stream_it == object_ref_streams_.end()) {
return false;
}
auto &stream = stream_it->second;
inserted_to_stream = stream.TemporarilyInsertToStreamIfNeeded(object_id);
// We shouldn't hold a lock when calling reference counter API.
if (inserted_to_stream) {
RAY_LOG(DEBUG) << "Added streaming ref " << object_id;
reference_counter_.OwnDynamicStreamingTaskReturnRef(object_id, generator_id);
return true;
}
return false;
}
void TaskManager::CompletePendingTask(const TaskID &task_id,
const rpc::PushTaskReply &reply,
const rpc::Address &worker_addr,
bool is_application_error) {
RAY_LOG(DEBUG) << "Completing task " << task_id;
bool first_execution = false;
const auto store_in_plasma_ids =
GetTaskReturnObjectsToStoreInPlasma(task_id, &first_execution);
std::vector<ObjectID> dynamic_return_ids;
std::vector<ObjectID> dynamic_returns_in_plasma;
std::vector<ObjectID> direct_return_ids;
if (reply.dynamic_return_objects_size() > 0) {
RAY_CHECK(reply.return_objects_size() == 1)
<< "Dynamic generators only supported for num_returns=1";
const auto generator_id = ObjectID::FromBinary(reply.return_objects(0).object_id());
for (const auto &return_object : reply.dynamic_return_objects()) {
const auto object_id = ObjectID::FromBinary(return_object.object_id());
if (first_execution) {
reference_counter_.AddDynamicReturn(object_id, generator_id);
dynamic_return_ids.push_back(object_id);
}
if (!HandleTaskReturn(object_id,
return_object,
NodeID::FromBinary(worker_addr.raylet_id()),
store_in_plasma_ids.contains(object_id))) {
if (first_execution) {
dynamic_returns_in_plasma.push_back(object_id);
}
}
}
}
for (const auto &return_object : reply.return_objects()) {
const auto object_id = ObjectID::FromBinary(return_object.object_id());
if (HandleTaskReturn(object_id,
return_object,
NodeID::FromBinary(worker_addr.raylet_id()),
store_in_plasma_ids.contains(object_id))) {
direct_return_ids.push_back(object_id);
}
}
TaskSpecification spec;
bool release_lineage = true;
int64_t min_lineage_bytes_to_evict = 0;
{
absl::MutexLock lock(&mu_);
auto it = submissible_tasks_.find(task_id);
RAY_CHECK(it != submissible_tasks_.end())
<< "Tried to complete task that was not pending " << task_id;
spec = it->second.spec;
// Record any dynamically returned objects. We need to store these with the
// task spec so that the worker will recreate them if the task gets
// re-executed.
// TODO(sang): Remove this logic once streaming generator is the default.
if (first_execution) {
for (const auto &dynamic_return_id : dynamic_return_ids) {
RAY_LOG(DEBUG) << "Task " << task_id << " produced dynamic return object "
<< dynamic_return_id;
spec.AddDynamicReturnId(dynamic_return_id);
}
for (const auto &dynamic_return_id : dynamic_returns_in_plasma) {
it->second.reconstructable_return_ids.insert(dynamic_return_id);
}
if (spec.IsStreamingGenerator()) {
// Upon the first complete execution, set the number of streaming
// generator returns.
auto num_streaming_generator_returns =
reply.streaming_generator_return_ids_size();
if (num_streaming_generator_returns > 0) {
spec.SetNumStreamingGeneratorReturns(num_streaming_generator_returns);
RAY_LOG(DEBUG) << "Completed streaming generator task " << spec.TaskId()
<< " has " << spec.NumStreamingGeneratorReturns()
<< " return objects.";
for (const auto &return_id_info : reply.streaming_generator_return_ids()) {
if (return_id_info.is_plasma_object()) {
// TODO(swang): It is possible that the dynamically returned refs
// have already been consumed by the caller and deleted. This can
// cause a memory leak of the task metadata, because we will
// never receive a callback from the ReferenceCounter to erase
// the task.
it->second.reconstructable_return_ids.insert(
ObjectID::FromBinary(return_id_info.object_id()));
}
}
}
}
}
// Release the lineage for any non-plasma return objects.
// TODO(sang): Remove this logic once streaming generator is the default.
for (const auto &direct_return_id : direct_return_ids) {
RAY_LOG(DEBUG) << "Task " << it->first << " returned direct object "
<< direct_return_id << ", now has "
<< it->second.reconstructable_return_ids.size()
<< " plasma returns in scope";
it->second.reconstructable_return_ids.erase(direct_return_id);
}
RAY_LOG(DEBUG) << "Task " << it->first << " now has "
<< it->second.reconstructable_return_ids.size()
<< " plasma returns in scope";
it->second.num_successful_executions++;
if (is_application_error) {
SetTaskStatus(
it->second,
rpc::TaskStatus::FAILED,
worker::TaskStatusEvent::TaskStateUpdate(gcs::GetRayErrorInfo(
rpc::ErrorType::TASK_EXECUTION_EXCEPTION, reply.task_execution_error())));
} else {
SetTaskStatus(it->second, rpc::TaskStatus::FINISHED);
}
num_pending_tasks_--;
// A finished task can only be re-executed if it has some number of
// retries left and returned at least one object that is still in use and
// stored in plasma.
bool task_retryable = it->second.num_retries_left != 0 &&
!it->second.reconstructable_return_ids.empty();
if (task_retryable) {
// Pin the task spec if it may be retried again.
release_lineage = false;
it->second.lineage_footprint_bytes = it->second.spec.GetMessage().ByteSizeLong();
total_lineage_footprint_bytes_ += it->second.lineage_footprint_bytes;
if (total_lineage_footprint_bytes_ > max_lineage_bytes_) {
RAY_LOG(INFO) << "Total lineage size is " << total_lineage_footprint_bytes_ / 1e6
<< "MB, which exceeds the limit of " << max_lineage_bytes_ / 1e6
<< "MB";
min_lineage_bytes_to_evict =
total_lineage_footprint_bytes_ - (max_lineage_bytes_ / 2);
}
} else {
submissible_tasks_.erase(it);
}
}
// If it is a streaming generator, mark the end of stream since the task is finished.
// We handle this logic here because the lock shouldn't be held while calling
// HandleTaskReturn.
if (spec.IsStreamingGenerator()) {
const auto generator_id = ObjectID::FromBinary(reply.return_objects(0).object_id());
if (first_execution) {
ObjectID last_ref_in_stream;
MarkEndOfStream(generator_id, reply.streaming_generator_return_ids_size());
} else {
// The end of the stream should already have been marked on the first
// successful execution.
if (is_application_error) {
// It means the task was re-executed but failed with an application
// error. In this case, we should fail the rest of known streaming
// generator returns with the same error.
RAY_LOG(DEBUG) << "Streaming generator task " << spec.TaskId()
<< " failed with application error, failing "
<< spec.NumStreamingGeneratorReturns() << " return objects.";
RAY_CHECK_EQ(reply.return_objects_size(), 1);
for (size_t i = 0; i < spec.NumStreamingGeneratorReturns(); i++) {
const auto generator_return_id = spec.StreamingGeneratorReturnId(i);
RAY_CHECK_EQ(reply.return_objects_size(), 1);
const auto &return_object = reply.return_objects(0);
HandleTaskReturn(generator_return_id,
return_object,
NodeID::FromBinary(worker_addr.raylet_id()),
store_in_plasma_ids.contains(generator_return_id));
}
}
}
}
RemoveFinishedTaskReferences(spec, release_lineage, worker_addr, reply.borrowed_refs());
if (min_lineage_bytes_to_evict > 0) {
// Evict at least half of the current lineage.
auto bytes_evicted = reference_counter_.EvictLineage(min_lineage_bytes_to_evict);
RAY_LOG(INFO) << "Evicted " << bytes_evicted / 1e6 << "MB of task lineage.";
}
ShutdownIfNeeded();
}
bool TaskManager::RetryTaskIfPossible(const TaskID &task_id,
const rpc::RayErrorInfo &error_info) {
TaskSpecification spec;
bool will_retry = false;
int32_t num_retries_left = 0;
int32_t num_oom_retries_left = 0;
bool task_failed_due_to_oom = error_info.error_type() == rpc::ErrorType::OUT_OF_MEMORY;
{
absl::MutexLock lock(&mu_);
auto it = submissible_tasks_.find(task_id);
RAY_CHECK(it != submissible_tasks_.end())
<< "Tried to retry task that was not pending " << task_id;
RAY_CHECK(it->second.IsPending())
<< "Tried to retry task that was not pending " << task_id;
spec = it->second.spec;
num_retries_left = it->second.num_retries_left;
num_oom_retries_left = it->second.num_oom_retries_left;
if (task_failed_due_to_oom) {
if (num_oom_retries_left > 0) {
will_retry = true;
it->second.num_oom_retries_left--;
} else if (num_oom_retries_left == -1) {
will_retry = true;
} else {
RAY_CHECK(num_oom_retries_left == 0);
}
} else {
if (num_retries_left > 0) {
will_retry = true;
it->second.num_retries_left--;
} else if (num_retries_left == -1) {
will_retry = true;
} else {
RAY_CHECK(num_retries_left == 0);
}
}