Merge pull request #52528 from m1093782566/refactor-proxy

Automatic merge from submit-queue. If you want to cherry-pick this change to another branch, please follow the instructions <a href="https://github.com/kubernetes/community/blob/master/contributors/devel/cherry-picks.md">here</a>.

Refactor kube-proxy service/endpoints update so that can be consumed among different proxiers

**What this PR does / why we need it**:

There are huge duplication among different proxiers.  For example, the service/endpoints list/watch part in iptables, ipvs and windows kernel mode(to be get in soon).

I think the more places this is replicated the harder it becomes to keep correct. We may need to refactor it and let different proxiers consume the same code.

**Which issue this PR fixes**: 

fixes #52464

**Special notes for your reviewer**:

* This refactor reduces **500** Lines in iptables proxy, so it will reduce **500*N**(number of proxiers) lines in total. People no need to care the service/endpoints update logic any more and can be more focus on proxy logic.

* I would like to do the following things in follow-ups:

1. rsync it to ipvs proxier

2. rsync it to winkernel proxier

**Release note**:

```release-note
Refactor kube-proxy service/endpoints update so that can be consumed among different proxiers
```
This commit is contained in:
Kubernetes Submit Queue
2018-02-12 23:29:50 -08:00
committed by GitHub
13 changed files with 2771 additions and 1478 deletions

View File

@@ -3,16 +3,25 @@ package(default_visibility = ["//visibility:public"])
load( load(
"@io_bazel_rules_go//go:def.bzl", "@io_bazel_rules_go//go:def.bzl",
"go_library", "go_library",
"go_test",
) )
go_library( go_library(
name = "go_default_library", name = "go_default_library",
srcs = [ srcs = [
"doc.go", "doc.go",
"endpoints.go",
"service.go",
"types.go", "types.go",
], ],
importpath = "k8s.io/kubernetes/pkg/proxy", importpath = "k8s.io/kubernetes/pkg/proxy",
deps = ["//vendor/k8s.io/apimachinery/pkg/types:go_default_library"], deps = [
"//pkg/apis/core:go_default_library",
"//pkg/proxy/util:go_default_library",
"//vendor/github.com/golang/glog:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/types:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/sets:go_default_library",
],
) )
filegroup( filegroup(
@@ -39,3 +48,22 @@ filegroup(
], ],
tags = ["automanaged"], tags = ["automanaged"],
) )
go_test(
name = "go_default_test",
srcs = [
"endpoints_test.go",
"service_test.go",
],
embed = [":go_default_library"],
importpath = "k8s.io/kubernetes/pkg/proxy",
deps = [
"//pkg/api/service:go_default_library",
"//pkg/apis/core:go_default_library",
"//vendor/github.com/davecgh/go-spew/spew:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/types:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/intstr:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/sets:go_default_library",
],
)

243
pkg/proxy/endpoints.go Normal file
View File

