ray-project / ray

Ray is a unified framework for scaling AI and Python applications. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
33.09k stars 5.6k forks source link

[core][Bug] Ray processes escaping hermetic python environment #22977

Closed ponner-github closed 1 year ago

ponner-github commented 2 years ago

Search before asking

Ray Component

Ray Core

Issue Severity

Medium: It contributes to significant difficulty to complete my task but I work arounds and get it resolved.

What happened + What you expected to happen

ray.init() the (grand)child python processes are escaping our hermetic python environment (specifically they start to look for modules on the system, instead of our bazel build sandbox)

This leads to Actor failures

(TemporaryActor pid=550)     import google.auth
(TemporaryActor pid=550) ModuleNotFoundError: No module named 'google.auth'
(TemporaryActor pid=544) 2022-03-09 23:15:03,378    ERROR worker.py:431 -- Exception raised in creation task: The actor died because of an error raised in its creation task, ray::Trainer.__init__() (pid=544, ip=xx.xxx.xxx.xx)
(TemporaryActor pid=544) RuntimeError: The actor with name Trainer failed to import on the worker. This may be because needed library dependencies are not installed in the worker environment:

Looking at python/ray/_private/services.py most python subprocesses are started as {sys.executable} <some script> without the -Ss flags that would prevent extending the module search path into site-packages

Please let us know if there are any workarounds that can be applied to deal with this or code references that show this should not be happening.

Versions / Dependencies

ray==1.9.1 python==3.7(via bazel)

Reproduction script

Working on a shareable repro...

Anything else

No response

Are you willing to submit a PR?

ponner-github commented 2 years ago

Also can confirm that the issue gets fixed by passing -Ss flags in our bazel environment Included the flag in start_worker_cmd, agent_command, java_command and start_ray_client_server in a forked version of ray for testing.

Will see if i can repro the issue in a shareable example

architkulkarni commented 2 years ago

Would deleting the PYTHONPATH environment variable on the worker processes solve this? This could be achieved by using runtime environments with runtime_env = {"env_vars": {"PYTHONPATH": ""}}

ponner-github commented 2 years ago

Thanks @architkulkarni , will try passing in the runtime environment args and confirm

architkulkarni commented 2 years ago

Sounds good, thanks! I'm not too familiar with bazel environments, but it might be appropriate to pass in a certain directory for PYTHONPATH instead of deleting it altogether. Looking forward to hearing the results.

ponner-github commented 2 years ago

Naively setting PYTHONPATH to the bazel python path where imports are present or an empty path fails with

    raise value
ray.exceptions.RuntimeEnvSetupError: The runtime_env failed to be set up.

I will continue checking other env variables that could possibly impact this behaviour.

The easiest solution from our perspective would be an additional param in ray.init that would toggle the -Ss flag in python/ray/_private/services.py where python subprocesses are started.

Something like ray.init(ignore_site_packages=True)

Do you think adding that is a feasible option at your end ?

architkulkarni commented 2 years ago

@ponner-github thanks for trying it out! I'd like to figure out why the runtime_env failed to be set up. Do you mind sharing your code that defines and passes in the runtime_env? Also, could you see if there are relevant logs in dashboard_agent.py or runtime_env_setup.py? These are stored at /tmp/ray/session_latest/logs on the head node by default. (These runtime env logs are automatically propagated to the driver in newer Ray versions, sorry about the bad experience here.)

Regarding the possibility of ignore_site_packages=True change, that would require an API proposal and approval. We'd have to balance the benefit of the adding this against keeping the Ray API simple, since we don't want to add too many arguments to ray.init. I don't have a good sense of whether or not the API proposal would be approved, @stephanie-wang do you have any insight here?

I'd say we should try to get the environment variable approach to work first. I think it should be possible using PYTHONPATH, PYTHONHOME or something similar.

richardliaw commented 2 years ago

So it looks like -s can be toggled using an environment variable:

