Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions airflow-core/tests/unit/always/test_project_structure.py
Original file line number Diff line number Diff line change
Expand Up @@ -470,6 +470,7 @@ class TestGoogleProviderProjectStructure(ExampleCoverageTest, AssetsCoverageTest
"airflow.providers.google.cloud.operators.managed_kafka.ManagedKafkaBaseOperator",
"airflow.providers.google.cloud.operators.vertex_ai.custom_job.CustomTrainingJobBaseOperator",
"airflow.providers.google.cloud.operators.vertex_ai.ray.RayBaseOperator",
"airflow.providers.google.cloud.operators.ray.RayJobBaseOperator",
"airflow.providers.google.cloud.operators.cloud_base.GoogleCloudBaseOperator",
"airflow.providers.google.marketing_platform.operators.search_ads._GoogleSearchAdsBaseOperator",
}
Expand Down
1 change: 1 addition & 0 deletions docs/spelling_wordlist.txt
Original file line number Diff line number Diff line change
Expand Up @@ -981,6 +981,7 @@ Jira
jira
jitter
JobComplete
JobDetails
JobExists
jobflow
jobId
Expand Down
90 changes: 90 additions & 0 deletions providers/google/docs/operators/cloud/ray.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
.. Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at

.. http://www.apache.org/licenses/LICENSE-2.0

.. Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.

Ray Job Operators
=================

The Ray Job operators provide a high-level interface for interacting with remote Ray clusters
using the Ray Jobs API. These operators can be used with clusters running on Google Cloud Vertex AI Ray,
GKE (self-managed Ray clusters) or any Ray cluster reachable through a dashboard address or Ray Client address.

The operators allow you to submit jobs, monitor their progress, retrieve logs,
and manage job lifecycle from Airflow.

Submitting Ray Jobs
^^^^^^^^^^^^^^^^^^^

The :class:`~airflow.providers.google.cloud.operators.ray.RaySubmitJobOperator`
submits a job to a Ray cluster and optionally waits for completion.

It supports waiting for job completion with ``wait_for_job_done``
and retrieving logs after completion with ``get_job_logs`` parameters.

.. exampleinclude:: /../../google/tests/system/google/cloud/ray/example_ray_job.py
:language: python
:dedent: 4
:start-after: [START how_to_ray_submit_job]
:end-before: [END how_to_ray_submit_job]

Stopping Ray Jobs
^^^^^^^^^^^^^^^^^

Use :class:`~airflow.providers.google.cloud.operators.ray.RayStopJobOperator`
to stop a running Ray job identified by its job ID.

.. exampleinclude:: /../../google/tests/system/google/cloud/ray/example_ray_job.py
:language: python
:dedent: 4
:start-after: [START how_to_ray_stop_job]
:end-before: [END how_to_ray_stop_job]

Deleting Ray Jobs
^^^^^^^^^^^^^^^^^

Use :class:`~airflow.providers.google.cloud.operators.ray.RayDeleteJobOperator`
to delete a job and its metadata after it reaches a terminal state.

.. exampleinclude:: /../../google/tests/system/google/cloud/ray/example_ray_job.py
:language: python
:dedent: 4
:start-after: [START how_to_ray_delete_job]
:end-before: [END how_to_ray_delete_job]

Retrieving Job Information
^^^^^^^^^^^^^^^^^^^^^^^^^^

The :class:`~airflow.providers.google.cloud.operators.ray.RayGetJobInfoOperator`
retrieves detailed information about a Ray job, including status, timestamps,
entrypoint, metadata, and runtime environment.

.. exampleinclude:: /../../google/tests/system/google/cloud/ray/example_ray_job.py
:language: python
:dedent: 4
:start-after: [START how_to_ray_get_job_info]
:end-before: [END how_to_ray_get_job_info]

Listing Jobs
^^^^^^^^^^^^

Use :class:`~airflow.providers.google.cloud.operators.ray.RayListJobsOperator`
to list all jobs that have run on the cluster.

