From 1405b6fbf48c7d27a4ebd7b1748310ed3cb8dd46 Mon Sep 17 00:00:00 2001 From: David Xia Date: Tue, 11 Feb 2025 08:52:29 -0500 Subject: [PATCH] [feat][kubectl-plugin] list Ray nodes with ```console kubectl ray (get|list) node[s] (NODE_NAME) \ [(-c/--ray-cluster) RAYCLUSTER] [(-n/--namespace) NAMESPACE] [(-A/--all-namespaces)] ``` Signed-off-by: David Xia --- kubectl-plugin/pkg/cmd/get/get.go | 1 + kubectl-plugin/pkg/cmd/get/get_nodes.go | 267 ++++++++ kubectl-plugin/pkg/cmd/get/get_nodes_test.go | 579 ++++++++++++++++++ .../get-nodes-print-nodes-with-namespaces.txt | 3 + .../get/testdata/get-nodes-print-nodes.txt | 3 + ...et-nodes-run-cluster-namespace-no-node.txt | 3 + .../get-nodes-run-cluster-namespace-node.txt | 3 + ...nodes-run-cluster-no-namespace-no-node.txt | 2 + ...et-nodes-run-cluster-no-namespace-node.txt | 2 + ...nodes-run-no-cluster-namespace-no-node.txt | 4 + ...et-nodes-run-no-cluster-namespace-node.txt | 2 + ...es-run-no-cluster-no-namespace-no-node.txt | 5 + ...nodes-run-no-cluster-no-namespace-node.txt | 2 + .../get-nodes-run-no-node-no-pods.txt | 1 + kubectl-plugin/pkg/util/constant.go | 8 + 15 files changed, 885 insertions(+) create mode 100644 kubectl-plugin/pkg/cmd/get/get_nodes.go create mode 100644 kubectl-plugin/pkg/cmd/get/get_nodes_test.go create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes-with-namespaces.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-no-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-no-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-no-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-no-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-node.txt create mode 100644 kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-node-no-pods.txt diff --git a/kubectl-plugin/pkg/cmd/get/get.go b/kubectl-plugin/pkg/cmd/get/get.go index 5b0af5a16ce..63892ad7c27 100644 --- a/kubectl-plugin/pkg/cmd/get/get.go +++ b/kubectl-plugin/pkg/cmd/get/get.go @@ -24,5 +24,6 @@ func NewGetCommand(streams genericclioptions.IOStreams) *cobra.Command { } cmd.AddCommand(NewGetClusterCommand(streams)) + cmd.AddCommand(NewGetNodesCommand(streams)) return cmd } diff --git a/kubectl-plugin/pkg/cmd/get/get_nodes.go b/kubectl-plugin/pkg/cmd/get/get_nodes.go new file mode 100644 index 00000000000..0561041b48d --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/get_nodes.go @@ -0,0 +1,267 @@ +package get + +import ( + "context" + "errors" + "fmt" + "io" + "strings" + "time" + + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util" + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util/client" + "github.com/spf13/cobra" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/util/duration" + "k8s.io/cli-runtime/pkg/genericclioptions" + "k8s.io/cli-runtime/pkg/printers" + cmdutil "k8s.io/kubectl/pkg/cmd/util" + "k8s.io/kubectl/pkg/util/templates" +) + +type GetNodesOptions struct { + configFlags *genericclioptions.ConfigFlags + ioStreams *genericclioptions.IOStreams + namespace string + cluster string + node string + allNamespaces bool +} + +type node struct { + creationTimestamp v1.Time + cpus resource.Quantity + gpus resource.Quantity + tpus resource.Quantity + memory resource.Quantity + namespace string + cluster string + _type string + workerGroup string + name string +} + +var getNodesExample = templates.Examples(` + # Get nodes in the default namespace + kubectl ray get node + + # Get nodes in all namespaces + kubectl ray get node --all-namespaces + + # Get all nodes in a namespace + kubectl ray get node --namespace my-namespace + + # Get all nodes for Ray clusters named my-raycluster in all namespaces + kubectl ray get node --ray-cluster my-raycluster --all-namespaces + + # Get all nodes in a namespace for a Ray cluster + kubectl ray get node --namespace my-namespace --ray-cluster my-raycluster + + # Get one node in a namespace for a Ray cluster + kubectl ray get node my-node --namespace my-namespace --ray-cluster my-raycluster + `) + +func NewGetNodesOptions(streams genericclioptions.IOStreams) *GetNodesOptions { + return &GetNodesOptions{ + configFlags: genericclioptions.NewConfigFlags(true), + ioStreams: &streams, + } +} + +func NewGetNodesCommand(streams genericclioptions.IOStreams) *cobra.Command { + options := NewGetNodesOptions(streams) + cmdFactory := cmdutil.NewFactory(options.configFlags) + + cmd := &cobra.Command{ + Use: "node [NODE] [(-c|--ray-cluster) RAYCLUSTER]", + Aliases: []string{"nodes"}, + Short: "Get Ray nodes", + Example: getNodesExample, + SilenceUsage: true, + Args: cobra.MaximumNArgs(1), + // ValidArgsFunction: TODO + RunE: func(cmd *cobra.Command, args []string) error { + if err := options.Complete(args); err != nil { + return err + } + if err := options.Validate(); err != nil { + return err + } + k8sClient, err := client.NewClient(cmdFactory) + if err != nil { + return fmt.Errorf("failed to create client: %w", err) + } + return options.Run(cmd.Context(), k8sClient) + }, + } + + cmd.Flags().StringVarP(&options.cluster, "ray-cluster", "c", "", "Ray cluster") + cmd.Flags().BoolVarP(&options.allNamespaces, "all-namespaces", "A", false, "If present, list nodes across all namespaces. Namespace in current context is ignored even if specified with --namespace.") + + options.configFlags.AddFlags(cmd.Flags()) + return cmd +} + +func (options *GetNodesOptions) Complete(args []string) error { + if options.allNamespaces { + options.namespace = "" + } else { + if options.configFlags.Namespace == nil || *options.configFlags.Namespace == "" { + options.namespace = "default" + } else { + options.namespace = *options.configFlags.Namespace + } + } + + if len(args) > 0 { + options.node = args[0] + } + + return nil +} + +func (options *GetNodesOptions) Validate() error { + // Overrides and binds the kube config then retrieves the merged result + config, err := options.configFlags.ToRawKubeConfigLoader().RawConfig() + if err != nil { + return fmt.Errorf("error retrieving raw config: %w", err) + } + if !util.HasKubectlContext(config, options.configFlags) { + return fmt.Errorf("no context is currently set, use %q or %q to select a new one", "--context", "kubectl config use-context ") + } + return nil +} + +func (options *GetNodesOptions) Run(ctx context.Context, k8sClient client.Client) error { + listopts := v1.ListOptions{ + LabelSelector: joinLabelMap(createLabelSelectors(options.cluster)), + } + + if options.node != "" { + listopts.FieldSelector = fmt.Sprintf("metadata.name=%s", options.node) + } + + pods, err := k8sClient.KubernetesClient().CoreV1().Pods(options.namespace).List(ctx, listopts) + if err != nil { + return fmt.Errorf("unable to get Ray nodes: %w", err) + } + + nodes := podsToNodes(pods.Items) + + if options.node != "" && len(nodes) == 0 { + return errors.New(errorMessageForNodeNotFound(options.node, options.cluster, options.namespace, options.allNamespaces)) + } + + return printNodes(nodes, options.allNamespaces, options.ioStreams.Out) +} + +// createLabelSelectors creates a map of K8s label selectors for Ray nodes +func createLabelSelectors(cluster string) map[string]string { + labelSelectors := map[string]string{ + util.RayIsRayNodeLabelKey: "yes", + } + if cluster != "" { + labelSelectors[util.RayClusterLabelKey] = cluster + } + return labelSelectors +} + +// joinLabelMap joins a map of K8s label key-val entries into a label selector string +func joinLabelMap(labelMap map[string]string) string { + var labels []string + for k, v := range labelMap { + labels = append(labels, fmt.Sprintf("%s=%s", k, v)) + } + return strings.Join(labels, ",") +} + +// errorMessageForNodeNotFound returns an error message for when a node is not found +func errorMessageForNodeNotFound(node, cluster, namespace string, allNamespaces bool) string { + errMsg := fmt.Sprintf("Ray node %s not found", node) + + if allNamespaces { + errMsg += " in any namespace" + } else { + errMsg += fmt.Sprintf(" in namespace %s", namespace) + } + + if cluster != "" { + errMsg += fmt.Sprintf(" in Ray cluster %s", cluster) + } else { + errMsg += " in any Ray cluster" + } + + return errMsg +} + +// podsToNodes converts an array of K8s Pods to a list of nodes +func podsToNodes(pods []corev1.Pod) []node { + var nodes []node + for _, pod := range pods { + nodes = append(nodes, node{ + namespace: pod.Namespace, + cluster: pod.Labels[util.RayClusterLabelKey], + _type: pod.Labels[util.RayNodeTypeLabelKey], + workerGroup: pod.Labels[util.RayNodeGroupLabelKey], + name: pod.Name, + cpus: *pod.Spec.Containers[0].Resources.Requests.Cpu(), + gpus: *pod.Spec.Containers[0].Resources.Requests.Name(corev1.ResourceName(util.ResourceNvidiaGPU), resource.DecimalSI), + tpus: *pod.Spec.Containers[0].Resources.Requests.Name(corev1.ResourceName(util.ResourceGoogleTPU), resource.DecimalSI), + memory: *pod.Spec.Containers[0].Resources.Requests.Memory(), + creationTimestamp: pod.CreationTimestamp, + }) + } + return nodes +} + +// printNodes prints a list of nodes to the output +func printNodes(nodes []node, allNamespaces bool, output io.Writer) error { + resultTablePrinter := printers.NewTablePrinter(printers.PrintOptions{}) + + columns := []v1.TableColumnDefinition{} + if allNamespaces { + columns = append(columns, v1.TableColumnDefinition{Name: "Namespace", Type: "string"}) + } + columns = append(columns, []v1.TableColumnDefinition{ + {Name: "Name", Type: "string"}, + {Name: "CPUs", Type: "string"}, + {Name: "GPUs", Type: "string"}, + {Name: "TPUs", Type: "string"}, + {Name: "Memory", Type: "string"}, + {Name: "Cluster", Type: "string"}, + {Name: "Type", Type: "string"}, + {Name: "Worker Group", Type: "string"}, + {Name: "Age", Type: "string"}, + }...) + + resTable := &v1.Table{ColumnDefinitions: columns} + + for _, node := range nodes { + age := duration.HumanDuration(time.Since(node.creationTimestamp.Time)) + if node.creationTimestamp.Time.IsZero() { + age = "" + } + + row := v1.TableRow{} + if allNamespaces { + row.Cells = append(row.Cells, node.namespace) + } + row.Cells = append(row.Cells, []interface{}{ + node.name, + node.cpus.String(), + node.gpus.String(), + node.tpus.String(), + node.memory.String(), + node.cluster, + node._type, + node.workerGroup, + age, + }...) + + resTable.Rows = append(resTable.Rows, row) + } + + return resultTablePrinter.PrintObj(resTable, output) +} diff --git a/kubectl-plugin/pkg/cmd/get/get_nodes_test.go b/kubectl-plugin/pkg/cmd/get/get_nodes_test.go new file mode 100644 index 00000000000..0af0bf5ac39 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/get_nodes_test.go @@ -0,0 +1,579 @@ +package get + +import ( + "bytes" + "context" + "os" + "strings" + "testing" + "time" + + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/cli-runtime/pkg/genericclioptions" + kubefake "k8s.io/client-go/kubernetes/fake" + "k8s.io/utils/ptr" + + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util/client" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + rayClientFake "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/fake" +) + +func TestRayNodesGetComplete(t *testing.T) { + tests := []struct { + opts *GetNodesOptions + name string + expectedNamespace string + expectedNode string + args []string + }{ + { + name: "specifying all namespaces should set namespace to empty string", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{}, + allNamespaces: true, + }, + expectedNamespace: "", + }, + { + name: "not specifying a namespace should set namespace to 'default'", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + Namespace: ptr.To(""), + }, + allNamespaces: false, + }, + expectedNamespace: "default", + }, + { + name: "specifying a namespace should set that namespace", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + Namespace: ptr.To("some-namespace"), + }, + allNamespaces: false, + }, + expectedNamespace: "some-namespace", + }, + { + name: "specifying all namespaces takes precedence over specifying a namespace", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + Namespace: ptr.To("some-namespace"), + }, + allNamespaces: true, + }, + expectedNamespace: "", + }, + { + name: "first positional argument should be set as the node name", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{}, + }, + args: []string{"my-node", "other-arg"}, + expectedNamespace: "default", + expectedNode: "my-node", + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + err := tc.opts.Complete(tc.args) + require.NoError(t, err) + assert.Equal(t, tc.expectedNamespace, tc.opts.namespace) + assert.Equal(t, tc.expectedNode, tc.opts.node) + }) + } +} + +func TestRayNodesGetValidate(t *testing.T) { + testContext := "test-context" + + kubeConfigWithCurrentContext, err := util.CreateTempKubeConfigFile(t, testContext) + require.NoError(t, err) + + kubeConfigWithoutCurrentContext, err := util.CreateTempKubeConfigFile(t, "") + require.NoError(t, err) + + tests := []struct { + name string + opts *GetNodesOptions + expect string + expectError string + }{ + { + name: "should error when no context is set", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + KubeConfig: &kubeConfigWithoutCurrentContext, + }, + }, + expectError: "no context is currently set, use \"--context\" or \"kubectl config use-context \" to select a new one", + }, + { + name: "no error when kubeconfig has current context and --context switch isn't set", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + KubeConfig: &kubeConfigWithCurrentContext, + }, + }, + }, + { + name: "no error when kubeconfig has no current context and --context switch is set", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + KubeConfig: &kubeConfigWithoutCurrentContext, + Context: &testContext, + }, + }, + }, + { + name: "no error when kubeconfig has current context and --context switch is set", + opts: &GetNodesOptions{ + configFlags: &genericclioptions.ConfigFlags{ + KubeConfig: &kubeConfigWithCurrentContext, + Context: &testContext, + }, + }, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + err := tc.opts.Validate() + if tc.expectError != "" { + require.EqualError(t, err, tc.expectError) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestRayNodesGetRun(t *testing.T) { + resources := corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("1Gi"), + util.ResourceNvidiaGPU: resource.MustParse("1"), + util.ResourceGoogleTPU: resource.MustParse("1"), + } + + pods := []runtime.Object{ + &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-1", + Name: "pod-1", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-1", + util.RayIsRayNodeLabelKey: "yes", + util.RayNodeTypeLabelKey: string(rayv1.HeadNode), + }, + CreationTimestamp: v1.NewTime(time.Now().Add(-1 * time.Hour)), + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-1", + Name: "pod-2", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-1", + util.RayIsRayNodeLabelKey: "yes", + util.RayNodeGroupLabelKey: "group-2", + util.RayNodeTypeLabelKey: string(rayv1.WorkerNode), + }, + CreationTimestamp: v1.NewTime(time.Now().Add(-2 * time.Hour)), + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-1", + Name: "pod-3", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-2", + util.RayIsRayNodeLabelKey: "yes", + util.RayNodeTypeLabelKey: string(rayv1.HeadNode), + }, + CreationTimestamp: v1.NewTime(time.Now().Add(-3 * time.Hour)), + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-2", + Name: "pod-1", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-1", + util.RayIsRayNodeLabelKey: "yes", + util.RayNodeGroupLabelKey: "group-1", + util.RayNodeTypeLabelKey: string(rayv1.WorkerNode), + }, + CreationTimestamp: v1.NewTime(time.Now().Add(-4 * time.Hour)), + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + } + + tests := []struct { + expectedOutput *string + name string + cluster string + namespace string + node string + expectedFixture string + expectedError string + pods []runtime.Object + allNamespaces bool + }{ + { + name: "no cluster; no namespace; no node", + allNamespaces: true, + pods: pods, + expectedFixture: "testdata/get-nodes-run-no-cluster-no-namespace-no-node.txt", + }, + { + name: "cluster; no namespace; no node", + cluster: "cluster-2", + allNamespaces: true, + pods: pods, + expectedFixture: "testdata/get-nodes-run-cluster-no-namespace-no-node.txt", + }, + { + name: "no cluster; namespace; no node", + namespace: "namespace-1", + allNamespaces: false, + pods: pods, + expectedFixture: "testdata/get-nodes-run-no-cluster-namespace-no-node.txt", + }, + { + name: "cluster; namespace; no node", + cluster: "cluster-1", + namespace: "namespace-1", + allNamespaces: false, + pods: pods, + expectedFixture: "testdata/get-nodes-run-cluster-namespace-no-node.txt", + }, + // We don't test for cases where the node is specified. + // We filter Pods by name with a field selector which k8s.io/client-go/kubernetes/fake doesn't support. + // See https://github.com/kubernetes/client-go/issues/326 + { + name: "node set but no Pods returned", + node: "pod-2", + allNamespaces: true, + pods: []runtime.Object{}, + expectedError: "Ray node pod-2 not found in any namespace in any Ray cluster", + }, + { + name: "no node set and no Pods returned", + pods: []runtime.Object{}, + expectedOutput: ptr.To(""), + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + testStreams, _, resBuf, _ := genericclioptions.NewTestIOStreams() + + fakeGetNodesOptions := GetNodesOptions{ + configFlags: genericclioptions.NewConfigFlags(true), + ioStreams: &testStreams, + cluster: tc.cluster, + namespace: tc.namespace, + allNamespaces: tc.allNamespaces, + node: tc.node, + } + + kubeClientSet := kubefake.NewClientset(tc.pods...) + rayClient := rayClientFake.NewSimpleClientset() + k8sClients := client.NewClientForTesting(kubeClientSet, rayClient) + + err := fakeGetNodesOptions.Run(context.Background(), k8sClients) + + if tc.expectedError != "" { + assert.EqualError(t, err, tc.expectedError) + return + } + + var expected []byte + if tc.expectedOutput != nil { + expected = []byte(*tc.expectedOutput) + } else if tc.expectedFixture != "" { + expected, err = os.ReadFile(tc.expectedFixture) + require.NoError(t, err) + } else { + t.Error("expectedOutput or expectedFixture must be set in test case") + } + + assert.Equal(t, string(expected), resBuf.String()) + }) + } +} + +func TestCreateLabelSelectors(t *testing.T) { + tests := []struct { + expected map[string]string + name string + cluster string + }{ + { + name: "should return the correct selectors if cluster isn't set", + cluster: "", + expected: map[string]string{ + util.RayIsRayNodeLabelKey: "yes", + }, + }, + { + name: "should return label selector for node name", + cluster: "my-cluster", + expected: map[string]string{ + util.RayIsRayNodeLabelKey: "yes", + util.RayClusterLabelKey: "my-cluster", + }, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + labelSelector := createLabelSelectors(tc.cluster) + assert.Equal(t, tc.expected, labelSelector) + }) + } +} + +func TestJoinLabelMap(t *testing.T) { + labels := map[string]string{ + "lain": "holsten", + "portia": "fabian", + } + output := joinLabelMap(labels) + parts := strings.Split(output, ",") + assert.Contains(t, parts, "lain=holsten") + assert.Contains(t, parts, "portia=fabian") +} + +func TestErrorMessageForNodeNotFound(t *testing.T) { + tests := []struct { + name string + cluster string + namespace string + expected string + allNamespaces bool + }{ + { + name: "neither cluster nor namespace are set", + allNamespaces: true, + expected: "Ray node my-node not found in any namespace in any Ray cluster", + }, + { + name: "cluster set, namespace not set", + cluster: "my-cluster", + allNamespaces: true, + expected: "Ray node my-node not found in any namespace in Ray cluster my-cluster", + }, + { + name: "cluster not set, namespace set", + namespace: "my-namespace", + allNamespaces: false, + expected: "Ray node my-node not found in namespace my-namespace in any Ray cluster", + }, + { + name: "both cluster and namespace are set", + cluster: "my-cluster", + namespace: "my-namespace", + allNamespaces: false, + expected: "Ray node my-node not found in namespace my-namespace in Ray cluster my-cluster", + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + msg := errorMessageForNodeNotFound("my-node", tc.cluster, tc.namespace, tc.allNamespaces) + assert.Equal(t, tc.expected, msg) + }) + } +} + +func TestPodsToNodes(t *testing.T) { + resources := corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("1Gi"), + util.ResourceNvidiaGPU: resource.MustParse("1"), + util.ResourceGoogleTPU: resource.MustParse("1"), + } + + pods := []corev1.Pod{ + { + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-1", + Name: "pod-1", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-1", + util.RayNodeGroupLabelKey: "group-1", + }, + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + { + ObjectMeta: v1.ObjectMeta{ + Namespace: "namespace-2", + Name: "pod-2", + Labels: map[string]string{ + util.RayClusterLabelKey: "cluster-2", + util.RayNodeGroupLabelKey: "group-2", + }, + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Requests: resources, + Limits: resources, + }, + }, + }, + }, + }, + } + + expectedNodes := []node{ + { + namespace: "namespace-1", + cluster: "cluster-1", + workerGroup: "group-1", + name: "pod-1", + cpus: *resources.Cpu(), + gpus: *resources.Name(util.ResourceNvidiaGPU, resource.DecimalSI), + tpus: *resources.Name(util.ResourceGoogleTPU, resource.DecimalSI), + memory: *resources.Memory(), + }, + { + namespace: "namespace-2", + cluster: "cluster-2", + workerGroup: "group-2", + name: "pod-2", + cpus: *resources.Cpu(), + gpus: *resources.Name(util.ResourceNvidiaGPU, resource.DecimalSI), + tpus: *resources.Name(util.ResourceGoogleTPU, resource.DecimalSI), + memory: *resources.Memory(), + }, + } + + assert.Equal(t, expectedNodes, podsToNodes(pods)) +} + +func TestPrintNodes(t *testing.T) { + resources := corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("1Gi"), + util.ResourceNvidiaGPU: resource.MustParse("1"), + util.ResourceGoogleTPU: resource.MustParse("1"), + } + + nodes := []node{ + { + namespace: "namespace-1", + cluster: "cluster-1", + _type: string(rayv1.HeadNode), + name: "pod-1", + cpus: *resources.Cpu(), + gpus: *resources.Name(util.ResourceNvidiaGPU, resource.DecimalSI), + tpus: *resources.Name(util.ResourceGoogleTPU, resource.DecimalSI), + memory: *resources.Memory(), + creationTimestamp: v1.NewTime(time.Now().Add(-1 * time.Hour)), + }, + { + namespace: "namespace-2", + cluster: "cluster-2", + _type: string(rayv1.WorkerNode), + workerGroup: "group-2", + name: "pod-2", + cpus: *resources.Cpu(), + gpus: *resources.Name(util.ResourceNvidiaGPU, resource.DecimalSI), + tpus: *resources.Name(util.ResourceGoogleTPU, resource.DecimalSI), + memory: *resources.Memory(), + creationTimestamp: v1.NewTime(time.Now().Add(-12 * time.Hour)), + }, + } + + tests := []struct { + name string + expectedFixture string + allNamespaces bool + }{ + { + name: "one namespace", + allNamespaces: false, + expectedFixture: "testdata/get-nodes-print-nodes.txt", + }, + { + name: "all namespaces", + allNamespaces: true, + expectedFixture: "testdata/get-nodes-print-nodes-with-namespaces.txt", + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + expected, err := os.ReadFile(tc.expectedFixture) + require.NoError(t, err) + + var output bytes.Buffer + err = printNodes(nodes, tc.allNamespaces, &output) + require.NoError(t, err) + + assert.Equal(t, string(expected), output.String()) + }) + } +} diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes-with-namespaces.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes-with-namespaces.txt new file mode 100644 index 00000000000..df426e8a999 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes-with-namespaces.txt @@ -0,0 +1,3 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +namespace-1 pod-1 1 1 1 1Gi cluster-1 head 60m +namespace-2 pod-2 1 1 1 1Gi cluster-2 worker group-2 12h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes.txt new file mode 100644 index 00000000000..c663299910f --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-print-nodes.txt @@ -0,0 +1,3 @@ +NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +pod-1 1 1 1 1Gi cluster-1 head 60m +pod-2 1 1 1 1Gi cluster-2 worker group-2 12h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-no-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-no-node.txt new file mode 100644 index 00000000000..f7856b23f54 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-no-node.txt @@ -0,0 +1,3 @@ +NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +pod-1 1 1 1 1Gi cluster-1 head 60m +pod-2 1 1 1 1Gi cluster-1 worker group-2 120m diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-node.txt new file mode 100644 index 00000000000..d3ce605d2f2 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-namespace-node.txt @@ -0,0 +1,3 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER WORKER GROUP AGE +namespace-1 pod-1 1 1 1 1Gi cluster-1 group-1 60m +namespace-2 pod-2 1 1 1 1Gi cluster-2 group-2 12h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-no-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-no-node.txt new file mode 100644 index 00000000000..c5e273736b5 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-no-node.txt @@ -0,0 +1,2 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +namespace-1 pod-3 1 1 1 1Gi cluster-2 head 3h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-node.txt new file mode 100644 index 00000000000..616a2d2a12a --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-cluster-no-namespace-node.txt @@ -0,0 +1,2 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER WORKER GROUP AGE +namespace-1 pod-1 1 1 1 1Gi cluster-1 group-1 60m diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-no-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-no-node.txt new file mode 100644 index 00000000000..c91bd43c84a --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-no-node.txt @@ -0,0 +1,4 @@ +NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +pod-1 1 1 1 1Gi cluster-1 head 60m +pod-2 1 1 1 1Gi cluster-1 worker group-2 120m +pod-3 1 1 1 1Gi cluster-2 head 3h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-node.txt new file mode 100644 index 00000000000..78c34ea6c90 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-namespace-node.txt @@ -0,0 +1,2 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER WORKER GROUP AGE +namespace-1 pod-2 1 1 1 1Gi cluster-1 group-2 2h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-no-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-no-node.txt new file mode 100644 index 00000000000..d71b35e792c --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-no-node.txt @@ -0,0 +1,5 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER TYPE WORKER GROUP AGE +namespace-1 pod-1 1 1 1 1Gi cluster-1 head 60m +namespace-1 pod-2 1 1 1 1Gi cluster-1 worker group-2 120m +namespace-1 pod-3 1 1 1 1Gi cluster-2 head 3h +namespace-2 pod-1 1 1 1 1Gi cluster-1 worker group-1 4h diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-node.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-node.txt new file mode 100644 index 00000000000..08ebecd14f8 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-cluster-no-namespace-node.txt @@ -0,0 +1,2 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER WORKER GROUP AGE +namespace-1 pod-2 1 1 1 1Gi cluster-1 group-2 120m diff --git a/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-node-no-pods.txt b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-node-no-pods.txt new file mode 100644 index 00000000000..ae4dd94c9e9 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/testdata/get-nodes-run-no-node-no-pods.txt @@ -0,0 +1 @@ +NAMESPACE NAME CPUS GPUS TPUS MEMORY CLUSTER WORKER GROUP AGE diff --git a/kubectl-plugin/pkg/util/constant.go b/kubectl-plugin/pkg/util/constant.go index 1c92dc30e54..4bea548de75 100644 --- a/kubectl-plugin/pkg/util/constant.go +++ b/kubectl-plugin/pkg/util/constant.go @@ -3,4 +3,12 @@ package util const ( RayVersion = "2.41.0" RayImage = "rayproject/ray:" + RayVersion + + RayClusterLabelKey = "ray.io/cluster" + RayIsRayNodeLabelKey = "ray.io/is-ray-node" + RayNodeGroupLabelKey = "ray.io/group" + RayNodeTypeLabelKey = "ray.io/node-type" + + ResourceNvidiaGPU = "nvidia.com/gpu" + ResourceGoogleTPU = "google.com/tpu" )