The user site directory can be suppressed with a new option -s or the environment variable PYTHONNOUSERSITE. The feature can be disabled globally by setting site.ENABLE_USER_SITE to the value False. It must be set by editing site.py. It can’t be altered in sitecustomize.py or later.

https://peps.python.org/pep-0370/#implementation

Looking for -S now...

richardliaw commented 2 years ago

It seems like -S does not have the same environment variable toggle for Python.

It doesn't seem like PYTHONPATH/PYTHONHOME will have the same effect -- these environment variables augment the import flow for Python, but do not disable the searching in site-packages.

ponner-github commented 2 years ago

Thanks @architkulkarni and @richardliaw , apologies for the delayed response as i was working on making an internal patch to unblock.

Here is a simple repro of the issue emulating the behaviour we are observing

mkdir /tmp/custom_install && \
python3.7 -m pip install -t /tmp/custom_install setuptools && \
python3.7 -m pip install -t /tmp/custom_install ray && \
python3.7 -m pip install --user google-auth==2.6.0 && \
PYTHONPATH=/tmp/custom_install python3.7 -S run_dummy_job.py

Content of run_dummy_job.py

# Third-party imports
import ray

def run_dummy_job() -> str:
    try:
        import google.auth
    except ImportError:
        print("Import Exception thrown")
    result = add_numbers.remote(42, 58)

@ray.remote
def add_numbers(a: int, b: int) -> int:
    import google.auth as ga
    print(f"Google auth from: {ga.__file__}")
    return a + b

if __name__ == "__main__":
    run_dummy_job()

Output

Import Exception thrown
(add_numbers pid=95964) Google auth from: /Users/ponner.github/local/lib/python3.7/site-packages/google/auth/__init__.py

The raylet extends the module search path into site-packages, as @richardliaw -S does not have an environment variable toggle and it would be beneficial to have ray.init take this in as a flag.

ponner-github commented 2 years ago

Also confirming that passing in env variables with runtime_env do not fix the issue

# Third-party imports
import ray

def run_dummy_job() -> str:
    try:
        import google.auth
    except ImportError:
        print("Import Exception thrown")
    result = add_numbers.remote(42, 58)

@ray.remote
def add_numbers(a: int, b: int) -> int:
    import google.auth as ga
    print(f"Google auth from: {ga.__file__}")
    return a + b

if __name__ == "__main__":
    runtime_env = {"env_vars": {"PYTHONPATH": "/tmp/custom_install"}}
    ray.init(runtime_env=runtime_env)
    run_dummy_job()

Output

> PYTHONPATH=/tmp/custom_install python3.7 -S run_dummy_job.py
Import Exception thrown
(add_numbers pid=96343) Google auth from: /Users/ponner.github/.local/lib/python3.7/site-packages/google/auth/__init__.py
architkulkarni commented 2 years ago

@ponner-github I see, thanks for running those tests. I'll make a patch for this and try to get the API change approved.

richardliaw commented 2 years ago

BTW @architkulkarni we should consider doing this via runtime envs rather than through the top-level API. But yeah, lets go through the api change process.

richardliaw commented 2 years ago

@architkulkarni this was labeled P1; have we made progress on this?

architkulkarni commented 2 years ago

@richardliaw Sorry for the delay, there hasn't been progress on this. I'll try to prioritize it.

architkulkarni commented 2 years ago

Hi @ponner-github , sorry for the late reply. I did some more thinking about this and I think your specific reproduction is actually supported by the current Ray if you add PYTHONNOUSERSITE=1 to the runtime env. For clarity here's the full repro:

# run_dummy_job.py
import ray

def run_dummy_job() -> str:
    try:
        import google.auth
    except ImportError:
        print("Import Exception thrown")
    result = ray.get(add_numbers.remote(42, 58))

@ray.remote
def add_numbers(a: int, b: int) -> int:
    import google.auth as ga
    print(f"Google auth from: {ga.__file__}")
    return a + b

