mirror of
https://github.com/cloudnativelabs/kube-router.git
synced 2025-11-26 23:41:06 +01:00
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:
parent
7699d165da
commit
afdf553fa8
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
81
docs/load-balancer-allocator.md
Normal file
81
docs/load-balancer-allocator.md
Normal 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.
|
||||
@ -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.
|
||||
|
||||
@ -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)
|
||||
|
||||
517
pkg/controllers/lballoc/lballoc.go
Normal file
517
pkg/controllers/lballoc/lballoc.go
Normal 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
|
||||
}
|
||||
692
pkg/controllers/lballoc/lballoc_test.go
Normal file
692
pkg/controllers/lballoc/lballoc_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
"",
|
||||
},
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user