mirror of
				https://github.com/optim-enterprises-bv/kubernetes.git
				synced 2025-11-04 04:08:16 +00:00 
			
		
		
		
	Merge pull request #123427 from alexzielenski/apiserver/policy/matching-refactor
ValidatingAdmissionPolicy: Factor out matching and params logic for reuse with MutatingAdmissionPolicy
This commit is contained in:
		@@ -17,14 +17,15 @@ limitations under the License.
 | 
			
		||||
package generic
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/types"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
type PolicyAccessor interface {
 | 
			
		||||
	GetName() string
 | 
			
		||||
	GetNamespace() string
 | 
			
		||||
	GetParamKind() *schema.GroupVersionKind
 | 
			
		||||
	GetParamKind() *v1beta1.ParamKind
 | 
			
		||||
	GetMatchConstraints() *v1beta1.MatchResources
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type BindingAccessor interface {
 | 
			
		||||
@@ -35,4 +36,7 @@ type BindingAccessor interface {
 | 
			
		||||
	// which is cluster-scoped, so namespace is usually left blank.
 | 
			
		||||
	// But we leave the door open to add a namespaced vesion in the future
 | 
			
		||||
	GetPolicyName() types.NamespacedName
 | 
			
		||||
	GetParamRef() *v1beta1.ParamRef
 | 
			
		||||
 | 
			
		||||
	GetMatchResources() *v1beta1.MatchResources
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -22,7 +22,6 @@ import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/meta"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	utilruntime "k8s.io/apimachinery/pkg/util/runtime"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/initializer"
 | 
			
		||||
@@ -37,12 +36,6 @@ import (
 | 
			
		||||
type sourceFactory[H any] func(informers.SharedInformerFactory, kubernetes.Interface, dynamic.Interface, meta.RESTMapper) Source[H]
 | 
			
		||||
type dispatcherFactory[H any] func(authorizer.Authorizer, *matching.Matcher) Dispatcher[H]
 | 
			
		||||
 | 
			
		||||
type Invocation struct {
 | 
			
		||||
	Resource    schema.GroupVersionResource
 | 
			
		||||
	Subresource string
 | 
			
		||||
	Kind        schema.GroupVersionKind
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// AdmissionPolicyManager is an abstract admission plugin with all the
 | 
			
		||||
// infrastructure to define Admit or Validate on-top.
 | 
			
		||||
type Plugin[H any] struct {
 | 
			
		||||
 
 | 
			
		||||
@@ -0,0 +1,354 @@
 | 
			
		||||
/*
 | 
			
		||||
Copyright 2024 The Kubernetes 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 generic
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"context"
 | 
			
		||||
	"errors"
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	apierrors "k8s.io/apimachinery/pkg/api/errors"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/meta"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	utilruntime "k8s.io/apimachinery/pkg/util/runtime"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/matching"
 | 
			
		||||
	webhookgeneric "k8s.io/apiserver/pkg/admission/plugin/webhook/generic"
 | 
			
		||||
	"k8s.io/client-go/informers"
 | 
			
		||||
	"k8s.io/client-go/tools/cache"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// A policy invocation is a single policy-binding-param tuple from a Policy Hook
 | 
			
		||||
// in the context of a specific request. The params have already been resolved
 | 
			
		||||
// and any error in configuration or setting up the invocation is stored in
 | 
			
		||||
// the Error field.
 | 
			
		||||
type PolicyInvocation[P runtime.Object, B runtime.Object, E Evaluator] struct {
 | 
			
		||||
	// Relevant policy for this hook.
 | 
			
		||||
	// This field is always populated
 | 
			
		||||
	Policy P
 | 
			
		||||
 | 
			
		||||
	// Matched Kind for the request given the policy's matchconstraints
 | 
			
		||||
	// May be empty if there was an error matching the resource
 | 
			
		||||
	Kind schema.GroupVersionKind
 | 
			
		||||
 | 
			
		||||
	// Matched Resource for the request given the policy's matchconstraints
 | 
			
		||||
	// May be empty if there was an error matching the resource
 | 
			
		||||
	Resource schema.GroupVersionResource
 | 
			
		||||
 | 
			
		||||
	// Relevant binding for this hook.
 | 
			
		||||
	// May be empty if there was an error with the policy's configuration itself
 | 
			
		||||
	Binding B
 | 
			
		||||
 | 
			
		||||
	// Compiled policy evaluator
 | 
			
		||||
	Evaluator E
 | 
			
		||||
 | 
			
		||||
	// Params fetched by the binding to use to evaluate the policy
 | 
			
		||||
	Param runtime.Object
 | 
			
		||||
 | 
			
		||||
	// Error is set if there was an error with the policy or binding or its
 | 
			
		||||
	// params, etc
 | 
			
		||||
	Error error
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// dispatcherDelegate is called during a request with a pre-filtered list
 | 
			
		||||
// of (Policy, Binding, Param) tuples that are active and match the request.
 | 
			
		||||
// The dispatcher delegate is responsible for updating the object on the
 | 
			
		||||
// admission attributes in the case of mutation, or returning a status error in
 | 
			
		||||
// the case of validation.
 | 
			
		||||
//
 | 
			
		||||
// The delegate provides the "validation" or "mutation" aspect of dispatcher functionality
 | 
			
		||||
// (in contrast to generic.PolicyDispatcher which only selects active policies and params)
 | 
			
		||||
type dispatcherDelegate[P, B runtime.Object, E Evaluator] func(ctx context.Context, a admission.Attributes, o admission.ObjectInterfaces, versionedAttributes webhookgeneric.VersionedAttributeAccessor, invocations []PolicyInvocation[P, B, E]) error
 | 
			
		||||
 | 
			
		||||
type policyDispatcher[P runtime.Object, B runtime.Object, E Evaluator] struct {
 | 
			
		||||
	newPolicyAccessor  func(P) PolicyAccessor
 | 
			
		||||
	newBindingAccessor func(B) BindingAccessor
 | 
			
		||||
	matcher            PolicyMatcher
 | 
			
		||||
	delegate           dispatcherDelegate[P, B, E]
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func NewPolicyDispatcher[P runtime.Object, B runtime.Object, E Evaluator](
 | 
			
		||||
	newPolicyAccessor func(P) PolicyAccessor,
 | 
			
		||||
	newBindingAccessor func(B) BindingAccessor,
 | 
			
		||||
	matcher *matching.Matcher,
 | 
			
		||||
	delegate dispatcherDelegate[P, B, E],
 | 
			
		||||
) Dispatcher[PolicyHook[P, B, E]] {
 | 
			
		||||
	return &policyDispatcher[P, B, E]{
 | 
			
		||||
		newPolicyAccessor:  newPolicyAccessor,
 | 
			
		||||
		newBindingAccessor: newBindingAccessor,
 | 
			
		||||
		matcher:            NewPolicyMatcher(matcher),
 | 
			
		||||
		delegate:           delegate,
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Dispatch implements generic.Dispatcher. It loops through all active hooks
 | 
			
		||||
// (policy x binding pairs) and selects those which are active for the current
 | 
			
		||||
// request. It then resolves all params and creates an Invocation for each
 | 
			
		||||
// matching policy-binding-param tuple. The delegate is then called with the
 | 
			
		||||
// list of tuples.
 | 
			
		||||
//
 | 
			
		||||
// Note: MatchConditions expressions are not evaluated here. The dispatcher delegate
 | 
			
		||||
// is expected to ignore the result of any policies whose match conditions dont pass.
 | 
			
		||||
// This may be possible to refactor so matchconditions are checked here instead.
 | 
			
		||||
func (d *policyDispatcher[P, B, E]) Dispatch(ctx context.Context, a admission.Attributes, o admission.ObjectInterfaces, hooks []PolicyHook[P, B, E]) error {
 | 
			
		||||
	var relevantHooks []PolicyInvocation[P, B, E]
 | 
			
		||||
	// Construct all the versions we need to call our webhooks
 | 
			
		||||
	versionedAttrAccessor := &versionedAttributeAccessor{
 | 
			
		||||
		versionedAttrs:   map[schema.GroupVersionKind]*admission.VersionedAttributes{},
 | 
			
		||||
		attr:             a,
 | 
			
		||||
		objectInterfaces: o,
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for _, hook := range hooks {
 | 
			
		||||
		policyAccessor := d.newPolicyAccessor(hook.Policy)
 | 
			
		||||
		matches, matchGVR, matchGVK, err := d.matcher.DefinitionMatches(a, o, policyAccessor)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// There was an error evaluating if this policy matches anything.
 | 
			
		||||
			utilruntime.HandleError(err)
 | 
			
		||||
			relevantHooks = append(relevantHooks, PolicyInvocation[P, B, E]{
 | 
			
		||||
				Policy: hook.Policy,
 | 
			
		||||
				Error:  err,
 | 
			
		||||
			})
 | 
			
		||||
			continue
 | 
			
		||||
		} else if !matches {
 | 
			
		||||
			continue
 | 
			
		||||
		} else if hook.ConfigurationError != nil {
 | 
			
		||||
			// The policy matches but there is a configuration error with the
 | 
			
		||||
			// policy itself
 | 
			
		||||
			relevantHooks = append(relevantHooks, PolicyInvocation[P, B, E]{
 | 
			
		||||
				Policy:   hook.Policy,
 | 
			
		||||
				Error:    hook.ConfigurationError,
 | 
			
		||||
				Resource: matchGVR,
 | 
			
		||||
				Kind:     matchGVK,
 | 
			
		||||
			})
 | 
			
		||||
			utilruntime.HandleError(hook.ConfigurationError)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		for _, binding := range hook.Bindings {
 | 
			
		||||
			bindingAccessor := d.newBindingAccessor(binding)
 | 
			
		||||
			matches, err = d.matcher.BindingMatches(a, o, bindingAccessor)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				// There was an error evaluating if this binding matches anything.
 | 
			
		||||
				utilruntime.HandleError(err)
 | 
			
		||||
				relevantHooks = append(relevantHooks, PolicyInvocation[P, B, E]{
 | 
			
		||||
					Policy:   hook.Policy,
 | 
			
		||||
					Binding:  binding,
 | 
			
		||||
					Error:    err,
 | 
			
		||||
					Resource: matchGVR,
 | 
			
		||||
					Kind:     matchGVK,
 | 
			
		||||
				})
 | 
			
		||||
				continue
 | 
			
		||||
			} else if !matches {
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// Collect params for this binding
 | 
			
		||||
			params, err := CollectParams(
 | 
			
		||||
				policyAccessor.GetParamKind(),
 | 
			
		||||
				hook.ParamInformer,
 | 
			
		||||
				hook.ParamScope,
 | 
			
		||||
				bindingAccessor.GetParamRef(),
 | 
			
		||||
				a.GetNamespace(),
 | 
			
		||||
			)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				// There was an error collecting params for this binding.
 | 
			
		||||
				utilruntime.HandleError(err)
 | 
			
		||||
				relevantHooks = append(relevantHooks, PolicyInvocation[P, B, E]{
 | 
			
		||||
					Policy:   hook.Policy,
 | 
			
		||||
					Binding:  binding,
 | 
			
		||||
					Error:    err,
 | 
			
		||||
					Resource: matchGVR,
 | 
			
		||||
					Kind:     matchGVK,
 | 
			
		||||
				})
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// If params is empty and there was no error, that means that
 | 
			
		||||
			// ParamNotFoundAction is ignore, so it shouldnt be added to list
 | 
			
		||||
			for _, param := range params {
 | 
			
		||||
				relevantHooks = append(relevantHooks, PolicyInvocation[P, B, E]{
 | 
			
		||||
					Policy:    hook.Policy,
 | 
			
		||||
					Binding:   binding,
 | 
			
		||||
					Kind:      matchGVK,
 | 
			
		||||
					Resource:  matchGVR,
 | 
			
		||||
					Param:     param,
 | 
			
		||||
					Evaluator: hook.Evaluator,
 | 
			
		||||
				})
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// VersionedAttr result will be cached and reused later during parallel
 | 
			
		||||
			// hook calls
 | 
			
		||||
			_, err = versionedAttrAccessor.VersionedAttribute(matchGVK)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return apierrors.NewInternalError(err)
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if len(relevantHooks) == 0 {
 | 
			
		||||
		// no matching hooks
 | 
			
		||||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return d.delegate(ctx, a, o, versionedAttrAccessor, relevantHooks)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Returns params to use to evaluate a policy-binding with given param
 | 
			
		||||
// configuration. If the policy-binding has no param configuration, it
 | 
			
		||||
// returns a single-element list with a nil param.
 | 
			
		||||
func CollectParams(
 | 
			
		||||
	paramKind *v1beta1.ParamKind,
 | 
			
		||||
	paramInformer informers.GenericInformer,
 | 
			
		||||
	paramScope meta.RESTScope,
 | 
			
		||||
	paramRef *v1beta1.ParamRef,
 | 
			
		||||
	namespace string,
 | 
			
		||||
) ([]runtime.Object, error) {
 | 
			
		||||
	// If definition has paramKind, paramRef is required in binding.
 | 
			
		||||
	// If definition has no paramKind, paramRef set in binding will be ignored.
 | 
			
		||||
	var params []runtime.Object
 | 
			
		||||
	var paramStore cache.GenericNamespaceLister
 | 
			
		||||
 | 
			
		||||
	// Make sure the param kind is ready to use
 | 
			
		||||
	if paramKind != nil && paramRef != nil {
 | 
			
		||||
		if paramInformer == nil {
 | 
			
		||||
			return nil, fmt.Errorf("paramKind kind `%v` not known",
 | 
			
		||||
				paramKind.String())
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Set up cluster-scoped, or namespaced access to the params
 | 
			
		||||
		// "default" if not provided, and paramKind is namespaced
 | 
			
		||||
		paramStore = paramInformer.Lister()
 | 
			
		||||
		if paramScope.Name() == meta.RESTScopeNameNamespace {
 | 
			
		||||
			paramsNamespace := namespace
 | 
			
		||||
			if len(paramRef.Namespace) > 0 {
 | 
			
		||||
				paramsNamespace = paramRef.Namespace
 | 
			
		||||
			} else if len(paramsNamespace) == 0 {
 | 
			
		||||
				// You must supply namespace if your matcher can possibly
 | 
			
		||||
				// match a cluster-scoped resource
 | 
			
		||||
				return nil, fmt.Errorf("cannot use namespaced paramRef in policy binding that matches cluster-scoped resources")
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			paramStore = paramInformer.Lister().ByNamespace(paramsNamespace)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// If the param informer for this admission policy has not yet
 | 
			
		||||
		// had time to perform an initial listing, don't attempt to use
 | 
			
		||||
		// it.
 | 
			
		||||
		timeoutCtx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		if !cache.WaitForCacheSync(timeoutCtx.Done(), paramInformer.Informer().HasSynced) {
 | 
			
		||||
			return nil, fmt.Errorf("paramKind kind `%v` not yet synced to use for admission",
 | 
			
		||||
				paramKind.String())
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Find params to use with policy
 | 
			
		||||
	switch {
 | 
			
		||||
	case paramKind == nil:
 | 
			
		||||
		// ParamKind is unset. Ignore any globalParamRef or namespaceParamRef
 | 
			
		||||
		// setting.
 | 
			
		||||
		return []runtime.Object{nil}, nil
 | 
			
		||||
	case paramRef == nil:
 | 
			
		||||
		// Policy ParamKind is set, but binding does not use it.
 | 
			
		||||
		// Validate with nil params
 | 
			
		||||
		return []runtime.Object{nil}, nil
 | 
			
		||||
	case len(paramRef.Namespace) > 0 && paramScope.Name() == meta.RESTScopeRoot.Name():
 | 
			
		||||
		// Not allowed to set namespace for cluster-scoped param
 | 
			
		||||
		return nil, fmt.Errorf("paramRef.namespace must not be provided for a cluster-scoped `paramKind`")
 | 
			
		||||
 | 
			
		||||
	case len(paramRef.Name) > 0:
 | 
			
		||||
		if paramRef.Selector != nil {
 | 
			
		||||
			// This should be validated, but just in case.
 | 
			
		||||
			return nil, fmt.Errorf("paramRef.name and paramRef.selector are mutually exclusive")
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		switch param, err := paramStore.Get(paramRef.Name); {
 | 
			
		||||
		case err == nil:
 | 
			
		||||
			params = []runtime.Object{param}
 | 
			
		||||
		case apierrors.IsNotFound(err):
 | 
			
		||||
			// Param not yet available. User may need to wait a bit
 | 
			
		||||
			// before being able to use it for validation.
 | 
			
		||||
			//
 | 
			
		||||
			// Set params to nil to prepare for not found action
 | 
			
		||||
			params = nil
 | 
			
		||||
		case apierrors.IsInvalid(err):
 | 
			
		||||
			// Param mis-configured
 | 
			
		||||
			// require to set namespace for namespaced resource
 | 
			
		||||
			// and unset namespace for cluster scoped resource
 | 
			
		||||
			return nil, err
 | 
			
		||||
		default:
 | 
			
		||||
			// Internal error
 | 
			
		||||
			utilruntime.HandleError(err)
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
	case paramRef.Selector != nil:
 | 
			
		||||
		// Select everything by default if empty name and selector
 | 
			
		||||
		selector, err := metav1.LabelSelectorAsSelector(paramRef.Selector)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// Cannot parse label selector: configuration error
 | 
			
		||||
			return nil, err
 | 
			
		||||
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		paramList, err := paramStore.List(selector)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// There was a bad internal error
 | 
			
		||||
			utilruntime.HandleError(err)
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Successfully grabbed params
 | 
			
		||||
		params = paramList
 | 
			
		||||
	default:
 | 
			
		||||
		// Should be unreachable due to validation
 | 
			
		||||
		return nil, fmt.Errorf("one of name or selector must be provided")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Apply fail action for params not found case
 | 
			
		||||
	if len(params) == 0 && paramRef.ParameterNotFoundAction != nil && *paramRef.ParameterNotFoundAction == v1beta1.DenyAction {
 | 
			
		||||
		return nil, errors.New("no params found for policy binding with `Deny` parameterNotFoundAction")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return params, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var _ webhookgeneric.VersionedAttributeAccessor = &versionedAttributeAccessor{}
 | 
			
		||||
 | 
			
		||||
type versionedAttributeAccessor struct {
 | 
			
		||||
	versionedAttrs   map[schema.GroupVersionKind]*admission.VersionedAttributes
 | 
			
		||||
	attr             admission.Attributes
 | 
			
		||||
	objectInterfaces admission.ObjectInterfaces
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *versionedAttributeAccessor) VersionedAttribute(gvk schema.GroupVersionKind) (*admission.VersionedAttributes, error) {
 | 
			
		||||
	if val, ok := v.versionedAttrs[gvk]; ok {
 | 
			
		||||
		return val, nil
 | 
			
		||||
	}
 | 
			
		||||
	versionedAttr, err := admission.NewVersionedAttributes(v.attr, gvk, v.objectInterfaces)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return nil, err
 | 
			
		||||
	}
 | 
			
		||||
	v.versionedAttrs[gvk] = versionedAttr
 | 
			
		||||
	return versionedAttr, nil
 | 
			
		||||
}
 | 
			
		||||
@@ -14,9 +14,11 @@ See the License for the specific language governing permissions and
 | 
			
		||||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package validating
 | 
			
		||||
package generic
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	corev1 "k8s.io/api/core/v1"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
@@ -26,6 +28,64 @@ import (
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/matching"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// Matcher is used for matching ValidatingAdmissionPolicy and ValidatingAdmissionPolicyBinding to attributes
 | 
			
		||||
type PolicyMatcher interface {
 | 
			
		||||
	admission.InitializationValidator
 | 
			
		||||
 | 
			
		||||
	// DefinitionMatches says whether this policy definition matches the provided admission
 | 
			
		||||
	// resource request
 | 
			
		||||
	DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition PolicyAccessor) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error)
 | 
			
		||||
 | 
			
		||||
	// BindingMatches says whether this policy definition matches the provided admission
 | 
			
		||||
	// resource request
 | 
			
		||||
	BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, binding BindingAccessor) (bool, error)
 | 
			
		||||
 | 
			
		||||
	// GetNamespace retrieves the Namespace resource by the given name. The name may be empty, in which case
 | 
			
		||||
	// GetNamespace must return nil, nil
 | 
			
		||||
	GetNamespace(name string) (*corev1.Namespace, error)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type matcher struct {
 | 
			
		||||
	Matcher *matching.Matcher
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func NewPolicyMatcher(m *matching.Matcher) PolicyMatcher {
 | 
			
		||||
	return &matcher{
 | 
			
		||||
		Matcher: m,
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// ValidateInitialization checks if Matcher is initialized.
 | 
			
		||||
func (c *matcher) ValidateInitialization() error {
 | 
			
		||||
	return c.Matcher.ValidateInitialization()
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// DefinitionMatches returns whether this ValidatingAdmissionPolicy matches the provided admission resource request
 | 
			
		||||
func (c *matcher) DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition PolicyAccessor) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error) {
 | 
			
		||||
	constraints := definition.GetMatchConstraints()
 | 
			
		||||
	if constraints == nil {
 | 
			
		||||
		return false, schema.GroupVersionResource{}, schema.GroupVersionKind{}, fmt.Errorf("policy contained no match constraints, a required field")
 | 
			
		||||
	}
 | 
			
		||||
	criteria := matchCriteria{constraints: constraints}
 | 
			
		||||
	return c.Matcher.Matches(a, o, &criteria)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// BindingMatches returns whether this ValidatingAdmissionPolicyBinding matches the provided admission resource request
 | 
			
		||||
func (c *matcher) BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, binding BindingAccessor) (bool, error) {
 | 
			
		||||
	matchResources := binding.GetMatchResources()
 | 
			
		||||
	if matchResources == nil {
 | 
			
		||||
		return true, nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	criteria := matchCriteria{constraints: matchResources}
 | 
			
		||||
	isMatch, _, _, err := c.Matcher.Matches(a, o, &criteria)
 | 
			
		||||
	return isMatch, err
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (c *matcher) GetNamespace(name string) (*corev1.Namespace, error) {
 | 
			
		||||
	return c.Matcher.GetNamespace(name)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var _ matching.MatchCriteria = &matchCriteria{}
 | 
			
		||||
 | 
			
		||||
type matchCriteria struct {
 | 
			
		||||
@@ -46,38 +106,3 @@ func (m *matchCriteria) GetParsedObjectSelector() (labels.Selector, error) {
 | 
			
		||||
func (m *matchCriteria) GetMatchResources() v1beta1.MatchResources {
 | 
			
		||||
	return *m.constraints
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type matcher struct {
 | 
			
		||||
	Matcher *matching.Matcher
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func NewMatcher(m *matching.Matcher) Matcher {
 | 
			
		||||
	return &matcher{
 | 
			
		||||
		Matcher: m,
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// ValidateInitialization checks if Matcher is initialized.
 | 
			
		||||
func (c *matcher) ValidateInitialization() error {
 | 
			
		||||
	return c.Matcher.ValidateInitialization()
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// DefinitionMatches returns whether this ValidatingAdmissionPolicy matches the provided admission resource request
 | 
			
		||||
func (c *matcher) DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition *v1beta1.ValidatingAdmissionPolicy) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error) {
 | 
			
		||||
	criteria := matchCriteria{constraints: definition.Spec.MatchConstraints}
 | 
			
		||||
	return c.Matcher.Matches(a, o, &criteria)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// BindingMatches returns whether this ValidatingAdmissionPolicyBinding matches the provided admission resource request
 | 
			
		||||
func (c *matcher) BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, binding *v1beta1.ValidatingAdmissionPolicyBinding) (bool, error) {
 | 
			
		||||
	if binding.Spec.MatchResources == nil {
 | 
			
		||||
		return true, nil
 | 
			
		||||
	}
 | 
			
		||||
	criteria := matchCriteria{constraints: binding.Spec.MatchResources}
 | 
			
		||||
	isMatch, _, _, err := c.Matcher.Matches(a, o, &criteria)
 | 
			
		||||
	return isMatch, err
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (c *matcher) GetNamespace(name string) (*corev1.Namespace, error) {
 | 
			
		||||
	return c.Matcher.GetNamespace(name)
 | 
			
		||||
}
 | 
			
		||||
@@ -85,6 +85,9 @@ type compiledPolicyEntry[E Evaluator] struct {
 | 
			
		||||
type PolicyHook[P runtime.Object, B runtime.Object, E Evaluator] struct {
 | 
			
		||||
	Policy   P
 | 
			
		||||
	Bindings []B
 | 
			
		||||
 | 
			
		||||
	// ParamInformer is the informer for the param CRD for this policy, or nil if
 | 
			
		||||
	// there is no param or if there was a configuration error
 | 
			
		||||
	ParamInformer informers.GenericInformer
 | 
			
		||||
	ParamScope    meta.RESTScope
 | 
			
		||||
 | 
			
		||||
@@ -157,7 +160,7 @@ func (s *policySource[P, B, E]) Run(ctx context.Context) error {
 | 
			
		||||
	}
 | 
			
		||||
	defer func() {
 | 
			
		||||
		if err := s.policyInformer.RemoveEventHandler(handle); err != nil {
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("failed to remove policy event handler: %v", err))
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("failed to remove policy event handler: %w", err))
 | 
			
		||||
		}
 | 
			
		||||
	}()
 | 
			
		||||
 | 
			
		||||
@@ -167,7 +170,7 @@ func (s *policySource[P, B, E]) Run(ctx context.Context) error {
 | 
			
		||||
	}
 | 
			
		||||
	defer func() {
 | 
			
		||||
		if err := s.bindingInformer.RemoveEventHandler(bindingHandle); err != nil {
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("failed to remove binding event handler: %v", err))
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("failed to remove binding event handler: %w", err))
 | 
			
		||||
		}
 | 
			
		||||
	}()
 | 
			
		||||
 | 
			
		||||
@@ -231,7 +234,7 @@ func (s *policySource[P, B, E]) refreshPolicies() {
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		// An error was generated while syncing policies. Mark it as dirty again
 | 
			
		||||
		// so we can retry later
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("encountered error syncing policies: %v. Rescheduling policy sync", err))
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("encountered error syncing policies: %w. Rescheduling policy sync", err))
 | 
			
		||||
		s.notify()
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
@@ -299,8 +302,26 @@ func (s *policySource[P, B, E]) calculatePolicyData() ([]PolicyHook[P, B, E], er
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		var parsedParamKind *schema.GroupVersionKind
 | 
			
		||||
		policyAccessor := s.newPolicyAccessor(policySpec)
 | 
			
		||||
		paramInformer, paramScope, configurationError := s.ensureParamsForPolicyLocked(policyAccessor.GetParamKind())
 | 
			
		||||
 | 
			
		||||
		if paramKind := policyAccessor.GetParamKind(); paramKind != nil {
 | 
			
		||||
			groupVersion, err := schema.ParseGroupVersion(paramKind.APIVersion)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				errs = append(errs, fmt.Errorf("failed to parse paramKind APIVersion: %w", err))
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
			parsedParamKind = &schema.GroupVersionKind{
 | 
			
		||||
				Group:   groupVersion.Group,
 | 
			
		||||
				Version: groupVersion.Version,
 | 
			
		||||
				Kind:    paramKind.Kind,
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// TEMPORARY UNTIL WE HAVE SHARED PARAM INFORMERS
 | 
			
		||||
			usedParams[*parsedParamKind] = struct{}{}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		paramInformer, paramScope, configurationError := s.ensureParamsForPolicyLocked(parsedParamKind)
 | 
			
		||||
		result = append(result, PolicyHook[P, B, E]{
 | 
			
		||||
			Policy:             policySpec,
 | 
			
		||||
			Bindings:           bindingSpecs,
 | 
			
		||||
@@ -310,11 +331,6 @@ func (s *policySource[P, B, E]) calculatePolicyData() ([]PolicyHook[P, B, E], er
 | 
			
		||||
			ConfigurationError: configurationError,
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
		// TEMPORARY UNTIL WE HAVE SHARED PARAM INFORMERS
 | 
			
		||||
		if paramKind := policyAccessor.GetParamKind(); paramKind != nil {
 | 
			
		||||
			usedParams[*paramKind] = struct{}{}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Should queue a re-sync for policy sync error. If our shared param
 | 
			
		||||
		// informer can notify us when CRD discovery changes we can remove this
 | 
			
		||||
		// and just rely on the informer to notify us when the CRDs change
 | 
			
		||||
 
 | 
			
		||||
@@ -20,9 +20,9 @@ import (
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/require"
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/types"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/generic"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/matching"
 | 
			
		||||
@@ -110,9 +110,8 @@ func TestPolicySourceHasSyncedInitialList(t *testing.T) {
 | 
			
		||||
			ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
				Name: "policy2",
 | 
			
		||||
			},
 | 
			
		||||
			ParamKind: &schema.GroupVersionKind{
 | 
			
		||||
				Group:   "policy.example.com",
 | 
			
		||||
				Version: "v1",
 | 
			
		||||
			ParamKind: &v1beta1.ParamKind{
 | 
			
		||||
				APIVersion: "policy.example.com/v1",
 | 
			
		||||
				Kind:       "FakeParam",
 | 
			
		||||
			},
 | 
			
		||||
		},
 | 
			
		||||
@@ -178,7 +177,7 @@ type FakePolicy struct {
 | 
			
		||||
	metav1.TypeMeta
 | 
			
		||||
	metav1.ObjectMeta
 | 
			
		||||
 | 
			
		||||
	ParamKind *schema.GroupVersionKind
 | 
			
		||||
	ParamKind *v1beta1.ParamKind
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var _ generic.PolicyAccessor = &FakePolicy{}
 | 
			
		||||
@@ -200,10 +199,14 @@ func (fp *FakePolicy) GetNamespace() string {
 | 
			
		||||
	return fp.Namespace
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fp *FakePolicy) GetParamKind() *schema.GroupVersionKind {
 | 
			
		||||
func (fp *FakePolicy) GetParamKind() *v1beta1.ParamKind {
 | 
			
		||||
	return fp.ParamKind
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fb *FakePolicy) GetMatchConstraints() *v1beta1.MatchResources {
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fb *FakeBinding) GetName() string {
 | 
			
		||||
	return fb.Name
 | 
			
		||||
}
 | 
			
		||||
@@ -218,6 +221,14 @@ func (fb *FakeBinding) GetPolicyName() types.NamespacedName {
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fb *FakeBinding) GetMatchResources() *v1beta1.MatchResources {
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fb *FakeBinding) GetParamRef() *v1beta1.ParamRef {
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fp *FakePolicy) DeepCopyObject() runtime.Object {
 | 
			
		||||
	// totally fudged deepcopy
 | 
			
		||||
	newFP := &FakePolicy{}
 | 
			
		||||
 
 | 
			
		||||
@@ -317,7 +317,11 @@ func (p *PolicyTestContext[P, B, E]) WaitForReconcile(timeoutCtx context.Context
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	objectGVK := object.GetObjectKind().GroupVersionKind()
 | 
			
		||||
	objectGVK, _, err := p.inferGVK(object)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	switch objectGVK {
 | 
			
		||||
	case p.policyGVK:
 | 
			
		||||
		return wait.PollUntilContextCancel(timeoutCtx, 100*time.Millisecond, true, func(ctx context.Context) (done bool, err error) {
 | 
			
		||||
@@ -443,32 +447,7 @@ func (p *PolicyTestContext[P, B, E]) updateOne(object runtime.Object) error {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	objectMeta.SetResourceVersion(string(uuid.NewUUID()))
 | 
			
		||||
	objectGVK := object.GetObjectKind().GroupVersionKind()
 | 
			
		||||
 | 
			
		||||
	if objectGVK.Empty() {
 | 
			
		||||
		// If the object doesn't have a GVK, ask the schema for preferred GVK
 | 
			
		||||
		knownKinds, _, err := p.scheme.ObjectKinds(object)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		} else if len(knownKinds) == 0 {
 | 
			
		||||
			return fmt.Errorf("no known GVKs for object in schema: %T", object)
 | 
			
		||||
		}
 | 
			
		||||
		toTake := 0
 | 
			
		||||
 | 
			
		||||
		// Prefer GVK if it is our fake policy or binding
 | 
			
		||||
		for i, knownKind := range knownKinds {
 | 
			
		||||
			if knownKind == p.policyGVK || knownKind == p.bindingGVK {
 | 
			
		||||
				toTake = i
 | 
			
		||||
				break
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		objectGVK = knownKinds[toTake]
 | 
			
		||||
		object.GetObjectKind().SetGroupVersionKind(objectGVK)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Make sure GVK is known to the fake rest mapper. To prevent cryptic error
 | 
			
		||||
	mapping, err := p.restMapper.RESTMapping(objectGVK.GroupKind(), objectGVK.Version)
 | 
			
		||||
	objectGVK, gvr, err := p.inferGVK(object)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
@@ -490,16 +469,16 @@ func (p *PolicyTestContext[P, B, E]) updateOne(object runtime.Object) error {
 | 
			
		||||
		return err
 | 
			
		||||
	default:
 | 
			
		||||
		if _, ok := object.(*unstructured.Unstructured); ok {
 | 
			
		||||
			if err := p.unstructuredTracker.Create(mapping.Resource, object, objectMeta.GetNamespace()); err != nil {
 | 
			
		||||
			if err := p.unstructuredTracker.Create(gvr, object, objectMeta.GetNamespace()); err != nil {
 | 
			
		||||
				if errors.IsAlreadyExists(err) {
 | 
			
		||||
					return p.unstructuredTracker.Update(mapping.Resource, object, objectMeta.GetNamespace())
 | 
			
		||||
					return p.unstructuredTracker.Update(gvr, object, objectMeta.GetNamespace())
 | 
			
		||||
				}
 | 
			
		||||
				return err
 | 
			
		||||
			}
 | 
			
		||||
			return nil
 | 
			
		||||
		} else if err := p.nativeTracker.Create(mapping.Resource, object, objectMeta.GetNamespace()); err != nil {
 | 
			
		||||
		} else if err := p.nativeTracker.Create(gvr, object, objectMeta.GetNamespace()); err != nil {
 | 
			
		||||
			if errors.IsAlreadyExists(err) {
 | 
			
		||||
				return p.nativeTracker.Update(mapping.Resource, object, objectMeta.GetNamespace())
 | 
			
		||||
				return p.nativeTracker.Update(gvr, object, objectMeta.GetNamespace())
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
		return nil
 | 
			
		||||
@@ -524,9 +503,14 @@ func (p *PolicyTestContext[P, B, E]) DeleteAndWait(object ...runtime.Object) err
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		objectGVK, _, err := p.inferGVK(object)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if err := p.waitForDelete(
 | 
			
		||||
			timeoutCtx,
 | 
			
		||||
			object.GetObjectKind().GroupVersionKind(),
 | 
			
		||||
			objectGVK,
 | 
			
		||||
			types.NamespacedName{Name: accessor.GetName(), Namespace: accessor.GetNamespace()}); err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
@@ -540,15 +524,73 @@ func (p *PolicyTestContext[P, B, E]) deleteOne(object runtime.Object) error {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	objectMeta.SetResourceVersion(string(uuid.NewUUID()))
 | 
			
		||||
	objectGVK := object.GetObjectKind().GroupVersionKind()
 | 
			
		||||
	objectGVK, gvr, err := p.inferGVK(object)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	switch objectGVK {
 | 
			
		||||
	case p.policyGVK:
 | 
			
		||||
		return p.policyAndBindingTracker.Delete(p.policyGVR, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
	case p.bindingGVK:
 | 
			
		||||
		return p.policyAndBindingTracker.Delete(p.bindingGVR, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
	default:
 | 
			
		||||
		if _, ok := object.(*unstructured.Unstructured); ok {
 | 
			
		||||
			return p.unstructuredTracker.Delete(gvr, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
		}
 | 
			
		||||
		return p.nativeTracker.Delete(gvr, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *PolicyTestContext[P, B, E]) Dispatch(
 | 
			
		||||
	new, old runtime.Object,
 | 
			
		||||
	operation admission.Operation,
 | 
			
		||||
) error {
 | 
			
		||||
	if old == nil && new == nil {
 | 
			
		||||
		return fmt.Errorf("both old and new objects cannot be nil")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	nonNilObject := new
 | 
			
		||||
	if nonNilObject == nil {
 | 
			
		||||
		nonNilObject = old
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	gvk, gvr, err := p.inferGVK(nonNilObject)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	nonNilMeta, err := meta.Accessor(nonNilObject)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return p.Plugin.Dispatch(
 | 
			
		||||
		p,
 | 
			
		||||
		admission.NewAttributesRecord(
 | 
			
		||||
			new,
 | 
			
		||||
			old,
 | 
			
		||||
			gvk,
 | 
			
		||||
			nonNilMeta.GetName(),
 | 
			
		||||
			nonNilMeta.GetNamespace(),
 | 
			
		||||
			gvr,
 | 
			
		||||
			"",
 | 
			
		||||
			operation,
 | 
			
		||||
			nil,
 | 
			
		||||
			false,
 | 
			
		||||
			nil,
 | 
			
		||||
		), admission.NewObjectInterfacesFromScheme(p.scheme))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *PolicyTestContext[P, B, E]) inferGVK(object runtime.Object) (schema.GroupVersionKind, schema.GroupVersionResource, error) {
 | 
			
		||||
	objectGVK := object.GetObjectKind().GroupVersionKind()
 | 
			
		||||
	if objectGVK.Empty() {
 | 
			
		||||
		// If the object doesn't have a GVK, ask the schema for preferred GVK
 | 
			
		||||
		knownKinds, _, err := p.scheme.ObjectKinds(object)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
			return schema.GroupVersionKind{}, schema.GroupVersionResource{}, err
 | 
			
		||||
		} else if len(knownKinds) == 0 {
 | 
			
		||||
			return fmt.Errorf("no known GVKs for object in schema: %T", object)
 | 
			
		||||
			return schema.GroupVersionKind{}, schema.GroupVersionResource{}, fmt.Errorf("no known GVKs for object in schema: %T", object)
 | 
			
		||||
		}
 | 
			
		||||
		toTake := 0
 | 
			
		||||
 | 
			
		||||
@@ -561,26 +603,14 @@ func (p *PolicyTestContext[P, B, E]) deleteOne(object runtime.Object) error {
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		objectGVK = knownKinds[toTake]
 | 
			
		||||
		object.GetObjectKind().SetGroupVersionKind(objectGVK)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Make sure GVK is known to the fake rest mapper. To prevent cryptic error
 | 
			
		||||
	mapping, err := p.restMapper.RESTMapping(objectGVK.GroupKind(), objectGVK.Version)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	switch objectGVK {
 | 
			
		||||
	case p.policyGVK:
 | 
			
		||||
		return p.policyAndBindingTracker.Delete(p.policyGVR, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
	case p.bindingGVK:
 | 
			
		||||
		return p.policyAndBindingTracker.Delete(p.bindingGVR, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
	default:
 | 
			
		||||
		if _, ok := object.(*unstructured.Unstructured); ok {
 | 
			
		||||
			return p.unstructuredTracker.Delete(mapping.Resource, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
		}
 | 
			
		||||
		return p.nativeTracker.Delete(mapping.Resource, objectMeta.GetNamespace(), objectMeta.GetName())
 | 
			
		||||
		return schema.GroupVersionKind{}, schema.GroupVersionResource{}, err
 | 
			
		||||
	}
 | 
			
		||||
	return objectGVK, mapping.Resource, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type FakeList[T runtime.Object] struct {
 | 
			
		||||
 
 | 
			
		||||
@@ -18,7 +18,6 @@ package validating
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/types"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/generic"
 | 
			
		||||
)
 | 
			
		||||
@@ -47,32 +46,12 @@ func (v *validatingAdmissionPolicyAccessor) GetName() string {
 | 
			
		||||
	return v.Name
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *validatingAdmissionPolicyAccessor) GetParamKind() *schema.GroupVersionKind {
 | 
			
		||||
	paramKind := v.Spec.ParamKind
 | 
			
		||||
	if paramKind == nil {
 | 
			
		||||
		return nil
 | 
			
		||||
func (v *validatingAdmissionPolicyAccessor) GetParamKind() *v1beta1.ParamKind {
 | 
			
		||||
	return v.Spec.ParamKind
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
	groupVersion, err := schema.ParseGroupVersion(paramKind.APIVersion)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		// A validatingadmissionpolicy which passes validation should have
 | 
			
		||||
		// a parseable APIVersion for its ParamKind, so this should never happen
 | 
			
		||||
		// if the policy is valid.
 | 
			
		||||
		//
 | 
			
		||||
		// Return a bogus but non-nil GVK that will throw an error about the
 | 
			
		||||
		// invalid APIVersion when the param is looked up.
 | 
			
		||||
		return &schema.GroupVersionKind{
 | 
			
		||||
			Group:   paramKind.APIVersion,
 | 
			
		||||
			Version: "",
 | 
			
		||||
			Kind:    paramKind.Kind,
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return &schema.GroupVersionKind{
 | 
			
		||||
		Group:   groupVersion.Group,
 | 
			
		||||
		Version: groupVersion.Version,
 | 
			
		||||
		Kind:    paramKind.Kind,
 | 
			
		||||
	}
 | 
			
		||||
func (v *validatingAdmissionPolicyAccessor) GetMatchConstraints() *v1beta1.MatchResources {
 | 
			
		||||
	return v.Spec.MatchConstraints
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type validatingAdmissionPolicyBindingAccessor struct {
 | 
			
		||||
@@ -93,3 +72,11 @@ func (v *validatingAdmissionPolicyBindingAccessor) GetPolicyName() types.Namespa
 | 
			
		||||
		Name:      v.Spec.PolicyName,
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *validatingAdmissionPolicyBindingAccessor) GetMatchResources() *v1beta1.MatchResources {
 | 
			
		||||
	return v.Spec.MatchResources
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *validatingAdmissionPolicyBindingAccessor) GetParamRef() *v1beta1.ParamRef {
 | 
			
		||||
	return v.Spec.ParamRef
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -261,7 +261,7 @@ func (f validateFunc) Validate(
 | 
			
		||||
	)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var _ validating.Matcher = &fakeMatcher{}
 | 
			
		||||
var _ generic.PolicyMatcher = &fakeMatcher{}
 | 
			
		||||
 | 
			
		||||
func (f *fakeMatcher) ValidateInitialization() error {
 | 
			
		||||
	return nil
 | 
			
		||||
@@ -273,11 +273,11 @@ func (f *fakeMatcher) GetNamespace(name string) (*v1.Namespace, error) {
 | 
			
		||||
 | 
			
		||||
type fakeMatcher struct {
 | 
			
		||||
	DefaultMatch         bool
 | 
			
		||||
	DefinitionMatchFuncs map[types.NamespacedName]func(*v1beta1.ValidatingAdmissionPolicy, admission.Attributes) bool
 | 
			
		||||
	BindingMatchFuncs    map[types.NamespacedName]func(*v1beta1.ValidatingAdmissionPolicyBinding, admission.Attributes) bool
 | 
			
		||||
	DefinitionMatchFuncs map[types.NamespacedName]func(generic.PolicyAccessor, admission.Attributes) bool
 | 
			
		||||
	BindingMatchFuncs    map[types.NamespacedName]func(generic.BindingAccessor, admission.Attributes) bool
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (f *fakeMatcher) RegisterDefinition(definition *v1beta1.ValidatingAdmissionPolicy, matchFunc func(*v1beta1.ValidatingAdmissionPolicy, admission.Attributes) bool) {
 | 
			
		||||
func (f *fakeMatcher) RegisterDefinition(definition *v1beta1.ValidatingAdmissionPolicy, matchFunc func(generic.PolicyAccessor, admission.Attributes) bool) {
 | 
			
		||||
	namespace, name := definition.Namespace, definition.Name
 | 
			
		||||
	key := types.NamespacedName{
 | 
			
		||||
		Name:      name,
 | 
			
		||||
@@ -286,13 +286,13 @@ func (f *fakeMatcher) RegisterDefinition(definition *v1beta1.ValidatingAdmission
 | 
			
		||||
 | 
			
		||||
	if matchFunc != nil {
 | 
			
		||||
		if f.DefinitionMatchFuncs == nil {
 | 
			
		||||
			f.DefinitionMatchFuncs = make(map[types.NamespacedName]func(*v1beta1.ValidatingAdmissionPolicy, admission.Attributes) bool)
 | 
			
		||||
			f.DefinitionMatchFuncs = make(map[types.NamespacedName]func(generic.PolicyAccessor, admission.Attributes) bool)
 | 
			
		||||
		}
 | 
			
		||||
		f.DefinitionMatchFuncs[key] = matchFunc
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (f *fakeMatcher) RegisterBinding(binding *v1beta1.ValidatingAdmissionPolicyBinding, matchFunc func(*v1beta1.ValidatingAdmissionPolicyBinding, admission.Attributes) bool) {
 | 
			
		||||
func (f *fakeMatcher) RegisterBinding(binding *v1beta1.ValidatingAdmissionPolicyBinding, matchFunc func(generic.BindingAccessor, admission.Attributes) bool) {
 | 
			
		||||
	namespace, name := binding.Namespace, binding.Name
 | 
			
		||||
	key := types.NamespacedName{
 | 
			
		||||
		Name:      name,
 | 
			
		||||
@@ -301,7 +301,7 @@ func (f *fakeMatcher) RegisterBinding(binding *v1beta1.ValidatingAdmissionPolicy
 | 
			
		||||
 | 
			
		||||
	if matchFunc != nil {
 | 
			
		||||
		if f.BindingMatchFuncs == nil {
 | 
			
		||||
			f.BindingMatchFuncs = make(map[types.NamespacedName]func(*v1beta1.ValidatingAdmissionPolicyBinding, admission.Attributes) bool)
 | 
			
		||||
			f.BindingMatchFuncs = make(map[types.NamespacedName]func(generic.BindingAccessor, admission.Attributes) bool)
 | 
			
		||||
		}
 | 
			
		||||
		f.BindingMatchFuncs[key] = matchFunc
 | 
			
		||||
	}
 | 
			
		||||
@@ -309,8 +309,8 @@ func (f *fakeMatcher) RegisterBinding(binding *v1beta1.ValidatingAdmissionPolicy
 | 
			
		||||
 | 
			
		||||
// Matches says whether this policy definition matches the provided admission
 | 
			
		||||
// resource request
 | 
			
		||||
func (f *fakeMatcher) DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition *v1beta1.ValidatingAdmissionPolicy) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error) {
 | 
			
		||||
	namespace, name := definition.Namespace, definition.Name
 | 
			
		||||
func (f *fakeMatcher) DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition generic.PolicyAccessor) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error) {
 | 
			
		||||
	namespace, name := definition.GetNamespace(), definition.GetName()
 | 
			
		||||
	key := types.NamespacedName{
 | 
			
		||||
		Name:      name,
 | 
			
		||||
		Namespace: namespace,
 | 
			
		||||
@@ -325,8 +325,8 @@ func (f *fakeMatcher) DefinitionMatches(a admission.Attributes, o admission.Obje
 | 
			
		||||
 | 
			
		||||
// Matches says whether this policy definition matches the provided admission
 | 
			
		||||
// resource request
 | 
			
		||||
func (f *fakeMatcher) BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, binding *v1beta1.ValidatingAdmissionPolicyBinding) (bool, error) {
 | 
			
		||||
	namespace, name := binding.Namespace, binding.Name
 | 
			
		||||
func (f *fakeMatcher) BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, binding generic.BindingAccessor) (bool, error) {
 | 
			
		||||
	namespace, name := binding.GetNamespace(), binding.GetName()
 | 
			
		||||
	key := types.NamespacedName{
 | 
			
		||||
		Name:      name,
 | 
			
		||||
		Namespace: namespace,
 | 
			
		||||
@@ -355,7 +355,7 @@ func setupFakeTest(t *testing.T, comp *fakeCompiler, match *fakeMatcher) *generi
 | 
			
		||||
func setupTestCommon(
 | 
			
		||||
	t *testing.T,
 | 
			
		||||
	compiler *fakeCompiler,
 | 
			
		||||
	matcher validating.Matcher,
 | 
			
		||||
	matcher generic.PolicyMatcher,
 | 
			
		||||
	shouldStartInformers bool,
 | 
			
		||||
) *generic.PolicyTestContext[*validating.Policy, *validating.PolicyBinding, validating.Validator] {
 | 
			
		||||
	testContext, testContextCancel, err := generic.NewPolicyTestContext(
 | 
			
		||||
@@ -367,7 +367,7 @@ func setupTestCommon(
 | 
			
		||||
		func(a authorizer.Authorizer, m *matching.Matcher) generic.Dispatcher[validating.PolicyHook] {
 | 
			
		||||
			coolMatcher := matcher
 | 
			
		||||
			if coolMatcher == nil {
 | 
			
		||||
				coolMatcher = validating.NewMatcher(m)
 | 
			
		||||
				coolMatcher = generic.NewPolicyMatcher(m)
 | 
			
		||||
			}
 | 
			
		||||
			return validating.NewDispatcher(a, coolMatcher)
 | 
			
		||||
		},
 | 
			
		||||
@@ -549,7 +549,7 @@ func TestDefinitionDoesntMatch(t *testing.T) {
 | 
			
		||||
		}
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	matcher.RegisterDefinition(denyPolicy, func(vap *v1beta1.ValidatingAdmissionPolicy, a admission.Attributes) bool {
 | 
			
		||||
	matcher.RegisterDefinition(denyPolicy, func(vap generic.PolicyAccessor, a admission.Attributes) bool {
 | 
			
		||||
		// Match names with even-numbered length
 | 
			
		||||
		obj := a.GetObject()
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -21,30 +21,25 @@ import (
 | 
			
		||||
	"errors"
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"strings"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	v1 "k8s.io/api/core/v1"
 | 
			
		||||
	k8serrors "k8s.io/apimachinery/pkg/api/errors"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/meta"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	utiljson "k8s.io/apimachinery/pkg/util/json"
 | 
			
		||||
	utilruntime "k8s.io/apimachinery/pkg/util/runtime"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission"
 | 
			
		||||
	"k8s.io/apiserver/pkg/admission/plugin/policy/generic"
 | 
			
		||||
	celmetrics "k8s.io/apiserver/pkg/admission/plugin/policy/validating/metrics"
 | 
			
		||||
	celconfig "k8s.io/apiserver/pkg/apis/cel"
 | 
			
		||||
	"k8s.io/apiserver/pkg/authorization/authorizer"
 | 
			
		||||
	"k8s.io/apiserver/pkg/warning"
 | 
			
		||||
	"k8s.io/client-go/informers"
 | 
			
		||||
	"k8s.io/client-go/tools/cache"
 | 
			
		||||
	"k8s.io/klog/v2"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
type dispatcher struct {
 | 
			
		||||
	matcher Matcher
 | 
			
		||||
	matcher generic.PolicyMatcher
 | 
			
		||||
	authz   authorizer.Authorizer
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@@ -52,7 +47,7 @@ var _ generic.Dispatcher[PolicyHook] = &dispatcher{}
 | 
			
		||||
 | 
			
		||||
func NewDispatcher(
 | 
			
		||||
	authorizer authorizer.Authorizer,
 | 
			
		||||
	matcher Matcher,
 | 
			
		||||
	matcher generic.PolicyMatcher,
 | 
			
		||||
) generic.Dispatcher[PolicyHook] {
 | 
			
		||||
	return &dispatcher{
 | 
			
		||||
		matcher: matcher,
 | 
			
		||||
@@ -124,7 +119,7 @@ func (c *dispatcher) Dispatch(ctx context.Context, a admission.Attributes, o adm
 | 
			
		||||
		var versionedAttr *admission.VersionedAttributes
 | 
			
		||||
 | 
			
		||||
		definition := hook.Policy
 | 
			
		||||
		matches, matchResource, matchKind, err := c.matcher.DefinitionMatches(a, o, definition)
 | 
			
		||||
		matches, matchResource, matchKind, err := c.matcher.DefinitionMatches(a, o, NewValidatingAdmissionPolicyAccessor(definition))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// Configuration error.
 | 
			
		||||
			addConfigError(err, definition, nil)
 | 
			
		||||
@@ -143,7 +138,7 @@ func (c *dispatcher) Dispatch(ctx context.Context, a admission.Attributes, o adm
 | 
			
		||||
		for _, binding := range hook.Bindings {
 | 
			
		||||
			// If the key is inside dependentBindings, there is guaranteed to
 | 
			
		||||
			// be a bindingInfo for it
 | 
			
		||||
			matches, err := c.matcher.BindingMatches(a, o, binding)
 | 
			
		||||
			matches, err := c.matcher.BindingMatches(a, o, NewValidatingAdmissionPolicyBindingAccessor(binding))
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				// Configuration error.
 | 
			
		||||
				addConfigError(err, definition, binding)
 | 
			
		||||
@@ -153,8 +148,8 @@ func (c *dispatcher) Dispatch(ctx context.Context, a admission.Attributes, o adm
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			params, err := c.collectParams(
 | 
			
		||||
				definition.Spec.ParamKind,
 | 
			
		||||
			params, err := generic.CollectParams(
 | 
			
		||||
				hook.Policy.Spec.ParamKind,
 | 
			
		||||
				hook.ParamInformer,
 | 
			
		||||
				hook.ParamScope,
 | 
			
		||||
				binding.Spec.ParamRef,
 | 
			
		||||
@@ -307,125 +302,6 @@ func (c *dispatcher) Dispatch(ctx context.Context, a admission.Attributes, o adm
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Returns objects to use to evaluate the policy
 | 
			
		||||
// Copied with minor modification to account for slightly different arguments
 | 
			
		||||
func (c *dispatcher) collectParams(
 | 
			
		||||
	paramKind *v1beta1.ParamKind,
 | 
			
		||||
	paramInformer informers.GenericInformer,
 | 
			
		||||
	paramScope meta.RESTScope,
 | 
			
		||||
	paramRef *v1beta1.ParamRef,
 | 
			
		||||
	namespace string,
 | 
			
		||||
) ([]runtime.Object, error) {
 | 
			
		||||
	// If definition has paramKind, paramRef is required in binding.
 | 
			
		||||
	// If definition has no paramKind, paramRef set in binding will be ignored.
 | 
			
		||||
	var params []runtime.Object
 | 
			
		||||
	var paramStore cache.GenericNamespaceLister
 | 
			
		||||
 | 
			
		||||
	// Make sure the param kind is ready to use
 | 
			
		||||
	if paramKind != nil && paramRef != nil {
 | 
			
		||||
		if paramInformer == nil {
 | 
			
		||||
			return nil, fmt.Errorf("paramKind kind `%v` not known",
 | 
			
		||||
				paramKind.String())
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Set up cluster-scoped, or namespaced access to the params
 | 
			
		||||
		// "default" if not provided, and paramKind is namespaced
 | 
			
		||||
		paramStore = paramInformer.Lister()
 | 
			
		||||
		if paramScope.Name() == meta.RESTScopeNameNamespace {
 | 
			
		||||
			paramsNamespace := namespace
 | 
			
		||||
			if len(paramRef.Namespace) > 0 {
 | 
			
		||||
				paramsNamespace = paramRef.Namespace
 | 
			
		||||
			} else if len(paramsNamespace) == 0 {
 | 
			
		||||
				// You must supply namespace if your matcher can possibly
 | 
			
		||||
				// match a cluster-scoped resource
 | 
			
		||||
				return nil, fmt.Errorf("cannot use namespaced paramRef in policy binding that matches cluster-scoped resources")
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			paramStore = paramInformer.Lister().ByNamespace(paramsNamespace)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// If the param informer for this admission policy has not yet
 | 
			
		||||
		// had time to perform an initial listing, don't attempt to use
 | 
			
		||||
		// it.
 | 
			
		||||
		timeoutCtx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		if !cache.WaitForCacheSync(timeoutCtx.Done(), paramInformer.Informer().HasSynced) {
 | 
			
		||||
			return nil, fmt.Errorf("paramKind kind `%v` not yet synced to use for admission",
 | 
			
		||||
				paramKind.String())
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Find params to use with policy
 | 
			
		||||
	switch {
 | 
			
		||||
	case paramKind == nil:
 | 
			
		||||
		// ParamKind is unset. Ignore any globalParamRef or namespaceParamRef
 | 
			
		||||
		// setting.
 | 
			
		||||
		return []runtime.Object{nil}, nil
 | 
			
		||||
	case paramRef == nil:
 | 
			
		||||
		// Policy ParamKind is set, but binding does not use it.
 | 
			
		||||
		// Validate with nil params
 | 
			
		||||
		return []runtime.Object{nil}, nil
 | 
			
		||||
	case len(paramRef.Namespace) > 0 && paramScope.Name() == meta.RESTScopeRoot.Name():
 | 
			
		||||
		// Not allowed to set namespace for cluster-scoped param
 | 
			
		||||
		return nil, fmt.Errorf("paramRef.namespace must not be provided for a cluster-scoped `paramKind`")
 | 
			
		||||
 | 
			
		||||
	case len(paramRef.Name) > 0:
 | 
			
		||||
		if paramRef.Selector != nil {
 | 
			
		||||
			// This should be validated, but just in case.
 | 
			
		||||
			return nil, fmt.Errorf("paramRef.name and paramRef.selector are mutually exclusive")
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		switch param, err := paramStore.Get(paramRef.Name); {
 | 
			
		||||
		case err == nil:
 | 
			
		||||
			params = []runtime.Object{param}
 | 
			
		||||
		case k8serrors.IsNotFound(err):
 | 
			
		||||
			// Param not yet available. User may need to wait a bit
 | 
			
		||||
			// before being able to use it for validation.
 | 
			
		||||
			//
 | 
			
		||||
			// Set params to nil to prepare for not found action
 | 
			
		||||
			params = nil
 | 
			
		||||
		case k8serrors.IsInvalid(err):
 | 
			
		||||
			// Param mis-configured
 | 
			
		||||
			// require to set namespace for namespaced resource
 | 
			
		||||
			// and unset namespace for cluster scoped resource
 | 
			
		||||
			return nil, err
 | 
			
		||||
		default:
 | 
			
		||||
			// Internal error
 | 
			
		||||
			utilruntime.HandleError(err)
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
	case paramRef.Selector != nil:
 | 
			
		||||
		// Select everything by default if empty name and selector
 | 
			
		||||
		selector, err := metav1.LabelSelectorAsSelector(paramRef.Selector)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// Cannot parse label selector: configuration error
 | 
			
		||||
			return nil, err
 | 
			
		||||
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		paramList, err := paramStore.List(selector)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// There was a bad internal error
 | 
			
		||||
			utilruntime.HandleError(err)
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Successfully grabbed params
 | 
			
		||||
		params = paramList
 | 
			
		||||
	default:
 | 
			
		||||
		// Should be unreachable due to validation
 | 
			
		||||
		return nil, fmt.Errorf("one of name or selector must be provided")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Apply fail action for params not found case
 | 
			
		||||
	if len(params) == 0 && paramRef.ParameterNotFoundAction != nil && *paramRef.ParameterNotFoundAction == v1beta1.DenyAction {
 | 
			
		||||
		return nil, errors.New("no params found for policy binding with `Deny` parameterNotFoundAction")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return params, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func publishValidationFailureAnnotation(binding *v1beta1.ValidatingAdmissionPolicyBinding, expressionIndex int, decision PolicyDecision, attributes admission.Attributes) {
 | 
			
		||||
	key := "validation.policy.admission.k8s.io/validation_failure"
 | 
			
		||||
	// Marshal to a list of failures since, in the future, we may need to support multiple failures
 | 
			
		||||
 
 | 
			
		||||
@@ -21,7 +21,6 @@ import (
 | 
			
		||||
 | 
			
		||||
	celgo "github.com/google/cel-go/cel"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/api/admissionregistration/v1beta1"
 | 
			
		||||
	corev1 "k8s.io/api/core/v1"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime"
 | 
			
		||||
@@ -80,23 +79,6 @@ func (v *Variable) GetName() string {
 | 
			
		||||
	return v.Name
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Matcher is used for matching ValidatingAdmissionPolicy and ValidatingAdmissionPolicyBinding to attributes
 | 
			
		||||
type Matcher interface {
 | 
			
		||||
	admission.InitializationValidator
 | 
			
		||||
 | 
			
		||||
	// DefinitionMatches says whether this policy definition matches the provided admission
 | 
			
		||||
	// resource request
 | 
			
		||||
	DefinitionMatches(a admission.Attributes, o admission.ObjectInterfaces, definition *v1beta1.ValidatingAdmissionPolicy) (bool, schema.GroupVersionResource, schema.GroupVersionKind, error)
 | 
			
		||||
 | 
			
		||||
	// BindingMatches says whether this policy definition matches the provided admission
 | 
			
		||||
	// resource request
 | 
			
		||||
	BindingMatches(a admission.Attributes, o admission.ObjectInterfaces, definition *v1beta1.ValidatingAdmissionPolicyBinding) (bool, error)
 | 
			
		||||
 | 
			
		||||
	// GetNamespace retrieves the Namespace resource by the given name. The name may be empty, in which case
 | 
			
		||||
	// GetNamespace must return nil, nil
 | 
			
		||||
	GetNamespace(name string) (*corev1.Namespace, error)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// ValidateResult defines the result of a Validator.Validate operation.
 | 
			
		||||
type ValidateResult struct {
 | 
			
		||||
	// Decisions specifies the outcome of the validation as well as the details about the decision.
 | 
			
		||||
 
 | 
			
		||||
@@ -94,7 +94,7 @@ func NewPlugin(_ io.Reader) *Plugin {
 | 
			
		||||
				)
 | 
			
		||||
			},
 | 
			
		||||
			func(a authorizer.Authorizer, m *matching.Matcher) generic.Dispatcher[PolicyHook] {
 | 
			
		||||
				return NewDispatcher(a, NewMatcher(m))
 | 
			
		||||
				return NewDispatcher(a, generic.NewPolicyMatcher(m))
 | 
			
		||||
			},
 | 
			
		||||
		),
 | 
			
		||||
	}
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user