mirror of
https://github.com/k3s-io/kubernetes.git
synced 2025-07-26 05:03:09 +00:00
Merge pull request #41195 from wojtek-t/remove_default_failure_domains
Automatic merge from submit-queue (batch tested with PRs 41401, 41195, 41664, 41521, 41651) Remove default failure domains from anti-affinity feature Removing it is necessary to make performance of this feature acceptable at some point. With default failure domains (or in general when multiple topology keys are possible), we don't have transitivity between node belonging to a topology. And without this, it's pretty much impossible to solve this effectively. @timothysc
This commit is contained in:
commit
97921ff38e
@ -572,6 +572,7 @@ type KubeSchedulerConfiguration struct {
|
|||||||
// HardPodAffinitySymmetricWeight represents the weight of implicit PreferredDuringScheduling affinity rule, in the range 0-100.
|
// HardPodAffinitySymmetricWeight represents the weight of implicit PreferredDuringScheduling affinity rule, in the range 0-100.
|
||||||
HardPodAffinitySymmetricWeight int
|
HardPodAffinitySymmetricWeight int
|
||||||
// Indicate the "all topologies" set for empty topologyKey when it's used for PreferredDuringScheduling pod anti-affinity.
|
// Indicate the "all topologies" set for empty topologyKey when it's used for PreferredDuringScheduling pod anti-affinity.
|
||||||
|
// DEPRECATED: This is no longer used.
|
||||||
FailureDomains string
|
FailureDomains string
|
||||||
// leaderElection defines the configuration of leader election client.
|
// leaderElection defines the configuration of leader election client.
|
||||||
LeaderElection LeaderElectionConfiguration
|
LeaderElection LeaderElectionConfiguration
|
||||||
|
@ -66,6 +66,9 @@ const (
|
|||||||
// and also prevents them from being evicted from a node.
|
// and also prevents them from being evicted from a node.
|
||||||
// Note: This feature is not supported for `BestEffort` pods.
|
// Note: This feature is not supported for `BestEffort` pods.
|
||||||
ExperimentalCriticalPodAnnotation utilfeature.Feature = "ExperimentalCriticalPodAnnotation"
|
ExperimentalCriticalPodAnnotation utilfeature.Feature = "ExperimentalCriticalPodAnnotation"
|
||||||
|
|
||||||
|
// Determines if affinity defined in annotations should bep rocessed
|
||||||
|
AffinityInAnnotations utilfeature.Feature = "AffinityInAnnotations"
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
|
@ -70,7 +70,7 @@ func createClient(s *options.SchedulerServer) (*clientset.Clientset, error) {
|
|||||||
|
|
||||||
// createScheduler encapsulates the entire creation of a runnable scheduler.
|
// createScheduler encapsulates the entire creation of a runnable scheduler.
|
||||||
func createScheduler(s *options.SchedulerServer, kubecli *clientset.Clientset, recorder record.EventRecorder) (*scheduler.Scheduler, error) {
|
func createScheduler(s *options.SchedulerServer, kubecli *clientset.Clientset, recorder record.EventRecorder) (*scheduler.Scheduler, error) {
|
||||||
configurator := factory.NewConfigFactory(kubecli, s.SchedulerName, s.HardPodAffinitySymmetricWeight, s.FailureDomains)
|
configurator := factory.NewConfigFactory(kubecli, s.SchedulerName, s.HardPodAffinitySymmetricWeight)
|
||||||
|
|
||||||
// Rebuild the configurator with a default Create(...) method.
|
// Rebuild the configurator with a default Create(...) method.
|
||||||
configurator = &schedulerConfigurator{
|
configurator = &schedulerConfigurator{
|
||||||
|
@ -74,6 +74,7 @@ func (s *SchedulerServer) AddFlags(fs *pflag.FlagSet) {
|
|||||||
"RequiredDuringScheduling affinity is not symmetric, but there is an implicit PreferredDuringScheduling affinity rule corresponding "+
|
"RequiredDuringScheduling affinity is not symmetric, but there is an implicit PreferredDuringScheduling affinity rule corresponding "+
|
||||||
"to every RequiredDuringScheduling affinity rule. --hard-pod-affinity-symmetric-weight represents the weight of implicit PreferredDuringScheduling affinity rule.")
|
"to every RequiredDuringScheduling affinity rule. --hard-pod-affinity-symmetric-weight represents the weight of implicit PreferredDuringScheduling affinity rule.")
|
||||||
fs.StringVar(&s.FailureDomains, "failure-domains", api.DefaultFailureDomains, "Indicate the \"all topologies\" set for an empty topologyKey when it's used for PreferredDuringScheduling pod anti-affinity.")
|
fs.StringVar(&s.FailureDomains, "failure-domains", api.DefaultFailureDomains, "Indicate the \"all topologies\" set for an empty topologyKey when it's used for PreferredDuringScheduling pod anti-affinity.")
|
||||||
|
fs.MarkDeprecated("failure-domains", "Doesn't have any effect. Will be removed in future version.")
|
||||||
leaderelection.BindFlags(&s.LeaderElection, fs)
|
leaderelection.BindFlags(&s.LeaderElection, fs)
|
||||||
|
|
||||||
utilfeature.DefaultFeatureGate.AddFlag(fs)
|
utilfeature.DefaultFeatureGate.AddFlag(fs)
|
||||||
|
@ -17,6 +17,7 @@ limitations under the License.
|
|||||||
package priorities
|
package priorities
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/golang/glog"
|
"github.com/golang/glog"
|
||||||
@ -35,21 +36,18 @@ type InterPodAffinity struct {
|
|||||||
nodeLister algorithm.NodeLister
|
nodeLister algorithm.NodeLister
|
||||||
podLister algorithm.PodLister
|
podLister algorithm.PodLister
|
||||||
hardPodAffinityWeight int
|
hardPodAffinityWeight int
|
||||||
failureDomains priorityutil.Topologies
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewInterPodAffinityPriority(
|
func NewInterPodAffinityPriority(
|
||||||
info predicates.NodeInfo,
|
info predicates.NodeInfo,
|
||||||
nodeLister algorithm.NodeLister,
|
nodeLister algorithm.NodeLister,
|
||||||
podLister algorithm.PodLister,
|
podLister algorithm.PodLister,
|
||||||
hardPodAffinityWeight int,
|
hardPodAffinityWeight int) algorithm.PriorityFunction {
|
||||||
failureDomains []string) algorithm.PriorityFunction {
|
|
||||||
interPodAffinity := &InterPodAffinity{
|
interPodAffinity := &InterPodAffinity{
|
||||||
info: info,
|
info: info,
|
||||||
nodeLister: nodeLister,
|
nodeLister: nodeLister,
|
||||||
podLister: podLister,
|
podLister: podLister,
|
||||||
hardPodAffinityWeight: hardPodAffinityWeight,
|
hardPodAffinityWeight: hardPodAffinityWeight,
|
||||||
failureDomains: priorityutil.Topologies{DefaultKeys: failureDomains},
|
|
||||||
}
|
}
|
||||||
return interPodAffinity.CalculateInterPodAffinityPriority
|
return interPodAffinity.CalculateInterPodAffinityPriority
|
||||||
}
|
}
|
||||||
@ -68,11 +66,11 @@ type podAffinityPriorityMap struct {
|
|||||||
firstError error
|
firstError error
|
||||||
}
|
}
|
||||||
|
|
||||||
func newPodAffinityPriorityMap(nodes []*v1.Node, failureDomains priorityutil.Topologies) *podAffinityPriorityMap {
|
func newPodAffinityPriorityMap(nodes []*v1.Node) *podAffinityPriorityMap {
|
||||||
return &podAffinityPriorityMap{
|
return &podAffinityPriorityMap{
|
||||||
nodes: nodes,
|
nodes: nodes,
|
||||||
counts: make(map[string]float64, len(nodes)),
|
counts: make(map[string]float64, len(nodes)),
|
||||||
failureDomains: failureDomains,
|
failureDomains: priorityutil.Topologies{DefaultKeys: strings.Split(v1.DefaultFailureDomains, ",")},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -132,7 +130,7 @@ func (ipa *InterPodAffinity) CalculateInterPodAffinityPriority(pod *v1.Pod, node
|
|||||||
var minCount float64
|
var minCount float64
|
||||||
// priorityMap stores the mapping from node name to so-far computed score of
|
// priorityMap stores the mapping from node name to so-far computed score of
|
||||||
// the node.
|
// the node.
|
||||||
pm := newPodAffinityPriorityMap(nodes, ipa.failureDomains)
|
pm := newPodAffinityPriorityMap(nodes)
|
||||||
|
|
||||||
processPod := func(existingPod *v1.Pod) error {
|
processPod := func(existingPod *v1.Pod) error {
|
||||||
existingPodNode, err := ipa.info.GetNodeInfo(existingPod.Spec.NodeName)
|
existingPodNode, err := ipa.info.GetNodeInfo(existingPod.Spec.NodeName)
|
||||||
|
@ -19,13 +19,11 @@ package priorities
|
|||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"reflect"
|
"reflect"
|
||||||
"strings"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/kubernetes/pkg/api/v1"
|
"k8s.io/kubernetes/pkg/api/v1"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm"
|
||||||
priorityutil "k8s.io/kubernetes/plugin/pkg/scheduler/algorithm/priorities/util"
|
|
||||||
schedulerapi "k8s.io/kubernetes/plugin/pkg/scheduler/api"
|
schedulerapi "k8s.io/kubernetes/plugin/pkg/scheduler/api"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/schedulercache"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/schedulercache"
|
||||||
)
|
)
|
||||||
@ -517,7 +515,6 @@ func TestInterPodAffinityPriority(t *testing.T) {
|
|||||||
nodeLister: algorithm.FakeNodeLister(test.nodes),
|
nodeLister: algorithm.FakeNodeLister(test.nodes),
|
||||||
podLister: algorithm.FakePodLister(test.pods),
|
podLister: algorithm.FakePodLister(test.pods),
|
||||||
hardPodAffinityWeight: v1.DefaultHardPodAffinitySymmetricWeight,
|
hardPodAffinityWeight: v1.DefaultHardPodAffinitySymmetricWeight,
|
||||||
failureDomains: priorityutil.Topologies{DefaultKeys: strings.Split(v1.DefaultFailureDomains, ",")},
|
|
||||||
}
|
}
|
||||||
list, err := interPodAffinity.CalculateInterPodAffinityPriority(test.pod, nodeNameToInfo, test.nodes)
|
list, err := interPodAffinity.CalculateInterPodAffinityPriority(test.pod, nodeNameToInfo, test.nodes)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -616,90 +613,3 @@ func TestHardPodAffinitySymmetricWeight(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestSoftPodAntiAffinityWithFailureDomains(t *testing.T) {
|
|
||||||
labelAzAZ1 := map[string]string{
|
|
||||||
"az": "az1",
|
|
||||||
}
|
|
||||||
LabelZoneFailureDomainAZ1 := map[string]string{
|
|
||||||
metav1.LabelZoneFailureDomain: "az1",
|
|
||||||
}
|
|
||||||
podLabel1 := map[string]string{
|
|
||||||
"security": "S1",
|
|
||||||
}
|
|
||||||
antiAffinity1 := &v1.Affinity{
|
|
||||||
PodAntiAffinity: &v1.PodAntiAffinity{
|
|
||||||
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
|
|
||||||
{
|
|
||||||
Weight: 5,
|
|
||||||
PodAffinityTerm: v1.PodAffinityTerm{
|
|
||||||
LabelSelector: &metav1.LabelSelector{
|
|
||||||
MatchExpressions: []metav1.LabelSelectorRequirement{
|
|
||||||
{
|
|
||||||
Key: "security",
|
|
||||||
Operator: metav1.LabelSelectorOpIn,
|
|
||||||
Values: []string{"S1"},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
TopologyKey: "",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}
|
|
||||||
tests := []struct {
|
|
||||||
pod *v1.Pod
|
|
||||||
pods []*v1.Pod
|
|
||||||
nodes []*v1.Node
|
|
||||||
failureDomains priorityutil.Topologies
|
|
||||||
expectedList schedulerapi.HostPriorityList
|
|
||||||
test string
|
|
||||||
}{
|
|
||||||
{
|
|
||||||
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: antiAffinity1}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
pods: []*v1.Pod{
|
|
||||||
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
},
|
|
||||||
nodes: []*v1.Node{
|
|
||||||
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: LabelZoneFailureDomainAZ1}},
|
|
||||||
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAZ1}},
|
|
||||||
},
|
|
||||||
failureDomains: priorityutil.Topologies{DefaultKeys: strings.Split(v1.DefaultFailureDomains, ",")},
|
|
||||||
expectedList: []schedulerapi.HostPriority{{Host: "machine1", Score: 0}, {Host: "machine2", Score: 10}},
|
|
||||||
test: "Soft Pod Anti Affinity: when the topologyKey is emtpy, match among topologyKeys indicated by failure domains.",
|
|
||||||
},
|
|
||||||
{
|
|
||||||
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: antiAffinity1}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
pods: []*v1.Pod{
|
|
||||||
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabel1}},
|
|
||||||
},
|
|
||||||
nodes: []*v1.Node{
|
|
||||||
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: LabelZoneFailureDomainAZ1}},
|
|
||||||
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAZ1}},
|
|
||||||
},
|
|
||||||
failureDomains: priorityutil.Topologies{},
|
|
||||||
expectedList: []schedulerapi.HostPriority{{Host: "machine1", Score: 0}, {Host: "machine2", Score: 0}},
|
|
||||||
test: "Soft Pod Anti Affinity: when the topologyKey is emtpy, and no failure domains indicated, regard as topologyKey not match.",
|
|
||||||
},
|
|
||||||
}
|
|
||||||
for _, test := range tests {
|
|
||||||
nodeNameToInfo := schedulercache.CreateNodeNameToInfoMap(test.pods, test.nodes)
|
|
||||||
ipa := InterPodAffinity{
|
|
||||||
info: FakeNodeListInfo(test.nodes),
|
|
||||||
nodeLister: algorithm.FakeNodeLister(test.nodes),
|
|
||||||
podLister: algorithm.FakePodLister(test.pods),
|
|
||||||
hardPodAffinityWeight: v1.DefaultHardPodAffinitySymmetricWeight,
|
|
||||||
failureDomains: test.failureDomains,
|
|
||||||
}
|
|
||||||
list, err := ipa.CalculateInterPodAffinityPriority(test.pod, nodeNameToInfo, test.nodes)
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("unexpected error: %v", err)
|
|
||||||
}
|
|
||||||
if !reflect.DeepEqual(test.expectedList, list) {
|
|
||||||
t.Errorf("%s: \nexpected \n\t%#v, \ngot \n\t%#v\n", test.test, test.expectedList, list)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
@ -17,9 +17,11 @@ go_library(
|
|||||||
tags = ["automanaged"],
|
tags = ["automanaged"],
|
||||||
deps = [
|
deps = [
|
||||||
"//pkg/api/v1:go_default_library",
|
"//pkg/api/v1:go_default_library",
|
||||||
|
"//pkg/features:go_default_library",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/apis/meta/v1",
|
"//vendor:k8s.io/apimachinery/pkg/apis/meta/v1",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/labels",
|
"//vendor:k8s.io/apimachinery/pkg/labels",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/util/sets",
|
"//vendor:k8s.io/apimachinery/pkg/util/sets",
|
||||||
|
"//vendor:k8s.io/apiserver/pkg/util/feature",
|
||||||
],
|
],
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -19,7 +19,9 @@ package util
|
|||||||
import (
|
import (
|
||||||
"k8s.io/apimachinery/pkg/labels"
|
"k8s.io/apimachinery/pkg/labels"
|
||||||
"k8s.io/apimachinery/pkg/util/sets"
|
"k8s.io/apimachinery/pkg/util/sets"
|
||||||
|
utilfeature "k8s.io/apiserver/pkg/util/feature"
|
||||||
"k8s.io/kubernetes/pkg/api/v1"
|
"k8s.io/kubernetes/pkg/api/v1"
|
||||||
|
"k8s.io/kubernetes/pkg/features"
|
||||||
)
|
)
|
||||||
|
|
||||||
// GetNamespacesFromPodAffinityTerm returns a set of names
|
// GetNamespacesFromPodAffinityTerm returns a set of names
|
||||||
@ -65,7 +67,7 @@ type Topologies struct {
|
|||||||
// NodesHaveSameTopologyKey checks if nodeA and nodeB have same label value with given topologyKey as label key.
|
// NodesHaveSameTopologyKey checks if nodeA and nodeB have same label value with given topologyKey as label key.
|
||||||
// If the topologyKey is empty, check if the two nodes have any of the default topologyKeys, and have same corresponding label value.
|
// If the topologyKey is empty, check if the two nodes have any of the default topologyKeys, and have same corresponding label value.
|
||||||
func (tps *Topologies) NodesHaveSameTopologyKey(nodeA, nodeB *v1.Node, topologyKey string) bool {
|
func (tps *Topologies) NodesHaveSameTopologyKey(nodeA, nodeB *v1.Node, topologyKey string) bool {
|
||||||
if len(topologyKey) == 0 {
|
if utilfeature.DefaultFeatureGate.Enabled(features.AffinityInAnnotations) && len(topologyKey) == 0 {
|
||||||
// assumes this is allowed only for PreferredDuringScheduling pod anti-affinity (ensured by api/validation)
|
// assumes this is allowed only for PreferredDuringScheduling pod anti-affinity (ensured by api/validation)
|
||||||
for _, defaultKey := range tps.DefaultKeys {
|
for _, defaultKey := range tps.DefaultKeys {
|
||||||
if NodesHaveSameTopologyKey(nodeA, nodeB, defaultKey) {
|
if NodesHaveSameTopologyKey(nodeA, nodeB, defaultKey) {
|
||||||
|
@ -340,7 +340,7 @@ func TestCompatibility_v1_Scheduler(t *testing.T) {
|
|||||||
defer server.Close()
|
defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
|
|
||||||
if _, err := factory.NewConfigFactory(client, "some-scheduler-name", v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains).CreateFromConfig(policy); err != nil {
|
if _, err := factory.NewConfigFactory(client, "some-scheduler-name", v1.DefaultHardPodAffinitySymmetricWeight).CreateFromConfig(policy); err != nil {
|
||||||
t.Errorf("%s: Error constructing: %v", v, err)
|
t.Errorf("%s: Error constructing: %v", v, err)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
@ -180,7 +180,7 @@ func defaultPriorities() sets.String {
|
|||||||
"InterPodAffinityPriority",
|
"InterPodAffinityPriority",
|
||||||
factory.PriorityConfigFactory{
|
factory.PriorityConfigFactory{
|
||||||
Function: func(args factory.PluginFactoryArgs) algorithm.PriorityFunction {
|
Function: func(args factory.PluginFactoryArgs) algorithm.PriorityFunction {
|
||||||
return priorities.NewInterPodAffinityPriority(args.NodeInfo, args.NodeLister, args.PodLister, args.HardPodAffinitySymmetricWeight, args.FailureDomains)
|
return priorities.NewInterPodAffinityPriority(args.NodeInfo, args.NodeLister, args.PodLister, args.HardPodAffinitySymmetricWeight)
|
||||||
},
|
},
|
||||||
Weight: 1,
|
Weight: 1,
|
||||||
},
|
},
|
||||||
|
@ -37,7 +37,6 @@ go_library(
|
|||||||
"//vendor:k8s.io/apimachinery/pkg/types",
|
"//vendor:k8s.io/apimachinery/pkg/types",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/util/runtime",
|
"//vendor:k8s.io/apimachinery/pkg/util/runtime",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/util/sets",
|
"//vendor:k8s.io/apimachinery/pkg/util/sets",
|
||||||
"//vendor:k8s.io/apimachinery/pkg/util/validation",
|
|
||||||
"//vendor:k8s.io/apiserver/pkg/endpoints/request",
|
"//vendor:k8s.io/apiserver/pkg/endpoints/request",
|
||||||
"//vendor:k8s.io/client-go/tools/cache",
|
"//vendor:k8s.io/client-go/tools/cache",
|
||||||
],
|
],
|
||||||
|
@ -20,7 +20,6 @@ package factory
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"strings"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/golang/glog"
|
"github.com/golang/glog"
|
||||||
@ -30,7 +29,6 @@ import (
|
|||||||
"k8s.io/apimachinery/pkg/types"
|
"k8s.io/apimachinery/pkg/types"
|
||||||
"k8s.io/apimachinery/pkg/util/runtime"
|
"k8s.io/apimachinery/pkg/util/runtime"
|
||||||
"k8s.io/apimachinery/pkg/util/sets"
|
"k8s.io/apimachinery/pkg/util/sets"
|
||||||
utilvalidation "k8s.io/apimachinery/pkg/util/validation"
|
|
||||||
genericapirequest "k8s.io/apiserver/pkg/endpoints/request"
|
genericapirequest "k8s.io/apiserver/pkg/endpoints/request"
|
||||||
"k8s.io/client-go/tools/cache"
|
"k8s.io/client-go/tools/cache"
|
||||||
"k8s.io/kubernetes/pkg/api/v1"
|
"k8s.io/kubernetes/pkg/api/v1"
|
||||||
@ -98,16 +96,13 @@ type ConfigFactory struct {
|
|||||||
// HardPodAffinitySymmetricWeight represents the weight of implicit PreferredDuringScheduling affinity rule, in the range 0-100.
|
// HardPodAffinitySymmetricWeight represents the weight of implicit PreferredDuringScheduling affinity rule, in the range 0-100.
|
||||||
hardPodAffinitySymmetricWeight int
|
hardPodAffinitySymmetricWeight int
|
||||||
|
|
||||||
// Indicate the "all topologies" set for empty topologyKey when it's used for PreferredDuringScheduling pod anti-affinity.
|
|
||||||
failureDomains []string
|
|
||||||
|
|
||||||
// Equivalence class cache
|
// Equivalence class cache
|
||||||
equivalencePodCache *core.EquivalenceCache
|
equivalencePodCache *core.EquivalenceCache
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewConfigFactory initializes the default implementation of a Configurator To encourage eventual privatization of the struct type, we only
|
// NewConfigFactory initializes the default implementation of a Configurator To encourage eventual privatization of the struct type, we only
|
||||||
// return the interface.
|
// return the interface.
|
||||||
func NewConfigFactory(client clientset.Interface, schedulerName string, hardPodAffinitySymmetricWeight int, failureDomains string) scheduler.Configurator {
|
func NewConfigFactory(client clientset.Interface, schedulerName string, hardPodAffinitySymmetricWeight int) scheduler.Configurator {
|
||||||
stopEverything := make(chan struct{})
|
stopEverything := make(chan struct{})
|
||||||
schedulerCache := schedulercache.New(30*time.Second, stopEverything)
|
schedulerCache := schedulercache.New(30*time.Second, stopEverything)
|
||||||
|
|
||||||
@ -132,7 +127,6 @@ func NewConfigFactory(client clientset.Interface, schedulerName string, hardPodA
|
|||||||
StopEverything: stopEverything,
|
StopEverything: stopEverything,
|
||||||
schedulerName: schedulerName,
|
schedulerName: schedulerName,
|
||||||
hardPodAffinitySymmetricWeight: hardPodAffinitySymmetricWeight,
|
hardPodAffinitySymmetricWeight: hardPodAffinitySymmetricWeight,
|
||||||
failureDomains: strings.Split(failureDomains, ","),
|
|
||||||
}
|
}
|
||||||
|
|
||||||
c.podLister = schedulerCache
|
c.podLister = schedulerCache
|
||||||
@ -200,10 +194,6 @@ func (c *ConfigFactory) GetHardPodAffinitySymmetricWeight() int {
|
|||||||
return c.hardPodAffinitySymmetricWeight
|
return c.hardPodAffinitySymmetricWeight
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ConfigFactory) GetFailureDomains() []string {
|
|
||||||
return c.failureDomains
|
|
||||||
}
|
|
||||||
|
|
||||||
func (f *ConfigFactory) GetSchedulerName() string {
|
func (f *ConfigFactory) GetSchedulerName() string {
|
||||||
return f.schedulerName
|
return f.schedulerName
|
||||||
}
|
}
|
||||||
@ -452,12 +442,6 @@ func (f *ConfigFactory) GetPredicates(predicateKeys sets.String) (map[string]alg
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (f *ConfigFactory) getPluginArgs() (*PluginFactoryArgs, error) {
|
func (f *ConfigFactory) getPluginArgs() (*PluginFactoryArgs, error) {
|
||||||
for _, failureDomain := range f.failureDomains {
|
|
||||||
if errs := utilvalidation.IsQualifiedName(failureDomain); len(errs) != 0 {
|
|
||||||
return nil, fmt.Errorf("invalid failure domain: %q: %s", failureDomain, strings.Join(errs, ";"))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return &PluginFactoryArgs{
|
return &PluginFactoryArgs{
|
||||||
PodLister: f.podLister,
|
PodLister: f.podLister,
|
||||||
ServiceLister: f.serviceLister,
|
ServiceLister: f.serviceLister,
|
||||||
@ -469,7 +453,6 @@ func (f *ConfigFactory) getPluginArgs() (*PluginFactoryArgs, error) {
|
|||||||
PVInfo: f.pVLister,
|
PVInfo: f.pVLister,
|
||||||
PVCInfo: &predicates.CachedPersistentVolumeClaimInfo{StoreToPersistentVolumeClaimLister: f.pVCLister},
|
PVCInfo: &predicates.CachedPersistentVolumeClaimInfo{StoreToPersistentVolumeClaimLister: f.pVCLister},
|
||||||
HardPodAffinitySymmetricWeight: f.hardPodAffinitySymmetricWeight,
|
HardPodAffinitySymmetricWeight: f.hardPodAffinitySymmetricWeight,
|
||||||
FailureDomains: sets.NewString(f.failureDomains...).List(),
|
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -49,7 +49,7 @@ func TestCreate(t *testing.T) {
|
|||||||
server := httptest.NewServer(&handler)
|
server := httptest.NewServer(&handler)
|
||||||
defer server.Close()
|
defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
factory.Create()
|
factory.Create()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -67,7 +67,7 @@ func TestCreateFromConfig(t *testing.T) {
|
|||||||
server := httptest.NewServer(&handler)
|
server := httptest.NewServer(&handler)
|
||||||
defer server.Close()
|
defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
|
|
||||||
// Pre-register some predicate and priority functions
|
// Pre-register some predicate and priority functions
|
||||||
RegisterFitPredicate("PredicateOne", PredicateOne)
|
RegisterFitPredicate("PredicateOne", PredicateOne)
|
||||||
@ -108,7 +108,7 @@ func TestCreateFromEmptyConfig(t *testing.T) {
|
|||||||
server := httptest.NewServer(&handler)
|
server := httptest.NewServer(&handler)
|
||||||
defer server.Close()
|
defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factory := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
|
|
||||||
configData = []byte(`{}`)
|
configData = []byte(`{}`)
|
||||||
if err := runtime.DecodeInto(latestschedulerapi.Codec, configData, &policy); err != nil {
|
if err := runtime.DecodeInto(latestschedulerapi.Codec, configData, &policy); err != nil {
|
||||||
@ -150,7 +150,7 @@ func TestDefaultErrorFunc(t *testing.T) {
|
|||||||
mux.Handle(testapi.Default.ResourcePath("pods", "bar", "foo"), &handler)
|
mux.Handle(testapi.Default.ResourcePath("pods", "bar", "foo"), &handler)
|
||||||
server := httptest.NewServer(mux)
|
server := httptest.NewServer(mux)
|
||||||
defer server.Close()
|
defer server.Close()
|
||||||
factory := NewConfigFactory(clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}}), v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factory := NewConfigFactory(clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}}), v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
queue := cache.NewFIFO(cache.MetaNamespaceKeyFunc)
|
queue := cache.NewFIFO(cache.MetaNamespaceKeyFunc)
|
||||||
podBackoff := util.CreatePodBackoff(1*time.Millisecond, 1*time.Second)
|
podBackoff := util.CreatePodBackoff(1*time.Millisecond, 1*time.Second)
|
||||||
errFunc := factory.MakeDefaultErrorFunc(podBackoff, queue)
|
errFunc := factory.MakeDefaultErrorFunc(podBackoff, queue)
|
||||||
@ -247,9 +247,9 @@ func TestResponsibleForPod(t *testing.T) {
|
|||||||
defer server.Close()
|
defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
// factory of "default-scheduler"
|
// factory of "default-scheduler"
|
||||||
factoryDefaultScheduler := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factoryDefaultScheduler := NewConfigFactory(client, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
// factory of "foo-scheduler"
|
// factory of "foo-scheduler"
|
||||||
factoryFooScheduler := NewConfigFactory(client, "foo-scheduler", v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
factoryFooScheduler := NewConfigFactory(client, "foo-scheduler", v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
// scheduler annotations to be tested
|
// scheduler annotations to be tested
|
||||||
schedulerFitsDefault := "default-scheduler"
|
schedulerFitsDefault := "default-scheduler"
|
||||||
schedulerFitsFoo := "foo-scheduler"
|
schedulerFitsFoo := "foo-scheduler"
|
||||||
@ -305,7 +305,7 @@ func TestInvalidHardPodAffinitySymmetricWeight(t *testing.T) {
|
|||||||
// defer server.Close()
|
// defer server.Close()
|
||||||
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
client := clientset.NewForConfigOrDie(&restclient.Config{Host: server.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
// factory of "default-scheduler"
|
// factory of "default-scheduler"
|
||||||
factory := NewConfigFactory(client, v1.DefaultSchedulerName, -1, v1.DefaultFailureDomains)
|
factory := NewConfigFactory(client, v1.DefaultSchedulerName, -1)
|
||||||
_, err := factory.Create()
|
_, err := factory.Create()
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Errorf("expected err: invalid hardPodAffinitySymmetricWeight, got nothing")
|
t.Errorf("expected err: invalid hardPodAffinitySymmetricWeight, got nothing")
|
||||||
@ -324,28 +324,20 @@ func TestInvalidFactoryArgs(t *testing.T) {
|
|||||||
|
|
||||||
testCases := []struct {
|
testCases := []struct {
|
||||||
hardPodAffinitySymmetricWeight int
|
hardPodAffinitySymmetricWeight int
|
||||||
failureDomains string
|
|
||||||
expectErr string
|
expectErr string
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
hardPodAffinitySymmetricWeight: -1,
|
hardPodAffinitySymmetricWeight: -1,
|
||||||
failureDomains: v1.DefaultFailureDomains,
|
|
||||||
expectErr: "invalid hardPodAffinitySymmetricWeight: -1, must be in the range 0-100",
|
expectErr: "invalid hardPodAffinitySymmetricWeight: -1, must be in the range 0-100",
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
hardPodAffinitySymmetricWeight: 101,
|
hardPodAffinitySymmetricWeight: 101,
|
||||||
failureDomains: v1.DefaultFailureDomains,
|
|
||||||
expectErr: "invalid hardPodAffinitySymmetricWeight: 101, must be in the range 0-100",
|
expectErr: "invalid hardPodAffinitySymmetricWeight: 101, must be in the range 0-100",
|
||||||
},
|
},
|
||||||
{
|
|
||||||
hardPodAffinitySymmetricWeight: 0,
|
|
||||||
failureDomains: "INVALID_FAILURE_DOMAINS",
|
|
||||||
expectErr: "invalid failure domain: INVALID_FAILURE_DOMAINS",
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, test := range testCases {
|
for _, test := range testCases {
|
||||||
factory := NewConfigFactory(client, v1.DefaultSchedulerName, test.hardPodAffinitySymmetricWeight, test.failureDomains)
|
factory := NewConfigFactory(client, v1.DefaultSchedulerName, test.hardPodAffinitySymmetricWeight)
|
||||||
_, err := factory.Create()
|
_, err := factory.Create()
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Errorf("expected err: %s, got nothing", test.expectErr)
|
t.Errorf("expected err: %s, got nothing", test.expectErr)
|
||||||
|
@ -43,7 +43,6 @@ type PluginFactoryArgs struct {
|
|||||||
PVInfo predicates.PersistentVolumeInfo
|
PVInfo predicates.PersistentVolumeInfo
|
||||||
PVCInfo predicates.PersistentVolumeClaimInfo
|
PVCInfo predicates.PersistentVolumeClaimInfo
|
||||||
HardPodAffinitySymmetricWeight int
|
HardPodAffinitySymmetricWeight int
|
||||||
FailureDomains []string
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// MetadataProducerFactory produces MetadataProducer from the given args.
|
// MetadataProducerFactory produces MetadataProducer from the given args.
|
||||||
|
@ -62,7 +62,6 @@ type Configurator interface {
|
|||||||
GetPredicateMetadataProducer() (algorithm.MetadataProducer, error)
|
GetPredicateMetadataProducer() (algorithm.MetadataProducer, error)
|
||||||
GetPredicates(predicateKeys sets.String) (map[string]algorithm.FitPredicate, error)
|
GetPredicates(predicateKeys sets.String) (map[string]algorithm.FitPredicate, error)
|
||||||
GetHardPodAffinitySymmetricWeight() int
|
GetHardPodAffinitySymmetricWeight() int
|
||||||
GetFailureDomains() []string
|
|
||||||
GetSchedulerName() string
|
GetSchedulerName() string
|
||||||
MakeDefaultErrorFunc(backoff *util.PodBackoff, podQueue *cache.FIFO) func(pod *v1.Pod, err error)
|
MakeDefaultErrorFunc(backoff *util.PodBackoff, podQueue *cache.FIFO) func(pod *v1.Pod, err error)
|
||||||
|
|
||||||
|
@ -239,7 +239,7 @@ func TestSchedulerExtender(t *testing.T) {
|
|||||||
}
|
}
|
||||||
policy.APIVersion = api.Registry.GroupOrDie(v1.GroupName).GroupVersion.String()
|
policy.APIVersion = api.Registry.GroupOrDie(v1.GroupName).GroupVersion.String()
|
||||||
|
|
||||||
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
schedulerConfig, err := schedulerConfigFactory.CreateFromConfig(policy)
|
schedulerConfig, err := schedulerConfigFactory.CreateFromConfig(policy)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Couldn't create scheduler config: %v", err)
|
t.Fatalf("Couldn't create scheduler config: %v", err)
|
||||||
|
@ -60,7 +60,7 @@ func TestUnschedulableNodes(t *testing.T) {
|
|||||||
|
|
||||||
clientSet := clientset.NewForConfigOrDie(&restclient.Config{Host: s.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
clientSet := clientset.NewForConfigOrDie(&restclient.Config{Host: s.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
|
|
||||||
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
schedulerConfig, err := schedulerConfigFactory.Create()
|
schedulerConfig, err := schedulerConfigFactory.Create()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Couldn't create scheduler config: %v", err)
|
t.Fatalf("Couldn't create scheduler config: %v", err)
|
||||||
@ -329,7 +329,7 @@ func TestMultiScheduler(t *testing.T) {
|
|||||||
// non-namespaced objects (Nodes).
|
// non-namespaced objects (Nodes).
|
||||||
defer clientSet.Core().Nodes().DeleteCollection(nil, metav1.ListOptions{})
|
defer clientSet.Core().Nodes().DeleteCollection(nil, metav1.ListOptions{})
|
||||||
|
|
||||||
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
schedulerConfig, err := schedulerConfigFactory.Create()
|
schedulerConfig, err := schedulerConfigFactory.Create()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Couldn't create scheduler config: %v", err)
|
t.Fatalf("Couldn't create scheduler config: %v", err)
|
||||||
@ -400,7 +400,7 @@ func TestMultiScheduler(t *testing.T) {
|
|||||||
// 5. create and start a scheduler with name "foo-scheduler"
|
// 5. create and start a scheduler with name "foo-scheduler"
|
||||||
clientSet2 := clientset.NewForConfigOrDie(&restclient.Config{Host: s.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
clientSet2 := clientset.NewForConfigOrDie(&restclient.Config{Host: s.URL, ContentConfig: restclient.ContentConfig{GroupVersion: &api.Registry.GroupOrDie(v1.GroupName).GroupVersion}})
|
||||||
|
|
||||||
schedulerConfigFactory2 := factory.NewConfigFactory(clientSet2, "foo-scheduler", v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigFactory2 := factory.NewConfigFactory(clientSet2, "foo-scheduler", v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
schedulerConfig2, err := schedulerConfigFactory2.Create()
|
schedulerConfig2, err := schedulerConfigFactory2.Create()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("Couldn't create scheduler config: %v", err)
|
t.Errorf("Couldn't create scheduler config: %v", err)
|
||||||
@ -495,7 +495,7 @@ func TestAllocatable(t *testing.T) {
|
|||||||
// non-namespaced objects (Nodes).
|
// non-namespaced objects (Nodes).
|
||||||
defer clientSet.Core().Nodes().DeleteCollection(nil, metav1.ListOptions{})
|
defer clientSet.Core().Nodes().DeleteCollection(nil, metav1.ListOptions{})
|
||||||
|
|
||||||
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigFactory := factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
schedulerConfig, err := schedulerConfigFactory.Create()
|
schedulerConfig, err := schedulerConfigFactory.Create()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Couldn't create scheduler config: %v", err)
|
t.Fatalf("Couldn't create scheduler config: %v", err)
|
||||||
|
@ -58,7 +58,7 @@ func mustSetupScheduler() (schedulerConfigurator scheduler.Configurator, destroy
|
|||||||
Burst: 5000,
|
Burst: 5000,
|
||||||
})
|
})
|
||||||
|
|
||||||
schedulerConfigurator = factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight, v1.DefaultFailureDomains)
|
schedulerConfigurator = factory.NewConfigFactory(clientSet, v1.DefaultSchedulerName, v1.DefaultHardPodAffinitySymmetricWeight)
|
||||||
|
|
||||||
eventBroadcaster := record.NewBroadcaster()
|
eventBroadcaster := record.NewBroadcaster()
|
||||||
eventBroadcaster.StartRecordingToSink(&clientv1core.EventSinkImpl{Interface: clientv1core.New(clientSet.Core().RESTClient()).Events("")})
|
eventBroadcaster.StartRecordingToSink(&clientv1core.EventSinkImpl{Interface: clientv1core.New(clientSet.Core().RESTClient()).Events("")})
|
||||||
|
Loading…
Reference in New Issue
Block a user