mirror of https://github.com/linkerd/linkerd2.git
dst: Update `GetProfile`'s stream when pod associated to HostPort lookup changes (#11334)
Followup to #11328 Implements a new pod watcher, instantiated along the other ones in the Destination server. It also watches on Servers and carries all the logic from ServerWatcher, which has now been decommissioned. The `CreateAddress()` function has been moved into a function of the PodWatcher, because now we're calling it on every update given the pod associated to an ip:port might change and we need to regenerate the Address object. That function also takes care of capturing opaque protocol info from associated Servers, which is not new and had some logic that was duped in the now defunct ServerWatcher. `getAnnotatedOpaquePorts()` got also moved for similar reasons. Other things to note about PodWatcher: - It publishes a new pair of metrics `ip_port_subscribers` and `ip_port_updates` leveraging the framework in `prometheus.go`. - The complexity in `updatePod()` is due to only send stream updates when there are changes in the pod's readiness, to avoid sending duped messages on every pod lifecycle event. - Finally, endpointProfileTranslator's `endpoint` (*pb.WeightedAddr) not being a static object anymore, the `Update()` function now receives an Address that allows it to rebuild the endpoint on the fly (and so `createEndpoint()` was converted into a method of endpointProfileTranslator).
This commit is contained in:
parent
0244282d96
commit
65ddba4e5d
|
@ -1,65 +1,107 @@
|
|||
package destination
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
pb "github.com/linkerd/linkerd2-proxy-api/go/destination"
|
||||
"github.com/sirupsen/logrus"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"github.com/linkerd/linkerd2/controller/api/destination/watcher"
|
||||
"github.com/linkerd/linkerd2/controller/k8s"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type endpointProfileTranslator struct {
|
||||
pod *v1.Pod
|
||||
port uint32
|
||||
endpoint *pb.WeightedAddr
|
||||
stream pb.Destination_GetProfileServer
|
||||
log *logrus.Entry
|
||||
enableH2Upgrade bool
|
||||
controllerNS string
|
||||
identityTrustDomain string
|
||||
defaultOpaquePorts map[uint32]struct{}
|
||||
stream pb.Destination_GetProfileServer
|
||||
lastMessage string
|
||||
|
||||
k8sAPI *k8s.API
|
||||
metadataAPI *k8s.MetadataAPI
|
||||
log *log.Entry
|
||||
}
|
||||
|
||||
// newEndpointProfileTranslator translates protocol updates to
|
||||
// DestinationProfiles for endpoints. When a Server on the cluster is updated
|
||||
// it is possible that it selects an endpoint that is being watched, if that
|
||||
// is the case then an update will be sent to the client if the Server has
|
||||
// changed the endpoint's supported protocol—mainly being opaque or not.
|
||||
func newEndpointProfileTranslator(pod *v1.Pod, port uint32, endpoint *pb.WeightedAddr, stream pb.Destination_GetProfileServer, log *logrus.Entry) *endpointProfileTranslator {
|
||||
// newEndpointProfileTranslator translates pod updates and protocol updates to
|
||||
// DestinationProfiles for endpoints
|
||||
func newEndpointProfileTranslator(
|
||||
enableH2Upgrade bool,
|
||||
controllerNS,
|
||||
identityTrustDomain string,
|
||||
defaultOpaquePorts map[uint32]struct{},
|
||||
stream pb.Destination_GetProfileServer,
|
||||
k8sAPI *k8s.API,
|
||||
metadataAPI *k8s.MetadataAPI,
|
||||
) *endpointProfileTranslator {
|
||||
return &endpointProfileTranslator{
|
||||
pod: pod,
|
||||
port: port,
|
||||
endpoint: endpoint,
|
||||
stream: stream,
|
||||
log: log,
|
||||
enableH2Upgrade: enableH2Upgrade,
|
||||
controllerNS: controllerNS,
|
||||
identityTrustDomain: identityTrustDomain,
|
||||
defaultOpaquePorts: defaultOpaquePorts,
|
||||
stream: stream,
|
||||
k8sAPI: k8sAPI,
|
||||
metadataAPI: metadataAPI,
|
||||
log: log.WithField("component", "endpoint-profile-translator"),
|
||||
}
|
||||
}
|
||||
|
||||
func (ept *endpointProfileTranslator) UpdateProtocol(opaqueProtocol bool) {
|
||||
// Update sends a DestinationProfile message into the stream, if the same
|
||||
// message hasn't been sent already. If it has, false is returned.
|
||||
func (ept *endpointProfileTranslator) Update(address *watcher.Address) (bool, error) {
|
||||
opaquePorts := watcher.GetAnnotatedOpaquePorts(address.Pod, ept.defaultOpaquePorts)
|
||||
endpoint, err := ept.createEndpoint(*address, opaquePorts)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("failed to create endpoint: %w", err)
|
||||
}
|
||||
|
||||
// The protocol for an endpoint should only be updated if there is a pod,
|
||||
// endpoint, and the endpoint has a protocol hint. If there is an endpoint
|
||||
// but it does not have a protocol hint, that means we could not determine
|
||||
// if it has a peer proxy so a opaque traffic would not be supported.
|
||||
if ept.pod != nil && ept.endpoint != nil && ept.endpoint.ProtocolHint != nil {
|
||||
if !opaqueProtocol {
|
||||
ept.endpoint.ProtocolHint.OpaqueTransport = nil
|
||||
} else if ept.endpoint.ProtocolHint.OpaqueTransport == nil {
|
||||
port, err := getInboundPort(&ept.pod.Spec)
|
||||
if address.Pod != nil && endpoint != nil && endpoint.ProtocolHint != nil {
|
||||
if !address.OpaqueProtocol {
|
||||
endpoint.ProtocolHint.OpaqueTransport = nil
|
||||
} else if endpoint.ProtocolHint.OpaqueTransport == nil {
|
||||
port, err := getInboundPort(&address.Pod.Spec)
|
||||
if err != nil {
|
||||
ept.log.Error(err)
|
||||
} else {
|
||||
ept.endpoint.ProtocolHint.OpaqueTransport = &pb.ProtocolHint_OpaqueTransport{
|
||||
InboundPort: port,
|
||||
}
|
||||
return false, err
|
||||
}
|
||||
|
||||
endpoint.ProtocolHint.OpaqueTransport = &pb.ProtocolHint_OpaqueTransport{
|
||||
InboundPort: port,
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
profile := ept.createDefaultProfile(opaqueProtocol)
|
||||
profile := &pb.DestinationProfile{
|
||||
RetryBudget: defaultRetryBudget(),
|
||||
Endpoint: endpoint,
|
||||
OpaqueProtocol: address.OpaqueProtocol,
|
||||
}
|
||||
msg := profile.String()
|
||||
if msg == ept.lastMessage {
|
||||
return false, nil
|
||||
}
|
||||
ept.lastMessage = msg
|
||||
ept.log.Debugf("sending protocol update: %+v", profile)
|
||||
if err := ept.stream.Send(profile); err != nil {
|
||||
ept.log.Errorf("failed to send protocol update: %s", err)
|
||||
return false, fmt.Errorf("failed to send protocol update: %w", err)
|
||||
}
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (ept *endpointProfileTranslator) createDefaultProfile(opaqueProtocol bool) *pb.DestinationProfile {
|
||||
return &pb.DestinationProfile{
|
||||
RetryBudget: defaultRetryBudget(),
|
||||
Endpoint: ept.endpoint,
|
||||
OpaqueProtocol: opaqueProtocol,
|
||||
func (ept *endpointProfileTranslator) createEndpoint(address watcher.Address, opaquePorts map[uint32]struct{}) (*pb.WeightedAddr, error) {
|
||||
weightedAddr, err := createWeightedAddr(address, opaquePorts, ept.enableH2Upgrade, ept.identityTrustDomain, ept.controllerNS, ept.log)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// `Get` doesn't include the namespace in the per-endpoint
|
||||
// metadata, so it needs to be special-cased.
|
||||
if address.Pod != nil {
|
||||
weightedAddr.MetricLabels["namespace"] = address.Pod.Namespace
|
||||
}
|
||||
|
||||
return weightedAddr, err
|
||||
}
|
||||
|
|
|
@ -273,7 +273,7 @@ func (et *endpointTranslator) sendClientAdd(set watcher.AddressSet) {
|
|||
err error
|
||||
)
|
||||
if address.Pod != nil {
|
||||
opaquePorts = getAnnotatedOpaquePorts(address.Pod, et.defaultOpaquePorts)
|
||||
opaquePorts = watcher.GetAnnotatedOpaquePorts(address.Pod, et.defaultOpaquePorts)
|
||||
wa, err = createWeightedAddr(address, opaquePorts, et.enableH2Upgrade, et.identityTrustDomain, et.controllerNS, et.log)
|
||||
} else {
|
||||
var authOverride *pb.AuthorityOverride
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package destination
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
|
@ -29,10 +28,10 @@ type (
|
|||
server struct {
|
||||
pb.UnimplementedDestinationServer
|
||||
|
||||
pods *watcher.PodWatcher
|
||||
endpoints *watcher.EndpointsWatcher
|
||||
opaquePorts *watcher.OpaquePortsWatcher
|
||||
profiles *watcher.ProfileWatcher
|
||||
servers *watcher.ServerWatcher
|
||||
|
||||
clusterStore *watcher.ClusterStore
|
||||
|
||||
|
@ -85,6 +84,10 @@ func NewServer(
|
|||
return nil, err
|
||||
}
|
||||
|
||||
pods, err := watcher.NewPodWatcher(k8sAPI, metadataAPI, log, defaultOpaquePorts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
endpoints, err := watcher.NewEndpointsWatcher(k8sAPI, metadataAPI, log, enableEndpointSlices, "local")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -97,17 +100,13 @@ func NewServer(
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
servers, err := watcher.NewServerWatcher(k8sAPI, log)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
srv := server{
|
||||
pb.UnimplementedDestinationServer{},
|
||||
pods,
|
||||
endpoints,
|
||||
opaquePorts,
|
||||
profiles,
|
||||
servers,
|
||||
clusterStore,
|
||||
enableH2Upgrade,
|
||||
controllerNS,
|
||||
|
@ -276,27 +275,7 @@ func (s *server) getProfileByIP(
|
|||
}
|
||||
|
||||
if svcID == nil {
|
||||
// If the IP does not map to a service, check if it maps to a pod
|
||||
var pod *corev1.Pod
|
||||
targetIP := ip.String()
|
||||
pod, err = getPodByPodIP(s.k8sAPI, targetIP, port, s.log)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pod != nil {
|
||||
targetIP = pod.Status.PodIP
|
||||
} else {
|
||||
pod, err = getPodByHostIP(s.k8sAPI, targetIP, port, s.log)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
address, err := s.createAddress(pod, targetIP, port)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create address: %w", err)
|
||||
}
|
||||
return s.subscribeToEndpointProfile(&address, port, stream)
|
||||
return s.subscribeToEndpointProfile(nil, "", ip.String(), port, stream)
|
||||
}
|
||||
|
||||
fqn := fmt.Sprintf("%s.%s.svc.%s", svcID.Name, svcID.Namespace, s.clusterDomain)
|
||||
|
@ -318,11 +297,7 @@ func (s *server) getProfileByName(
|
|||
// name. When we fetch the profile using a pod's DNS name, we want to
|
||||
// return an endpoint in the profile response.
|
||||
if hostname != "" {
|
||||
address, err := s.getEndpointByHostname(s.k8sAPI, hostname, service, port)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod for hostname %s: %w", hostname, err)
|
||||
}
|
||||
return s.subscribeToEndpointProfile(address, port, stream)
|
||||
return s.subscribeToEndpointProfile(&service, hostname, "", port, stream)
|
||||
}
|
||||
|
||||
return s.subscribeToServiceProfile(service, token, host, port, stream)
|
||||
|
@ -467,88 +442,37 @@ func (s *server) subscribeToServiceWithoutContext(
|
|||
//
|
||||
// This function does not return until the stream is closed.
|
||||
func (s *server) subscribeToEndpointProfile(
|
||||
address *watcher.Address,
|
||||
service *watcher.ServiceID,
|
||||
hostname,
|
||||
ip string,
|
||||
port uint32,
|
||||
stream pb.Destination_GetProfileServer,
|
||||
) error {
|
||||
log := s.log
|
||||
if address.Pod != nil {
|
||||
log = log.WithField("ns", address.Pod.Namespace).WithField("pod", address.Pod.Name)
|
||||
} else {
|
||||
log = log.WithField("ip", address.IP)
|
||||
}
|
||||
translator := newEndpointProfileTranslator(
|
||||
s.enableH2Upgrade,
|
||||
s.controllerNS,
|
||||
s.identityTrustDomain,
|
||||
s.defaultOpaquePorts,
|
||||
stream,
|
||||
s.k8sAPI,
|
||||
s.metadataAPI,
|
||||
)
|
||||
|
||||
opaquePorts := getAnnotatedOpaquePorts(address.Pod, s.defaultOpaquePorts)
|
||||
var endpoint *pb.WeightedAddr
|
||||
endpoint, err := s.createEndpoint(*address, opaquePorts)
|
||||
var err error
|
||||
ip, err = s.pods.Subscribe(service, hostname, ip, port, translator)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create endpoint: %w", err)
|
||||
}
|
||||
translator := newEndpointProfileTranslator(address.Pod, port, endpoint, stream, log)
|
||||
|
||||
// If the endpoint's port is annotated as opaque, we don't need to
|
||||
// subscribe for updates because it will always be opaque
|
||||
// regardless of any Servers that may select it.
|
||||
if _, ok := opaquePorts[port]; ok {
|
||||
translator.UpdateProtocol(true)
|
||||
} else if address.Pod == nil {
|
||||
translator.UpdateProtocol(false)
|
||||
} else {
|
||||
translator.UpdateProtocol(address.OpaqueProtocol)
|
||||
s.servers.Subscribe(address.Pod, port, translator)
|
||||
defer s.servers.Unsubscribe(address.Pod, port, translator)
|
||||
return err
|
||||
}
|
||||
defer s.pods.Unsubscribe(ip, port, translator)
|
||||
|
||||
select {
|
||||
case <-s.shutdown:
|
||||
case <-stream.Context().Done():
|
||||
log.Debugf("Cancelled")
|
||||
s.log.Debugf("Cancelled")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *server) createAddress(pod *corev1.Pod, targetIP string, port uint32) (watcher.Address, error) {
|
||||
var ownerKind, ownerName string
|
||||
var err error
|
||||
if pod != nil {
|
||||
ownerKind, ownerName, err = s.metadataAPI.GetOwnerKindAndName(context.Background(), pod, true)
|
||||
if err != nil {
|
||||
return watcher.Address{}, err
|
||||
}
|
||||
}
|
||||
|
||||
address := watcher.Address{
|
||||
IP: targetIP,
|
||||
Port: port,
|
||||
Pod: pod,
|
||||
OwnerName: ownerName,
|
||||
OwnerKind: ownerKind,
|
||||
}
|
||||
|
||||
if address.Pod != nil {
|
||||
if err := watcher.SetToServerProtocol(s.k8sAPI, &address, port); err != nil {
|
||||
return watcher.Address{}, fmt.Errorf("failed to set address OpaqueProtocol: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
return address, nil
|
||||
}
|
||||
|
||||
func (s *server) createEndpoint(address watcher.Address, opaquePorts map[uint32]struct{}) (*pb.WeightedAddr, error) {
|
||||
weightedAddr, err := createWeightedAddr(address, opaquePorts, s.enableH2Upgrade, s.identityTrustDomain, s.controllerNS, s.log)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// `Get` doesn't include the namespace in the per-endpoint
|
||||
// metadata, so it needs to be special-cased.
|
||||
if address.Pod != nil {
|
||||
weightedAddr.MetricLabels["namespace"] = address.Pod.Namespace
|
||||
}
|
||||
|
||||
return weightedAddr, err
|
||||
}
|
||||
|
||||
// getSvcID returns the service that corresponds to a Cluster IP address if one
|
||||
// exists.
|
||||
func getSvcID(k8sAPI *k8s.API, clusterIP string, log *logging.Entry) (*watcher.ServiceID, error) {
|
||||
|
@ -579,116 +503,6 @@ func getSvcID(k8sAPI *k8s.API, clusterIP string, log *logging.Entry) (*watcher.S
|
|||
return service, nil
|
||||
}
|
||||
|
||||
// getEndpointByHostname returns a pod that maps to the given hostname (or an
|
||||
// instanceID). The hostname is generally the prefix of the pod's DNS name;
|
||||
// since it may be arbitrary we need to look at the corresponding service's
|
||||
// Endpoints object to see whether the hostname matches a pod.
|
||||
func (s *server) getEndpointByHostname(k8sAPI *k8s.API, hostname string, svcID watcher.ServiceID, port uint32) (*watcher.Address, error) {
|
||||
ep, err := k8sAPI.Endpoint().Lister().Endpoints(svcID.Namespace).Get(svcID.Name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, subset := range ep.Subsets {
|
||||
for _, addr := range subset.Addresses {
|
||||
|
||||
if hostname == addr.Hostname {
|
||||
if addr.TargetRef != nil && addr.TargetRef.Kind == "Pod" {
|
||||
podName := addr.TargetRef.Name
|
||||
podNamespace := addr.TargetRef.Namespace
|
||||
pod, err := k8sAPI.Pod().Lister().Pods(podNamespace).Get(podName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
address, err := s.createAddress(pod, addr.IP, port)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &address, nil
|
||||
}
|
||||
return &watcher.Address{
|
||||
IP: addr.IP,
|
||||
Port: port,
|
||||
}, nil
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("no pod found in Endpoints %s/%s for hostname %s", svcID.Namespace, svcID.Name, hostname)
|
||||
}
|
||||
|
||||
// getPodByHostIP returns a pod that maps to the given IP address in the host
|
||||
// network. It must have a container port that exposes `port` as a host port.
|
||||
func getPodByHostIP(k8sAPI *k8s.API, hostIP string, port uint32, log *logging.Entry) (*corev1.Pod, error) {
|
||||
addr := net.JoinHostPort(hostIP, fmt.Sprintf("%d", port))
|
||||
hostIPPods, err := getIndexedPods(k8sAPI, watcher.HostIPIndex, addr)
|
||||
if err != nil {
|
||||
return nil, status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
if len(hostIPPods) == 1 {
|
||||
log.Debugf("found %s:%d on the host network", hostIP, port)
|
||||
return hostIPPods[0], nil
|
||||
}
|
||||
if len(hostIPPods) > 1 {
|
||||
conflictingPods := []string{}
|
||||
for _, pod := range hostIPPods {
|
||||
conflictingPods = append(conflictingPods, fmt.Sprintf("%s:%s", pod.Namespace, pod.Name))
|
||||
}
|
||||
log.Warnf("found conflicting %s:%d endpoint on the host network: %s", hostIP, port, strings.Join(conflictingPods, ","))
|
||||
return nil, status.Errorf(codes.FailedPrecondition, "found %d pods with a conflicting host network endpoint %s:%d", len(hostIPPods), hostIP, port)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// getPodByPodIP returns a pod that maps to the given IP address in the pod network
|
||||
func getPodByPodIP(k8sAPI *k8s.API, podIP string, port uint32, log *logging.Entry) (*corev1.Pod, error) {
|
||||
podIPPods, err := getIndexedPods(k8sAPI, watcher.PodIPIndex, podIP)
|
||||
if err != nil {
|
||||
return nil, status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
if len(podIPPods) == 1 {
|
||||
log.Debugf("found %s on the pod network", podIP)
|
||||
return podIPPods[0], nil
|
||||
}
|
||||
if len(podIPPods) > 1 {
|
||||
conflictingPods := []string{}
|
||||
for _, pod := range podIPPods {
|
||||
conflictingPods = append(conflictingPods, fmt.Sprintf("%s:%s", pod.Namespace, pod.Name))
|
||||
}
|
||||
log.Warnf("found conflicting %s IP on the pod network: %s", podIP, strings.Join(conflictingPods, ","))
|
||||
return nil, status.Errorf(codes.FailedPrecondition, "found %d pods with a conflicting pod network IP %s", len(podIPPods), podIP)
|
||||
}
|
||||
|
||||
log.Debugf("no pod found for %s:%d", podIP, port)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func getIndexedPods(k8sAPI *k8s.API, indexName string, podIP string) ([]*corev1.Pod, error) {
|
||||
objs, err := k8sAPI.Pod().Informer().GetIndexer().ByIndex(indexName, podIP)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed getting %s indexed pods: %w", indexName, err)
|
||||
}
|
||||
pods := make([]*corev1.Pod, 0)
|
||||
for _, obj := range objs {
|
||||
pod := obj.(*corev1.Pod)
|
||||
if !podReceivingTraffic(pod) {
|
||||
continue
|
||||
}
|
||||
pods = append(pods, pod)
|
||||
}
|
||||
return pods, nil
|
||||
}
|
||||
|
||||
func podReceivingTraffic(pod *corev1.Pod) bool {
|
||||
phase := pod.Status.Phase
|
||||
podTerminated := phase == corev1.PodSucceeded || phase == corev1.PodFailed
|
||||
podTerminating := pod.DeletionTimestamp != nil
|
||||
|
||||
return !podTerminating && !podTerminated
|
||||
}
|
||||
|
||||
////////////
|
||||
/// util ///
|
||||
////////////
|
||||
|
@ -805,25 +619,6 @@ func hasSuffix(slice []string, suffix []string) bool {
|
|||
return true
|
||||
}
|
||||
|
||||
func getAnnotatedOpaquePorts(pod *corev1.Pod, defaultPorts map[uint32]struct{}) map[uint32]struct{} {
|
||||
if pod == nil {
|
||||
return defaultPorts
|
||||
}
|
||||
annotation, ok := pod.Annotations[labels.ProxyOpaquePortsAnnotation]
|
||||
if !ok {
|
||||
return defaultPorts
|
||||
}
|
||||
opaquePorts := make(map[uint32]struct{})
|
||||
if annotation != "" {
|
||||
for _, pr := range util.ParseContainerOpaquePorts(annotation, pod.Spec.Containers) {
|
||||
for _, port := range pr.Ports() {
|
||||
opaquePorts[uint32(port)] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
return opaquePorts
|
||||
}
|
||||
|
||||
func getPodSkippedInboundPortsAnnotations(pod *corev1.Pod) (map[uint32]struct{}, error) {
|
||||
annotation, ok := pod.Annotations[labels.ProxyIgnoreInboundPortsAnnotation]
|
||||
if !ok || annotation == "" {
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package destination
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -12,7 +12,10 @@ import (
|
|||
"github.com/linkerd/linkerd2/controller/api/util"
|
||||
"github.com/linkerd/linkerd2/controller/k8s"
|
||||
"github.com/linkerd/linkerd2/pkg/addr"
|
||||
"github.com/linkerd/linkerd2/testutil"
|
||||
logging "github.com/sirupsen/logrus"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
)
|
||||
|
||||
const fullyQualifiedName = "name1.ns.svc.mycluster.local"
|
||||
|
@ -157,6 +160,7 @@ func TestGetProfiles(t *testing.T) {
|
|||
updates: []*pb.DestinationProfile{},
|
||||
MockServerStream: util.NewMockServerStream(),
|
||||
}
|
||||
defer stream.Cancel()
|
||||
err := server.GetProfile(&pb.GetDestination{Scheme: "k8s", Path: "linkerd.io"}, stream)
|
||||
if err == nil {
|
||||
t.Fatalf("Expecting error, got nothing")
|
||||
|
@ -167,7 +171,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Returns server profile", func(t *testing.T) {
|
||||
stream, server := profileStream(t, fullyQualifiedName, port, "ns:other")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.FullyQualifiedName != fullyQualifiedName {
|
||||
t.Fatalf("Expected fully qualified name '%s', but got '%s'",
|
||||
fullyQualifiedName, profile.FullyQualifiedName)
|
||||
|
@ -185,7 +190,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return service profile when using json token", func(t *testing.T) {
|
||||
stream, server := profileStream(t, fullyQualifiedName, port, `{"ns":"other"}`)
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.FullyQualifiedName != fullyQualifiedName {
|
||||
t.Fatalf("Expected fully qualified name '%s', but got '%s'", fullyQualifiedName, profile.FullyQualifiedName)
|
||||
}
|
||||
|
@ -199,7 +205,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Returns client profile", func(t *testing.T) {
|
||||
stream, server := profileStream(t, fullyQualifiedName, port, `{"ns":"client-ns"}`)
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
routes := profile.GetRoutes()
|
||||
if len(routes) != 1 {
|
||||
t.Fatalf("Expected 1 route but got %d: %v", len(routes), routes)
|
||||
|
@ -213,7 +220,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile when using cluster IP", func(t *testing.T) {
|
||||
stream, server := profileStream(t, clusterIP, port, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.FullyQualifiedName != fullyQualifiedName {
|
||||
t.Fatalf("Expected fully qualified name '%s', but got '%s'", fullyQualifiedName, profile.FullyQualifiedName)
|
||||
}
|
||||
|
@ -230,6 +238,7 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with endpoint when using pod DNS", func(t *testing.T) {
|
||||
stream, server := profileStream(t, fullyQualifiedPodDNS, port, "ns:ns")
|
||||
defer stream.Cancel()
|
||||
|
||||
epAddr, err := toAddress(podIPStatefulSet, port)
|
||||
if err != nil {
|
||||
|
@ -238,11 +247,12 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
// An explanation for why we expect 1 to 3 updates is in test cases
|
||||
// above
|
||||
if len(stream.updates) == 0 || len(stream.updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(stream.updates), stream.updates)
|
||||
updates := stream.Updates()
|
||||
if len(updates) == 0 || len(updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(updates), updates)
|
||||
}
|
||||
|
||||
first := stream.updates[0]
|
||||
first := updates[0]
|
||||
if first.Endpoint == nil {
|
||||
t.Fatalf("Expected response to have endpoint field")
|
||||
}
|
||||
|
@ -268,6 +278,7 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with endpoint when using pod IP", func(t *testing.T) {
|
||||
stream, server := profileStream(t, podIP1, port, "ns:ns")
|
||||
defer stream.Cancel()
|
||||
|
||||
epAddr, err := toAddress(podIP1, port)
|
||||
if err != nil {
|
||||
|
@ -276,11 +287,12 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
// An explanation for why we expect 1 to 3 updates is in test cases
|
||||
// above
|
||||
if len(stream.updates) == 0 || len(stream.updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(stream.updates), stream.updates)
|
||||
updates := stream.Updates()
|
||||
if len(updates) == 0 || len(updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(updates), updates)
|
||||
}
|
||||
|
||||
first := stream.updates[0]
|
||||
first := updates[0]
|
||||
if first.Endpoint == nil {
|
||||
t.Fatalf("Expected response to have endpoint field")
|
||||
}
|
||||
|
@ -306,7 +318,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return default profile when IP does not map to service or pod", func(t *testing.T) {
|
||||
stream, server := profileStream(t, "172.0.0.0", 1234, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.RetryBudget == nil {
|
||||
t.Fatalf("Expected default profile to have a retry budget")
|
||||
}
|
||||
|
@ -316,7 +329,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with no protocol hint when pod does not have label", func(t *testing.T) {
|
||||
stream, server := profileStream(t, podIP2, port, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.Endpoint == nil {
|
||||
t.Fatalf("Expected response to have endpoint field")
|
||||
}
|
||||
|
@ -329,7 +343,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return non-opaque protocol profile when using cluster IP and opaque protocol port", func(t *testing.T) {
|
||||
stream, server := profileStream(t, clusterIPOpaque, opaquePort, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.FullyQualifiedName != fullyQualifiedNameOpaque {
|
||||
t.Fatalf("Expected fully qualified name '%s', but got '%s'", fullyQualifiedNameOpaque, profile.FullyQualifiedName)
|
||||
}
|
||||
|
@ -342,6 +357,7 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return opaque protocol profile with endpoint when using pod IP and opaque protocol port", func(t *testing.T) {
|
||||
stream, server := profileStream(t, podIPOpaque, opaquePort, "")
|
||||
defer stream.Cancel()
|
||||
|
||||
epAddr, err := toAddress(podIPOpaque, opaquePort)
|
||||
if err != nil {
|
||||
|
@ -350,11 +366,12 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
// An explanation for why we expect 1 to 3 updates is in test cases
|
||||
// above
|
||||
if len(stream.updates) == 0 || len(stream.updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(stream.updates), stream.updates)
|
||||
updates := stream.Updates()
|
||||
if len(updates) == 0 || len(updates) > 3 {
|
||||
t.Fatalf("Expected 1 to 3 updates but got %d: %v", len(updates), updates)
|
||||
}
|
||||
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
profile := assertSingleProfile(t, updates)
|
||||
if profile.Endpoint == nil {
|
||||
t.Fatalf("Expected response to have endpoint field")
|
||||
}
|
||||
|
@ -380,7 +397,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return opaque protocol profile when using service name with opaque port annotation", func(t *testing.T) {
|
||||
stream, server := profileStream(t, fullyQualifiedNameOpaqueService, opaquePort, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.FullyQualifiedName != fullyQualifiedNameOpaqueService {
|
||||
t.Fatalf("Expected fully qualified name '%s', but got '%s'", fullyQualifiedNameOpaqueService, profile.FullyQualifiedName)
|
||||
}
|
||||
|
@ -393,7 +411,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with unknown protocol hint and identity when pod contains skipped inbound port", func(t *testing.T) {
|
||||
stream, server := profileStream(t, podIPSkipped, skippedPort, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
addr := profile.GetEndpoint()
|
||||
if addr == nil {
|
||||
t.Fatalf("Expected to not be nil")
|
||||
|
@ -410,7 +429,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with opaque protocol when using Pod IP selected by a Server", func(t *testing.T) {
|
||||
stream, server := profileStream(t, podIPPolicy, 80, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.Endpoint == nil {
|
||||
t.Fatalf("Expected response to have endpoint field")
|
||||
}
|
||||
|
@ -429,7 +449,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with opaque protocol when using an opaque port with an external IP", func(t *testing.T) {
|
||||
stream, server := profileStream(t, externalIP, 3306, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if !profile.OpaqueProtocol {
|
||||
t.Fatalf("Expected port %d to be an opaque protocol, but it was not", 3306)
|
||||
}
|
||||
|
@ -439,7 +460,8 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile with non-opaque protocol when using an arbitrary port with an external IP", func(t *testing.T) {
|
||||
stream, server := profileStream(t, externalIP, 80, "")
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
defer stream.Cancel()
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
if profile.OpaqueProtocol {
|
||||
t.Fatalf("Expected port %d to be a non-opaque protocol, but it was opaque", 80)
|
||||
}
|
||||
|
@ -449,10 +471,12 @@ func TestGetProfiles(t *testing.T) {
|
|||
|
||||
t.Run("Return profile for host port pods", func(t *testing.T) {
|
||||
hostPort := uint32(7777)
|
||||
containerPort := uint32(80)
|
||||
stream, server := profileStream(t, externalIP, hostPort, "")
|
||||
defer stream.Cancel()
|
||||
|
||||
// HostPort maps to pod.
|
||||
profile := assertSingleProfile(t, stream.updates)
|
||||
profile := assertSingleProfile(t, stream.Updates())
|
||||
dstPod := profile.Endpoint.MetricLabels["pod"]
|
||||
if dstPod != "hostport-mapping" {
|
||||
t.Fatalf("Expected dst_pod to be %s got %s", "hostport-mapping", dstPod)
|
||||
|
@ -467,6 +491,82 @@ func TestGetProfiles(t *testing.T) {
|
|||
t.Fatalf("Expected endpoint addr to be %s port:%d got %s", ip, hostPort, addr)
|
||||
}
|
||||
|
||||
// HostPort pod is deleted.
|
||||
err = server.k8sAPI.Client.CoreV1().Pods("ns").Delete(context.Background(), "hostport-mapping", metav1.DeleteOptions{})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to delete pod: %s", err)
|
||||
}
|
||||
err = testutil.RetryFor(time.Second*10, func() error {
|
||||
updates := stream.Updates()
|
||||
if len(updates) < 2 {
|
||||
return fmt.Errorf("expected 2 updates, got %d", len(updates))
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
profile = stream.Updates()[1]
|
||||
dstPod = profile.Endpoint.MetricLabels["pod"]
|
||||
if dstPod != "" {
|
||||
t.Fatalf("Expected no dst_pod but got %s", dstPod)
|
||||
}
|
||||
|
||||
// New HostPort pod is created.
|
||||
_, err = server.k8sAPI.Client.CoreV1().Pods("ns").Create(context.Background(), &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "hostport-mapping-2",
|
||||
Namespace: "ns",
|
||||
},
|
||||
Spec: corev1.PodSpec{
|
||||
Containers: []corev1.Container{
|
||||
{
|
||||
Name: "nginx",
|
||||
Image: "nginx",
|
||||
Ports: []corev1.ContainerPort{
|
||||
{
|
||||
Name: "nginx-7777",
|
||||
ContainerPort: (int32)(containerPort),
|
||||
HostPort: (int32)(hostPort),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
Status: corev1.PodStatus{
|
||||
Phase: "Running",
|
||||
Conditions: []corev1.PodCondition{
|
||||
{
|
||||
Type: corev1.PodReady,
|
||||
Status: corev1.ConditionTrue,
|
||||
},
|
||||
},
|
||||
HostIP: externalIP,
|
||||
PodIP: "172.17.0.55",
|
||||
PodIPs: []corev1.PodIP{{IP: "172.17.0.55"}},
|
||||
},
|
||||
}, metav1.CreateOptions{})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create pod: %s", err)
|
||||
}
|
||||
|
||||
err = testutil.RetryFor(time.Second*10, func() error {
|
||||
updates := stream.Updates()
|
||||
if len(updates) < 3 {
|
||||
return fmt.Errorf("expected 3 updates, got %d", len(updates))
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
profile = stream.Updates()[2]
|
||||
dstPod = profile.Endpoint.MetricLabels["pod"]
|
||||
if dstPod != "hostport-mapping-2" {
|
||||
t.Fatalf("Expected dst_pod to be %s got %s", "hostport-mapping-2", dstPod)
|
||||
}
|
||||
|
||||
server.clusterStore.UnregisterGauges()
|
||||
})
|
||||
}
|
||||
|
@ -589,101 +689,6 @@ spec:
|
|||
})
|
||||
}
|
||||
|
||||
func TestIpWatcherGetPod(t *testing.T) {
|
||||
podIP := "10.255.0.1"
|
||||
hostIP := "172.0.0.1"
|
||||
var hostPort1 uint32 = 22345
|
||||
var hostPort2 uint32 = 22346
|
||||
expectedPodName := "hostPortPod1"
|
||||
k8sConfigs := []string{`
|
||||
apiVersion: v1
|
||||
kind: Pod
|
||||
metadata:
|
||||
name: hostPortPod1
|
||||
namespace: ns
|
||||
spec:
|
||||
containers:
|
||||
- image: test
|
||||
name: hostPortContainer1
|
||||
ports:
|
||||
- containerPort: 12345
|
||||
hostIP: 172.0.0.1
|
||||
hostPort: 22345
|
||||
- image: test
|
||||
name: hostPortContainer2
|
||||
ports:
|
||||
- containerPort: 12346
|
||||
hostIP: 172.0.0.1
|
||||
hostPort: 22346
|
||||
status:
|
||||
phase: Running
|
||||
podIP: 10.255.0.1
|
||||
hostIP: 172.0.0.1`,
|
||||
`
|
||||
apiVersion: v1
|
||||
kind: Pod
|
||||
metadata:
|
||||
name: pod
|
||||
namespace: ns
|
||||
status:
|
||||
phase: Running
|
||||
podIP: 10.255.0.1`,
|
||||
}
|
||||
t.Run("get pod by host IP and host port", func(t *testing.T) {
|
||||
k8sAPI, err := k8s.NewFakeAPI(k8sConfigs...)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create new fake API: %s", err)
|
||||
}
|
||||
|
||||
err = watcher.InitializeIndexers(k8sAPI)
|
||||
if err != nil {
|
||||
t.Fatalf("initializeIndexers returned an error: %s", err)
|
||||
}
|
||||
|
||||
k8sAPI.Sync(nil)
|
||||
// Get host IP pod that is mapped to the port `hostPort1`
|
||||
pod, err := getPodByHostIP(k8sAPI, hostIP, hostPort1, logging.WithFields(nil))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get pod: %s", err)
|
||||
}
|
||||
if pod == nil {
|
||||
t.Fatalf("failed to find pod mapped to %s:%d", hostIP, hostPort1)
|
||||
}
|
||||
if pod.Name != expectedPodName {
|
||||
t.Fatalf("expected pod name to be %s, but got %s", expectedPodName, pod.Name)
|
||||
}
|
||||
// Get host IP pod that is mapped to the port `hostPort2`; this tests
|
||||
// that the indexer properly adds multiple containers from a single
|
||||
// pod.
|
||||
pod, err = getPodByHostIP(k8sAPI, hostIP, hostPort2, logging.WithFields(nil))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get pod: %s", err)
|
||||
}
|
||||
if pod == nil {
|
||||
t.Fatalf("failed to find pod mapped to %s:%d", hostIP, hostPort2)
|
||||
}
|
||||
if pod.Name != expectedPodName {
|
||||
t.Fatalf("expected pod name to be %s, but got %s", expectedPodName, pod.Name)
|
||||
}
|
||||
// Get host IP pod with unmapped host port
|
||||
pod, err = getPodByHostIP(k8sAPI, hostIP, 12347, logging.WithFields(nil))
|
||||
if err != nil {
|
||||
t.Fatalf("expected no error when getting host IP pod with unmapped host port, but got: %s", err)
|
||||
}
|
||||
if pod != nil {
|
||||
t.Fatal("expected no pod to be found with unmapped host port")
|
||||
}
|
||||
// Get pod IP pod and expect an error
|
||||
_, err = getPodByPodIP(k8sAPI, podIP, 12346, logging.WithFields(nil))
|
||||
if err == nil {
|
||||
t.Fatal("expected error when getting by pod IP and unmapped host port, but got none")
|
||||
}
|
||||
if !strings.Contains(err.Error(), "pods with a conflicting pod network IP") {
|
||||
t.Fatalf("expected error to be pod IP address conflict, but got: %s", err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func assertSingleProfile(t *testing.T, updates []*pb.DestinationProfile) *pb.DestinationProfile {
|
||||
t.Helper()
|
||||
// Under normal conditions the creation of resources by the fake API will
|
||||
|
@ -717,21 +722,18 @@ func profileStream(t *testing.T, host string, port uint32, token string) (*buffe
|
|||
MockServerStream: util.NewMockServerStream(),
|
||||
}
|
||||
|
||||
// We cancel the stream before even sending the request so that we don't
|
||||
// need to call server.Get in a separate goroutine. By preemptively
|
||||
// cancelling, the behavior of Get becomes effectively synchronous and
|
||||
// we will get only the initial update, which is what we want for this
|
||||
// test.
|
||||
stream.Cancel()
|
||||
|
||||
err := server.GetProfile(&pb.GetDestination{
|
||||
Scheme: "k8s",
|
||||
Path: fmt.Sprintf("%s:%d", host, port),
|
||||
ContextToken: token,
|
||||
}, stream)
|
||||
if err != nil {
|
||||
t.Fatalf("Got error: %s", err)
|
||||
}
|
||||
go func() {
|
||||
err := server.GetProfile(&pb.GetDestination{
|
||||
Scheme: "k8s",
|
||||
Path: fmt.Sprintf("%s:%d", host, port),
|
||||
ContextToken: token,
|
||||
}, stream)
|
||||
if err != nil {
|
||||
logging.Fatalf("Got error: %s", err)
|
||||
}
|
||||
}()
|
||||
// Give GetProfile some slack
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
|
||||
return stream, server
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package destination
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
pb "github.com/linkerd/linkerd2-proxy-api/go/destination"
|
||||
|
@ -453,7 +454,7 @@ spec:
|
|||
t.Fatalf("NewFakeMetadataAPI returned an error: %s", err)
|
||||
}
|
||||
log := logging.WithField("test", t.Name())
|
||||
logging.SetLevel(logging.DebugLevel)
|
||||
logging.SetLevel(logging.TraceLevel)
|
||||
defaultOpaquePorts := map[uint32]struct{}{
|
||||
25: {},
|
||||
443: {},
|
||||
|
@ -468,6 +469,10 @@ spec:
|
|||
t.Fatalf("initializeIndexers returned an error: %s", err)
|
||||
}
|
||||
|
||||
pods, err := watcher.NewPodWatcher(k8sAPI, metadataAPI, log, defaultOpaquePorts)
|
||||
if err != nil {
|
||||
t.Fatalf("can't create Pods watcher: %s", err)
|
||||
}
|
||||
endpoints, err := watcher.NewEndpointsWatcher(k8sAPI, metadataAPI, log, false, "local")
|
||||
if err != nil {
|
||||
t.Fatalf("can't create Endpoints watcher: %s", err)
|
||||
|
@ -480,10 +485,6 @@ spec:
|
|||
if err != nil {
|
||||
t.Fatalf("can't create profile watcher: %s", err)
|
||||
}
|
||||
servers, err := watcher.NewServerWatcher(k8sAPI, log)
|
||||
if err != nil {
|
||||
t.Fatalf("can't create Server watcher: %s", err)
|
||||
}
|
||||
|
||||
clusterStore, err := watcher.NewClusterStoreWithDecoder(k8sAPI.Client, "linkerd", false, watcher.CreateMockDecoder(exportedServiceResources...))
|
||||
if err != nil {
|
||||
|
@ -498,10 +499,10 @@ spec:
|
|||
|
||||
return &server{
|
||||
pb.UnimplementedDestinationServer{},
|
||||
pods,
|
||||
endpoints,
|
||||
opaquePorts,
|
||||
profiles,
|
||||
servers,
|
||||
clusterStore,
|
||||
true,
|
||||
"linkerd",
|
||||
|
@ -528,13 +529,22 @@ func (bgs *bufferingGetStream) Send(update *pb.Update) error {
|
|||
type bufferingGetProfileStream struct {
|
||||
updates []*pb.DestinationProfile
|
||||
util.MockServerStream
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
func (bgps *bufferingGetProfileStream) Send(profile *pb.DestinationProfile) error {
|
||||
bgps.mu.Lock()
|
||||
defer bgps.mu.Unlock()
|
||||
bgps.updates = append(bgps.updates, profile)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bgps *bufferingGetProfileStream) Updates() []*pb.DestinationProfile {
|
||||
bgps.mu.Lock()
|
||||
defer bgps.mu.Unlock()
|
||||
return bgps.updates
|
||||
}
|
||||
|
||||
type mockDestinationGetServer struct {
|
||||
util.MockServerStream
|
||||
updatesReceived []*pb.Update
|
||||
|
|
|
@ -1321,7 +1321,8 @@ func SetToServerProtocol(k8sAPI *k8s.API, address *Address, port Port) error {
|
|||
case intstr.String:
|
||||
for _, c := range address.Pod.Spec.Containers {
|
||||
for _, p := range c.Ports {
|
||||
if p.ContainerPort == int32(port) && p.Name == server.Spec.Port.StrVal {
|
||||
if (p.ContainerPort == int32(port) || p.HostPort == int32(port)) &&
|
||||
p.Name == server.Spec.Port.StrVal {
|
||||
portMatch = true
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,12 @@ const (
|
|||
)
|
||||
|
||||
type (
|
||||
// IPPort holds the IP and port for some destination
|
||||
IPPort struct {
|
||||
IP string
|
||||
Port Port
|
||||
}
|
||||
|
||||
// ID is a namespace-qualified name.
|
||||
ID struct {
|
||||
Namespace string
|
||||
|
@ -117,3 +123,27 @@ func InitializeIndexers(k8sAPI *k8s.API) error {
|
|||
|
||||
return nil
|
||||
}
|
||||
|
||||
func getIndexedPods(k8sAPI *k8s.API, indexName string, podIP string) ([]*corev1.Pod, error) {
|
||||
objs, err := k8sAPI.Pod().Informer().GetIndexer().ByIndex(indexName, podIP)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed getting %s indexed pods: %w", indexName, err)
|
||||
}
|
||||
pods := make([]*corev1.Pod, 0)
|
||||
for _, obj := range objs {
|
||||
pod := obj.(*corev1.Pod)
|
||||
if !podReceivingTraffic(pod) {
|
||||
continue
|
||||
}
|
||||
pods = append(pods, pod)
|
||||
}
|
||||
return pods, nil
|
||||
}
|
||||
|
||||
func podReceivingTraffic(pod *corev1.Pod) bool {
|
||||
phase := pod.Status.Phase
|
||||
podTerminated := phase == corev1.PodSucceeded || phase == corev1.PodFailed
|
||||
podTerminating := pod.DeletionTimestamp != nil
|
||||
|
||||
return !podTerminating && !podTerminated
|
||||
}
|
||||
|
|
|
@ -0,0 +1,546 @@
|
|||
package watcher
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/linkerd/linkerd2/controller/k8s"
|
||||
consts "github.com/linkerd/linkerd2/pkg/k8s"
|
||||
"github.com/linkerd/linkerd2/pkg/util"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
logging "github.com/sirupsen/logrus"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
)
|
||||
|
||||
type (
|
||||
// PodWatcher watches all pods in the cluster. It keeps a map of publishers
|
||||
// keyed by IP and port.
|
||||
PodWatcher struct {
|
||||
defaultOpaquePorts map[uint32]struct{}
|
||||
k8sAPI *k8s.API
|
||||
metadataAPI *k8s.MetadataAPI
|
||||
publishers map[IPPort]*podPublisher
|
||||
log *logging.Entry
|
||||
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
// podPublisher represents an ip:port along with the backing pod (if any).
|
||||
// It keeps a list of listeners to be notified whenever the pod or the
|
||||
// associated opaque protocol config changes.
|
||||
podPublisher struct {
|
||||
defaultOpaquePorts map[uint32]struct{}
|
||||
k8sAPI *k8s.API
|
||||
metadataAPI *k8s.MetadataAPI
|
||||
ip string
|
||||
port Port
|
||||
pod *corev1.Pod
|
||||
listeners []PodUpdateListener
|
||||
metrics metrics
|
||||
log *logging.Entry
|
||||
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
// PodUpdateListener is the interface subscribers must implement.
|
||||
PodUpdateListener interface {
|
||||
Update(*Address) (bool, error)
|
||||
}
|
||||
)
|
||||
|
||||
var ipPortVecs = newMetricsVecs("ip_port", []string{"ip", "port"})
|
||||
|
||||
func NewPodWatcher(k8sAPI *k8s.API, metadataAPI *k8s.MetadataAPI, log *logging.Entry, defaultOpaquePorts map[uint32]struct{}) (*PodWatcher, error) {
|
||||
pw := &PodWatcher{
|
||||
defaultOpaquePorts: defaultOpaquePorts,
|
||||
k8sAPI: k8sAPI,
|
||||
metadataAPI: metadataAPI,
|
||||
publishers: make(map[IPPort]*podPublisher),
|
||||
log: log.WithFields(logging.Fields{
|
||||
"component": "pod-watcher",
|
||||
}),
|
||||
}
|
||||
|
||||
_, err := k8sAPI.Pod().Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: pw.addPod,
|
||||
DeleteFunc: pw.deletePod,
|
||||
UpdateFunc: pw.updatePod,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
_, err = k8sAPI.Srv().Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: pw.updateServer,
|
||||
DeleteFunc: pw.updateServer,
|
||||
UpdateFunc: func(_, obj interface{}) { pw.updateServer(obj) },
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return pw, nil
|
||||
}
|
||||
|
||||
// Subscribe notifies the listener on changes on any pod backing the passed
|
||||
// host/ip:port or changes to its associated opaque protocol config. If service
|
||||
// and hostname are empty then ip should be set and vice-versa. If ip is empty,
|
||||
// the corresponding ip is found for the given service/hostname, and returned.
|
||||
func (pw *PodWatcher) Subscribe(service *ServiceID, hostname, ip string, port Port, listener PodUpdateListener) (string, error) {
|
||||
if hostname != "" {
|
||||
pw.log.Debugf("Establishing watch on %s.%s.%s:%d", hostname, service.Name, service.Namespace, port)
|
||||
} else if service != nil {
|
||||
pw.log.Debugf("Establishing watch on %s.%s:%d", service.Name, service.Namespace, port)
|
||||
} else {
|
||||
pw.log.Debugf("Establishing watch on %s:%d", ip, port)
|
||||
}
|
||||
pp, err := pw.getOrNewPodPublisher(service, hostname, ip, port)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
pp.subscribe(listener)
|
||||
|
||||
address, err := pp.createAddress()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
sent, err := listener.Update(&address)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
if sent {
|
||||
pp.metrics.incUpdates()
|
||||
}
|
||||
|
||||
return pp.ip, nil
|
||||
}
|
||||
|
||||
// Subscribe stops notifying the listener on chages on any pod backing the
|
||||
// passed ip:port or its associated protocol config
|
||||
func (pw *PodWatcher) Unsubscribe(ip string, port Port, listener PodUpdateListener) {
|
||||
pw.mu.Lock()
|
||||
defer pw.mu.Unlock()
|
||||
|
||||
pw.log.Debugf("Stopping watch on %s:%d", ip, port)
|
||||
pp, ok := pw.getPodPublisher(ip, port)
|
||||
if !ok {
|
||||
pw.log.Errorf("Cannot unsubscribe from unknown ip:port [%s:%d]", ip, port)
|
||||
return
|
||||
}
|
||||
pp.unsubscribe(listener)
|
||||
|
||||
if len(pp.listeners) == 0 {
|
||||
delete(pw.publishers, IPPort{pp.ip, pp.port})
|
||||
}
|
||||
}
|
||||
|
||||
// addPod is an event handler so it cannot block
|
||||
func (pw *PodWatcher) addPod(obj any) {
|
||||
pod := obj.(*corev1.Pod)
|
||||
pw.log.Tracef("Added pod %s.%s", pod.Name, pod.Namespace)
|
||||
go pw.submitPodUpdate(pod, false)
|
||||
}
|
||||
|
||||
// deletePod is an event handler so it cannot block
|
||||
func (pw *PodWatcher) deletePod(obj any) {
|
||||
pod, ok := obj.(*corev1.Pod)
|
||||
if !ok {
|
||||
tombstone, ok := obj.(cache.DeletedFinalStateUnknown)
|
||||
if !ok {
|
||||
pw.log.Errorf("Couldn't get object from DeletedFinalStateUnknown %#v", obj)
|
||||
return
|
||||
}
|
||||
pod, ok = tombstone.Obj.(*corev1.Pod)
|
||||
if !ok {
|
||||
pw.log.Errorf("DeletedFinalStateUnknown contained object that is not a Pod %#v", obj)
|
||||
return
|
||||
}
|
||||
}
|
||||
pw.log.Tracef("Deleted pod %s.%s", pod.Name, pod.Namespace)
|
||||
go pw.submitPodUpdate(pod, true)
|
||||
}
|
||||
|
||||
// updatePod is an event handler so it cannot block
|
||||
func (pw *PodWatcher) updatePod(oldObj any, newObj any) {
|
||||
oldPod := oldObj.(*corev1.Pod)
|
||||
newPod := newObj.(*corev1.Pod)
|
||||
if oldPod.DeletionTimestamp == nil && newPod.DeletionTimestamp != nil {
|
||||
// this is just a mark, wait for actual deletion event
|
||||
return
|
||||
}
|
||||
pw.log.Tracef("Updated pod %s.%s", newPod.Name, newPod.Namespace)
|
||||
go pw.submitPodUpdate(newPod, false)
|
||||
}
|
||||
|
||||
func (pw *PodWatcher) submitPodUpdate(pod *corev1.Pod, remove bool) {
|
||||
pw.mu.RLock()
|
||||
defer pw.mu.RUnlock()
|
||||
|
||||
submitPod := pod
|
||||
if remove {
|
||||
submitPod = nil
|
||||
}
|
||||
for _, container := range pod.Spec.Containers {
|
||||
for _, containerPort := range container.Ports {
|
||||
if containerPort.ContainerPort != 0 {
|
||||
if pp, ok := pw.getPodPublisher(pod.Status.PodIP, Port(containerPort.ContainerPort)); ok {
|
||||
pp.updatePod(submitPod)
|
||||
}
|
||||
}
|
||||
if containerPort.HostPort != 0 {
|
||||
if pp, ok := pw.getPodPublisher(pod.Status.HostIP, Port(containerPort.HostPort)); ok {
|
||||
pp.updatePod(submitPod)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// updateServer triggers an Update() call to the listeners of the podPublishers
|
||||
// whose pod matches the Server's selector. This function is an event handler
|
||||
// so it cannot block.
|
||||
func (pw *PodWatcher) updateServer(_ any) {
|
||||
pw.mu.RLock()
|
||||
defer pw.mu.RUnlock()
|
||||
|
||||
for _, pp := range pw.publishers {
|
||||
if pp.pod == nil {
|
||||
continue
|
||||
}
|
||||
opaquePorts := GetAnnotatedOpaquePorts(pp.pod, pw.defaultOpaquePorts)
|
||||
_, isOpaque := opaquePorts[pp.port]
|
||||
// if port is annotated to be always opaque we can disregard Server updates
|
||||
if isOpaque {
|
||||
continue
|
||||
}
|
||||
|
||||
go func(pp *podPublisher) {
|
||||
pp.mu.RLock()
|
||||
defer pp.mu.RUnlock()
|
||||
|
||||
updated := false
|
||||
for _, listener := range pp.listeners {
|
||||
// the Server in question doesn't carry information about other
|
||||
// Servers that might target this podPublisher; createAddress()
|
||||
// queries all the relevant Servers to determine the full state
|
||||
addr, err := pp.createAddress()
|
||||
if err != nil {
|
||||
pw.log.Errorf("Error creating address for pod: %s", err)
|
||||
continue
|
||||
}
|
||||
sent, err := listener.Update(&addr)
|
||||
if err != nil {
|
||||
pw.log.Errorf("Error calling pod watcher listener for server update: %s", err)
|
||||
}
|
||||
updated = updated || sent
|
||||
}
|
||||
if updated {
|
||||
pp.metrics.incUpdates()
|
||||
}
|
||||
}(pp)
|
||||
}
|
||||
}
|
||||
|
||||
// getOrNewPodPublisher returns the podPublisher for the given target if it
|
||||
// exists. Otherwise, it creates a new one and returns it.
|
||||
func (pw *PodWatcher) getOrNewPodPublisher(service *ServiceID, hostname, ip string, port Port) (*podPublisher, error) {
|
||||
pw.mu.Lock()
|
||||
defer pw.mu.Unlock()
|
||||
|
||||
var pod *corev1.Pod
|
||||
var err error
|
||||
if hostname != "" {
|
||||
pod, err = pw.getEndpointByHostname(hostname, service)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to get pod for hostname %s: %w", hostname, err)
|
||||
}
|
||||
ip = pod.Status.PodIP
|
||||
} else {
|
||||
pod, err = pw.getPodByPodIP(ip, port)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if pod == nil {
|
||||
pod, err = pw.getPodByHostIP(ip, port)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ipPort := IPPort{ip, port}
|
||||
pp, ok := pw.publishers[ipPort]
|
||||
if !ok {
|
||||
pp = &podPublisher{
|
||||
defaultOpaquePorts: pw.defaultOpaquePorts,
|
||||
k8sAPI: pw.k8sAPI,
|
||||
metadataAPI: pw.metadataAPI,
|
||||
ip: ip,
|
||||
port: port,
|
||||
pod: pod,
|
||||
metrics: ipPortVecs.newMetrics(prometheus.Labels{
|
||||
"ip": ip,
|
||||
"port": strconv.FormatUint(uint64(port), 10),
|
||||
}),
|
||||
log: pw.log.WithFields(logging.Fields{
|
||||
"component": "pod-publisher",
|
||||
"ip": ip,
|
||||
"port": port,
|
||||
}),
|
||||
}
|
||||
pw.publishers[ipPort] = pp
|
||||
}
|
||||
return pp, nil
|
||||
}
|
||||
|
||||
func (pw *PodWatcher) getPodPublisher(ip string, port Port) (pp *podPublisher, ok bool) {
|
||||
ipPort := IPPort{ip, port}
|
||||
pp, ok = pw.publishers[ipPort]
|
||||
return
|
||||
}
|
||||
|
||||
// getPodByPodIP returns a pod that maps to the given IP address in the pod network
|
||||
func (pw *PodWatcher) getPodByPodIP(podIP string, port uint32) (*corev1.Pod, error) {
|
||||
podIPPods, err := getIndexedPods(pw.k8sAPI, PodIPIndex, podIP)
|
||||
if err != nil {
|
||||
return nil, status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
if len(podIPPods) == 1 {
|
||||
pw.log.Debugf("found %s on the pod network", podIP)
|
||||
return podIPPods[0], nil
|
||||
}
|
||||
if len(podIPPods) > 1 {
|
||||
conflictingPods := []string{}
|
||||
for _, pod := range podIPPods {
|
||||
conflictingPods = append(conflictingPods, fmt.Sprintf("%s:%s", pod.Namespace, pod.Name))
|
||||
}
|
||||
pw.log.Warnf("found conflicting %s IP on the pod network: %s", podIP, strings.Join(conflictingPods, ","))
|
||||
return nil, status.Errorf(codes.FailedPrecondition, "found %d pods with a conflicting pod network IP %s", len(podIPPods), podIP)
|
||||
}
|
||||
|
||||
pw.log.Debugf("no pod found for %s:%d", podIP, port)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// getPodByHostIP returns a pod that maps to the given IP address in the host
|
||||
// network. It must have a container port that exposes `port` as a host port.
|
||||
func (pw *PodWatcher) getPodByHostIP(hostIP string, port uint32) (*corev1.Pod, error) {
|
||||
addr := net.JoinHostPort(hostIP, fmt.Sprintf("%d", port))
|
||||
hostIPPods, err := getIndexedPods(pw.k8sAPI, HostIPIndex, addr)
|
||||
if err != nil {
|
||||
return nil, status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
if len(hostIPPods) == 1 {
|
||||
pw.log.Debugf("found %s:%d on the host network", hostIP, port)
|
||||
return hostIPPods[0], nil
|
||||
}
|
||||
if len(hostIPPods) > 1 {
|
||||
conflictingPods := []string{}
|
||||
for _, pod := range hostIPPods {
|
||||
conflictingPods = append(conflictingPods, fmt.Sprintf("%s:%s", pod.Namespace, pod.Name))
|
||||
}
|
||||
pw.log.Warnf("found conflicting %s:%d endpoint on the host network: %s", hostIP, port, strings.Join(conflictingPods, ","))
|
||||
return nil, status.Errorf(codes.FailedPrecondition, "found %d pods with a conflicting host network endpoint %s:%d", len(hostIPPods), hostIP, port)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// getEndpointByHostname returns a pod that maps to the given hostname (or an
|
||||
// instanceID). The hostname is generally the prefix of the pod's DNS name;
|
||||
// since it may be arbitrary we need to look at the corresponding service's
|
||||
// Endpoints object to see whether the hostname matches a pod.
|
||||
func (pw *PodWatcher) getEndpointByHostname(hostname string, svcID *ServiceID) (*corev1.Pod, error) {
|
||||
ep, err := pw.k8sAPI.Endpoint().Lister().Endpoints(svcID.Namespace).Get(svcID.Name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, subset := range ep.Subsets {
|
||||
for _, addr := range subset.Addresses {
|
||||
|
||||
if hostname == addr.Hostname {
|
||||
if addr.TargetRef != nil && addr.TargetRef.Kind == "Pod" {
|
||||
podName := addr.TargetRef.Name
|
||||
podNamespace := addr.TargetRef.Namespace
|
||||
pod, err := pw.k8sAPI.Pod().Lister().Pods(podNamespace).Get(podName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return pod, nil
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("no pod found in Endpoints %s/%s for hostname %s", svcID.Namespace, svcID.Name, hostname)
|
||||
}
|
||||
|
||||
func (pp *podPublisher) subscribe(listener PodUpdateListener) {
|
||||
pp.mu.Lock()
|
||||
defer pp.mu.Unlock()
|
||||
|
||||
pp.listeners = append(pp.listeners, listener)
|
||||
pp.metrics.setSubscribers(len(pp.listeners))
|
||||
}
|
||||
|
||||
func (pp *podPublisher) unsubscribe(listener PodUpdateListener) {
|
||||
pp.mu.Lock()
|
||||
defer pp.mu.Unlock()
|
||||
|
||||
for i, e := range pp.listeners {
|
||||
if e == listener {
|
||||
n := len(pp.listeners)
|
||||
pp.listeners[i] = pp.listeners[n-1]
|
||||
pp.listeners[n-1] = nil
|
||||
pp.listeners = pp.listeners[:n-1]
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
pp.metrics.setSubscribers(len(pp.listeners))
|
||||
}
|
||||
|
||||
// updatePod creates an Address instance for the given pod, that is passed to
|
||||
// the listener's Update() method, only if the pod's readiness state has
|
||||
// changed. If the passed pod is nil, it means the pod (still referred to in
|
||||
// pp.pod) has been deleted.
|
||||
func (pp *podPublisher) updatePod(pod *corev1.Pod) {
|
||||
pp.mu.Lock()
|
||||
defer pp.mu.Unlock()
|
||||
|
||||
// pod wasn't ready or there was no backing pod - check if passed pod is ready
|
||||
if pp.pod == nil {
|
||||
if pod == nil {
|
||||
pp.log.Trace("Pod deletion event already consumed - ignore")
|
||||
return
|
||||
}
|
||||
|
||||
if !isRunningAndReady(pod) {
|
||||
pp.log.Tracef("Pod %s.%s not ready - ignore", pod.Name, pod.Namespace)
|
||||
return
|
||||
}
|
||||
|
||||
pp.log.Debugf("Pod %s.%s became ready", pod.Name, pod.Namespace)
|
||||
pp.pod = pod
|
||||
updated := false
|
||||
for _, l := range pp.listeners {
|
||||
addr, err := pp.createAddress()
|
||||
if err != nil {
|
||||
pp.log.Errorf("Error creating address for pod: %s", err)
|
||||
continue
|
||||
}
|
||||
sent, err := l.Update(&addr)
|
||||
if err != nil {
|
||||
pp.log.Errorf("Error calling pod watcher listener for pod update: %s", err)
|
||||
}
|
||||
updated = updated || sent
|
||||
}
|
||||
if updated {
|
||||
pp.metrics.incUpdates()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// backing pod becoming unready or getting deleted
|
||||
if pod == nil || !isRunningAndReady(pod) {
|
||||
pp.log.Debugf("Pod %s.%s deleted or it became unready - remove", pp.pod.Name, pp.pod.Namespace)
|
||||
pp.pod = nil
|
||||
updated := false
|
||||
for _, l := range pp.listeners {
|
||||
addr, err := pp.createAddress()
|
||||
if err != nil {
|
||||
pp.log.Errorf("Error creating address for pod: %s", err)
|
||||
continue
|
||||
}
|
||||
sent, err := l.Update(&addr)
|
||||
if err != nil {
|
||||
pp.log.Errorf("Error calling pod watcher listener for pod deletion: %s", err)
|
||||
}
|
||||
updated = updated || sent
|
||||
}
|
||||
if updated {
|
||||
pp.metrics.incUpdates()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
pp.log.Tracef("Ignored event on pod %s.%s", pod.Name, pod.Namespace)
|
||||
}
|
||||
|
||||
// createAddress returns an Address instance for the given ip, port and pod. It
|
||||
// completes the ownership and opaque protocol information
|
||||
func (pp *podPublisher) createAddress() (Address, error) {
|
||||
var ownerKind, ownerName string
|
||||
var err error
|
||||
if pp.pod != nil {
|
||||
ownerKind, ownerName, err = pp.metadataAPI.GetOwnerKindAndName(context.Background(), pp.pod, true)
|
||||
if err != nil {
|
||||
return Address{}, err
|
||||
}
|
||||
}
|
||||
|
||||
address := Address{
|
||||
IP: pp.ip,
|
||||
Port: pp.port,
|
||||
Pod: pp.pod,
|
||||
OwnerName: ownerName,
|
||||
OwnerKind: ownerKind,
|
||||
}
|
||||
|
||||
// Override opaqueProtocol if the endpoint's port is annotated as opaque
|
||||
opaquePorts := GetAnnotatedOpaquePorts(pp.pod, pp.defaultOpaquePorts)
|
||||
if _, ok := opaquePorts[pp.port]; ok {
|
||||
address.OpaqueProtocol = true
|
||||
} else if pp.pod != nil {
|
||||
if err := SetToServerProtocol(pp.k8sAPI, &address, pp.port); err != nil {
|
||||
return Address{}, fmt.Errorf("failed to set address OpaqueProtocol: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
return address, nil
|
||||
}
|
||||
|
||||
// GetAnnotatedOpaquePorts returns the opaque ports for the pod given its
|
||||
// annotations, or the default opaque ports if it's not annotated
|
||||
func GetAnnotatedOpaquePorts(pod *corev1.Pod, defaultPorts map[uint32]struct{}) map[uint32]struct{} {
|
||||
if pod == nil {
|
||||
return defaultPorts
|
||||
}
|
||||
annotation, ok := pod.Annotations[consts.ProxyOpaquePortsAnnotation]
|
||||
if !ok {
|
||||
return defaultPorts
|
||||
}
|
||||
opaquePorts := make(map[uint32]struct{})
|
||||
if annotation != "" {
|
||||
for _, pr := range util.ParseContainerOpaquePorts(annotation, pod.Spec.Containers) {
|
||||
for _, port := range pr.Ports() {
|
||||
opaquePorts[uint32(port)] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
return opaquePorts
|
||||
}
|
||||
|
||||
func isRunningAndReady(pod *corev1.Pod) bool {
|
||||
if pod == nil || pod.Status.Phase != corev1.PodRunning {
|
||||
return false
|
||||
}
|
||||
for _, condition := range pod.Status.Conditions {
|
||||
if condition.Type == corev1.PodReady && condition.Status == corev1.ConditionTrue {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
package watcher
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/linkerd/linkerd2/controller/k8s"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
func TestIpWatcherGetPod(t *testing.T) {
|
||||
podIP := "10.255.0.1"
|
||||
hostIP := "172.0.0.1"
|
||||
var hostPort1 uint32 = 22345
|
||||
var hostPort2 uint32 = 22346
|
||||
expectedPodName := "hostPortPod1"
|
||||
k8sConfigs := []string{`
|
||||
apiVersion: v1
|
||||
kind: Pod
|
||||
metadata:
|
||||
name: hostPortPod1
|
||||
namespace: ns
|
||||
spec:
|
||||
containers:
|
||||
- image: test
|
||||
name: hostPortContainer1
|
||||
ports:
|
||||
- containerPort: 12345
|
||||
hostIP: 172.0.0.1
|
||||
hostPort: 22345
|
||||
- image: test
|
||||
name: hostPortContainer2
|
||||
ports:
|
||||
- containerPort: 12346
|
||||
hostIP: 172.0.0.1
|
||||
hostPort: 22346
|
||||
status:
|
||||
phase: Running
|
||||
podIP: 10.255.0.1
|
||||
hostIP: 172.0.0.1`,
|
||||
`
|
||||
apiVersion: v1
|
||||
kind: Pod
|
||||
metadata:
|
||||
name: pod
|
||||
namespace: ns
|
||||
status:
|
||||
phase: Running
|
||||
podIP: 10.255.0.1`,
|
||||
}
|
||||
t.Run("get pod by host IP and host port", func(t *testing.T) {
|
||||
k8sAPI, err := k8s.NewFakeAPI(k8sConfigs...)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create new fake API: %s", err)
|
||||
}
|
||||
|
||||
err = InitializeIndexers(k8sAPI)
|
||||
if err != nil {
|
||||
t.Fatalf("initializeIndexers returned an error: %s", err)
|
||||
}
|
||||
|
||||
k8sAPI.Sync(nil)
|
||||
pw := PodWatcher{
|
||||
k8sAPI: k8sAPI,
|
||||
log: log.WithFields(log.Fields{
|
||||
"component": "pod-watcher",
|
||||
}),
|
||||
}
|
||||
|
||||
// Get host IP pod that is mapped to the port `hostPort1`
|
||||
pod, err := pw.getPodByHostIP(hostIP, hostPort1)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get pod: %s", err)
|
||||
}
|
||||
if pod == nil {
|
||||
t.Fatalf("failed to find pod mapped to %s:%d", hostIP, hostPort1)
|
||||
}
|
||||
if pod.Name != expectedPodName {
|
||||
t.Fatalf("expected pod name to be %s, but got %s", expectedPodName, pod.Name)
|
||||
}
|
||||
// Get host IP pod that is mapped to the port `hostPort2`; this tests
|
||||
// that the indexer properly adds multiple containers from a single
|
||||
// pod.
|
||||
pod, err = pw.getPodByHostIP(hostIP, hostPort2)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get pod: %s", err)
|
||||
}
|
||||
if pod == nil {
|
||||
t.Fatalf("failed to find pod mapped to %s:%d", hostIP, hostPort2)
|
||||
}
|
||||
if pod.Name != expectedPodName {
|
||||
t.Fatalf("expected pod name to be %s, but got %s", expectedPodName, pod.Name)
|
||||
}
|
||||
// Get host IP pod with unmapped host port
|
||||
pod, err = pw.getPodByHostIP(hostIP, 12347)
|
||||
if err != nil {
|
||||
t.Fatalf("expected no error when getting host IP pod with unmapped host port, but got: %s", err)
|
||||
}
|
||||
if pod != nil {
|
||||
t.Fatal("expected no pod to be found with unmapped host port")
|
||||
}
|
||||
// Get pod IP pod and expect an error
|
||||
_, err = pw.getPodByPodIP(podIP, 12346)
|
||||
if err == nil {
|
||||
t.Fatal("expected error when getting by pod IP and unmapped host port, but got none")
|
||||
}
|
||||
if !strings.Contains(err.Error(), "pods with a conflicting pod network IP") {
|
||||
t.Fatalf("expected error to be pod IP address conflict, but got: %s", err)
|
||||
}
|
||||
})
|
||||
}
|
|
@ -1,211 +0,0 @@
|
|||
package watcher
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/linkerd/linkerd2/controller/gen/apis/server/v1beta1"
|
||||
"github.com/linkerd/linkerd2/controller/k8s"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
logging "github.com/sirupsen/logrus"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/labels"
|
||||
"k8s.io/apimachinery/pkg/util/intstr"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
)
|
||||
|
||||
// ServerWatcher watches all the servers in the cluster. When there is an
|
||||
// update, it only sends updates to listeners if their endpoint's protocol
|
||||
// is changed by the Server.
|
||||
type ServerWatcher struct {
|
||||
subscriptions map[podPort]podPortPublisher
|
||||
k8sAPI *k8s.API
|
||||
subscribesCounter *prometheus.CounterVec
|
||||
unsubscribesCounter *prometheus.CounterVec
|
||||
log *logging.Entry
|
||||
sync.RWMutex
|
||||
}
|
||||
|
||||
type podPort struct {
|
||||
podID PodID
|
||||
port Port
|
||||
}
|
||||
|
||||
type podPortPublisher struct {
|
||||
pod *corev1.Pod
|
||||
listeners []ServerUpdateListener
|
||||
}
|
||||
|
||||
// ServerUpdateListener is the interface that subscribers must implement.
|
||||
type ServerUpdateListener interface {
|
||||
// UpdateProtocol takes a bool which is set to true if the endpoint is
|
||||
// opaque and false otherwise. This value is used to send a
|
||||
// DestinationProfile update to listeners for that endpoint.
|
||||
UpdateProtocol(bool)
|
||||
}
|
||||
|
||||
var (
|
||||
serverSubscribeCounter = promauto.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "server_port_subscribes",
|
||||
Help: "Counter of subscribes to Server changes associated with a pod's port.",
|
||||
},
|
||||
[]string{"namespace", "name", "port"},
|
||||
)
|
||||
serverUnsubscribeCounter = promauto.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "server_port_unsubscribes",
|
||||
Help: "Counter of unsubscribes to Server changes associated with a pod's port.",
|
||||
},
|
||||
[]string{"namespace", "name", "port"},
|
||||
)
|
||||
)
|
||||
|
||||
// NewServerWatcher creates a new ServerWatcher.
|
||||
func NewServerWatcher(k8sAPI *k8s.API, log *logging.Entry) (*ServerWatcher, error) {
|
||||
sw := &ServerWatcher{
|
||||
subscriptions: make(map[podPort]podPortPublisher),
|
||||
k8sAPI: k8sAPI,
|
||||
subscribesCounter: serverSubscribeCounter,
|
||||
unsubscribesCounter: serverUnsubscribeCounter,
|
||||
log: log,
|
||||
}
|
||||
_, err := k8sAPI.Srv().Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: sw.addServer,
|
||||
DeleteFunc: sw.deleteServer,
|
||||
UpdateFunc: func(_, obj interface{}) { sw.addServer(obj) },
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return sw, nil
|
||||
}
|
||||
|
||||
// Subscribe subscribes a listener for any Server updates that may select the
|
||||
// endpoint and change its expected protocol.
|
||||
func (sw *ServerWatcher) Subscribe(pod *corev1.Pod, port Port, listener ServerUpdateListener) {
|
||||
sw.Lock()
|
||||
defer sw.Unlock()
|
||||
pp := podPort{
|
||||
podID: PodID{
|
||||
Namespace: pod.Namespace,
|
||||
Name: pod.Name,
|
||||
},
|
||||
port: port,
|
||||
}
|
||||
ppp, ok := sw.subscriptions[pp]
|
||||
if !ok {
|
||||
ppp = podPortPublisher{
|
||||
pod: pod,
|
||||
listeners: []ServerUpdateListener{},
|
||||
}
|
||||
}
|
||||
ppp.listeners = append(ppp.listeners, listener)
|
||||
sw.subscriptions[pp] = ppp
|
||||
|
||||
sw.subscribesCounter.With(serverMetricLabels(pod, port)).Inc()
|
||||
}
|
||||
|
||||
// Unsubscribe unsubcribes a listener from any Server updates.
|
||||
func (sw *ServerWatcher) Unsubscribe(pod *corev1.Pod, port Port, listener ServerUpdateListener) {
|
||||
sw.Lock()
|
||||
defer sw.Unlock()
|
||||
pp := podPort{
|
||||
podID: PodID{
|
||||
Namespace: pod.Namespace,
|
||||
Name: pod.Name,
|
||||
},
|
||||
port: port,
|
||||
}
|
||||
ppp, ok := sw.subscriptions[pp]
|
||||
if !ok {
|
||||
sw.log.Errorf("cannot unsubscribe from unknown Pod: %s/%s:%d", pod.Namespace, pod.Name, port)
|
||||
return
|
||||
}
|
||||
for i, l := range ppp.listeners {
|
||||
if l == listener {
|
||||
n := len(ppp.listeners)
|
||||
ppp.listeners[i] = ppp.listeners[n-1]
|
||||
ppp.listeners[n-1] = nil
|
||||
ppp.listeners = ppp.listeners[:n-1]
|
||||
}
|
||||
}
|
||||
|
||||
if len(ppp.listeners) > 0 {
|
||||
sw.subscriptions[pp] = ppp
|
||||
} else {
|
||||
delete(sw.subscriptions, pp)
|
||||
}
|
||||
|
||||
sw.unsubscribesCounter.With(serverMetricLabels(pod, port)).Inc()
|
||||
}
|
||||
|
||||
func (sw *ServerWatcher) addServer(obj interface{}) {
|
||||
server := obj.(*v1beta1.Server)
|
||||
selector, err := metav1.LabelSelectorAsSelector(server.Spec.PodSelector)
|
||||
if err != nil {
|
||||
sw.log.Errorf("failed to create Selector: %s", err)
|
||||
return
|
||||
}
|
||||
sw.updateServer(server, selector, true)
|
||||
}
|
||||
|
||||
func (sw *ServerWatcher) deleteServer(obj interface{}) {
|
||||
server := obj.(*v1beta1.Server)
|
||||
selector, err := metav1.LabelSelectorAsSelector(server.Spec.PodSelector)
|
||||
if err != nil {
|
||||
sw.log.Errorf("failed to create Selector: %s", err)
|
||||
return
|
||||
}
|
||||
sw.updateServer(server, selector, false)
|
||||
}
|
||||
|
||||
func (sw *ServerWatcher) updateServer(server *v1beta1.Server, selector labels.Selector, isAdd bool) {
|
||||
sw.Lock()
|
||||
defer sw.Unlock()
|
||||
for pp, ppp := range sw.subscriptions {
|
||||
if selector.Matches(labels.Set(ppp.pod.Labels)) {
|
||||
var portMatch bool
|
||||
switch server.Spec.Port.Type {
|
||||
case intstr.Int:
|
||||
if server.Spec.Port.IntVal == int32(pp.port) {
|
||||
portMatch = true
|
||||
}
|
||||
case intstr.String:
|
||||
for _, c := range ppp.pod.Spec.Containers {
|
||||
for _, p := range c.Ports {
|
||||
if p.ContainerPort == int32(pp.port) && p.Name == server.Spec.Port.StrVal {
|
||||
portMatch = true
|
||||
}
|
||||
}
|
||||
}
|
||||
default:
|
||||
continue
|
||||
}
|
||||
if portMatch {
|
||||
var isOpaque bool
|
||||
if isAdd && server.Spec.ProxyProtocol == opaqueProtocol {
|
||||
isOpaque = true
|
||||
} else {
|
||||
isOpaque = false
|
||||
}
|
||||
for _, listener := range ppp.listeners {
|
||||
listener.UpdateProtocol(isOpaque)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func serverMetricLabels(pod *corev1.Pod, port Port) prometheus.Labels {
|
||||
podName, _, _ := strings.Cut(pod.Name, "-")
|
||||
return prometheus.Labels{
|
||||
"namespace": pod.Namespace,
|
||||
"name": podName,
|
||||
"port": strconv.FormatUint(uint64(port), 10),
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue