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 2021/12/08 19:59:03 UTC

[GitHub] [incubator-yunikorn-k8shim] craigcondit opened a new pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   ### What is this PR for?
   This PR implements a new build mode for YuniKorn which allows it to run as a set of plugins on top of the default Kubernetes scheduler. This should allow for closer behavior to the default scheduler as well as the ability to bypass YuniKorn-specific functionality for pods which are not annotated with an `applicationId` label.
   
   This PR makes no functional changes to the default YuniKorn build, but adds new Makefile targets to support build / running YuniKorn as a scheduler plugin:
   
   - `build_plugin` Build the plugin code.
   - `run_plugin` Run the plugin code locally.
   - `plugin_image` Create a Docker image from the plugin code. this should mostly be a drop-in replacement for the default YuniKorn image.
   
   This feature should be considered `experimental` at this stage. Hopefully we can provide it as a tech-preview for 1.0.
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [ ] - Improvement
   * [x] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-971
   
   ### How should this be tested?
   Some unit tests added for configuration, remainder is covered via e2e tests (all passing).
   
   ### 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] [incubator-yunikorn-k8shim] craigcondit closed pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

Posted by GitBox <gi...@apache.org>.
craigcondit closed pull request #336:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/336


   


-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -132,6 +132,45 @@ func (cache *SchedulerCache) removeNode(node *v1.Node) error {
 	return nil
 }
 
+func (cache *SchedulerCache) AddPendingPodAllocation(podKey string, nodeID string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.inProgressAllocations, podKey)
+	cache.pendingAllocations[podKey] = nodeID
+}
+
+func (cache *SchedulerCache) RemovePendingPodAllocation(podKey string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.pendingAllocations, podKey)
+	delete(cache.inProgressAllocations, podKey)
+}
+
+func (cache *SchedulerCache) GetPendingPodAllocation(podKey string) (nodeID string, ok bool) {
+	cache.lock.RLock()
+	defer cache.lock.RUnlock()
+	res, ok := cache.pendingAllocations[podKey]
+	return res, ok
+}
+
+func (cache *SchedulerCache) GetInProgressPodAllocation(podKey string) (nodeID string, ok bool) {
+	cache.lock.RLock()
+	defer cache.lock.RUnlock()
+	res, ok := cache.inProgressAllocations[podKey]
+	return res, ok
+}
+
+func (cache *SchedulerCache) StartPodAllocation(podKey string, nodeID string) bool {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	expectedNodeID, ok := cache.pendingAllocations[podKey]
+	if ok && expectedNodeID == nodeID {

Review comment:
       Ok, from what I understand `expectedNodeID` comes from YK itself, and `nodeID` from the default scheduler. Questions:
   * How often do these two mismatch?
   * What happens if we return `false` here?
   * Can YK and the def. scheduler end up repeatedly allocating to different nodes? 
   
   Another thing is that do we even need to consider YK's allocation? Because in `task.postTaskAllocated()`, the original code path calls `KubeClient.Bind()` but in plugin mode, we delegate this operation as I can see. 




-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,209 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+// YuniKornSchedulerPlugin provides an implementation of several lifecycle methods of the Kubernetes scheduling framework:
+//   https://kubernetes.io/docs/concepts/scheduling-eviction/scheduling-framework/
+//
+// PreFilter: Used to notify the default scheduler that a particular pod has been marked ready for scheduling by YuniKorn
+//
+// Filter: Used to notify the default scheduler that a particular pod/node combination is ready to be scheduled
+//
+// PostBind: Used to notify YuniKorn that a pod has been scheduled successfully
+//
+// Pod Allocations:
+//
+// The YuniKorn scheduler is always running in the background, making decisions about which pods to allocate to which
+// nodes. When a decision is made, that pod is marked as having a "pending" pod allocation, which means YuniKorn has
+// allocated the pod, but the default scheduler (via the plugin interface) has not yet been notified.
+//
+// Once PreFilter() has been called for a particular pod, that allocation is marked as "in progress" meaning it has been
+// communicated to the default scheduler, but has not yet been fulfilled.
+//
+// Finally, in PostBind(), the allocation is removed as we now know that the pod has been allocated successfully.
+// If a pending or in-progress allocation is detected for a pod in PreFilter(), we remove the allocation and force the
+// pod to be rescheduled, as this means the prior allocation could not be completed successfully by the default
+// scheduler for some reason.
+type YuniKornSchedulerPlugin struct {
+	sync.RWMutex
+	context *cache.Context
+}
+
+// ensure all required interfaces are implemented
+var _ framework.PreFilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.FilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.PostBindPlugin = &YuniKornSchedulerPlugin{}
+
+// Name returns the name of the plugin
+func (sp *YuniKornSchedulerPlugin) Name() string {
+	return SchedulerPluginName
+}
+
+// PreFilter is used to release pods to scheduler
+func (sp *YuniKornSchedulerPlugin) PreFilter(_ context.Context, _ *framework.CycleState, pod *v1.Pod) *framework.Status {
+	log.Logger().Debug("PreFilter check",
+		zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)))
+
+	// we don't process pods without appID defined
+	appID, err := utils.GetApplicationIDFromPod(pod)
+	if err != nil {
+		log.Logger().Info(fmt.Sprintf("Skipping pod %s/%s in the prefilter plugin because no applicationID is defined",
+			pod.Namespace, pod.Name))
+		return framework.NewStatus(framework.Success, "Deferring to default scheduler")
+	}
+
+	if app := sp.context.GetApplication(appID); app != nil {
+		if task, err := app.GetTask(string(pod.UID)); err == nil {
+			_, ok := sp.context.GetInProgressPodAllocation(string(pod.UID))
+			if ok {
+				// pod must have failed scheduling, reject it and return unschedulable
+				log.Logger().Info("Task failed scheduling, marking as rejected",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()))
+				sp.context.RemovePodAllocation(string(pod.UID))
+				dispatcher.Dispatch(cache.NewRejectTaskEvent(app.GetApplicationID(), task.GetTaskID(),
+					fmt.Sprintf("task %s rejected by scheduler", task.GetTaskID())))
+				return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+			}
+
+			nodeID, ok := sp.context.GetPendingPodAllocation(string(pod.UID))
+			if task.GetTaskState() == events.States().Task.Bound && ok {
+				log.Logger().Info("Releasing pod for scheduling (prefilter phase)",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()),
+					zap.String("assignedNode", nodeID))
+
+				return framework.NewStatus(framework.Success, "")
+			}
+		}
+	}
+
+	return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+}
+
+// PreFilterExtensions is unused
+func (sp *YuniKornSchedulerPlugin) PreFilterExtensions() framework.PreFilterExtensions {
+	return nil
+}
+
+// Filter is used to release specific pod/node combinations to scheduler

