Skip to content

Commit

Permalink
Add JAX controller (#2194)
Browse files Browse the repository at this point in the history
* Add JAX controller

Add JAX controller, controller tests, webhook validations, examples,
e2e tests for JAXJob

Extend the Training Operator Python SDK to simplify the creation and
management of JAXJob resources.

Signed-off-by: Sandipan Panda <[email protected]>

* Add envvar tests

Signed-off-by: Sandipan Panda <[email protected]>

---------

Signed-off-by: Sandipan Panda <[email protected]>
  • Loading branch information
sandipanpanda committed Sep 20, 2024
1 parent ee6756b commit 8285aff
Show file tree
Hide file tree
Showing 22 changed files with 1,835 additions and 1 deletion.
4 changes: 4 additions & 0 deletions .github/workflows/publish-example-images.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -69,3 +69,7 @@ jobs:
platforms: linux/amd64,linux/arm64
dockerfile: examples/pytorch/mnist/Dockerfile-mpi
context: examples/pytorch/mnist
- component-name: jaxjob-simple
platforms: linux/amd64,linux/arm64
dockerfile: examples/jax/cpu-demo/Dockerfile
context: examples/jax/cpu-demo
8 changes: 8 additions & 0 deletions PROJECT
Original file line number Diff line number Diff line change
Expand Up @@ -27,4 +27,12 @@ resources:
kind: TFJob
path: github.com/kubeflow/training-operator/pkg/apis/kubeflow.org/v1
version: v1
- api:
crdVersion: v1
namespaced: true
controller: true
group: kubeflow.org
kind: JAXJob
path: github.com/kubeflow/training-operator/pkg/apis/kubeflow.org/v1
version: v1
version: "3"
2 changes: 1 addition & 1 deletion cmd/training-operator.v1/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func main() {
"Enabling this will ensure there is only one active controller manager.")
flag.StringVar(&leaderElectionID, "leader-election-id", "1ca428e5.training-operator.kubeflow.org", "The ID for leader election.")
flag.Var(&enabledSchemes, "enable-scheme", "Enable scheme(s) as --enable-scheme=tfjob --enable-scheme=pytorchjob, case insensitive."+
" Now supporting TFJob, PyTorchJob, XGBoostJob, PaddleJob. By default, all supported schemes will be enabled.")
" Now supporting TFJob, PyTorchJob, XGBoostJob, PaddleJob, JAXJob. By default, all supported schemes will be enabled.")
flag.StringVar(&gangSchedulerName, "gang-scheduler-name", "", "Now Supporting volcano and scheduler-plugins."+
" Note: If you set another scheduler name, the training-operator assumes it's the scheduler-plugins.")
flag.StringVar(&namespace, "namespace", os.Getenv(EnvKubeflowNamespace), "The namespace to monitor kubeflow jobs. If unset, it monitors all namespaces cluster-wide."+
Expand Down
25 changes: 25 additions & 0 deletions examples/jax/cpu-demo/Dockerfile
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
FROM python:3.12

RUN pip install jax absl-py kubernetes

RUN apt-get update && apt-get install -y \
build-essential \
cmake \
git \
libgoogle-glog-dev \
libgflags-dev \
libprotobuf-dev \
protobuf-compiler \
&& rm -rf /var/lib/apt/lists/*

RUN git clone https://github.com/facebookincubator/gloo.git \
&& cd gloo \
&& mkdir build \
&& cd build \
&& cmake ../ \
&& make \
&& make install

WORKDIR /app

ADD train.py /app
19 changes: 19 additions & 0 deletions examples/jax/cpu-demo/demo.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
apiVersion: "kubeflow.org/v1"
kind: JAXJob
metadata:
name: jaxjob-simple
namespace: kubeflow
spec:
jaxReplicaSpecs:
Worker:
replicas: 2
restartPolicy: OnFailure
template:
spec:
containers:
- name: jax
image: docker.io/sandipanify/jaxgoogle:latest
command:
- "python3"
- "train.py"
imagePullPolicy: Always
43 changes: 43 additions & 0 deletions examples/jax/cpu-demo/train.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
# example ref:
# https://jax.readthedocs.io/en/latest/multi_process.html#running-multi-process-computations
# https://github.com/GoogleCloudPlatform/ai-on-gke/blob/main/tutorials-and-examples/gpu-examples/a100-jax/train.py # noqa

import os
import socket

import jax
from absl import app

jax.config.update("jax_cpu_collectives_implementation", "gloo")


def _main(argv):

process_id = int(os.getenv("PROCESS_ID"))
num_processes = int(os.getenv("NUM_PROCESSES"))
coordinator_address = os.getenv("COORDINATOR_ADDRESS")
coordinator_port = int(os.getenv("COORDINATOR_PORT"))
coordinator_address = f"{coordinator_address}:{coordinator_port}"

jax.distributed.initialize(
coordinator_address=coordinator_address,
num_processes=num_processes,
process_id=process_id,
)

print(
f"JAX process {jax.process_index()}/{jax.process_count() - 1} initialized on "
f"{socket.gethostname()}"
)
print(f"JAX global devices:{jax.devices()}")
print(f"JAX local devices:{jax.local_devices()}")

print(f"JAX device count:{jax.device_count()}")
print(f"JAX local device count:{jax.local_device_count()}")

xs = jax.numpy.ones(jax.local_device_count())
print(jax.pmap(lambda x: jax.lax.psum(x, "i"), axis_name="i")(xs))


if __name__ == "__main__":
app.run(_main)
1 change: 1 addition & 0 deletions manifests/base/crds/kustomization.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -6,3 +6,4 @@ resources:
- kubeflow.org_xgboostjobs.yaml
- kubeflow.org_mpijobs.yaml
- kubeflow.org_paddlejobs.yaml
- kubeflow.org_jaxjobs.yaml
26 changes: 26 additions & 0 deletions manifests/base/rbac/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,32 @@ rules:
- patch
- update
- watch
- apiGroups:
- kubeflow.org
resources:
- jaxjobs
verbs:
- create
- delete
- get
- list
- patch
- update
- watch
- apiGroups:
- kubeflow.org
resources:
- jaxjobs/finalizers
verbs:
- update
- apiGroups:
- kubeflow.org
resources:
- jaxjobs/status
verbs:
- get
- patch
- update
- apiGroups:
- kubeflow.org
resources:
Expand Down
20 changes: 20 additions & 0 deletions manifests/base/webhook/manifests.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,26 @@ kind: ValidatingWebhookConfiguration
metadata:
name: validating-webhook-configuration
webhooks:
- admissionReviewVersions:
- v1
clientConfig:
service:
name: webhook-service
namespace: system
path: /validate-kubeflow-org-v1-jaxjob
failurePolicy: Fail
name: validator.jaxjob.training-operator.kubeflow.org
rules:
- apiGroups:
- kubeflow.org
apiVersions:
- v1
operations:
- CREATE
- UPDATE
resources:
- jaxjobs
sideEffects: None
- admissionReviewVersions:
- v1
clientConfig:
Expand Down
3 changes: 3 additions & 0 deletions manifests/base/webhook/patch.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,9 @@
- op: replace
path: /webhooks/3/clientConfig/service/name
value: training-operator
- op: replace
path: /webhooks/4/clientConfig/service/name
value: training-operator
- op: replace
path: /metadata/name
value: validator.training-operator.kubeflow.org
107 changes: 107 additions & 0 deletions pkg/controller.v1/jax/envvar.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
// Copyright 2024 The Kubeflow Authors
//
// Licensed 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

package jax

import (
"errors"
"strconv"
"strings"

corev1 "k8s.io/api/core/v1"
"k8s.io/utils/ptr"

kubeflowv1 "github.com/kubeflow/training-operator/pkg/apis/kubeflow.org/v1"
)

var (
errorDefaulContainerPortNotExposed = errors.New("default container port is not exposed")
errorFailedToRecognizeRank = errors.New("failed to recognize the JAXJob Rank")
)

type EnvVarGenerator interface {
Generate(job *kubeflowv1.JAXJob) ([]corev1.EnvVar, error)
}

func setPodEnv(jaxjob *kubeflowv1.JAXJob, podTemplateSpec *corev1.PodTemplateSpec, rtype, index string) error {

coordinatorAddr := replicaName(jaxjob.Name, kubeflowv1.JAXJobReplicaTypeWorker, 0)

coordinatorPort, err := getPortFromJAXJob(jaxjob, kubeflowv1.JAXJobReplicaTypeWorker)
if err != nil {
return err
}

totalReplicas := getTotalReplicas(jaxjob)

for i := range podTemplateSpec.Spec.Containers {

rank, err := strconv.Atoi(index)
if err != nil {
return errorFailedToRecognizeRank
}
// Set PYTHONUNBUFFERED to true, to disable output buffering.
// Ref https://stackoverflow.com/questions/59812009/what-is-the-use-of-pythonunbuffered-in-docker-file.
podTemplateSpec.Spec.Containers[i].Env = append(podTemplateSpec.Spec.Containers[i].Env, corev1.EnvVar{
Name: "PYTHONUNBUFFERED",
Value: "1",
})
podTemplateSpec.Spec.Containers[i].Env = append(podTemplateSpec.Spec.Containers[i].Env, corev1.EnvVar{
Name: "COORDINATOR_PORT",
Value: strconv.Itoa(int(coordinatorPort)),
})
podTemplateSpec.Spec.Containers[i].Env = append(podTemplateSpec.Spec.Containers[i].Env, corev1.EnvVar{
Name: "COORDINATOR_ADDRESS",
Value: coordinatorAddr,
})
podTemplateSpec.Spec.Containers[i].Env = append(podTemplateSpec.Spec.Containers[i].Env, corev1.EnvVar{
Name: "NUM_PROCESSES",
Value: strconv.Itoa(int(totalReplicas)),
})
podTemplateSpec.Spec.Containers[i].Env = append(podTemplateSpec.Spec.Containers[i].Env, corev1.EnvVar{
Name: "PROCESS_ID",
Value: strconv.Itoa(rank),
})
}

return nil
}

func getTotalReplicas(job *kubeflowv1.JAXJob) int {
jobReplicas := 0
for _, r := range job.Spec.JAXReplicaSpecs {
jobReplicas += int(ptr.Deref[int32](r.Replicas, 0))
}
return jobReplicas
}

func replicaName(jobName string, rtype kubeflowv1.ReplicaType, index int) string {
n := jobName + "-" + strings.ToLower(string(rtype)) + "-" + strconv.Itoa(index)
return strings.Replace(n, "/", "-", -1)
}

func getPortFromJAXJob(job *kubeflowv1.JAXJob, rtype kubeflowv1.ReplicaType) (int32, error) {
containers := job.Spec.JAXReplicaSpecs[rtype].Template.Spec.Containers
for _, container := range containers {
if container.Name == kubeflowv1.JAXJobDefaultContainerName {
ports := container.Ports
for _, port := range ports {
if port.Name == kubeflowv1.JAXJobDefaultPortName {
return port.ContainerPort, nil
}
}
}
}
return -1, errorDefaulContainerPortNotExposed
}
Loading

0 comments on commit 8285aff

Please sign in to comment.