if __name__ == "__main__":
    ray.init(runtime_env={"env_vars": {"PYTHONNOUSERSITE": "1"}})
    ray.init(runtime_env=runtime_env)
    run_dummy_job()

Running your command:

mkdir /tmp/custom_install && \
python3.7 -m pip install -t /tmp/custom_install setuptools && \
python3.7 -m pip install -t /tmp/custom_install ray && \
python3.7 -m pip install --user google-auth==2.6.0 && \
PYTHONPATH=/tmp/custom_install python3.7 -S run_dummy_job.py

The output is now

Import Exception thrown
Traceback (most recent call last):
  File "/Users/archit/test/run_dummy_job.py", line 22, in <module>
    run_dummy_job()
  File "/Users/archit/test/run_dummy_job.py", line 10, in run_dummy_job
    result = ray.get(add_numbers.remote(42, 58))
  File "/tmp/custom_install/ray/_private/client_mode_hook.py", line 105, in wrapper
    return func(*args, **kwargs)
  File "/tmp/custom_install/ray/worker.py", line 1809, in get
    raise value.as_instanceof_cause()
ray.exceptions.RayTaskError(ModuleNotFoundError): ray::add_numbers() (pid=68037, ip=127.0.0.1)
  File "/Users/archit/test/run_dummy_job.py", line 15, in add_numbers
    import google.auth as ga
ModuleNotFoundError: No module named 'google.auth'

which is what we wanted.

There is a separate issue, which is that this only handles the -s flag; the -S flag doesn't have an environment variable flag as discussed earlier in the thread.

Could you help me confirm my understanding of the gap between just having -s and having both -sS flags for your use case? I read through the -S documentation which links to the site docs. Is the problem that it imports packages from sys.prefix, which points to /usr/local which contains undesired packages? The docs mention that if you use a virtual environment, sys.prefix just points to that environment--is that a viable workaround?

architkulkarni commented 2 years ago

One more question--you mentioned you added the flags in four places for it to work: start_worker_cmd, agent_command, java_command and start_ray_client_server

Would it be enough to have it just for the Ray workers (the start_worker_cmd)? One of the implementations I have in mind only works for Ray workers. If not, I can think of a different approach.

ponner-github commented 2 years ago

Thanks @architkulkarni

Could you help me confirm my understanding of the gap between just having -s and having both -sS flags for your use case? I read through the -S documentation which links to the site docs. Is the problem that it imports packages from sys.prefix, which points to /usr/local which contains undesired packages? The docs mention that if you use a virtual environment, sys.prefix just points to that environment--is that a viable workaround?

let me revisit my notes and retry the setup on

Would it be enough to have it just for the Ray workers (the start_worker_cmd)? One of the implementations I have in mind only works for Ray workers. If not, I can think of a different approach.

I think the start worker command and the java_command should be sufficient, when i was initially trying to fix the issue i added the -Ss flag whereever sys.executable was invoked. However from later testing found only passing it in for the worker_cmds (start_worker and java_command) were sufficient.

WaterKnight1998 commented 2 years ago

Hi @ponner-github @architkulkarni @richardliaw , I have seen the same issue while running it in databricks. I am starting the cluster with ray start.

What can I do to solve it? Trigger it like: PYTHONNOUSERSITE=1 ray start?

architkulkarni commented 2 years ago

@WaterKnight1998 can you give more details about your issue?

You can pass PYTHONNOUSERSITE=1 in your runtime environment: runtime_env={"env_vars": {"PYTHONNOUSERSITE": "1"}}. See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for more details.

Can you try the solution in https://github.com/ray-project/ray/issues/22977#issuecomment-1110325835? If you also need the -s flag in addition to PYTHONNOUSERSITE, can you try using a virtual environment to prevent Python from picking up packages in usr/local as discussed in that comment?

WaterKnight1998 commented 2 years ago

Hi @architkulkarni ,

@WaterKnight1998 can you give more details about your issue?

https://discuss.ray.io/t/ray-tune-not-working-inside-databricks/6594/6