.. exampleinclude:: /../../google/tests/system/google/cloud/ray/example_ray_job.py
:language: python
:dedent: 4
:start-after: [START how_to_ray_list_jobs]
:end-before: [END how_to_ray_list_jobs]
12 changes: 12 additions & 0 deletions providers/google/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -462,6 +462,11 @@ integrations:
how-to-guide:
- /docs/apache-airflow-providers-google/operators/cloud/gen_ai.rst
tags: [gcp]
- integration-name: Google Ray
external-doc-url: https://docs.cloud.google.com/vertex-ai/docs/open-source/ray-on-vertex-ai/overview
how-to-guide:
- /docs/apache-airflow-providers-google/operators/cloud/ray.rst
tags: [gcp]

operators:
- integration-name: Google Ads
Expand Down Expand Up @@ -624,6 +629,9 @@ operators:
- integration-name: Google Cloud Generative AI
python-modules:
- airflow.providers.google.cloud.operators.gen_ai
- integration-name: Google Ray
python-modules:
- airflow.providers.google.cloud.operators.ray

sensors:
- integration-name: Google BigQuery
Expand Down Expand Up @@ -905,6 +913,9 @@ hooks:
- integration-name: Google Cloud Generative AI
python-modules:
- airflow.providers.google.cloud.hooks.gen_ai
- integration-name: Google Ray
python-modules:
- airflow.providers.google.cloud.hooks.ray

bundles:
- integration-name: Google Cloud Storage (GCS)
Expand Down Expand Up @@ -1258,6 +1269,7 @@ extra-links:
- airflow.providers.google.cloud.links.managed_kafka.ApacheKafkaClusterListLink
- airflow.providers.google.cloud.links.managed_kafka.ApacheKafkaTopicLink
- airflow.providers.google.cloud.links.managed_kafka.ApacheKafkaConsumerGroupLink
- airflow.providers.google.cloud.links.ray.RayJobLink

secrets-backends:
- airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
Expand Down
234 changes: 234 additions & 0 deletions providers/google/src/airflow/providers/google/cloud/hooks/ray.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,234 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""This module contains a Google Cloud Ray Job hook."""

from __future__ import annotations

from typing import TYPE_CHECKING, Any
from urllib.parse import urlparse

from ray.job_submission import JobSubmissionClient

from airflow.providers.google.common.hooks.base_google import GoogleBaseHook

if TYPE_CHECKING:
from ray.dashboard.modules.job.common import JobStatus
from ray.dashboard.modules.job.pydantic_models import JobDetails

VERTEX_RAY_DOMAIN = "aiplatform-training.googleusercontent.com"


class RayJobHook(GoogleBaseHook):
"""Hook for Jobs APIs."""

def _is_vertex_ray_address(self, address: str) -> bool:
"""Return True if address points to Vertex Ray dashboard host."""
parsed = urlparse(address if "://" in address else f"https://{address}")
hostname = parsed.hostname
if not hostname:
return False
return hostname.endswith(VERTEX_RAY_DOMAIN)

def get_client(self, address: str) -> JobSubmissionClient:
"""
Create a client for submitting and interacting with jobs on a remote cluster.

:param address: Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
"""
if self._is_vertex_ray_address(address):
return JobSubmissionClient(f"vertex_ray://{address}")
return JobSubmissionClient(address=address)

def serialize_job_obj(self, job_obj: JobDetails) -> dict:
"""Serialize JobDetails to a plain dict."""
if hasattr(job_obj, "model_dump"): # Pydantic v2
return job_obj.model_dump(exclude_none=True)
if hasattr(job_obj, "dict"): # Pydantic v1
return job_obj.dict(exclude_none=True)
return dict(job_obj)

def submit_job(
self,
entrypoint: str,
cluster_address: str,
runtime_env: dict[str, Any] | None = None,
metadata: dict[str, str] | None = None,
submission_id: str | None = None,
entrypoint_num_cpus: int | float | None = None,
entrypoint_num_gpus: int | float | None = None,
entrypoint_memory: int | None = None,
entrypoint_resources: dict[str, float] | None = None,
) -> str:
"""
Submit and execute Job on Ray cluster.

