Skip to content

Commit

Permalink
separate out out of disk space test
Browse files Browse the repository at this point in the history
Signed-off-by: Lonnie Liu <[email protected]>
  • Loading branch information
aslonnie committed Jan 9, 2024
1 parent b87ed2c commit f0bc974
Show file tree
Hide file tree
Showing 8 changed files with 66 additions and 4 deletions.
25 changes: 23 additions & 2 deletions .buildkite/core.rayci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ steps:
commands:
- bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core
--workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3
--except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,container,manual
--except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual

- label: ":ray: core: redis tests"
tags: python
Expand All @@ -41,7 +41,7 @@ steps:
- bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core
--workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3
--test-env=TEST_EXTERNAL_REDIS=1
--except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,container,manual
--except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual

- label: ":ray: core: :windows: python tests"
tags: python
Expand Down Expand Up @@ -75,6 +75,27 @@ steps:
--test_tag_filters=mem_pressure -- //python/ray/tests/...
job_env: corebuild

- label: ":ray: core: out of disk tests"
tags:
- python
- oss
instance_type: small
commands:
- bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core
--only-tags=tmpfs --tmp-filesystem=tmpfs
job_env: corebuild

- label: ":ray: core: out of disk redis tests"
tags:
- python
- oss
instance_type: small
commands:
- bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core
--test-env=TEST_EXTERNAL_REDIS=1
--only-tags=tmpfs --tmp-filesystem=tmpfs
job_env: corebuild

- label: ":ray: core: workflow tests"
tags:
- python
Expand Down
11 changes: 11 additions & 0 deletions ci/ray_ci/linux_container.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,16 @@ def __init__(
docker_tag: str,
volumes: Optional[List[str]] = None,
envs: Optional[List[str]] = None,
tmp_filesystem: Optional[str] = None,
) -> None:
super().__init__(docker_tag, envs)
self.volumes = volumes or []

if tmp_filesystem is not None:
if tmp_filesystem != "tmpfs":
raise ValueError("Only tmpfs is supported for tmp filesystem")
self.tmp_filesystem = tmp_filesystem

def install_ray(self, build_type: Optional[str] = None) -> List[str]:
env = os.environ.copy()
env["DOCKER_BUILDKIT"] = "1"
Expand Down Expand Up @@ -58,6 +64,11 @@ def get_run_command_extra_args(
"--add-host",
"rayci.localhost:host-gateway",
]
if self.tmp_filesystem:
extra_args += [
"--mount",
f"type={self.tmp_filesystem},destination=/tmp",
]
for volume in self.volumes:
extra_args += ["--volume", volume]
for cap in _DOCKER_CAP_ADD:
Expand Down
2 changes: 2 additions & 0 deletions ci/ray_ci/linux_tester_container.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ def __init__(
shard_ids: Optional[List[int]] = None,
skip_ray_installation: bool = False,
build_type: Optional[str] = None,
tmp_filesystem: Optional[str] = None,
) -> None:
LinuxContainer.__init__(
self,
Expand All @@ -25,6 +26,7 @@ def __init__(
f"{os.environ.get('RAYCI_CHECKOUT_DIR')}:/ray-mount",
"/var/run/docker.sock:/var/run/docker.sock",
],
tmp_filesystem=tmp_filesystem,
)
TesterContainer.__init__(
self,
Expand Down
6 changes: 6 additions & 0 deletions ci/ray_ci/test_linux_container.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,5 +18,11 @@ def test_get_run_command() -> None:
assert "/bin/bash -iecuo pipefail -- hi\nhello" in command


def test_get_run_command_tmpfs() -> None:
container = LinuxContainer("test", tmp_filesystem="tmpfs")
command = " ".join(container.get_run_command(["hi", "hello"]))
assert "--mount type=tmpfs,destination=/tmp" in command


if __name__ == "__main__":
sys.exit(pytest.main(["-v", __file__]))
1 change: 1 addition & 0 deletions ci/ray_ci/test_tester.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ def test_get_container() -> None:
parallelism_per_worker=2,
network=None,
gpus=0,
tmp_filesystem=None,
)
assert isinstance(container, LinuxTesterContainer)
assert container.docker_tag == "corebuild"
Expand Down
11 changes: 10 additions & 1 deletion ci/ray_ci/tester.py
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,11 @@
type=click.Choice(["linux", "windows"]),
help=("Operating system to run tests on"),
)
@click.option(
"--tmp-filesystem",
type=str,
help=("Filesystem to use for /tmp"),
)
def main(
targets: List[str],
team: str,
Expand All @@ -165,6 +170,7 @@ def main(
test_arg: Optional[str],
build_name: Optional[str],
build_type: Optional[str],
tmp_filesystem: Optional[str],
) -> None:
if not bazel_workspace_dir:
raise Exception("Please use `bazelisk run //ci/ray_ci`")
Expand All @@ -182,7 +188,8 @@ def main(
worker_id,
parallelism_per_worker,
gpus,
network,
network=network,
tmp_filesystem=tmp_filesystem,
test_env=list(test_env),
build_name=build_name,
build_type=build_type,
Expand Down Expand Up @@ -218,6 +225,7 @@ def _get_container(
parallelism_per_worker: int,
gpus: int,
network: Optional[str],
tmp_filesystem: Optional[str] = None,
test_env: Optional[List[str]] = None,
build_name: Optional[str] = None,
build_type: Optional[str] = None,
Expand All @@ -237,6 +245,7 @@ def _get_container(
network=network,
skip_ray_installation=skip_ray_installation,
build_type=build_type,
tmp_filesystem=tmp_filesystem,
)

if operating_system == "windows":
Expand Down
10 changes: 9 additions & 1 deletion python/ray/tests/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -465,8 +465,16 @@ py_test(

py_test_module_list(
files = [
"test_tqdm.py",
"test_out_of_disk_space.py",
],
size = "large",
tags = ["exclusive", "tmpfs", "team:core"],
deps = ["//:ray_lib", ":conftest"],
)

py_test_module_list(
files = [
"test_tqdm.py",
"test_failure_4.py",
"test_iter.py",
"test_object_spilling.py",
Expand Down
4 changes: 4 additions & 0 deletions python/ray/tests/test_out_of_disk_space.py
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,10 @@ def foo():
except ray.exceptions.RayTaskError as e:
assert isinstance(e.cause, ray.exceptions.OutOfDiskError)

# Give it some time for events to appear.
# TODO(core-team): provide some way to wait for events to be flushed.
time.sleep(2)

events = list_cluster_events()
print(events)
# There could be more than 1 event depending on the test timing.
Expand Down

0 comments on commit f0bc974

Please sign in to comment.