Skip to content

Commit

Permalink
[Test][Autoscaling] Add custom resource test
Browse files Browse the repository at this point in the history
Signed-off-by: Chi-Sheng Liu <[email protected]>
  • Loading branch information
MortalHappiness committed Jun 14, 2024
1 parent 40a946a commit aebd2ee
Show file tree
Hide file tree
Showing 2 changed files with 60 additions and 1 deletion.
3 changes: 2 additions & 1 deletion ray-operator/test/e2e/create_detached_actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,10 @@
parser.add_argument('name')
parser.add_argument('--num-cpus', type=float, default=1)
parser.add_argument('--num-gpus', type=float, default=0)
parser.add_argument('--num-custom-resources', type=float, default=0)
args = parser.parse_args()

@ray.remote(num_cpus=args.num_cpus, num_gpus=args.num_gpus)
@ray.remote(num_cpus=args.num_cpus, num_gpus=args.num_gpus, resources={"CustomResource": args.num_custom_resources})
class Actor:
pass

Expand Down
58 changes: 58 additions & 0 deletions ray-operator/test/e2e/raycluster_autoscaler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,3 +133,61 @@ func TestRayClusterAutoscalerWithFakeGPU(t *testing.T) {
Should(WithTransform(RayClusterDesiredWorkerReplicas, Equal(int32(0))))
})
}

func TestRayClusterAutoscalerWithCustomResource(t *testing.T) {
test := With(t)

// Create a namespace
namespace := test.NewTestNamespace()
test.StreamKubeRayOperatorLogs()

// Scripts for creating and terminating detached actors to trigger autoscaling
scriptsAC := newConfigMap(namespace.Name, "scripts", files(test, "create_detached_actor.py", "terminate_detached_actor.py"))
scripts, err := test.Client().Core().CoreV1().ConfigMaps(namespace.Name).Apply(test.Ctx(), scriptsAC, TestApplyOptions)
test.Expect(err).NotTo(HaveOccurred())
test.T().Logf("Created ConfigMap %s/%s successfully", scripts.Namespace, scripts.Name)

test.T().Run("Create a RayCluster with autoscaling enabled", func(_ *testing.T) {
groupName := "custom-resource-group"

rayClusterSpecAC := rayv1ac.RayClusterSpec().
WithEnableInTreeAutoscaling(true).
WithRayVersion(GetRayVersion()).
WithHeadGroupSpec(rayv1ac.HeadGroupSpec().
WithRayStartParams(map[string]string{"num-cpus": "0"}).
WithTemplate(headPodTemplateApplyConfiguration())).
WithWorkerGroupSpecs(rayv1ac.WorkerGroupSpec().
WithReplicas(0).
WithMinReplicas(0).
WithMaxReplicas(3).
WithGroupName(groupName).
WithRayStartParams(map[string]string{"num-cpus": "1", "resources": "\"{\\\"CustomResource\\\": 1}\""}).
WithTemplate(workerPodTemplateApplyConfiguration()))
rayClusterAC := rayv1ac.RayCluster("ray-cluster", namespace.Name).
WithSpec(apply(rayClusterSpecAC, mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](scripts, "/home/ray/test_scripts")))

rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions)
test.Expect(err).NotTo(HaveOccurred())
test.T().Logf("Created RayCluster %s/%s successfully", rayCluster.Namespace, rayCluster.Name)

// Wait for RayCluster to become ready and verify the number of available worker replicas.
test.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium).
Should(WithTransform(RayClusterState, Equal(rayv1.Ready)))
rayCluster = GetRayCluster(test, rayCluster.Namespace, rayCluster.Name)
test.Expect(rayCluster.Status.DesiredWorkerReplicas).To(Equal(int32(0)))

headPod := GetHeadPod(test, rayCluster)
test.T().Logf("Found head pod %s/%s", headPod.Namespace, headPod.Name)

// Create a detached custom resource actor, and a worker in the "custom-resource-group" should be created.
ExecPodCmd(test, headPod, common.RayHeadContainer, []string{"python", "/home/ray/test_scripts/create_detached_actor.py", "custom_resource_actor", "--num-custom-resources=1"})
test.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium).
Should(WithTransform(RayClusterDesiredWorkerReplicas, Equal(int32(1))))
test.Expect(GetGroupPods(test, rayCluster, groupName)).To(HaveLen(1))

// Terminate the custom resource detached actor, and the worker should be deleted.
ExecPodCmd(test, headPod, common.RayHeadContainer, []string{"python", "/home/ray/test_scripts/terminate_detached_actor.py", "custom_resource_actor"})
test.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium).
Should(WithTransform(RayClusterDesiredWorkerReplicas, Equal(int32(0))))
})
}

0 comments on commit aebd2ee

Please sign in to comment.