Skip to content

Commit

Permalink
[core][state][log] Cherry pick changes to prevent state API from read…
Browse files Browse the repository at this point in the history
…ing files outside the Ray log directory (#41520)

* [core][state][log] State API should not allow reading files outside of the ray log directory on all ray nodes. (#41467)

State API log retrieval has a security bug where one could pass:

relative paths like "../../../xxx" to get file outside of ray's log dir
absolute path that's refers to other files to get file somewhere else.
This PR fixes both issues such that one could only read logs under the ray logs directory.

---------

Signed-off-by: rickyyx <rickyx@anyscale.com>
Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>

* [core][state][log] Enable following symlinks that point outside of the `root_log_dir` when resolving paths (#41502)

Follow-up to: #41467. The change incidentally broke log retrieval on mac os because /tmp is a symlink to /private/tmp.

This PR avoids resolving the symlink until after we do the subdir check. This solves the mac os problem and generically enables file paths that contain symlinks outside of the root_log_dir.

---------

Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>

---------

Signed-off-by: rickyyx <rickyx@anyscale.com>
Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
Co-authored-by: Ricky Xu <xuchen727@hotmail.com>
  • Loading branch information
edoakes and rickyyx authored Nov 30, 2023
1 parent d97943e commit 7459639
Show file tree
Hide file tree
Showing 7 changed files with 143 additions and 12 deletions.
3 changes: 3 additions & 0 deletions dashboard/agent.py
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,9 @@ def publisher(self):
), "Accessing unsupported API (GcsAioPublisher) in a minimal ray."
return self.aio_publisher

def get_node_id(self) -> str:
return self.node_id

async def run(self):
# Start a grpc asyncio server.
if self.server:
Expand Down
55 changes: 46 additions & 9 deletions dashboard/modules/log/log_agent.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
import logging
from typing import Tuple
from typing import Optional, Tuple

import concurrent.futures
import ray.dashboard.modules.log.log_utils as log_utils
Expand Down Expand Up @@ -275,6 +275,10 @@ async def run(self, server):
if server:
reporter_pb2_grpc.add_LogServiceServicer_to_server(self, server)

@property
def node_id(self) -> Optional[str]:
return self._dashboard_agent.get_node_id()

@staticmethod
def is_minimal_module():
# Dashboard is only available with non-minimal install now.
Expand Down Expand Up @@ -370,6 +374,42 @@ async def _find_task_log_offsets(

return start_offset, end_offset

@classmethod
def _resolve_filename(cls, root_log_dir: Path, filename: str) -> Path:
"""
Resolves the file path relative to the root log directory.
Args:
root_log_dir: Root log directory.
filename: File path relative to the root log directory.
Raises:
FileNotFoundError: If the file path is invalid.
Returns:
The absolute file path resolved from the root log directory.
"""
if not Path(filename).is_absolute():
filepath = root_log_dir / filename
else:
filepath = Path(filename)

# We want to allow relative paths that include symlinks pointing outside of the
# `root_log_dir`, so use `os.path.abspath` instead of `Path.resolve()` because
# `os.path.abspath` does not resolve symlinks.
filepath = Path(os.path.abspath(filepath))

if not filepath.is_file():
raise FileNotFoundError(f"A file is not found at: {filepath}")

try:
filepath.relative_to(root_log_dir)
except ValueError as e:
raise FileNotFoundError(f"{filepath} not in {root_log_dir}: {e}")

# Fully resolve the path before returning (including following symlinks).
return filepath.resolve()

async def StreamLog(self, request, context):
"""
Streams the log in real time starting from `request.lines` number of lines from
Expand All @@ -384,15 +424,12 @@ async def StreamLog(self, request, context):
# be automatically terminated.
lines = request.lines if request.lines else 1000

if not Path(request.log_file_name).is_absolute():
filepath = Path(self._dashboard_agent.log_dir) / request.log_file_name
else:
filepath = Path(request.log_file_name)

if not filepath.is_file():
await context.send_initial_metadata(
[[log_consts.LOG_GRPC_ERROR, log_consts.FILE_NOT_FOUND]]
try:
filepath = self._resolve_filename(
Path(self._dashboard_agent.log_dir), request.log_file_name
)
except FileNotFoundError as e:
await context.send_initial_metadata([[log_consts.LOG_GRPC_ERROR, str(e)]])
else:
with open(filepath, "rb") as f:
await context.send_initial_metadata([])
Expand Down
1 change: 0 additions & 1 deletion dashboard/modules/log/log_consts.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@
}

LOG_GRPC_ERROR = "log_grpc_status"
FILE_NOT_FOUND = "LOG_GRPC_ERROR: file_not_found"

# 10 seconds
GRPC_TIMEOUT = 10
6 changes: 6 additions & 0 deletions python/ray/tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -1248,6 +1248,12 @@ def temp_file(request):
yield fp


@pytest.fixture(scope="function")
def temp_dir(request):
with tempfile.TemporaryDirectory("r+b") as d:
yield d


@pytest.fixture(scope="module")
def random_ascii_file(request):
import random
Expand Down
85 changes: 85 additions & 0 deletions python/ray/tests/test_state_api_log.py
Original file line number Diff line number Diff line change
Expand Up @@ -417,6 +417,56 @@ async def test_log_agent_find_task_log_offsets(temp_file):
assert start_offset == expected_start + len(exclude_tail_content)


def test_log_agent_resolve_filename(temp_dir):
"""
Test that LogAgentV1Grpc.resolve_filename(root, filename) works:
1. Not possible to resolve a file that doesn't exist.
2. Not able to resolve files outside of the temp dir root.
- with a absolute path.
- with a relative path recursive up.
3. Permits a file in a directory that's symlinked into the root dir.
"""
root = Path(temp_dir)
# Create a file in the temp dir.
file = root / "valid_file"
file.touch()
subdir = root / "subdir"
subdir.mkdir()

# Create a directory in the root that contains a valid file and
# is symlinked to by a path in the subdir.
symlinked_dir = root / "symlinked"
symlinked_dir.mkdir()
symlinked_file = symlinked_dir / "valid_file"
symlinked_file.touch()
symlinked_path_in_subdir = subdir / "symlink_to_outside_dir"
symlinked_path_in_subdir.symlink_to(symlinked_dir)

# Test file doesn't exist
with pytest.raises(FileNotFoundError):
LogAgentV1Grpc._resolve_filename(root, "non-exist-file")

# Test absolute path outside of root is not allowed
with pytest.raises(FileNotFoundError):
LogAgentV1Grpc._resolve_filename(subdir, root.resolve() / "valid_file")

# Test relative path recursive up is not allowed
with pytest.raises(FileNotFoundError):
LogAgentV1Grpc._resolve_filename(subdir, "../valid_file")

# Test relative path a valid file is allowed
assert (
LogAgentV1Grpc._resolve_filename(root, "valid_file")
== (root / "valid_file").resolve()
)

# Test relative path to a valid file following a symlink is allowed
assert (
LogAgentV1Grpc._resolve_filename(subdir, "symlink_to_outside_dir/valid_file")
== (root / "symlinked" / "valid_file").resolve()
)


# Unit Tests (LogsManager)


Expand Down Expand Up @@ -1066,6 +1116,41 @@ def verify():
wait_for_condition(verify)


def test_log_get_invalid_filenames(ray_start_with_dashboard, temp_file):
assert (
wait_until_server_available(ray_start_with_dashboard.address_info["webui_url"])
is True
)
webui_url = ray_start_with_dashboard.address_info["webui_url"]
webui_url = format_web_url(webui_url)
node_id = list_nodes()[0]["node_id"]

# log_dir = ray._private.worker.global_worker.node.get_logs_dir_path()

def verify():
# Kind of hack that we know the file node_ip_address.json exists in ray.
with pytest.raises(RayStateApiException) as e:
logs = "".join(get_log(node_id=node_id, filename="../node_ip_address.json"))
print(logs)
assert "does not start with " in str(e.value)
return True

wait_for_condition(verify)

# Verify that reading file outside of the log directory is not allowed
# with absolute path.
def verify():
# Kind of hack that we know the file node_ip_address.json exists in ray.
temp_file_abs_path = str(Path(temp_file.name).resolve())
with pytest.raises(RayStateApiException) as e:
logs = "".join(get_log(node_id=node_id, filename=temp_file_abs_path))
print(logs)
assert "does not start with " in str(e.value)
return True

wait_for_condition(verify)


def test_log_get_subdir(ray_start_with_dashboard):
assert (
wait_until_server_available(ray_start_with_dashboard.address_info["webui_url"])
Expand Down
1 change: 1 addition & 0 deletions python/ray/util/state/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -1355,6 +1355,7 @@ def list_logs(
r = requests.get(
f"{api_server_url}/api/v0/logs?{urllib.parse.urlencode(options_dict)}"
)
# TODO(rickyx): we could do better at error handling here.
r.raise_for_status()

response = r.json()
Expand Down
4 changes: 2 additions & 2 deletions python/ray/util/state/state_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -477,6 +477,6 @@ async def stream_log(
timeout=timeout,
)
metadata = await stream.initial_metadata()
if metadata.get(log_consts.LOG_GRPC_ERROR) == log_consts.FILE_NOT_FOUND:
raise ValueError(f'File "{log_file_name}" not found on node {node_id}')
if metadata.get(log_consts.LOG_GRPC_ERROR) is not None:
raise ValueError(metadata.get(log_consts.LOG_GRPC_ERROR))
return stream

0 comments on commit 7459639

Please sign in to comment.