From 0e3677288a64e58cf19429ce93169465ba1e2815 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Mon, 15 Jul 2024 09:42:33 -0700 Subject: [PATCH 1/2] Make working_dir support files created before 1980 Signed-off-by: Jiajun Yao --- python/ray/_private/runtime_env/packaging.py | 2 +- .../tests/test_runtime_env_working_dir_2.py | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index abf854315155..7e386dba8607 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -393,7 +393,7 @@ def _zip_directory( directory inside the zip file. """ pkg_file = Path(output_path).absolute() - with ZipFile(pkg_file, "w") as zip_handler: + with ZipFile(pkg_file, "w", strict_timestamps=False) as zip_handler: # Put all files in the directory into the zip file. dir_path = Path(directory).absolute() diff --git a/python/ray/tests/test_runtime_env_working_dir_2.py b/python/ray/tests/test_runtime_env_working_dir_2.py index b70f2536f552..e8777f63ec46 100644 --- a/python/ray/tests/test_runtime_env_working_dir_2.py +++ b/python/ray/tests/test_runtime_env_working_dir_2.py @@ -2,6 +2,7 @@ from pathlib import Path import sys import tempfile +import datetime import pytest from ray._private.test_utils import ( @@ -279,6 +280,31 @@ def g(): ray.get(refs) +def test_file_created_before_1980(shutdown_only, tmp_working_dir): + # Make sure working_dir supports file created before 1980 + # https://github.com/ray-project/ray/issues/46379 + working_path = Path(tmp_working_dir) + file_1970 = working_path / "1970" + with file_1970.open(mode="w") as f: + f.write("1970") + os.utime( + file_1970, + ( + datetime.datetime(1970, 1, 1, 10, 30).timestamp(), + datetime.datetime(1970, 1, 1, 10, 30).timestamp(), + ), + ) + + ray.init(runtime_env={"working_dir": tmp_working_dir}) + + @ray.remote + def task(): + with open("1970") as f: + assert f.read() == "1970" + + ray.get(task.remote()) + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From c682f180ef13c242b42a8baf69559fd122ba6879 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Tue, 16 Jul 2024 09:18:25 -0700 Subject: [PATCH 2/2] up Signed-off-by: Jiajun Yao --- python/ray/tests/test_runtime_env_working_dir_2.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/python/ray/tests/test_runtime_env_working_dir_2.py b/python/ray/tests/test_runtime_env_working_dir_2.py index e8777f63ec46..9fc3a9e70b4c 100644 --- a/python/ray/tests/test_runtime_env_working_dir_2.py +++ b/python/ray/tests/test_runtime_env_working_dir_2.py @@ -2,7 +2,6 @@ from pathlib import Path import sys import tempfile -import datetime import pytest from ray._private.test_utils import ( @@ -289,10 +288,7 @@ def test_file_created_before_1980(shutdown_only, tmp_working_dir): f.write("1970") os.utime( file_1970, - ( - datetime.datetime(1970, 1, 1, 10, 30).timestamp(), - datetime.datetime(1970, 1, 1, 10, 30).timestamp(), - ), + (0, 0), ) ray.init(runtime_env={"working_dir": tmp_working_dir})