Bound object spilling file size to avoid disk increase pressure#60098
Bound object spilling file size to avoid disk increase pressure#60098edoakes merged 7 commits intoray-project:masterfrom
Conversation
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
There was a problem hiding this comment.
Code Review
This pull request introduces a valuable feature to restrict the maximum size of object spill files, addressing potential disk-full issues in large-scale scenarios. The implementation is clean, adding a new configuration max_spilling_file_size and correctly modifying the object spilling logic in TryToSpillObjects. The changes are well-documented and include a new test case that validates the core functionality. My only suggestion is to enhance the test coverage to include a scenario where a single object's size exceeds the new limit, to fully verify the intended behavior.
|
This pull request has been automatically marked as stale because it has not had You can always ask for help on our discussion forum or Ray's public slack channel. If you'd like to keep this open, just leave any comment, and the stale label will be removed. |
|
Thanks for another contribution @yancanmao and apologies for the delay reviewing. @Kunchd will review shortly |
Kunchd
left a comment
There was a problem hiding this comment.
Thanks for the contribution! The content is very helpful.
I left a couple of nits and questions, but we're almost there.
| break; | ||
| } | ||
|
|
||
| bytes_to_spill += object_size; |
There was a problem hiding this comment.
Out of curiosity, if you limit the amount of bytes that can be spilled each time we attempt to spill, would that result in the system being unable to spill fast enough resulting in significant amount of OOMs in plasma? Wondering if experiments were done for this scenario.
There was a problem hiding this comment.
Thanks for the question.
We haven’t run a full “OOM frequency vs. cap size” study yet. Capping spill bytes per attempt (via max_spilling_file_size) reduces spill fusion and can lower effective throughput due to higher per-file overhead.
In a small local benchmark (fixed object_store=75MiB, min_spilling_size=10MiB), a small cap (i.e., 10MB) led to more fragmented spill output and increased end-to-end ray.get / ray.put time by ~20% compared to the unlimited case.
This is an expected trade-off: smaller caps bound spill file sizes and smooth disk usage but may hurt throughput, while larger or unlimited caps favor efficiency. The default remains unlimited, with the cap provided as a tuning knob for users who need tighter control over spill I/O or disk usage.
There was a problem hiding this comment.
Thanks for the explanation! It make sense that the default is -1. I'm wondering if this should be documented in the ray_config_def or some other place, what do you think?
There was a problem hiding this comment.
In a small local benchmark (fixed object_store=75MiB, min_spilling_size=10MiB), a small cap (i.e., 10MB) led to more fragmented spill output and increased end-to-end ray.get / ray.put time by ~20% compared to the unlimited case.
We should dissect this a little bit more. It would be ideal to have it on by default at a reasonably tuned value. 20% is a lot of overhead for file creation.
There was a problem hiding this comment.
@yancanmao We believe that the tradeoff here might reveal an area that could deliver impactful improvement to the system.
Could you try the benchmark with a higher number of I/O workers (for example export RAY_max_io_workers=8) and see if the overhead drops. This will help identify whether or not the 20% drop in throughput is caused by artificial bottleneck within the system.
We really appreciate the work!
There was a problem hiding this comment.
@israbbani @Kunchd Thanks for the great suggestions!
I realized my previous benchmark was a bit too "quick and dirty" and represented an extreme edge case (very small cap). To address your concerns about the overhead and whether it's an artificial bottleneck, I re-ran a more rigorous benchmark with a fixed high-pressure workload (Writing 2GB data continuously).
Here are the results with max_io_workers=4:

As you can see in Groups B, C, and D (10MB - 50MB), the throughput is comparable to (or even slightly better than) the Baseline. The "20% overhead" I mentioned earlier only appears in Group E (5MB), where the system is forced to create hundreds of tiny files, causing syscall overheads to dominate. Interestingly, even in Group A (Unlimited), Ray generated small files (~8.8MB) instead of large chunks. This confirms that Ray's current batching (min_spilling_size) is "optimistic"—under memory pressure, it abandons batching to prevent OOM.
Since Ray naturally fragments files under pressure anyway (as seen in Baseline), setting a max_spilling_file_size doesn't necessarily hurt performance. Instead, it provides a predictable upper bound. This is crucial for disk space management—ensuring that we don't end up with massive "zombie files" that can't be deleted because a few small objects inside are still alive.
Here are the results with max_io_workers=8:

