"ResourceRequest" now uses 2 containers: a vector for predefined resources, and a map for custom resources.
This was intended to be a perf optimization. However, in practice, this makes the code more complex, and, moreover, prevents optimizations for some methods (e.g., "ResourceIds", "Size").
This PR removes the vector and makes ResourceRequest use only one map for all resources. Also, "ResourceIds" now returns a "boost:range" to allow iterating resource IDs without having to construct temporary sets.
microbenchmark shows a slight perf improvement.
last nightly: `placement group create/removal per second 837.76 +- 16.68`.
this PR: `placement group create/removal per second 895.76 +- 16.99`.
As we (@scv119 @iycheng @raulchen @Chong-Li @WangTaoTheTonic ) discussed offline, the GcsResourceScheduler on the GCS side should be unified to ClusterResourceScheduler.
There is already a big PR( #23268 ) to do this, but in order to make review easy, I will split it to two or mall small PRs.
This is [3/n]:
Move the implementation of all policies from gcs_resource_scheduler to bundle_scheduling_plocy
Delete gcs_resource_scheduler
Refactor gcs_resource_scheduler_test to cluster_resource_scheduler_2_test
BTW: The interface inside ISchedulingPolicy should be refactor in another PR, see the discussion #23323 (comment)
To be clear:
scorer related codes are moved out from gcs_resoruce_scheduler to scorer.h/.cc and no logic changes.
Policy related codes are moved out from gcs_resoruce_scheduler to bundle_scheduling_policy.h/.cc, and a small part of the logic in "GcsResourceScheduler::Schedule" is distributed into each policy.
Some codes inside gcs_placement_group_scheduler.h/.cc are changed to adapt to new data structure (SchedulingResult and SchedulingContext)
## Why are these changes needed?
This PR refactor the resource syncer to decouple it from GCS and raylet. GCS and raylet will use the same module to sync data. The integration will happen in the next PR.
There are several new introduced components:
* RaySyncer: the place where remote and local information sits. It's a coordinator layer.
* NodeState: keeps track of the local status, similar to NodeSyncConnection.
* NodeSyncConnection: keeps track of the sending and receiving information and make sure not sending the information the remote node knows.
The core protocol is that each node will send {what it has} - {what the target has} to the target.
For example, think about node A <-> B. A will send all A has exclude what B has to B.
Whenever when there is new information (from NodeState or NodeSyncConnection), it'll be passed to RaySyncer broadcast message to broadcast.
NodeSyncConnection is for the communication layer. It has two implementations Client and Server:
* Server => Client: client will send a long-polling request and server will response every 100ms if there is data to be sent.
* Client => Server: client will check every 100ms to see whether there is new data to be sent. If there is, just use RPC call to send the data.
Here is one example:
```mermaid
flowchart LR;
A-->B;
B-->C;
B-->D;
```
It means A initialize the connection to B and B initialize the connections to C and D
Now C generate a message M:
1. [C] RaySyncer check whether there is new message generated in C and get M
2. [C] RaySyncer will push M to NodeSyncConnection in local component (B)
3. [C] ServerSyncConnection will wait until B send a long polling and send the data to B
4. [B] B received the message from C and push it to local sync connection (C, A, D)
5. [B] ClientSyncConnection of C will not push it to its local queue since it's received by this channel.
6. [B] ClientSyncConnection of D will send this message to D
7. [B] ServerSyncConnection of A will be used to send this message to A (long-polling here)
8. [B] B will update NodeState (local component) with this message M
9. [D] D's pipelines is similar to 5) (with ServerSyncConnection) and 8)
10. [A] A's pipeline is similar to 5) and 8)
Running Datasets shuffle with 1TB data and 2k partitions sometimes times out due to a failed object fetch. This happens because the object directory notifies the PullManager that the object is already on the local node, even though it isn't. This seems to be a bug in the object directory.
To work around this on the PullManager side, this PR filters out the current node from the list of locations provided by the object directory. @jjyao confirmed that this fixes the issue for Datasets shuffle.
This is the first PR to refactor scheduler data structures (See #22850).
Major changes:
- Hid the implementation details in the `ResourceRequest` and `TaskResourceInstnaces` classes, which expose public methods such as algebra operators and comparison operators.
- Hid the differences between "predefined" and "custom" resources inside these 2 classes. Call sites can simply use the resource ID to access the resource, no matter it is predefined or custom.
- The predefined_resources vector now always has the full length. So no more "resize"s are needed.
- Removed the `ResourceCapacity` class. Now "total" and "available" resources are stored in separate fields in "NodeResources".
- Moved helper functions for FixedPoint vectors from "cluster_resource_data.h" to "fixed_point.h"
- "ResourceID" now has static methods to get the resource ids of predefined resources, e.g. "ResourceID::CPU()".
- Encapsulated unit-instance resource logic to "ResourceID"
Other planned changes that are not included in this PR:
- Rename ResourceRequest to ResourceSet, and move it to its own file.
- Remove the predefined vectors and always use maps.
Co-authored-by: Chong-Li <lc300133@antgroup.com>
This PR adds the API `setRuntimeEnv` for submitting a normal task, for the usage:
```java
RuntimeEnv runtimeEnv =
new RuntimeEnv.Builder()
.addEnvVar("KEY1", "A")
.build();
/// Return `A`
Ray.task(RuntimeEnvTest::getEnvVar, "KEY1").setRuntimeEnv(runtimeEnv).remote().get();
```
Getting or creating a named actor is a common pattern, however it is somewhat esoteric in how to achieve this. Add a utility function and test that it doesn't cause any scary error messages.
Actor.options(name="my_singleton", get_if_exists=True).remote(args)
- Move the URI reference logic from raylet to agent.
- Redefine the runtime env agent RPC to `CreateRuntimeEnvOrGet` and `DeleteRuntimeEnvIfPossible`
- More details https://github.com/ray-project/ray/issues/21695#issuecomment-1032161528
Future works
- We don't remove the `RuntimeEnvUris` from `RuntimeEnv` protobuf in current PR because gcs also uses those URIs to do GC by runtime_env_manager. We should also clear this.
- Ray client server shouldn't interact with agent directly. Or Ray client server should also decrease the reference count.
- Currently, `WorkerPool::HandleJobStarted` will be called multiple times for one job. So we should make sure this function is idempotent. Can we change this logic and make this function be called only once?
* Add new interface to policy for batch scheduling and unify the scheduling result and context
* Remove the dependence of GcsClient on ClusterResourceScheduler
* fix compile error
* fix lint error
Co-authored-by: 黑驰 <senlin.zsl@antgroup.com>
Make sure Python dependencies can be imported on demand, without the background importer thread. Use cases are:
If the pubsub notification for a new export is lost, importing can still be done.
Allow not running the background importer thread, without affecting Ray's functionalities.
Add a feature flag to support forking from Python workers, by
Enable fork support in gRPC.
Disable importer thread and only leave the main thread in the Python worker. The importer thread will not run after forking anyway.
In linux the thread name could not be longer than 15 chars.
When we use command like top, we are easy being confused by similar thread name like `resource_report_poller` and `resource_report_broadcaster` because they are both show `resource_report`.
This pr uses abbr to make the thread names shorter.
Next move of #22932. This pr replace unordered_map to flat_hash_map in core worker and object manager module.
Also some interfaces, like GetAllReferenceCounts, which expose user interfaces in Java/Python, is exclusive as it's a little bit complicated. We save them to deal with pg together.
The follow-up PRs would be migrating in reference counting, placement group and others.
In this PR, we disabled every 10min FullGC which is not triggered by a global gc event in Java worker. As detail, we added `triggered_by_global_gc` flag to indicate whether the gc event is triggered by a global gc event. If it's triggered by global gc, we still need to do FullGC.
Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
Currently, when a spill/restore worker fails and the state of it in the worker pool is idle, the worker pool will not clean up the metadata of the worker. Subsequent spill/restore requests will reuse this dead worker and RPC requests cannot succeed. This results in broken object spilling functionality.
This PR addresses the issue by removing disconnected IO workers from `registered_io_workers` and `idle_io_workers`.
It's really annoying to deal with parameter/argument conflicts. This is even frustrating when we merge code from the community to Ant's internal code base with hundreds of conflicts caused by parameters/arguments.
In this PR, I updated the clang-format style to make parameters/arguments stay on different lines if they can't fit into a single line.
There are several benefits:
* Conflict resolving is easier.
* Less potential human mistakes when resolving conflicts.
* Git history and Git blame are more straightforward.
* Better readability.
* Align with the new Python format style.
* Fix the normal task resources at GCS
* Fix comments
* Leave a TODO
* Bring back a UT
* consider object memory
* Fix
Co-authored-by: Chong-Li <lc300133@antgroup.com>
Previously, placement group had suboptimal bin-packing resulting in unexpected placement group stalls for users.
The root cause is lack of implementation for sorting of pg bundles by resource priority and size.
This PR implements a naive priority mechanism for bundles that can be improved upon (and even config by user in the future) in the GCS resource scheduler.
The behaviour is to schedule: "GPU" first, custom resources in int64_t order next, and finally, memory and then "CPU" last.
Next move of #19220. This pr replace unordered_map to flat_hash_map in most GCS code and some util & common modules.
The placement group part, which exposes user interfaces in Java/Python, is exclusive as it's a little bit complicated.
The follow-up PRs would be migrating in core worker, placement group and others.
This PR is part of resource reporting refactoring. In this PR ray syncer is moved from gcs_resource_manager to gcs_placement_group_scheduler. With this one, gcs_resource_manager is totally decoupled from resource broadcasting.
* refactor resource data structure in gcs
* fix comment
* fix lint error
* fix
* DISABLED_TestRejectedRequestWorkerLeaseReply as it depends on the update of normal task
Co-authored-by: 黑驰 <senlin.zsl@antgroup.com>