You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by GitBox <gi...@apache.org> on 2022/12/30 07:06:31 UTC

[GitHub] [yunikorn-k8shim] wilfred-s commented on a diff in pull request #507: [YUNIKORN-1492] Add gang scheduling test suites

wilfred-s commented on code in PR #507:
URL: https://github.com/apache/yunikorn-k8shim/pull/507#discussion_r1059266841


##########
test/e2e/Wrappers.go:
##########
@@ -0,0 +1,143 @@
+/*
+ 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.
+*/
+
+package e2e
+
+import (
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"time"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/gomega"
+	"gopkg.in/yaml.v2"
+	v1 "k8s.io/api/core/v1"
+
+	"github.com/apache/yunikorn-core/pkg/common/configs"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/configmanager"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/common"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
+)
+
+var k = k8s.KubeCtl{}
+
+func UpdateConfigMapWrapper(oldConfigMap *v1.ConfigMap, schedPolicy string, annotation string) {
+	By("Enabling new scheduling config")
+
+	// Save old configMap
+	Ω(k.SetClient()).To(BeNil())
+	var c, err = k.GetConfigMaps(configmanager.YuniKornTestConfig.YkNamespace,
+		configmanager.GetConfigMapName())
+	Ω(err).NotTo(HaveOccurred())
+	Ω(c).NotTo(BeNil())
+	c.DeepCopyInto(oldConfigMap)
+	Ω(c).Should(BeEquivalentTo(oldConfigMap))
+
+	// Create fair configMap with timestamp
+	sc := common.CreateBasicConfigMap()
+	if schedPolicy != "" {
+		err = common.SetSchedulingPolicy(sc, "default", "root", schedPolicy)
+		Ω(err).NotTo(HaveOccurred())
+	}
+	ts, tsErr := common.SetQueueTimestamp(sc, "default", "root")
+	Ω(tsErr).NotTo(HaveOccurred())
+	configStr, yamlErr := common.ToYAML(sc)
+	Ω(yamlErr).NotTo(HaveOccurred())
+	c.Data[configmanager.DefaultPolicyGroup] = configStr
+	var d, err3 = k.UpdateConfigMap(c, configmanager.YuniKornTestConfig.YkNamespace)
+	Ω(err3).NotTo(HaveOccurred())
+	Ω(d).NotTo(BeNil())
+
+	// Updating scheduler pod annotation to trigger force refresh of configmaps
+	// https://jira.cloudera.com/browse/COMPX-4042
+	Ω(k.UpdateYunikornSchedulerPodAnnotation(annotation)).NotTo(HaveOccurred())
+	err = yunikorn.WaitForQueueTS("root", ts, 2*time.Minute)
+	Ω(err).NotTo(HaveOccurred())
+}
+
+func RestoreConfigMapWrapper(oldConfigMap *v1.ConfigMap, annotation string) {
+	Ω(k.SetClient()).To(BeNil())
+	By("Restoring the old config maps")
+	var c, err = k.GetConfigMaps(configmanager.YuniKornTestConfig.YkNamespace,
+		configmanager.GetConfigMapName())
+	Ω(err).NotTo(HaveOccurred())
+	Ω(c).NotTo(BeNil())
+
+	oldSC := new(configs.SchedulerConfig)
+	err = yaml.Unmarshal([]byte(oldConfigMap.Data[configmanager.DefaultPolicyGroup]), oldSC)
+	Ω(err).NotTo(HaveOccurred())
+	ts, tsErr := common.SetQueueTimestamp(oldSC, "default", "root")
+	Ω(tsErr).NotTo(HaveOccurred())
+	c.Data[configmanager.DefaultPolicyGroup], err = common.ToYAML(oldSC)
+	Ω(err).NotTo(HaveOccurred())
+
+	var e, err3 = k.UpdateConfigMap(c, configmanager.YuniKornTestConfig.YkNamespace)
+	Ω(err3).NotTo(HaveOccurred())
+	Ω(e).NotTo(BeNil())
+	// Updating scheduler pod annotation to trigger force refresh of configmaps
+	// https://jira.cloudera.com/browse/COMPX-4042
+	Ω(k.RemoveYunikornSchedulerPodAnnotation(annotation)).NotTo(HaveOccurred())

Review Comment:
   This should not be needed anymore with the new config map listener
   Internal jira reference must be removed



##########
test/e2e/Wrappers.go:
##########
@@ -0,0 +1,143 @@
+/*
+ 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.
+*/
+
+package e2e
+
+import (
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"time"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/gomega"
+	"gopkg.in/yaml.v2"
+	v1 "k8s.io/api/core/v1"
+
+	"github.com/apache/yunikorn-core/pkg/common/configs"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/configmanager"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/common"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
+)
+
+var k = k8s.KubeCtl{}
+
+func UpdateConfigMapWrapper(oldConfigMap *v1.ConfigMap, schedPolicy string, annotation string) {
+	By("Enabling new scheduling config")
+
+	// Save old configMap
+	Ω(k.SetClient()).To(BeNil())
+	var c, err = k.GetConfigMaps(configmanager.YuniKornTestConfig.YkNamespace,
+		configmanager.GetConfigMapName())
+	Ω(err).NotTo(HaveOccurred())
+	Ω(c).NotTo(BeNil())
+	c.DeepCopyInto(oldConfigMap)
+	Ω(c).Should(BeEquivalentTo(oldConfigMap))
+
+	// Create fair configMap with timestamp
+	sc := common.CreateBasicConfigMap()
+	if schedPolicy != "" {
+		err = common.SetSchedulingPolicy(sc, "default", "root", schedPolicy)
+		Ω(err).NotTo(HaveOccurred())
+	}
+	ts, tsErr := common.SetQueueTimestamp(sc, "default", "root")
+	Ω(tsErr).NotTo(HaveOccurred())
+	configStr, yamlErr := common.ToYAML(sc)
+	Ω(yamlErr).NotTo(HaveOccurred())
+	c.Data[configmanager.DefaultPolicyGroup] = configStr
+	var d, err3 = k.UpdateConfigMap(c, configmanager.YuniKornTestConfig.YkNamespace)
+	Ω(err3).NotTo(HaveOccurred())
+	Ω(d).NotTo(BeNil())
+
+	// Updating scheduler pod annotation to trigger force refresh of configmaps
+	// https://jira.cloudera.com/browse/COMPX-4042
+	Ω(k.UpdateYunikornSchedulerPodAnnotation(annotation)).NotTo(HaveOccurred())

Review Comment:
   This should not be needed anymore with the new config map listener.
   Internal jira reference must be removed



##########
test/e2e/framework/helpers/k8s/k8s_utils.go:
##########
@@ -391,6 +414,24 @@ func (k *KubeCtl) ConfigMapExists(name string, namespace string) (bool, error) {
 	return true, nil
 }
 
+// Filters out master/liftie infra nodes. Gets schedulable nodes

Review Comment:
   `master` or `control-plane` nodes: internal reference to liftie



##########
test/e2e/framework/helpers/yunikorn/rest_api_utils.go:
##########
@@ -174,13 +191,13 @@ func (c *RClient) WaitForAllocationLog(partition string, queueName string, appID
 	return nil
 }
 
-func (c *RClient) isAppInDesiredState(partition string, queueName string, appID string, state string) wait.ConditionFunc {
+func (c *RClient) isAppInDesiredState(partition string, queue string, appID string, state string) wait.ConditionFunc {
 	return func() (bool, error) {
-		appInfo, err := c.GetAppInfo(partition, queueName, appID)
+		appInfo, err := c.GetAppInfo(partition, queue, appID)
 		if err != nil {
 			return false, nil // returning nil here for wait & loop
 		}
-
+		fmt.Printf("current application state is %+v, required app state is %+v\n", appInfo.State, state)

Review Comment:
   remove rogue `fmt.Printf` statement



##########
test/e2e/framework/configmanager/constants.go:
##########
@@ -54,5 +54,13 @@ const (
 	DefaultYuniKornScheme = "http"
 
 	DefaultYuniKornConfigMap = "yunikorn-configs"
+	DefaultPluginConfigMap   = "yunikorn-configs"
 	DefaultPolicyGroup       = "queues.yaml"
 )
+
+func GetConfigMapName() string {
+	if YuniKornTestConfig.Plugin {
+		return DefaultPluginConfigMap
+	}
+	return DefaultYuniKornConfigMap
+}

Review Comment:
   The config names do not differ between plugin and default mode.
   Code overhead is not needed.



##########
test/e2e/gang_scheduling/gang_scheduling_test.go:
##########
@@ -0,0 +1,540 @@
+/*
+ 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.
+*/
+
+package gangscheduling_test
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/onsi/gomega"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/api/resource"
+
+	"github.com/apache/yunikorn-k8shim/pkg/apis/yunikorn.apache.org/v1alpha1"
+	tests "github.com/apache/yunikorn-k8shim/test/e2e"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/common"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
+)
+
+var _ = Describe("", func() {
+	var kClient k8s.KubeCtl //nolint
+	var restClient yunikorn.RClient
+	var ns string
+
+	BeforeEach(func() {
+		kClient = k8s.KubeCtl{}
+		Ω(kClient.SetClient()).To(BeNil())
+		ns = "ns-" + common.RandSeq(10)
+		By(fmt.Sprintf("Creating namespace: %s for sleep jobs", ns))
+		var ns1, err1 = kClient.CreateNamespace(ns, nil)
+		Ω(err1).NotTo(HaveOccurred())
+		Ω(ns1.Status.Phase).To(Equal(v1.NamespaceActive))
+	})
+
+	It("Verify_Annotation_TaskGroup_Def", func() {
+		/*
+			1. Deploy 1 job with tg definition
+			3. Poll for 5 placeholders
+			4. App running
+			5. Deploy 1 job with 5 real pods of taskgroup
+			6. Check placeholders deleted
+			7. Real pods running and app running
+		*/

Review Comment:
   place test description before the `It(....)`
   use multiple single line `//` comments instead of a multi line comment`/* */`
   apply to all tests in the file



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org