@@ -0,0 +1,243 @@
/*
Copyright 2017 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package proxy
import (
"reflect"
"sync"
"github.com/golang/glog"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/sets"
api "k8s.io/kubernetes/pkg/apis/core"
)
// EndpointChangeTracker carries state about uncommitted changes to an arbitrary number of
// Endpoints, keyed by their namespace and name.
type EndpointChangeTracker struct {
// lock protects items.
lock sync.Mutex
// hostname is the host where kube-proxy is running.
hostname string
// items maps a service to is endpointsChange.
items map[types.NamespacedName]*endpointsChange
}
// NewEndpointChangeTracker initializes an EndpointsChangeMap
func NewEndpointChangeTracker(hostname string) *EndpointChangeTracker {
return &EndpointChangeTracker{
hostname: hostname,
items: make(map[types.NamespacedName]*endpointsChange),
}
}
// Update updates given service's endpoints change map based on the <previous, current> endpoints pair. It returns true
// if items changed, otherwise return false. Update can be used to add/update/delete items of EndpointsChangeMap. For example,
// Add item
// - pass <nil, endpoints> as the <previous, current> pair.
// Update item
// - pass <oldEndpoints, endpoints> as the <previous, current> pair.
// Delete item
// - pass <endpoints, nil> as the <previous, current> pair.
func (ect *EndpointChangeTracker) Update(previous, current *api.Endpoints, makeEndpoints func(IP string, port int, isLocal bool) Endpoint) bool {
endpoints := current
if endpoints == nil {
endpoints = previous
}
// previous == nil && current == nil is unexpected, we should return false directly.
if endpoints == nil {
return false
}
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name}
ect.lock.Lock()
defer ect.lock.Unlock()
change, exists := ect.items[namespacedName]
if !exists {
change = &endpointsChange{}
change.previous = endpointsToEndpointsMap(previous, ect.hostname, makeEndpoints)
ect.items[namespacedName] = change
}
change.current = endpointsToEndpointsMap(current, ect.hostname, makeEndpoints)
// if change.previous equal to change.current, it means no change
if reflect.DeepEqual(change.previous, change.current) {
delete(ect.items, namespacedName)
}
return len(ect.items) > 0
}
// endpointsChange contains all changes to endpoints that happened since proxy rules were synced. For a single object,
// changes are accumulated, i.e. previous is state from before applying the changes,
// current is state after applying the changes.
type endpointsChange struct {
previous EndpointsMap
current EndpointsMap
}
// UpdateEndpointMapResult is the updated results after applying endpoints changes.
type UpdateEndpointMapResult struct {
// HCEndpointsLocalIPSize maps an endpoints name to the length of its local IPs.
HCEndpointsLocalIPSize map[types.NamespacedName]int
// StaleEndpoints identifies if an endpoints service pair is stale.
StaleEndpoints []ServiceEndpoint
// StaleServiceNames identifies if a service is stale.
StaleServiceNames []ServicePortName
}
// UpdateEndpointsMap updates endpointsMap base on the given changes.
func UpdateEndpointsMap(endpointsMap EndpointsMap, changes *EndpointChangeTracker) (result UpdateEndpointMapResult) {
result.StaleEndpoints = make([]ServiceEndpoint, 0)
result.StaleServiceNames = make([]ServicePortName, 0)
endpointsMap.apply(changes, &result.StaleEndpoints, &result.StaleServiceNames)
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to endpointsMap.
result.HCEndpointsLocalIPSize = make(map[types.NamespacedName]int)
localIPs := GetLocalEndpointIPs(endpointsMap)
for nsn, ips := range localIPs {
result.HCEndpointsLocalIPSize[nsn] = len(ips)
}
return result
}
// EndpointsMap maps a service to one of its endpoint.
type EndpointsMap map[ServicePortName][]Endpoint
// endpointsToEndpointsMap translates single Endpoints object to EndpointsMap.
// This function is used for incremental updated of endpointsMap.
//
// NOTE: endpoints object should NOT be modified.
func endpointsToEndpointsMap(endpoints *api.Endpoints, hostname string, makeEndpoints func(IP string, port int, isLocal bool) Endpoint) EndpointsMap {
if endpoints == nil {
return nil
}
endpointsMap := make(EndpointsMap)
// We need to build a map of portname -> all ip:ports for that
// portname. Explode Endpoints.Subsets[*] into this structure.
for i := range endpoints.Subsets {
ss := &endpoints.Subsets[i]
for i := range ss.Ports {
port := &ss.Ports[i]
if port.Port == 0 {
glog.Warningf("ignoring invalid endpoint port %s", port.Name)
continue
}
svcPortName := ServicePortName{
NamespacedName: types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name},
Port: port.Name,
}
for i := range ss.Addresses {
addr := &ss.Addresses[i]
if addr.IP == "" {
glog.Warningf("ignoring invalid endpoint port %s with empty host", port.Name)
continue
}
isLocal := addr.NodeName != nil && *addr.NodeName == hostname
epInfo := makeEndpoints(addr.IP, int(port.Port), isLocal)
endpointsMap[svcPortName] = append(endpointsMap[svcPortName], epInfo)
}
if glog.V(3) {
newEPList := []string{}
for _, ep := range endpointsMap[svcPortName] {
newEPList = append(newEPList, ep.String())
}
glog.Infof("Setting endpoints for %q to %+v", svcPortName, newEPList)
}
}
}
return endpointsMap
}
// apply the changes to EndpointsMap and updates stale endpoints and service-endpoints pair. The `staleEndpoints` argument
// is passed in to store the stale udp endpoints and `staleServiceNames` argument is passed in to store the stale udp service.
// The changes map is cleared after applying them.
func (endpointsMap EndpointsMap) apply(changes *EndpointChangeTracker, staleEndpoints *[]ServiceEndpoint, staleServiceNames *[]ServicePortName) {
if changes == nil {
return
}
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
endpointsMap.Unmerge(change.previous)
endpointsMap.Merge(change.current)
detectStaleConnections(change.previous, change.current, staleEndpoints, staleServiceNames)
}
changes.items = make(map[types.NamespacedName]*endpointsChange)
}
// Merge ensures that the current EndpointsMap contains all <service, endpoints> pairs from the EndpointsMap passed in.
func (em EndpointsMap) Merge(other EndpointsMap) {
for svcPortName := range other {
em[svcPortName] = other[svcPortName]
}
}
// Unmerge removes the <service, endpoints> pairs from the current EndpointsMap which are contained in the EndpointsMap passed in.
func (em EndpointsMap) Unmerge(other EndpointsMap) {
for svcPortName := range other {
delete(em, svcPortName)
}
}
// GetLocalEndpointIPs returns endpoints IPs if given endpoint is local - local means the endpoint is running in same host as kube-proxy.
func GetLocalEndpointIPs(endpointsMap EndpointsMap) map[types.NamespacedName]sets.String {
localIPs := make(map[types.NamespacedName]sets.String)
for svcPortName, epList := range endpointsMap {
for _, ep := range epList {
if ep.IsLocal() {
nsn := svcPortName.NamespacedName
if localIPs[nsn] == nil {
localIPs[nsn] = sets.NewString()
}
localIPs[nsn].Insert(ep.IP())
}
}
}
return localIPs
}
// detectStaleConnections modifies <staleEndpoints> and <staleServices> with detected stale connections. <staleServiceNames>
// is used to store stale udp service in order to clear udp conntrack later.
func detectStaleConnections(oldEndpointsMap, newEndpointsMap EndpointsMap, staleEndpoints *[]ServiceEndpoint, staleServiceNames *[]ServicePortName) {
for svcPortName, epList := range oldEndpointsMap {
for _, ep := range epList {
stale := true
for i := range newEndpointsMap[svcPortName] {
if newEndpointsMap[svcPortName][i].Equal(ep) {
stale = false
break
}
}
if stale {
glog.V(4).Infof("Stale endpoint %v -> %v", svcPortName, ep.String())
*staleEndpoints = append(*staleEndpoints, ServiceEndpoint{Endpoint: ep.String(), ServicePortName: svcPortName})
}
}
}
for svcPortName, epList := range newEndpointsMap {
// For udp service, if its backend changes from 0 to non-0. There may exist a conntrack entry that could blackhole traffic to the service.
if len(epList) > 0 && len(oldEndpointsMap[svcPortName]) == 0 {
*staleServiceNames = append(*staleServiceNames, svcPortName)
}
}
}

1247
pkg/proxy/endpoints_test.go Normal file

File diff suppressed because it is too large Load Diff

View File

@@ -27,7 +27,6 @@ go_library(
"//vendor/github.com/golang/glog:go_default_library", "//vendor/github.com/golang/glog:go_default_library",
"//vendor/k8s.io/api/core/v1:go_default_library", "//vendor/k8s.io/api/core/v1:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/types:go_default_library", "//vendor/k8s.io/apimachinery/pkg/types:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/sets:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/wait:go_default_library", "//vendor/k8s.io/apimachinery/pkg/util/wait:go_default_library",
"//vendor/k8s.io/client-go/tools/record:go_default_library", "//vendor/k8s.io/client-go/tools/record:go_default_library",
"//vendor/k8s.io/utils/exec:go_default_library", "//vendor/k8s.io/utils/exec:go_default_library",
@@ -46,12 +45,10 @@ go_test(
"//pkg/util/async:go_default_library", "//pkg/util/async:go_default_library",
"//pkg/util/iptables:go_default_library", "//pkg/util/iptables:go_default_library",
"//pkg/util/iptables/testing:go_default_library", "//pkg/util/iptables/testing:go_default_library",
"//vendor/github.com/davecgh/go-spew/spew:go_default_library",
"//vendor/github.com/golang/glog:go_default_library", "//vendor/github.com/golang/glog:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library", "//vendor/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/types:go_default_library", "//vendor/k8s.io/apimachinery/pkg/types:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/intstr:go_default_library", "//vendor/k8s.io/apimachinery/pkg/util/intstr:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/sets:go_default_library",
"//vendor/k8s.io/utils/exec:go_default_library", "//vendor/k8s.io/utils/exec:go_default_library",
"//vendor/k8s.io/utils/exec/testing:go_default_library", "//vendor/k8s.io/utils/exec/testing:go_default_library",
], ],

View File

@@ -26,7 +26,6 @@ import (
"encoding/base32" "encoding/base32"
"fmt" "fmt"
"net" "net"
"reflect"
"strconv" "strconv"
"strings" "strings"
"sync" "sync"
@@ -37,7 +36,6 @@ import (
"k8s.io/api/core/v1" "k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/apimachinery/pkg/util/wait" "k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/tools/record" "k8s.io/client-go/tools/record"
apiservice "k8s.io/kubernetes/pkg/api/service" apiservice "k8s.io/kubernetes/pkg/api/service"
@@ -157,37 +155,8 @@ type serviceInfo struct {
serviceLBChainName utiliptables.Chain serviceLBChainName utiliptables.Chain
} }
// internal struct for endpoints information // returns a new proxy.ServicePort which abstracts a serviceInfo
type endpointsInfo struct { func newServiceInfo(port *api.ServicePort, service *api.Service) proxy.ServicePort {
endpoint string // TODO: should be an endpointString type
isLocal bool
// The following fields we lazily compute and store here for performance
// reasons. If the protocol is the same as you expect it to be, then the
// chainName can be reused, otherwise it should be recomputed.
protocol string
chainName utiliptables.Chain
}
// IPPart returns just the IP part of the endpoint.
func (e *endpointsInfo) IPPart() string {
return utilproxy.IPPart(e.endpoint)
}
// Returns the endpoint chain name for a given endpointsInfo.
func (e *endpointsInfo) endpointChain(svcNameString, protocol string) utiliptables.Chain {
if e.protocol != protocol {
e.protocol = protocol
e.chainName = servicePortEndpointChainName(svcNameString, protocol, e.endpoint)
}
return e.chainName
}
func (e *endpointsInfo) String() string {
return fmt.Sprintf("%v", *e)
}
// returns a new serviceInfo struct
func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, service *api.Service) *serviceInfo {
onlyNodeLocalEndpoints := false onlyNodeLocalEndpoints := false
if apiservice.RequestsOnlyLocalTraffic(service) { if apiservice.RequestsOnlyLocalTraffic(service) {
onlyNodeLocalEndpoints = true onlyNodeLocalEndpoints = true
@@ -214,10 +183,13 @@ func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, se
copy(info.loadBalancerSourceRanges, service.Spec.LoadBalancerSourceRanges) copy(info.loadBalancerSourceRanges, service.Spec.LoadBalancerSourceRanges)
copy(info.externalIPs, service.Spec.ExternalIPs) copy(info.externalIPs, service.Spec.ExternalIPs)
svcName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name}
svcPortName := proxy.ServicePortName{NamespacedName: svcName, Port: port.Name}
if apiservice.NeedsHealthCheck(service) { if apiservice.NeedsHealthCheck(service) {
p := service.Spec.HealthCheckNodePort p := service.Spec.HealthCheckNodePort
if p == 0 { if p == 0 {
glog.Errorf("Service %q has no healthcheck nodeport", svcPortName.NamespacedName.String()) glog.Errorf("Service %q has no healthcheck nodeport", svcName.String())
} else { } else {
info.healthCheckNodePort = int(p) info.healthCheckNodePort = int(p)
} }
@@ -233,134 +205,90 @@ func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, se
return info return info
} }
type endpointsChange struct { // ClusterIP is part of proxy.ServicePort interface.
previous proxyEndpointsMap func (info *serviceInfo) ClusterIP() string {
current proxyEndpointsMap return info.clusterIP.String()
} }
type endpointsChangeMap struct { // Port is part of proxy.ServicePort interface.
lock sync.Mutex func (info *serviceInfo) Port() int {
hostname string return info.port
items map[types.NamespacedName]*endpointsChange
} }
type serviceChange struct { // Protocol is part of proxy.ServicePort interface.
previous proxyServiceMap func (info *serviceInfo) Protocol() api.Protocol {
current proxyServiceMap return info.protocol
} }
type serviceChangeMap struct { // String is part of proxy.ServicePort interface.
lock sync.Mutex func (info *serviceInfo) String() string {
items map[types.NamespacedName]*serviceChange return fmt.Sprintf("%s:%d/%s", info.clusterIP, info.port, info.protocol)
} }
type updateEndpointMapResult struct { // HealthCheckNodePort is part of proxy.ServicePort interface.
hcEndpoints map[types.NamespacedName]int func (info *serviceInfo) HealthCheckNodePort() int {
staleEndpoints map[endpointServicePair]bool return info.healthCheckNodePort
staleServiceNames map[proxy.ServicePortName]bool
} }
type updateServiceMapResult struct { var _ proxy.ServicePort = &serviceInfo{}
hcServices map[types.NamespacedName]uint16
staleServices sets.String // internal struct for endpoints information
type endpointsInfo struct {
endpoint string // TODO: should be an endpointString type
isLocal bool
// The following fields we lazily compute and store here for performance
// reasons. If the protocol is the same as you expect it to be, then the
// chainName can be reused, otherwise it should be recomputed.
protocol string
chainName utiliptables.Chain
} }
type proxyServiceMap map[proxy.ServicePortName]*serviceInfo // returns a new proxy.Endpoint which abstracts a endpointsInfo
type proxyEndpointsMap map[proxy.ServicePortName][]*endpointsInfo func newEndpointsInfo(IP string, port int, isLocal bool) proxy.Endpoint {
return &endpointsInfo{
func newEndpointsChangeMap(hostname string) endpointsChangeMap { endpoint: net.JoinHostPort(IP, strconv.Itoa(port)),
return endpointsChangeMap{ isLocal: isLocal,
hostname: hostname,
items: make(map[types.NamespacedName]*endpointsChange),
} }
} }
func (ecm *endpointsChangeMap) update(namespacedName *types.NamespacedName, previous, current *api.Endpoints) bool { // IsLocal is part of proxy.Endpoint interface.
ecm.lock.Lock() func (e *endpointsInfo) IsLocal() bool {
defer ecm.lock.Unlock() return e.isLocal
change, exists := ecm.items[*namespacedName]
if !exists {
change = &endpointsChange{}
change.previous = endpointsToEndpointsMap(previous, ecm.hostname)
ecm.items[*namespacedName] = change
}
change.current = endpointsToEndpointsMap(current, ecm.hostname)
if reflect.DeepEqual(change.previous, change.current) {
delete(ecm.items, *namespacedName)
}
return len(ecm.items) > 0
} }
func newServiceChangeMap() serviceChangeMap { // IP is part of proxy.Endpoint interface.
return serviceChangeMap{ func (e *endpointsInfo) IP() string {
items: make(map[types.NamespacedName]*serviceChange), return utilproxy.IPPart(e.endpoint)
}
} }
func (scm *serviceChangeMap) update(namespacedName *types.NamespacedName, previous, current *api.Service) bool { // Equal is part of proxy.Endpoint interface.
scm.lock.Lock() func (e *endpointsInfo) Equal(other proxy.Endpoint) bool {
defer scm.lock.Unlock() o, ok := other.(*endpointsInfo)
if !ok {
change, exists := scm.items[*namespacedName] glog.Errorf("Failed to cast endpointsInfo")
if !exists { return false
change = &serviceChange{}
change.previous = serviceToServiceMap(previous)
scm.items[*namespacedName] = change
} }
change.current = serviceToServiceMap(current) return e.endpoint == o.endpoint &&
if reflect.DeepEqual(change.previous, change.current) { e.isLocal == o.isLocal &&
delete(scm.items, *namespacedName) e.protocol == o.protocol &&
} e.chainName == o.chainName
return len(scm.items) > 0
} }
func (sm *proxyServiceMap) merge(other proxyServiceMap) sets.String { // String is part of proxy.Endpoint interface.
existingPorts := sets.NewString() func (e *endpointsInfo) String() string {
for svcPortName, info := range other { return e.endpoint
port := strconv.Itoa(info.port)
clusterIPPort := net.JoinHostPort(info.clusterIP.String(), port)
existingPorts.Insert(svcPortName.Port)
_, exists := (*sm)[svcPortName]
if !exists {
glog.V(1).Infof("Adding new service port %q at %s/%s", svcPortName, clusterIPPort, info.protocol)
} else {
glog.V(1).Infof("Updating existing service port %q at %s/%s", svcPortName, clusterIPPort, info.protocol)
}
(*sm)[svcPortName] = info
}
return existingPorts
} }
func (sm *proxyServiceMap) unmerge(other proxyServiceMap, existingPorts, staleServices sets.String) { // Returns the endpoint chain name for a given endpointsInfo.
for svcPortName := range other { func (e *endpointsInfo) endpointChain(svcNameString, protocol string) utiliptables.Chain {
if existingPorts.Has(svcPortName.Port) { if e.protocol != protocol {
continue e.protocol = protocol
} e.chainName = servicePortEndpointChainName(svcNameString, protocol, e.endpoint)
info, exists := (*sm)[svcPortName]
if exists {
glog.V(1).Infof("Removing service port %q", svcPortName)
if info.protocol == api.ProtocolUDP {
staleServices.Insert(info.clusterIP.String())
}
delete(*sm, svcPortName)
} else {
glog.Errorf("Service port %q removed, but doesn't exists", svcPortName)
}
} }
return e.chainName
} }
func (em proxyEndpointsMap) merge(other proxyEndpointsMap) { var _ proxy.Endpoint = &endpointsInfo{}
for svcPortName := range other {
em[svcPortName] = other[svcPortName]
}
}
func (em proxyEndpointsMap) unmerge(other proxyEndpointsMap) {
for svcPortName := range other {
delete(em, svcPortName)
}
}
// Proxier is an iptables based proxy for connections between a localhost:lport // Proxier is an iptables based proxy for connections between a localhost:lport
// and services that provide the actual backends. // and services that provide the actual backends.
@@ -369,12 +297,12 @@ type Proxier struct {
// services that happened since iptables was synced. For a single object, // services that happened since iptables was synced. For a single object,
// changes are accumulated, i.e. previous is state from before all of them, // changes are accumulated, i.e. previous is state from before all of them,
// current is state after applying all of those. // current is state after applying all of those.
endpointsChanges endpointsChangeMap endpointsChanges *proxy.EndpointChangeTracker
serviceChanges serviceChangeMap serviceChanges *proxy.ServiceChangeTracker
mu sync.Mutex // protects the following fields mu sync.Mutex // protects the following fields
serviceMap proxyServiceMap serviceMap proxy.ServiceMap
endpointsMap proxyEndpointsMap endpointsMap proxy.EndpointsMap
portsMap map[utilproxy.LocalPort]utilproxy.Closeable portsMap map[utilproxy.LocalPort]utilproxy.Closeable
// endpointsSynced and servicesSynced are set to true when corresponding // endpointsSynced and servicesSynced are set to true when corresponding
// objects are synced after startup. This is used to avoid updating iptables // objects are synced after startup. This is used to avoid updating iptables
@@ -469,10 +397,10 @@ func NewProxier(ipt utiliptables.Interface,
proxier := &Proxier{ proxier := &Proxier{
portsMap: make(map[utilproxy.LocalPort]utilproxy.Closeable), portsMap: make(map[utilproxy.LocalPort]utilproxy.Closeable),
serviceMap: make(proxyServiceMap), serviceMap: make(proxy.ServiceMap),
serviceChanges: newServiceChangeMap(), serviceChanges: proxy.NewServiceChangeTracker(),
endpointsMap: make(proxyEndpointsMap), endpointsMap: make(proxy.EndpointsMap),
endpointsChanges: newEndpointsChangeMap(hostname), endpointsChanges: proxy.NewEndpointChangeTracker(hostname),
iptables: ipt, iptables: ipt,
masqueradeAll: masqueradeAll, masqueradeAll: masqueradeAll,
masqueradeMark: masqueradeMark, masqueradeMark: masqueradeMark,
@@ -643,22 +571,19 @@ func (proxier *Proxier) isInitialized() bool {
} }
func (proxier *Proxier) OnServiceAdd(service *api.Service) { func (proxier *Proxier) OnServiceAdd(service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(nil, service, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, nil, service) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
func (proxier *Proxier) OnServiceUpdate(oldService, service *api.Service) { func (proxier *Proxier) OnServiceUpdate(oldService, service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(oldService, service, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, oldService, service) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
func (proxier *Proxier) OnServiceDelete(service *api.Service) { func (proxier *Proxier) OnServiceDelete(service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(service, nil, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, service, nil) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
@@ -673,52 +598,20 @@ func (proxier *Proxier) OnServiceSynced() {
proxier.syncProxyRules() proxier.syncProxyRules()
} }
// <serviceMap> is updated by this function (based on the given changes).
// <changes> map is cleared after applying them.
func updateServiceMap(
serviceMap proxyServiceMap,
changes *serviceChangeMap) (result updateServiceMapResult) {
result.staleServices = sets.NewString()
func() {
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
existingPorts := serviceMap.merge(change.current)
serviceMap.unmerge(change.previous, existingPorts, result.staleServices)
}
changes.items = make(map[types.NamespacedName]*serviceChange)
}()
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to serviceMap.
result.hcServices = make(map[types.NamespacedName]uint16)
for svcPortName, info := range serviceMap {
if info.healthCheckNodePort != 0 {
result.hcServices[svcPortName.NamespacedName] = uint16(info.healthCheckNodePort)
}
}
return result
}
func (proxier *Proxier) OnEndpointsAdd(endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsAdd(endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(nil, endpoints, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, nil, endpoints) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
func (proxier *Proxier) OnEndpointsUpdate(oldEndpoints, endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsUpdate(oldEndpoints, endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(oldEndpoints, endpoints, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, oldEndpoints, endpoints) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
func (proxier *Proxier) OnEndpointsDelete(endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsDelete(endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(endpoints, nil, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, endpoints, nil) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
@@ -733,152 +626,6 @@ func (proxier *Proxier) OnEndpointsSynced() {
proxier.syncProxyRules() proxier.syncProxyRules()
} }
// <endpointsMap> is updated by this function (based on the given changes).
// <changes> map is cleared after applying them.
func updateEndpointsMap(
endpointsMap proxyEndpointsMap,
changes *endpointsChangeMap,
hostname string) (result updateEndpointMapResult) {
result.staleEndpoints = make(map[endpointServicePair]bool)
result.staleServiceNames = make(map[proxy.ServicePortName]bool)
func() {
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
endpointsMap.unmerge(change.previous)
endpointsMap.merge(change.current)
detectStaleConnections(change.previous, change.current, result.staleEndpoints, result.staleServiceNames)
}
changes.items = make(map[types.NamespacedName]*endpointsChange)
}()
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to endpointsMap.
result.hcEndpoints = make(map[types.NamespacedName]int)
localIPs := getLocalIPs(endpointsMap)
for nsn, ips := range localIPs {
result.hcEndpoints[nsn] = len(ips)
}
return result
}
// <staleEndpoints> and <staleServices> are modified by this function with detected stale connections.
func detectStaleConnections(oldEndpointsMap, newEndpointsMap proxyEndpointsMap, staleEndpoints map[endpointServicePair]bool, staleServiceNames map[proxy.ServicePortName]bool) {
for svcPortName, epList := range oldEndpointsMap {
for _, ep := range epList {
stale := true
for i := range newEndpointsMap[svcPortName] {
if *newEndpointsMap[svcPortName][i] == *ep {
stale = false
break
}
}
if stale {
glog.V(4).Infof("Stale endpoint %v -> %v", svcPortName, ep.endpoint)
staleEndpoints[endpointServicePair{endpoint: ep.endpoint, servicePortName: svcPortName}] = true
}
}
}
for svcPortName, epList := range newEndpointsMap {
// For udp service, if its backend changes from 0 to non-0. There may exist a conntrack entry that could blackhole traffic to the service.
if len(epList) > 0 && len(oldEndpointsMap[svcPortName]) == 0 {
staleServiceNames[svcPortName] = true
}
}
}
func getLocalIPs(endpointsMap proxyEndpointsMap) map[types.NamespacedName]sets.String {
localIPs := make(map[types.NamespacedName]sets.String)
for svcPortName := range endpointsMap {
for _, ep := range endpointsMap[svcPortName] {
if ep.isLocal {
// If the endpoint has a bad format, utilproxy.IPPart() will log an
// error and ep.IPPart() will return a null string.
if ip := ep.IPPart(); ip != "" {
nsn := svcPortName.NamespacedName
if localIPs[nsn] == nil {
localIPs[nsn] = sets.NewString()
}
localIPs[nsn].Insert(ip)
}
}
}
}
return localIPs
}
// Translates single Endpoints object to proxyEndpointsMap.
// This function is used for incremental updated of endpointsMap.
//
// NOTE: endpoints object should NOT be modified.
func endpointsToEndpointsMap(endpoints *api.Endpoints, hostname string) proxyEndpointsMap {
if endpoints == nil {
return nil
}
endpointsMap := make(proxyEndpointsMap)
// We need to build a map of portname -> all ip:ports for that
// portname. Explode Endpoints.Subsets[*] into this structure.
for i := range endpoints.Subsets {
ss := &endpoints.Subsets[i]
for i := range ss.Ports {
port := &ss.Ports[i]
if port.Port == 0 {
glog.Warningf("ignoring invalid endpoint port %s", port.Name)
continue
}
svcPortName := proxy.ServicePortName{
NamespacedName: types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name},
Port: port.Name,
}
for i := range ss.Addresses {
addr := &ss.Addresses[i]
if addr.IP == "" {
glog.Warningf("ignoring invalid endpoint port %s with empty host", port.Name)
continue
}
epInfo := &endpointsInfo{
endpoint: net.JoinHostPort(addr.IP, strconv.Itoa(int(port.Port))),
isLocal: addr.NodeName != nil && *addr.NodeName == hostname,
}
endpointsMap[svcPortName] = append(endpointsMap[svcPortName], epInfo)
}
if glog.V(3) {
newEPList := []string{}
for _, ep := range endpointsMap[svcPortName] {
newEPList = append(newEPList, ep.endpoint)
}
glog.Infof("Setting endpoints for %q to %+v", svcPortName, newEPList)
}
}
}
return endpointsMap
}
// Translates single Service object to proxyServiceMap.
//
// NOTE: service object should NOT be modified.
func serviceToServiceMap(service *api.Service) proxyServiceMap {
if service == nil {
return nil
}
svcName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name}
if utilproxy.ShouldSkipService(svcName, service) {
return nil
}
serviceMap := make(proxyServiceMap)
for i := range service.Spec.Ports {
servicePort := &service.Spec.Ports[i]
svcPortName := proxy.ServicePortName{NamespacedName: svcName, Port: servicePort.Name}
serviceMap[svcPortName] = newServiceInfo(svcPortName, servicePort, service)
}
return serviceMap
}
// portProtoHash takes the ServicePortName and protocol for a service // portProtoHash takes the ServicePortName and protocol for a service
// returns the associated 16 character hash. This is computed by hashing (sha256) // returns the associated 16 character hash. This is computed by hashing (sha256)
// then encoding to base32 and truncating to 16 chars. We do this because IPTables // then encoding to base32 and truncating to 16 chars. We do this because IPTables
@@ -919,25 +666,17 @@ func servicePortEndpointChainName(servicePortName string, protocol string, endpo
return utiliptables.Chain("KUBE-SEP-" + encoded[:16]) return utiliptables.Chain("KUBE-SEP-" + encoded[:16])
} }
type endpointServicePair struct {
endpoint string
servicePortName proxy.ServicePortName
}
func (esp *endpointServicePair) IPPart() string {
return utilproxy.IPPart(esp.endpoint)
}
// After a UDP endpoint has been removed, we must flush any pending conntrack entries to it, or else we // After a UDP endpoint has been removed, we must flush any pending conntrack entries to it, or else we
// risk sending more traffic to it, all of which will be lost (because UDP). // risk sending more traffic to it, all of which will be lost (because UDP).
// This assumes the proxier mutex is held // This assumes the proxier mutex is held
func (proxier *Proxier) deleteEndpointConnections(connectionMap map[endpointServicePair]bool) { // TODO: move it to util
for epSvcPair := range connectionMap { func (proxier *Proxier) deleteEndpointConnections(connectionMap []proxy.ServiceEndpoint) {
if svcInfo, ok := proxier.serviceMap[epSvcPair.servicePortName]; ok && svcInfo.protocol == api.ProtocolUDP { for _, epSvcPair := range connectionMap {
endpointIP := utilproxy.IPPart(epSvcPair.endpoint) if svcInfo, ok := proxier.serviceMap[epSvcPair.ServicePortName]; ok && svcInfo.Protocol() == api.ProtocolUDP {
err := utilproxy.ClearUDPConntrackForPeers(proxier.exec, svcInfo.clusterIP.String(), endpointIP) endpointIP := utilproxy.IPPart(epSvcPair.Endpoint)
err := utilproxy.ClearUDPConntrackForPeers(proxier.exec, svcInfo.ClusterIP(), endpointIP)
if err != nil { if err != nil {
glog.Errorf("Failed to delete %s endpoint connections, error: %v", epSvcPair.servicePortName.String(), err) glog.Errorf("Failed to delete %s endpoint connections, error: %v", epSvcPair.ServicePortName.String(), err)
} }
} }
} }
@@ -964,17 +703,15 @@ func (proxier *Proxier) syncProxyRules() {
// We assume that if this was called, we really want to sync them, // We assume that if this was called, we really want to sync them,
// even if nothing changed in the meantime. In other words, callers are // even if nothing changed in the meantime. In other words, callers are
// responsible for detecting no-op changes and not calling this function. // responsible for detecting no-op changes and not calling this function.
serviceUpdateResult := updateServiceMap( serviceUpdateResult := proxy.UpdateServiceMap(proxier.serviceMap, proxier.serviceChanges)
proxier.serviceMap, &proxier.serviceChanges) endpointUpdateResult := proxy.UpdateEndpointsMap(proxier.endpointsMap, proxier.endpointsChanges)
endpointUpdateResult := updateEndpointsMap(
proxier.endpointsMap, &proxier.endpointsChanges, proxier.hostname)
staleServices := serviceUpdateResult.staleServices staleServices := serviceUpdateResult.UDPStaleClusterIP
// merge stale services gathered from updateEndpointsMap // merge stale services gathered from updateEndpointsMap
for svcPortName := range endpointUpdateResult.staleServiceNames { for _, svcPortName := range endpointUpdateResult.StaleServiceNames {
if svcInfo, ok := proxier.serviceMap[svcPortName]; ok && svcInfo != nil && svcInfo.protocol == api.ProtocolUDP { if svcInfo, ok := proxier.serviceMap[svcPortName]; ok && svcInfo != nil && svcInfo.Protocol() == api.ProtocolUDP {
glog.V(2).Infof("Stale udp service %v -> %s", svcPortName, svcInfo.clusterIP.String()) glog.V(2).Infof("Stale udp service %v -> %s", svcPortName, svcInfo.ClusterIP())
staleServices.Insert(svcInfo.clusterIP.String()) staleServices.Insert(svcInfo.ClusterIP())
} }
} }
@@ -1088,7 +825,12 @@ func (proxier *Proxier) syncProxyRules() {
// Build rules for each service. // Build rules for each service.
var svcNameString string var svcNameString string
for svcName, svcInfo := range proxier.serviceMap { for svcName, svc := range proxier.serviceMap {
svcInfo, ok := svc.(*serviceInfo)
if !ok {
glog.Errorf("Failed to cast serviceInfo %q", svcName.String())
continue
}
isIPv6 := utilproxy.IsIPv6(svcInfo.clusterIP) isIPv6 := utilproxy.IsIPv6(svcInfo.clusterIP)
protocol := strings.ToLower(string(svcInfo.protocol)) protocol := strings.ToLower(string(svcInfo.protocol))
svcNameString = svcInfo.serviceNameString svcNameString = svcInfo.serviceNameString
@@ -1148,7 +890,7 @@ func (proxier *Proxier) syncProxyRules() {
lp := utilproxy.LocalPort{ lp := utilproxy.LocalPort{
Description: "externalIP for " + svcNameString, Description: "externalIP for " + svcNameString,
IP: externalIP, IP: externalIP,
Port: svcInfo.port, Port: svcInfo.Port(),
Protocol: protocol, Protocol: protocol,
} }
if proxier.portsMap[lp] != nil { if proxier.portsMap[lp] != nil {
@@ -1372,8 +1114,13 @@ func (proxier *Proxier) syncProxyRules() {
endpointChains = endpointChains[:0] endpointChains = endpointChains[:0]
var endpointChain utiliptables.Chain var endpointChain utiliptables.Chain
for _, ep := range proxier.endpointsMap[svcName] { for _, ep := range proxier.endpointsMap[svcName] {
endpoints = append(endpoints, ep) epInfo, ok := ep.(*endpointsInfo)
endpointChain = ep.endpointChain(svcNameString, protocol) if !ok {
glog.Errorf("Failed to cast endpointsInfo %q", ep.String())
continue
}
endpoints = append(endpoints, epInfo)
endpointChain = epInfo.endpointChain(svcNameString, protocol)
endpointChains = append(endpointChains, endpointChain) endpointChains = append(endpointChains, endpointChain)
// Create the endpoint chain, retaining counters if possible. // Create the endpoint chain, retaining counters if possible.
@@ -1400,7 +1147,7 @@ func (proxier *Proxier) syncProxyRules() {
// Now write loadbalancing & DNAT rules. // Now write loadbalancing & DNAT rules.
n := len(endpointChains) n := len(endpointChains)
for i, endpointChain := range endpointChains { for i, endpointChain := range endpointChains {
epIP := endpoints[i].IPPart() epIP := endpoints[i].IP()
if epIP == "" { if epIP == "" {
// Error parsing this endpoint has been logged. Skip to next endpoint. // Error parsing this endpoint has been logged. Skip to next endpoint.
continue continue
@@ -1611,10 +1358,10 @@ func (proxier *Proxier) syncProxyRules() {
// Update healthchecks. The endpoints list might include services that are // Update healthchecks. The endpoints list might include services that are
// not "OnlyLocal", but the services list will not, and the healthChecker // not "OnlyLocal", but the services list will not, and the healthChecker
// will just drop those endpoints. // will just drop those endpoints.
if err := proxier.healthChecker.SyncServices(serviceUpdateResult.hcServices); err != nil { if err := proxier.healthChecker.SyncServices(serviceUpdateResult.HCServiceNodePorts); err != nil {
glog.Errorf("Error syncing healtcheck services: %v", err) glog.Errorf("Error syncing healtcheck services: %v", err)
} }
if err := proxier.healthChecker.SyncEndpoints(endpointUpdateResult.hcEndpoints); err != nil { if err := proxier.healthChecker.SyncEndpoints(endpointUpdateResult.HCEndpointsLocalIPSize); err != nil {
glog.Errorf("Error syncing healthcheck endoints: %v", err) glog.Errorf("Error syncing healthcheck endoints: %v", err)
} }
@@ -1625,7 +1372,7 @@ func (proxier *Proxier) syncProxyRules() {
glog.Errorf("Failed to delete stale service IP %s connections, error: %v", svcIP, err) glog.Errorf("Failed to delete stale service IP %s connections, error: %v", svcIP, err)
} }
} }
proxier.deleteEndpointConnections(endpointUpdateResult.staleEndpoints) proxier.deleteEndpointConnections(endpointUpdateResult.StaleEndpoints)
} }
// Join all words with spaces, terminate with newline and write to buf. // Join all words with spaces, terminate with newline and write to buf.

View File

@@ -18,22 +18,19 @@ package iptables
import ( import (
"bytes" "bytes"
"fmt"
"net"
"reflect" "reflect"
"strconv" "strconv"
"strings"
"testing" "testing"
"time" "time"
"github.com/davecgh/go-spew/spew"
"github.com/golang/glog" "github.com/golang/glog"
"fmt"
"net"
"strings"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/intstr" "k8s.io/apimachinery/pkg/util/intstr"
"k8s.io/apimachinery/pkg/util/sets"
api "k8s.io/kubernetes/pkg/apis/core" api "k8s.io/kubernetes/pkg/apis/core"
"k8s.io/kubernetes/pkg/proxy" "k8s.io/kubernetes/pkg/proxy"
utilproxy "k8s.io/kubernetes/pkg/proxy/util" utilproxy "k8s.io/kubernetes/pkg/proxy/util"
@@ -201,7 +198,7 @@ func TestDeleteEndpointConnections(t *testing.T) {
svcPort int svcPort int
protocol api.Protocol protocol api.Protocol
endpoint string // IP:port endpoint endpoint string // IP:port endpoint
epSvcPair endpointServicePair // Will be generated by test epSvcPair proxy.ServiceEndpoint // Will be generated by test
simulatedErr string simulatedErr string
}{ }{
{ {
@@ -253,16 +250,16 @@ func TestDeleteEndpointConnections(t *testing.T) {
// Create a service map that has service info entries for all test cases // Create a service map that has service info entries for all test cases
// and generate an endpoint service pair for each test case // and generate an endpoint service pair for each test case
serviceMap := make(map[proxy.ServicePortName]*serviceInfo) serviceMap := make(map[proxy.ServicePortName]proxy.ServicePort)
for i, tc := range testCases { for i, tc := range testCases {
svc := proxy.ServicePortName{ svc := proxy.ServicePortName{
NamespacedName: types.NamespacedName{Namespace: "ns1", Name: tc.svcName}, NamespacedName: types.NamespacedName{Namespace: "ns1", Name: tc.svcName},
Port: "p80", Port: "p80",
} }
serviceMap[svc] = newFakeServiceInfo(svc, net.ParseIP(tc.svcIP), 80, tc.protocol, false) serviceMap[svc] = newFakeServiceInfo(svc, net.ParseIP(tc.svcIP), 80, tc.protocol, false)
testCases[i].epSvcPair = endpointServicePair{ testCases[i].epSvcPair = proxy.ServiceEndpoint{
endpoint: tc.endpoint, Endpoint: tc.endpoint,
servicePortName: svc, ServicePortName: svc,
} }
} }
@@ -298,7 +295,7 @@ func TestDeleteEndpointConnections(t *testing.T) {
priorExecs := fexec.CommandCalls priorExecs := fexec.CommandCalls
priorGlogErrs := glog.Stats.Error.Lines() priorGlogErrs := glog.Stats.Error.Lines()
input := map[endpointServicePair]bool{tc.epSvcPair: true} input := []proxy.ServiceEndpoint{tc.epSvcPair}
fakeProxier.deleteEndpointConnections(input) fakeProxier.deleteEndpointConnections(input)
// For UDP connections, check the executed conntrack command // For UDP connections, check the executed conntrack command
@@ -391,10 +388,10 @@ func NewFakeProxier(ipt utiliptables.Interface) *Proxier {
// invocation into a Run() method. // invocation into a Run() method.
p := &Proxier{ p := &Proxier{
exec: &fakeexec.FakeExec{}, exec: &fakeexec.FakeExec{},
serviceMap: make(proxyServiceMap), serviceMap: make(proxy.ServiceMap),
serviceChanges: newServiceChangeMap(), serviceChanges: proxy.NewServiceChangeTracker(),
endpointsMap: make(proxyEndpointsMap), endpointsMap: make(proxy.EndpointsMap),
endpointsChanges: newEndpointsChangeMap(testHostname), endpointsChanges: proxy.NewEndpointChangeTracker(testHostname),
iptables: ipt, iptables: ipt,
clusterCIDR: "10.0.0.0/24", clusterCIDR: "10.0.0.0/24",
hostname: testHostname, hostname: testHostname,
@@ -720,7 +717,6 @@ func TestLoadBalancer(t *testing.T) {
proto := strings.ToLower(string(api.ProtocolTCP)) proto := strings.ToLower(string(api.ProtocolTCP))
fwChain := string(serviceFirewallChainName(svcPortName.String(), proto)) fwChain := string(serviceFirewallChainName(svcPortName.String(), proto))
svcChain := string(servicePortChainName(svcPortName.String(), proto)) svcChain := string(servicePortChainName(svcPortName.String(), proto))
//lbChain := string(serviceLBChainName(svcPortName.String(), proto))
kubeSvcRules := ipt.GetRules(string(kubeServicesChain)) kubeSvcRules := ipt.GetRules(string(kubeServicesChain))
if !hasJump(kubeSvcRules, fwChain, svcLBIP, svcPort) { if !hasJump(kubeSvcRules, fwChain, svcLBIP, svcPort) {
@@ -1111,24 +1107,24 @@ func TestBuildServiceMapAddRemove(t *testing.T) {
for i := range services { for i := range services {
fp.OnServiceAdd(services[i]) fp.OnServiceAdd(services[i])
} }
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 8 { if len(fp.serviceMap) != 8 {
t.Errorf("expected service map length 8, got %v", fp.serviceMap) t.Errorf("expected service map length 8, got %v", fp.serviceMap)
} }
// The only-local-loadbalancer ones get added // The only-local-loadbalancer ones get added
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected 1 healthcheck port, got %v", result.hcServices) t.Errorf("expected 1 healthcheck port, got %v", result.HCServiceNodePorts)
} else { } else {
nsn := makeNSN("somewhere", "only-local-load-balancer") nsn := makeNSN("somewhere", "only-local-load-balancer")
if port, found := result.hcServices[nsn]; !found || port != 345 { if port, found := result.HCServiceNodePorts[nsn]; !found || port != 345 {
t.Errorf("expected healthcheck port [%q]=345: got %v", nsn, result.hcServices) t.Errorf("expected healthcheck port [%q]=345: got %v", nsn, result.HCServiceNodePorts)
} }
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
// Remove some stuff // Remove some stuff
@@ -1144,24 +1140,24 @@ func TestBuildServiceMapAddRemove(t *testing.T) {
fp.OnServiceDelete(services[2]) fp.OnServiceDelete(services[2])
fp.OnServiceDelete(services[3]) fp.OnServiceDelete(services[3])
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 1 { if len(fp.serviceMap) != 1 {
t.Errorf("expected service map length 1, got %v", fp.serviceMap) t.Errorf("expected service map length 1, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected 0 healthcheck ports, got %v", result.hcServices) t.Errorf("expected 0 healthcheck ports, got %v", result.HCServiceNodePorts)
} }
// All services but one were deleted. While you'd expect only the ClusterIPs // All services but one were deleted. While you'd expect only the ClusterIPs
// from the three deleted services here, we still have the ClusterIP for // from the three deleted services here, we still have the ClusterIP for
// the not-deleted service, because one of it's ServicePorts was deleted. // the not-deleted service, because one of it's ServicePorts was deleted.
expectedStaleUDPServices := []string{"172.16.55.10", "172.16.55.4", "172.16.55.11", "172.16.55.12"} expectedStaleUDPServices := []string{"172.16.55.10", "172.16.55.4", "172.16.55.11", "172.16.55.12"}
if len(result.staleServices) != len(expectedStaleUDPServices) { if len(result.UDPStaleClusterIP) != len(expectedStaleUDPServices) {
t.Errorf("expected stale UDP services length %d, got %v", len(expectedStaleUDPServices), result.staleServices.UnsortedList()) t.Errorf("expected stale UDP services length %d, got %v", len(expectedStaleUDPServices), result.UDPStaleClusterIP.UnsortedList())
} }
for _, ip := range expectedStaleUDPServices { for _, ip := range expectedStaleUDPServices {
if !result.staleServices.Has(ip) { if !result.UDPStaleClusterIP.Has(ip) {
t.Errorf("expected stale UDP service service %s", ip) t.Errorf("expected stale UDP service service %s", ip)
} }
} }
@@ -1184,18 +1180,18 @@ func TestBuildServiceMapServiceHeadless(t *testing.T) {
) )
// Headless service should be ignored // Headless service should be ignored
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 { if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %d", len(fp.serviceMap)) t.Errorf("expected service map length 0, got %d", len(fp.serviceMap))
} }
// No proxied services, so no healthchecks // No proxied services, so no healthchecks
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %d", len(result.hcServices)) t.Errorf("expected healthcheck ports length 0, got %d", len(result.HCServiceNodePorts))
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
} }
@@ -1212,16 +1208,16 @@ func TestBuildServiceMapServiceTypeExternalName(t *testing.T) {
}), }),
) )
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 { if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %v", fp.serviceMap) t.Errorf("expected service map length 0, got %v", fp.serviceMap)
} }
// No proxied services, so no healthchecks // No proxied services, so no healthchecks
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP)
} }
} }
@@ -1252,328 +1248,57 @@ func TestBuildServiceMapServiceUpdate(t *testing.T) {
fp.OnServiceAdd(servicev1) fp.OnServiceAdd(servicev1)
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
// Change service to load-balancer // Change service to load-balancer
fp.OnServiceUpdate(servicev1, servicev2) fp.OnServiceUpdate(servicev1, servicev2)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices.UnsortedList()) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.UnsortedList())
} }
// No change; make sure the service map stays the same and there are // No change; make sure the service map stays the same and there are
// no health-check changes // no health-check changes
fp.OnServiceUpdate(servicev2, servicev2) fp.OnServiceUpdate(servicev2, servicev2)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices.UnsortedList()) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.UnsortedList())
} }
// And back to ClusterIP // And back to ClusterIP
fp.OnServiceUpdate(servicev2, servicev1) fp.OnServiceUpdate(servicev2, servicev1)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
}
}
func Test_getLocalIPs(t *testing.T) {
testCases := []struct {
endpointsMap map[proxy.ServicePortName][]*endpointsInfo
expected map[types.NamespacedName]sets.String
}{{
// Case[0]: nothing
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{},
expected: map[types.NamespacedName]sets.String{},
}, {
// Case[1]: unnamed port
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
expected: map[types.NamespacedName]sets.String{},
}, {
// Case[2]: unnamed port local
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{endpoint: "1.1.1.1:11", isLocal: true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns1", Name: "ep1"}: sets.NewString("1.1.1.1"),
},
}, {
// Case[3]: named local and non-local ports for the same IP.
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{endpoint: "1.1.1.1:11", isLocal: false},
{endpoint: "1.1.1.2:11", isLocal: true},
},
makeServicePortName("ns1", "ep1", "p12"): {
{endpoint: "1.1.1.1:12", isLocal: false},
{endpoint: "1.1.1.2:12", isLocal: true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns1", Name: "ep1"}: sets.NewString("1.1.1.2"),
},
}, {
// Case[4]: named local and non-local ports for different IPs.
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
makeServicePortName("ns2", "ep2", "p22"): {
{endpoint: "2.2.2.2:22", isLocal: true},
{endpoint: "2.2.2.22:22", isLocal: true},
},
makeServicePortName("ns2", "ep2", "p23"): {
{endpoint: "2.2.2.3:23", isLocal: true},
},
makeServicePortName("ns4", "ep4", "p44"): {
{endpoint: "4.4.4.4:44", isLocal: true},
{endpoint: "4.4.4.5:44", isLocal: false},
},
makeServicePortName("ns4", "ep4", "p45"): {
{endpoint: "4.4.4.6:45", isLocal: true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns2", Name: "ep2"}: sets.NewString("2.2.2.2", "2.2.2.22", "2.2.2.3"),
{Namespace: "ns4", Name: "ep4"}: sets.NewString("4.4.4.4", "4.4.4.6"),
},
}, {
// Case[5]: named port local and bad endpoints IP
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{endpoint: "bad ip:11", isLocal: true},
},
},
expected: map[types.NamespacedName]sets.String{},
}}
for tci, tc := range testCases {
// outputs
localIPs := getLocalIPs(tc.endpointsMap)
if !reflect.DeepEqual(localIPs, tc.expected) {
t.Errorf("[%d] expected %#v, got %#v", tci, tc.expected, localIPs)
}
}
}
// This is a coarse test, but it offers some modicum of confidence as the code is evolved.
func Test_endpointsToEndpointsMap(t *testing.T) {
testCases := []struct {
newEndpoints *api.Endpoints
expected map[proxy.ServicePortName][]*endpointsInfo
}{{
// Case[0]: nothing
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {}),
expected: map[proxy.ServicePortName][]*endpointsInfo{},
}, {
// Case[1]: no changes, unnamed port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
}, {
// Case[2]: no changes, named port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "port",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "port"): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
}, {
// Case[3]: new port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
}, {
// Case[4]: remove port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {}),
expected: map[proxy.ServicePortName][]*endpointsInfo{},
}, {
// Case[5]: new IP and port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}, {
IP: "2.2.2.2",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 11,
}, {
Name: "p2",
Port: 22,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{endpoint: "1.1.1.1:11", isLocal: false},
{endpoint: "2.2.2.2:11", isLocal: false},
},
makeServicePortName("ns1", "ep1", "p2"): {
{endpoint: "1.1.1.1:22", isLocal: false},
{endpoint: "2.2.2.2:22", isLocal: false},
},
},
}, {
// Case[6]: remove IP and port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
}, {
// Case[7]: rename port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p2",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p2"): {
{endpoint: "1.1.1.1:11", isLocal: false},
},
},
}, {
// Case[8]: renumber port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 22,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{endpoint: "1.1.1.1:22", isLocal: false},
},
},
}}
for tci, tc := range testCases {
// outputs
newEndpoints := endpointsToEndpointsMap(tc.newEndpoints, "host")
if len(newEndpoints) != len(tc.expected) {
t.Errorf("[%d] expected %d new, got %d: %v", tci, len(tc.expected), len(newEndpoints), spew.Sdump(newEndpoints))
}
for x := range tc.expected {
if len(newEndpoints[x]) != len(tc.expected[x]) {
t.Errorf("[%d] expected %d endpoints for %v, got %d", tci, len(tc.expected[x]), x, len(newEndpoints[x]))
} else {
for i := range newEndpoints[x] {
if *(newEndpoints[x][i]) != *(tc.expected[x][i]) {
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, tc.expected[x][i], *(newEndpoints[x][i]))
}
}
}
}
} }
} }
@@ -1619,7 +1344,7 @@ func makeServiceMap(proxier *Proxier, allServices ...*api.Service) {
proxier.servicesSynced = true proxier.servicesSynced = true
} }
func compareEndpointsMaps(t *testing.T, tci int, newMap, expected map[proxy.ServicePortName][]*endpointsInfo) { func compareEndpointsMaps(t *testing.T, tci int, newMap proxy.EndpointsMap, expected map[proxy.ServicePortName][]*endpointsInfo) {
if len(newMap) != len(expected) { if len(newMap) != len(expected) {
t.Errorf("[%d] expected %d results, got %d: %v", tci, len(expected), len(newMap), newMap) t.Errorf("[%d] expected %d results, got %d: %v", tci, len(expected), len(newMap), newMap)
} }
@@ -1628,8 +1353,13 @@ func compareEndpointsMaps(t *testing.T, tci int, newMap, expected map[proxy.Serv
t.Errorf("[%d] expected %d endpoints for %v, got %d", tci, len(expected[x]), x, len(newMap[x])) t.Errorf("[%d] expected %d endpoints for %v, got %d", tci, len(expected[x]), x, len(newMap[x]))
} else { } else {
for i := range expected[x] { for i := range expected[x] {
if *(newMap[x][i]) != *(expected[x][i]) { newEp, ok := newMap[x][i].(*endpointsInfo)
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, expected[x][i], newMap[x][i]) if !ok {
t.Errorf("Failed to cast endpointsInfo")
continue
}
if *newEp != *(expected[x][i]) {
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, expected[x][i], newEp)
} }
} }
} }
@@ -1950,14 +1680,14 @@ func Test_updateEndpointsMap(t *testing.T) {
currentEndpoints []*api.Endpoints currentEndpoints []*api.Endpoints
oldEndpoints map[proxy.ServicePortName][]*endpointsInfo oldEndpoints map[proxy.ServicePortName][]*endpointsInfo
expectedResult map[proxy.ServicePortName][]*endpointsInfo expectedResult map[proxy.ServicePortName][]*endpointsInfo
expectedStaleEndpoints []endpointServicePair expectedStaleEndpoints []proxy.ServiceEndpoint
expectedStaleServiceNames map[proxy.ServicePortName]bool expectedStaleServiceNames map[proxy.ServicePortName]bool
expectedHealthchecks map[types.NamespacedName]int expectedHealthchecks map[types.NamespacedName]int
}{{ }{{
// Case[0]: nothing // Case[0]: nothing
oldEndpoints: map[proxy.ServicePortName][]*endpointsInfo{}, oldEndpoints: map[proxy.ServicePortName][]*endpointsInfo{},
expectedResult: map[proxy.ServicePortName][]*endpointsInfo{}, expectedResult: map[proxy.ServicePortName][]*endpointsInfo{},
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -1978,7 +1708,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -1999,7 +1729,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: true}, {endpoint: "1.1.1.1:11", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 1, makeNSN("ns1", "ep1"): 1,
@@ -2028,7 +1758,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: false}, {endpoint: "1.1.1.2:12", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -2061,7 +1791,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.3:13", isLocal: false}, {endpoint: "1.1.1.3:13", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 1, makeNSN("ns1", "ep1"): 1,
@@ -2128,7 +1858,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "2.2.2.2:22", isLocal: true}, {endpoint: "2.2.2.2:22", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 2, makeNSN("ns1", "ep1"): 2,
@@ -2148,7 +1878,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: true}, {endpoint: "1.1.1.1:11", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", ""): true, makeServicePortName("ns1", "ep1", ""): true,
}, },
@@ -2169,9 +1899,9 @@ func Test_updateEndpointsMap(t *testing.T) {
}, },
}, },
expectedResult: map[proxy.ServicePortName][]*endpointsInfo{}, expectedResult: map[proxy.ServicePortName][]*endpointsInfo{},
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", ""), ServicePortName: makeServicePortName("ns1", "ep1", ""),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -2198,7 +1928,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: true}, {endpoint: "1.1.1.2:12", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
}, },
@@ -2228,15 +1958,15 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.2:11", Endpoint: "1.1.1.2:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}, { }, {
endpoint: "1.1.1.1:12", Endpoint: "1.1.1.1:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}, { }, {
endpoint: "1.1.1.2:12", Endpoint: "1.1.1.2:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -2261,7 +1991,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: true}, {endpoint: "1.1.1.2:12", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
}, },
@@ -2289,9 +2019,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.2:12", Endpoint: "1.1.1.2:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -2313,9 +2043,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p11-2"): true, makeServicePortName("ns1", "ep1", "p11-2"): true,
@@ -2339,9 +2069,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:22", isLocal: false}, {endpoint: "1.1.1.1:22", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -2396,21 +2126,21 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "4.4.4.4:44", isLocal: true}, {endpoint: "4.4.4.4:44", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "2.2.2.2:22", Endpoint: "2.2.2.2:22",
servicePortName: makeServicePortName("ns2", "ep2", "p22"), ServicePortName: makeServicePortName("ns2", "ep2", "p22"),
}, { }, {
endpoint: "2.2.2.22:22", Endpoint: "2.2.2.22:22",
servicePortName: makeServicePortName("ns2", "ep2", "p22"), ServicePortName: makeServicePortName("ns2", "ep2", "p22"),
}, { }, {
endpoint: "2.2.2.3:23", Endpoint: "2.2.2.3:23",
servicePortName: makeServicePortName("ns2", "ep2", "p23"), ServicePortName: makeServicePortName("ns2", "ep2", "p23"),
}, { }, {
endpoint: "4.4.4.5:44", Endpoint: "4.4.4.5:44",
servicePortName: makeServicePortName("ns4", "ep4", "p44"), ServicePortName: makeServicePortName("ns4", "ep4", "p44"),
}, { }, {
endpoint: "4.4.4.6:45", Endpoint: "4.4.4.6:45",
servicePortName: makeServicePortName("ns4", "ep4", "p45"), ServicePortName: makeServicePortName("ns4", "ep4", "p45"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
@@ -2434,7 +2164,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", ""): true, makeServicePortName("ns1", "ep1", ""): true,
}, },
@@ -2454,7 +2184,7 @@ func Test_updateEndpointsMap(t *testing.T) {
fp.OnEndpointsAdd(tc.previousEndpoints[i]) fp.OnEndpointsAdd(tc.previousEndpoints[i])
} }
} }
updateEndpointsMap(fp.endpointsMap, &fp.endpointsChanges, fp.hostname) proxy.UpdateEndpointsMap(fp.endpointsMap, fp.endpointsChanges)
compareEndpointsMaps(t, tci, fp.endpointsMap, tc.oldEndpoints) compareEndpointsMaps(t, tci, fp.endpointsMap, tc.oldEndpoints)
// Now let's call appropriate handlers to get to state we want to be. // Now let's call appropriate handlers to get to state we want to be.
@@ -2474,27 +2204,40 @@ func Test_updateEndpointsMap(t *testing.T) {
fp.OnEndpointsUpdate(prev, curr) fp.OnEndpointsUpdate(prev, curr)
} }
} }
result := updateEndpointsMap(fp.endpointsMap, &fp.endpointsChanges, fp.hostname) result := proxy.UpdateEndpointsMap(fp.endpointsMap, fp.endpointsChanges)
newMap := fp.endpointsMap newMap := fp.endpointsMap
compareEndpointsMaps(t, tci, newMap, tc.expectedResult) compareEndpointsMaps(t, tci, newMap, tc.expectedResult)
if len(result.staleEndpoints) != len(tc.expectedStaleEndpoints) { if len(result.StaleEndpoints) != len(tc.expectedStaleEndpoints) {
t.Errorf("[%d] expected %d staleEndpoints, got %d: %v", tci, len(tc.expectedStaleEndpoints), len(result.staleEndpoints), result.staleEndpoints) t.Errorf("[%d] expected %d staleEndpoints, got %d: %v", tci, len(tc.expectedStaleEndpoints), len(result.StaleEndpoints), result.StaleEndpoints)
} }
for _, x := range tc.expectedStaleEndpoints { for _, x := range tc.expectedStaleEndpoints {
if result.staleEndpoints[x] != true { found := false
t.Errorf("[%d] expected staleEndpoints[%v], but didn't find it: %v", tci, x, result.staleEndpoints) for _, stale := range result.StaleEndpoints {
if stale == x {
found = true
break
} }
} }
if len(result.staleServiceNames) != len(tc.expectedStaleServiceNames) { if !found {
t.Errorf("[%d] expected %d staleServiceNames, got %d: %v", tci, len(tc.expectedStaleServiceNames), len(result.staleServiceNames), result.staleServiceNames) t.Errorf("[%d] expected staleEndpoints[%v], but didn't find it: %v", tci, x, result.StaleEndpoints)
}
}
if len(result.StaleServiceNames) != len(tc.expectedStaleServiceNames) {
t.Errorf("[%d] expected %d staleServiceNames, got %d: %v", tci, len(tc.expectedStaleServiceNames), len(result.StaleServiceNames), result.StaleServiceNames)
} }
for svcName := range tc.expectedStaleServiceNames { for svcName := range tc.expectedStaleServiceNames {
if result.staleServiceNames[svcName] != true { found := false
t.Errorf("[%d] expected staleServiceNames[%v], but didn't find it: %v", tci, svcName, result.staleServiceNames) for _, stale := range result.StaleServiceNames {
if stale == svcName {
found = true
} }
} }
if !reflect.DeepEqual(result.hcEndpoints, tc.expectedHealthchecks) { if !found {
t.Errorf("[%d] expected healthchecks %v, got %v", tci, tc.expectedHealthchecks, result.hcEndpoints) t.Errorf("[%d] expected staleServiceNames[%v], but didn't find it: %v", tci, svcName, result.StaleServiceNames)
}
}
if !reflect.DeepEqual(result.HCEndpointsLocalIPSize, tc.expectedHealthchecks) {
t.Errorf("[%d] expected healthchecks %v, got %v", tci, tc.expectedHealthchecks, result.HCEndpointsLocalIPSize)
} }
} }
} }

View File

@@ -25,7 +25,6 @@ go_test(
"//pkg/util/iptables/testing:go_default_library", "//pkg/util/iptables/testing:go_default_library",
"//pkg/util/ipvs:go_default_library", "//pkg/util/ipvs:go_default_library",
"//pkg/util/ipvs/testing:go_default_library", "//pkg/util/ipvs/testing:go_default_library",
"//vendor/github.com/davecgh/go-spew/spew:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library", "//vendor/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/types:go_default_library", "//vendor/k8s.io/apimachinery/pkg/types:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/intstr:go_default_library", "//vendor/k8s.io/apimachinery/pkg/util/intstr:go_default_library",

View File

@@ -24,7 +24,6 @@ import (
"bytes" "bytes"
"fmt" "fmt"
"net" "net"
"reflect"
"strconv" "strconv"
"strings" "strings"
"sync" "sync"
@@ -109,12 +108,12 @@ type Proxier struct {
// services that happened since last syncProxyRules call. For a single object, // services that happened since last syncProxyRules call. For a single object,
// changes are accumulated, i.e. previous is state from before all of them, // changes are accumulated, i.e. previous is state from before all of them,
// current is state after applying all of those. // current is state after applying all of those.
endpointsChanges endpointsChangeMap endpointsChanges *proxy.EndpointChangeTracker
serviceChanges serviceChangeMap serviceChanges *proxy.ServiceChangeTracker
mu sync.Mutex // protects the following fields mu sync.Mutex // protects the following fields
serviceMap proxyServiceMap serviceMap proxy.ServiceMap
endpointsMap proxyEndpointsMap endpointsMap proxy.EndpointsMap
portsMap map[utilproxy.LocalPort]utilproxy.Closeable portsMap map[utilproxy.LocalPort]utilproxy.Closeable
// endpointsSynced and servicesSynced are set to true when corresponding // endpointsSynced and servicesSynced are set to true when corresponding
// objects are synced after startup. This is used to avoid updating ipvs rules // objects are synced after startup. This is used to avoid updating ipvs rules
@@ -302,10 +301,10 @@ func NewProxier(ipt utiliptables.Interface,
proxier := &Proxier{ proxier := &Proxier{
portsMap: make(map[utilproxy.LocalPort]utilproxy.Closeable), portsMap: make(map[utilproxy.LocalPort]utilproxy.Closeable),
serviceMap: make(proxyServiceMap), serviceMap: make(proxy.ServiceMap),
serviceChanges: newServiceChangeMap(), serviceChanges: proxy.NewServiceChangeTracker(),
endpointsMap: make(proxyEndpointsMap), endpointsMap: make(proxy.EndpointsMap),
endpointsChanges: newEndpointsChangeMap(hostname), endpointsChanges: proxy.NewEndpointChangeTracker(hostname),
syncPeriod: syncPeriod, syncPeriod: syncPeriod,
minSyncPeriod: minSyncPeriod, minSyncPeriod: minSyncPeriod,
iptables: ipt, iptables: ipt,
@@ -343,8 +342,6 @@ func NewProxier(ipt utiliptables.Interface,
return proxier, nil return proxier, nil
} }
type proxyServiceMap map[proxy.ServicePortName]*serviceInfo
// internal struct for string service information // internal struct for string service information
type serviceInfo struct { type serviceInfo struct {
clusterIP net.IP clusterIP net.IP
@@ -362,37 +359,8 @@ type serviceInfo struct {
serviceNameString string serviceNameString string
} }
// <serviceMap> is updated by this function (based on the given changes). // returns a new proxy.ServicePort which abstracts a serviceInfo
// <changes> map is cleared after applying them. func newServiceInfo(port *api.ServicePort, service *api.Service) proxy.ServicePort {
func updateServiceMap(
serviceMap proxyServiceMap,
changes *serviceChangeMap) (result updateServiceMapResult) {
result.staleServices = sets.NewString()
func() {
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
existingPorts := serviceMap.merge(change.current)
serviceMap.unmerge(change.previous, existingPorts, result.staleServices)
}
changes.items = make(map[types.NamespacedName]*serviceChange)
}()
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to serviceMap.
result.hcServices = make(map[types.NamespacedName]uint16)
for svcPortName, info := range serviceMap {
if info.healthCheckNodePort != 0 {
result.hcServices[svcPortName.NamespacedName] = uint16(info.healthCheckNodePort)
}
}
return result
}
// returns a new serviceInfo struct
func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, service *api.Service) *serviceInfo {
onlyNodeLocalEndpoints := false onlyNodeLocalEndpoints := false
if apiservice.RequestsOnlyLocalTraffic(service) { if apiservice.RequestsOnlyLocalTraffic(service) {
onlyNodeLocalEndpoints = true onlyNodeLocalEndpoints = true
@@ -418,10 +386,13 @@ func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, se
copy(info.loadBalancerSourceRanges, service.Spec.LoadBalancerSourceRanges) copy(info.loadBalancerSourceRanges, service.Spec.LoadBalancerSourceRanges)
copy(info.externalIPs, service.Spec.ExternalIPs) copy(info.externalIPs, service.Spec.ExternalIPs)
svcName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name}
svcPortName := proxy.ServicePortName{NamespacedName: svcName, Port: port.Name}
if apiservice.NeedsHealthCheck(service) { if apiservice.NeedsHealthCheck(service) {
p := service.Spec.HealthCheckNodePort p := service.Spec.HealthCheckNodePort
if p == 0 { if p == 0 {
glog.Errorf("Service %q has no healthcheck nodeport", svcPortName.NamespacedName.String()) glog.Errorf("Service %q has no healthcheck nodeport", svcName.String())
} else { } else {
info.healthCheckNodePort = int(p) info.healthCheckNodePort = int(p)
} }
@@ -433,103 +404,32 @@ func newServiceInfo(svcPortName proxy.ServicePortName, port *api.ServicePort, se
return info return info
} }
func (sm *proxyServiceMap) merge(other proxyServiceMap) sets.String { // ClusterIP is part of ServicePort interface.
existingPorts := sets.NewString() func (info *serviceInfo) ClusterIP() string {
for svcPortName, info := range other { return info.clusterIP.String()
existingPorts.Insert(svcPortName.Port)
_, exists := (*sm)[svcPortName]
if !exists {
glog.V(1).Infof("Adding new service port %q at %s:%d/%s", svcPortName, info.clusterIP, info.port, info.protocol)
} else {
glog.V(1).Infof("Updating existing service port %q at %s:%d/%s", svcPortName, info.clusterIP, info.port, info.protocol)
}
(*sm)[svcPortName] = info
}
return existingPorts
} }
func (sm *proxyServiceMap) unmerge(other proxyServiceMap, existingPorts, staleServices sets.String) { // Port is part of ServicePort interface.
for svcPortName := range other { func (info *serviceInfo) Port() int {
if existingPorts.Has(svcPortName.Port) { return info.port
continue
}
info, exists := (*sm)[svcPortName]
if exists {
glog.V(1).Infof("Removing service port %q", svcPortName)
if info.protocol == api.ProtocolUDP {
staleServices.Insert(info.clusterIP.String())
}
delete(*sm, svcPortName)
} else {
glog.Errorf("Service port %q removed, but doesn't exists", svcPortName)
}
}
} }
type serviceChangeMap struct { // Protocol is part of ServicePort interface.
lock sync.Mutex func (info *serviceInfo) Protocol() api.Protocol {
items map[types.NamespacedName]*serviceChange return info.protocol
} }
type serviceChange struct { // String is part of ServicePort interface.
previous proxyServiceMap func (info *serviceInfo) String() string {
current proxyServiceMap return fmt.Sprintf("%s:%d/%s", info.clusterIP, info.port, info.protocol)
} }
type updateEndpointMapResult struct { // HealthCheckNodePort is part of ServicePort interface.
hcEndpoints map[types.NamespacedName]int func (info *serviceInfo) HealthCheckNodePort() int {
staleEndpoints map[endpointServicePair]bool return info.healthCheckNodePort
staleServiceNames map[proxy.ServicePortName]bool
} }
type updateServiceMapResult struct { var _ proxy.ServicePort = &serviceInfo{}
hcServices map[types.NamespacedName]uint16
staleServices sets.String
}
func newServiceChangeMap() serviceChangeMap {
return serviceChangeMap{
items: make(map[types.NamespacedName]*serviceChange),
}
}
func (scm *serviceChangeMap) update(namespacedName *types.NamespacedName, previous, current *api.Service) bool {
scm.lock.Lock()
defer scm.lock.Unlock()
change, exists := scm.items[*namespacedName]
if !exists {
change = &serviceChange{}
change.previous = serviceToServiceMap(previous)
scm.items[*namespacedName] = change
}
change.current = serviceToServiceMap(current)
if reflect.DeepEqual(change.previous, change.current) {
delete(scm.items, *namespacedName)
}
return len(scm.items) > 0
}
// Translates single Service object to proxyServiceMap.
//
// NOTE: service object should NOT be modified.
func serviceToServiceMap(service *api.Service) proxyServiceMap {
if service == nil {
return nil
}
svcName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name}
if utilproxy.ShouldSkipService(svcName, service) {
return nil
}
serviceMap := make(proxyServiceMap)
for i := range service.Spec.Ports {
servicePort := &service.Spec.Ports[i]
svcPortName := proxy.ServicePortName{NamespacedName: svcName, Port: servicePort.Name}
serviceMap[svcPortName] = newServiceInfo(svcPortName, servicePort, service)
}
return serviceMap
}
// internal struct for endpoints information // internal struct for endpoints information
type endpointsInfo struct { type endpointsInfo struct {
@@ -537,12 +437,26 @@ type endpointsInfo struct {
isLocal bool isLocal bool
} }
func (e *endpointsInfo) String() string { // returns a new proxy.Endpoint which abstracts a endpointsInfo
return fmt.Sprintf("%v", *e) func newEndpointsInfo(IP string, port int, isLocal bool) proxy.Endpoint {
return &endpointsInfo{
endpoint: net.JoinHostPort(IP, strconv.Itoa(port)),
isLocal: isLocal,
}
} }
// IPPart returns just the IP part of the endpoint. // IsLocal is part of proxy.Endpoint interface.
func (e *endpointsInfo) IPPart() string { func (e *endpointsInfo) IsLocal() bool {
return e.isLocal
}
// String is part of proxy.Endpoint interface.
func (e *endpointsInfo) String() string {
return fmt.Sprintf("%v", e.endpoint)
}
// IP returns just the IP part of the endpoint, it's a part of proxy.Endpoints interface.
func (e *endpointsInfo) IP() string {
return utilproxy.IPPart(e.endpoint) return utilproxy.IPPart(e.endpoint)
} }
@@ -551,164 +465,18 @@ func (e *endpointsInfo) PortPart() (int, error) {
return utilproxy.PortPart(e.endpoint) return utilproxy.PortPart(e.endpoint)
} }
type endpointServicePair struct { // Equal is part of proxy.Endpoint interface.
endpoint string func (e *endpointsInfo) Equal(other proxy.Endpoint) bool {
servicePortName proxy.ServicePortName o, ok := other.(*endpointsInfo)
if !ok {
glog.Errorf("Failed to cast endpointsInfo")
return false
}
return e.endpoint == o.endpoint &&
e.isLocal == o.isLocal
} }
type proxyEndpointsMap map[proxy.ServicePortName][]*endpointsInfo var _ proxy.Endpoint = &endpointsInfo{}
type endpointsChange struct {
previous proxyEndpointsMap
current proxyEndpointsMap
}
type endpointsChangeMap struct {
lock sync.Mutex
hostname string
items map[types.NamespacedName]*endpointsChange
}
// <staleEndpoints> and <staleServices> are modified by this function with detected stale connections.
func detectStaleConnections(oldEndpointsMap, newEndpointsMap proxyEndpointsMap, staleEndpoints map[endpointServicePair]bool, staleServiceNames map[proxy.ServicePortName]bool) {
for svcPortName, epList := range oldEndpointsMap {
for _, ep := range epList {
stale := true
for i := range newEndpointsMap[svcPortName] {
if *newEndpointsMap[svcPortName][i] == *ep {
stale = false
break
}
}
if stale {
glog.V(4).Infof("Stale endpoint %v -> %v", svcPortName, ep.endpoint)
staleEndpoints[endpointServicePair{endpoint: ep.endpoint, servicePortName: svcPortName}] = true
}
}
}
for svcPortName, epList := range newEndpointsMap {
// For udp service, if its backend changes from 0 to non-0. There may exist a conntrack entry that could blackhole traffic to the service.
if len(epList) > 0 && len(oldEndpointsMap[svcPortName]) == 0 {
staleServiceNames[svcPortName] = true
}
}
}
// <endpointsMap> is updated by this function (based on the given changes).
// <changes> map is cleared after applying them.
func updateEndpointsMap(
endpointsMap proxyEndpointsMap,
changes *endpointsChangeMap,
hostname string) (result updateEndpointMapResult) {
result.staleEndpoints = make(map[endpointServicePair]bool)
result.staleServiceNames = make(map[proxy.ServicePortName]bool)
func() {
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
endpointsMap.unmerge(change.previous)
endpointsMap.merge(change.current)
detectStaleConnections(change.previous, change.current, result.staleEndpoints, result.staleServiceNames)
}
changes.items = make(map[types.NamespacedName]*endpointsChange)
}()
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to endpointsMap.
result.hcEndpoints = make(map[types.NamespacedName]int)
localIPs := getLocalIPs(endpointsMap)
for nsn, ips := range localIPs {
result.hcEndpoints[nsn] = len(ips)
}
return result
}
// Translates single Endpoints object to proxyEndpointsMap.
// This function is used for incremental updated of endpointsMap.
//
// NOTE: endpoints object should NOT be modified.
func endpointsToEndpointsMap(endpoints *api.Endpoints, hostname string) proxyEndpointsMap {
if endpoints == nil {
return nil
}
endpointsMap := make(proxyEndpointsMap)
// We need to build a map of portname -> all ip:ports for that
// portname. Explode Endpoints.Subsets[*] into this structure.
for i := range endpoints.Subsets {
ss := &endpoints.Subsets[i]
for i := range ss.Ports {
port := &ss.Ports[i]
if port.Port == 0 {
glog.Warningf("ignoring invalid endpoint port %s", port.Name)
continue
}
svcPort := proxy.ServicePortName{
NamespacedName: types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name},
Port: port.Name,
}
for i := range ss.Addresses {
addr := &ss.Addresses[i]
if addr.IP == "" {
glog.Warningf("ignoring invalid endpoint port %s with empty host", port.Name)
continue
}
epInfo := &endpointsInfo{
endpoint: net.JoinHostPort(addr.IP, strconv.Itoa(int(port.Port))),
isLocal: addr.NodeName != nil && *addr.NodeName == hostname,
}
endpointsMap[svcPort] = append(endpointsMap[svcPort], epInfo)
}
if glog.V(3) {
newEPList := []string{}
for _, ep := range endpointsMap[svcPort] {
newEPList = append(newEPList, ep.endpoint)
}
glog.Infof("Setting endpoints for %q to %+v", svcPort, newEPList)
}
}
}
return endpointsMap
}
func newEndpointsChangeMap(hostname string) endpointsChangeMap {
return endpointsChangeMap{
hostname: hostname,
items: make(map[types.NamespacedName]*endpointsChange),
}
}
func (ecm *endpointsChangeMap) update(namespacedName *types.NamespacedName, previous, current *api.Endpoints) bool {
ecm.lock.Lock()
defer ecm.lock.Unlock()
change, exists := ecm.items[*namespacedName]
if !exists {
change = &endpointsChange{}
change.previous = endpointsToEndpointsMap(previous, ecm.hostname)
ecm.items[*namespacedName] = change
}
change.current = endpointsToEndpointsMap(current, ecm.hostname)
if reflect.DeepEqual(change.previous, change.current) {
delete(ecm.items, *namespacedName)
}
return len(ecm.items) > 0
}
func (em proxyEndpointsMap) merge(other proxyEndpointsMap) {
for svcPort := range other {
em[svcPort] = other[svcPort]
}
}
func (em proxyEndpointsMap) unmerge(other proxyEndpointsMap) {
for svcPort := range other {
delete(em, svcPort)
}
}
// KernelHandler can handle the current installed kernel modules. // KernelHandler can handle the current installed kernel modules.
type KernelHandler interface { type KernelHandler interface {
@@ -891,24 +659,21 @@ func (proxier *Proxier) isInitialized() bool {
// OnServiceAdd is called whenever creation of new service object is observed. // OnServiceAdd is called whenever creation of new service object is observed.
func (proxier *Proxier) OnServiceAdd(service *api.Service) { func (proxier *Proxier) OnServiceAdd(service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(nil, service, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, nil, service) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
// OnServiceUpdate is called whenever modification of an existing service object is observed. // OnServiceUpdate is called whenever modification of an existing service object is observed.
func (proxier *Proxier) OnServiceUpdate(oldService, service *api.Service) { func (proxier *Proxier) OnServiceUpdate(oldService, service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(oldService, service, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, oldService, service) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
// OnServiceDelete is called whenever deletion of an existing service object is observed. // OnServiceDelete is called whenever deletion of an existing service object is observed.
func (proxier *Proxier) OnServiceDelete(service *api.Service) { func (proxier *Proxier) OnServiceDelete(service *api.Service) {
namespacedName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name} if proxier.serviceChanges.Update(service, nil, newServiceInfo) && proxier.isInitialized() {
if proxier.serviceChanges.update(&namespacedName, service, nil) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
@@ -926,24 +691,21 @@ func (proxier *Proxier) OnServiceSynced() {
// OnEndpointsAdd is called whenever creation of new endpoints object is observed. // OnEndpointsAdd is called whenever creation of new endpoints object is observed.
func (proxier *Proxier) OnEndpointsAdd(endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsAdd(endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(nil, endpoints, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, nil, endpoints) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
// OnEndpointsUpdate is called whenever modification of an existing endpoints object is observed. // OnEndpointsUpdate is called whenever modification of an existing endpoints object is observed.
func (proxier *Proxier) OnEndpointsUpdate(oldEndpoints, endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsUpdate(oldEndpoints, endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(oldEndpoints, endpoints, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, oldEndpoints, endpoints) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
// OnEndpointsDelete is called whenever deletion of an existing endpoints object is observed. // OnEndpointsDelete is called whenever deletion of an existing endpoints object is observed.
func (proxier *Proxier) OnEndpointsDelete(endpoints *api.Endpoints) { func (proxier *Proxier) OnEndpointsDelete(endpoints *api.Endpoints) {
namespacedName := types.NamespacedName{Namespace: endpoints.Namespace, Name: endpoints.Name} if proxier.endpointsChanges.Update(endpoints, nil, newEndpointsInfo) && proxier.isInitialized() {
if proxier.endpointsChanges.update(&namespacedName, endpoints, nil) && proxier.isInitialized() {
proxier.syncRunner.Run() proxier.syncRunner.Run()
} }
} }
@@ -980,17 +742,15 @@ func (proxier *Proxier) syncProxyRules() {
// We assume that if this was called, we really want to sync them, // We assume that if this was called, we really want to sync them,
// even if nothing changed in the meantime. In other words, callers are // even if nothing changed in the meantime. In other words, callers are
// responsible for detecting no-op changes and not calling this function. // responsible for detecting no-op changes and not calling this function.
serviceUpdateResult := updateServiceMap( serviceUpdateResult := proxy.UpdateServiceMap(proxier.serviceMap, proxier.serviceChanges)
proxier.serviceMap, &proxier.serviceChanges) endpointUpdateResult := proxy.UpdateEndpointsMap(proxier.endpointsMap, proxier.endpointsChanges)
endpointUpdateResult := updateEndpointsMap(
proxier.endpointsMap, &proxier.endpointsChanges, proxier.hostname)
staleServices := serviceUpdateResult.staleServices staleServices := serviceUpdateResult.UDPStaleClusterIP
// merge stale services gathered from updateEndpointsMap // merge stale services gathered from updateEndpointsMap
for svcPortName := range endpointUpdateResult.staleServiceNames { for _, svcPortName := range endpointUpdateResult.StaleServiceNames {
if svcInfo, ok := proxier.serviceMap[svcPortName]; ok && svcInfo != nil && svcInfo.protocol == api.ProtocolUDP { if svcInfo, ok := proxier.serviceMap[svcPortName]; ok && svcInfo != nil && svcInfo.Protocol() == api.ProtocolUDP {
glog.V(2).Infof("Stale udp service %v -> %s", svcPortName, svcInfo.clusterIP.String()) glog.V(2).Infof("Stale udp service %v -> %s", svcPortName, svcInfo.ClusterIP())
staleServices.Insert(svcInfo.clusterIP.String()) staleServices.Insert(svcInfo.ClusterIP())
} }
} }
@@ -1093,15 +853,25 @@ func (proxier *Proxier) syncProxyRules() {
} }
// Build IPVS rules for each service. // Build IPVS rules for each service.
for svcName, svcInfo := range proxier.serviceMap { for svcName, svc := range proxier.serviceMap {
svcInfo, ok := svc.(*serviceInfo)
if !ok {
glog.Errorf("Failed to cast serviceInfo %q", svcName.String())
continue
}
protocol := strings.ToLower(string(svcInfo.protocol)) protocol := strings.ToLower(string(svcInfo.protocol))
// Precompute svcNameString; with many services the many calls // Precompute svcNameString; with many services the many calls
// to ServicePortName.String() show up in CPU profiles. // to ServicePortName.String() show up in CPU profiles.
svcNameString := svcName.String() svcNameString := svcName.String()
// Handle traffic that loops back to the originator with SNAT. // Handle traffic that loops back to the originator with SNAT.
for _, ep := range proxier.endpointsMap[svcName] { for _, e := range proxier.endpointsMap[svcName] {
epIP := ep.IPPart() ep, ok := e.(*endpointsInfo)
if !ok {
glog.Errorf("Failed to cast endpointsInfo %q", e.String())
continue
}
epIP := ep.IP()
epPort, err := ep.PortPart() epPort, err := ep.PortPart()
// Error parsing this endpoint has been logged. Skip to next endpoint. // Error parsing this endpoint has been logged. Skip to next endpoint.
if epIP == "" || err != nil { if epIP == "" || err != nil {
@@ -1569,10 +1339,10 @@ func (proxier *Proxier) syncProxyRules() {
// Update healthchecks. The endpoints list might include services that are // Update healthchecks. The endpoints list might include services that are
// not "OnlyLocal", but the services list will not, and the healthChecker // not "OnlyLocal", but the services list will not, and the healthChecker
// will just drop those endpoints. // will just drop those endpoints.
if err := proxier.healthChecker.SyncServices(serviceUpdateResult.hcServices); err != nil { if err := proxier.healthChecker.SyncServices(serviceUpdateResult.HCServiceNodePorts); err != nil {
glog.Errorf("Error syncing healtcheck services: %v", err) glog.Errorf("Error syncing healtcheck services: %v", err)
} }
if err := proxier.healthChecker.SyncEndpoints(endpointUpdateResult.hcEndpoints); err != nil { if err := proxier.healthChecker.SyncEndpoints(endpointUpdateResult.HCEndpointsLocalIPSize); err != nil {
glog.Errorf("Error syncing healthcheck endpoints: %v", err) glog.Errorf("Error syncing healthcheck endpoints: %v", err)
} }
@@ -1583,19 +1353,19 @@ func (proxier *Proxier) syncProxyRules() {
glog.Errorf("Failed to delete stale service IP %s connections, error: %v", svcIP, err) glog.Errorf("Failed to delete stale service IP %s connections, error: %v", svcIP, err)
} }
} }
proxier.deleteEndpointConnections(endpointUpdateResult.staleEndpoints) proxier.deleteEndpointConnections(endpointUpdateResult.StaleEndpoints)
} }
// After a UDP endpoint has been removed, we must flush any pending conntrack entries to it, or else we // After a UDP endpoint has been removed, we must flush any pending conntrack entries to it, or else we
// risk sending more traffic to it, all of which will be lost (because UDP). // risk sending more traffic to it, all of which will be lost (because UDP).
// This assumes the proxier mutex is held // This assumes the proxier mutex is held
func (proxier *Proxier) deleteEndpointConnections(connectionMap map[endpointServicePair]bool) { func (proxier *Proxier) deleteEndpointConnections(connectionMap []proxy.ServiceEndpoint) {
for epSvcPair := range connectionMap { for _, epSvcPair := range connectionMap {
if svcInfo, ok := proxier.serviceMap[epSvcPair.servicePortName]; ok && svcInfo.protocol == api.ProtocolUDP { if svcInfo, ok := proxier.serviceMap[epSvcPair.ServicePortName]; ok && svcInfo.Protocol() == api.ProtocolUDP {
endpointIP := utilproxy.IPPart(epSvcPair.endpoint) endpointIP := utilproxy.IPPart(epSvcPair.Endpoint)
err := utilproxy.ClearUDPConntrackForPeers(proxier.exec, svcInfo.clusterIP.String(), endpointIP) err := utilproxy.ClearUDPConntrackForPeers(proxier.exec, svcInfo.ClusterIP(), endpointIP)
if err != nil { if err != nil {
glog.Errorf("Failed to delete %s endpoint connections, error: %v", epSvcPair.servicePortName.String(), err) glog.Errorf("Failed to delete %s endpoint connections, error: %v", epSvcPair.ServicePortName.String(), err)
} }
} }
} }
@@ -1656,8 +1426,13 @@ func (proxier *Proxier) syncEndpoint(svcPortName proxy.ServicePortName, onlyNode
} }
for _, eps := range proxier.endpointsMap[svcPortName] { for _, eps := range proxier.endpointsMap[svcPortName] {
if !onlyNodeLocalEndpoints || onlyNodeLocalEndpoints && eps.isLocal { epInfo, ok := eps.(*endpointsInfo)
newEndpoints.Insert(eps.endpoint) if !ok {
glog.Errorf("Failed to cast endpointsInfo")
continue
}
if !onlyNodeLocalEndpoints || onlyNodeLocalEndpoints && epInfo.isLocal {
newEndpoints.Insert(epInfo.endpoint)
} }
} }
@@ -1796,26 +1571,6 @@ func writeLine(buf *bytes.Buffer, words ...string) {
} }
} }
func getLocalIPs(endpointsMap proxyEndpointsMap) map[types.NamespacedName]sets.String {
localIPs := make(map[types.NamespacedName]sets.String)
for svcPortName := range endpointsMap {
for _, ep := range endpointsMap[svcPortName] {
if ep.isLocal {
// If the endpoint has a bad format, utilproxy.IPPart() will log an
// error and ep.IPPart() will return a null string.
if ip := ep.IPPart(); ip != "" {
nsn := svcPortName.NamespacedName
if localIPs[nsn] == nil {
localIPs[nsn] = sets.NewString()
}
localIPs[nsn].Insert(ip)
}
}
}
}
return localIPs
}
// listenPortOpener opens ports by calling bind() and listen(). // listenPortOpener opens ports by calling bind() and listen().
type listenPortOpener struct{} type listenPortOpener struct{}

View File

@@ -40,8 +40,6 @@ import (
iptablestest "k8s.io/kubernetes/pkg/util/iptables/testing" iptablestest "k8s.io/kubernetes/pkg/util/iptables/testing"
utilipvs "k8s.io/kubernetes/pkg/util/ipvs" utilipvs "k8s.io/kubernetes/pkg/util/ipvs"
ipvstest "k8s.io/kubernetes/pkg/util/ipvs/testing" ipvstest "k8s.io/kubernetes/pkg/util/ipvs/testing"
"github.com/davecgh/go-spew/spew"
) )
const testHostname = "test-hostname" const testHostname = "test-hostname"
@@ -121,10 +119,10 @@ func NewFakeProxier(ipt utiliptables.Interface, ipvs utilipvs.Interface, ipset u
} }
return &Proxier{ return &Proxier{
exec: fexec, exec: fexec,
serviceMap: make(proxyServiceMap), serviceMap: make(proxy.ServiceMap),
serviceChanges: newServiceChangeMap(), serviceChanges: proxy.NewServiceChangeTracker(),
endpointsMap: make(proxyEndpointsMap), endpointsMap: make(proxy.EndpointsMap),
endpointsChanges: newEndpointsChangeMap(testHostname), endpointsChanges: proxy.NewEndpointChangeTracker(testHostname),
iptables: ipt, iptables: ipt,
ipvs: ipvs, ipvs: ipvs,
ipset: ipset, ipset: ipset,
@@ -997,24 +995,24 @@ func TestBuildServiceMapAddRemove(t *testing.T) {
for i := range services { for i := range services {
fp.OnServiceAdd(services[i]) fp.OnServiceAdd(services[i])
} }
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 8 { if len(fp.serviceMap) != 8 {
t.Errorf("expected service map length 8, got %v", fp.serviceMap) t.Errorf("expected service map length 8, got %v", fp.serviceMap)
} }
// The only-local-loadbalancer ones get added // The only-local-loadbalancer ones get added
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected 1 healthcheck port, got %v", result.hcServices) t.Errorf("expected 1 healthcheck port, got %v", result.HCServiceNodePorts)
} else { } else {
nsn := makeNSN("somewhere", "only-local-load-balancer") nsn := makeNSN("somewhere", "only-local-load-balancer")
if port, found := result.hcServices[nsn]; !found || port != 345 { if port, found := result.HCServiceNodePorts[nsn]; !found || port != 345 {
t.Errorf("expected healthcheck port [%q]=345: got %v", nsn, result.hcServices) t.Errorf("expected healthcheck port [%q]=345: got %v", nsn, result.HCServiceNodePorts)
} }
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
// Remove some stuff // Remove some stuff
@@ -1030,24 +1028,24 @@ func TestBuildServiceMapAddRemove(t *testing.T) {
fp.OnServiceDelete(services[2]) fp.OnServiceDelete(services[2])
fp.OnServiceDelete(services[3]) fp.OnServiceDelete(services[3])
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 1 { if len(fp.serviceMap) != 1 {
t.Errorf("expected service map length 1, got %v", fp.serviceMap) t.Errorf("expected service map length 1, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected 0 healthcheck ports, got %v", result.hcServices) t.Errorf("expected 0 healthcheck ports, got %v", result.HCServiceNodePorts)
} }
// All services but one were deleted. While you'd expect only the ClusterIPs // All services but one were deleted. While you'd expect only the ClusterIPs
// from the three deleted services here, we still have the ClusterIP for // from the three deleted services here, we still have the ClusterIP for
// the not-deleted service, because one of it's ServicePorts was deleted. // the not-deleted service, because one of it's ServicePorts was deleted.
expectedStaleUDPServices := []string{"172.16.55.10", "172.16.55.4", "172.16.55.11", "172.16.55.12"} expectedStaleUDPServices := []string{"172.16.55.10", "172.16.55.4", "172.16.55.11", "172.16.55.12"}
if len(result.staleServices) != len(expectedStaleUDPServices) { if len(result.UDPStaleClusterIP) != len(expectedStaleUDPServices) {
t.Errorf("expected stale UDP services length %d, got %v", len(expectedStaleUDPServices), result.staleServices.List()) t.Errorf("expected stale UDP services length %d, got %v", len(expectedStaleUDPServices), result.UDPStaleClusterIP.List())
} }
for _, ip := range expectedStaleUDPServices { for _, ip := range expectedStaleUDPServices {
if !result.staleServices.Has(ip) { if !result.UDPStaleClusterIP.Has(ip) {
t.Errorf("expected stale UDP service service %s", ip) t.Errorf("expected stale UDP service service %s", ip)
} }
} }
@@ -1072,18 +1070,18 @@ func TestBuildServiceMapServiceHeadless(t *testing.T) {
) )
// Headless service should be ignored // Headless service should be ignored
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 { if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %d", len(fp.serviceMap)) t.Errorf("expected service map length 0, got %d", len(fp.serviceMap))
} }
// No proxied services, so no healthchecks // No proxied services, so no healthchecks
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %d", len(result.hcServices)) t.Errorf("expected healthcheck ports length 0, got %d", len(result.HCServiceNodePorts))
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
} }
@@ -1102,16 +1100,16 @@ func TestBuildServiceMapServiceTypeExternalName(t *testing.T) {
}), }),
) )
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 { if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %v", fp.serviceMap) t.Errorf("expected service map length 0, got %v", fp.serviceMap)
} }
// No proxied services, so no healthchecks // No proxied services, so no healthchecks
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP)
} }
} }
@@ -1144,57 +1142,57 @@ func TestBuildServiceMapServiceUpdate(t *testing.T) {
fp.OnServiceAdd(servicev1) fp.OnServiceAdd(servicev1)
result := updateServiceMap(fp.serviceMap, &fp.serviceChanges) result := proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
// Change service to load-balancer // Change service to load-balancer
fp.OnServiceUpdate(servicev1, servicev2) fp.OnServiceUpdate(servicev1, servicev2)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices.List()) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.List())
} }
// No change; make sure the service map stays the same and there are // No change; make sure the service map stays the same and there are
// no health-check changes // no health-check changes
fp.OnServiceUpdate(servicev2, servicev2) fp.OnServiceUpdate(servicev2, servicev2)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 1 { if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.staleServices.List()) t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.List())
} }
// And back to ClusterIP // And back to ClusterIP
fp.OnServiceUpdate(servicev2, servicev1) fp.OnServiceUpdate(servicev2, servicev1)
result = updateServiceMap(fp.serviceMap, &fp.serviceChanges) result = proxy.UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 { if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap) t.Errorf("expected service map length 2, got %v", fp.serviceMap)
} }
if len(result.hcServices) != 0 { if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.hcServices) t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
} }
if len(result.staleServices) != 0 { if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet // Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.staleServices)) t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
} }
} }
@@ -1570,14 +1568,14 @@ func Test_updateEndpointsMap(t *testing.T) {
currentEndpoints []*api.Endpoints currentEndpoints []*api.Endpoints
oldEndpoints map[proxy.ServicePortName][]*endpointsInfo oldEndpoints map[proxy.ServicePortName][]*endpointsInfo
expectedResult map[proxy.ServicePortName][]*endpointsInfo expectedResult map[proxy.ServicePortName][]*endpointsInfo
expectedStaleEndpoints []endpointServicePair expectedStaleEndpoints []proxy.ServiceEndpoint
expectedStaleServiceNames map[proxy.ServicePortName]bool expectedStaleServiceNames map[proxy.ServicePortName]bool
expectedHealthchecks map[types.NamespacedName]int expectedHealthchecks map[types.NamespacedName]int
}{{ }{{
// Case[0]: nothing // Case[0]: nothing
oldEndpoints: map[proxy.ServicePortName][]*endpointsInfo{}, oldEndpoints: map[proxy.ServicePortName][]*endpointsInfo{},
expectedResult: map[proxy.ServicePortName][]*endpointsInfo{}, expectedResult: map[proxy.ServicePortName][]*endpointsInfo{},
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -1598,7 +1596,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -1619,7 +1617,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: true}, {endpoint: "1.1.1.1:11", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 1, makeNSN("ns1", "ep1"): 1,
@@ -1648,7 +1646,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: false}, {endpoint: "1.1.1.2:12", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
}, { }, {
@@ -1681,7 +1679,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.3:13", isLocal: false}, {endpoint: "1.1.1.3:13", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 1, makeNSN("ns1", "ep1"): 1,
@@ -1748,7 +1746,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "2.2.2.2:22", isLocal: true}, {endpoint: "2.2.2.2:22", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{ expectedHealthchecks: map[types.NamespacedName]int{
makeNSN("ns1", "ep1"): 2, makeNSN("ns1", "ep1"): 2,
@@ -1768,7 +1766,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: true}, {endpoint: "1.1.1.1:11", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", ""): true, makeServicePortName("ns1", "ep1", ""): true,
}, },
@@ -1789,9 +1787,9 @@ func Test_updateEndpointsMap(t *testing.T) {
}, },
}, },
expectedResult: map[proxy.ServicePortName][]*endpointsInfo{}, expectedResult: map[proxy.ServicePortName][]*endpointsInfo{},
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", ""), ServicePortName: makeServicePortName("ns1", "ep1", ""),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -1818,7 +1816,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: true}, {endpoint: "1.1.1.2:12", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
}, },
@@ -1848,15 +1846,15 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.2:11", Endpoint: "1.1.1.2:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}, { }, {
endpoint: "1.1.1.1:12", Endpoint: "1.1.1.1:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}, { }, {
endpoint: "1.1.1.2:12", Endpoint: "1.1.1.2:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -1881,7 +1879,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.2:12", isLocal: true}, {endpoint: "1.1.1.2:12", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
}, },
@@ -1909,9 +1907,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.2:12", Endpoint: "1.1.1.2:12",
servicePortName: makeServicePortName("ns1", "ep1", "p12"), ServicePortName: makeServicePortName("ns1", "ep1", "p12"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -1933,9 +1931,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p11-2"): true, makeServicePortName("ns1", "ep1", "p11-2"): true,
@@ -1959,9 +1957,9 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:22", isLocal: false}, {endpoint: "1.1.1.1:22", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "1.1.1.1:11", Endpoint: "1.1.1.1:11",
servicePortName: makeServicePortName("ns1", "ep1", "p11"), ServicePortName: makeServicePortName("ns1", "ep1", "p11"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{}, expectedStaleServiceNames: map[proxy.ServicePortName]bool{},
expectedHealthchecks: map[types.NamespacedName]int{}, expectedHealthchecks: map[types.NamespacedName]int{},
@@ -2016,21 +2014,21 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "4.4.4.4:44", isLocal: true}, {endpoint: "4.4.4.4:44", isLocal: true},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{{ expectedStaleEndpoints: []proxy.ServiceEndpoint{{
endpoint: "2.2.2.2:22", Endpoint: "2.2.2.2:22",
servicePortName: makeServicePortName("ns2", "ep2", "p22"), ServicePortName: makeServicePortName("ns2", "ep2", "p22"),
}, { }, {
endpoint: "2.2.2.22:22", Endpoint: "2.2.2.22:22",
servicePortName: makeServicePortName("ns2", "ep2", "p22"), ServicePortName: makeServicePortName("ns2", "ep2", "p22"),
}, { }, {
endpoint: "2.2.2.3:23", Endpoint: "2.2.2.3:23",
servicePortName: makeServicePortName("ns2", "ep2", "p23"), ServicePortName: makeServicePortName("ns2", "ep2", "p23"),
}, { }, {
endpoint: "4.4.4.5:44", Endpoint: "4.4.4.5:44",
servicePortName: makeServicePortName("ns4", "ep4", "p44"), ServicePortName: makeServicePortName("ns4", "ep4", "p44"),
}, { }, {
endpoint: "4.4.4.6:45", Endpoint: "4.4.4.6:45",
servicePortName: makeServicePortName("ns4", "ep4", "p45"), ServicePortName: makeServicePortName("ns4", "ep4", "p45"),
}}, }},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", "p12"): true, makeServicePortName("ns1", "ep1", "p12"): true,
@@ -2054,7 +2052,7 @@ func Test_updateEndpointsMap(t *testing.T) {
{endpoint: "1.1.1.1:11", isLocal: false}, {endpoint: "1.1.1.1:11", isLocal: false},
}, },
}, },
expectedStaleEndpoints: []endpointServicePair{}, expectedStaleEndpoints: []proxy.ServiceEndpoint{},
expectedStaleServiceNames: map[proxy.ServicePortName]bool{ expectedStaleServiceNames: map[proxy.ServicePortName]bool{
makeServicePortName("ns1", "ep1", ""): true, makeServicePortName("ns1", "ep1", ""): true,
}, },
@@ -2076,7 +2074,7 @@ func Test_updateEndpointsMap(t *testing.T) {
fp.OnEndpointsAdd(tc.previousEndpoints[i]) fp.OnEndpointsAdd(tc.previousEndpoints[i])
} }
} }
updateEndpointsMap(fp.endpointsMap, &fp.endpointsChanges, fp.hostname) proxy.UpdateEndpointsMap(fp.endpointsMap, fp.endpointsChanges)
compareEndpointsMaps(t, tci, fp.endpointsMap, tc.oldEndpoints) compareEndpointsMaps(t, tci, fp.endpointsMap, tc.oldEndpoints)
// Now let's call appropriate handlers to get to state we want to be. // Now let's call appropriate handlers to get to state we want to be.
@@ -2096,313 +2094,61 @@ func Test_updateEndpointsMap(t *testing.T) {
fp.OnEndpointsUpdate(prev, curr) fp.OnEndpointsUpdate(prev, curr)
} }
} }
result := updateEndpointsMap(fp.endpointsMap, &fp.endpointsChanges, fp.hostname) result := proxy.UpdateEndpointsMap(fp.endpointsMap, fp.endpointsChanges)
newMap := fp.endpointsMap newMap := fp.endpointsMap
compareEndpointsMaps(t, tci, newMap, tc.expectedResult) compareEndpointsMaps(t, tci, newMap, tc.expectedResult)
if len(result.staleEndpoints) != len(tc.expectedStaleEndpoints) { if len(result.StaleEndpoints) != len(tc.expectedStaleEndpoints) {
t.Errorf("[%d] expected %d staleEndpoints, got %d: %v", tci, len(tc.expectedStaleEndpoints), len(result.staleEndpoints), result.staleEndpoints) t.Errorf("[%d] expected %d staleEndpoints, got %d: %v", tci, len(tc.expectedStaleEndpoints), len(result.StaleEndpoints), result.StaleEndpoints)
} }
for _, x := range tc.expectedStaleEndpoints { for _, x := range tc.expectedStaleEndpoints {
if result.staleEndpoints[x] != true { found := false
t.Errorf("[%d] expected staleEndpoints[%v], but didn't find it: %v", tci, x, result.staleEndpoints) for _, stale := range result.StaleEndpoints {
if stale == x {
found = true
break
} }
} }
if len(result.staleServiceNames) != len(tc.expectedStaleServiceNames) { if !found {
t.Errorf("[%d] expected %d staleServiceNames, got %d: %v", tci, len(tc.expectedStaleServiceNames), len(result.staleServiceNames), result.staleServiceNames) t.Errorf("[%d] expected staleEndpoints[%v], but didn't find it: %v", tci, x, result.StaleEndpoints)
}
}
if len(result.StaleServiceNames) != len(tc.expectedStaleServiceNames) {
t.Errorf("[%d] expected %d staleServiceNames, got %d: %v", tci, len(tc.expectedStaleServiceNames), len(result.StaleServiceNames), result.StaleServiceNames)
} }
for svcName := range tc.expectedStaleServiceNames { for svcName := range tc.expectedStaleServiceNames {
if result.staleServiceNames[svcName] != true { found := false
t.Errorf("[%d] expected staleServiceNames[%v], but didn't find it: %v", tci, svcName, result.staleServiceNames) for _, stale := range result.StaleServiceNames {
if stale == svcName {
found = true
break
} }
} }
if !reflect.DeepEqual(result.hcEndpoints, tc.expectedHealthchecks) { if !found {
t.Errorf("[%d] expected healthchecks %v, got %v", tci, tc.expectedHealthchecks, result.hcEndpoints) t.Errorf("[%d] expected staleServiceNames[%v], but didn't find it: %v", tci, svcName, result.StaleServiceNames)
}
}
if !reflect.DeepEqual(result.HCEndpointsLocalIPSize, tc.expectedHealthchecks) {
t.Errorf("[%d] expected healthchecks %v, got %v", tci, tc.expectedHealthchecks, result.HCEndpointsLocalIPSize)
} }
} }
} }
func compareEndpointsMaps(t *testing.T, tci int, newMap, expected map[proxy.ServicePortName][]*endpointsInfo) { func compareEndpointsMaps(t *testing.T, tci int, newMap proxy.EndpointsMap, expected map[proxy.ServicePortName][]*endpointsInfo) {
if len(newMap) != len(expected) { if len(newMap) != len(expected) {
t.Errorf("[%d] expected %d results, got %d: %v", tci, len(expected), len(newMap), newMap) t.Errorf("[%d] expected %d results, got %d: %v", tci, len(expected), len(newMap), newMap)
} }
for x := range expected { for x := range expected {
if len(newMap[x]) != len(expected[x]) { if len(newMap[x]) != len(expected[x]) {
t.Errorf("[%d] expected %d Endpoints for %v, got %d", tci, len(expected[x]), x, len(newMap[x])) t.Errorf("[%d] expected %d endpoints for %v, got %d", tci, len(expected[x]), x, len(newMap[x]))
} else { } else {
for i := range expected[x] { for i := range expected[x] {
if *(newMap[x][i]) != *(expected[x][i]) { newEp, ok := newMap[x][i].(*endpointsInfo)
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, expected[x][i], newMap[x][i]) if !ok {
} t.Errorf("Failed to cast endpointsInfo")
} continue
}
}
}
func Test_getLocalIPs(t *testing.T) {
testCases := []struct {
endpointsMap map[proxy.ServicePortName][]*endpointsInfo
expected map[types.NamespacedName]sets.String
}{{
// Case[0]: nothing
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{},
expected: map[types.NamespacedName]sets.String{},
}, {
// Case[1]: unnamed port
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{"1.1.1.1:11", false},
},
},
expected: map[types.NamespacedName]sets.String{},
}, {
// Case[2]: unnamed port local
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{"1.1.1.1:11", true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns1", Name: "ep1"}: sets.NewString("1.1.1.1"),
},
}, {
// Case[3]: named local and non-local ports for the same IP.
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{"1.1.1.1:11", false},
{"1.1.1.2:11", true},
},
makeServicePortName("ns1", "ep1", "p12"): {
{"1.1.1.1:12", false},
{"1.1.1.2:12", true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns1", Name: "ep1"}: sets.NewString("1.1.1.2"),
},
}, {
// Case[4]: named local and non-local ports for different IPs.
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{"1.1.1.1:11", false},
},
makeServicePortName("ns2", "ep2", "p22"): {
{"2.2.2.2:22", true},
{"2.2.2.22:22", true},
},
makeServicePortName("ns2", "ep2", "p23"): {
{"2.2.2.3:23", true},
},
makeServicePortName("ns4", "ep4", "p44"): {
{"4.4.4.4:44", true},
{"4.4.4.5:44", false},
},
makeServicePortName("ns4", "ep4", "p45"): {
{"4.4.4.6:45", true},
},
},
expected: map[types.NamespacedName]sets.String{
{Namespace: "ns2", Name: "ep2"}: sets.NewString("2.2.2.2", "2.2.2.22", "2.2.2.3"),
{Namespace: "ns4", Name: "ep4"}: sets.NewString("4.4.4.4", "4.4.4.6"),
},
}, {
// Case[5]: named port local and bad endpoints IP
endpointsMap: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p11"): {
{endpoint: "bad ip:11", isLocal: true},
},
},
expected: map[types.NamespacedName]sets.String{},
}}
for tci, tc := range testCases {
// outputs
localIPs := getLocalIPs(tc.endpointsMap)
if !reflect.DeepEqual(localIPs, tc.expected) {
t.Errorf("[%d] expected %#v, got %#v", tci, tc.expected, localIPs)
}
}
}
// This is a coarse test, but it offers some modicum of confidence as the code is evolved.
func Test_endpointsToEndpointsMap(t *testing.T) {
testCases := []struct {
newEndpoints *api.Endpoints
expected map[proxy.ServicePortName][]*endpointsInfo
}{{
// Case[0]: nothing
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {}),
expected: map[proxy.ServicePortName][]*endpointsInfo{},
}, {
// Case[1]: no changes, unnamed port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{"1.1.1.1:11", false},
},
},
}, {
// Case[2]: no changes, named port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "port",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "port"): {
{"1.1.1.1:11", false},
},
},
}, {
// Case[3]: new port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", ""): {
{"1.1.1.1:11", false},
},
},
}, {
// Case[4]: remove port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {}),
expected: map[proxy.ServicePortName][]*endpointsInfo{},
}, {
// Case[5]: new IP and port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}, {
IP: "2.2.2.2",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 11,
}, {
Name: "p2",
Port: 22,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{"1.1.1.1:11", false},
{"2.2.2.2:11", false},
},
makeServicePortName("ns1", "ep1", "p2"): {
{"1.1.1.1:22", false},
{"2.2.2.2:22", false},
},
},
}, {
// Case[6]: remove IP and port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{"1.1.1.1:11", false},
},
},
}, {
// Case[7]: rename port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p2",
Port: 11,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p2"): {
{"1.1.1.1:11", false},
},
},
}, {
// Case[8]: renumber port
newEndpoints: makeTestEndpoints("ns1", "ep1", func(ept *api.Endpoints) {
ept.Subsets = []api.EndpointSubset{
{
Addresses: []api.EndpointAddress{{
IP: "1.1.1.1",
}},
Ports: []api.EndpointPort{{
Name: "p1",
Port: 22,
}},
},
}
}),
expected: map[proxy.ServicePortName][]*endpointsInfo{
makeServicePortName("ns1", "ep1", "p1"): {
{"1.1.1.1:22", false},
},
},
}}
for tci, tc := range testCases {
// outputs
newEndpoints := endpointsToEndpointsMap(tc.newEndpoints, "host")
if len(newEndpoints) != len(tc.expected) {
t.Errorf("[%d] expected %d new, got %d: %v", tci, len(tc.expected), len(newEndpoints), spew.Sdump(newEndpoints))
}
for x := range tc.expected {
if len(newEndpoints[x]) != len(tc.expected[x]) {
t.Errorf("[%d] expected %d endpoints for %v, got %d", tci, len(tc.expected[x]), x, len(newEndpoints[x]))
} else {
for i := range newEndpoints[x] {
if *(newEndpoints[x][i]) != *(tc.expected[x][i]) {
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, tc.expected[x][i], *(newEndpoints[x][i]))
} }
if *newEp != *(expected[x][i]) {
t.Errorf("[%d] expected new[%v][%d] to be %v, got %v", tci, x, i, expected[x][i], newEp)
} }
} }
} }

224
pkg/proxy/service.go Normal file
View File

@@ -0,0 +1,224 @@
/*
Copyright 2017 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package proxy
import (
"reflect"
"sync"
"github.com/golang/glog"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/sets"
api "k8s.io/kubernetes/pkg/apis/core"
proxyutil "k8s.io/kubernetes/pkg/proxy/util"
)
// serviceChange contains all changes to services that happened since proxy rules were synced. For a single object,
// changes are accumulated, i.e. previous is state from before applying the changes,
// current is state after applying all of the changes.
type serviceChange struct {
previous ServiceMap
current ServiceMap
}
// ServiceChangeTracker carries state about uncommitted changes to an arbitrary number of
// Services, keyed by their namespace and name.
type ServiceChangeTracker struct {
// lock protects items.
lock sync.Mutex
// items maps a service to its serviceChange.
items map[types.NamespacedName]*serviceChange
}
// NewServiceChangeTracker initializes a ServiceChangeTracker
func NewServiceChangeTracker() *ServiceChangeTracker {
return &ServiceChangeTracker{
items: make(map[types.NamespacedName]*serviceChange),
}
}
// Update updates given service's change map based on the <previous, current> service pair. It returns true if items changed,
// otherwise return false. Update can be used to add/update/delete items of ServiceChangeMap. For example,
// Add item
// - pass <nil, service> as the <previous, current> pair.
// Update item
// - pass <oldService, service> as the <previous, current> pair.
// Delete item
// - pass <service, nil> as the <previous, current> pair.
//
// makeServicePort() return a proxy.ServicePort based on the given Service and its ServicePort. We inject makeServicePort()
// so that giving caller side a chance to initialize proxy.ServicePort interface.
func (sct *ServiceChangeTracker) Update(previous, current *api.Service, makeServicePort func(servicePort *api.ServicePort, service *api.Service) ServicePort) bool {
svc := current
if svc == nil {
svc = previous
}
// previous == nil && current == nil is unexpected, we should return false directly.
if svc == nil {
return false
}
namespacedName := types.NamespacedName{Namespace: svc.Namespace, Name: svc.Name}
sct.lock.Lock()
defer sct.lock.Unlock()
change, exists := sct.items[namespacedName]
if !exists {
change = &serviceChange{}
change.previous = serviceToServiceMap(previous, makeServicePort)
sct.items[namespacedName] = change
}
change.current = serviceToServiceMap(current, makeServicePort)
// if change.previous equal to change.current, it means no change
if reflect.DeepEqual(change.previous, change.current) {
delete(sct.items, namespacedName)
}
return len(sct.items) > 0
}
// UpdateServiceMapResult is the updated results after applying service changes.
type UpdateServiceMapResult struct {
// HCServiceNodePorts is a map of Service names to node port numbers which indicate the health of that Service on this Node.
// The value(uint16) of HCServices map is the service health check node port.
HCServiceNodePorts map[types.NamespacedName]uint16
// UDPStaleClusterIP holds stale (no longer assigned to a Service) Service IPs that had UDP ports.
// Callers can use this to abort timeout-waits or clear connection-tracking information.
UDPStaleClusterIP sets.String
}
// UpdateServiceMap updates ServiceMap based on the given changes.
func UpdateServiceMap(serviceMap ServiceMap, changes *ServiceChangeTracker) (result UpdateServiceMapResult) {
result.UDPStaleClusterIP = sets.NewString()
serviceMap.apply(changes, result.UDPStaleClusterIP)
// TODO: If this will appear to be computationally expensive, consider
// computing this incrementally similarly to serviceMap.
result.HCServiceNodePorts = make(map[types.NamespacedName]uint16)
for svcPortName, info := range serviceMap {
if info.HealthCheckNodePort() != 0 {
result.HCServiceNodePorts[svcPortName.NamespacedName] = uint16(info.HealthCheckNodePort())
}
}
return result
}
// ServiceMap maps a service to its ServicePort information.
type ServiceMap map[ServicePortName]ServicePort
// serviceToServiceMap translates a single Service object to a ServiceMap.
// makeServicePort() return a proxy.ServicePort based on the given Service and its ServicePort. We inject makeServicePort()
// so that giving caller side a chance to initialize proxy.ServicePort interface.
//
// NOTE: service object should NOT be modified.
func serviceToServiceMap(service *api.Service, makeServicePort func(servicePort *api.ServicePort, service *api.Service) ServicePort) ServiceMap {
if service == nil {
return nil
}
svcName := types.NamespacedName{Namespace: service.Namespace, Name: service.Name}
if proxyutil.ShouldSkipService(svcName, service) {
return nil
}
serviceMap := make(ServiceMap)
for i := range service.Spec.Ports {
servicePort := &service.Spec.Ports[i]
svcPortName := ServicePortName{NamespacedName: svcName, Port: servicePort.Name}
serviceMap[svcPortName] = makeServicePort(servicePort, service)
}
return serviceMap
}
// apply the changes to ServiceMap and update the stale udp cluster IP set. The UDPStaleClusterIP argument is passed in to store the
// udp protocol service cluster ip when service is deleted from the ServiceMap.
func (serviceMap *ServiceMap) apply(changes *ServiceChangeTracker, UDPStaleClusterIP sets.String) {
changes.lock.Lock()
defer changes.lock.Unlock()
for _, change := range changes.items {
serviceMap.merge(change.current)
// filter out the Update event of current changes from previous changes before calling unmerge() so that can
// skip deleting the Update events.
change.previous.filter(change.current)
serviceMap.unmerge(change.previous, UDPStaleClusterIP)
}
// clear changes after applying them to ServiceMap.
changes.items = make(map[types.NamespacedName]*serviceChange)
return
}
// merge adds other ServiceMap's elements to current ServiceMap.
// If collision, other ALWAYS win. Otherwise add the other to current.
// In other words, if some elements in current collisions with other, update the current by other.
// It returns a string type set which stores all the newly merged services' identifier, ServicePortName.String(), to help users
// tell if a service is deleted or updated.
// The returned value is one of the arguments of ServiceMap.unmerge().
// ServiceMap A Merge ServiceMap B will do following 2 things:
// * update ServiceMap A.
// * produce a string set which stores all other ServiceMap's ServicePortName.String().
// For example,
// - A{}
// - B{{"ns", "cluster-ip", "http"}: {"172.16.55.10", 1234, "TCP"}}
// - A updated to be {{"ns", "cluster-ip", "http"}: {"172.16.55.10", 1234, "TCP"}}
// - produce string set {"ns/cluster-ip:http"}
// - A{{"ns", "cluster-ip", "http"}: {"172.16.55.10", 345, "UDP"}}
// - B{{"ns", "cluster-ip", "http"}: {"172.16.55.10", 1234, "TCP"}}
// - A updated to be {{"ns", "cluster-ip", "http"}: {"172.16.55.10", 1234, "TCP"}}
// - produce string set {"ns/cluster-ip:http"}
func (sm *ServiceMap) merge(other ServiceMap) sets.String {
// existingPorts is going to store all identifiers of all services in `other` ServiceMap.
existingPorts := sets.NewString()
for svcPortName, info := range other {
// Take ServicePortName.String() as the newly merged service's identifier and put it into existingPorts.
existingPorts.Insert(svcPortName.String())
_, exists := (*sm)[svcPortName]
if !exists {
glog.V(1).Infof("Adding new service port %q at %s", svcPortName, info.String())
} else {
glog.V(1).Infof("Updating existing service port %q at %s", svcPortName, info.String())
}
(*sm)[svcPortName] = info
}
return existingPorts
}
// filter filters out elements from ServiceMap base on given ports string sets.
func (sm *ServiceMap) filter(other ServiceMap) {
for svcPortName := range *sm {
// skip the delete for Update event.
if _, ok := other[svcPortName]; ok {
delete(*sm, svcPortName)
}
}
}
// unmerge deletes all other ServiceMap's elements from current ServiceMap. We pass in the UDPStaleClusterIP strings sets
// for storing the stale udp service cluster IPs. We will clear stale udp connection base on UDPStaleClusterIP later
func (sm *ServiceMap) unmerge(other ServiceMap, UDPStaleClusterIP sets.String) {
for svcPortName := range other {
info, exists := (*sm)[svcPortName]
if exists {
glog.V(1).Infof("Removing service port %q", svcPortName)
if info.Protocol() == api.ProtocolUDP {
UDPStaleClusterIP.Insert(info.ClusterIP())
}
delete(*sm, svcPortName)
} else {
glog.Errorf("Service port %q doesn't exists", svcPortName)
}
}
}

531
pkg/proxy/service_test.go Normal file
View File

@@ -0,0 +1,531 @@
/*
Copyright 2017 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package proxy
import (
"fmt"
"net"
"reflect"
"testing"
"github.com/davecgh/go-spew/spew"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/intstr"
apiservice "k8s.io/kubernetes/pkg/api/service"
api "k8s.io/kubernetes/pkg/apis/core"
)
const testHostname = "test-hostname"
// fake implementation for service info.
type fakeServiceInfo struct {
clusterIP net.IP
port int
protocol api.Protocol
healthCheckNodePort int
}
func (f *fakeServiceInfo) String() string {
return fmt.Sprintf("%s:%d/%s", f.clusterIP, f.port, f.protocol)
}
func (f *fakeServiceInfo) ClusterIP() string {
return f.clusterIP.String()
}
func (f *fakeServiceInfo) Protocol() api.Protocol {
return f.protocol
}
func (f *fakeServiceInfo) HealthCheckNodePort() int {
return f.healthCheckNodePort
}
func makeTestServiceInfo(clusterIP string, port int, protocol string, healthcheckNodePort int) *fakeServiceInfo {
info := &fakeServiceInfo{
clusterIP: net.ParseIP(clusterIP),
port: port,
protocol: api.Protocol(protocol),
}
if healthcheckNodePort != 0 {
info.healthCheckNodePort = healthcheckNodePort
}
return info
}
func newFakeServiceInfo(servicePort *api.ServicePort, service *api.Service) ServicePort {
info := &fakeServiceInfo{
clusterIP: net.ParseIP(service.Spec.ClusterIP),
port: int(servicePort.Port),
protocol: servicePort.Protocol,
}
if apiservice.NeedsHealthCheck(service) {
p := service.Spec.HealthCheckNodePort
if p != 0 {
info.healthCheckNodePort = int(p)
}
}
return info
}
func makeTestService(namespace, name string, svcFunc func(*api.Service)) *api.Service {
svc := &api.Service{
ObjectMeta: metav1.ObjectMeta{
Name: name,
Namespace: namespace,
Annotations: map[string]string{},
},
Spec: api.ServiceSpec{},
Status: api.ServiceStatus{},
}
svcFunc(svc)
return svc
}
func addTestPort(array []api.ServicePort, name string, protocol api.Protocol, port, nodeport int32, targetPort int) []api.ServicePort {
svcPort := api.ServicePort{
Name: name,
Protocol: protocol,
Port: port,
NodePort: nodeport,
TargetPort: intstr.FromInt(targetPort),
}
return append(array, svcPort)
}
func makeNSN(namespace, name string) types.NamespacedName {
return types.NamespacedName{Namespace: namespace, Name: name}
}
func makeServicePortName(ns, name, port string) ServicePortName {
return ServicePortName{
NamespacedName: makeNSN(ns, name),
Port: port,
}
}
func Test_serviceToServiceMap(t *testing.T) {
testCases := []struct {
service *api.Service
expected map[ServicePortName]*fakeServiceInfo
}{
{
// Case[0]: nothing
service: nil,
expected: map[ServicePortName]*fakeServiceInfo{},
},
{
// Case[1]: headless service
service: makeTestService("ns2", "headless", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = api.ClusterIPNone
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "rpc", "UDP", 1234, 0, 0)
}),
expected: map[ServicePortName]*fakeServiceInfo{},
},
{
// Case[2]: headless service without port
service: makeTestService("ns2", "headless-without-port", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = api.ClusterIPNone
}),
expected: map[ServicePortName]*fakeServiceInfo{},
},
{
// Case[3]: cluster ip service
service: makeTestService("ns2", "cluster-ip", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = "172.16.55.4"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p1", "UDP", 1234, 4321, 0)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p2", "UDP", 1235, 5321, 0)
}),
expected: map[ServicePortName]*fakeServiceInfo{
makeServicePortName("ns2", "cluster-ip", "p1"): makeTestServiceInfo("172.16.55.4", 1234, "UDP", 0),
makeServicePortName("ns2", "cluster-ip", "p2"): makeTestServiceInfo("172.16.55.4", 1235, "UDP", 0),
},
},
{
// Case[4]: nodeport service
service: makeTestService("ns2", "node-port", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeNodePort
svc.Spec.ClusterIP = "172.16.55.10"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port1", "UDP", 345, 678, 0)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port2", "TCP", 344, 677, 0)
}),
expected: map[ServicePortName]*fakeServiceInfo{
makeServicePortName("ns2", "node-port", "port1"): makeTestServiceInfo("172.16.55.10", 345, "UDP", 0),
makeServicePortName("ns2", "node-port", "port2"): makeTestServiceInfo("172.16.55.10", 344, "TCP", 0),
},
},
{
// Case[5]: load balancer service
service: makeTestService("ns1", "load-balancer", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeLoadBalancer
svc.Spec.ClusterIP = "172.16.55.11"
svc.Spec.LoadBalancerIP = "5.6.7.8"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port3", "UDP", 8675, 30061, 7000)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port4", "UDP", 8676, 30062, 7001)
svc.Status.LoadBalancer = api.LoadBalancerStatus{
Ingress: []api.LoadBalancerIngress{
{IP: "10.1.2.4"},
},
}
}),
expected: map[ServicePortName]*fakeServiceInfo{
makeServicePortName("ns1", "load-balancer", "port3"): makeTestServiceInfo("172.16.55.11", 8675, "UDP", 0),
makeServicePortName("ns1", "load-balancer", "port4"): makeTestServiceInfo("172.16.55.11", 8676, "UDP", 0),
},
},
{
// Case[6]: load balancer service with only local traffic policy
service: makeTestService("ns1", "only-local-load-balancer", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeLoadBalancer
svc.Spec.ClusterIP = "172.16.55.12"
svc.Spec.LoadBalancerIP = "5.6.7.8"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "portx", "UDP", 8677, 30063, 7002)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "porty", "UDP", 8678, 30064, 7003)
svc.Status.LoadBalancer = api.LoadBalancerStatus{
Ingress: []api.LoadBalancerIngress{
{IP: "10.1.2.3"},
},
}
svc.Spec.ExternalTrafficPolicy = api.ServiceExternalTrafficPolicyTypeLocal
svc.Spec.HealthCheckNodePort = 345
}),
expected: map[ServicePortName]*fakeServiceInfo{
makeServicePortName("ns1", "only-local-load-balancer", "portx"): makeTestServiceInfo("172.16.55.12", 8677, "UDP", 345),
makeServicePortName("ns1", "only-local-load-balancer", "porty"): makeTestServiceInfo("172.16.55.12", 8678, "UDP", 345),
},
},
{
// Case[7]: external name service
service: makeTestService("ns2", "external-name", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeExternalName
svc.Spec.ClusterIP = "172.16.55.4" // Should be ignored
svc.Spec.ExternalName = "foo2.bar.com"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "portz", "UDP", 1235, 5321, 0)
}),
expected: map[ServicePortName]*fakeServiceInfo{},
},
}
for tci, tc := range testCases {
// outputs
newServices := serviceToServiceMap(tc.service, newFakeServiceInfo)
if len(newServices) != len(tc.expected) {
t.Errorf("[%d] expected %d new, got %d: %v", tci, len(tc.expected), len(newServices), spew.Sdump(newServices))
}
for x := range tc.expected {
svc := newServices[x].(*fakeServiceInfo)
if !reflect.DeepEqual(svc, tc.expected[x]) {
t.Errorf("[%d] expected new[%v]to be %v, got %v", tci, x, tc.expected[x], *svc)
}
}
}
}
type FakeProxier struct {
endpointsChanges *EndpointChangeTracker
serviceChanges *ServiceChangeTracker
serviceMap ServiceMap
endpointsMap EndpointsMap
hostname string
}
func newFakeProxier() *FakeProxier {
return &FakeProxier{
serviceMap: make(ServiceMap),
serviceChanges: NewServiceChangeTracker(),
endpointsMap: make(EndpointsMap),
endpointsChanges: NewEndpointChangeTracker(testHostname),
}
}
func makeServiceMap(fake *FakeProxier, allServices ...*api.Service) {
for i := range allServices {
fake.addService(allServices[i])
}
}
func (fake *FakeProxier) addService(service *api.Service) {
fake.serviceChanges.Update(nil, service, makeServicePort)
}
func (fake *FakeProxier) updateService(oldService *api.Service, service *api.Service) {
fake.serviceChanges.Update(oldService, service, makeServicePort)
}
func (fake *FakeProxier) deleteService(service *api.Service) {
fake.serviceChanges.Update(service, nil, makeServicePort)
}
func makeServicePort(port *api.ServicePort, service *api.Service) ServicePort {
info := &fakeServiceInfo{
clusterIP: net.ParseIP(service.Spec.ClusterIP),
port: int(port.Port),
protocol: port.Protocol,
}
if apiservice.NeedsHealthCheck(service) {
p := service.Spec.HealthCheckNodePort
if p != 0 {
info.healthCheckNodePort = int(p)
}
}
return info
}
func TestUpdateServiceMapHeadless(t *testing.T) {
fp := newFakeProxier()
makeServiceMap(fp,
makeTestService("ns2", "headless", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = api.ClusterIPNone
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "rpc", "UDP", 1234, 0, 0)
}),
makeTestService("ns2", "headless-without-port", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = api.ClusterIPNone
}),
)
// Headless service should be ignored
result := UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %d", len(fp.serviceMap))
}
// No proxied services, so no healthchecks
if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %d", len(result.HCServiceNodePorts))
}
if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
}
}
func TestUpdateServiceTypeExternalName(t *testing.T) {
fp := newFakeProxier()
makeServiceMap(fp,
makeTestService("ns2", "external-name", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeExternalName
svc.Spec.ClusterIP = "172.16.55.4" // Should be ignored
svc.Spec.ExternalName = "foo2.bar.com"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "blah", "UDP", 1235, 5321, 0)
}),
)
result := UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 0 {
t.Errorf("expected service map length 0, got %v", fp.serviceMap)
}
// No proxied services, so no healthchecks
if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
}
if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP)
}
}
func TestBuildServiceMapAddRemove(t *testing.T) {
fp := newFakeProxier()
services := []*api.Service{
makeTestService("ns2", "cluster-ip", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = "172.16.55.4"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port1", "UDP", 1234, 4321, 0)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port2", "UDP", 1235, 5321, 0)
}),
makeTestService("ns2", "node-port", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeNodePort
svc.Spec.ClusterIP = "172.16.55.10"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port1", "UDP", 345, 678, 0)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "port2", "TCP", 344, 677, 0)
}),
makeTestService("ns1", "load-balancer", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeLoadBalancer
svc.Spec.ClusterIP = "172.16.55.11"
svc.Spec.LoadBalancerIP = "5.6.7.8"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "foobar", "UDP", 8675, 30061, 7000)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "baz", "UDP", 8676, 30062, 7001)
svc.Status.LoadBalancer = api.LoadBalancerStatus{
Ingress: []api.LoadBalancerIngress{
{IP: "10.1.2.4"},
},
}
}),
makeTestService("ns1", "only-local-load-balancer", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeLoadBalancer
svc.Spec.ClusterIP = "172.16.55.12"
svc.Spec.LoadBalancerIP = "5.6.7.8"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "foobar2", "UDP", 8677, 30063, 7002)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "baz", "UDP", 8678, 30064, 7003)
svc.Status.LoadBalancer = api.LoadBalancerStatus{
Ingress: []api.LoadBalancerIngress{
{IP: "10.1.2.3"},
},
}
svc.Spec.ExternalTrafficPolicy = api.ServiceExternalTrafficPolicyTypeLocal
svc.Spec.HealthCheckNodePort = 345
}),
}
for i := range services {
fp.addService(services[i])
}
result := UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 8 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap)
}
// The only-local-loadbalancer ones get added
if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected 1 healthcheck port, got %v", result.HCServiceNodePorts)
} else {
nsn := makeNSN("ns1", "only-local-load-balancer")
if port, found := result.HCServiceNodePorts[nsn]; !found || port != 345 {
t.Errorf("expected healthcheck port [%q]=345: got %v", nsn, result.HCServiceNodePorts)
}
}
if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
}
// Remove some stuff
// oneService is a modification of services[0] with removed first port.
oneService := makeTestService("ns2", "cluster-ip", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = "172.16.55.4"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p2", "UDP", 1235, 5321, 0)
})
fp.updateService(services[0], oneService)
fp.deleteService(services[1])
fp.deleteService(services[2])
fp.deleteService(services[3])
result = UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 1 {
t.Errorf("expected service map length 1, got %v", fp.serviceMap)
}
if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected 0 healthcheck ports, got %v", result.HCServiceNodePorts)
}
// All services but one were deleted. While you'd expect only the ClusterIPs
// from the three deleted services here, we still have the ClusterIP for
// the not-deleted service, because one of it's ServicePorts was deleted.
expectedStaleUDPServices := []string{"172.16.55.10", "172.16.55.4", "172.16.55.11", "172.16.55.12"}
if len(result.UDPStaleClusterIP) != len(expectedStaleUDPServices) {
t.Errorf("expected stale UDP services length %d, got %v", len(expectedStaleUDPServices), result.UDPStaleClusterIP.UnsortedList())
}
for _, ip := range expectedStaleUDPServices {
if !result.UDPStaleClusterIP.Has(ip) {
t.Errorf("expected stale UDP service service %s", ip)
}
}
}
func TestBuildServiceMapServiceUpdate(t *testing.T) {
fp := newFakeProxier()
servicev1 := makeTestService("ns1", "svc1", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeClusterIP
svc.Spec.ClusterIP = "172.16.55.4"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p1", "UDP", 1234, 4321, 0)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p2", "TCP", 1235, 5321, 0)
})
servicev2 := makeTestService("ns1", "svc1", func(svc *api.Service) {
svc.Spec.Type = api.ServiceTypeLoadBalancer
svc.Spec.ClusterIP = "172.16.55.4"
svc.Spec.LoadBalancerIP = "5.6.7.8"
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p1", "UDP", 1234, 4321, 7002)
svc.Spec.Ports = addTestPort(svc.Spec.Ports, "p2", "TCP", 1235, 5321, 7003)
svc.Status.LoadBalancer = api.LoadBalancerStatus{
Ingress: []api.LoadBalancerIngress{
{IP: "10.1.2.3"},
},
}
svc.Spec.ExternalTrafficPolicy = api.ServiceExternalTrafficPolicyTypeLocal
svc.Spec.HealthCheckNodePort = 345
})
fp.addService(servicev1)
result := UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap)
}
if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
}
if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
}
// Change service to load-balancer
fp.updateService(servicev1, servicev2)
result = UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap)
}
if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
}
if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.UnsortedList())
}
// No change; make sure the service map stays the same and there are
// no health-check changes
fp.updateService(servicev2, servicev2)
result = UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap)
}
if len(result.HCServiceNodePorts) != 1 {
t.Errorf("expected healthcheck ports length 1, got %v", result.HCServiceNodePorts)
}
if len(result.UDPStaleClusterIP) != 0 {
t.Errorf("expected stale UDP services length 0, got %v", result.UDPStaleClusterIP.UnsortedList())
}
// And back to ClusterIP
fp.updateService(servicev2, servicev1)
result = UpdateServiceMap(fp.serviceMap, fp.serviceChanges)
if len(fp.serviceMap) != 2 {
t.Errorf("expected service map length 2, got %v", fp.serviceMap)
}
if len(result.HCServiceNodePorts) != 0 {
t.Errorf("expected healthcheck ports length 0, got %v", result.HCServiceNodePorts)
}
if len(result.UDPStaleClusterIP) != 0 {
// Services only added, so nothing stale yet
t.Errorf("expected stale UDP services length 0, got %d", len(result.UDPStaleClusterIP))
}
}

View File

@@ -20,11 +20,12 @@ import (
"fmt" "fmt"
"k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/types"
api "k8s.io/kubernetes/pkg/apis/core"
) )
// ProxyProvider is the interface provided by proxier implementations. // ProxyProvider is the interface provided by proxier implementations.
type ProxyProvider interface { type ProxyProvider interface {
// Sync immediately synchronizes the ProxyProvider's current state to iptables. // Sync immediately synchronizes the ProxyProvider's current state to proxy rules.
Sync() Sync()
// SyncLoop runs periodic work. // SyncLoop runs periodic work.
// This is expected to run as a goroutine or as the main loop of the app. // This is expected to run as a goroutine or as the main loop of the app.
@@ -33,7 +34,7 @@ type ProxyProvider interface {
} }
// ServicePortName carries a namespace + name + portname. This is the unique // ServicePortName carries a namespace + name + portname. This is the unique
// identfier for a load-balanced service. // identifier for a load-balanced service.
type ServicePortName struct { type ServicePortName struct {
types.NamespacedName types.NamespacedName
Port string Port string
@@ -42,3 +43,34 @@ type ServicePortName struct {
func (spn ServicePortName) String() string { func (spn ServicePortName) String() string {
return fmt.Sprintf("%s:%s", spn.NamespacedName.String(), spn.Port) return fmt.Sprintf("%s:%s", spn.NamespacedName.String(), spn.Port)
} }
// ServicePort is an interface which abstracts information about a service.
type ServicePort interface {
// String returns service string. An example format can be: `IP:Port/Protocol`.
String() string
// ClusterIP returns service cluster IP.
ClusterIP() string
// Protocol returns service protocol.
Protocol() api.Protocol
// HealthCheckNodePort returns service health check node port if present. If return 0, it means not present.
HealthCheckNodePort() int
}
// Endpoint in an interface which abstracts information about an endpoint.
type Endpoint interface {
// String returns endpoint string. An example format can be: `IP:Port`.
// We take the returned value as ServiceEndpoint.Endpoint.
String() string
// IsLocal returns true if the endpoint is running in same host as kube-proxy, otherwise returns false.
IsLocal() bool
// IP returns IP part of endpoints.
IP() string
// Equal checks if two endpoints are equal.
Equal(Endpoint) bool
}
// ServiceEndpoint is used to identify a service and one of its endpoint pair.
type ServiceEndpoint struct {
Endpoint string
ServicePortName ServicePortName
}

View File

@@ -47,6 +47,7 @@ func IPPart(s string) string {
return "" return ""
} }
// PortPart returns just the port part of an endpoint string.
func PortPart(s string) (int, error) { func PortPart(s string) (int, error) {
// Must be IP:port // Must be IP:port
_, port, err := net.SplitHostPort(s) _, port, err := net.SplitHostPort(s)