diff --git a/cmd/katalyst-agent/app/options/orm/orm_base.go b/cmd/katalyst-agent/app/options/orm/orm_base.go index 48eece7817..7b1c20c79c 100644 --- a/cmd/katalyst-agent/app/options/orm/orm_base.go +++ b/cmd/katalyst-agent/app/options/orm/orm_base.go @@ -25,16 +25,20 @@ import ( ) type GenericORMPluginOptions struct { - ORMRconcilePeriod time.Duration - ORMResourceNamesMap map[string]string - ORMPodNotifyChanLen int + ORMRconcilePeriod time.Duration + ORMResourceNamesMap map[string]string + ORMPodNotifyChanLen int + TopologyPolicyName string + NumericAlignResources []string } func NewGenericORMPluginOptions() *GenericORMPluginOptions { return &GenericORMPluginOptions{ - ORMRconcilePeriod: time.Second * 5, - ORMResourceNamesMap: map[string]string{}, - ORMPodNotifyChanLen: 10, + ORMRconcilePeriod: time.Second * 5, + ORMResourceNamesMap: map[string]string{}, + ORMPodNotifyChanLen: 10, + TopologyPolicyName: "none", + NumericAlignResources: []string{"cpu", "memory"}, } } @@ -50,12 +54,18 @@ func (o *GenericORMPluginOptions) AddFlags(fss *cliflag.NamedFlagSets) { "will also be allocated by [cpu] and [memory] QRM plugins") fs.IntVar(&o.ORMPodNotifyChanLen, "orm-pod-notify-chan-len", o.ORMPodNotifyChanLen, "length of pod addition and movement notifying channel") + fs.StringVar(&o.TopologyPolicyName, "topology-policy-name", + o.TopologyPolicyName, "topology merge policy name used by ORM") + fs.StringSliceVar(&o.NumericAlignResources, "numeric-align-resources", o.NumericAlignResources, + "resources which should be aligned in numeric topology policy") } func (o *GenericORMPluginOptions) ApplyTo(conf *ormconfig.GenericORMConfiguration) error { conf.ORMRconcilePeriod = o.ORMRconcilePeriod conf.ORMResourceNamesMap = o.ORMResourceNamesMap conf.ORMPodNotifyChanLen = o.ORMPodNotifyChanLen + conf.TopologyPolicyName = o.TopologyPolicyName + conf.NumericAlignResources = o.NumericAlignResources return nil } diff --git a/pkg/agent/resourcemanager/outofband/checkpoint_test.go b/pkg/agent/resourcemanager/outofband/checkpoint_test.go index 329dc9af39..e19c0df5ab 100644 --- a/pkg/agent/resourcemanager/outofband/checkpoint_test.go +++ b/pkg/agent/resourcemanager/outofband/checkpoint_test.go @@ -30,21 +30,21 @@ import ( func TestCheckpoint(t *testing.T) { t.Parallel() - checkpointManager, err := checkpointmanager.NewCheckpointManager("/tmp") + checkpointManager, err := checkpointmanager.NewCheckpointManager("/tmp/checkpoint/") assert.NoError(t, err) m := &ManagerImpl{ - socketdir: "/tmp", + socketdir: "/tmp/checkpoint", endpoints: make(map[string]endpoint.EndpointInfo), podResources: newPodResourcesChk(), checkpointManager: checkpointManager, } defer func() { - _ = os.Remove("/tmp/kubelet_qrm_checkpoint") + _ = os.Remove("/tmp/checkpoint/kubelet_qrm_checkpoint") }() file := m.checkpointFile() - assert.Equal(t, file, "/tmp/kubelet_qrm_checkpoint") + assert.Equal(t, file, "/tmp/checkpoint/kubelet_qrm_checkpoint") allocationInfo := generateResourceAllocationInfo() m.podResources.insert("testPod", "testContainer", "cpu", allocationInfo) diff --git a/pkg/agent/resourcemanager/outofband/endpoint/endpoint.go b/pkg/agent/resourcemanager/outofband/endpoint/endpoint.go index 59ca06e26c..21941b2ecd 100644 --- a/pkg/agent/resourcemanager/outofband/endpoint/endpoint.go +++ b/pkg/agent/resourcemanager/outofband/endpoint/endpoint.go @@ -36,6 +36,7 @@ import ( type Endpoint interface { Stop() Allocate(c context.Context, resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceAllocationResponse, error) + GetTopologyHints(c context.Context, resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceHintsResponse, error) GetResourceAllocation(c context.Context, request *pluginapi.GetResourcesAllocationRequest) (*pluginapi.GetResourcesAllocationResponse, error) RemovePod(c context.Context, removePodRequest *pluginapi.RemovePodRequest) (*pluginapi.RemovePodResponse, error) IsStopped() bool @@ -64,7 +65,7 @@ type EndpointImpl struct { func NewEndpointImpl(socketPath, resourceName string) (*EndpointImpl, error) { client, c, err := dial(socketPath) if err != nil { - klog.Errorf("[qosresourcemanager] Can't create new endpoint with path %s err %v", socketPath, err) + klog.Errorf("[ORM] Can't create new endpoint with path %s err %v", socketPath, err) return nil, err } @@ -150,6 +151,16 @@ func (e *EndpointImpl) GetResourcePluginOptions(ctx context.Context, in *plugina return e.client.GetResourcePluginOptions(ctx, in, opts...) } +func (e *EndpointImpl) GetTopologyHints(c context.Context, resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceHintsResponse, error) { + if e.IsStopped() { + return nil, fmt.Errorf(errEndpointStopped, e) + } + ctx, cancel := context.WithTimeout(c, pluginapi.KubeletResourcePluginGetTopologyHintsRPCTimeoutInSecs*time.Second) + defer cancel() + + return e.client.GetTopologyHints(ctx, resourceRequest) +} + // dial establishes the gRPC communication with the registered resource plugin. https://godoc.org/google.golang.org/grpc#Dial func dial(unixSocketPath string) (pluginapi.ResourcePluginClient, *grpc.ClientConn, error) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) diff --git a/pkg/agent/resourcemanager/outofband/manager.go b/pkg/agent/resourcemanager/outofband/manager.go index 9b2c85b685..ab25e91286 100644 --- a/pkg/agent/resourcemanager/outofband/manager.go +++ b/pkg/agent/resourcemanager/outofband/manager.go @@ -19,6 +19,7 @@ package outofband import ( "context" "fmt" + "math" "net" "os" "path/filepath" @@ -38,10 +39,12 @@ import ( "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/endpoint" "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/executor" "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/metamanager" + "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/topology" "github.com/kubewharf/katalyst-core/pkg/config" "github.com/kubewharf/katalyst-core/pkg/config/generic" "github.com/kubewharf/katalyst-core/pkg/metaserver" "github.com/kubewharf/katalyst-core/pkg/metrics" + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" cgroupmgr "github.com/kubewharf/katalyst-core/pkg/util/cgroup/manager" "github.com/kubewharf/katalyst-core/pkg/util/native" ) @@ -59,6 +62,8 @@ type ManagerImpl struct { metaManager *metamanager.Manager + topologyManager topology.Manager + server *grpc.Server wg sync.WaitGroup @@ -110,6 +115,14 @@ func NewManager(socketPath string, emitter metrics.MetricEmitter, metaServer *me metaManager := metamanager.NewManager(emitter, m.podResources.pods, metaServer) m.metaManager = metaManager + topologyManager, err := topology.NewManager(metaServer.Topology, config.TopologyPolicyName, config.NumericAlignResources) + if err != nil { + klog.Error(err) + return nil, err + } + topologyManager.AddHintProvider(m) + m.topologyManager = topologyManager + if err := m.removeContents(m.socketdir); err != nil { err = fmt.Errorf("[ORM] Fail to clean up stale contents under %s: %v", m.socketdir, err) klog.Error(err) @@ -179,6 +192,105 @@ func (m *ManagerImpl) GetHandlerType() string { return pluginregistration.ResourcePlugin } +func (m *ManagerImpl) GetTopologyHints(pod *v1.Pod, container *v1.Container) map[string][]topology.TopologyHint { + if pod == nil || container == nil { + klog.Errorf("[ORM] GetTopologyHints got nil pod: %v or container: %v", pod, container) + return nil + } + + podUID := string(pod.UID) + contName := container.Name + containerType, containerIndex, err := GetContainerTypeAndIndex(pod, container) + if err != nil { + return nil + } + + resourceHints := make(map[string][]topology.TopologyHint) + for resourceObj, requestedObj := range container.Resources.Requests { + requested := int(requestedObj.Value()) + resource, err := m.getMappedResourceName(string(resourceObj), container.Resources.Requests) + if err != nil { + klog.Errorf("resource %s getMappedResourceName fail: %v", string(resourceObj), err) + return nil + } + + if requestedObj.IsZero() { + continue + } + + allocationInfo := m.podResources.containerResource(podUID, contName, resource) + if allocationInfo != nil && allocationInfo.ResourceHints != nil && len(allocationInfo.ResourceHints.Hints) > 0 { + + allocated := int(math.Ceil(allocationInfo.AllocatedQuantity)) + + if allocationInfo.IsScalarResource && allocated >= requested { + resourceHints[resource] = ParseListOfTopologyHints(allocationInfo.ResourceHints) + klog.Warningf("[ORM] resource %s already allocated to (pod %s/%s, container %v) with larger number than request: requested: %d, allocated: %d; not to getTopologyHints", + resource, pod.GetNamespace(), pod.GetName(), container.Name, requested, allocated) + continue + } else { + klog.Warningf("[ORM] resource %s already allocated to (pod %s/%s, container %v) with smaller number than request: requested: %d, allocated: %d; continue to getTopologyHints", + resource, pod.GetNamespace(), pod.GetName(), container.Name, requested, int(math.Ceil(allocationInfo.AllocatedQuantity))) + } + } + + m.mutex.Lock() + e, ok := m.endpoints[resource] + m.mutex.Unlock() + if !ok || e.Opts == nil || !e.Opts.WithTopologyAlignment { + klog.V(5).Infof("[ORM] GetTopologyHints resource %s not supported", resource) + continue + } + + resourceReq := &pluginapi.ResourceRequest{ + PodUid: podUID, + PodNamespace: pod.GetNamespace(), + PodName: pod.GetName(), + ContainerName: container.Name, + ContainerType: containerType, + ContainerIndex: containerIndex, + PodRole: pod.Labels[pluginapi.PodRoleLabelKey], + PodType: pod.Annotations[pluginapi.PodTypeAnnotationKey], + Labels: maputil.CopySS(pod.Labels), + Annotations: maputil.CopySS(pod.Annotations), + // use mapped resource name in "ResourceName" to indicates which endpoint to request + ResourceName: resource, + // use original requested resource name in "ResourceRequests" in order to make plugin identity real requested resource name + ResourceRequests: map[string]float64{string(resourceObj): requestedObj.AsApproximateFloat64()}, + } + + resp, err := e.E.GetTopologyHints(context.Background(), resourceReq) + if err != nil { + klog.Errorf("[ORM] call GetTopologyHints of %s resource plugin for pod: %s/%s, container: %s failed with error: %v", + resource, pod.GetNamespace(), pod.GetName(), contName, err) + + resourceHints[resource] = []topology.TopologyHint{} + continue + } + + resourceHints[resource] = ParseListOfTopologyHints(resp.ResourceHints[resource]) + + klog.Infof("[ORM] GetTopologyHints for resource: %s, pod: %s/%s, container: %s, result: %+v", + resource, pod.Namespace, pod.Name, contName, resourceHints[resource]) + } + + return resourceHints +} + +func (m *ManagerImpl) Allocate(pod *v1.Pod, container *v1.Container) error { + if pod == nil || container == nil { + return fmt.Errorf("Allocate got nil pod: %v or container: %v", pod, container) + } + + err := m.addContainer(pod, container) + if err != nil { + return err + } + + err = m.syncContainer(pod, container) + return err +} + func (m *ManagerImpl) onPodAdd(podUID string) { klog.V(5).Infof("[ORM] onPodAdd: %v", podUID) @@ -240,17 +352,7 @@ func (m *ManagerImpl) processAddPod(podUID string) error { return err } - for _, container := range append(pod.Spec.InitContainers, pod.Spec.Containers...) { - err = m.addContainer(pod, &container) - if err != nil { - klog.Errorf("[ORM] add container fail, pod: %v, container: %v, err: %v", pod.Name, container.Name, err) - return err - } - - _ = m.syncContainer(pod, &container) - } - - return nil + return m.topologyManager.Admit(pod) } func (m *ManagerImpl) processDeletePod(podUID string) error { @@ -271,6 +373,7 @@ func (m *ManagerImpl) processDeletePod(podUID string) error { if allSuccess { m.podResources.deletePod(podUID) + m.topologyManager.RemovePod(podUID) } return m.writeCheckpoint() @@ -291,13 +394,33 @@ func (m *ManagerImpl) addContainer(pod *v1.Pod, container *v1.Container) error { return nil } + containerType, containerIndex, err := GetContainerTypeAndIndex(pod, container) + if err != nil { + return err + } + for k, v := range container.Resources.Requests { + needed := int(v.Value()) resource, err := m.getMappedResourceName(string(k), container.Resources.Requests) if err != nil { klog.Errorf("resource %s getMappedResourceName fail: %v", string(k), err) return err } + allocationInfo := m.podResources.containerResource(string(pod.UID), container.Name, resource) + if allocationInfo != nil { + allocated := int(math.Ceil(allocationInfo.AllocatedQuantity)) + + if allocationInfo.IsScalarResource && allocated >= needed { + klog.Infof("[ORM] resource %s already allocated to (pod %s/%s, container %v) with larger number than request: requested: %d, allocated: %d; not to allocate", + resource, pod.GetNamespace(), pod.GetName(), container.Name, needed, allocated) + continue + } else { + klog.Warningf("[ORM] resource %s already allocated to (pod %s/%s, container %v) with smaller number than request: requested: %d, allocated: %d; continue to allocate", + resource, pod.GetNamespace(), pod.GetName(), container.Name, needed, allocated) + } + } + m.mutex.Lock() e, ok := m.endpoints[resource] m.mutex.Unlock() @@ -306,11 +429,6 @@ func (m *ManagerImpl) addContainer(pod *v1.Pod, container *v1.Container) error { continue } - containerType, containerIndex, err := GetContainerTypeAndIndex(pod, container) - if err != nil { - return err - } - resourceReq := &pluginapi.ResourceRequest{ PodUid: string(pod.UID), PodNamespace: pod.GetNamespace(), @@ -327,8 +445,20 @@ func (m *ManagerImpl) addContainer(pod *v1.Pod, container *v1.Container) error { ResourceRequests: map[string]float64{resource: v.AsApproximateFloat64()}, Labels: maputil.CopySS(pod.Labels), Annotations: maputil.CopySS(pod.Annotations), - // hint is not used in ORM but it can not be nil - Hint: &pluginapi.TopologyHint{}, + } + + if e.Opts != nil && e.Opts.WithTopologyAlignment { + hint := m.topologyManager.GetAffinity(string(pod.UID), container.Name, resource) + + if hint.NUMANodeAffinity == nil { + klog.Warningf("[ORM] pod: %s/%s; container: %s allocate resource: %s without numa nodes affinity", + pod.Namespace, pod.Name, container.Name, resource) + } else { + klog.Warningf("[ORM] pod: %s/%s; container: %s allocate resource: %s get hint: %v from store", + pod.Namespace, pod.Name, container.Name, resource, hint) + } + + resourceReq.Hint = ParseTopologyManagerHint(hint) } response, err := e.E.Allocate(m.ctx, resourceReq) @@ -578,3 +708,46 @@ func isPodKatalystQoSLevelSystemCores(qosConfig *generic.QoSConfiguration, pod * return qosLevel == pluginapi.KatalystQoSLevelSystemCores, nil } + +func ParseListOfTopologyHints(hintsList *pluginapi.ListOfTopologyHints) []topology.TopologyHint { + if hintsList == nil { + return nil + } + + resultHints := make([]topology.TopologyHint, 0, len(hintsList.Hints)) + + for _, hint := range hintsList.Hints { + if hint != nil { + + mask := bitmask.NewEmptyBitMask() + + for _, node := range hint.Nodes { + mask.Add(int(node)) + } + + resultHints = append(resultHints, topology.TopologyHint{ + NUMANodeAffinity: mask, + Preferred: hint.Preferred, + }) + } + } + + return resultHints +} + +func ParseTopologyManagerHint(hint topology.TopologyHint) *pluginapi.TopologyHint { + var nodes []uint64 + + if hint.NUMANodeAffinity != nil { + bits := hint.NUMANodeAffinity.GetBits() + + for _, node := range bits { + nodes = append(nodes, uint64(node)) + } + } + + return &pluginapi.TopologyHint{ + Nodes: nodes, + Preferred: hint.Preferred, + } +} diff --git a/pkg/agent/resourcemanager/outofband/manager_test.go b/pkg/agent/resourcemanager/outofband/manager_test.go index dbd4c1ac37..9b8924e97a 100644 --- a/pkg/agent/resourcemanager/outofband/manager_test.go +++ b/pkg/agent/resourcemanager/outofband/manager_test.go @@ -24,6 +24,7 @@ import ( "testing" "time" + cadvisorapi "github.com/google/cadvisor/info/v1" "github.com/stretchr/testify/assert" "google.golang.org/grpc" v1 "k8s.io/api/core/v1" @@ -39,6 +40,7 @@ import ( "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/endpoint" "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/executor" "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/metamanager" + "github.com/kubewharf/katalyst-core/pkg/agent/resourcemanager/outofband/topology" "github.com/kubewharf/katalyst-core/pkg/config" "github.com/kubewharf/katalyst-core/pkg/config/generic" "github.com/kubewharf/katalyst-core/pkg/metaserver" @@ -109,7 +111,13 @@ func TestProcess(t *testing.T) { qosConfig: generic.NewQoSConfiguration(), } defer func() { _ = os.Remove("/tmp/process/kubelet_qrm_checkpoint") }() - + topologyManager, _ := topology.NewManager([]cadvisorapi.Node{ + { + Id: 0, + }, + }, "restricted", nil) + topologyManager.AddHintProvider(m) + m.topologyManager = topologyManager err = registerEndpointByRes(m, testResources) assert.NoError(t, err) @@ -188,7 +196,13 @@ func TestReconcile(t *testing.T) { qosConfig: generic.NewQoSConfiguration(), } defer func() { _ = os.Remove("/tmp/reconcile/kubelet_qrm_checkpoint") }() - + topologyManager, _ := topology.NewManager([]cadvisorapi.Node{ + { + Id: 0, + }, + }, "none", nil) + topologyManager.AddHintProvider(m) + m.topologyManager = topologyManager err = registerEndpointByPods(m, pods) assert.NoError(t, err) @@ -366,6 +380,14 @@ func TestRun(t *testing.T) { metaManager := metamanager.NewManager(metrics.DummyMetrics{}, m.podResources.pods, metaServer) m.metaManager = metaManager + topologyManager, _ := topology.NewManager([]cadvisorapi.Node{ + { + Id: 0, + }, + }, "none", nil) + topologyManager.AddHintProvider(m) + m.topologyManager = topologyManager + err = registerEndpointByPods(m, pods) assert.NoError(t, err) @@ -469,6 +491,12 @@ func registerEndpointByRes(manager *ManagerImpl, testRes []TestResource) error { resp.AllocationResult.ResourceAllocation[curResourceName].OciPropertyName = OciPropertyName return resp, nil }, + topologyHints: []*pluginapi.TopologyHint{ + { + Nodes: []uint64{0}, + Preferred: true, + }, + }, }) } else if res.resourceName == "domain3.com/resource3" { manager.registerEndpoint(curResourceName, &pluginapi.ResourcePluginOptions{ @@ -479,6 +507,12 @@ func registerEndpointByRes(manager *ManagerImpl, testRes []TestResource) error { allocateFunc: func(req *pluginapi.ResourceRequest) (*pluginapi.ResourceAllocationResponse, error) { return nil, fmt.Errorf("mock error") }, + topologyHints: []*pluginapi.TopologyHint{ + { + Nodes: []uint64{0}, + Preferred: true, + }, + }, }) } } @@ -528,6 +562,12 @@ func registerEndpointByPods(manager *ManagerImpl, pods []*v1.Pod) error { allocateFunc: func(resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceAllocationResponse, error) { return &pluginapi.ResourceAllocationResponse{}, nil }, + topologyHints: []*pluginapi.TopologyHint{ + { + Nodes: []uint64{0}, + Preferred: true, + }, + }, }) } @@ -539,6 +579,7 @@ type MockEndpoint struct { allocateFunc func(resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceAllocationResponse, error) resourceAlloc func(ctx context.Context, request *pluginapi.GetResourcesAllocationRequest) (*pluginapi.GetResourcesAllocationResponse, error) stopTime time.Time + topologyHints []*pluginapi.TopologyHint } func (m *MockEndpoint) Stop() { @@ -556,6 +597,27 @@ func (m *MockEndpoint) Allocate(ctx context.Context, resourceRequest *pluginapi. return nil, nil } +func (m *MockEndpoint) GetTopologyHints(c context.Context, resourceRequest *pluginapi.ResourceRequest) (*pluginapi.ResourceHintsResponse, error) { + return &pluginapi.ResourceHintsResponse{ + PodUid: resourceRequest.PodUid, + PodNamespace: resourceRequest.PodNamespace, + PodName: resourceRequest.PodName, + ContainerName: resourceRequest.ContainerName, + ContainerIndex: resourceRequest.ContainerIndex, + ContainerType: resourceRequest.ContainerType, + PodRole: resourceRequest.PodRole, + PodType: resourceRequest.PodType, + ResourceName: resourceRequest.ResourceName, + Labels: resourceRequest.Labels, + Annotations: resourceRequest.Annotations, + ResourceHints: map[string]*pluginapi.ListOfTopologyHints{ + resourceRequest.ResourceName: { + Hints: m.topologyHints, + }, + }, + }, nil +} + func (m *MockEndpoint) IsStopped() bool { return !m.stopTime.IsZero() } diff --git a/pkg/agent/resourcemanager/outofband/pluginhandler.go b/pkg/agent/resourcemanager/outofband/pluginhandler.go index 052304ff8b..e5fb224118 100644 --- a/pkg/agent/resourcemanager/outofband/pluginhandler.go +++ b/pkg/agent/resourcemanager/outofband/pluginhandler.go @@ -168,7 +168,7 @@ func (m *ManagerImpl) Register(ctx context.Context, r *pluginapi.RegisterRequest func (m *ManagerImpl) addEndpoint(r *pluginapi.RegisterRequest, success chan<- bool) { new, err := endpoint2.NewEndpointImpl(filepath.Join(m.socketdir, r.Endpoint), r.ResourceName) if err != nil { - klog.Errorf("[qosresourcemanager] Failed to dial resource plugin with request %v: %v", r, err) + klog.Errorf("[ORM] Failed to dial resource plugin with request %v: %v", r, err) success <- false return } diff --git a/pkg/agent/resourcemanager/outofband/pod_resource.go b/pkg/agent/resourcemanager/outofband/pod_resource.go index 86a4280c8e..8c1abd79ec 100644 --- a/pkg/agent/resourcemanager/outofband/pod_resource.go +++ b/pkg/agent/resourcemanager/outofband/pod_resource.go @@ -168,6 +168,23 @@ func (pres *podResourcesChk) containerAllResources(podUID, contName string) Reso return pres.resources[podUID][contName].DeepCopy() } +func (pres *podResourcesChk) containerResource(podUID, contName, resource string) *pluginapi.ResourceAllocationInfo { + pres.RLock() + defer pres.RUnlock() + + if _, podExists := pres.resources[podUID]; !podExists { + return nil + } + if _, contExists := pres.resources[podUID][contName]; !contExists { + return nil + } + resourceAllocationInfo, resourceExists := pres.resources[podUID][contName][resource] + if !resourceExists || resourceAllocationInfo == nil { + return nil + } + return proto.Clone(resourceAllocationInfo).(*pluginapi.ResourceAllocationInfo) +} + // Turns podResourcesChk to checkpointData. func (pres *podResourcesChk) toCheckpointData() []checkpoint.PodResourcesEntry { pres.RLock() diff --git a/pkg/agent/resourcemanager/outofband/topology/manager.go b/pkg/agent/resourcemanager/outofband/topology/manager.go new file mode 100644 index 0000000000..e19ddafb3e --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/manager.go @@ -0,0 +1,224 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "fmt" + "sync" + + cadvisorapi "github.com/google/cadvisor/info/v1" + v1 "k8s.io/api/core/v1" + "k8s.io/klog/v2" +) + +const ( + // maxAllowableNUMANodes specifies the maximum number of NUMA Nodes that + // the TopologyManager supports on the underlying machine. + // + // At present, having more than this number of NUMA Nodes will result in a + // state explosion when trying to enumerate possible NUMAAffinity masks and + // generate hints for them. As such, if more NUMA Nodes than this are + // present on a machine and the TopologyManager is enabled, an error will + // be returned and the TopologyManager will not be loaded. + maxAllowableNUMANodes = 8 + // defaultResourceKey is the key to store the default hint for those resourceNames + // which don't specify hint. + defaultResourceKey = "*" +) + +type Manager interface { + Admit(pod *v1.Pod) error + + AddHintProvider(provider HintProvider) + + GetAffinity(podUID string, containerName string, resourceName string) TopologyHint + + RemovePod(podUID string) +} + +// HintProvider is an interface for components that want to collaborate to +// achieve globally optimal concrete resource alignment with respect to +// NUMA locality. +type HintProvider interface { + // GetTopologyHints returns a map of resource names to a list of possible + // concrete resource allocations in terms of NUMA locality hints. Each hint + // is optionally marked "preferred" and indicates the set of NUMA nodes + // involved in the hypothetical allocation. The topology manager calls + // this function for each hint provider, and merges the hints to produce + // a consensus "best" hint. The hint providers may subsequently query the + // topology manager to influence actual resource assignment. + GetTopologyHints(pod *v1.Pod, container *v1.Container) map[string][]TopologyHint + // Allocate triggers resource allocation to occur on the HintProvider after + // all hints have been gathered and the aggregated Hint is available via a + // call to GetAffinity(). + Allocate(pod *v1.Pod, container *v1.Container) error +} + +type manager struct { + mutex sync.Mutex + // Mapping of a Pods mapping of Containers and their TopologyHints + // Indexed by PodUID to ContainerName + podTopologyHints map[string]podTopologyHints + // The list of components registered with the Manager + hintProviders []HintProvider + // Topology Manager Policy + policy Policy +} + +func NewManager(topology []cadvisorapi.Node, topologyPolicyName string, alignResources []string) (Manager, error) { + klog.InfoS("Creating topology manager with policy per scope", "topologyPolicyName", topologyPolicyName) + + var numaNodes []int + for _, node := range topology { + numaNodes = append(numaNodes, node.Id) + } + + if topologyPolicyName != PolicyNone && len(numaNodes) > maxAllowableNUMANodes { + return nil, fmt.Errorf("unsupported on machines with more than %v NUMA Nodes", maxAllowableNUMANodes) + } + + var policy Policy + switch topologyPolicyName { + case PolicyNone: + policy = NewNonePolicy() + + case PolicyBestEffort: + policy = NewBestEffortPolicy(numaNodes) + + case PolicyRestricted: + policy = NewRestrictedPolicy(numaNodes) + + case PolicySingleNumaNode: + policy = NewSingleNumaNodePolicy(numaNodes) + + case PolicyNumeric: + policy = NewNumericPolicy(alignResources) + + default: + return nil, fmt.Errorf("unknown policy: \"%s\"", topologyPolicyName) + } + + m := &manager{ + podTopologyHints: map[string]podTopologyHints{}, + hintProviders: make([]HintProvider, 0), + policy: policy, + } + return m, nil +} + +func (m *manager) Admit(pod *v1.Pod) error { + if m.policy.Name() == PolicyNone { + return m.admitPolicyNone(pod) + } + + for _, container := range append(pod.Spec.InitContainers, pod.Spec.Containers...) { + bestHint, admit := m.calculateAffinity(pod, &container) + klog.V(3).Infof("Best TopologyHint, bestHint: %v, pod: %v, containerName: %v", bestHint, klog.KObj(pod), container.Name) + + if !admit { + err := fmt.Errorf("pod: %v, containerName: %v not admit", pod.Name, container.Name) + return err + } + klog.V(3).Infof("Topology Affinity, bestHint: %v, pod: %v, containerName: %v", bestHint, klog.KObj(pod), container.Name) + m.setTopologyHints(string(pod.UID), container.Name, bestHint) + + err := m.allocateAlignedResources(pod, &container) + if err != nil { + klog.Errorf("allocateAlignedResources fail, pod: %v, containerName: %v, err: %v", klog.KObj(pod), container.Name, err) + return err + } + } + + return nil +} + +func (m *manager) admitPolicyNone(pod *v1.Pod) error { + for _, container := range append(pod.Spec.InitContainers, pod.Spec.Containers...) { + err := m.allocateAlignedResources(pod, &container) + if err != nil { + klog.Errorf("allocateAlignedResources fail, pod: %v, containerName: %v, err: %v", klog.KObj(pod), container.Name, err) + return err + } + } + + return nil +} + +func (m *manager) AddHintProvider(provider HintProvider) { + m.hintProviders = append(m.hintProviders, provider) +} + +func (m *manager) GetAffinity(podUID string, containerName string, resourceName string) TopologyHint { + return m.getTopologyHints(podUID, containerName, resourceName) +} + +func (m *manager) calculateAffinity(pod *v1.Pod, container *v1.Container) (map[string]TopologyHint, bool) { + providersHints := m.accumulateProvidersHints(pod, container) + bestHint, admit := m.policy.Merge(providersHints) + klog.V(3).Infof("ContainerTopologyHint, bestHint: %v", bestHint) + return bestHint, admit +} + +func (m *manager) accumulateProvidersHints(pod *v1.Pod, container *v1.Container) []map[string][]TopologyHint { + var providersHints []map[string][]TopologyHint + + for _, provider := range m.hintProviders { + // Get the TopologyHints for a Container from a provider. + hints := provider.GetTopologyHints(pod, container) + providersHints = append(providersHints, hints) + klog.V(3).Infof("TopologyHints, hints: %v, pod: %v, containerName: %v", hints, klog.KObj(pod), container.Name) + } + return providersHints +} + +func (m *manager) allocateAlignedResources(pod *v1.Pod, container *v1.Container) error { + for _, provider := range m.hintProviders { + err := provider.Allocate(pod, container) + if err != nil { + return err + } + } + return nil +} + +func (m *manager) setTopologyHints(podUID string, containerName string, th map[string]TopologyHint) { + m.mutex.Lock() + defer m.mutex.Unlock() + + if m.podTopologyHints[podUID] == nil { + m.podTopologyHints[podUID] = make(map[string]map[string]TopologyHint) + } + m.podTopologyHints[podUID][containerName] = th +} + +func (m *manager) getTopologyHints(podUID string, containerName string, resourceName string) TopologyHint { + m.mutex.Lock() + defer m.mutex.Unlock() + hint, ok := m.podTopologyHints[podUID][containerName][resourceName] + if ok { + return hint + } + return m.podTopologyHints[podUID][containerName][defaultResourceKey] +} + +func (m *manager) RemovePod(podUID string) { + m.mutex.Lock() + defer m.mutex.Unlock() + + klog.V(3).Infof("RemovePod, podUID: %v", podUID) + delete(m.podTopologyHints, podUID) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/manager_test.go b/pkg/agent/resourcemanager/outofband/topology/manager_test.go new file mode 100644 index 0000000000..01aee71654 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/manager_test.go @@ -0,0 +1,535 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "fmt" + "reflect" + "strings" + "testing" + + "github.com/stretchr/testify/assert" + v1 "k8s.io/api/core/v1" + v12 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/kubernetes/pkg/kubelet/lifecycle" + + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" +) + +func NewTestBitMask(sockets ...int) bitmask.BitMask { + s, _ := bitmask.NewBitMask(sockets...) + return s +} + +type mockHintProvider struct { + th map[string][]TopologyHint + //TODO: Add this field and add some tests to make sure things error out + //appropriately on allocation errors. + //allocateError error +} + +func (m *mockHintProvider) GetTopologyHints(pod *v1.Pod, container *v1.Container) map[string][]TopologyHint { + return m.th +} + +func (m *mockHintProvider) Allocate(pod *v1.Pod, container *v1.Container) error { + //return allocateError + return nil +} + +func TestNewManager(t *testing.T) { + t.Parallel() + tcases := []struct { + description string + policyName string + expectedPolicy string + expectedError error + }{ + { + description: "Policy is set to none", + policyName: "none", + expectedPolicy: "none", + }, + { + description: "Policy is set to best-effort", + policyName: "best-effort", + expectedPolicy: "best-effort", + }, + { + description: "Policy is set to restricted", + policyName: "restricted", + expectedPolicy: "restricted", + }, + { + description: "Policy is set to single-numa-node", + policyName: "single-numa-node", + expectedPolicy: "single-numa-node", + }, + { + description: "Policy is set to unknown", + policyName: "unknown", + expectedError: fmt.Errorf("unknown policy: \"unknown\""), + }, + } + + for _, tc := range tcases { + mngr, err := NewManager(nil, tc.policyName, nil) + + if tc.expectedError != nil { + if !strings.Contains(err.Error(), tc.expectedError.Error()) { + t.Errorf("Unexpected error message. Have: %s wants %s", err.Error(), tc.expectedError.Error()) + } + } else { + rawMgr := mngr.(*manager) + if rawMgr.policy.Name() != tc.expectedPolicy { + t.Errorf("Unexpected policy name. Have: %q wants %q", rawMgr.policy.Name(), tc.expectedPolicy) + } + } + } +} + +func TestAddHintProvider(t *testing.T) { + t.Parallel() + tcases := []struct { + name string + hp []HintProvider + }{ + { + name: "Add HintProvider", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{}, + &mockHintProvider{}, + }, + }, + } + mngr := manager{ + hintProviders: make([]HintProvider, 0), + } + for _, tc := range tcases { + for _, hp := range tc.hp { + mngr.AddHintProvider(hp) + } + if len(tc.hp) != len(mngr.hintProviders) { + t.Errorf("error") + } + } +} + +func TestGetAffinity(t *testing.T) { + t.Parallel() + tcases := []struct { + name string + resourceName string + containerName string + podUID string + expected TopologyHint + }{ + { + name: "case1", + resourceName: "*", + containerName: "nginx", + podUID: "0aafa4c4-38e8-11e9-bcb1-a4bf01040474", + expected: TopologyHint{}, + }, + { + name: "case2", + containerName: "preferredContainer", + resourceName: "cpu", + podUID: "testpoduid", + expected: TopologyHint{ + Preferred: true, + NUMANodeAffinity: NewTestBitMask(0), + }, + }, + { + name: "case3", + resourceName: "cpu", + containerName: "notpreferedContainer", + podUID: "testpoduid", + expected: TopologyHint{ + Preferred: false, + NUMANodeAffinity: NewTestBitMask(0, 1), + }, + }, + } + + mngr := manager{ + podTopologyHints: map[string]podTopologyHints{}, + } + mngr.setTopologyHints("testpoduid", "preferredContainer", map[string]TopologyHint{ + "cpu": { + Preferred: true, + NUMANodeAffinity: NewTestBitMask(0), + }, + }) + mngr.setTopologyHints("testpoduid", "notpreferedContainer", map[string]TopologyHint{ + "cpu": { + Preferred: false, + NUMANodeAffinity: NewTestBitMask(0, 1), + }, + }) + + for _, tc := range tcases { + actual := mngr.GetAffinity(tc.podUID, tc.containerName, tc.resourceName) + if !reflect.DeepEqual(actual, tc.expected) { + t.Errorf("Expected Affinity in result to be %v, got %v", tc.expected, actual) + } + } +} + +func TestRemovePod(t *testing.T) { + t.Parallel() + mngr := manager{ + podTopologyHints: map[string]podTopologyHints{}, + } + mngr.setTopologyHints("testpoduid", "testContainer", map[string]TopologyHint{ + "cpu": { + Preferred: true, + NUMANodeAffinity: NewTestBitMask(0), + }, + }) + + mngr.RemovePod("none") + assert.Equal(t, 1, len(mngr.podTopologyHints)) + mngr.RemovePod("testpoduid") + assert.Equal(t, 0, len(mngr.podTopologyHints)) +} + +func TestAdmit(t *testing.T) { + t.Parallel() + numaNodes := []int{0, 1} + + tcases := []struct { + name string + result lifecycle.PodAdmitResult + policy Policy + hp []HintProvider + expectedErr error + expected TopologyHint + }{ + { + name: "None Policy. No Hints.", + policy: NewNonePolicy(), + hp: []HintProvider{}, + expectedErr: nil, + expected: TopologyHint{}, + }, + { + name: "None Policy. No Hints.", + policy: NewNonePolicy(), + hp: []HintProvider{}, + expectedErr: nil, + expected: TopologyHint{}, + }, + { + name: "single-numa-node Policy. No Hints.", + policy: NewSingleNumaNodePolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{}, + }, + expectedErr: nil, + expected: TopologyHint{}, + }, + { + name: "Restricted Policy. No Hints.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{}, + }, + expectedErr: nil, + expected: TopologyHint{}, + }, + { + name: "BestEffort Policy. Preferred Affinity.", + policy: NewBestEffortPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "BestEffort Policy. More than one Preferred Affinity.", + policy: NewBestEffortPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "BestEffort Policy. More than one Preferred Affinity.", + policy: NewBestEffortPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "BestEffort Policy. No Preferred Affinity.", + policy: NewBestEffortPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + { + name: "Restricted Policy. Preferred Affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "Restricted Policy. Preferred Affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "Restricted Policy. More than one Preferred affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "Restricted Policy. More than one Preferred affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: nil, + expected: TopologyHint{ + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + { + name: "Restricted Policy. No Preferred affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: fmt.Errorf("pod: %v, containerName: %v not admit", "testPod", "testContainer"), + expected: TopologyHint{}, + }, + { + name: "Restricted Policy. No Preferred affinity.", + policy: NewRestrictedPolicy(numaNodes), + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expectedErr: fmt.Errorf("pod: %v, containerName: %v not admit", "testPod", "testContainer"), + expected: TopologyHint{}, + }, + } + for _, tc := range tcases { + topologyManager := manager{ + policy: tc.policy, + hintProviders: tc.hp, + podTopologyHints: map[string]podTopologyHints{}, + } + + pod := &v1.Pod{ + ObjectMeta: v12.ObjectMeta{ + Name: "testPod", + UID: "testUID", + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "testContainer", + Resources: v1.ResourceRequirements{}, + }, + }, + }, + Status: v1.PodStatus{}, + } + + err := topologyManager.Admit(pod) + assert.Equal(t, tc.expectedErr, err) + if err != nil { + assert.Equal(t, tc.expected, topologyManager.GetAffinity("testUID", "testContainer", "resource")) + } + } +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy.go b/pkg/agent/resourcemanager/outofband/topology/policy.go new file mode 100644 index 0000000000..46c497677c --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy.go @@ -0,0 +1,349 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "k8s.io/klog/v2" + + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" +) + +// Policy interface for Topology Manager Pod Admit Result +type Policy interface { + // Returns Policy Name + Name() string + // Returns a merged TopologyHint based on input from hint providers + // and a Pod Admit Handler Response based on hints and policy type + Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) +} + +// Merge a TopologyHints permutation to a single hint by performing a bitwise-AND +// of their affinity masks. The hint shall be preferred if all hits in the permutation +// are preferred. +func mergePermutation(numaNodes []int, permutation []TopologyHint) TopologyHint { + // Get the NUMANodeAffinity from each hint in the permutation and see if any + // of them encode unpreferred allocations. + preferred := true + defaultAffinity, _ := bitmask.NewBitMask(numaNodes...) + var numaAffinities []bitmask.BitMask + for _, hint := range permutation { + // Only consider hints that have an actual NUMANodeAffinity set. + if hint.NUMANodeAffinity != nil { + numaAffinities = append(numaAffinities, hint.NUMANodeAffinity) + // Only mark preferred if all affinities are equal. + if !hint.NUMANodeAffinity.IsEqual(numaAffinities[0]) { + preferred = false + } + } + // Only mark preferred if all affinities are preferred. + if !hint.Preferred { + preferred = false + } + } + + // Merge the affinities using a bitwise-and operation. + mergedAffinity := bitmask.And(defaultAffinity, numaAffinities...) + // Build a mergedHint from the merged affinity mask, setting preferred as + // appropriate based on the logic above. + return TopologyHint{mergedAffinity, preferred} +} + +func filterProvidersHints(providersHints []map[string][]TopologyHint) ([][]TopologyHint, []string) { + // Loop through all hint providers and save an accumulated list of the + // hints returned by each hint provider. If no hints are provided, assume + // that provider has no preference for topology-aware allocation. + var ( + allProviderHints [][]TopologyHint + resourceNames []string + ) + for _, hints := range providersHints { + // If hints is nil, insert a single, preferred any-numa hint into allProviderHints. + if len(hints) == 0 { + klog.InfoS("Hint Provider has no preference for NUMA affinity with any resource") + allProviderHints = append(allProviderHints, []TopologyHint{{nil, true}}) + // Here, we add a defaultResourceKey to resourceNames because we don't know + // which resource this hint is for. + resourceNames = append(resourceNames, defaultResourceKey) + continue + } + + // Otherwise, accumulate the hints for each resource type into allProviderHints. + for resource := range hints { + resourceNames = append(resourceNames, resource) + if hints[resource] == nil { + klog.InfoS("Hint Provider has no preference for NUMA affinity with resource", "resource", resource) + allProviderHints = append(allProviderHints, []TopologyHint{{nil, true}}) + continue + } + + if len(hints[resource]) == 0 { + klog.InfoS("Hint Provider has no possible NUMA affinities for resource", "resource", resource) + allProviderHints = append(allProviderHints, []TopologyHint{{nil, false}}) + continue + } + + allProviderHints = append(allProviderHints, hints[resource]) + } + } + return allProviderHints, resourceNames +} + +func narrowestHint(hints []TopologyHint) *TopologyHint { + if len(hints) == 0 { + return nil + } + var narrowestHint *TopologyHint + for i := range hints { + if hints[i].NUMANodeAffinity == nil { + continue + } + if narrowestHint == nil { + narrowestHint = &hints[i] + } + if hints[i].NUMANodeAffinity.IsNarrowerThan(narrowestHint.NUMANodeAffinity) { + narrowestHint = &hints[i] + } + } + return narrowestHint +} + +func maxOfMinAffinityCounts(filteredHints [][]TopologyHint) int { + maxOfMinCount := 0 + for _, resourceHints := range filteredHints { + narrowestHint := narrowestHint(resourceHints) + if narrowestHint == nil { + continue + } + if narrowestHint.NUMANodeAffinity.Count() > maxOfMinCount { + maxOfMinCount = narrowestHint.NUMANodeAffinity.Count() + } + } + return maxOfMinCount +} + +func compareHints(bestNonPreferredAffinityCount int, current *TopologyHint, candidate *TopologyHint) *TopologyHint { + // Only consider candidates that result in a NUMANodeAffinity > 0 to + // replace the current bestHint. + if candidate.NUMANodeAffinity.Count() == 0 { + return current + } + + // If no current bestHint is set, return the candidate as the bestHint. + if current == nil { + return candidate + } + + // If the current bestHint is non-preferred and the candidate hint is + // preferred, always choose the preferred hint over the non-preferred one. + if !current.Preferred && candidate.Preferred { + return candidate + } + + // If the current bestHint is preferred and the candidate hint is + // non-preferred, never update the bestHint, regardless of the + // candidate hint's narowness. + if current.Preferred && !candidate.Preferred { + return current + } + + // If the current bestHint and the candidate hint are both preferred, + // then only consider candidate hints that have a narrower + // NUMANodeAffinity than the NUMANodeAffinity in the current bestHint. + if current.Preferred && candidate.Preferred { + if candidate.NUMANodeAffinity.IsNarrowerThan(current.NUMANodeAffinity) { + return candidate + } + return current + } + + // The only case left is if the current best bestHint and the candidate + // hint are both non-preferred. In this case, try and find a hint whose + // affinity count is as close to (but not higher than) the + // bestNonPreferredAffinityCount as possible. To do this we need to + // consider the following cases and react accordingly: + // + // 1. current.NUMANodeAffinity.Count() > bestNonPreferredAffinityCount + // 2. current.NUMANodeAffinity.Count() == bestNonPreferredAffinityCount + // 3. current.NUMANodeAffinity.Count() < bestNonPreferredAffinityCount + // + // For case (1), the current bestHint is larger than the + // bestNonPreferredAffinityCount, so updating to any narrower mergeHint + // is preferred over staying where we are. + // + // For case (2), the current bestHint is equal to the + // bestNonPreferredAffinityCount, so we would like to stick with what + // we have *unless* the candidate hint is also equal to + // bestNonPreferredAffinityCount and it is narrower. + // + // For case (3), the current bestHint is less than + // bestNonPreferredAffinityCount, so we would like to creep back up to + // bestNonPreferredAffinityCount as close as we can. There are three + // cases to consider here: + // + // 3a. candidate.NUMANodeAffinity.Count() > bestNonPreferredAffinityCount + // 3b. candidate.NUMANodeAffinity.Count() == bestNonPreferredAffinityCount + // 3c. candidate.NUMANodeAffinity.Count() < bestNonPreferredAffinityCount + // + // For case (3a), we just want to stick with the current bestHint + // because choosing a new hint that is greater than + // bestNonPreferredAffinityCount would be counter-productive. + // + // For case (3b), we want to immediately update bestHint to the + // candidate hint, making it now equal to bestNonPreferredAffinityCount. + // + // For case (3c), we know that *both* the current bestHint and the + // candidate hint are less than bestNonPreferredAffinityCount, so we + // want to choose one that brings us back up as close to + // bestNonPreferredAffinityCount as possible. There are three cases to + // consider here: + // + // 3ca. candidate.NUMANodeAffinity.Count() > current.NUMANodeAffinity.Count() + // 3cb. candidate.NUMANodeAffinity.Count() < current.NUMANodeAffinity.Count() + // 3cc. candidate.NUMANodeAffinity.Count() == current.NUMANodeAffinity.Count() + // + // For case (3ca), we want to immediately update bestHint to the + // candidate hint because that will bring us closer to the (higher) + // value of bestNonPreferredAffinityCount. + // + // For case (3cb), we want to stick with the current bestHint because + // choosing the candidate hint would strictly move us further away from + // the bestNonPreferredAffinityCount. + // + // Finally, for case (3cc), we know that the current bestHint and the + // candidate hint are equal, so we simply choose the narrower of the 2. + + // Case 1 + if current.NUMANodeAffinity.Count() > bestNonPreferredAffinityCount { + if candidate.NUMANodeAffinity.IsNarrowerThan(current.NUMANodeAffinity) { + return candidate + } + return current + } + // Case 2 + if current.NUMANodeAffinity.Count() == bestNonPreferredAffinityCount { + if candidate.NUMANodeAffinity.Count() != bestNonPreferredAffinityCount { + return current + } + if candidate.NUMANodeAffinity.IsNarrowerThan(current.NUMANodeAffinity) { + return candidate + } + return current + } + // Case 3a + if candidate.NUMANodeAffinity.Count() > bestNonPreferredAffinityCount { + return current + } + // Case 3b + if candidate.NUMANodeAffinity.Count() == bestNonPreferredAffinityCount { + return candidate + } + // Case 3ca + if candidate.NUMANodeAffinity.Count() > current.NUMANodeAffinity.Count() { + return candidate + } + // Case 3cb + if candidate.NUMANodeAffinity.Count() < current.NUMANodeAffinity.Count() { + return current + } + // Case 3cc + if candidate.NUMANodeAffinity.IsNarrowerThan(current.NUMANodeAffinity) { + return candidate + } + return current +} + +func mergeFilteredHints(numaNodes []int, filteredHints [][]TopologyHint) TopologyHint { + // Set bestNonPreferredAffinityCount to help decide which affinity mask is + // preferred amongst all non-preferred hints. We calculate this value as + // the maximum of the minimum affinity counts supplied for any given hint + // provider. In other words, prefer a hint that has an affinity mask that + // includes all of the NUMA nodes from the provider that requires the most + // NUMA nodes to satisfy its allocation. + bestNonPreferredAffinityCount := maxOfMinAffinityCounts(filteredHints) + + var bestHint *TopologyHint + iterateAllProviderTopologyHints(filteredHints, func(permutation []TopologyHint) { + // Get the NUMANodeAffinity from each hint in the permutation and see if any + // of them encode unpreferred allocations. + mergedHint := mergePermutation(numaNodes, permutation) + + // Compare the current bestHint with the candidate mergedHint and + // update bestHint if appropriate. + bestHint = compareHints(bestNonPreferredAffinityCount, bestHint, &mergedHint) + }) + + if bestHint == nil { + defaultAffinity, _ := bitmask.NewBitMask(numaNodes...) + bestHint = &TopologyHint{defaultAffinity, false} + } + + return *bestHint +} + +// Iterate over all permutations of hints in 'allProviderHints [][]TopologyHint'. +// +// This procedure is implemented as a recursive function over the set of hints +// in 'allproviderHints[i]'. It applies the function 'callback' to each +// permutation as it is found. It is the equivalent of: +// +// for i := 0; i < len(providerHints[0]); i++ +// +// for j := 0; j < len(providerHints[1]); j++ +// for k := 0; k < len(providerHints[2]); k++ +// ... +// for z := 0; z < len(providerHints[-1]); z++ +// permutation := []TopologyHint{ +// providerHints[0][i], +// providerHints[1][j], +// providerHints[2][k], +// ... +// providerHints[-1][z] +// } +// callback(permutation) +func iterateAllProviderTopologyHints(allProviderHints [][]TopologyHint, callback func([]TopologyHint)) { + // Internal helper function to accumulate the permutation before calling the callback. + var iterate func(i int, accum []TopologyHint) + iterate = func(i int, accum []TopologyHint) { + // Base case: we have looped through all providers and have a full permutation. + if i == len(allProviderHints) { + callback(accum) + return + } + + // Loop through all hints for provider 'i', and recurse to build the + // the permutation of this hint with all hints from providers 'i++'. + for j := range allProviderHints[i] { + iterate(i+1, append(accum, allProviderHints[i][j])) + } + } + iterate(0, []TopologyHint{}) +} + +// generateResourceHints generates the map from resourceName to given hint. +// all providers get a same bestHint under native policy(None,best_effort,restricted,single_numa_node), +// we just map resources to the bestHint +func generateResourceHints(resourceNames []string, hint TopologyHint) map[string]TopologyHint { + result := make(map[string]TopologyHint) + for _, resource := range resourceNames { + result[resource] = hint + } + // If non resourceNames are provided, we add defaultResourceKey here. + if len(resourceNames) == 0 { + result[defaultResourceKey] = hint + } + return result +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_best_effort.go b/pkg/agent/resourcemanager/outofband/topology/policy_best_effort.go new file mode 100644 index 0000000000..f17396b77c --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_best_effort.go @@ -0,0 +1,47 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +type bestEffortPolicy struct { + //List of NUMA Nodes available on the underlying machine + numaNodes []int +} + +var _ Policy = &bestEffortPolicy{} + +// PolicyBestEffort policy name. +const PolicyBestEffort string = "best-effort" + +// NewBestEffortPolicy returns best-effort policy. +func NewBestEffortPolicy(numaNodes []int) Policy { + return &bestEffortPolicy{numaNodes: numaNodes} +} + +func (p *bestEffortPolicy) Name() string { + return PolicyBestEffort +} + +func (p *bestEffortPolicy) canAdmitPodResult(hint *TopologyHint) bool { + return true +} + +func (p *bestEffortPolicy) Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) { + filteredProvidersHints, resourceNames := filterProvidersHints(providersHints) + bestHint := mergeFilteredHints(p.numaNodes, filteredProvidersHints) + admit := p.canAdmitPodResult(&bestHint) + return generateResourceHints(resourceNames, bestHint), admit +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_best_effort_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_best_effort_test.go new file mode 100644 index 0000000000..a78fa7b554 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_best_effort_test.go @@ -0,0 +1,60 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "testing" +) + +func TestPolicyBestEffortCanAdmitPodResult(t *testing.T) { + tcases := []struct { + name string + hint TopologyHint + expected bool + }{ + { + name: "Preferred is set to false in topology hints", + hint: TopologyHint{nil, false}, + expected: true, + }, + { + name: "Preferred is set to true in topology hints", + hint: TopologyHint{nil, true}, + expected: true, + }, + } + + for _, tc := range tcases { + numaNodes := []int{0, 1} + policy := NewBestEffortPolicy(numaNodes) + result := policy.(*bestEffortPolicy).canAdmitPodResult(&tc.hint) + + if result != tc.expected { + t.Errorf("Expected result to be %t, got %t", tc.expected, result) + } + } +} + +func TestPolicyBestEffortMerge(t *testing.T) { + numaNodes := []int{0, 1, 2, 3} + policy := NewBestEffortPolicy(numaNodes) + + tcases := commonPolicyMergeTestCases(numaNodes) + tcases = append(tcases, policy.(*bestEffortPolicy).mergeTestCases(numaNodes)...) + + testPolicyMerge(policy, tcases, t) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_none.go b/pkg/agent/resourcemanager/outofband/topology/policy_none.go new file mode 100644 index 0000000000..4cdd686101 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_none.go @@ -0,0 +1,41 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +type nonePolicy struct{} + +var _ Policy = &nonePolicy{} + +// PolicyNone policy name. +const PolicyNone string = "none" + +// NewNonePolicy returns none policy. +func NewNonePolicy() Policy { + return &nonePolicy{} +} + +func (p *nonePolicy) Name() string { + return PolicyNone +} + +func (p *nonePolicy) canAdmitPodResult(hint *TopologyHint) bool { + return true +} + +func (p *nonePolicy) Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) { + return map[string]TopologyHint{}, p.canAdmitPodResult(nil) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_none_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_none_test.go new file mode 100644 index 0000000000..3e1a363248 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_none_test.go @@ -0,0 +1,117 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "testing" +) + +func TestPolicyNoneName(t *testing.T) { + tcases := []struct { + name string + expected string + }{ + { + name: "New None Policy", + expected: "none", + }, + } + for _, tc := range tcases { + policy := NewNonePolicy() + if policy.Name() != tc.expected { + t.Errorf("Expected Policy Name to be %s, got %s", tc.expected, policy.Name()) + } + } +} + +func TestPolicyNoneCanAdmitPodResult(t *testing.T) { + tcases := []struct { + name string + hint TopologyHint + expected bool + }{ + { + name: "Preferred is set to false in topology hints", + hint: TopologyHint{nil, false}, + expected: true, + }, + { + name: "Preferred is set to true in topology hints", + hint: TopologyHint{nil, true}, + expected: true, + }, + } + + for _, tc := range tcases { + policy := NewNonePolicy() + result := policy.(*nonePolicy).canAdmitPodResult(&tc.hint) + + if result != tc.expected { + t.Errorf("Expected result to be %t, got %t", tc.expected, result) + } + } +} + +func TestPolicyNoneMerge(t *testing.T) { + tcases := []struct { + name string + providersHints []map[string][]TopologyHint + expectedHint map[string]TopologyHint + expectedAdmit bool + }{ + { + name: "merged empty providers hints", + providersHints: []map[string][]TopologyHint{}, + expectedHint: map[string]TopologyHint{}, + expectedAdmit: true, + }, + { + name: "merge with a single provider with a single preferred resource", + providersHints: []map[string][]TopologyHint{ + { + "resource": {{NUMANodeAffinity: NewTestBitMask(0, 1), Preferred: true}}, + }, + }, + expectedHint: map[string]TopologyHint{}, + expectedAdmit: true, + }, + { + name: "merge with a single provider with a single non-preferred resource", + providersHints: []map[string][]TopologyHint{ + { + "resource": {{NUMANodeAffinity: NewTestBitMask(0, 1), Preferred: false}}, + }, + }, + expectedHint: map[string]TopologyHint{}, + expectedAdmit: true, + }, + } + + for _, tc := range tcases { + policy := NewNonePolicy() + result, admit := policy.Merge(tc.providersHints) + for resource, hint := range result { + expected := tc.expectedHint[resource] + if !hint.IsEqual(expected) { + t.Errorf("Test Case: %s: Expected merge hint to be %v, got %v", tc.name, tc.expectedHint, result) + } + } + if admit != tc.expectedAdmit { + t.Errorf("Test Case: %s: Expected merge hint to be %v, got %v", tc.name, tc.expectedHint, result) + } + } +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_numeric.go b/pkg/agent/resourcemanager/outofband/topology/policy_numeric.go new file mode 100644 index 0000000000..7478dcb9f8 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_numeric.go @@ -0,0 +1,254 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "sort" + + v1 "k8s.io/api/core/v1" + "k8s.io/klog/v2" + + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" +) + +// numericPolicy implements a policy: +// 1. for align resources, hints should be totally equal and at-least-one preferred. +// 2. for other resources, bigger hints always contains smaller hints. +// 3. more preferredHint count hints permutation is preferred. +// 4. smaller maxNumaCount hints permutation is preferred. +type numericPolicy struct { + // alignResourceNames are those resources which should be aligned in numa node. + alignResourceNames []string +} + +// PolicyNumeric policy name. +const PolicyNumeric string = "numeric" + +var defaultAlignResourceNames = []string{v1.ResourceCPU.String(), v1.ResourceMemory.String()} + +// NewNumericPolicy returns numeric policy. +func NewNumericPolicy(alignResourceNames []string) Policy { + if alignResourceNames == nil { + alignResourceNames = defaultAlignResourceNames + } + return &numericPolicy{ + alignResourceNames: alignResourceNames, + } +} + +// Name returns numericPolicy name +func (p *numericPolicy) Name() string { + return PolicyNumeric +} + +func (p *numericPolicy) Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) { + if len(providersHints) == 0 { + return map[string]TopologyHint{ + defaultResourceKey: {nil, true}, + }, true + } + + if existEmptyHintSlice(providersHints) { + klog.Infof("[numeric_policy] admit failed due to existing empty hint slice") + return nil, false + } + + filteredHints, resourceNames := filterProvidersHints(providersHints) + + bestHints := findBestNumericPermutation(filteredHints, getAlignResourceIndexes(resourceNames, p.alignResourceNames)) + // no permutation fits the policy + if bestHints == nil { + return nil, false + } + + if len(bestHints) != len(resourceNames) { + // This should not happen. + klog.Warningf("[numeric policy] wrong hints length %d vs resource length %d", len(bestHints), len(resourceNames)) + return nil, false + } + + result := make(map[string]TopologyHint) + for i := range resourceNames { + result[resourceNames[i]] = bestHints[i] + } + return result, true +} + +// existEmptyHintSlice returns true if there is empty hint slice in providersHints +func existEmptyHintSlice(providersHints []map[string][]TopologyHint) bool { + for _, hints := range providersHints { + for resource := range hints { + // hint providers return nil if there is no possible NUMA affinity for resource + // hint providers return empty slice if there is no preference NUMA affinity for resource + if hints[resource] != nil && len(hints[resource]) == 0 { + klog.Infof("[numeric_policy] hint Provider has no possible NUMA affinity for resource: %s", resource) + return true + } + } + } + + return false +} + +// findBestNumericPermutation finds the best numeric permutation. +func findBestNumericPermutation(filteredHints [][]TopologyHint, alignResourceIndexes []int) []TopologyHint { + var bestHints []TopologyHint + + iterateAllProviderTopologyHints(filteredHints, func(permutation []TopologyHint) { + // the length of permutation and the order of the resources hints in it are equal to filteredHints, + // align and unaligned resource hints can be filtered by alignResourceIndexes + + // 1. check if align resource hints are equal, + // and there should be at least one preferred hint. + var alignHasPreferred bool + for i := 0; i < len(alignResourceIndexes)-1; i++ { + cur := alignResourceIndexes[i] + next := alignResourceIndexes[i+1] + + if !numaAffinityAligned(permutation[cur], permutation[next]) { + // hints are not aligned + return + } + alignHasPreferred = permutation[cur].Preferred || permutation[next].Preferred + } + if len(alignResourceIndexes) == 1 { + alignHasPreferred = permutation[alignResourceIndexes[0]].Preferred + } + if len(alignResourceIndexes) > 0 && !alignHasPreferred { + // all hints are not preferred + return + } + + // 2. check if bigger numa-node hints contains smaller numa-node hints. + if !isSubsetPermutation(permutation) { + return + } + + if bestHints == nil { + bestHints = DeepCopyTopologyHints(permutation) + } + + // 3. If preferredHint count beside align resources in this permutation is larger than + // that in current bestHints, always choose more preferredHint permutation. + if preferredCountBesideAlign(permutation, alignResourceIndexes) > + preferredCountBesideAlign(bestHints, alignResourceIndexes) { + bestHints = DeepCopyTopologyHints(permutation) + return + } + + // 4. Only Consider permutation that have smaller maxNumaCount than the + // maxNumaCount in the current bestHint. + if getMaxNumaCount(permutation) < getMaxNumaCount(bestHints) { + bestHints = DeepCopyTopologyHints(permutation) + return + } + }) + + return bestHints +} + +// getAlignResourceIndexes gets align resource indexes in resources array. +func getAlignResourceIndexes(resources []string, alignResourceNames []string) []int { + resourceIndexes := make(map[string]int) + for i, rn := range resources { + resourceIndexes[rn] = i + } + var result []int + for _, align := range alignResourceNames { + index, ok := resourceIndexes[align] + if ok { + result = append(result, index) + } + } + return result +} + +// getMaxNumaCount returns the max numa count in the given hints. +func getMaxNumaCount(permutation []TopologyHint) int { + var result int + for _, hint := range permutation { + if hint.NUMANodeAffinity == nil { + continue + } + if hint.NUMANodeAffinity.Count() > result { + result = hint.NUMANodeAffinity.Count() + } + } + return result +} + +// preferredCountBesideAlign counts the preferred hints beside align resources. +func preferredCountBesideAlign(hints []TopologyHint, alignIndexes []int) int { + var result int + alignIndexesMap := map[int]bool{} + for _, index := range alignIndexes { + alignIndexesMap[index] = true + } + for i, hint := range hints { + if _, ok := alignIndexesMap[i]; ok { + continue + } + if hint.Preferred { + result++ + } + } + return result +} + +// numaAffinityAligned checks a,b TopologyHint could be aligned or not. +func numaAffinityAligned(a, b TopologyHint) bool { + if a.NUMANodeAffinity == nil && b.NUMANodeAffinity == nil { + return a.Preferred == b.Preferred + } else if a.NUMANodeAffinity == nil { // b.NUMANodeAffinity != nil + // if a.Preferred, there is no NUMA preference for a, so it can be aligned with b. + return a.Preferred + } else if b.NUMANodeAffinity == nil { // a.NUMANodeAffinity != nil + // if b.Preferred, there is no NUMA preference for b, so it can be aligned with a. + return b.Preferred + } + + // NUMANodeAffinity of alignResources should be totally equal + return a.NUMANodeAffinity.IsEqual(b.NUMANodeAffinity) +} + +// isSubsetPermutation checks whether permutation meets that bigger numa-node hints always +// contain smaller numa-node hints or not. +func isSubsetPermutation(permutation []TopologyHint) bool { + // When NUMANodeAffinity is nil, means this has no preference. + // We should ignore it. + var filters []TopologyHint + for _, hint := range permutation { + if hint.NUMANodeAffinity != nil { + filters = append(filters, hint) + } + } + + // Sort from small numa node count to big count. + sort.Slice(filters, func(i, j int) bool { + return filters[i].NUMANodeAffinity.Count() <= filters[j].NUMANodeAffinity.Count() + }) + + for i := 0; i < len(filters)-1; i++ { + cur := filters[i] + next := filters[i+1] + if !bitmask.And(next.NUMANodeAffinity, cur.NUMANodeAffinity).IsEqual(cur.NUMANodeAffinity) { + return false + } + } + + return true +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_numeric_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_numeric_test.go new file mode 100644 index 0000000000..0990a71a41 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_numeric_test.go @@ -0,0 +1,909 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import "testing" + +func TestPolicyNumericMerge(t *testing.T) { + policy := NewNumericPolicy(defaultAlignResourceNames) + + tcases := []policyMergeTestCase{ + { + name: "Two providers, 1 hint each, same mask, both preferred 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, both preferred 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 no hints, 1 single hint preferred 1/2", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 no hints, 1 single hint preferred 2/2", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single hint matching 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + + { + name: "Two providers, 1 with 2 hints, 1 with single hint matching 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Two providers, both with 2 hints, matching narrower preferred hint from both", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Ensure less narrow preferred hints are chosen over narrower non-preferred hints", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Multiple resources, same provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "TopologyHint not set", + hp: []HintProvider{}, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{}, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns -nil map[string][]TopologyHint from provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": nil, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint from provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": {}, + }, + }, + }, + expected: nil, + }, + { + name: "Single TopologyHint with Preferred as true and NUMANodeAffinity as nil", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 hint each, no common mask", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: nil, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single non-preferred hint matching", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + { + name: "Numeric hint generation, two resource", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Align cpu, memory, cpu with 2/2 hit, memory with 1/2, 2/2 hint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "memory": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "cpu": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + "memory": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + "gpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Align cpu, memory, cpu with 2/2 hit, memory with 1/2, 2/2 hint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "memory": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: nil, + }, + { + name: "Align cpu, cpu with 2/2 hit", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: nil, + }, + { + name: "Align cpu, cpu with 1/2, 2/2 hit", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "cpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "gpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + { + name: "Align cpu, cpu with 1/2 hit, gpu with 1/2, 2/2 hint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "cpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "gpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Align cpu, memory, cpu with 2/2 hit, memory with 1/2, 2/2 hint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "gpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "memory": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "cpu": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + "memory": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + "gpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Align cpu, memory, nil, nil provider to test append bug", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{}, + &mockHintProvider{ + map[string][]TopologyHint{ + "cpu": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + "memory": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + "cpu": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "memory": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Align cpu, memory. cpu nil preferred nil hint, memory with 2/2 hint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "memory": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "cpu": { + { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "memory": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + "cpu": { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + } + testPolicyMerge(policy, tcases, t) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_restricted.go b/pkg/agent/resourcemanager/outofband/topology/policy_restricted.go new file mode 100644 index 0000000000..d2d7681882 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_restricted.go @@ -0,0 +1,46 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +type restrictedPolicy struct { + bestEffortPolicy +} + +var _ Policy = &restrictedPolicy{} + +// PolicyRestricted policy name. +const PolicyRestricted string = "restricted" + +// NewRestrictedPolicy returns restricted policy. +func NewRestrictedPolicy(numaNodes []int) Policy { + return &restrictedPolicy{bestEffortPolicy{numaNodes: numaNodes}} +} + +func (p *restrictedPolicy) Name() string { + return PolicyRestricted +} + +func (p *restrictedPolicy) canAdmitPodResult(hint *TopologyHint) bool { + return hint.Preferred +} + +func (p *restrictedPolicy) Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) { + filteredHints, resourceNames := filterProvidersHints(providersHints) + hint := mergeFilteredHints(p.numaNodes, filteredHints) + admit := p.canAdmitPodResult(&hint) + return generateResourceHints(resourceNames, hint), admit +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_restricted_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_restricted_test.go new file mode 100644 index 0000000000..e9d3730d15 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_restricted_test.go @@ -0,0 +1,78 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "testing" +) + +func TestPolicyRestrictedName(t *testing.T) { + tcases := []struct { + name string + expected string + }{ + { + name: "New Restricted Policy", + expected: "restricted", + }, + } + for _, tc := range tcases { + policy := NewRestrictedPolicy([]int{0, 1}) + if policy.Name() != tc.expected { + t.Errorf("Expected Policy Name to be %s, got %s", tc.expected, policy.Name()) + } + } +} + +func TestPolicyRestrictedCanAdmitPodResult(t *testing.T) { + tcases := []struct { + name string + hint TopologyHint + expected bool + }{ + { + name: "Preferred is set to false in topology hints", + hint: TopologyHint{nil, false}, + expected: false, + }, + { + name: "Preferred is set to true in topology hints", + hint: TopologyHint{nil, true}, + expected: true, + }, + } + + for _, tc := range tcases { + numaNodes := []int{0, 1} + policy := NewRestrictedPolicy(numaNodes) + result := policy.(*restrictedPolicy).canAdmitPodResult(&tc.hint) + + if result != tc.expected { + t.Errorf("Expected result to be %t, got %t", tc.expected, result) + } + } +} + +func TestPolicyRestrictedMerge(t *testing.T) { + numaNodes := []int{0, 1, 2, 3} + policy := NewRestrictedPolicy(numaNodes) + + tcases := commonPolicyMergeTestCases(numaNodes) + tcases = append(tcases, policy.(*restrictedPolicy).mergeTestCases(numaNodes)...) + + testPolicyMerge(policy, tcases, t) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node.go b/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node.go new file mode 100644 index 0000000000..796cab8b69 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node.go @@ -0,0 +1,77 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" +) + +type singleNumaNodePolicy struct { + //List of NUMA Nodes available on the underlying machine + numaNodes []int +} + +var _ Policy = &singleNumaNodePolicy{} + +// PolicySingleNumaNode policy name. +const PolicySingleNumaNode string = "single-numa-node" + +// NewSingleNumaNodePolicy returns single-numa-node policy. +func NewSingleNumaNodePolicy(numaNodes []int) Policy { + return &singleNumaNodePolicy{numaNodes: numaNodes} +} + +func (p *singleNumaNodePolicy) Name() string { + return PolicySingleNumaNode +} + +func (p *singleNumaNodePolicy) canAdmitPodResult(hint *TopologyHint) bool { + return hint.Preferred +} + +// Return hints that have valid bitmasks with exactly one bit set. +func filterSingleNumaHints(allResourcesHints [][]TopologyHint) [][]TopologyHint { + var filteredResourcesHints [][]TopologyHint + for _, oneResourceHints := range allResourcesHints { + var filtered []TopologyHint + for _, hint := range oneResourceHints { + if hint.NUMANodeAffinity == nil && hint.Preferred { + filtered = append(filtered, hint) + } + if hint.NUMANodeAffinity != nil && hint.NUMANodeAffinity.Count() == 1 && hint.Preferred { + filtered = append(filtered, hint) + } + } + filteredResourcesHints = append(filteredResourcesHints, filtered) + } + return filteredResourcesHints +} + +func (p *singleNumaNodePolicy) Merge(providersHints []map[string][]TopologyHint) (map[string]TopologyHint, bool) { + filteredHints, resourceNames := filterProvidersHints(providersHints) + // Filter to only include don't cares and hints with a single NUMA node. + singleNumaHints := filterSingleNumaHints(filteredHints) + bestHint := mergeFilteredHints(p.numaNodes, singleNumaHints) + + defaultAffinity, _ := bitmask.NewBitMask(p.numaNodes...) + if bestHint.NUMANodeAffinity.IsEqual(defaultAffinity) { + bestHint = TopologyHint{nil, bestHint.Preferred} + } + + admit := p.canAdmitPodResult(&bestHint) + return generateResourceHints(resourceNames, bestHint), admit +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node_test.go new file mode 100644 index 0000000000..3be0b41228 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_single_numa_node_test.go @@ -0,0 +1,166 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "reflect" + "testing" +) + +func TestPolicySingleNumaNodeCanAdmitPodResult(t *testing.T) { + tcases := []struct { + name string + hint TopologyHint + expected bool + }{ + { + name: "Preferred is set to false in topology hints", + hint: TopologyHint{nil, false}, + expected: false, + }, + } + + for _, tc := range tcases { + numaNodes := []int{0, 1} + policy := NewSingleNumaNodePolicy(numaNodes) + result := policy.(*singleNumaNodePolicy).canAdmitPodResult(&tc.hint) + + if result != tc.expected { + t.Errorf("Expected result to be %t, got %t", tc.expected, result) + } + } +} + +func TestPolicySingleNumaNodeFilterHints(t *testing.T) { + tcases := []struct { + name string + allResources [][]TopologyHint + expectedResources [][]TopologyHint + }{ + { + name: "filter empty resources", + allResources: [][]TopologyHint{}, + expectedResources: [][]TopologyHint(nil), + }, + { + name: "filter hints with nil socket mask 1/2", + allResources: [][]TopologyHint{ + { + {NUMANodeAffinity: nil, Preferred: false}, + }, + { + {NUMANodeAffinity: nil, Preferred: true}, + }, + }, + expectedResources: [][]TopologyHint{ + []TopologyHint(nil), + { + {NUMANodeAffinity: nil, Preferred: true}, + }, + }, + }, + { + name: "filter hints with nil socket mask 2/2", + allResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + {NUMANodeAffinity: nil, Preferred: false}, + }, + { + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + {NUMANodeAffinity: nil, Preferred: true}, + }, + }, + expectedResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + }, + { + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + {NUMANodeAffinity: nil, Preferred: true}, + }, + }, + }, + { + name: "filter hints with empty resource socket mask", + allResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + {NUMANodeAffinity: nil, Preferred: false}, + }, + {}, + }, + expectedResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + }, + []TopologyHint(nil), + }, + }, + { + name: "filter hints with wide sockemask", + allResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + {NUMANodeAffinity: NewTestBitMask(1, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(0, 1, 2), Preferred: false}, + {NUMANodeAffinity: nil, Preferred: false}, + }, + { + {NUMANodeAffinity: NewTestBitMask(1, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(0, 1, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(0, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(3), Preferred: false}, + }, + { + {NUMANodeAffinity: NewTestBitMask(1, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(0, 1, 2), Preferred: false}, + {NUMANodeAffinity: NewTestBitMask(0, 2), Preferred: false}, + }, + }, + expectedResources: [][]TopologyHint{ + { + {NUMANodeAffinity: NewTestBitMask(0), Preferred: true}, + {NUMANodeAffinity: NewTestBitMask(1), Preferred: true}, + }, + []TopologyHint(nil), + []TopologyHint(nil), + }, + }, + } + + for _, tc := range tcases { + actual := filterSingleNumaHints(tc.allResources) + if !reflect.DeepEqual(tc.expectedResources, actual) { + t.Errorf("Test Case: %s", tc.name) + t.Errorf("Expected result to be %v, got %v", tc.expectedResources, actual) + } + } +} + +func TestPolicySingleNumaNodeMerge(t *testing.T) { + numaNodes := []int{0, 1, 2, 3} + policy := NewSingleNumaNodePolicy(numaNodes) + + tcases := commonPolicyMergeTestCases(numaNodes) + tcases = append(tcases, policy.(*singleNumaNodePolicy).mergeTestCases(numaNodes)...) + + testPolicyMerge(policy, tcases, t) +} diff --git a/pkg/agent/resourcemanager/outofband/topology/policy_test.go b/pkg/agent/resourcemanager/outofband/topology/policy_test.go new file mode 100644 index 0000000000..26255fe746 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/policy_test.go @@ -0,0 +1,1595 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import ( + "reflect" + "testing" + + v1 "k8s.io/api/core/v1" + + "github.com/kubewharf/katalyst-core/pkg/util/bitmask" +) + +type policyMergeTestCase struct { + name string + hp []HintProvider + expected map[string]TopologyHint +} + +func commonPolicyMergeTestCases(numaNodes []int) []policyMergeTestCase { + return []policyMergeTestCase{ + { + name: "Two providers, 1 hint each, same mask, both preferred 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, both preferred 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 no hints, 1 single hint preferred 1/2", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 no hints, 1 single hint preferred 2/2", + hp: []HintProvider{ + &mockHintProvider{}, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single hint matching 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single hint matching 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Two providers, both with 2 hints, matching narrower preferred hint from both", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + { + name: "Ensure less narrow preferred hints are chosen over narrower non-preferred hints", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + { + name: "Multiple resources, same provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + } +} + +func (p *bestEffortPolicy) mergeTestCases(numaNodes []int) []policyMergeTestCase { + return []policyMergeTestCase{ + { + name: "Two providers, 2 hints each, same mask (some with different bits), same preferred", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 2), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 2), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + }, + { + name: "TopologyHint not set", + hp: []HintProvider{}, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{}, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns -nil map[string][]TopologyHint from provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": nil, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint from provider", hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": {}, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: false, + }, + }, + }, + { + name: "Single TopologyHint with Preferred as true and NUMANodeAffinity as nil", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: true, + }, + }, + }, + { + name: "Single TopologyHint with Preferred as false and NUMANodeAffinity as nil", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, no common mask", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(numaNodes...), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, 1 wider mask, both preferred 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single non-preferred hint matching", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, 1 wider mask, both preferred 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (1)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (2)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 3), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0, 3), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(0, 3), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (3)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (4)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(2, 3), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(2, 3), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(2, 3), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (5)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(2, 3), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + }, + }, + { + name: "bestNonPreferredAffinityCount (6)", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1, 2, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(1, 3), + Preferred: false, + }, + { + NUMANodeAffinity: NewTestBitMask(2, 3), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: NewTestBitMask(1, 2), + Preferred: false, + }, + }, + }, + } +} + +func (p *singleNumaNodePolicy) mergeTestCases(numaNodes []int) []policyMergeTestCase { + return []policyMergeTestCase{ + { + name: "TopologyHint not set", + hp: []HintProvider{}, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{}, + }, + }, + expected: map[string]TopologyHint{ + defaultResourceKey: { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns -nil map[string][]TopologyHint from provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": nil, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "HintProvider returns empty non-nil map[string][]TopologyHint from provider", hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": {}, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Single TopologyHint with Preferred as true and NUMANodeAffinity as nil", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: true, + }, + }, + }, + { + name: "Single TopologyHint with Preferred as false and NUMANodeAffinity as nil", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource": { + { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, no common mask", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: nil, + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 1/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: nil, + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 hint each, same mask, 1 preferred, 1 not 2/2", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: nil, + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Two providers, 1 with 2 hints, 1 with single non-preferred hint matching", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + }, + }, + }, + &mockHintProvider{ + map[string][]TopologyHint{ + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: nil, + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "Single NUMA hint generation", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: true, + }, + }, + "resource2": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: nil, + Preferred: false, + }, + "resource2": { + NUMANodeAffinity: nil, + Preferred: false, + }, + }, + }, + { + name: "One no-preference provider", + hp: []HintProvider{ + &mockHintProvider{ + map[string][]TopologyHint{ + "resource1": { + { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(1), + Preferred: true, + }, + { + NUMANodeAffinity: NewTestBitMask(0, 1), + Preferred: false, + }, + }, + }, + }, + &mockHintProvider{ + nil, + }, + }, + expected: map[string]TopologyHint{ + "resource1": { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + defaultResourceKey: { + NUMANodeAffinity: NewTestBitMask(0), + Preferred: true, + }, + }, + }, + } +} + +func testPolicyMerge(policy Policy, tcases []policyMergeTestCase, t *testing.T) { + for _, tc := range tcases { + var providersHints []map[string][]TopologyHint + for _, provider := range tc.hp { + hints := provider.GetTopologyHints(&v1.Pod{}, &v1.Container{}) + providersHints = append(providersHints, hints) + } + + actual, _ := policy.Merge(providersHints) + if !reflect.DeepEqual(actual, tc.expected) { + t.Errorf("%v: Expected Topology Hint to be %v, got %v:", tc.name, tc.expected, actual) + } + } +} + +func TestMaxOfMinAffinityCounts(t *testing.T) { + tcases := []struct { + hints [][]TopologyHint + expected int + }{ + { + [][]TopologyHint{}, + 0, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(), true}, + }, + }, + 0, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(0), true}, + }, + }, + 1, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(0, 1), true}, + }, + }, + 2, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(0, 1), true}, + TopologyHint{NewTestBitMask(0, 1, 2), true}, + }, + }, + 2, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(0, 1), true}, + TopologyHint{NewTestBitMask(0, 1, 2), true}, + }, + { + TopologyHint{NewTestBitMask(0, 1, 2), true}, + }, + }, + 3, + }, + { + [][]TopologyHint{ + { + TopologyHint{NewTestBitMask(0, 1), true}, + TopologyHint{NewTestBitMask(0, 1, 2), true}, + }, + { + TopologyHint{NewTestBitMask(0, 1, 2), true}, + TopologyHint{NewTestBitMask(0, 1, 2, 3), true}, + }, + }, + 3, + }, + } + + for _, tc := range tcases { + t.Run("", func(t *testing.T) { + result := maxOfMinAffinityCounts(tc.hints) + if result != tc.expected { + t.Errorf("Expected result to be %v, got %v", tc.expected, result) + } + }) + } +} + +func TestCompareHints(t *testing.T) { + tcases := []struct { + description string + bestNonPreferredAffinityCount int + current *TopologyHint + candidate *TopologyHint + expected string + }{ + { + "candidate.NUMANodeAffinity.Count() == 0 (1)", + -1, + nil, + &TopologyHint{bitmask.NewEmptyBitMask(), false}, + "current", + }, + { + "candidate.NUMANodeAffinity.Count() == 0 (2)", + -1, + &TopologyHint{NewTestBitMask(), true}, + &TopologyHint{NewTestBitMask(), false}, + "current", + }, + { + "current == nil (1)", + -1, + nil, + &TopologyHint{NewTestBitMask(0), true}, + "candidate", + }, + { + "current == nil (2)", + -1, + nil, + &TopologyHint{NewTestBitMask(0), false}, + "candidate", + }, + { + "!current.Preferred && candidate.Preferred", + -1, + &TopologyHint{NewTestBitMask(0), false}, + &TopologyHint{NewTestBitMask(0), true}, + "candidate", + }, + { + "current.Preferred && !candidate.Preferred", + -1, + &TopologyHint{NewTestBitMask(0), true}, + &TopologyHint{NewTestBitMask(0), false}, + "current", + }, + { + "current.Preferred && candidate.Preferred (1)", + -1, + &TopologyHint{NewTestBitMask(0), true}, + &TopologyHint{NewTestBitMask(0), true}, + "current", + }, + { + "current.Preferred && candidate.Preferred (2)", + -1, + &TopologyHint{NewTestBitMask(0, 1), true}, + &TopologyHint{NewTestBitMask(0), true}, + "candidate", + }, + { + "current.Preferred && candidate.Preferred (3)", + -1, + &TopologyHint{NewTestBitMask(0), true}, + &TopologyHint{NewTestBitMask(0, 1), true}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (1.1)", + 1, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (1.2)", + 1, + &TopologyHint{NewTestBitMask(1, 2), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (1.3)", + 1, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(1, 2), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (2.1)", + 2, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(0), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (2.2)", + 2, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (2.3)", + 2, + &TopologyHint{NewTestBitMask(1, 2), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (2.4)", + 2, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(1, 2), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (3a)", + 2, + &TopologyHint{NewTestBitMask(0), false}, + &TopologyHint{NewTestBitMask(0, 1, 2), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (3b)", + 2, + &TopologyHint{NewTestBitMask(0), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (3ca.1)", + 3, + &TopologyHint{NewTestBitMask(0), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (3ca.2)", + 3, + &TopologyHint{NewTestBitMask(0), false}, + &TopologyHint{NewTestBitMask(1, 2), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (3ca.3)", + 4, + &TopologyHint{NewTestBitMask(0, 1), false}, + &TopologyHint{NewTestBitMask(1, 2, 3), false}, + "candidate", + }, + { + "!current.Preferred && !candidate.Preferred (3cb)", + 4, + &TopologyHint{NewTestBitMask(1, 2, 3), false}, + &TopologyHint{NewTestBitMask(0, 1), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (3cc.1)", + 4, + &TopologyHint{NewTestBitMask(0, 1, 2), false}, + &TopologyHint{NewTestBitMask(0, 1, 2), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (3cc.2)", + 4, + &TopologyHint{NewTestBitMask(0, 1, 2), false}, + &TopologyHint{NewTestBitMask(1, 2, 3), false}, + "current", + }, + { + "!current.Preferred && !candidate.Preferred (3cc.3)", + 4, + &TopologyHint{NewTestBitMask(1, 2, 3), false}, + &TopologyHint{NewTestBitMask(0, 1, 2), false}, + "candidate", + }, + } + + for _, tc := range tcases { + t.Run(tc.description, func(t *testing.T) { + result := compareHints(tc.bestNonPreferredAffinityCount, tc.current, tc.candidate) + if result != tc.current && result != tc.candidate { + t.Errorf("Expected result to be either 'current' or 'candidate' hint") + } + if tc.expected == "current" && result != tc.current { + t.Errorf("Expected result to be %v, got %v", tc.current, result) + } + if tc.expected == "candidate" && result != tc.candidate { + t.Errorf("Expected result to be %v, got %v", tc.candidate, result) + } + }) + } +} diff --git a/pkg/agent/resourcemanager/outofband/topology/topologyhint.go b/pkg/agent/resourcemanager/outofband/topology/topologyhint.go new file mode 100644 index 0000000000..2230407273 --- /dev/null +++ b/pkg/agent/resourcemanager/outofband/topology/topologyhint.go @@ -0,0 +1,61 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 topology + +import "github.com/kubewharf/katalyst-core/pkg/util/bitmask" + +type podTopologyHints map[string]map[string]TopologyHint + +// TopologyHint is a struct containing the NUMANodeAffinity for a Container +type TopologyHint struct { + NUMANodeAffinity bitmask.BitMask + // Preferred is set to true when the NUMANodeAffinity encodes a preferred + // allocation for the Container. It is set to false otherwise. + Preferred bool +} + +// IsEqual checks if TopologyHint are equal +func (th *TopologyHint) IsEqual(topologyHint TopologyHint) bool { + if th.Preferred == topologyHint.Preferred { + if th.NUMANodeAffinity == nil || topologyHint.NUMANodeAffinity == nil { + return th.NUMANodeAffinity == topologyHint.NUMANodeAffinity + } + return th.NUMANodeAffinity.IsEqual(topologyHint.NUMANodeAffinity) + } + return false +} + +// LessThan checks if TopologyHint `a` is less than TopologyHint `b` +// this means that either `a` is a preferred hint and `b` is not +// or `a` NUMANodeAffinity attribute is narrower than `b` NUMANodeAffinity attribute. +func (th *TopologyHint) LessThan(other TopologyHint) bool { + if th.Preferred != other.Preferred { + return th.Preferred + } + return th.NUMANodeAffinity.IsNarrowerThan(other.NUMANodeAffinity) +} + +// DeepCopyTopologyHints returns deep copied hints of source hints +func DeepCopyTopologyHints(srcHints []TopologyHint) []TopologyHint { + if srcHints == nil { + return nil + } + + dstHints := make([]TopologyHint, 0, len(srcHints)) + + return append(dstHints, srcHints...) +} diff --git a/pkg/config/agent/orm/orm_base.go b/pkg/config/agent/orm/orm_base.go index 301c1f4d90..4644c77352 100644 --- a/pkg/config/agent/orm/orm_base.go +++ b/pkg/config/agent/orm/orm_base.go @@ -19,15 +19,19 @@ package orm import "time" type GenericORMConfiguration struct { - ORMRconcilePeriod time.Duration - ORMResourceNamesMap map[string]string - ORMPodNotifyChanLen int + ORMRconcilePeriod time.Duration + ORMResourceNamesMap map[string]string + ORMPodNotifyChanLen int + TopologyPolicyName string + NumericAlignResources []string } func NewGenericORMConfiguration() *GenericORMConfiguration { return &GenericORMConfiguration{ - ORMRconcilePeriod: time.Second * 5, - ORMResourceNamesMap: map[string]string{}, - ORMPodNotifyChanLen: 10, + ORMRconcilePeriod: time.Second * 5, + ORMResourceNamesMap: map[string]string{}, + ORMPodNotifyChanLen: 10, + TopologyPolicyName: "none", + NumericAlignResources: []string{"cpu", "memory"}, } } diff --git a/pkg/util/bitmask/bitmask.go b/pkg/util/bitmask/bitmask.go new file mode 100644 index 0000000000..2b0371ec05 --- /dev/null +++ b/pkg/util/bitmask/bitmask.go @@ -0,0 +1,212 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 bitmask + +import ( + "fmt" + "math/bits" + "strconv" +) + +// BitMask interface allows hint providers to create BitMasks for TopologyHints +type BitMask interface { + Add(bits ...int) error + Remove(bits ...int) error + And(masks ...BitMask) + Or(masks ...BitMask) + Clear() + Fill() + IsEqual(mask BitMask) bool + IsEmpty() bool + IsSet(bit int) bool + AnySet(bits []int) bool + IsNarrowerThan(mask BitMask) bool + String() string + Count() int + GetBits() []int +} + +type bitMask uint64 + +// NewEmptyBitMask creates a new, empty BitMask +func NewEmptyBitMask() BitMask { + s := bitMask(0) + return &s +} + +// NewBitMask creates a new BitMask +func NewBitMask(bits ...int) (BitMask, error) { + s := bitMask(0) + err := (&s).Add(bits...) + if err != nil { + return nil, err + } + return &s, nil +} + +// Add adds the bits with topology affinity to the BitMask +func (s *bitMask) Add(bits ...int) error { + mask := *s + for _, i := range bits { + if i < 0 || i >= 64 { + return fmt.Errorf("bit number must be in range 0-63") + } + mask |= 1 << uint64(i) + } + *s = mask + return nil +} + +// Remove removes specified bits from BitMask +func (s *bitMask) Remove(bits ...int) error { + mask := *s + for _, i := range bits { + if i < 0 || i >= 64 { + return fmt.Errorf("bit number must be in range 0-63") + } + mask &^= 1 << uint64(i) + } + *s = mask + return nil +} + +// And performs and operation on all bits in masks +func (s *bitMask) And(masks ...BitMask) { + for _, m := range masks { + *s &= *m.(*bitMask) + } +} + +// Or performs or operation on all bits in masks +func (s *bitMask) Or(masks ...BitMask) { + for _, m := range masks { + *s |= *m.(*bitMask) + } +} + +// Clear resets all bits in mask to zero +func (s *bitMask) Clear() { + *s = 0 +} + +// Fill sets all bits in mask to one +func (s *bitMask) Fill() { + *s = bitMask(^uint64(0)) +} + +// IsEmpty checks mask to see if all bits are zero +func (s *bitMask) IsEmpty() bool { + return *s == 0 +} + +// IsSet checks bit in mask to see if bit is set to one +func (s *bitMask) IsSet(bit int) bool { + if bit < 0 || bit >= 64 { + return false + } + return (*s & (1 << uint64(bit))) > 0 +} + +// AnySet checks bit in mask to see if any provided bit is set to one +func (s *bitMask) AnySet(bits []int) bool { + for _, b := range bits { + if s.IsSet(b) { + return true + } + } + return false +} + +// IsEqual checks if masks are equal +func (s *bitMask) IsEqual(mask BitMask) bool { + return *s == *mask.(*bitMask) +} + +// IsNarrowerThan checks if one mask is narrower than another. +// +// A mask is said to be "narrower" than another if it has lets bits set. If the +// same number of bits are set in both masks, then the mask with more +// lower-numbered bits set wins out. +func (s *bitMask) IsNarrowerThan(mask BitMask) bool { + if s.Count() == mask.Count() { + if *s < *mask.(*bitMask) { + return true + } + } + return s.Count() < mask.Count() +} + +// String converts mask to string +func (s *bitMask) String() string { + grouping := 2 + for shift := 64 - grouping; shift > 0; shift -= grouping { + if *s > (1 << uint(shift)) { + return fmt.Sprintf("%0"+strconv.Itoa(shift+grouping)+"b", *s) + } + } + return fmt.Sprintf("%0"+strconv.Itoa(grouping)+"b", *s) +} + +// Count counts number of bits in mask set to one +func (s *bitMask) Count() int { + return bits.OnesCount64(uint64(*s)) +} + +// Getbits returns each bit number with bits set to one +func (s *bitMask) GetBits() []int { + var bits []int + for i := uint64(0); i < 64; i++ { + if (*s & (1 << i)) > 0 { + bits = append(bits, int(i)) + } + } + return bits +} + +// And is a package level implementation of 'and' between first and masks +func And(first BitMask, masks ...BitMask) BitMask { + s := *first.(*bitMask) + s.And(masks...) + return &s +} + +// Or is a package level implementation of 'or' between first and masks +func Or(first BitMask, masks ...BitMask) BitMask { + s := *first.(*bitMask) + s.Or(masks...) + return &s +} + +// IterateBitMasks iterates all possible masks from a list of bits, +// issuing a callback on each mask. +func IterateBitMasks(bits []int, callback func(BitMask)) { + var iterate func(bits, accum []int, size int) + iterate = func(bits, accum []int, size int) { + if len(accum) == size { + mask, _ := NewBitMask(accum...) + callback(mask) + return + } + for i := range bits { + iterate(bits[i+1:], append(accum, bits[i]), size) + } + } + + for i := 1; i <= len(bits); i++ { + iterate(bits, []int{}, i) + } +} diff --git a/pkg/util/bitmask/bitmask_test.go b/pkg/util/bitmask/bitmask_test.go new file mode 100644 index 0000000000..b4d6531ef2 --- /dev/null +++ b/pkg/util/bitmask/bitmask_test.go @@ -0,0 +1,632 @@ +/* +Copyright 2022 The Katalyst Authors. + +Licensed 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 bitmask + +import ( + "reflect" + "testing" +) + +func TestNewEmptyiBitMask(t *testing.T) { + tcases := []struct { + name string + expectedMask string + }{ + { + name: "New empty BitMask", + expectedMask: "00", + }, + } + for _, tc := range tcases { + bm := NewEmptyBitMask() + if bm.String() != tc.expectedMask { + t.Errorf("Expected mask to be %v, got %v", tc.expectedMask, bm) + } + } +} + +func TestNewBitMask(t *testing.T) { + tcases := []struct { + name string + bits []int + expectedMask string + }{ + { + name: "New BitMask with bit 0 set", + bits: []int{0}, + expectedMask: "01", + }, + { + name: "New BitMask with bit 1 set", + bits: []int{1}, + expectedMask: "10", + }, + { + name: "New BitMask with bit 0 and bit 1 set", + bits: []int{0, 1}, + expectedMask: "11", + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.bits...) + if mask.String() != tc.expectedMask { + t.Errorf("Expected mask to be %v, got %v", tc.expectedMask, mask) + } + } +} + +func TestAdd(t *testing.T) { + tcases := []struct { + name string + bits []int + expectedMask string + }{ + { + name: "Add BitMask with bit 0 set", + bits: []int{0}, + expectedMask: "01", + }, + { + name: "Add BitMask with bit 1 set", + bits: []int{1}, + expectedMask: "10", + }, + { + name: "Add BitMask with bits 0 and 1 set", + bits: []int{0, 1}, + expectedMask: "11", + }, + { + name: "Add BitMask with bits outside range 0-63", + bits: []int{-1, 64}, + expectedMask: "00", + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask() + mask.Add(tc.bits...) + if mask.String() != tc.expectedMask { + t.Errorf("Expected mask to be %v, got %v", tc.expectedMask, mask) + } + } +} + +func TestRemove(t *testing.T) { + tcases := []struct { + name string + bitsSet []int + bitsRemove []int + expectedMask string + }{ + { + name: "Set bit 0. Remove bit 0", + bitsSet: []int{0}, + bitsRemove: []int{0}, + expectedMask: "00", + }, + { + name: "Set bits 0 and 1. Remove bit 1", + bitsSet: []int{0, 1}, + bitsRemove: []int{1}, + expectedMask: "01", + }, + { + name: "Set bits 0 and 1. Remove bits 0 and 1", + bitsSet: []int{0, 1}, + bitsRemove: []int{0, 1}, + expectedMask: "00", + }, + { + name: "Set bit 0. Attempt to remove bits outside range 0-63", + bitsSet: []int{0}, + bitsRemove: []int{-1, 64}, + expectedMask: "01", + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.bitsSet...) + mask.Remove(tc.bitsRemove...) + if mask.String() != tc.expectedMask { + t.Errorf("Expected mask to be %v, got %v", tc.expectedMask, mask) + } + } +} + +func TestAnd(t *testing.T) { + tcases := []struct { + name string + masks [][]int + andMask string + }{ + { + name: "Mask 11 AND mask 11", + masks: [][]int{{0, 1}, {0, 1}}, + andMask: "11", + }, + { + name: "Mask 11 AND mask 10", + masks: [][]int{{0, 1}, {1}}, + andMask: "10", + }, + { + name: "Mask 01 AND mask 11", + masks: [][]int{{0}, {0, 1}}, + andMask: "01", + }, + { + name: "Mask 11 AND mask 11 AND mask 10", + masks: [][]int{{0, 1}, {0, 1}, {1}}, + andMask: "10", + }, + { + name: "Mask 01 AND mask 01 AND mask 10 AND mask 11", + masks: [][]int{{0}, {0}, {1}, {0, 1}}, + andMask: "00", + }, + { + name: "Mask 1111 AND mask 1110 AND mask 1100 AND mask 1000", + masks: [][]int{{0, 1, 2, 3}, {1, 2, 3}, {2, 3}, {3}}, + andMask: "1000", + }, + } + for _, tc := range tcases { + var bitMasks []BitMask + for i := range tc.masks { + bitMask, _ := NewBitMask(tc.masks[i]...) + bitMasks = append(bitMasks, bitMask) + } + resultMask := And(bitMasks[0], bitMasks...) + if resultMask.String() != tc.andMask { + t.Errorf("Expected mask to be %v, got %v", tc.andMask, resultMask) + } + + } +} + +func TestOr(t *testing.T) { + tcases := []struct { + name string + masks [][]int + orMask string + }{ + { + name: "Mask 01 OR mask 00", + masks: [][]int{{0}, {}}, + orMask: "01", + }, + { + name: "Mask 10 OR mask 10", + masks: [][]int{{1}, {1}}, + orMask: "10", + }, + { + name: "Mask 01 OR mask 10", + masks: [][]int{{0}, {1}}, + orMask: "11", + }, + { + name: "Mask 11 OR mask 11", + masks: [][]int{{0, 1}, {0, 1}}, + orMask: "11", + }, + { + name: "Mask 01 OR mask 10 OR mask 11", + masks: [][]int{{0}, {1}, {0, 1}}, + orMask: "11", + }, + { + name: "Mask 1000 OR mask 0100 OR mask 0010 OR mask 0001", + masks: [][]int{{3}, {2}, {1}, {0}}, + orMask: "1111", + }, + } + for _, tc := range tcases { + var bitMasks []BitMask + for i := range tc.masks { + bitMask, _ := NewBitMask(tc.masks[i]...) + bitMasks = append(bitMasks, bitMask) + } + resultMask := Or(bitMasks[0], bitMasks...) + if resultMask.String() != tc.orMask { + t.Errorf("Expected mask to be %v, got %v", tc.orMask, resultMask) + } + } +} + +func TestClear(t *testing.T) { + tcases := []struct { + name string + mask []int + clearedMask string + }{ + { + name: "Clear mask 01", + mask: []int{0}, + clearedMask: "00", + }, + { + name: "Clear mask 10", + mask: []int{1}, + clearedMask: "00", + }, + { + name: "Clear mask 11", + mask: []int{0, 1}, + clearedMask: "00", + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.mask...) + mask.Clear() + if mask.String() != tc.clearedMask { + t.Errorf("Expected mask to be %v, got %v", tc.clearedMask, mask) + } + } +} + +func TestFill(t *testing.T) { + tcases := []struct { + name string + mask []int + filledMask string + }{ + { + name: "Fill empty mask", + mask: nil, + filledMask: "1111111111111111111111111111111111111111111111111111111111111111", + }, + { + name: "Fill mask 10", + mask: []int{0}, + filledMask: "1111111111111111111111111111111111111111111111111111111111111111", + }, + { + name: "Fill mask 11", + mask: []int{0, 1}, + filledMask: "1111111111111111111111111111111111111111111111111111111111111111", + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.mask...) + mask.Fill() + if mask.String() != tc.filledMask { + t.Errorf("Expected mask to be %v, got %v", tc.filledMask, mask) + } + } +} + +func TestIsEmpty(t *testing.T) { + tcases := []struct { + name string + mask []int + expectedEmpty bool + }{ + { + name: "Check if mask 00 is empty", + mask: nil, + expectedEmpty: true, + }, + { + name: "Check if mask 01 is empty", + mask: []int{0}, + expectedEmpty: false, + }, + { + name: "Check if mask 11 is empty", + mask: []int{0, 1}, + expectedEmpty: false, + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.mask...) + empty := mask.IsEmpty() + if empty != tc.expectedEmpty { + t.Errorf("Expected value to be %v, got %v", tc.expectedEmpty, empty) + } + } +} + +func TestIsSet(t *testing.T) { + tcases := []struct { + name string + mask []int + checkBit int + expectedSet bool + }{ + { + name: "Check if bit 0 in mask 00 is set", + mask: nil, + checkBit: 0, + expectedSet: false, + }, + { + name: "Check if bit 0 in mask 01 is set", + mask: []int{0}, + checkBit: 0, + expectedSet: true, + }, + { + name: "Check if bit 1 in mask 11 is set", + mask: []int{0, 1}, + checkBit: 1, + expectedSet: true, + }, + { + name: "Check if bit outside range 0-63 is set", + mask: []int{0, 1}, + checkBit: 64, + expectedSet: false, + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.mask...) + set := mask.IsSet(tc.checkBit) + if set != tc.expectedSet { + t.Errorf("Expected value to be %v, got %v", tc.expectedSet, set) + } + } +} + +func TestAnySet(t *testing.T) { + tcases := []struct { + name string + mask []int + checkBits []int + expectedSet bool + }{ + { + name: "Check if any bits from 11 in mask 00 is set", + mask: nil, + checkBits: []int{0, 1}, + expectedSet: false, + }, + { + name: "Check if any bits from 11 in mask 01 is set", + mask: []int{0}, + checkBits: []int{0, 1}, + expectedSet: true, + }, + { + name: "Check if any bits from 11 in mask 11 is set", + mask: []int{0, 1}, + checkBits: []int{0, 1}, + expectedSet: true, + }, + { + name: "Check if any bit outside range 0-63 is set", + mask: []int{0, 1}, + checkBits: []int{64, 65}, + expectedSet: false, + }, + { + name: "Check if any bits from 1001 in mask 0110 is set", + mask: []int{1, 2}, + checkBits: []int{0, 3}, + expectedSet: false, + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.mask...) + set := mask.AnySet(tc.checkBits) + if set != tc.expectedSet { + t.Errorf("Expected value to be %v, got %v", tc.expectedSet, set) + } + } +} + +func TestIsEqual(t *testing.T) { + tcases := []struct { + name string + firstMask []int + secondMask []int + expectedEqual bool + }{ + { + name: "Check if mask 00 equals mask 00", + firstMask: nil, + secondMask: nil, + expectedEqual: true, + }, + { + name: "Check if mask 00 equals mask 01", + firstMask: nil, + secondMask: []int{0}, + expectedEqual: false, + }, + { + name: "Check if mask 01 equals mask 01", + firstMask: []int{0}, + secondMask: []int{0}, + expectedEqual: true, + }, + { + name: "Check if mask 01 equals mask 10", + firstMask: []int{0}, + secondMask: []int{1}, + expectedEqual: false, + }, + { + name: "Check if mask 11 equals mask 11", + firstMask: []int{0, 1}, + secondMask: []int{0, 1}, + expectedEqual: true, + }, + } + for _, tc := range tcases { + firstMask, _ := NewBitMask(tc.firstMask...) + secondMask, _ := NewBitMask(tc.secondMask...) + isEqual := firstMask.IsEqual(secondMask) + if isEqual != tc.expectedEqual { + t.Errorf("Expected mask to be %v, got %v", tc.expectedEqual, isEqual) + } + } +} + +func TestCount(t *testing.T) { + tcases := []struct { + name string + bits []int + expectedCount int + }{ + { + name: "Count number of bits set in mask 00", + bits: nil, + expectedCount: 0, + }, + { + name: "Count number of bits set in mask 01", + bits: []int{0}, + expectedCount: 1, + }, + { + name: "Count number of bits set in mask 11", + bits: []int{0, 1}, + expectedCount: 2, + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.bits...) + count := mask.Count() + if count != tc.expectedCount { + t.Errorf("Expected value to be %v, got %v", tc.expectedCount, count) + } + } +} + +func TestGetBits(t *testing.T) { + tcases := []struct { + name string + bits []int + expectedBits []int + }{ + { + name: "Get bits of mask 00", + bits: nil, + expectedBits: nil, + }, + { + name: "Get bits of mask 01", + bits: []int{0}, + expectedBits: []int{0}, + }, + { + name: "Get bits of mask 11", + bits: []int{0, 1}, + expectedBits: []int{0, 1}, + }, + } + for _, tc := range tcases { + mask, _ := NewBitMask(tc.bits...) + bits := mask.GetBits() + if !reflect.DeepEqual(bits, tc.expectedBits) { + t.Errorf("Expected value to be %v, got %v", tc.expectedBits, bits) + } + } +} + +func TestIsNarrowerThan(t *testing.T) { + tcases := []struct { + name string + firstMask []int + secondMask []int + expectedFirstNarrower bool + }{ + { + name: "Check narrowness of masks with unequal bits set 1/2", + firstMask: []int{0}, + secondMask: []int{0, 1}, + expectedFirstNarrower: true, + }, + { + name: "Check narrowness of masks with unequal bits set 2/2", + firstMask: []int{0, 1}, + secondMask: []int{0}, + expectedFirstNarrower: false, + }, + { + name: "Check narrowness of masks with equal bits set 1/2", + firstMask: []int{0}, + secondMask: []int{1}, + expectedFirstNarrower: true, + }, + { + name: "Check narrowness of masks with equal bits set 2/2", + firstMask: []int{1}, + secondMask: []int{0}, + expectedFirstNarrower: false, + }, + } + for _, tc := range tcases { + firstMask, _ := NewBitMask(tc.firstMask...) + secondMask, _ := NewBitMask(tc.secondMask...) + expectedFirstNarrower := firstMask.IsNarrowerThan(secondMask) + if expectedFirstNarrower != tc.expectedFirstNarrower { + t.Errorf("Expected value to be %v, got %v", tc.expectedFirstNarrower, expectedFirstNarrower) + } + } +} + +func TestIterateBitMasks(t *testing.T) { + tcases := []struct { + name string + numbits int + }{ + { + name: "1 bit", + numbits: 1, + }, + { + name: "2 bits", + numbits: 2, + }, + { + name: "4 bits", + numbits: 4, + }, + { + name: "8 bits", + numbits: 8, + }, + { + name: "16 bits", + numbits: 16, + }, + } + for _, tc := range tcases { + // Generate a list of bits from tc.numbits. + var bits []int + for i := 0; i < tc.numbits; i++ { + bits = append(bits, i) + } + + // Calculate the expected number of masks. Since we always have masks + // with bits from 0..n, this is just (2^n - 1) since we want 1 mask + // represented by each integer between 1 and 2^n-1. + expectedNumMasks := (1 << uint(tc.numbits)) - 1 + + // Iterate all masks and count them. + numMasks := 0 + IterateBitMasks(bits, func(BitMask) { + numMasks++ + }) + + // Compare the number of masks generated to the expected amount. + if expectedNumMasks != numMasks { + t.Errorf("Expected to iterate %v masks, got %v", expectedNumMasks, numMasks) + } + } +}