This uses the generators introduced in #25247 to reduce memory usage during the merge stage in push-based shuffle. These tasks merge groups of map outputs, so it fits a generator pattern where we want to return merged outputs one at a time. Verified that this allows for merging more/larger objects at a time than the current list-based version.
I also tried this for the map stage in random_shuffle, but it didn't seem to make a difference in memory usage for Arrow blocks. I think this is probably because Arrow is already doing some zero-copy optimizations when selecting rows?
Also adds a new line to Dataset stats for memory usage. Unfortunately it's hard to get an accurate reading of physical memory usage in Python and this value will probably be an overestimate in a lot of cases. I didn't see a difference before and after this PR for the merge stage, for example. Arguably this field should be opt-in. For 100MB partitions, for example:
```
Substage 0 read->random_shuffle_map: 10/10 blocks executed
* Remote wall time: 1.44s min, 3.32s max, 2.57s mean, 25.74s total
* Remote cpu time: 1.42s min, 2.53s max, 2.03s mean, 20.25s total
* Worker memory usage (MB): 462 min, 864 max, 552 mean
* Output num rows: 12500000 min, 12500000 max, 12500000 mean, 125000000 total
* Output size bytes: 101562500 min, 101562500 max, 101562500 mean, 1015625000 total
* Tasks per node: 10 min, 10 max, 10 mean; 1 nodes used
Substage 1 random_shuffle_reduce: 10/10 blocks executed
* Remote wall time: 1.47s min, 2.94s max, 2.17s mean, 21.69s total
* Remote cpu time: 1.45s min, 1.88s max, 1.71s mean, 17.09s total
* Worker memory usage (MB): 462 min, 1047 max, 831 mean
* Output num rows: 12500000 min, 12500000 max, 12500000 mean, 125000000 total
* Output size bytes: 101562500 min, 101562500 max, 101562500 mean, 1015625000 total
* Tasks per node: 10 min, 10 max, 10 mean; 1 nodes used
```
## Checks
- [ ] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for https://docs.ray.io/en/master/.
- [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
- [ ] Unit tests
- [ ] Release tests
- [ ] This PR is not tested :(
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Followup from #24622. This is another step towards pluggability for runtime_env. Previously some plugin classes had `get_uri` which returned a single URI, while others had `get_uris` which returned a list. This PR makes all plugins use `get_uris`, which simplifies the code overall.
Most of the lines in the diff just come from the new `format.sh` which sorts the imports.
Fixes a bug in push-based shuffle in computing the merge task <> reduce tasks mapping when the number of reduce tasks does not divide evenly by the number of merge tasks. Previously, if there were N reduce tasks for one merge task, we would do:
[N + 1, N + 1, ..., N + 1, all left over tasks]
which could lead to negative many reduce tasks n the last merge partition.
This PR changes it to:
[N + 1, N + 1, ..., N + 1, N, N, N, ...]
Related issue number
Closes#25863.
This PR ensures that GCS keeps the IP and PID information about a job so that it can be used to find the job's logs in the dashboard after the job terminates.
@alanwguo will handle any dashboard work in a separate PR.
Co-authored-by: Alex Wu <alex@anyscale.com>
pyarrow.json.ReadOptions are not picklable until Arrow 8.0.0, which we do not yet support. This PR adds a custom serializer for this type and ensures that said serializer is registered before each Ray task submission.
Previously it was not possible to set the object spill config from the RAY_xxx environment variable, unlike other system configs. This is because the config is initialized in Python first, before the C++ config is parsed.
We often need to convert our internal `_TrackedCheckpoint` objects to `ray.air.Checkpoint`s - we should do this in a utility method in the `_TrackedCheckpoint` class.
This PR also fixes cases where we haven't resolved the saved checkpoint futures, yet.
This is the first step to enable the parallel tests for ray core ci. To reduce the noise this test only add the option and not enable them. Parallel CI can be 40%-60% faster compared with running them one-by-one.
We'll enable them by bk jobs one-by-one.
Prototype here #25612
## Why are these changes needed?
We currently only record usage stats from drivers. This can lose some of information when libraries are imported from workers (e.g., doing some rllib import from trainable).
@jjyao just for the future reference.
When GCS restarts, it'll recover the placement group and make sure no resource is leaking. The protocol now is like:
- Sending the committed PGs to raylets
- Raylets will check whether any worker is using resources from the PG not in this group
- If there is any, it'll kill that worker.
Right now there is a bug, which will kill the worker using bundle index equals -1.
Un-reverting https://github.com/ray-project/ray/pull/24934 which caused `test_cluster` to become flaky. This was due to an oversight: we need to update the `HTTPState` logic to account for the controller not necessarily running on the head node.
This will require using the new `SchedulingPolicy` API, but I'm not quite sure the best way to do it. Context here: https://github.com/ray-project/ray/issues/25090.
Followup PR to https://github.com/ray-project/ray/pull/20273.
- Hides cache logic behind a class.
- Adds "name" field to runtime env plugin class and makes existing conda, pip, working_dir, and py_modules inherit from the plugin class.
Future work will unify the codepath for these "base plugins" with the codepath for third-party plugins; currently these are different, and URI support is missing for third-party plugins.
This is a follow-up to the previous PR (GitHub did some funky things when I did a rebase, so I had to create a new one)
On Windows systems, the `exec_worker` method may fail due to spaces being present in arguments that are file paths. This addresses said issue.
Unfortunately, ray.data.read_parquet() doesn't work with multiple directories since it uses Arrow's Dataset abstraction under-the-hood, which doesn't accept multiple directories as a source: https://arrow.apache.org/docs/python/generated/pyarrow.dataset.dataset.html
This PR makes this clear in the docs, and as a driveby, adds ray.data.read_parquet_bulk() to the API docs.
Push-based shuffle has some extra metadata involving merge and reduce tasks. Previously we were serializing an O(n) (n = reduce tasks) metadata and sending this to tasks, which caused a lot of unnecessary plasma usage on the head node. This PR splits up the metadata into parts that can be kept on the driver and a relatively cheap part that is sent to all tasks.
Related issue number
One of the issues needed for #24480.
Adds a _transform_arrow method to Preprocessors that allows them to implement logic for arrow-based Datasets.
- If only _transform_arrow is implemented, will convert the data to arrow.
- If only _transform_pandas is implemented, will convert the data to pandas.
- If both are implemented, will pick the method corresponding to the format for best performance.
Implementation is defined as overriding the method in a sub-class.
This is only a change to the base Preprocessor class. Implementations for sub-classes will come in the future.
this is a temp fix of #25556. When the dtype from the pandas dataframe gives object, we set the dtype to be None and make use of the auto-inferring of the type in the conversion.