We can also see increasing number of io workers can increase the throughput effectively.
My benchmark code is as follows:
import ray
import numpy as np
import time
import os
import shutil
import gc
from tqdm import tqdm
# --- Global configuration ---
TOTAL_SIZE_GB = 2
OBJECT_SIZE_MB = 5
NUM_OBJECTS = (TOTAL_SIZE_GB * 1024) // OBJECT_SIZE_MB
SPILL_DIR_ROOT = "/tmp/ray_spill_experiment"
# Experiment configurations
EXPERIMENTS = [
{
"name": "A: Splitting 100",
"min_spill": 100 * 1024 * 1024,
"max_cap": 100 * 1024 * 1024,
"desc": "Accumulate 100MB -> Write 100MB file"
},
{
"name": "B: Splitting 50",
"min_spill": 50 * 1024 * 1024,
"max_cap": 50 * 1024 * 1024,
"desc": "Accumulate 50MB -> Write 50MB file"
},
{
"name": "C: Splitting 20",
"min_spill": 20 * 1024 * 1024,
"max_cap": 20 * 1024 * 1024,
"desc": "Accumulate 20MB -> Write 20MB file"
},
{
"name": "D: Splitting 10",
"min_spill": 10 * 1024 * 1024,
"max_cap": 10 * 1024 * 1024,
"desc": "Accumulate 10MB -> Write 10MB file"
},
{
"name": "E: Splitting 5",
"min_spill": 5 * 1024 * 1024,
"max_cap": 5 * 1024 * 1024,
"desc": "Accumulate 5MB -> Write 5MB file (More wakeups)"
}
]
def get_real_spill_dir(root_dir):
"""Return the actual spill directory under the given root.
Uses startswith("ray_spilled_objects") to match the per-node spill directory.
"""
for dirpath, dirnames, filenames in os.walk(root_dir):
for dirname in dirnames:
if dirname.startswith("ray_spilled_objects"):
return os.path.join(dirpath, dirname)
return None
def get_file_stats(directory):
if not directory or not os.path.exists(directory):
return 0, 0
total_size = 0
file_count = 0
for dirpath, dirnames, filenames in os.walk(directory):
for f in filenames:
fp = os.path.join(dirpath, f)
if not os.path.islink(fp):
total_size += os.path.getsize(fp)
file_count += 1
return file_count, total_size / 1024 / 1024
def run_experiment(config):
print(f"\n>>> Running: {config['name']} <<<")
print(f" ({config['desc']})")
# 1. Clean up the environment
if os.path.exists(SPILL_DIR_ROOT): shutil.rmtree(SPILL_DIR_ROOT)
os.makedirs(SPILL_DIR_ROOT, exist_ok=True)
if ray.is_initialized(): ray.shutdown()
# 2. Ray configuration
sys_config = {
"automatic_object_spilling_enabled": True,
"object_spilling_threshold": 0.8,
"min_spilling_size": config['min_spill'],
"max_spilling_file_size_bytes": config['max_cap'],
"max_io_workers": 8
}
# 3. Start Ray
mem_limit = 100 * 1024 * 1024
print(f" [System] ObjStore: {mem_limit//1024//1024} MB | MinSpill: {config['min_spill']//1024//1024} MB")
ray.init(
object_store_memory=mem_limit,
_system_config=sys_config,
_temp_dir=SPILL_DIR_ROOT,
logging_level="ERROR"
)
data_shape = (OBJECT_SIZE_MB * 1024 * 1024 // 8, )
refs = []
# Reuse a shared payload object to reduce Python GC noise
payload_ref = ray.put(np.zeros(data_shape, dtype=np.int64))
payload = ray.get(payload_ref)
# --- Phase 1: Write Stress ---
start_time = time.time()
# Put as fast as possible
for _ in tqdm(range(NUM_OBJECTS), desc=" Writing", leave=False):
ref = ray.put(payload)
refs.append(ref)
# Important: give Ray enough time to finish the last async flush.
# This matters more when there are many small spill files and the IO queue is long.
time.sleep(5)
end_time = time.time()
write_duration = end_time - start_time
write_bw = (TOTAL_SIZE_GB * 1024) / write_duration
# --- Spill file stats ---
real_dir = get_real_spill_dir(SPILL_DIR_ROOT)
print(f" [Debug] Found Spill Dir: {real_dir}") # Print to confirm the directory was found
file_count, disk_usage = get_file_stats(real_dir)
avg_file_size = disk_usage / file_count if file_count > 0 else 0
print(f" [Result] Throughput: {write_bw:.2f} MB/s | Files: {file_count} | AvgSize: {avg_file_size:.2f} MB")
# --- Phase 2: Cleanup ---
del refs
gc.collect()
time.sleep(2)
ray.shutdown()
return {
"Group": config['name'],
"Write Speed (MB/s)": write_bw,
"File Count": file_count,
"Avg File Size (MB)": avg_file_size
}
if __name__ == "__main__":
results = []
for exp_config in EXPERIMENTS:
try:
res = run_experiment(exp_config)
results.append(res)
except Exception as e:
print(f"Experiment failed: {e}")
import traceback
traceback.print_exc()
print("\n" + "="*90)
print(f"{'Group':<30} | {'Speed (MB/s)':<15} | {'Files':<10} | {'Avg Size (MB)':<15}")
print("-" * 90)
for res in results:
print(f"{res['Group']:<30} | {res['Write Speed (MB/s)']:<15.2f} | {res['File Count']:<10} | {res['Avg File Size (MB)']:<15.2f}")
print("="*90)
There was a problem hiding this comment.
Thanks for the in depth analysis! I think this is an indicator that there exists a potential artificial bottleneck in how the system spills right now that can be a project for improvement.
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
|
Thanks @Kunchd for reviewing this PR! I have revised the pr according to your comments. |
Kunchd
left a comment
There was a problem hiding this comment.
Thanks for addressing all my previous comments! I've left two quick nits, but we should be close to done.
| break; | ||
| } | ||
|
|
||
| bytes_to_spill += object_size; |
There was a problem hiding this comment.
Thanks for the explanation! It make sense that the default is -1. I'm wondering if this should be documented in the ray_config_def or some other place, what do you think?
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
|
Thanks @Kunchd for the comment! I have done the nit update according to the comment. |
Kunchd
left a comment
There was a problem hiding this comment.
Looks good! Thanks for the contribution!
|
I'll will approve the PR once we pass pre-merge tests. You may want to update the branch to the latest version of master. |
|
@yancanmao this is an interesting proposal. I think your approach makes sense. I wonder if we can take it further still. If I summarize the problem statement as: Infinite spilling has 20% (which may not be accurate because of artificial bottlenecks) higher throughput than a small spill file. However, infinite spilling will run out of disk. Then, I'm inclined to believe that compaction of spill files might be a good compromise. Especially if you have SSD throughput available and CPU (worker) cores stalling because of a lack of available memory. |
src/ray/common/ray_config_def.h
Outdated
| /// due to higher per-file overhead. If spilling cannot keep up with allocation under | ||
| /// memory pressure, this may increase the likelihood of object store OOMs. | ||
| /// Set to -1 to disable this limit. | ||
| RAY_CONFIG(int64_t, max_spilling_file_size, -1) |
There was a problem hiding this comment.
Can you please add the unit to the name of the config? E.g. max_spilling_file_size_bytes. It makes it self-documenting for readers. I know min_spilling_size doesn't follow this, but that's not a good thing.
| RAY_LOG(DEBUG) << absl::StrFormat( | ||
| "Choosing objects to spill with minimum total size %lld, max fused file size %s " | ||
| "or with total # of objects = %lld", | ||
| static_cast<long long>(min_spilling_size_), | ||
| max_spilling_file_size_str, | ||
| static_cast<long long>(max_fused_object_count_)); |
There was a problem hiding this comment.
Since you're improving the code, can you move this to the ctor of LocalObjectManager? It's good to log static configuration once on startup instead of every time we try to spill.
|
|
||
| /// Maximum bytes to include in a single spill request (i.e. fused spill file). | ||
| /// If <= 0, the limit is disabled. | ||
| int64_t max_spilling_file_size_; |
There was a problem hiding this comment.
Same comment as above. Please add the unit to the variable name.
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
Signed-off-by: Mao Yancan <yancan.mao@bytedance.com>
|
Thanks @Kunchd @israbbani for your insightful comments and discussion! This helped me understand deeper on the spilling mechanisms. I have updated the code accordingly. |
Kunchd
left a comment
There was a problem hiding this comment.
@yancanmao Thank you for the contribution, and we would love to collaborate on further improving the object spilling logic!
From our discussions and your investigation, it's clear that there are many opportunities for improving the object spilling logic. We will be discussing more on a comprehensive effort to improve object spilling offline, but if you're interested, the most immediate action item would be to investigate the existing implementation of IO workers (e.g. how it performs the write to disk and how many workers are kept around) and whether that's causing an artificial bottleneck. Compaction, like you've mentioned, would be an optimization further down the line.
If you are on the ray slack, we can also loop you in on any discussions we have regarding object spilling improvements there.
|
Thanks @Kunchd @israbbani! I'll review the current IO worker implementation and run some isolated benchmarks on file read/write performance as suggested. I would definitely love to join the discussion on Slack. I'll ping you there to follow up! |
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: tiennguyentony <46289799+tiennguyentony@users.noreply.github.com>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: tiennguyentony <46289799+tiennguyentony@users.noreply.github.com>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com>
### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes #60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes #60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: Adel Nour <ans9868@nyu.edu>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: peterxcli <peterxcli@gmail.com>
…project#60098) ### Problem Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by `min_spilling_size_` and `max_fused_object_count_`, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count. In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource. Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures. While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead. ### Proposal We propose to add an optional upper bound on fused spill file size. - Introduce `max_spilling_file_size` (bytes, default disabled with value `-1`). - When forming a spill batch, stop adding objects once the accumulated size would exceed the size cap. - Always allow spilling at least one object, even if it exceeds the cap. This complements `max_fused_object_count_` by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation. ## Related issues Closes ray-project#60097 ## Additional information - added a new config `max_spilling_file_size`, default to `-1` --------- Signed-off-by: Mao Yancan <yancan.mao@bytedance.com> Co-authored-by: Mao Yancan <yancan.mao@bytedance.com> Signed-off-by: peterxcli <peterxcli@gmail.com>
Description
Problem
Ray object spilling fuses multiple objects into a single spill file. Fusion is bounded only by
min_spilling_size_andmax_fused_object_count_, not by total spill file size. In practice, individual objects can be very large, so a fused spill file can still grow to many GBs even with a small object count.In large-scale training setups where memory and local disk are comparable in size (e.g., ~2 TB RAM / ~2 TB local disk per node), sustained memory pressure triggers frequent spilling and makes disk a constrained resource.
Spill file reclamation is file-granular and driven by object reference counts. When large objects are fused together, a spill file is often referenced by multiple long-lived objects. As a result, the file cannot be reclaimed until all references are released. Under sustained spilling, new large spill files keep being created while old ones remain pinned, causing disk usage to monotonically increase and eventually leading to disk-full failures.
While a strict one-object-per-file approach would avoid this retention coupling, it is not practical due to excessive file counts and I/O overhead.
Proposal
We propose to add an optional upper bound on fused spill file size.
max_spilling_file_size(bytes, default disabled with value-1).This complements
max_fused_object_count_by bounding spill files in terms of bytes, not just object count. Large objects can naturally spill into smaller or single-object files, allowing more flexible and timely disk reclamation.Related issues
Closes #60097
Additional information
max_spilling_file_size, default to-1