Review comment:
       The thing that's probably worth mentioning here is what happens when `expectedNode != node`, pointing out the minimal performance impact and the immediate invocation of `Filter()` again. You can probably also add that currently this is the only way to force the default scheduler to match YK's decision.




-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,209 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+// YuniKornSchedulerPlugin provides an implementation of several lifecycle methods of the Kubernetes scheduling framework:
+//   https://kubernetes.io/docs/concepts/scheduling-eviction/scheduling-framework/
+//
+// PreFilter: Used to notify the default scheduler that a particular pod has been marked ready for scheduling by YuniKorn
+//
+// Filter: Used to notify the default scheduler that a particular pod/node combination is ready to be scheduled
+//
+// PostBind: Used to notify YuniKorn that a pod has been scheduled successfully
+//
+// Pod Allocations:
+//
+// The YuniKorn scheduler is always running in the background, making decisions about which pods to allocate to which
+// nodes. When a decision is made, that pod is marked as having a "pending" pod allocation, which means YuniKorn has
+// allocated the pod, but the default scheduler (via the plugin interface) has not yet been notified.
+//
+// Once PreFilter() has been called for a particular pod, that allocation is marked as "in progress" meaning it has been
+// communicated to the default scheduler, but has not yet been fulfilled.
+//
+// Finally, in PostBind(), the allocation is removed as we now know that the pod has been allocated successfully.
+// If a pending or in-progress allocation is detected for a pod in PreFilter(), we remove the allocation and force the
+// pod to be rescheduled, as this means the prior allocation could not be completed successfully by the default
+// scheduler for some reason.
+type YuniKornSchedulerPlugin struct {
+	sync.RWMutex
+	context *cache.Context
+}
+
+// ensure all required interfaces are implemented
+var _ framework.PreFilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.FilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.PostBindPlugin = &YuniKornSchedulerPlugin{}
+
+// Name returns the name of the plugin
+func (sp *YuniKornSchedulerPlugin) Name() string {
+	return SchedulerPluginName
+}
+
+// PreFilter is used to release pods to scheduler
+func (sp *YuniKornSchedulerPlugin) PreFilter(_ context.Context, _ *framework.CycleState, pod *v1.Pod) *framework.Status {
+	log.Logger().Debug("PreFilter check",
+		zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)))
+
+	// we don't process pods without appID defined
+	appID, err := utils.GetApplicationIDFromPod(pod)
+	if err != nil {
+		log.Logger().Info(fmt.Sprintf("Skipping pod %s/%s in the prefilter plugin because no applicationID is defined",
+			pod.Namespace, pod.Name))
+		return framework.NewStatus(framework.Success, "Deferring to default scheduler")
+	}
+
+	if app := sp.context.GetApplication(appID); app != nil {
+		if task, err := app.GetTask(string(pod.UID)); err == nil {
+			_, ok := sp.context.GetInProgressPodAllocation(string(pod.UID))
+			if ok {
+				// pod must have failed scheduling, reject it and return unschedulable
+				log.Logger().Info("Task failed scheduling, marking as rejected",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()))
+				sp.context.RemovePodAllocation(string(pod.UID))
+				dispatcher.Dispatch(cache.NewRejectTaskEvent(app.GetApplicationID(), task.GetTaskID(),
+					fmt.Sprintf("task %s rejected by scheduler", task.GetTaskID())))
+				return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+			}
+
+			nodeID, ok := sp.context.GetPendingPodAllocation(string(pod.UID))
+			if task.GetTaskState() == events.States().Task.Bound && ok {
+				log.Logger().Info("Releasing pod for scheduling (prefilter phase)",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()),
+					zap.String("assignedNode", nodeID))
+
+				return framework.NewStatus(framework.Success, "")
+			}
+		}
+	}
+
+	return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+}
+
+// PreFilterExtensions is unused
+func (sp *YuniKornSchedulerPlugin) PreFilterExtensions() framework.PreFilterExtensions {
+	return nil
+}
+
+// Filter is used to release specific pod/node combinations to scheduler

