To prepare for additional changes in pubsub to fix#22339 and #22340,
- Use structs instead of std::pair to hold per-subscription data, in case we need to expand the data fields.
- Rename variables in tests to indicate non-object pubsub testing.
- Pass full request to long poll handler in Publisher.
- Simplify logic when possible.
There should be no behavior change. Most of the code changes are based on #20276
Enables lineage reconstruction, which allows automatic recovery of task outputs, by default.
Also adds an info message to the driver whenever objects need to be reconstructed (not including recursive reconstruction).
When there is no scheduling task of scheduling class in local raylet, the backlog resource will not be reported. It usually will happen when core worker try to schedule the task on other node and report backlog to local node.
This will lead to the wrong demands.
This PR move the poller and broadcaster from gcs server to ray syncer.
TODO in next PR: deprecate the code path of placement group resource reporting and move the broadcaster out of gcs cluster resource manager.
Don't re-use task workers for actors, since those workers may own objects that will be lost on actor exit.
This adds a slight performance penalty for actor startup.
This PR fixes some minor bugs in `to_dict` and `from_dict` for the runtime env protobuf and adds a test to cover this codepath. The test checks that `to_dict` and `from_dict` are inverses. This PR contains all fixes required to make the test pass.
This PR supports setting actor level env vars for Java worker in runtime env.
General API looks like:
```java
RuntimeEnv runtimeEnv = new RuntimeEnv.Builder()
.addEnvVar("KEY1", "A")
.addEnvVar("KEY2", "B")
.addEnvVar("KEY1", "C") // This overwrites "KEY1" to "C"
.build();
ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
```
If `num-java-workers-per-process` > 1, it will never reuse the worker process except they have the same runtime envs.
Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
To hidden symbols of thirdparty library, this pull request reuses internal namespace that can be imported by any external native projects without side effects.
Besides, we suggest all of contributors to make sure it'd better use thirdparty library in ray scopes/namspaces and only ray::internal should be exported.
More details in https://github.com/ray-project/ray/pull/22526
Mobius has applied this change in https://github.com/ray-project/mobius/pull/28.
Co-authored-by: 林濯 <lingxuan.zlx@antgroup.com>
Whenever we spill, we try to spill all spillable objects. We also try to fuse small objects together to reduce total IOPS. If there aren't enough objects in the object store to meet the fusion threshold, we spill the objects anyway to avoid liveness issues.
However, the current logic always spills once we reach the end of the spillable objects or once we've reached the fusion threshold. This can produce lots of unfused objects if they are created concurrently with the spill.
This PR changes the spill logic: once we reach the end of the spillable objects, if the last batch of spilled objects is under the fusion threshold, we'll only spill it if we don't have other spills pending too. This gives the pending spills time to finish, and then we can re-evaluate whether it's necessary to spill the remaining objects. Liveness is also preserved.
Currently object transfers assume that the object size is fixed. This is a bad assumption during failures, especially with lineage reconstruction enabled and tasks with nondeterministic outputs.
This PR cleans up the handling and hopefully guards against two cases where the object size may change during a transfer:
1. The object manager's size information does not match the object in the local plasma store (due to async notifications). --> the object manager overwrites its own information if it finds that the physical object has a different size.
2. The receiver's created buffer size does not match the sender's object size. --> the receiver destroys the previous buffer and creates a new buffer with the correct size. This might cause some transient errors but eventually object transfer should succeed.
Unfortunately I couldn't trigger this from Python because it depends on some pretty specific timing conditions. However, I did add some unit tests for case 2 (this is the majority of the PR).
When get a bunch of data from redis, we first initialize local variables and then put them in vector, which bring so much copies from stack to heap or from local variables to vector.
This tiny little change would save the copies.
According to the description of #21218 , in this PR, we support the ability specifying a frontend-defined in-memory object allocator. So that we can specify an allocator to allocate the buffers from JVM heap. This is the basic functionality for the next PR #21441 that the JVM is able to be aware of the memory pressure of the in-memeory store objects.
Note that, if we use a frontend defined allocator, it may break the zerocopy ability. In Java, JVM buffers is in heap and we should copy it to native memory if needed.
Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
Why are these changes needed?
Data from PutRequests is chunked into 64MiB messages over the datastream, to avoid the 2GiB message size limit from gRPC. This will allow users to transfer objects larger than 2GiB over the network.
Proto changes
Put requests now have fields for chunk_id to identify which chunk data belongs to, total_chunks to identify the total number of chunks in the object, and total_size for total size in bytes of the object (useful for raising warnings).
PutObject is still unary-unary. The dataservicer handles reassembling the chunks before passing the result to the underlying RayletServicer.
Dataclient changes
If a put request is inserted into the request queue, self._requests will chunk it lazily. Doing this lazily is important since inserting all of the chunks onto the request queue immediately would double the amount of memory needed to handle a large request. This also guarantees that the chunks of a given putrequest will be contiguous
Dataservicer changes
The dataservicer now maintains some state to track received chunks. Once all chunks for a putrequest are received, the combined chunks are passed to the raylet servicer.
Opencenus symobls haven been exported in linux version of libcore_worker_library_java.so, but deleted from ray_exported_symbols.lds, which makes streaming macos test case failed.
This pr add this exporting record and rename *ray*streaming* stuff to *ray*internal* that's a united entry to ray cpp.
Co-authored-by: 林濯 <lingxuan.zlx@antgroup.com>
Improve observability for general objects and lineage reconstruction by adding a "Status" field to `ray memory`. The value of the field can be:
```
// The task is waiting for its dependencies to be created.
WAITING_FOR_DEPENDENCIES = 1;
// All dependencies have been created and the task is scheduled to execute.
SCHEDULED = 2;
// The task finished successfully.
FINISHED = 3;
```
In addition, tasks that failed or that needed to be re-executed due to lineage reconstruction will have a field listing the attempt number. Example output:
```
IP Address | PID | Type | Call Site | Status | Size | Reference Type | Object Ref
192.168.4.22 | 279475 | Driver | (task call) ... | Attempt #2: FINISHED | 10000254.0 B | LOCAL_REFERENCE | c2668a65bda616c1ffffffffffffffffffffffff0100000001000000
```
Runtime Environments is already GA in Ray 1.6.0. The latest doc is [here](https://docs.ray.io/en/master/ray-core/handling-dependencies.html#runtime-environments). And now, we already supported a [inheritance](https://docs.ray.io/en/master/ray-core/handling-dependencies.html#inheritance) behavior as follows (copied from the doc):
- The runtime_env["env_vars"] field will be merged with the runtime_env["env_vars"] field of the parent. This allows for environment variables set in the parent’s runtime environment to be automatically propagated to the child, even if new environment variables are set in the child’s runtime environment.
- Every other field in the runtime_env will be overridden by the child, not merged. For example, if runtime_env["py_modules"] is specified, it will replace the runtime_env["py_modules"] field of the parent.
We think this runtime env merging logic is so complex and confusing to users because users can't know the final runtime env before the jobs are run.
Current PR tries to do a refactor and change the behavior of Runtime Environments inheritance. Here is the new behavior:
- **If there is no runtime env option when we create actor, inherit the parent runtime env.**
- **Otherwise, use the optional runtime env directly and don't do the merging.**
Add a new API named `ray.runtime_env.get_current_runtime_env()` to get the parent runtime env and modify this dict by yourself. Like:
```Actor.options(runtime_env=ray.runtime_env.get_current_runtime_env().update({"X": "Y"}))```
This new API also can be used in ray client.
- Separate spread scheduling and default hydra scheduling (i.e. SpreadScheduling != HybridScheduling(threshold=0)): they are already separated in the API layer and they have the different end goals so it makes sense to separate their implementations and evolve them independently.
- Simple round robin for spread scheduling: this is just a starting implementation, can be optimized later.
- Prefer not to spill back tasks that are waiting for args since the pull is already in progress.
This flag has been turned on by default for almost 4 months. Delete the old code so that when refactoring, we don't need to take care of the legacy code path.
In the implementation of `GcsResourceManager::UpdateResourceCapacity`, 'cluster_scheduling_resources_' is modified, but this method is only used in c++ unit test, it is easy to cause confuse when reading the code. Since this method can be completely replaced by `GcsResourceManager::OnNodeAdd`, just remove it.
Co-authored-by: 黑驰 <senlin.zsl@antgroup.com>
* fix null pointer crash when GcsResourceManager::SetAvailableResources
* add warning log when node does not exist
* add unit test
Co-authored-by: 黑驰 <senlin.zsl@antgroup.com>
This is a working in progress PR that splits cluster_task_manager into local and distributed parts.
For the distributed scheduler (cluster_task_manager_:
/// Schedules a task onto one node of the cluster. The logic is as follows:
/// 1. Queue tasks for scheduling.
/// 2. Pick a node on the cluster which has the available resources to run a
/// task.
/// * Step 2 should occur any time the state of the cluster is
/// changed, or a new task is queued.
/// 3. For tasks that's infeasible, put them into infeasible queue and reports
/// it to gcs, where the auto scaler will be notified and start new node
/// to accommodate the requirement.
For the local task manager:
/// It Manages the lifetime of a task on the local node. It receives request from
/// cluster_task_manager (the distributed scheduler) and does the following
/// steps:
/// 1. Pulling task dependencies, add the task into to_dispatch queue.
/// 2. Once task's dependencies are all pulled locally, the task becomes ready
/// to dispatch.
/// 3. For all tasks that are dispatch-ready, we schedule them by acquiring
/// local resources (including pinning the objects in memory and deduct
/// cpu/gpu and other resources from local resource manager), and start
/// a worker.
/// 4. If task failed to acquire resources in step 3, we will try to
/// spill it to a different remote node.
/// 5. When a worker finishes executing its task(s), the requester will return
/// it and we should release the resources in our view of the node's state.
/// 6. If a task has been waiting for arguments for too long, it will also be
/// spilled back to a different node.
///
Why are these changes needed?
Switches GetObject from unary-unary to unary-streaming so that large objects can be streamed across multiple messages (currently hardcoded to 64MiB chunks). This will allow users to retrieve objects larger than 2GiB from a remote cluster. If the transfer is interrupted by a recoverable gRPC error (i.e. temporary disconnect), then the request will be retried starting from the first chunk that hasn't been received yet.
Proto changes
GetRequest's now have the field start_chunk_id, to indicate which chunk to start from (useful if the we have to retry a request after already receiving some chunks). GetResponses now have a chunk_id (0 indexed chunk of the serialized object), total_chunks (total number of chunks, used in async transfers to determine when all chunks have been received), and total_size (the total size of the object in bytes, used to raise user warnings if the object being retrieved is very large).
Server changes
Mainly just updating GetObject logic to yield chunks instead of returning
Client changes
At the moment, objects can be retrieved directly from the raylet servicer (ray.get) or asynchronously over the datapath (await some_remote_func.remote()). In both cases, the request will error if the chunk isn't valid (server side error) or if a chunk is received out of order (shouldn't happen in practice, since gRPC guarantees that messages in a stream either arrive in order or not at all).
ray.get is fairly straightforward, and changes are mainly to accommodate yielding from the stub instead of taking the value directly.
await some_remote_func.remote() is similar, but to keep things consistent with other async handling collecting the chunks is handled by a ChunkCollector, which wraps around the original callback.