add loadbalancer address allocator

This adds a simple controller that will watch for services of type LoadBalancer
and try to allocated addresses from the specified IPv4 and/or IPv6 ranges.
It's assumed that kube-router (or another network controller) will announce the addresses.

As the controller uses leases for leader election and updates the service status new
RBAC permissions are required.
This commit is contained in:
Erik Larsson 2023-07-11 00:12:51 +02:00 committed by Aaron U'Ren
parent 7699d165da
commit afdf553fa8
17 changed files with 1592 additions and 35 deletions

View File

@ -84,6 +84,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -206,6 +210,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding

View File

@ -80,6 +80,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -202,6 +206,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding
apiVersion: rbac.authorization.k8s.io/v1

View File

@ -43,6 +43,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
livenessProbe:
httpGet:
path: /healthz
@ -114,6 +118,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding

View File

@ -60,6 +60,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -169,6 +173,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding

View File

@ -61,6 +61,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -185,6 +189,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding
apiVersion: rbac.authorization.k8s.io/v1

View File

@ -68,6 +68,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -184,6 +188,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding
apiVersion: rbac.authorization.k8s.io/v1

View File

@ -61,6 +61,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -177,6 +181,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding
apiVersion: rbac.authorization.k8s.io/v1

View File

@ -60,6 +60,10 @@ spec:
valueFrom:
fieldRef:
fieldPath: spec.nodeName
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
- name: KUBE_ROUTER_CNI_CONF_FILE
value: /etc/cni/net.d/10-kuberouter.conflist
livenessProbe:
@ -173,6 +177,20 @@ rules:
- get
- list
- watch
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
---
kind: ClusterRoleBinding
apiVersion: rbac.authorization.k8s.io/v1

View File

@ -0,0 +1,81 @@
# Load Balancer allocator
## What does it do
The load balancer allocator controller looks for services with the type LoadBalancer and tries to allocate addresses for it if needed.
The controller doesn't enable any announcement of the addresses by default, so `--advertise-loadbalancer-ip` should be set to true and BGP peers configured.
## Load balancer classes
By default the controller allocates addresses for all LoadBalancer services with the where `loadBalancerClass` is empty or set to one of "default" or "kube-router".
If `--loadbalancer-default-class` is set to false, the controller will only handle services with the class set to "kube-router".
## RBAC permissions
The controller needs some extra permissions to get, create and update leases for leader election and to update services with allocated addresses.
Example permissions:
```yaml
kind: ClusterRole
apiVersion: rbac.authorization.k8s.io/v1
metadata:
name: kube-router
namespace: kube-system
rules:
- apiGroups:
- "coordination.k8s.io"
resources:
- leases
verbs:
- get
- create
- update
- apiGroups:
- ""
resources:
- services/status
verbs:
- update
```
## Environment variables
The controller uses the environment variable `POD_NAME` as the identify for the lease used for leader election.
Using the kubernetes downward api to set `POD_NAME` to the pod name the lease identify will match the current leader.
```yaml
---
apiVersion: apps/v1
kind: DaemonSet
metadata:
labels:
k8s-app: kube-router
tier: node
name: kube-router
namespace: kube-system
spec:
...
template:
metadata:
....
spec:
...
env:
- name: POD_NAME
valueFrom:
fieldRef:
fieldPath: metadata.name
...
```
The environment variable `POD_NAMESPACE` can also be specified to set the namespace used for the lease.
By default the namespace is looked up from within the pod using `/var/run/secrets/kubernetes.io/serviceaccount/namespace`.
## Running outside kubernetes
When running the controller outside a pod, both `POD_NAME` and `POD_NAMESPACE` must set for the controller to work.
`POD_NAME` should be unique per instance, so using for example the hostname of the machine might be a good idea.
`POD_NAMESPACE` must be the same across all instances running in the same cluster.
## Notes
It's not possible to specify the addresses for the load balancer services. A externalIP service can be used instead.

View File

@ -94,6 +94,9 @@ Usage of kube-router:
--ipvs-permit-all Enables rule to accept all incoming traffic to service VIP's on the node. (default true)
--ipvs-sync-period duration The delay between ipvs config synchronizations (e.g. '5s', '1m', '2h22m'). Must be greater than 0. (default 5m0s)
--kubeconfig string Path to kubeconfig file with authorization information (the master location is set by the master flag).
--loadbalancer-default-class Handle loadbalancer services without a class (default true)
--loadbalancer-ip-range strings CIDR values from which loadbalancer services addresses are assigned (can be specified multiple times)
--loadbalancer-sync-period duration The delay between checking for missed services (e.g. '5s', '1m'). Must be greater than 0. (default 1m0s)
--masquerade-all SNAT all traffic to cluster IP/node port.
--master string The address of the Kubernetes API server (overrides any value in kubeconfig).
--metrics-path string Prometheus metrics path (default "/metrics")
@ -113,6 +116,7 @@ Usage of kube-router:
--router-id string BGP router-id. Must be specified in a ipv6 only cluster, "generate" can be specified to generate the router id.
--routes-sync-period duration The delay between route updates and advertisements (e.g. '5s', '1m', '2h22m'). Must be greater than 0. (default 5m0s)
--run-firewall Enables Network Policy -- sets up iptables to provide ingress firewall for pods. (default true)
--run-loadbalancer Enable loadbalancer address allocator
--run-router Enables Pod Networking -- Advertises and learns the routes to Pods via iBGP. (default true)
--run-service-proxy Enables Service Proxy -- sets up IPVS for Kubernetes Services. (default true)
--runtime-endpoint string Path to CRI compatible container runtime socket (used for DSR mode). Currently known working with containerd.

View File

