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/21 15:34:52 UTC

[GitHub] [yunikorn-k8shim] manirajv06 opened a new pull request, #507: [YUNIKORN-1492] Add gang scheduling test suites

manirajv06 opened a new pull request, #507:
URL: https://github.com/apache/yunikorn-k8shim/pull/507

   ### What is this PR for?
   
   Add gang scheduling test methods to cover all the flows. Added/Modified helper methods as needed.
   
   
   ### What type of PR is it?
   * [ ] - Improvement
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1492
   
   ### How should this be tested?
   
   ### Screenshots (if appropriate)
   
   ### Questions:
   * [ ] - The licenses files need update.
   * [ ] - There is breaking changes for older versions.
   * [ ] - It needs documentation.
   


-- 
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


[GitHub] [yunikorn-k8shim] wilfred-s closed pull request #507: [YUNIKORN-1492] Add gang scheduling test suites

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #507: [YUNIKORN-1492] Add gang scheduling test suites
URL: https://github.com/apache/yunikorn-k8shim/pull/507


-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [yunikorn-k8shim] codecov[bot] commented on pull request #507: [YUNIKORN-1492] Add gang scheduling test suites

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #507:
URL: https://github.com/apache/yunikorn-k8shim/pull/507#issuecomment-1362721616

   # [Codecov](https://codecov.io/gh/apache/yunikorn-k8shim/pull/507?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#507](https://codecov.io/gh/apache/yunikorn-k8shim/pull/507?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d69e872) into [master](https://codecov.io/gh/apache/yunikorn-k8shim/commit/8dad421916fc0693adf0ac002e15a2ea8ee84d17?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8dad421) will **not change** coverage.
   > The diff coverage is `n/a`.
   
   ```diff
   @@           Coverage Diff           @@
   ##           master     #507   +/-   ##
   =======================================
     Coverage   69.41%   69.41%           
   =======================================
     Files          45       45           
     Lines        7714     7714           
   =======================================
     Hits         5355     5355           
     Misses       2162     2162           
     Partials      197      197           
   ```
   
   
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on PR #507:
URL: https://github.com/apache/yunikorn-k8shim/pull/507#issuecomment-1366255927

   Can you check the call you do in the before suite? looks like the REAT call is failing in all setups.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on PR #507:
URL: https://github.com/apache/yunikorn-k8shim/pull/507#issuecomment-1362223246

   I understand that this is still a draft but these comments should be solved before moving out of draft stage:
   * license headers for all files
   * no filenames with capitals (Wrappers.go)
   * sort imports in 3 groups based on origin: golang system, others packages and yunikorn package imports
   * Remove any company specific info or messages


-- 
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