Review comment:
       Added 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.

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

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



[GitHub] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   @pbacsko can you re-review? I've addressed the comments you made on the last revision.


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/task.go
##########
@@ -440,6 +460,23 @@ func (task *Task) beforeTaskAllocated(event *fsm.Event) {
 }
 
 func (task *Task) postTaskBound(event *fsm.Event) {
+	if task.pluginMode {
+		// when the pod is scheduling by yunikorn, it is moved to the default-scheduler's
+		// unschedulable queue, if nothing changes, the pod will be staying in the unschedulable
+		// queue for unschedulableQTimeInterval long (default 1 minute). hence, we are updating
+		// the pod status explicitly, when there is a status change, the default scheduler will
+		// move the pod back to the active queue immediately.
+		podCopy := task.pod.DeepCopy()

Review comment:
       This cost is negligible, because the copy happens very infrequently (typically once per pod), when YuniKorn has decided to allocate the pod. We update the pod status once at this point to force the default scheduler logic to re-consider the pod for scheduling. It will very shortly call PreFilter() / Filter() and push the Pod through the scheduling cycle.




-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   Changes in latest version:
   
   - Rebased PR on master
   - Fixed event recorder changes after YUNIKORN-937
   - Implemented `--plugin` flag for `scripts/run-e2e-tests.sh`. E2E tests can be run using the plugin implementation with something like:
   
   `scripts/run-e2e-tests.sh -a test -n "yk8s" -v "kindest/node:v1.21.2" --plugin`


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -36,23 +36,23 @@ import (
 // nodes are cached in the form of de-scheduler nodeInfo, instead of re-creating all nodes info from scratch,
 // we replicate nodes info from de-scheduler, in order to re-use predicates functions.
 type SchedulerCache struct {
-	// node name to NodeInfo map
-	nodesMap map[string]*framework.NodeInfo
-	podsMap  map[string]*v1.Pod
-	// this is a map of assumed pods,
-	// the value indicates if a pod volumes are all bound
-	assumedPods map[string]bool
-	lock        sync.RWMutex
-	// client APIs
-	clients *client.Clients
+	nodesMap              map[string]*framework.NodeInfo // node name to NodeInfo map
+	podsMap               map[string]*v1.Pod
+	assumedPods           map[string]bool   // map of assumed pods, value indicates if pod volumes are all bound
+	pendingAllocations    map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler
+	inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler

Review comment:
       A pending allocation is one which YuniKorn has decided on, but which has not yet been communicated to the default scheduler via the scheduling framework (the PreFilter()/Filter() calls in the plugin). An in-progress allocation is one which HAS been communicated to the default scheduler via PreFilter() / Filter() but which we have not yet got confirmation of the Pod being bound (PostBind() in the plugin has not yet been called). This is used to track state to ensure that we don't reprocess the same Pod or leak in the case where the default scheduler is unable to bind the pod for some reason.




-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -36,23 +36,23 @@ import (
 // nodes are cached in the form of de-scheduler nodeInfo, instead of re-creating all nodes info from scratch,
 // we replicate nodes info from de-scheduler, in order to re-use predicates functions.
 type SchedulerCache struct {
-	// node name to NodeInfo map
-	nodesMap map[string]*framework.NodeInfo
-	podsMap  map[string]*v1.Pod
-	// this is a map of assumed pods,
-	// the value indicates if a pod volumes are all bound
-	assumedPods map[string]bool
-	lock        sync.RWMutex
-	// client APIs
-	clients *client.Clients
+	nodesMap              map[string]*framework.NodeInfo // node name to NodeInfo map
+	podsMap               map[string]*v1.Pod
+	assumedPods           map[string]bool   // map of assumed pods, value indicates if pod volumes are all bound
+	pendingAllocations    map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler
+	inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler

Review comment:
       We need some extra comment which explains what is the difference between a "pending" and an "in progress" allocation. Just by looking at the code, it's not straightforward which comes first and what is the lifecycle of the entries here.




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4c05a97) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `1.70%`.
   > The diff coverage is `62.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   61.46%   +1.70%     
   ==========================================
     Files          35       39       +4     
     Lines        3133     5353    +2220     
   ==========================================
   + Hits         1872     3290    +1418     
   - Misses       1180     1951     +771     
   - Partials       81      112      +31     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `30.41% <6.74%> (-2.33%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.80%> (-3.87%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `40.00% <28.57%> (+10.00%)` | :arrow_up: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <31.81%> (-3.25%)` | :arrow_down: |
   | ... and [56 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [72d7cbc...4c05a97](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   Rebased on latest master (post 0.12.1).


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -36,23 +36,23 @@ import (
 // nodes are cached in the form of de-scheduler nodeInfo, instead of re-creating all nodes info from scratch,
 // we replicate nodes info from de-scheduler, in order to re-use predicates functions.
 type SchedulerCache struct {
-	// node name to NodeInfo map
-	nodesMap map[string]*framework.NodeInfo
-	podsMap  map[string]*v1.Pod
-	// this is a map of assumed pods,
-	// the value indicates if a pod volumes are all bound
-	assumedPods map[string]bool
-	lock        sync.RWMutex
-	// client APIs
-	clients *client.Clients
+	nodesMap              map[string]*framework.NodeInfo // node name to NodeInfo map
+	podsMap               map[string]*v1.Pod
+	assumedPods           map[string]bool   // map of assumed pods, value indicates if pod volumes are all bound
+	pendingAllocations    map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler
+	inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler

Review comment:
       Added additional documentation in scheduler_cache.go.




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] removed a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce032c) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `3.15%`.
   > The diff coverage is `64.28%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   62.90%   +3.15%     
   ==========================================
     Files          35       41       +6     
     Lines        3133     6104    +2971     
   ==========================================
   + Hits         1872     3840    +1968     
   - Misses       1180     2112     +932     
   - Partials       81      152      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/plugin/admissioncontrollers/webhook/webhook.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL3dlYmhvb2suZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <17.64%> (+3.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `36.90% <20.73%> (+4.16%)` | :arrow_up: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <21.68%> (-3.87%)` | :arrow_down: |
   | ... and [59 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a316283...6ce032c](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -132,6 +132,45 @@ func (cache *SchedulerCache) removeNode(node *v1.Node) error {
 	return nil
 }
 
+func (cache *SchedulerCache) AddPendingPodAllocation(podKey string, nodeID string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.inProgressAllocations, podKey)

Review comment:
       Is this delete necessary? Under what circumstances can we have an entry?
   
   As I can see it, `inProgressAllocations` is populated from `StartPodAllocation()` but that already requires that pending has an entry which we just add in line 139. Am I missing something here?
   
   Is it a bug if we already have something in `inProgressAllocations` at this point for the same `podKey`?




-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,192 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+type YuniKornSchedulerPlugin struct {
+	sync.RWMutex
+	context *cache.Context
+}
+
+// ensure all required interfaces are implemented
+var _ framework.PreFilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.FilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.PostBindPlugin = &YuniKornSchedulerPlugin{}
+
+// Plugin
+func (sp *YuniKornSchedulerPlugin) Name() string {
+	return SchedulerPluginName
+}
+
+// PreFilterPlugin: PreFilter is used to release pods to scheduler
+func (sp *YuniKornSchedulerPlugin) PreFilter(_ context.Context, _ *framework.CycleState, pod *v1.Pod) *framework.Status {
+	log.Logger().Debug("PreFilter check",
+		zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)))
+
+	// we don't process pods without appID defined
+	appID, err := utils.GetApplicationIDFromPod(pod)
+	if err != nil {
+		log.Logger().Info(fmt.Sprintf("Skipping pod %s/%s in the prefilter plugin because no applicationID is defined",
+			pod.Namespace, pod.Name))
+		return framework.NewStatus(framework.Success, "Deferring to default scheduler")
+	}
+
+	if app := sp.context.GetApplication(appID); app != nil {
+		if task, err := app.GetTask(string(pod.UID)); err == nil {
+			_, ok := sp.context.GetInProgressPodAllocation(string(pod.UID))
+			if ok {
+				// pod must have failed scheduling, reject it and return unschedulable
+				log.Logger().Info("Task failed scheduling, marking as rejected",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()))
+				sp.context.RemovePendingPodAllocation(string(pod.UID))
+				dispatcher.Dispatch(cache.NewRejectTaskEvent(app.GetApplicationID(), task.GetTaskID(),
+					fmt.Sprintf("task %s rejected by scheduler", task.GetTaskID())))
+				return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+			}
+
+			nodeID, ok := sp.context.GetPendingPodAllocation(string(pod.UID))
+			if !ok {
+				nodeID = "<none>"

Review comment:
       Is this value used for anything?




-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   @pbacsko can you review again?


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,192 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+type YuniKornSchedulerPlugin struct {
+	sync.RWMutex
+	context *cache.Context
+}
+
+// ensure all required interfaces are implemented
+var _ framework.PreFilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.FilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.PostBindPlugin = &YuniKornSchedulerPlugin{}
+
+// Plugin
+func (sp *YuniKornSchedulerPlugin) Name() string {
+	return SchedulerPluginName
+}
+
+// PreFilterPlugin: PreFilter is used to release pods to scheduler
+func (sp *YuniKornSchedulerPlugin) PreFilter(_ context.Context, _ *framework.CycleState, pod *v1.Pod) *framework.Status {
+	log.Logger().Debug("PreFilter check",
+		zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)))
+
+	// we don't process pods without appID defined
+	appID, err := utils.GetApplicationIDFromPod(pod)
+	if err != nil {
+		log.Logger().Info(fmt.Sprintf("Skipping pod %s/%s in the prefilter plugin because no applicationID is defined",
+			pod.Namespace, pod.Name))
+		return framework.NewStatus(framework.Success, "Deferring to default scheduler")
+	}
+
+	if app := sp.context.GetApplication(appID); app != nil {
+		if task, err := app.GetTask(string(pod.UID)); err == nil {
+			_, ok := sp.context.GetInProgressPodAllocation(string(pod.UID))
+			if ok {
+				// pod must have failed scheduling, reject it and return unschedulable
+				log.Logger().Info("Task failed scheduling, marking as rejected",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()))
+				sp.context.RemovePendingPodAllocation(string(pod.UID))
+				dispatcher.Dispatch(cache.NewRejectTaskEvent(app.GetApplicationID(), task.GetTaskID(),
+					fmt.Sprintf("task %s rejected by scheduler", task.GetTaskID())))
+				return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+			}
+
+			nodeID, ok := sp.context.GetPendingPodAllocation(string(pod.UID))
+			if !ok {
+				nodeID = "<none>"

Review comment:
       in the logging message below to make things clearer.




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9a9f4e2) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/023a63ce512c7c8b237b70e10c21aefc3f02bb2b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (023a63c) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.30%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <12.50%> (-0.62%)` | :arrow_down: |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <21.62%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [023a63c...9a9f4e2](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3b36f2b) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `1.70%`.
   > The diff coverage is `63.21%`.
   
   > :exclamation: Current head 3b36f2b differs from pull request most recent head 6ce032c. Consider uploading reports for the commit 6ce032c to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   61.46%   +1.70%     
   ==========================================
     Files          35       39       +4     
     Lines        3133     5353    +2220     
   ==========================================
   + Hits         1872     3290    +1418     
   - Misses       1180     1951     +771     
   - Partials       81      112      +31     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `30.41% <7.79%> (-2.33%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.80%> (-3.87%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `40.00% <28.57%> (+10.00%)` | :arrow_up: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <31.81%> (-3.25%)` | :arrow_down: |
   | ... and [56 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a316283...6ce032c](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cb5ecde) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/023a63ce512c7c8b237b70e10c21aefc3f02bb2b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (023a63c) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.07%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.64%> (-0.62%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <20.51%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [023a63c...cb5ecde](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9a9f4e2) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/023a63ce512c7c8b237b70e10c21aefc3f02bb2b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (023a63c) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.30%`.
   
   > :exclamation: Current head 9a9f4e2 differs from pull request most recent head cb5ecde. Consider uploading reports for the commit cb5ecde to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <12.50%> (-0.62%)` | :arrow_down: |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <21.62%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [023a63c...cb5ecde](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,192 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+type YuniKornSchedulerPlugin struct {
+	sync.RWMutex
+	context *cache.Context
+}
+
+// ensure all required interfaces are implemented
+var _ framework.PreFilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.FilterPlugin = &YuniKornSchedulerPlugin{}
+var _ framework.PostBindPlugin = &YuniKornSchedulerPlugin{}
+
+// Plugin
+func (sp *YuniKornSchedulerPlugin) Name() string {
+	return SchedulerPluginName
+}
+
+// PreFilterPlugin: PreFilter is used to release pods to scheduler
+func (sp *YuniKornSchedulerPlugin) PreFilter(_ context.Context, _ *framework.CycleState, pod *v1.Pod) *framework.Status {
+	log.Logger().Debug("PreFilter check",
+		zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)))
+
+	// we don't process pods without appID defined
+	appID, err := utils.GetApplicationIDFromPod(pod)
+	if err != nil {
+		log.Logger().Info(fmt.Sprintf("Skipping pod %s/%s in the prefilter plugin because no applicationID is defined",
+			pod.Namespace, pod.Name))
+		return framework.NewStatus(framework.Success, "Deferring to default scheduler")
+	}
+
+	if app := sp.context.GetApplication(appID); app != nil {
+		if task, err := app.GetTask(string(pod.UID)); err == nil {
+			_, ok := sp.context.GetInProgressPodAllocation(string(pod.UID))
+			if ok {
+				// pod must have failed scheduling, reject it and return unschedulable
+				log.Logger().Info("Task failed scheduling, marking as rejected",
+					zap.String("pod", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)),
+					zap.String("taskID", task.GetTaskID()))
+				sp.context.RemovePendingPodAllocation(string(pod.UID))
+				dispatcher.Dispatch(cache.NewRejectTaskEvent(app.GetApplicationID(), task.GetTaskID(),
+					fmt.Sprintf("task %s rejected by scheduler", task.GetTaskID())))
+				return framework.NewStatus(framework.Unschedulable, "Pod is not ready for scheduling")
+			}
+
+			nodeID, ok := sp.context.GetPendingPodAllocation(string(pod.UID))
+			if !ok {
+				nodeID = "<none>"

Review comment:
       Per offline discussion, removed this as the log message will never print it anyway.




-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/schedulerplugin/scheduler_plugin.go
##########
@@ -0,0 +1,209 @@
+/*
+ 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 schedulerplugin
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"go.uber.org/zap"
+	v1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/informers"
+	"k8s.io/kubernetes/pkg/scheduler/framework"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/entrypoint"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/cache"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/utils"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/dispatcher"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/log"
+	pluginconf "github.com/apache/incubator-yunikorn-k8shim/pkg/schedulerplugin/conf"
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/shim"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/api"
+)
+
+const (
+	SchedulerPluginName = "YuniKornPlugin"
+)
+
+// YuniKornSchedulerPlugin provides an implementation of several lifecycle methods of the Kubernetes scheduling framework:
+//   https://kubernetes.io/docs/concepts/scheduling-eviction/scheduling-framework/
+//
+// PreFilter: Used to notify the default scheduler that a particular pod has been marked ready for scheduling by YuniKorn
+//
+// Filter: Used to notify the default scheduler that a particular pod/node combination is ready to be scheduled
+//
+// PostBind: Used to notify YuniKorn that a pod has been scheduled successfully
+//
+// Pod Allocations:
+//
+// The YuniKorn scheduler is always running in the background, making decisions about which pods to allocate to which
+// nodes. When a decision is made, that pod is marked as having a "pending" pod allocation, which means YuniKorn has
+// allocated the pod, but the default scheduler (via the plugin interface) has not yet been notified.
+//
+// Once PreFilter() has been called for a particular pod, that allocation is marked as "in progress" meaning it has been
+// communicated to the default scheduler, but has not yet been fulfilled.
+//
+// Finally, in PostBind(), the allocation is removed as we now know that the pod has been allocated successfully.
+// If a pending or in-progress allocation is detected for a pod in PreFilter(), we remove the allocation and force the
+// pod to be rescheduled, as this means the prior allocation could not be completed successfully by the default
+// scheduler for some reason.
+type YuniKornSchedulerPlugin struct {

Review comment:
       Solid comment, it's great to have this here.




-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/task.go
##########
@@ -440,6 +460,23 @@ func (task *Task) beforeTaskAllocated(event *fsm.Event) {
 }
 
 func (task *Task) postTaskBound(event *fsm.Event) {
+	if task.pluginMode {
+		// when the pod is scheduling by yunikorn, it is moved to the default-scheduler's
+		// unschedulable queue, if nothing changes, the pod will be staying in the unschedulable
+		// queue for unschedulableQTimeInterval long (default 1 minute). hence, we are updating
+		// the pod status explicitly, when there is a status change, the default scheduler will
+		// move the pod back to the active queue immediately.
+		podCopy := task.pod.DeepCopy()

Review comment:
       Can we reason about the performance effects of this copy? If we do this all the time (and that's what seems to be happening), we end up creating extra objects which affects GC time. Maybe not a big deal, just something to think about.




-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -132,6 +132,45 @@ func (cache *SchedulerCache) removeNode(node *v1.Node) error {
 	return nil
 }
 
+func (cache *SchedulerCache) AddPendingPodAllocation(podKey string, nodeID string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.inProgressAllocations, podKey)
+	cache.pendingAllocations[podKey] = nodeID
+}
+
+func (cache *SchedulerCache) RemovePendingPodAllocation(podKey string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.pendingAllocations, podKey)
+	delete(cache.inProgressAllocations, podKey)
+}
+
+func (cache *SchedulerCache) GetPendingPodAllocation(podKey string) (nodeID string, ok bool) {
+	cache.lock.RLock()
+	defer cache.lock.RUnlock()
+	res, ok := cache.pendingAllocations[podKey]
+	return res, ok
+}
+
+func (cache *SchedulerCache) GetInProgressPodAllocation(podKey string) (nodeID string, ok bool) {
+	cache.lock.RLock()
+	defer cache.lock.RUnlock()
+	res, ok := cache.inProgressAllocations[podKey]
+	return res, ok
+}
+
+func (cache *SchedulerCache) StartPodAllocation(podKey string, nodeID string) bool {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	expectedNodeID, ok := cache.pendingAllocations[podKey]
+	if ok && expectedNodeID == nodeID {

Review comment:
       This code is used to verify that the requested node (by the default scheduler) is the one that YuniKorn decided to allocate the pod to. Returning `false` here will result in the Filter() call in the plugin returning `Unschedulable` to the default scheduler for this pod / node combination, and a subsequent Filter() call with a matching node will return `Success` instead. Effectively, we use this code to veto node allocations other than the node that YK has decided on.
   
   The bind is deferred to the default scheduler logic once Filter() succeeds, which is why we don't handle the bind logic in postTaskAllocated().




-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   Rebased on master.


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -132,6 +132,45 @@ func (cache *SchedulerCache) removeNode(node *v1.Node) error {
 	return nil
 }
 
+func (cache *SchedulerCache) AddPendingPodAllocation(podKey string, nodeID string) {
+	cache.lock.Lock()
+	defer cache.lock.Unlock()
+	delete(cache.inProgressAllocations, podKey)

Review comment:
       If a pod fails to bind in the default scheduler for some reason, the only notice we may get (as there is not a callback in the scheduler framework for failed binds) is another call to PreFilter() / Filter() for this pod. So it is possible in that case for us to have an in-progress allocation. So, we remove it and reset the state back to pending to be safe.
   
   




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce032c) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/023a63ce512c7c8b237b70e10c21aefc3f02bb2b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (023a63c) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.30%`.
   
   > :exclamation: Current head 6ce032c differs from pull request most recent head 9a9f4e2. Consider uploading reports for the commit 9a9f4e2 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <12.50%> (-0.62%)` | :arrow_down: |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <21.62%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [023a63c...9a9f4e2](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   Added additional documentation to the scheduler_plugin.go and scheduler_cache.go files. Hopefully this is now clear.


-- 
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] [incubator-yunikorn-k8shim] craigcondit commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   @wilfred-s, @yangwwei, can you review?


-- 
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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -36,23 +36,23 @@ import (
 // nodes are cached in the form of de-scheduler nodeInfo, instead of re-creating all nodes info from scratch,
 // we replicate nodes info from de-scheduler, in order to re-use predicates functions.
 type SchedulerCache struct {
-	// node name to NodeInfo map
-	nodesMap map[string]*framework.NodeInfo
-	podsMap  map[string]*v1.Pod
-	// this is a map of assumed pods,
-	// the value indicates if a pod volumes are all bound
-	assumedPods map[string]bool
-	lock        sync.RWMutex
-	// client APIs
-	clients *client.Clients
+	nodesMap              map[string]*framework.NodeInfo // node name to NodeInfo map
+	podsMap               map[string]*v1.Pod
+	assumedPods           map[string]bool   // map of assumed pods, value indicates if pod volumes are all bound
+	pendingAllocations    map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler
+	inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler

Review comment:
       We need some extra comment which explains what is the difference between a "pending" and an "in progress" allocation. Just by looking at the code, it's not straightforward which comes first and what is the lifecycle of the pods.




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce032c) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `3.15%`.
   > The diff coverage is `64.28%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   62.90%   +3.15%     
   ==========================================
     Files          35       41       +6     
     Lines        3133     6104    +2971     
   ==========================================
   + Hits         1872     3840    +1968     
   - Misses       1180     2112     +932     
   - Partials       81      152      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/plugin/admissioncontrollers/webhook/webhook.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL3dlYmhvb2suZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <17.64%> (+3.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `36.90% <20.73%> (+4.16%)` | :arrow_up: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <21.68%> (-3.87%)` | :arrow_down: |
   | ... and [59 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a316283...6ce032c](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3b36f2b) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `1.70%`.
   > The diff coverage is `63.21%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   61.46%   +1.70%     
   ==========================================
     Files          35       39       +4     
     Lines        3133     5353    +2220     
   ==========================================
   + Hits         1872     3290    +1418     
   - Misses       1180     1951     +771     
   - Partials       81      112      +31     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `30.41% <7.79%> (-2.33%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.80%> (-3.87%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `40.00% <28.57%> (+10.00%)` | :arrow_up: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <31.81%> (-3.25%)` | :arrow_down: |
   | ... and [56 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [2c7581d...3b36f2b](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] commented on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4c05a97) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `1.70%`.
   > The diff coverage is `62.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   61.46%   +1.70%     
   ==========================================
     Files          35       39       +4     
     Lines        3133     5353    +2220     
   ==========================================
   + Hits         1872     3290    +1418     
   - Misses       1180     1951     +771     
   - Partials       81      112      +31     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `30.41% <6.74%> (-2.33%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.80%> (-3.87%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `40.00% <28.57%> (+10.00%)` | :arrow_up: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <31.81%> (-3.25%)` | :arrow_down: |
   | ... and [56 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [72d7cbc...4c05a97](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4c05a97) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/c47ed51f075c5af5910f71da40e7e68699a9abae?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c47ed51) will **increase** coverage by `1.70%`.
   > The diff coverage is `62.75%`.
   
   > :exclamation: Current head 4c05a97 differs from pull request most recent head 3b36f2b. Consider uploading reports for the commit 3b36f2b to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   + Coverage   59.75%   61.46%   +1.70%     
   ==========================================
     Files          35       39       +4     
     Lines        3133     5353    +2220     
   ==========================================
   + Hits         1872     3290    +1418     
   - Misses       1180     1951     +771     
   - Partials       81      112      +31     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/cache/amprotocol\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2FtcHJvdG9jb2xfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/cache/context\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHRfcmVjb3ZlcnkuZ28=) | `40.67% <0.00%> (-5.67%)` | :arrow_down: |
   | [pkg/common/events/recorder\_mock.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXJfbW9jay5nbw==) | `0.00% <0.00%> (ø)` | |
   | [pkg/common/events/states.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvc3RhdGVzLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (+33.33%)` | :arrow_up: |
   | [...missioncontrollers/webhook/admission\_controller.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3BsdWdpbi9hZG1pc3Npb25jb250cm9sbGVycy93ZWJob29rL2FkbWlzc2lvbl9jb250cm9sbGVyLmdv) | `30.41% <6.74%> (-2.33%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `67.92% <16.66%> (-7.76%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.80%> (-3.87%)` | :arrow_down: |
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `40.00% <28.57%> (+10.00%)` | :arrow_up: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <31.81%> (-3.25%)` | :arrow_down: |
   | ... and [56 more](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [2c7581d...3b36f2b](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cb5ecde) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/023a63ce512c7c8b237b70e10c21aefc3f02bb2b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (023a63c) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.07%`.
   
   > :exclamation: Current head cb5ecde differs from pull request most recent head 2ee49fb. Consider uploading reports for the commit 2ee49fb to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.64%> (-0.62%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <20.51%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09240e6...2ee49fb](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] pbacsko commented on a change in pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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



##########
File path: pkg/cache/external/scheduler_cache.go
##########
@@ -36,23 +36,23 @@ import (
 // nodes are cached in the form of de-scheduler nodeInfo, instead of re-creating all nodes info from scratch,
 // we replicate nodes info from de-scheduler, in order to re-use predicates functions.
 type SchedulerCache struct {
-	// node name to NodeInfo map
-	nodesMap map[string]*framework.NodeInfo
-	podsMap  map[string]*v1.Pod
-	// this is a map of assumed pods,
-	// the value indicates if a pod volumes are all bound
-	assumedPods map[string]bool
-	lock        sync.RWMutex
-	// client APIs
-	clients *client.Clients
+	nodesMap              map[string]*framework.NodeInfo // node name to NodeInfo map
+	podsMap               map[string]*v1.Pod
+	assumedPods           map[string]bool   // map of assumed pods, value indicates if pod volumes are all bound
+	pendingAllocations    map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler
+	inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler

Review comment:
       Thanks, this definitely helps to understand the code.




-- 
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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2ee49fb) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/09240e6a73c90d8733a9ffb90dafae207bdf1a46?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09240e6) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.07%`.
   
   > :exclamation: Current head 2ee49fb differs from pull request most recent head 7bcc9cc. Consider uploading reports for the commit 7bcc9cc to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.64%> (-0.62%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <20.51%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09240e6...7bcc9cc](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (7bcc9cc) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/09240e6a73c90d8733a9ffb90dafae207bdf1a46?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09240e6) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.07%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.64%> (-0.62%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <20.51%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09240e6...7bcc9cc](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-k8shim] codecov[bot] edited a comment on pull request #336: [YUNIKORN-971] Implement YuniKorn as a Kubernetes scheduler plugin.

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


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?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 [#336](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2ee49fb) into [master](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/commit/09240e6a73c90d8733a9ffb90dafae207bdf1a46?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09240e6) will **decrease** coverage by `0.24%`.
   > The diff coverage is `53.07%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/graphs/tree.svg?width=650&height=150&src=pr&token=LZImIuvleR&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #336      +/-   ##
   ==========================================
   - Coverage   63.15%   62.90%   -0.25%     
   ==========================================
     Files          41       41              
     Lines        5906     6104     +198     
   ==========================================
   + Hits         3730     3840     +110     
   - Misses       2030     2112      +82     
   - Partials      146      152       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/common/events/recorder.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbW1vbi9ldmVudHMvcmVjb3JkZXIuZ28=) | `33.33% <0.00%> (-8.78%)` | :arrow_down: |
   | [pkg/shim/scheduler\_events.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyX2V2ZW50cy5nbw==) | `100.00% <ø> (ø)` | |
   | [pkg/shim/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NoaW0vc2NoZWR1bGVyLmdv) | `76.85% <15.00%> (-4.72%)` | :arrow_down: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `37.21% <17.64%> (-0.62%)` | :arrow_down: |
   | [pkg/cache/external/scheduler\_cache.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL2V4dGVybmFsL3NjaGVkdWxlcl9jYWNoZS5nbw==) | `30.91% <20.51%> (-2.71%)` | :arrow_down: |
   | [pkg/cache/task.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NhY2hlL3Rhc2suZ28=) | `73.05% <37.70%> (-4.48%)` | :arrow_down: |
   | [pkg/conf/schedulerconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `66.35% <75.00%> (-0.32%)` | :arrow_down: |
   | [pkg/schedulerplugin/conf/pluginconf.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlcnBsdWdpbi9jb25mL3BsdWdpbmNvbmYuZ28=) | `86.36% <86.36%> (ø)` | |
   | [pkg/cmd/shim/main.go](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL2NtZC9zaGltL21haW4uZ28=) | | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09240e6...2ee49fb](https://codecov.io/gh/apache/incubator-yunikorn-k8shim/pull/336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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