@ -8,6 +8,7 @@ import (
"syscall"
"time"
"github.com/cloudnativelabs/kube-router/v2/pkg/controllers/lballoc"
"github.com/cloudnativelabs/kube-router/v2/pkg/controllers/netpol"
"github.com/cloudnativelabs/kube-router/v2/pkg/controllers/proxy"
"github.com/cloudnativelabs/kube-router/v2/pkg/controllers/routing"
@ -228,6 +229,19 @@ func (kr *KubeRouter) Run() error {
go npc.Run(healthChan, stopCh, &wg)
}
if kr.Config.RunLoadBalancer {
klog.V(0).Info("running load balancer allocator controller")
lbc, err := lballoc.NewLoadBalancerController(kr.Client, kr.Config, svcInformer)
if err != nil {
return errors.New("Failed to create load balancer allocator: " + err.Error())
}
svcInformer.AddEventHandler(lbc)
wg.Add(1)
go lbc.Run(healthChan, stopCh, &wg)
}
// Handle SIGINT and SIGTERM
ch := make(chan os.Signal, 1)
signal.Notify(ch, syscall.SIGINT, syscall.SIGTERM)

View File

@ -0,0 +1,517 @@
package lballoc
import (
"context"
"errors"
"net"
"os"
"sync"
"time"
"github.com/cloudnativelabs/kube-router/v2/pkg/healthcheck"
"github.com/cloudnativelabs/kube-router/v2/pkg/options"
v1core "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/client-go/kubernetes"
"k8s.io/client-go/tools/cache"
"k8s.io/client-go/tools/leaderelection"
"k8s.io/client-go/tools/leaderelection/resourcelock"
"k8s.io/client-go/util/retry"
"k8s.io/klog/v2"
)
const loadBalancerClassName = "kube-router"
type ipRanges struct {
ipRanges []net.IPNet
rangeIndex int
currentIP net.IP
}
type LoadBalancerController struct {
ipv4Ranges *ipRanges
ipv6Ranges *ipRanges
svcLister cache.Indexer
lock *resourcelock.LeaseLock
addChan chan v1core.Service
allocateChan chan v1core.Service
clientset kubernetes.Interface
isDefault bool
syncPeriod time.Duration
}
func getNamespace() (namespace string, err error) {
ns := os.Getenv("POD_NAMESPACE")
if ns != "" {
return ns, nil
}
nb, err := os.ReadFile("/var/run/secrets/kubernetes.io/serviceaccount/namespace")
if err == nil {
return string(nb), nil
}
return "", errors.New("unable to get namespace from kubernetes environment or $POD_NAMESPACE")
}
func getPodname() (podname string, err error) {
podname = os.Getenv("POD_NAME")
if podname != "" {
return podname, nil
}
return "", errors.New("unable to get pod name from $POD_NAME")
}
func copyIP(ip net.IP) net.IP {
return append(net.IP{}, ip...)
}
func newipRanges(ranges []net.IPNet) *ipRanges {
var cip net.IP
if len(ranges) == 0 {
cip = nil
} else {
cip = copyIP(ranges[0].IP)
}
ir := &ipRanges{
ipRanges: ranges,
currentIP: cip,
}
return ir
}
func (ir *ipRanges) inc() {
cn := ir.ipRanges[ir.rangeIndex]
ci := copyIP(ir.currentIP)
// Increment the current IP address
// 10.0.0.3 will increment to 10.0.0.4
// 10.0.0.255 will increment to 10.0.1.0
for i := len(ci) - 1; i >= 0; i-- {
ci[i]++
if ci[i] > 0 { // if the byte didn't overflow to zero, don't increment the byte to the left
break
}
}
// If the new address is not in the current IP range, move to the first IP in the next range
// If the current range is the last, move to the first IP in the first range
if !cn.Contains(ci) {
if ir.rangeIndex == len(ir.ipRanges)-1 {
ir.rangeIndex = 0
} else {
ir.rangeIndex++
}
ci = copyIP(ir.ipRanges[ir.rangeIndex].IP)
}
ir.currentIP = ci
}
func ipInAllocated(ip net.IP, allocated []net.IP) bool {
for _, cip := range allocated {
if cip.Equal(ip) {
return true
}
}
return false
}
func (ir *ipRanges) getNextFreeIP(allocated []net.IP) (net.IP, error) {
startIP := copyIP(ir.currentIP)
if len(startIP) == 0 {
return nil, errors.New("no IPs left to allocate")
}
ip := startIP
for {
if !ipInAllocated(ip, allocated) {
return ip, nil
}
ir.inc()
ip = ir.currentIP
if ip.Equal(startIP) {
break
}
}
return nil, errors.New("no IPs left to allocate")
}
func (ir *ipRanges) Len() int {
return len(ir.ipRanges)
}
func (ir *ipRanges) Contains(ip net.IP) bool {
for _, in := range ir.ipRanges {
if in.Contains(ip) {
return true
}
}
return false
}
func (lbc *LoadBalancerController) runLeaderElection(ctx context.Context, isLeaderChan chan<- bool) {
leaderelection.RunOrDie(ctx, leaderelection.LeaderElectionConfig{
Lock: lbc.lock,
ReleaseOnCancel: true,
LeaseDuration: 15 * time.Second, //nolint:gomnd // No reason for a 15 second constant
RenewDeadline: 10 * time.Second,
RetryPeriod: 2 * time.Second,
Callbacks: leaderelection.LeaderCallbacks{
OnStartedLeading: func(c context.Context) {
isLeaderChan <- true
},
OnStoppedLeading: func() {
isLeaderChan <- false
},
OnNewLeader: func(_ string) {},
},
})
}
func (lbc *LoadBalancerController) OnAdd(obj interface{}) {
if svc, ok := obj.(*v1core.Service); ok {
lbc.addChan <- *svc
}
}
func (lbc *LoadBalancerController) OnDelete(obj interface{}) {
}
func (lbc *LoadBalancerController) OnUpdate(oldObj interface{}, newObj interface{}) {
if svc, ok := newObj.(*v1core.Service); ok {
lbc.addChan <- *svc
}
}
func getIPFamilies(families []v1core.IPFamily) (v4, v6 bool) {
for _, ipf := range families {
switch ipf {
case v1core.IPv4Protocol:
v4 = true
case v1core.IPv6Protocol:
v6 = true
}
}
return v4, v6
}
func getCurrentIngressFamilies(svc *v1core.Service) (v4, v6 bool) {
for _, lbi := range svc.Status.LoadBalancer.Ingress {
ip := net.ParseIP(lbi.IP)
switch {
case ip == nil:
continue
case ip.To4() != nil:
v4 = true
case ip.To4() == nil:
v6 = true
}
}
return v4, v6
}
func checkIngress(svc *v1core.Service) bool {
want4, want6 := getIPFamilies(svc.Spec.IPFamilies)
have4, have6 := getCurrentIngressFamilies(svc)
if want4 != have4 {
return true
}
if want6 != have6 {
return true
}
return false
}
func (lbc *LoadBalancerController) checkClass(svc *v1core.Service) bool {
cls := ""
if svc.Spec.LoadBalancerClass != nil {
cls = *svc.Spec.LoadBalancerClass
}
switch {
case cls == loadBalancerClassName:
return true
case lbc.isDefault && cls == "default":
return true
case lbc.isDefault && cls == "":
return true
}
return false
}
func (lbc *LoadBalancerController) shouldAllocate(svc *v1core.Service) bool {
if svc.Spec.Type != v1core.ServiceTypeLoadBalancer {
return false
}
if !lbc.checkClass(svc) {
return false
}
if !checkIngress(svc) {
return false
}
return true
}
func (lbc *LoadBalancerController) walkServices() {
var svc *v1core.Service
var ok bool
for _, obj := range lbc.svcLister.List() {
if svc, ok = obj.(*v1core.Service); !ok {
continue
}
if lbc.shouldAllocate(svc) {
lbc.addChan <- *svc
}
}
}
func (lbc *LoadBalancerController) canAllocate(svc v1core.Service) error {
canV4 := lbc.ipv4Ranges.Len() != 0
canV6 := lbc.ipv6Ranges.Len() != 0
requireDual := (svc.Spec.IPFamilyPolicy != nil && *svc.Spec.IPFamilyPolicy == v1core.IPFamilyPolicyRequireDualStack)
if requireDual && !canV4 {
return errors.New("IPv4 address required, but no IPv4 ranges available")
}
if requireDual && !canV6 {
return errors.New("IPv6 address required, but no IPv6 ranges available")
}
ipv4, ipv6 := getIPFamilies(svc.Spec.IPFamilies)
if ipv4 && !canV4 && !ipv6 {
return errors.New("no IPv4 ranges specified")
}
if ipv6 && !canV6 && !ipv4 {
return errors.New("no IPv6 ranges specified")
}
return nil
}
func (lbc *LoadBalancerController) getIPsFromService(svc *v1core.Service) ([]net.IP, []net.IP) {
v4 := make([]net.IP, 0)
v6 := make([]net.IP, 0)
allips := make([]string, 0)
allips = append(allips, svc.Spec.ExternalIPs...)
for _, lin := range svc.Status.LoadBalancer.Ingress {
if lin.IP == "" {
continue
}
allips = append(allips, lin.IP)
}
for _, sip := range allips {
ip := net.ParseIP(sip)
if ip == nil {
continue
}
ip4 := ip.To4()
switch {
case ip4 != nil && lbc.ipv4Ranges.Contains(ip4):
v4 = append(v4, ip4)
case lbc.ipv6Ranges.Contains(ip):
v6 = append(v6, ip)
}
}
return v4, v6
}
func (lbc *LoadBalancerController) getAllocatedIPs() ([]net.IP, []net.IP) {
allocated4 := make([]net.IP, 0)
allocated6 := make([]net.IP, 0)
var svc *v1core.Service
var ok bool
for _, obj := range lbc.svcLister.List() {
if svc, ok = obj.(*v1core.Service); !ok {
continue
}
ips4, ips6 := lbc.getIPsFromService(svc)
allocated4 = append(allocated4, ips4...)
allocated6 = append(allocated6, ips6...)
}
return allocated4, allocated6
}
func appendIngressIP(svc *v1core.Service, ip net.IP) {
lbi := v1core.LoadBalancerIngress{
IP: ip.String(),
}
svc.Status.LoadBalancer.Ingress = append(svc.Status.LoadBalancer.Ingress, lbi)
}
func (lbc *LoadBalancerController) updateService(svc *v1core.Service, ips ...net.IP) {
if lbc.clientset == nil {
panic("clientset")
}
if lbc.clientset.CoreV1() == nil {
panic("corev1")
}
svcClient := lbc.clientset.CoreV1().Services(svc.Namespace)
err := retry.RetryOnConflict(retry.DefaultRetry, func() error {
curSvc, err := svcClient.Get(context.TODO(), svc.Name, metav1.GetOptions{})
if err != nil {
return err
}
for _, ip := range ips {
if ip == nil {
continue
}
appendIngressIP(curSvc, ip)
}
_, err = svcClient.UpdateStatus(context.TODO(), curSvc, metav1.UpdateOptions{})
return err
})
if err != nil {
klog.Errorf("unable to update %s in %s: %s", svc.Name, svc.Namespace, err)
}
}
func (lbc *LoadBalancerController) allocateService(svc *v1core.Service) error {
allocated4, allocated6 := lbc.getAllocatedIPs()
requireDual := (svc.Spec.IPFamilyPolicy != nil && *svc.Spec.IPFamilyPolicy == v1core.IPFamilyPolicyRequireDualStack)
want4, want6 := getIPFamilies(svc.Spec.IPFamilies)
have4, have6 := getCurrentIngressFamilies(svc)
var ipv4, ipv6 net.IP
var err4, err6 error
if want4 && !have4 {
ipv4, err4 = lbc.ipv4Ranges.getNextFreeIP(allocated4)
}
if want6 && !have6 {
ipv6, err6 = lbc.ipv6Ranges.getNextFreeIP(allocated6)
}
err := err6
if err4 != nil {
err = err4
}
if ipv4 == nil && ipv6 == nil {
return errors.New("unable to allocate address: " + err.Error())
}
if (ipv4 == nil || ipv6 == nil) && requireDual {
return errors.New("unable to allocate dual-stack addresses: " + err.Error())
}
go lbc.updateService(svc, ipv4, ipv6)
return nil
}
func (lbc *LoadBalancerController) allocator() {
for svc := range lbc.allocateChan {
err := lbc.canAllocate(svc)
if err != nil {
klog.Errorf("can not allocate address for %s in %s: %s",
svc.Name, svc.Namespace, err)
continue
}
err = lbc.allocateService(&svc)
if err != nil {
klog.Errorf("failed to allocate address for %s in %s: %s",
svc.Name, svc.Namespace, err)
continue
}
}
}
func (lbc *LoadBalancerController) Run(healthChan chan<- *healthcheck.ControllerHeartbeat,
stopCh <-chan struct{}, wg *sync.WaitGroup) {
isLeader := false
isLeaderChan := make(chan bool)
ctx, cancel := context.WithCancel(context.Background())
timer := time.NewTimer(lbc.syncPeriod)
defer wg.Done()
defer cancel()
defer close(lbc.allocateChan)
go lbc.runLeaderElection(ctx, isLeaderChan)
go lbc.allocator()
for {
select {
case <-stopCh:
klog.Info("shutting down load balancer allocator controller")
return
case isLeader = <-isLeaderChan:
if isLeader {
klog.Info("became the load balancer controller leader, syncing...")
go lbc.walkServices()
}
case svc := <-lbc.addChan:
if isLeader && lbc.shouldAllocate(&svc) {
lbc.allocateChan <- svc
}
case <-timer.C:
timer.Reset(time.Minute)
healthcheck.SendHeartBeat(healthChan, "LBC")
if isLeader {
go lbc.walkServices()
}
}
}
}
func NewLoadBalancerController(clientset kubernetes.Interface,
config *options.KubeRouterConfig, svcInformer cache.SharedIndexInformer,
) (*LoadBalancerController, error) {
ranges4 := make([]net.IPNet, 0)
ranges6 := make([]net.IPNet, 0)
for _, ir := range config.LoadBalancerCIDRs {
ip, cidr, err := net.ParseCIDR(ir)
if err != nil {
return nil, err
}
if ip.To4() != nil && !config.EnableIPv4 {
return nil, errors.New("IPv4 loadbalancer CIDR specified while IPv4 is disabled")
}
if ip.To4() == nil && !config.EnableIPv6 {
return nil, errors.New("IPv6 loadbalancer CIDR specified while IPv6 is disabled")
}
if ip.To4() != nil {
ranges4 = append(ranges4, *cidr)
} else {
ranges6 = append(ranges6, *cidr)
}
}
lbc := &LoadBalancerController{
ipv4Ranges: newipRanges(ranges4),
ipv6Ranges: newipRanges(ranges6),
addChan: make(chan v1core.Service),
allocateChan: make(chan v1core.Service),
clientset: clientset,
isDefault: config.LoadBalancerDefaultClass,
syncPeriod: config.LoadBalancerSyncPeriod,
}
lbc.svcLister = svcInformer.GetIndexer()
namespace, err := getNamespace()
if err != nil {
return nil, err
}
podname, err := getPodname()
if err != nil {
return nil, err
}
lbc.lock = &resourcelock.LeaseLock{
LeaseMeta: metav1.ObjectMeta{
Name: "kube-router-lballoc",
Namespace: namespace,
},
Client: clientset.CoordinationV1(),
LockConfig: resourcelock.ResourceLockConfig{
Identity: podname,
},
}
return lbc, nil
}

View File

@ -0,0 +1,692 @@
package lballoc
import (
"errors"
"net"
"testing"
"time"
"github.com/cloudnativelabs/kube-router/v2/pkg/options"
v1core "k8s.io/api/core/v1"
"k8s.io/client-go/kubernetes/fake"
"k8s.io/client-go/tools/cache"
)
const (
testName = "falafel"
testDefaultClass = "default"
)
func TestGetNamespace(t *testing.T) {
errExp := error(nil)
t.Setenv("POD_NAMESPACE", testName)
ns, err := getNamespace()
if ns != testName {
t.Fatalf("expected %s, got %s", testName, ns)
}
if err != errExp {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestGetNamespaceFail(t *testing.T) {
nsExp := ""
errExp := errors.New("unable to get namespace from kubernetes environment or $POD_NAMESPACE")
ns, err := getNamespace()
if ns != nsExp {
t.Fatalf("expected \"%s\", got %s", nsExp, ns)
}
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestGetPodName(t *testing.T) {
errExp := error(nil)
t.Setenv("POD_NAME", testName)
name, err := getPodname()
if name != testName {
t.Fatalf("expected %s, got %s", testName, name)
}
if err != errExp {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestGetPodNameFail(t *testing.T) {
nameExp := ""
errExp := errors.New("unable to get pod name from $POD_NAME")
name, err := getPodname()
if name != nameExp {
t.Fatalf("expected \"%s\", got %s", nameExp, name)
}
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestIPRangesEmpty(t *testing.T) {
lenExp := 0
ipExp := net.IP(nil)
errExp := errors.New("no IPs left to allocate")
allocated := make([]net.IP, 0)
ir := newipRanges(nil)
l := ir.Len()
if l != lenExp {
t.Fatalf("expected %d, got %d", lenExp, l)
}
ip, err := ir.getNextFreeIP(allocated)
if ip != nil {
t.Fatalf("expected %s, got %s", ipExp, ip)
}
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestIPRange(t *testing.T) {
lenExp := 1
ipExp := net.ParseIP("ffff::")
onesExp := 128
bitsExp := 128
errExp := errors.New("no IPs left to allocate")
containsExp := true
allocated := make([]net.IP, 0)
_, ipnet, err := net.ParseCIDR("ffff::/128")
if err != nil {
t.Fatalf("expected %s, got %s", error(nil), err)
}
ipnets := append([]net.IPNet(nil), *ipnet)
ir := newipRanges(ipnets)
l := ir.Len()
if l != lenExp {
t.Fatalf("expected %d, got %d", lenExp, l)
}
if !ir.ipRanges[0].IP.Equal(ipExp) {
t.Fatalf("expected %s, got %s", ipExp, ir.ipRanges[0].IP)
}
ones, bits := ir.ipRanges[0].Mask.Size()
if ones != onesExp {
t.Fatalf("expected %d, got %d", onesExp, ones)
}
if bits != bitsExp {
t.Fatalf("expected %d, got %d", bitsExp, bits)
}
ip, err := ir.getNextFreeIP(allocated)
if !ip.Equal(ipExp) {
t.Fatalf("expected %s, got %s", ipExp, ip)
}
if err != nil {
t.Fatalf("expected %s, got %s", error(nil), err)
}
allocated = append(allocated, ip)
ip, err = ir.getNextFreeIP(allocated)
if ip != nil {
t.Fatalf("expected %s, got %s", net.IP(nil), ip)
}
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
contains := ir.Contains(ipExp)
if contains != containsExp {
t.Fatalf("expected %t, got %t", containsExp, contains)
}
}
func TestGetIPFamilies(t *testing.T) {
v4Exp := true
v6Exp := true
families := append([]v1core.IPFamily{}, v1core.IPv4Protocol, v1core.IPv6Protocol)
v4, v6 := getIPFamilies(families)
if v4 != v4Exp {
t.Fatalf("expected %t, got %t", v4Exp, v4)
}
if v6 != v6Exp {
t.Fatalf("expected %t, got %t", v6Exp, v6)
}
}
func makeTestService() v1core.Service {
svc := v1core.Service{
Spec: v1core.ServiceSpec{
Type: v1core.ServiceTypeLoadBalancer,
},
}
svc.Name = testName
svc.Namespace = "tahini"
svc.Spec.LoadBalancerClass = nil
svc.Spec.IPFamilies = append([]v1core.IPFamily{}, v1core.IPv4Protocol, v1core.IPv6Protocol)
return svc
}
func TestGetCurrentIngressFamilies(t *testing.T) {
svc := makeTestService()
for _, tip := range []string{"ffff::", "127.127.127.127"} {
ing := v1core.LoadBalancerIngress{
IP: tip,
}
svc.Status.LoadBalancer.Ingress = append(svc.Status.LoadBalancer.Ingress, ing)
}
expV4 := true
expV6 := true
v4, v6 := getCurrentIngressFamilies(&svc)
if expV4 != v4 {
t.Fatalf("expected %t, got %t", expV4, v4)
}
if expV6 != v6 {
t.Fatalf("expected %t, got %t", expV6, v6)
}
}
func TestCheckIngress(t *testing.T) {
svc := makeTestService()
check := checkIngress(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
v6Ingress := v1core.LoadBalancerIngress{
IP: "ffff::",
}
svc.Status.LoadBalancer.Ingress = append(svc.Status.LoadBalancer.Ingress, v6Ingress)
check = checkIngress(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
v4Ingress := v1core.LoadBalancerIngress{
IP: "127.127.127.127",
}
svc.Status.LoadBalancer.Ingress = append(svc.Status.LoadBalancer.Ingress, v4Ingress)
check = checkIngress(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
}
func TestCheckClass(t *testing.T) {
lbc := &LoadBalancerController{
isDefault: true,
}
svc := makeTestService()
svc.Spec.LoadBalancerClass = nil
check := lbc.checkClass(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
lbc.isDefault = false
check = lbc.checkClass(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
cls := ""
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
cls = testDefaultClass
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
cls = loadBalancerClassName
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
lbc.isDefault = true
cls = ""
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
cls = testDefaultClass
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
cls = loadBalancerClassName
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
cls = testName
svc.Spec.LoadBalancerClass = &cls
check = lbc.checkClass(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
}
func TestShouldAllocate(t *testing.T) {
lbc := &LoadBalancerController{
isDefault: true,
}
svc := makeTestService()
check := lbc.shouldAllocate(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
svc.Spec.Type = v1core.ServiceTypeExternalName
check = lbc.shouldAllocate(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
svc.Spec.Type = v1core.ServiceTypeLoadBalancer
cls := testName
svc.Spec.LoadBalancerClass = &cls
check = lbc.shouldAllocate(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
svc.Spec.LoadBalancerClass = nil
svc.Spec.IPFamilies = append([]v1core.IPFamily{}, v1core.IPv4Protocol)
ingress := v1core.LoadBalancerIngress{
IP: "127.127.127.127",
}
svc.Status.LoadBalancer.Ingress = append([]v1core.LoadBalancerIngress{}, ingress)
check = lbc.shouldAllocate(&svc)
if check {
t.Fatalf("expected %t, got %t", false, check)
}
ingress = v1core.LoadBalancerIngress{
IP: "ffff::",
}
svc.Status.LoadBalancer.Ingress = append([]v1core.LoadBalancerIngress{}, ingress)
check = lbc.shouldAllocate(&svc)
if !check {
t.Fatalf("expected %t, got %t", true, check)
}
}
type mockIndexer struct {
cache.FakeCustomStore
objects []interface{}
}
func (mi *mockIndexer) Index(_ string, _ interface{}) ([]interface{}, error) {
return nil, errors.New("unsupported")
}
func (mi *mockIndexer) IndexKeys(_, _ string) ([]string, error) {
return nil, errors.New("unsupported")
}
func (mi *mockIndexer) ListIndexFuncValues(_ string) []string {
return nil
}
func (mi *mockIndexer) ByIndex(_, _ string) ([]interface{}, error) {
return nil, errors.New("unsupported")
}
func (mi *mockIndexer) GetIndexers() cache.Indexers {
return nil
}
func (mi *mockIndexer) AddIndexers(_ cache.Indexers) error {
return errors.New("unsupported")
}
func (mi *mockIndexer) List() []interface{} {
return mi.objects
}
func newMockIndexer(objects ...interface{}) *mockIndexer {
mi := &mockIndexer{
objects: make([]interface{}, 0),
}
mi.objects = append(mi.objects, objects...)
return mi
}
func TestWalkServices(t *testing.T) {
svc1 := makeTestService()
svc2 := true
mi := newMockIndexer(svc1, svc2)
addChan := make(chan v1core.Service, 2)
lbc := &LoadBalancerController{
svcLister: mi,
addChan: addChan,
}
lbc.walkServices()
close(lbc.addChan)
out := make([]v1core.Service, 1)
for svc := range lbc.addChan {
out = append(out, svc)
}
l := 1
lenExp := 1
if len(out) != lenExp {
t.Fatalf("expected %d, got %d", lenExp, l)
}
}
func makeIPRanges(ips ...string) (ir4, ir6 *ipRanges) {
var v4, v6 []net.IPNet
for _, sip := range ips {
_, ipn, _ := net.ParseCIDR(sip)
if ipn == nil {
continue
}
if ipn.IP.To4() != nil {
v4 = append(v4, *ipn)
} else {
v6 = append(v6, *ipn)
}
}
ir4 = newipRanges(v4)
ir6 = newipRanges(v6)
return ir4, ir6
}
func TestCanAllocate(t *testing.T) {
ir4, ir6 := makeIPRanges("127.127.127.127/32", "ffff::/32")
lbc := &LoadBalancerController{
ipv4Ranges: ir4,
ipv6Ranges: ir6,
}
ippol := v1core.IPFamilyPolicy("RequireDualStack")
svc := makeTestService()
svc.Spec.IPFamilyPolicy = &ippol
err := lbc.canAllocate(svc)
if err != nil {
t.Fatalf("expected %v, got %s", nil, err)
}
lbc.ipv4Ranges = newipRanges(nil)
errExp := errors.New("IPv4 address required, but no IPv4 ranges available")
err = lbc.canAllocate(svc)
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
lbc.ipv4Ranges = ir4
lbc.ipv6Ranges = newipRanges(nil)
errExp = errors.New("IPv6 address required, but no IPv6 ranges available")
err = lbc.canAllocate(svc)
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
ippol = v1core.IPFamilyPolicy("PreferDualStack")
svc.Spec.IPFamilyPolicy = &ippol
svc.Spec.IPFamilies = append([]v1core.IPFamily{}, v1core.IPv4Protocol)
err = lbc.canAllocate(svc)
if err != nil {
t.Fatalf("expected %v, got %s", nil, err)
}
svc.Spec.IPFamilies = append([]v1core.IPFamily{}, v1core.IPv6Protocol)
err = lbc.canAllocate(svc)
errExp = errors.New("no IPv6 ranges specified")
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
lbc.ipv4Ranges = newipRanges(nil)
lbc.ipv6Ranges = ir6
svc.Spec.IPFamilies = append([]v1core.IPFamily{}, v1core.IPv4Protocol)
err = lbc.canAllocate(svc)
errExp = errors.New("no IPv4 ranges specified")
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
lbc.ipv6Ranges = newipRanges(nil)
err = lbc.canAllocate(svc)
errExp = errors.New("no IPv4 ranges specified")
if err.Error() != errExp.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
func TestGetIPsFromService(t *testing.T) {
svc := makeTestService()
ir4, ir6 := makeIPRanges("127.127.127.127/32", "ffff::/32")
lbc := &LoadBalancerController{
ipv4Ranges: ir4,
ipv6Ranges: ir6,
}
svc.Spec.ExternalIPs = append([]string{}, "falafel", "127.127.127.127")
for _, is := range []string{"ffff::", "aaaa::", "tahini"} {
ing := v1core.LoadBalancerIngress{
IP: is,
}
svc.Status.LoadBalancer.Ingress = append(svc.Status.LoadBalancer.Ingress, ing)
}
addresses4, addresses6 := lbc.getIPsFromService(&svc)
l4Exp := 1
l6Exp := 1
l4 := len(addresses4)
l6 := len(addresses6)
if l4 != l4Exp {
t.Fatalf("expected %d, got %d", l4Exp, l4)
}
if l6 != l6Exp {
t.Fatalf("expected %d, got %d", l6Exp, l6)
}
}
func TestGetAllocatedIPs(t *testing.T) {
svcExt := makeTestService()
svcExt.Spec.ExternalIPs = append([]string{}, "ffff::", "kaka", "255.255.255.255")
svcLB := makeTestService()
for _, is := range []string{"aaaa::", "127.127.127.127"} {
ing := v1core.LoadBalancerIngress{
IP: is,
}
svcLB.Status.LoadBalancer.Ingress = append(svcLB.Status.LoadBalancer.Ingress, ing)
}
mi := newMockIndexer(&svcExt, &svcLB, 1234)
ir4, ir6 := makeIPRanges("127.127.127.127/32", "ffff::/32")
lbc := &LoadBalancerController{
ipv4Ranges: ir4,
ipv6Ranges: ir6,
svcLister: mi,
}
allocated4, allocated6 := lbc.getAllocatedIPs()
l4Exp := 1
l4 := len(allocated4)
if l4 != l4Exp {
t.Fatalf("expected %d, got %d", l4Exp, l4)
}
l6Exp := 1
l6 := len(allocated6)
if l6 != l6Exp {
t.Fatalf("expected %d, got %d", l6Exp, l6)
}
}
func TestAppendIngressIP(t *testing.T) {
svc := makeTestService()
ip := net.ParseIP("127.127.127.127")
appendIngressIP(&svc, ip)
ilExp := 1
il := len(svc.Status.LoadBalancer.Ingress)
if ilExp != il {
t.Fatalf("expected %d, got %d", ilExp, il)
}
ipExp := "127.127.127.127"
if ipExp != svc.Status.LoadBalancer.Ingress[0].IP {
t.Fatalf("expected %s, got %s", ipExp, svc.Status.LoadBalancer.Ingress[0].IP)
}
}
func TestAllocateService(t *testing.T) {
mlbc := &LoadBalancerController{
clientset: fake.NewSimpleClientset(),
}
ir4, ir6 := makeIPRanges("127.127.127.127/30", "ffff::/80")
mlbc.ipv4Ranges = ir4
mlbc.ipv6Ranges = ir6
mi := newMockIndexer()
mlbc.svcLister = mi
svc := makeTestService()
err := mlbc.allocateService(&svc)
if err != nil {
t.Fatalf("expected %v, got %s", nil, err)
}
svc = makeTestService()
mlbc.ipv4Ranges = newipRanges(nil)
fp := v1core.IPFamilyPolicyRequireDualStack
svc.Spec.IPFamilyPolicy = &fp
err = mlbc.allocateService(&svc)
errExp := "unable to allocate dual-stack addresses: no IPs left to allocate"
if errExp != err.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
mlbc.ipv4Ranges = ir4
mlbc.ipv6Ranges = newipRanges(nil)
err = mlbc.allocateService(&svc)
if errExp != err.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
mlbc.ipv4Ranges = newipRanges(nil)
fp = v1core.IPFamilyPolicyPreferDualStack
svc.Spec.IPFamilyPolicy = &fp
err = mlbc.allocateService(&svc)
errExp = "unable to allocate address: no IPs left to allocate"
if errExp != err.Error() {
t.Fatalf("expected %s, got %s", errExp, err)
}
}
type mockInformer struct {
}
func (mf *mockInformer) GetIndexer() cache.Indexer {
return newMockIndexer()
}
func (mf *mockInformer) AddIndexers(_ cache.Indexers) error {
return nil
}
func (mf *mockInformer) AddEventHandler(_ cache.ResourceEventHandler) {
}
func (mf *mockInformer) AddEventHandlerWithResyncPeriod(_ cache.ResourceEventHandler, _ time.Duration) {
}
func (mf *mockInformer) GetController() cache.Controller {
return nil
}
func (mf *mockInformer) GetStore() cache.Store {
return nil
}
func (mf *mockInformer) HasSynced() bool {
return false
}
func (mf *mockInformer) LastSyncResourceVersion() string {
return ""
}
func (mf *mockInformer) Run(_ <-chan struct{}) {
}
func (mf *mockInformer) SetTransform(_ cache.TransformFunc) error {
return nil
}
func (mf *mockInformer) SetWatchErrorHandler(_ cache.WatchErrorHandler) error {
return nil
}
func TestNewLoadBalancerController(t *testing.T) {
t.Setenv("POD_NAMESPACE", testName)
t.Setenv("POD_NAME", testName)
mf := &mockInformer{}
config := &options.KubeRouterConfig{
LoadBalancerCIDRs: []string{"127.127.127.127/30", "ffff::/80"},
EnableIPv4: true,
EnableIPv6: true,
}
fs := fake.NewSimpleClientset()
_, err := NewLoadBalancerController(fs, config, mf)
if err != nil {
t.Fatalf("expected %v, got %s", nil, err)
}
config.EnableIPv4 = false
_, err = NewLoadBalancerController(fs, config, mf)
errExp := "IPv4 loadbalancer CIDR specified while IPv4 is disabled"
if err.Error() != errExp {
t.Fatalf("expected %s, got %s", errExp, err)
}
config.EnableIPv4 = true
config.EnableIPv6 = false
_, err = NewLoadBalancerController(fs, config, mf)
errExp = "IPv6 loadbalancer CIDR specified while IPv6 is disabled"
if err.Error() != errExp {
t.Fatalf("expected %s, got %s", errExp, err)
}
}

View File

@ -64,16 +64,17 @@ var (
// NetworkPolicyController struct to hold information required by NetworkPolicyController
type NetworkPolicyController struct {
nodeHostName string
serviceClusterIPRanges []net.IPNet
serviceExternalIPRanges []net.IPNet
serviceNodePortRange string
mu sync.Mutex
syncPeriod time.Duration
MetricsEnabled bool
healthChan chan<- *healthcheck.ControllerHeartbeat
fullSyncRequestChan chan struct{}
ipsetMutex *sync.Mutex
nodeHostName string
serviceClusterIPRanges []net.IPNet
serviceExternalIPRanges []net.IPNet
serviceLoadBalancerIPRanges []net.IPNet
serviceNodePortRange string
mu sync.Mutex
syncPeriod time.Duration
MetricsEnabled bool
healthChan chan<- *healthcheck.ControllerHeartbeat
fullSyncRequestChan chan struct{}
ipsetMutex *sync.Mutex
iptablesCmdHandlers map[v1core.IPFamily]utils.IPTablesHandler
iptablesSaveRestore map[v1core.IPFamily]utils.IPTablesSaveRestorer
@ -494,6 +495,32 @@ func (npc *NetworkPolicyController) ensureTopLevelChains() {
kubeInputChainName, whitelistServiceVips, uuid, rulePosition[family])
rulePosition[family]++
}
for idx, loadBalancerIPRange := range npc.serviceLoadBalancerIPRanges {
var family v1core.IPFamily
if loadBalancerIPRange.IP.To4() != nil {
family = v1core.IPv4Protocol
} else {
family = v1core.IPv6Protocol
}
whitelistServiceVips := []string{"-m", "comment", "--comment",
"allow traffic to load balancer IP range: " + loadBalancerIPRange.String(), "-d", loadBalancerIPRange.String(),
"-j", "RETURN"}
uuid, err := addUUIDForRuleSpec(kubeInputChainName, &whitelistServiceVips)
if err != nil {
klog.Fatalf("Failed to get uuid for rule: %s", err.Error())
}
// Access loadBalancerIPRange via index to avoid implicit memory aliasing
cidrHandler, err := npc.iptablesCmdHandlerForCIDR(&npc.serviceLoadBalancerIPRanges[idx])
if err != nil {
klog.Fatalf("Failed to get iptables handler: %s", err.Error())
}
klog.V(2).Infof("Allow traffic to ingress towards Load Balancer IP Range: %s for family: %s",
loadBalancerIPRange.String(), family)
ensureRuleAtPosition(cidrHandler,
kubeInputChainName, whitelistServiceVips, uuid, rulePosition[family])
rulePosition[family]++
}
}
func (npc *NetworkPolicyController) ensureExplicitAccept() {
@ -806,6 +833,16 @@ func NewNetworkPolicyController(clientset kubernetes.Interface,
npc.serviceExternalIPRanges = append(npc.serviceExternalIPRanges, *ipnet)
}
// Validate and parse LoadBalancerIP service range
for _, loadBalancerIPRange := range config.LoadBalancerCIDRs {
_, ipnet, err := net.ParseCIDR(loadBalancerIPRange)
if err != nil {
return nil, fmt.Errorf("failed to get parse --loadbalancer-ip-range parameter: '%s'. Error: %s",
loadBalancerIPRange, err.Error())
}
npc.serviceLoadBalancerIPRanges = append(npc.serviceLoadBalancerIPRanges, *ipnet)
}
if config.MetricsEnabled {
// Register the metrics for this controller
prometheus.MustRegister(metrics.ControllerIptablesSyncTime)

View File

@ -275,7 +275,7 @@ func testForMissingOrUnwanted(t *testing.T, targetMsg string, got []podInfo, wan
}
}
func newMinimalKubeRouterConfig(clusterIPCIDRs []string, nodePortRange string, hostNameOverride string, externalIPs []string, enableIPv6 bool) *options.KubeRouterConfig {
func newMinimalKubeRouterConfig(clusterIPCIDRs []string, nodePortRange string, hostNameOverride string, externalIPs []string, loadBalancerIPs []string, enableIPv6 bool) *options.KubeRouterConfig {
kubeConfig := options.NewKubeRouterConfig()
if len(clusterIPCIDRs) > 0 && clusterIPCIDRs[0] != "" {
kubeConfig.ClusterIPCIDRs = clusterIPCIDRs
@ -289,6 +289,9 @@ func newMinimalKubeRouterConfig(clusterIPCIDRs []string, nodePortRange string, h
if externalIPs != nil {
kubeConfig.ExternalIPCIDRs = externalIPs
}
if loadBalancerIPs != nil {
kubeConfig.LoadBalancerCIDRs = loadBalancerIPs
}
kubeConfig.EnableIPv4 = true
kubeConfig.EnableIPv6 = enableIPv6
return kubeConfig
@ -750,145 +753,181 @@ func TestNetworkPolicyController(t *testing.T) {
testCases := []tNetPolConfigTestCase{
{
"Default options are successful",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, nil, false),
false,
"",
},
{
"Missing nodename fails appropriately",
newMinimalKubeRouterConfig([]string{""}, "", "", nil, false),
newMinimalKubeRouterConfig([]string{""}, "", "", nil, nil, false),
true,
"failed to identify the node by NODE_NAME, hostname or --hostname-override",
},
{
"Test bad cluster CIDR (not properly formatting ip address)",
newMinimalKubeRouterConfig([]string{"10.10.10"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.10.10"}, "", "node", nil, nil, false),
true,
"failed to get parse --service-cluster-ip-range parameter: invalid CIDR address: 10.10.10",
},
{
"Test bad cluster CIDR (not using an ip address)",
newMinimalKubeRouterConfig([]string{"foo"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"foo"}, "", "node", nil, nil, false),
true,
"failed to get parse --service-cluster-ip-range parameter: invalid CIDR address: foo",
},
{
"Test bad cluster CIDR (using an ip address that is not a CIDR)",
newMinimalKubeRouterConfig([]string{"10.10.10.10"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.10.10.10"}, "", "node", nil, nil, false),
true,
"failed to get parse --service-cluster-ip-range parameter: invalid CIDR address: 10.10.10.10",
},
{
"Test bad cluster CIDRs (using more than 2 ip addresses, including 2 ipv4)",
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "10.244.0.0/16", "2001:db8:42:1::/112"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "10.244.0.0/16", "2001:db8:42:1::/112"}, "", "node", nil, nil, false),
true,
"too many CIDRs provided in --service-cluster-ip-range parameter: dual-stack must be enabled to provide two addresses",
},
{
"Test bad cluster CIDRs (using more than 2 ip addresses, including 2 ipv6)",
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "2001:db8:42:0::/56", "2001:db8:42:1::/112"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "2001:db8:42:0::/56", "2001:db8:42:1::/112"}, "", "node", nil, nil, false),
true,
"too many CIDRs provided in --service-cluster-ip-range parameter: dual-stack must be enabled to provide two addresses",
},
{
"Test good cluster CIDR (using single IP with a /32)",
newMinimalKubeRouterConfig([]string{"10.10.10.10/32"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.10.10.10/32"}, "", "node", nil, nil, false),
false,
"",
},
{
"Test good cluster CIDR (using normal range with /24)",
newMinimalKubeRouterConfig([]string{"10.10.10.0/24"}, "", "node", nil, false),
newMinimalKubeRouterConfig([]string{"10.10.10.0/24"}, "", "node", nil, nil, false),
false,
"",
},
{
"Test good cluster CIDR (using ipv6)",
newMinimalKubeRouterConfig([]string{"2001:db8:42:1::/112"}, "", "node", []string{"2001:db8:42:1::/112"}, true),
newMinimalKubeRouterConfig([]string{"2001:db8:42:1::/112"}, "", "node", []string{"2001:db8:42:1::/112"}, []string{"2001:db8:43:1::/112"}, true),
false,
"",
},
{
"Test good cluster CIDRs (with dual-stack)",
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "2001:db8:42:1::/112"}, "", "node", []string{"10.96.0.0/12", "2001:db8:42:1::/112"}, true),
newMinimalKubeRouterConfig([]string{"10.96.0.0/12", "2001:db8:42:1::/112"}, "", "node", []string{"10.96.0.0/12", "2001:db8:42:1::/112"}, []string{"10.97.0.0/12", "2001:db8:43:1::/112"}, true),
false,
"",
},
{
"Test bad node port specification (using commas)",
newMinimalKubeRouterConfig([]string{""}, "8080,8081", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "8080,8081", "node", nil, nil, false),
true,
"failed to parse node port range given: '8080,8081' please see specification in help text",
},
{
"Test bad node port specification (not using numbers)",
newMinimalKubeRouterConfig([]string{""}, "foo:bar", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "foo:bar", "node", nil, nil, false),
true,
"failed to parse node port range given: 'foo:bar' please see specification in help text",
},
{
"Test bad node port specification (using anything in addition to range)",
newMinimalKubeRouterConfig([]string{""}, "8080,8081-8090", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "8080,8081-8090", "node", nil, nil, false),
true,
"failed to parse node port range given: '8080,8081-8090' please see specification in help text",
},
{
"Test bad node port specification (using reversed range)",
newMinimalKubeRouterConfig([]string{""}, "8090-8080", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "8090-8080", "node", nil, nil, false),
true,
"port 1 is greater than or equal to port 2 in range given: '8090-8080'",
},
{
"Test bad node port specification (port out of available range)",
newMinimalKubeRouterConfig([]string{""}, "132000-132001", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "132000-132001", "node", nil, nil, false),
true,
"could not parse first port number from range given: '132000-132001'",
},
{
"Test good node port specification (using colon separator)",
newMinimalKubeRouterConfig([]string{""}, "8080:8090", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "8080:8090", "node", nil, nil, false),
false,
"",
},
{
"Test good node port specification (using hyphen separator)",
newMinimalKubeRouterConfig([]string{""}, "8080-8090", "node", nil, false),
newMinimalKubeRouterConfig([]string{""}, "8080-8090", "node", nil, nil, false),
false,
"",
},
{
"Test bad external IP CIDR (not properly formatting ip address)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10"}, nil, false),
true,
"failed to get parse --service-external-ip-range parameter: '199.10.10'. Error: invalid CIDR address: 199.10.10",
},
{
"Test bad external IP CIDR (not using an ip address)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"foo"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"foo"}, nil, false),
true,
"failed to get parse --service-external-ip-range parameter: 'foo'. Error: invalid CIDR address: foo",
},
{
"Test bad external IP CIDR (using an ip address that is not a CIDR)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10"}, nil, false),
true,
"failed to get parse --service-external-ip-range parameter: '199.10.10.10'. Error: invalid CIDR address: 199.10.10.10",
},
{
"Test bad external IP CIDR (making sure that it processes all items in the list)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/32", "199.10.10.11"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/32", "199.10.10.11"}, nil, false),
true,
"failed to get parse --service-external-ip-range parameter: '199.10.10.11'. Error: invalid CIDR address: 199.10.10.11",
},
{
"Test good external IP CIDR (using single IP with a /32)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/32"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/32"}, nil, false),
false,
"",
},
{
"Test good external IP CIDR (using normal range with /24)",
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/24"}, false),
newMinimalKubeRouterConfig([]string{""}, "", "node", []string{"199.10.10.10/24"}, nil, false),
false,
"",
},
{
"Test bad load balancer CIDR (not properly formatting ip address)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"199.10.10"}, false),
true,
"failed to get parse --loadbalancer-ip-range parameter: '199.10.10'. Error: invalid CIDR address: 199.10.10",
},
{
"Test bad load balancer CIDR (not using an ip address)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"foo"}, false),
true,
"failed to get parse --loadbalancer-ip-range parameter: 'foo'. Error: invalid CIDR address: foo",
},
{
"Test bad load balancer CIDR (using an ip address that is not a CIDR)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"199.10.10.10"}, false),
true,
"failed to get parse --loadbalancer-ip-range parameter: '199.10.10.10'. Error: invalid CIDR address: 199.10.10.10",
},
{
"Test bad load balancer CIDR (making sure that it processes all items in the list)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"199.10.10.10/32", "199.10.10.11"}, false),
true,
"failed to get parse --loadbalancer-ip-range parameter: '199.10.10.11'. Error: invalid CIDR address: 199.10.10.11",
},
{
"Test good load balancer CIDR (using single IP with a /32)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"199.10.10.10/32"}, false),
false,
"",
},
{
"Test good load balancer CIDR (using normal range with /24)",
newMinimalKubeRouterConfig([]string{""}, "", "node", nil, []string{"199.10.10.10/24"}, false),
false,
"",
},

View File

@ -34,6 +34,8 @@ type HealthController struct {
type HealthStats struct {
sync.Mutex
Healthy bool
LoadBalancerControllerAlive time.Time
LoadBalancerControllerAliveTTL time.Duration
MetricsControllerAlive time.Time
NetworkPolicyControllerAlive time.Time
NetworkPolicyControllerAliveTTL time.Duration
@ -90,6 +92,11 @@ func (hc *HealthController) HandleHeartbeat(beat *ControllerHeartbeat) {
switch {
// The first heartbeat will set the initial gracetime the controller has to report in, A static time is added as
// well when checking to allow for load variation in sync time
case beat.Component == "LBC":
if hc.Status.LoadBalancerControllerAliveTTL == 0 {
hc.Status.LoadBalancerControllerAliveTTL = time.Since(hc.Status.LoadBalancerControllerAlive)
}
hc.Status.LoadBalancerControllerAlive = beat.LastHeartBeat
case beat.Component == "NSC":
if hc.Status.NetworkServicesControllerAliveTTL == 0 {
hc.Status.NetworkServicesControllerAliveTTL = time.Since(hc.Status.NetworkServicesControllerAlive)
@ -126,6 +133,14 @@ func (hc *HealthController) CheckHealth() bool {
}
}
if hc.Config.RunLoadBalancer {
if time.Since(hc.Status.LoadBalancerControllerAlive) >
hc.Config.LoadBalancerSyncPeriod+hc.Status.LoadBalancerControllerAliveTTL+graceTime {
klog.Error("Load Balancer Allocator Controller heartbeat missed")
health = false
}
}
if hc.Config.RunRouter {
if time.Since(hc.Status.NetworkRoutingControllerAlive) >
hc.Config.RoutesSyncPeriod+hc.Status.NetworkRoutingControllerAliveTTL+graceTime {
@ -205,6 +220,7 @@ func (hc *HealthController) SetAlive() {
now := time.Now()
hc.Status.LoadBalancerControllerAlive = now
hc.Status.MetricsControllerAlive = now
hc.Status.NetworkPolicyControllerAlive = now
hc.Status.NetworkRoutingControllerAlive = now

View File

@ -52,6 +52,9 @@ type KubeRouterConfig struct {
IpvsPermitAll bool
IpvsSyncPeriod time.Duration
Kubeconfig string
LoadBalancerCIDRs []string
LoadBalancerDefaultClass bool
LoadBalancerSyncPeriod time.Duration
MasqueradeAll bool
Master string
MetricsEnabled bool
@ -74,6 +77,7 @@ type KubeRouterConfig struct {
RunFirewall bool
RunRouter bool
RunServiceProxy bool
RunLoadBalancer bool
RuntimeEndpoint string
Version bool
VLevel string
@ -92,6 +96,7 @@ func NewKubeRouterConfig() *KubeRouterConfig {
IPTablesSyncPeriod: 5 * time.Minute,
IpvsGracefulPeriod: 30 * time.Second,
IpvsSyncPeriod: 5 * time.Minute,
LoadBalancerSyncPeriod: time.Minute,
NodePortRange: "30000-32767",
OverlayType: "subnet",
RoutesSyncPeriod: 5 * time.Minute,
@ -173,6 +178,12 @@ func (s *KubeRouterConfig) AddFlags(fs *pflag.FlagSet) {
"The delay between ipvs config synchronizations (e.g. '5s', '1m', '2h22m'). Must be greater than 0.")
fs.StringVar(&s.Kubeconfig, "kubeconfig", s.Kubeconfig,
"Path to kubeconfig file with authorization information (the master location is set by the master flag).")
fs.BoolVar(&s.LoadBalancerDefaultClass, "loadbalancer-default-class", true,
"Handle loadbalancer services without a class")
fs.StringSliceVar(&s.LoadBalancerCIDRs, "loadbalancer-ip-range", s.LoadBalancerCIDRs,
"CIDR values from which loadbalancer services addresses are assigned (can be specified multiple times)")
fs.DurationVar(&s.LoadBalancerSyncPeriod, "loadbalancer-sync-period", s.LoadBalancerSyncPeriod,
"The delay between checking for missed services (e.g. '5s', '1m'). Must be greater than 0.")
fs.BoolVar(&s.MasqueradeAll, "masquerade-all", false,
"SNAT all traffic to cluster IP/node port.")
fs.StringVar(&s.Master, "master", s.Master,
@ -216,6 +227,8 @@ func (s *KubeRouterConfig) AddFlags(fs *pflag.FlagSet) {
"The delay between route updates and advertisements (e.g. '5s', '1m', '2h22m'). Must be greater than 0.")
fs.BoolVar(&s.RunFirewall, "run-firewall", true,
"Enables Network Policy -- sets up iptables to provide ingress firewall for pods.")
fs.BoolVar(&s.RunLoadBalancer, "run-loadbalancer", false,
"Enable loadbalancer address allocator")
fs.BoolVar(&s.RunRouter, "run-router", true,
"Enables Pod Networking -- Advertises and learns the routes to Pods via iBGP.")
fs.BoolVar(&s.RunServiceProxy, "run-service-proxy", true,