You can pass PYTHONNOUSERSITE=1 in your runtime environment: runtime_env={"env_vars": {"PYTHONNOUSERSITE": "1"}}. See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for more details.

I tried this with Ray on Databricks but it didn't work, it looks like it is getting packages from pyspark:

covid_df = (spark
            .read
            .option("header", "true") 
            .option('inferSchema', 'true')
            .csv('/databricks-datasets/COVID/USAFacts/covid_deaths_usafacts.csv'))

select_cols = covid_df.columns[4:]

df = (covid_df
     .select(
       col('County Name').alias('county_name'),
       array([col(n) for n in select_cols]
       ).alias('deaths')))

@ray.remote
def linear_pred(x,y, i):
    import sys
    sys.path="/databricks/python/lib/python3.8/site-packages"
    from os import listdir
    raise Exception(listdir("/databricks/python/lib/python3.8/site-packages"))
    import imblearn
    import pandas as pd
    reg = linear_model.ElasticNet().fit(x, y)
    p = reg.predict(np.array([[i + 1]]))
    return p[0]

@pandas_udf(ArrayType(LongType()))
def ray_udf(s):
    ray.init(ignore_reinit_error=True, address='auto', _redis_password='d4t4bricks', runtime_env = {"env_vars": {"PYTHONPATH": "/databricks/python/lib/python3.8/site-packages", "PYTHONNOUSERSITE": "1"}})
    s = list(s)

    pred = []
    workers = []
    for i in range(len(s)):
        x = list(range(i+1))
        x = np.asarray([[n] for n in x])
        y = s[:i+1]
        y = np.asarray(y)

        workers.append(linear_pred.remote(x, y, i))

    pred = ray.get(workers)
    return pd.Series(pred)

res = df.select("county_name", "deaths", ray_udf("deaths").alias("preds"))
display(res)

Can you try the solution in https://github.com/ray-project/ray/issues/22977#issuecomment-1110325835?

Maybe I could try to do the same at databricks init script, I guess I just need to replace installation by this:

mkdir /tmp/custom_install && \
python3.7 -m pip install -t /tmp/custom_install setuptools && \
python3.7 -m pip install -t /tmp/custom_install ray

So init script would look like:

#!/bin/bash

#RAY PORT
RAY_PORT=9339
REDIS_PASS="d4t4bricks"

# install ray
mkdir /tmp/custom_install && \
python3.7 -m pip install -t /tmp/custom_install setuptools && \
python3.7 -m pip install -t /tmp/custom_install ray==1.13.0 && \
python3.7 -m pip install -t /tmp/custom_install ray[debug,dashboard,tune,rllib,serve]==1.13.0

# If starting on the Spark driver node, initialize the Ray head node
# If starting on the Spark worker node, connect to the head Ray node
if [ ! -z $DB_IS_DRIVER ] && [ $DB_IS_DRIVER = TRUE ] ; then
  echo "Starting the head node"
  PYTHONPATH=/tmp/custom_install ray start  --min-worker-port=20000 --max-worker-port=25000 --temp-dir="/tmp/ray" --head --port=$RAY_PORT --redis-password="$REDIS_PASS"  --include-dashboard=false
else
  sleep 40
  echo "Starting the non-head node - connecting to $DB_DRIVER_IP:$RAY_PORT"
  PYTHONPATH=/tmp/custom_install ray start  --min-worker-port=20000 --max-worker-port=25000 --temp-dir="/tmp/ray" --address="$DB_DRIVER_IP:$RAY_PORT" --redis-password="$REDIS_PASS"
fi
DmitriGekhtman commented 1 year ago

For the record, we've been patching the -S flag into the Ray code each time we upgrade Ray.

Adding myself as an assignee just so I can easily track Ray issues that affect Cruise -- I don't necessarily intend to fix this myself.

DmitriGekhtman commented 1 year ago

Proposed quick fix that wouldn't require an explicit API change is to pass the -S and -s from the root Ray process down to the Python workers. https://github.com/ray-project/ray/pull/32690