When a job is submitted, it runs once to completion or failure. Retries or
different runs with different parameters should be handled by the
submitter. Jobs are bound to the lifetime of a Ray cluster, so if the
cluster goes down, all running jobs on that cluster will be terminated.

:param entrypoint: Required. The shell command to run for this job.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:param submission_id: A unique ID for this job.
:param runtime_env: The runtime environment to install and run this job in.
:param metadata: Arbitrary data to store along with this job.
:param entrypoint_num_cpus: The quantity of CPU cores to reserve for the execution
of the entrypoint command, separately from any tasks or actors launched
by it. Defaults to 0.
:param entrypoint_num_gpus: The quantity of GPUs to reserve for the execution
of the entrypoint command, separately from any tasks or actors launched
by it. Defaults to 0.
:param entrypoint_memory: The quantity of memory to reserve for the
execution of the entrypoint command, separately from any tasks or
actors launched by it. Defaults to 0.
:param entrypoint_resources: The quantity of custom resources to reserve for the
execution of the entrypoint command, separately from any tasks or
actors launched by it.
"""
job_id = self.get_client(address=cluster_address).submit_job(
entrypoint=entrypoint,
runtime_env=runtime_env,
metadata=metadata,
submission_id=submission_id,
entrypoint_num_cpus=entrypoint_num_cpus,
entrypoint_num_gpus=entrypoint_num_gpus,
entrypoint_memory=entrypoint_memory,
entrypoint_resources=entrypoint_resources,
)
return job_id

def stop_job(
self,
job_id: str,
cluster_address: str,
) -> bool:
"""
Stop Job on Ray cluster.

:param job_id: Required. The job ID or submission ID for the job to be stopped.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:return: True if the job was stopped, otherwise False.
"""
return self.get_client(address=cluster_address).stop_job(job_id=job_id)

def delete_job(
self,
job_id: str,
cluster_address: str,
) -> bool:
"""
Delete Job on Ray cluster in a terminal state and all of its associated data.

If the job is not already in a terminal state, raises an error.
This does not delete the job logs from disk.
Submitting a job with the same submission ID as a previously
deleted job is not supported and may lead to unexpected behavior.

:param job_id: Required. The job ID or submission ID for the job to be deleted.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:return: True if the job was deleted, otherwise False.
"""
return self.get_client(address=cluster_address).delete_job(job_id=job_id)

def get_job_info(
self,
job_id: str,
cluster_address: str,
) -> JobDetails:
"""
Get the latest status and other information associated with a Job on Ray cluster.

:param job_id: Required. The job ID or submission ID for the job to be retrieved.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:return: The JobDetails for the job.
"""
return self.get_client(address=cluster_address).get_job_info(job_id=job_id)

def list_jobs(
self,
cluster_address: str,
) -> list[JobDetails]:
"""
List all jobs along with their status and other information.

Lists all jobs that have ever run on the cluster, including jobs that are
currently running and jobs that are no longer running.

:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
"""
return self.get_client(address=cluster_address).list_jobs()

def get_job_status(
self,
job_id: str,
cluster_address: str,
) -> JobStatus:
"""
Get the most recent status of a Job on Ray cluster.

:param job_id: Required. The job ID or submission ID for the job to be retrieved.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:return: The JobStatus of the job.
"""
return self.get_client(address=cluster_address).get_job_status(job_id=job_id)

def get_job_logs(
self,
job_id: str,
cluster_address: str,
) -> str:
"""
Get all logs produced by a Job on Ray cluster.

:param job_id: Required. The job ID or submission ID for the job to be retrieved.
:param cluster_address: Required. Either (1) the address of the Ray cluster, or (2) the HTTP address
of the dashboard server on the head node, e.g. "http://<head-node-ip>:8265".
In case (1) it must be specified as an address that can be passed to
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
or "auto", or "localhost:<port>".
:return: A string containing the full logs of the job.
"""
return self.get_client(address=cluster_address).get_job_logs(job_id=job_id)
Loading
Loading