Skip to content

[Bug] Objects not being spilled in 100MB bundles #22485

@frank-lsf

Description

@frank-lsf

Search before asking

  • I searched the issues and found no similar issues.

Ray Component

Ray Core

What happened + What you expected to happen

Ray setup: 1GB object store memory.

We put 16000 of 1MB objects into the object store using ray.put(), causing it to spill most of them to disk, then schedule tasks that require these objects so that they get restored into memory.

Default RAY_min_spilling_size = 100 * 1024 * 1024.

Expected behavior: close to 16GB objects get spilled out, in 100MB chunks.

Reality: only the first 8 spills were over 100MB; all the subsequent spill requests are just 1--2MB. In the end, there were ~14000 spill requests, as opposed to ~160 as expected.

In raylet.out:

[2022-02-17 23:40:43,668 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,678 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,690 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,701 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,712 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,723 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,734 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,745 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 105026250 num objects 105
[2022-02-17 23:40:43,756 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 26006500 num objects 26
[2022-02-17 23:40:43,767 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,778 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,789 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,801 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,812 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,823 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,834 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,845 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,856 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,867 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,878 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,889 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1
[2022-02-17 23:40:43,900 D 753034 753034] (raylet) local_object_manager.cc:191: Spilling objects of total size 1000250 num objects 1

Versions / Dependencies

1.10.0 and master

Reproduction script

def get_args(*args, **kwargs):
    parser = argparse.ArgumentParser()
    parser.add_argument(
        "--total_data_size",
        default=16_000_000_000,
        type=int,
    )
    parser.add_argument(
        "--num_objects",
        default=16000,  # 1MB
        # default=16000 * 10,  # 100KB
        # default=16000 * 20,  # 50KB
        type=int,
    )
    parser.add_argument(
        "--num_objects_per_task",
        default=200,
        type=int,
    )
    parser.add_argument(
        "--object_store_memory",
        default=1 * 1024 * 1024 * 1024,
        type=int,
    )
    parser.add_argument(
        "--task_parallelism",
        default=1,
        type=int,
    )
    parser.add_argument(
        "--no_fusing",
        default=False,
        action="store_true",
    )
    parser.add_argument(
        "--no_prefetching",
        default=False,
        action="store_true",
    )
    args = parser.parse_args(args, **kwargs)
    args.object_size = args.total_data_size // args.num_objects
    args.num_tasks = args.num_objects // args.num_objects_per_task
    assert args.object_size * args.num_objects_per_task < args.object_store_memory, args
    return args

args = get_args()

@ray.remote
def consume(*xs):
    time.sleep(1)
    return sum(x.size for x in xs)

def consume_all(args, refs):
    tasks = [
        consume.remote(
            *refs[t * args.num_objects_per_task : (t + 1) * args.num_objects_per_task]
        )
        for t in range(args.num_tasks)
    ]
    with tqdm.tqdm(total=len(tasks)) as pbar:
        not_ready = tasks
        while not_ready:
            _, not_ready = ray.wait(not_ready, fetch_local=False)
            pbar.update(1)
    print(ray.get(tasks))


def produce_all(args):
    refs = []
    for i in tqdm.tqdm(range(args.num_objects)):
        obj = np.full(args.object_size, i % 256, dtype=np.uint8)
        refs.append(ray.put(obj))
    return refs

def microbenchmark(args):
    logging.info("Produce")
    refs = produce_all(args)

    logging.info("Dropping filesystem cache")
    subprocess.run("sudo bash -c 'sync; echo 3 > /proc/sys/vm/drop_caches'", shell=True)

    logging.info("Consume")
    consume_one_by_one(args, refs)

Anything else

The object store should always be under pressure since we are putting 16GB objects into a 1GB object store, so the object manager shouldn't have trouble finding eligible objects to spill. But why is it only spilling 1--2 objects at a time?

cc @stephanie-wang @rkooo567 @scv119 @ericl

Are you willing to submit a PR?

  • Yes I am willing to submit a PR!

Metadata

Metadata

Labels

P1Issue that should be fixed within a few weeksbugSomething that is supposed to be working; but isn't

Type

No type

Projects

No projects

Milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions