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 2020/08/04 01:46:58 UTC

[GitHub] [incubator-yunikorn-k8shim] ayubpathan opened a new pull request #171: [YUNIKORN-344]: Add tests for queue quota management

ayubpathan opened a new pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171


   And code refactoring..


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

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



[GitHub] [incubator-yunikorn-k8shim] ayubpathan commented on a change in pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
ayubpathan commented on a change in pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#discussion_r465362047



##########
File path: test/e2e/QueueQuotaMgmt/QueueQuotaMgmt_suite_test.go
##########
@@ -0,0 +1,104 @@
+/*
+ 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 queuequotamgmt_test
+
+import (
+	"path/filepath"
+	"testing"
+
+	v1 "k8s.io/api/core/v1"
+
+	"github.com/onsi/ginkgo/reporters"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/ginkgo/extensions/table"
+	"github.com/onsi/gomega"
+
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/configmanager"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+)
+
+func init() {
+	configmanager.YuniKornTestConfig.ParseFlags()
+}
+
+var k = k8s.KubeCtl{}
+var oldConfigMap *v1.ConfigMap
+
+var _ = BeforeSuite(func() {
+	By("Enable basic scheduling config over config maps")
+	Ω(k.SetClient()).To(BeNil())
+	var c, err = k.GetConfigMaps(configmanager.YuniKornTestConfig.YkNamespace,
+		configmanager.DefaultYuniKornConfigMap)
+	Ω(err).NotTo(HaveOccurred())
+	Ω(c).NotTo(BeNil())
+
+	oldConfigMap = c.DeepCopy()
+	Ω(c).Should(BeEquivalentTo(oldConfigMap))
+	var configStr = "partitions:\n  -\n    name: default\n    placementrules:\n      - name: tag\n        " +

Review comment:
       Ok.




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

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



[GitHub] [incubator-yunikorn-k8shim] ayubpathan commented on pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
ayubpathan commented on pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#issuecomment-668863081


   Thanks for the review @yangwwei 


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

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



[GitHub] [incubator-yunikorn-k8shim] ayubpathan commented on a change in pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
ayubpathan commented on a change in pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#discussion_r465372490



##########
File path: test/e2e/BasicScheduling/BasicScheduling_test.go
##########
@@ -0,0 +1,115 @@
+/*
+ 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 basicscheduling_test
+
+import (
+	"fmt"
+	"regexp"
+	"strconv"
+
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/common"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/gomega"
+	v1 "k8s.io/api/core/v1"
+)
+
+var _ = ginkgo.Describe("", func() {
+	var kClient k8s.KubeCtl
+	var restClient yunikorn.RClient
+	var sleepPodDef string
+	var err error
+	var sleepRespPod *v1.Pod
+	var dev = "dev" + common.RandSeq(5)
+	var appsInfo map[string]interface{}
+	var r = regexp.MustCompile(`memory:(\d+) vcore:(\d+)`)
+
+	ginkgo.BeforeSuite(func() {
+		// Initializing kubectl client
+		sleepPodDef, err = common.GetAbsPath("../testdata/sleeppod_template.yaml")
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		kClient = k8s.KubeCtl{}
+		gomega.Ω(kClient.SetClient()).To(gomega.BeNil())
+		// Initializing rest client
+		restClient = yunikorn.RClient{}
+		ginkgo.By("create development namespace")
+		ns1, err := kClient.CreateNamespace(dev, nil)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		gomega.Ω(ns1.Status.Phase).To(gomega.Equal(v1.NamespaceActive))
+
+		ginkgo.By("Deploy the sleep pod to the development namespace")
+		sleepObj, err := k8s.GetPodObj(sleepPodDef)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		sleepObj.Namespace = dev
+		sleepObj.ObjectMeta.Labels["applicationId"] = common.GetUUID()
+		sleepRespPod, err = kClient.CreatePod(sleepObj, dev)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		//Wait for pod to move to running state
+		err = kClient.WaitForPodBySelectorRunning(dev,
+			fmt.Sprintf("app=%s", sleepRespPod.ObjectMeta.Labels["app"]),
+			10)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+
+		appsInfo, err = restClient.GetAppInfo(sleepRespPod.ObjectMeta.Labels["applicationId"])
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		gomega.Ω(appsInfo).NotTo(gomega.BeNil())
+	})
+
+	ginkgo.It("Verify_App_Queue_Info", func() {
+		ginkgo.By("Verify that the sleep pod is mapped to development queue")
+		gomega.Ω(appsInfo["applicationID"]).To(gomega.Equal(sleepRespPod.ObjectMeta.Labels["applicationId"]))
+		gomega.Ω(appsInfo["queueName"]).To(gomega.ContainSubstring(sleepRespPod.ObjectMeta.Namespace))
+	})
+
+	ginkgo.It("Verify_Job_State", func() {
+		ginkgo.By("Verify that the job is scheduled & starting by YuniKorn")
+		gomega.Ω(appsInfo["applicationState"]).To(gomega.Equal("Starting"))
+		gomega.Ω("yunikorn").To(gomega.Equal(sleepRespPod.Spec.SchedulerName))
+	})

Review comment:
       ok




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

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



[GitHub] [incubator-yunikorn-k8shim] ayubpathan commented on pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
ayubpathan commented on pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#issuecomment-668383831


   @yangwwei  The tests are failing with below error.. 
   `  
   Unexpected error:
         <*errors.StatusError | 0xc000264240>: {
             ErrStatus: {
                 TypeMeta: {Kind: "", APIVersion: ""},
                 ListMeta: {SelfLink: "", ResourceVersion: "", Continue: ""},
                 Status: "Failure",
                 Message: "admission webhook \"admission-webhook.yunikorn.validate-conf\" denied the request: Post http://yunikorn-service.yunikorn.svc:9080/ws/v1/validate-conf: dial tcp: lookup yunikorn-service.yunikorn.svc on 10.96.0.10:53: write udp 172.18.0.2:53050->10.96.0.10:53: write: operation not permitted",
                 Reason: "",
                 Details: nil,
                 Code: 400,
             },
         }
   `
   
   Seems like some commits are missing, can you please check?


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

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



[GitHub] [incubator-yunikorn-k8shim] yangwwei merged pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
yangwwei merged pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171


   


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

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



[GitHub] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#issuecomment-669362804


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=h1) Report
   > Merging [#171](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/20a1f09f25714db9ee3805f65b01bb8cacda54f1&el=desc) will **increase** coverage by `0.06%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #171      +/-   ##
   ==========================================
   + Coverage   57.55%   57.62%   +0.06%     
   ==========================================
     Files          32       32              
     Lines        3155     3155              
   ==========================================
   + Hits         1816     1818       +2     
   + Misses       1270     1268       -2     
     Partials       69       69              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/dispatcher/dispatcher.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/diff?src=pr&el=tree#diff-cGtnL2Rpc3BhdGNoZXIvZGlzcGF0Y2hlci5nbw==) | `87.15% <0.00%> (+1.83%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=footer). Last update [20a1f09...26327a7](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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



[GitHub] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#issuecomment-669362804


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=h1) Report
   > Merging [#171](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/20a1f09f25714db9ee3805f65b01bb8cacda54f1&el=desc) will **increase** coverage by `0.06%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #171      +/-   ##
   ==========================================
   + Coverage   57.55%   57.62%   +0.06%     
   ==========================================
     Files          32       32              
     Lines        3155     3155              
   ==========================================
   + Hits         1816     1818       +2     
   + Misses       1270     1268       -2     
     Partials       69       69              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/dispatcher/dispatcher.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/diff?src=pr&el=tree#diff-cGtnL2Rpc3BhdGNoZXIvZGlzcGF0Y2hlci5nbw==) | `87.15% <0.00%> (+1.83%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=footer). Last update [20a1f09...26327a7](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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



[GitHub] [incubator-yunikorn-k8shim] yangwwei commented on a change in pull request #171: [YUNIKORN-344]: Add tests for queue quota management

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #171:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/171#discussion_r465328323



##########
File path: test/e2e/framework/helpers/k8s/k8s_utils.go
##########
@@ -88,38 +91,52 @@ func (k *KubeCtl) SetClient() error {
 	return err
 }
 
+func (k *KubeCtl) GetKubeConfig() (*rest.Config, error) {
+	if k.kubeConfig != nil {
+		return k.kubeConfig, nil
+	}
+	return nil, err

Review comment:
       why there is a global `err` defined in this file?
   why this returns that global `err` if kubeconfig is nil ?

##########
File path: test/e2e/BasicScheduling/BasicScheduling_test.go
##########
@@ -0,0 +1,115 @@
+/*
+ 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 basicscheduling_test
+
+import (
+	"fmt"
+	"regexp"
+	"strconv"
+
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/common"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/gomega"
+	v1 "k8s.io/api/core/v1"
+)
+
+var _ = ginkgo.Describe("", func() {
+	var kClient k8s.KubeCtl
+	var restClient yunikorn.RClient
+	var sleepPodDef string
+	var err error
+	var sleepRespPod *v1.Pod
+	var dev = "dev" + common.RandSeq(5)
+	var appsInfo map[string]interface{}
+	var r = regexp.MustCompile(`memory:(\d+) vcore:(\d+)`)
+
+	ginkgo.BeforeSuite(func() {
+		// Initializing kubectl client
+		sleepPodDef, err = common.GetAbsPath("../testdata/sleeppod_template.yaml")
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		kClient = k8s.KubeCtl{}
+		gomega.Ω(kClient.SetClient()).To(gomega.BeNil())
+		// Initializing rest client
+		restClient = yunikorn.RClient{}
+		ginkgo.By("create development namespace")
+		ns1, err := kClient.CreateNamespace(dev, nil)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		gomega.Ω(ns1.Status.Phase).To(gomega.Equal(v1.NamespaceActive))
+
+		ginkgo.By("Deploy the sleep pod to the development namespace")
+		sleepObj, err := k8s.GetPodObj(sleepPodDef)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		sleepObj.Namespace = dev
+		sleepObj.ObjectMeta.Labels["applicationId"] = common.GetUUID()
+		sleepRespPod, err = kClient.CreatePod(sleepObj, dev)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		//Wait for pod to move to running state
+		err = kClient.WaitForPodBySelectorRunning(dev,
+			fmt.Sprintf("app=%s", sleepRespPod.ObjectMeta.Labels["app"]),
+			10)
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+
+		appsInfo, err = restClient.GetAppInfo(sleepRespPod.ObjectMeta.Labels["applicationId"])
+		gomega.Ω(err).NotTo(gomega.HaveOccurred())
+		gomega.Ω(appsInfo).NotTo(gomega.BeNil())
+	})
+
+	ginkgo.It("Verify_App_Queue_Info", func() {
+		ginkgo.By("Verify that the sleep pod is mapped to development queue")
+		gomega.Ω(appsInfo["applicationID"]).To(gomega.Equal(sleepRespPod.ObjectMeta.Labels["applicationId"]))
+		gomega.Ω(appsInfo["queueName"]).To(gomega.ContainSubstring(sleepRespPod.ObjectMeta.Namespace))
+	})
+
+	ginkgo.It("Verify_Job_State", func() {
+		ginkgo.By("Verify that the job is scheduled & starting by YuniKorn")
+		gomega.Ω(appsInfo["applicationState"]).To(gomega.Equal("Starting"))
+		gomega.Ω("yunikorn").To(gomega.Equal(sleepRespPod.Spec.SchedulerName))
+	})

Review comment:
       This test may break if we use a different sorting policy other than `StateAware`.
   Since this is the `BasicSchedulingTest`, I suggest to remove it.

##########
File path: test/e2e/QueueQuotaMgmt/QueueQuotaMgmt_suite_test.go
##########
@@ -0,0 +1,104 @@
+/*
+ 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 queuequotamgmt_test
+
+import (
+	"path/filepath"
+	"testing"
+
+	v1 "k8s.io/api/core/v1"
+
+	"github.com/onsi/ginkgo/reporters"
+
+	"github.com/onsi/ginkgo"
+	"github.com/onsi/ginkgo/extensions/table"
+	"github.com/onsi/gomega"
+
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/configmanager"
+	"github.com/apache/incubator-yunikorn-k8shim/test/e2e/framework/helpers/k8s"
+)
+
+func init() {
+	configmanager.YuniKornTestConfig.ParseFlags()
+}
+
+var k = k8s.KubeCtl{}
+var oldConfigMap *v1.ConfigMap
+
+var _ = BeforeSuite(func() {
+	By("Enable basic scheduling config over config maps")
+	Ω(k.SetClient()).To(BeNil())
+	var c, err = k.GetConfigMaps(configmanager.YuniKornTestConfig.YkNamespace,
+		configmanager.DefaultYuniKornConfigMap)
+	Ω(err).NotTo(HaveOccurred())
+	Ω(c).NotTo(BeNil())
+
+	oldConfigMap = c.DeepCopy()
+	Ω(c).Should(BeEquivalentTo(oldConfigMap))
+	var configStr = "partitions:\n  -\n    name: default\n    placementrules:\n      - name: tag\n        " +

Review comment:
       Could you pls check if the config can be defined like: https://github.com/apache/incubator-yunikorn-core/blob/357d71fa290619e583b5da0484215c5d927807b7/pkg/scheduler/tests/scheduler_smoke_test.go#L41-L52
   
   




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

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



[GitHub] [incubator-yunikorn-k8shim] codecov[bot] commented on pull request #171: [YUNIKORN-344]: Add tests for queue quota management

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=h1) Report
   > Merging [#171](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/20a1f09f25714db9ee3805f65b01bb8cacda54f1&el=desc) will **decrease** coverage by `0.31%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #171      +/-   ##
   ==========================================
   - Coverage   57.55%   57.24%   -0.32%     
   ==========================================
     Files          32       32              
     Lines        3155     3155              
   ==========================================
   - Hits         1816     1806      -10     
   - Misses       1270     1279       +9     
   - Partials       69       70       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171/diff?src=pr&el=tree#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `67.78% <0.00%> (-4.19%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=footer). Last update [20a1f09...26327a7](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/171?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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