created docker-prometheus compose file and edited the Prometheus yml file

This commit is contained in:
Brian Christner
2015-08-18 14:36:46 +02:00
parent 4b9b5b70e3
commit bf34f627cf
664 changed files with 150581 additions and 0 deletions

View File

@ -0,0 +1,577 @@
// Copyright 2015 The Prometheus 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 kubernetes
import (
"crypto/tls"
"crypto/x509"
"encoding/json"
"fmt"
"io/ioutil"
"net"
"net/http"
"os"
"sync"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/log"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/util/httputil"
"github.com/prometheus/prometheus/util/strutil"
)
const (
sourceServicePrefix = "services"
// kubernetesMetaLabelPrefix is the meta prefix used for all meta labels.
// in this discovery.
metaLabelPrefix = clientmodel.MetaLabelPrefix + "kubernetes_"
// nodeLabel is the name for the label containing a target's node name.
nodeLabel = metaLabelPrefix + "node"
// serviceNamespaceLabel is the name for the label containing a target's service namespace.
serviceNamespaceLabel = metaLabelPrefix + "service_namespace"
// serviceNameLabel is the name for the label containing a target's service name.
serviceNameLabel = metaLabelPrefix + "service_name"
// nodeLabelPrefix is the prefix for the node labels.
nodeLabelPrefix = metaLabelPrefix + "node_label_"
// serviceLabelPrefix is the prefix for the service labels.
serviceLabelPrefix = metaLabelPrefix + "service_label_"
// serviceAnnotationPrefix is the prefix for the service annotations.
serviceAnnotationPrefix = metaLabelPrefix + "service_annotation_"
// nodesTargetGroupName is the name given to the target group for nodes.
nodesTargetGroupName = "nodes"
serviceAccountToken = "/var/run/secrets/kubernetes.io/serviceaccount/token"
serviceAccountCACert = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt"
apiVersion = "v1"
apiPrefix = "api/" + apiVersion
nodesURL = apiPrefix + "/nodes"
servicesURL = apiPrefix + "/services"
endpointsURL = apiPrefix + "/endpoints"
serviceEndpointsURL = apiPrefix + "/namespaces/%s/endpoints/%s"
)
type KubernetesDiscovery struct {
client *http.Client
Conf *config.KubernetesSDConfig
nodesResourceVersion string
servicesResourceVersion string
endpointsResourceVersion string
nodes map[string]*Node
services map[string]map[string]*Service
nodesMu sync.RWMutex
servicesMu sync.RWMutex
runDone chan struct{}
}
func (kd *KubernetesDiscovery) Initialize() error {
client, err := newKubernetesHTTPClient(kd.Conf)
if err != nil {
return err
}
kd.client = client
kd.nodes = map[string]*Node{}
kd.services = map[string]map[string]*Service{}
kd.runDone = make(chan struct{})
return nil
}
// Sources implements the TargetProvider interface.
func (kd *KubernetesDiscovery) Sources() []string {
res, err := kd.client.Get(kd.Conf.Server + nodesURL)
if err != nil {
// If we can't list nodes then we can't watch them. Assume this is a misconfiguration
// & log & return empty.
log.Errorf("Unable to list Kubernetes nodes: %s", err)
return []string{}
}
if res.StatusCode != http.StatusOK {
log.Errorf("Unable to list Kubernetes nodes. Unexpected response: %d %s", res.StatusCode, res.Status)
return []string{}
}
var nodes NodeList
if err := json.NewDecoder(res.Body).Decode(&nodes); err != nil {
body, _ := ioutil.ReadAll(res.Body)
log.Errorf("Unable to list Kubernetes nodes. Unexpected response body: %s", string(body))
return []string{}
}
kd.nodesMu.Lock()
defer kd.nodesMu.Unlock()
sourceNames := make([]string, 0, len(nodes.Items))
kd.nodesResourceVersion = nodes.ResourceVersion
for idx, node := range nodes.Items {
sourceNames = append(sourceNames, nodesTargetGroupName+":"+node.ObjectMeta.Name)
kd.nodes[node.ObjectMeta.Name] = &nodes.Items[idx]
}
res, err = kd.client.Get(kd.Conf.Server + servicesURL)
if err != nil {
// If we can't list services then we can't watch them. Assume this is a misconfiguration
// & log & return empty.
log.Errorf("Unable to list Kubernetes services: %s", err)
return []string{}
}
if res.StatusCode != http.StatusOK {
log.Errorf("Unable to list Kubernetes services. Unexpected response: %d %s", res.StatusCode, res.Status)
return []string{}
}
var services ServiceList
if err := json.NewDecoder(res.Body).Decode(&services); err != nil {
body, _ := ioutil.ReadAll(res.Body)
log.Errorf("Unable to list Kubernetes services. Unexpected response body: %s", string(body))
return []string{}
}
kd.servicesMu.Lock()
defer kd.servicesMu.Unlock()
kd.servicesResourceVersion = services.ResourceVersion
for idx, service := range services.Items {
sourceNames = append(sourceNames, serviceSource(&service))
namespace, ok := kd.services[service.ObjectMeta.Namespace]
if !ok {
namespace = map[string]*Service{}
kd.services[service.ObjectMeta.Namespace] = namespace
}
namespace[service.ObjectMeta.Name] = &services.Items[idx]
}
return sourceNames
}
// Run implements the TargetProvider interface.
func (kd *KubernetesDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
defer close(ch)
select {
case ch <- kd.updateNodesTargetGroup():
case <-done:
return
}
for _, ns := range kd.services {
for _, service := range ns {
select {
case ch <- kd.addService(service):
case <-done:
return
}
}
}
retryInterval := time.Duration(kd.Conf.RetryInterval)
update := make(chan interface{}, 10)
go kd.watchNodes(update, done, retryInterval)
go kd.watchServices(update, done, retryInterval)
go kd.watchServiceEndpoints(update, done, retryInterval)
var tg *config.TargetGroup
for {
select {
case <-done:
return
case event := <-update:
switch obj := event.(type) {
case *nodeEvent:
kd.updateNode(obj.Node, obj.EventType)
tg = kd.updateNodesTargetGroup()
case *serviceEvent:
tg = kd.updateService(obj.Service, obj.EventType)
case *endpointsEvent:
tg = kd.updateServiceEndpoints(obj.Endpoints, obj.EventType)
}
}
select {
case ch <- tg:
case <-done:
return
}
}
}
func (kd *KubernetesDiscovery) updateNodesTargetGroup() *config.TargetGroup {
kd.nodesMu.Lock()
defer kd.nodesMu.Unlock()
tg := &config.TargetGroup{Source: nodesTargetGroupName}
// Now let's loop through the nodes & add them to the target group with appropriate labels.
for nodeName, node := range kd.nodes {
address := fmt.Sprintf("%s:%d", node.Status.Addresses[0].Address, kd.Conf.KubeletPort)
t := clientmodel.LabelSet{
clientmodel.AddressLabel: clientmodel.LabelValue(address),
nodeLabel: clientmodel.LabelValue(nodeName),
}
for k, v := range node.ObjectMeta.Labels {
labelName := strutil.SanitizeLabelName(nodeLabelPrefix + k)
t[clientmodel.LabelName(labelName)] = clientmodel.LabelValue(v)
}
tg.Targets = append(tg.Targets, t)
}
return tg
}
func (kd *KubernetesDiscovery) updateNode(node *Node, eventType EventType) {
kd.nodesMu.Lock()
defer kd.nodesMu.Unlock()
updatedNodeName := node.ObjectMeta.Name
switch eventType {
case deleted:
// Deleted - remove from nodes map.
delete(kd.nodes, updatedNodeName)
case added, modified:
// Added/Modified - update the node in the nodes map.
kd.nodes[updatedNodeName] = node
}
}
// watchNodes watches nodes as they come & go.
func (kd *KubernetesDiscovery) watchNodes(events chan interface{}, done <-chan struct{}, retryInterval time.Duration) {
until(func() {
req, err := http.NewRequest("GET", kd.Conf.Server+nodesURL, nil)
if err != nil {
log.Errorf("Failed to watch nodes: %s", err)
return
}
values := req.URL.Query()
values.Add("watch", "true")
values.Add("resourceVersion", kd.nodesResourceVersion)
req.URL.RawQuery = values.Encode()
res, err := kd.client.Do(req)
if err != nil {
log.Errorf("Failed to watch nodes: %s", err)
return
}
if res.StatusCode != http.StatusOK {
log.Errorf("Failed to watch nodes: %s", res.StatusCode)
return
}
d := json.NewDecoder(res.Body)
for {
var event nodeEvent
if err := d.Decode(&event); err != nil {
log.Errorf("Failed to watch nodes: %s", err)
return
}
kd.nodesResourceVersion = event.Node.ObjectMeta.ResourceVersion
select {
case events <- &event:
case <-done:
}
}
}, retryInterval, done)
}
// watchServices watches services as they come & go.
func (kd *KubernetesDiscovery) watchServices(events chan interface{}, done <-chan struct{}, retryInterval time.Duration) {
until(func() {
req, err := http.NewRequest("GET", kd.Conf.Server+servicesURL, nil)
if err != nil {
log.Errorf("Failed to watch services: %s", err)
return
}
values := req.URL.Query()
values.Add("watch", "true")
values.Add("resourceVersion", kd.servicesResourceVersion)
req.URL.RawQuery = values.Encode()
res, err := kd.client.Do(req)
if err != nil {
log.Errorf("Failed to watch services: %s", err)
return
}
if res.StatusCode != http.StatusOK {
log.Errorf("Failed to watch services: %s", res.StatusCode)
return
}
d := json.NewDecoder(res.Body)
for {
var event serviceEvent
if err := d.Decode(&event); err != nil {
log.Errorf("Unable to watch services: %s", err)
return
}
kd.servicesResourceVersion = event.Service.ObjectMeta.ResourceVersion
select {
case events <- &event:
case <-done:
}
}
}, retryInterval, done)
}
func (kd *KubernetesDiscovery) updateService(service *Service, eventType EventType) *config.TargetGroup {
kd.servicesMu.Lock()
defer kd.servicesMu.Unlock()
var (
name = service.ObjectMeta.Name
namespace = service.ObjectMeta.Namespace
_, exists = kd.services[namespace][name]
)
switch eventType {
case deleted:
if exists {
return kd.deleteService(service)
}
case added, modified:
return kd.addService(service)
}
return nil
}
func (kd *KubernetesDiscovery) deleteService(service *Service) *config.TargetGroup {
tg := &config.TargetGroup{Source: serviceSource(service)}
delete(kd.services[service.ObjectMeta.Namespace], service.ObjectMeta.Name)
if len(kd.services[service.ObjectMeta.Namespace]) == 0 {
delete(kd.services, service.ObjectMeta.Namespace)
}
return tg
}
func (kd *KubernetesDiscovery) addService(service *Service) *config.TargetGroup {
namespace, ok := kd.services[service.ObjectMeta.Namespace]
if !ok {
namespace = map[string]*Service{}
kd.services[service.ObjectMeta.Namespace] = namespace
}
namespace[service.ObjectMeta.Name] = service
endpointURL := fmt.Sprintf(serviceEndpointsURL, service.ObjectMeta.Namespace, service.ObjectMeta.Name)
res, err := kd.client.Get(kd.Conf.Server + endpointURL)
if err != nil {
log.Errorf("Error getting service endpoints: %s", err)
return nil
}
if res.StatusCode != http.StatusOK {
log.Errorf("Failed to get service endpoints: %s", res.StatusCode)
return nil
}
var endpoints Endpoints
if err := json.NewDecoder(res.Body).Decode(&endpoints); err != nil {
log.Errorf("Error getting service endpoints: %s", err)
return nil
}
return kd.updateServiceTargetGroup(service, &endpoints)
}
func (kd *KubernetesDiscovery) updateServiceTargetGroup(service *Service, endpoints *Endpoints) *config.TargetGroup {
tg := &config.TargetGroup{
Source: serviceSource(service),
Labels: clientmodel.LabelSet{
serviceNamespaceLabel: clientmodel.LabelValue(service.ObjectMeta.Namespace),
serviceNameLabel: clientmodel.LabelValue(service.ObjectMeta.Name),
},
}
for k, v := range service.ObjectMeta.Labels {
labelName := strutil.SanitizeLabelName(serviceLabelPrefix + k)
tg.Labels[clientmodel.LabelName(labelName)] = clientmodel.LabelValue(v)
}
for k, v := range service.ObjectMeta.Annotations {
labelName := strutil.SanitizeLabelName(serviceAnnotationPrefix + k)
tg.Labels[clientmodel.LabelName(labelName)] = clientmodel.LabelValue(v)
}
// Now let's loop through the endpoints & add them to the target group with appropriate labels.
for _, eps := range endpoints.Subsets {
epPort := eps.Ports[0].Port
for _, addr := range eps.Addresses {
ipAddr := addr.IP
if len(ipAddr) == net.IPv6len {
ipAddr = "[" + ipAddr + "]"
}
address := fmt.Sprintf("%s:%d", ipAddr, epPort)
t := clientmodel.LabelSet{clientmodel.AddressLabel: clientmodel.LabelValue(address)}
tg.Targets = append(tg.Targets, t)
}
}
return tg
}
// watchServiceEndpoints watches service endpoints as they come & go.
func (kd *KubernetesDiscovery) watchServiceEndpoints(events chan interface{}, done <-chan struct{}, retryInterval time.Duration) {
until(func() {
req, err := http.NewRequest("GET", kd.Conf.Server+endpointsURL, nil)
if err != nil {
log.Errorf("Failed to watch service endpoints: %s", err)
return
}
values := req.URL.Query()
values.Add("watch", "true")
values.Add("resourceVersion", kd.servicesResourceVersion)
req.URL.RawQuery = values.Encode()
res, err := kd.client.Do(req)
if err != nil {
log.Errorf("Failed to watch service endpoints: %s", err)
return
}
if res.StatusCode != http.StatusOK {
log.Errorf("Failed to watch service endpoints: %s", res.StatusCode)
return
}
d := json.NewDecoder(res.Body)
for {
var event endpointsEvent
if err := d.Decode(&event); err != nil {
log.Errorf("Unable to watch service endpoints: %s", err)
return
}
kd.servicesResourceVersion = event.Endpoints.ObjectMeta.ResourceVersion
select {
case events <- &event:
case <-done:
}
}
}, retryInterval, done)
}
func (kd *KubernetesDiscovery) updateServiceEndpoints(endpoints *Endpoints, eventType EventType) *config.TargetGroup {
kd.servicesMu.Lock()
defer kd.servicesMu.Unlock()
serviceNamespace := endpoints.ObjectMeta.Namespace
serviceName := endpoints.ObjectMeta.Name
if service, ok := kd.services[serviceNamespace][serviceName]; ok {
return kd.updateServiceTargetGroup(service, endpoints)
}
return nil
}
func newKubernetesHTTPClient(conf *config.KubernetesSDConfig) (*http.Client, error) {
bearerTokenFile := conf.BearerTokenFile
caFile := conf.CAFile
if conf.InCluster {
if len(bearerTokenFile) == 0 {
bearerTokenFile = serviceAccountToken
}
if len(caFile) == 0 {
// With recent versions, the CA certificate is provided as a token
// but we need to handle older versions too. In this case, don't
// set the CAFile & the configuration will have to use Insecure.
if _, err := os.Stat(serviceAccountCACert); err == nil {
caFile = serviceAccountCACert
}
}
}
tlsConfig := &tls.Config{InsecureSkipVerify: conf.Insecure}
// Load client cert if specified.
if len(conf.CertFile) > 0 && len(conf.KeyFile) > 0 {
cert, err := tls.LoadX509KeyPair(conf.CertFile, conf.KeyFile)
if err != nil {
return nil, err
}
tlsConfig.Certificates = []tls.Certificate{cert}
}
caCertPool := x509.NewCertPool()
if len(caFile) > 0 {
// Load CA cert.
caCert, err := ioutil.ReadFile(caFile)
if err != nil {
return nil, err
}
caCertPool.AppendCertsFromPEM(caCert)
}
tlsConfig.RootCAs = caCertPool
tlsConfig.BuildNameToCertificate()
tr := &http.Transport{
Dial: func(netw, addr string) (c net.Conn, err error) {
c, err = net.DialTimeout(netw, addr, time.Duration(conf.RequestTimeout))
return
},
}
tr.TLSClientConfig = tlsConfig
var rt http.RoundTripper
rt = tr
bearerToken, err := ioutil.ReadFile(bearerTokenFile)
if err != nil {
return nil, err
}
if len(bearerToken) > 0 {
rt = httputil.NewBearerAuthRoundTripper(string(bearerToken), rt)
}
if len(conf.Username) > 0 && len(conf.Password) > 0 {
rt = httputil.NewBasicAuthRoundTripper(conf.Username, conf.Password, rt)
}
return &http.Client{
Transport: rt,
}, nil
}
func serviceSource(service *Service) string {
return sourceServicePrefix + ":" + service.ObjectMeta.Namespace + "/" + service.ObjectMeta.Name
}
// Until loops until stop channel is closed, running f every period.
// f may not be invoked if stop channel is already closed.
func until(f func(), period time.Duration, stopCh <-chan struct{}) {
select {
case <-stopCh:
return
default:
f()
}
for {
select {
case <-stopCh:
return
case <-time.After(period):
f()
}
}
}

View File

@ -0,0 +1,210 @@
// Copyright 2015 The Prometheus 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 kubernetes
type EventType string
const (
added EventType = "ADDED"
modified EventType = "MODIFIED"
deleted EventType = "DELETED"
)
type nodeEvent struct {
EventType EventType `json:"type"`
Node *Node `json:"object"`
}
type serviceEvent struct {
EventType EventType `json:"type"`
Service *Service `json:"object"`
}
type endpointsEvent struct {
EventType EventType `json:"type"`
Endpoints *Endpoints `json:"object"`
}
// From here down types are copied from
// https://github.com/GoogleCloudPlatform/kubernetes/blob/master/pkg/api/v1/types.go
// with all currently irrelevant types/fields stripped out. This removes the
// need for any kubernetes dependencies, with the drawback of having to keep
// this file up to date.
// ListMeta describes metadata that synthetic resources must have, including lists and
// various status objects.
type ListMeta struct {
// An opaque value that represents the version of this response for use with optimistic
// concurrency and change monitoring endpoints. Clients must treat these values as opaque
// and values may only be valid for a particular resource or set of resources. Only servers
// will generate resource versions.
ResourceVersion string `json:"resourceVersion,omitempty" description:"string that identifies the internal version of this object that can be used by clients to determine when objects have changed; populated by the system, read-only; value must be treated as opaque by clients and passed unmodified back to the server: http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#concurrency-control-and-consistency"`
}
// ObjectMeta is metadata that all persisted resources must have, which includes all objects
// users must create.
type ObjectMeta struct {
// Name is unique within a namespace. Name is required when creating resources, although
// some resources may allow a client to request the generation of an appropriate name
// automatically. Name is primarily intended for creation idempotence and configuration
// definition.
Name string `json:"name,omitempty" description:"string that identifies an object. Must be unique within a namespace; cannot be updated; see http://releases.k8s.io/HEAD/docs/user-guide/identifiers.md#names"`
// Namespace defines the space within which name must be unique. An empty namespace is
// equivalent to the "default" namespace, but "default" is the canonical representation.
// Not all objects are required to be scoped to a namespace - the value of this field for
// those objects will be empty.
Namespace string `json:"namespace,omitempty" description:"namespace of the object; must be a DNS_LABEL; cannot be updated; see http://releases.k8s.io/HEAD/docs/user-guide/namespaces.md"`
ResourceVersion string `json:"resourceVersion,omitempty" description:"string that identifies the internal version of this object that can be used by clients to determine when objects have changed; populated by the system, read-only; value must be treated as opaque by clients and passed unmodified back to the server: http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#concurrency-control-and-consistency"`
// TODO: replace map[string]string with labels.LabelSet type
Labels map[string]string `json:"labels,omitempty" description:"map of string keys and values that can be used to organize and categorize objects; may match selectors of replication controllers and services; see http://releases.k8s.io/HEAD/docs/user-guide/labels.md"`
// Annotations are unstructured key value data stored with a resource that may be set by
// external tooling. They are not queryable and should be preserved when modifying
// objects.
Annotations map[string]string `json:"annotations,omitempty" description:"map of string keys and values that can be used by external tooling to store and retrieve arbitrary metadata about objects; see http://releases.k8s.io/HEAD/docs/user-guide/annotations.md"`
}
// Protocol defines network protocols supported for things like conatiner ports.
type Protocol string
const (
// ProtocolTCP is the TCP protocol.
ProtocolTCP Protocol = "TCP"
// ProtocolUDP is the UDP protocol.
ProtocolUDP Protocol = "UDP"
)
const (
// NamespaceAll is the default argument to specify on a context when you want to list or filter resources across all namespaces
NamespaceAll string = ""
)
// Container represents a single container that is expected to be run on the host.
type Container struct {
// Required: This must be a DNS_LABEL. Each container in a pod must
// have a unique name.
Name string `json:"name" description:"name of the container; must be a DNS_LABEL and unique within the pod; cannot be updated"`
// Optional.
Image string `json:"image,omitempty" description:"Docker image name; see http://releases.k8s.io/HEAD/docs/user-guide/images.md"`
}
// Service is a named abstraction of software service (for example, mysql) consisting of local port
// (for example 3306) that the proxy listens on, and the selector that determines which pods
// will answer requests sent through the proxy.
type Service struct {
ObjectMeta `json:"metadata,omitempty" description:"standard object metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
}
// ServiceList holds a list of services.
type ServiceList struct {
ListMeta `json:"metadata,omitempty" description:"standard list metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
Items []Service `json:"items" description:"list of services"`
}
// Endpoints is a collection of endpoints that implement the actual service. Example:
// Name: "mysvc",
// Subsets: [
// {
// Addresses: [{"ip": "10.10.1.1"}, {"ip": "10.10.2.2"}],
// Ports: [{"name": "a", "port": 8675}, {"name": "b", "port": 309}]
// },
// {
// Addresses: [{"ip": "10.10.3.3"}],
// Ports: [{"name": "a", "port": 93}, {"name": "b", "port": 76}]
// },
// ]
type Endpoints struct {
ObjectMeta `json:"metadata,omitempty" description:"standard object metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
// The set of all endpoints is the union of all subsets.
Subsets []EndpointSubset `json:"subsets" description:"sets of addresses and ports that comprise a service"`
}
// EndpointSubset is a group of addresses with a common set of ports. The
// expanded set of endpoints is the Cartesian product of Addresses x Ports.
// For example, given:
// {
// Addresses: [{"ip": "10.10.1.1"}, {"ip": "10.10.2.2"}],
// Ports: [{"name": "a", "port": 8675}, {"name": "b", "port": 309}]
// }
// The resulting set of endpoints can be viewed as:
// a: [ 10.10.1.1:8675, 10.10.2.2:8675 ],
// b: [ 10.10.1.1:309, 10.10.2.2:309 ]
type EndpointSubset struct {
Addresses []EndpointAddress `json:"addresses,omitempty" description:"IP addresses which offer the related ports"`
Ports []EndpointPort `json:"ports,omitempty" description:"port numbers available on the related IP addresses"`
}
// EndpointAddress is a tuple that describes single IP address.
type EndpointAddress struct {
// The IP of this endpoint.
// TODO: This should allow hostname or IP, see #4447.
IP string `json:"ip" description:"IP address of the endpoint"`
}
// EndpointPort is a tuple that describes a single port.
type EndpointPort struct {
// The port number.
Port int `json:"port" description:"port number of the endpoint"`
// The IP protocol for this port.
Protocol Protocol `json:"protocol,omitempty" description:"protocol for this port; must be UDP or TCP; TCP if unspecified"`
}
// EndpointsList is a list of endpoints.
type EndpointsList struct {
ListMeta `json:"metadata,omitempty" description:"standard list metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
Items []Endpoints `json:"items" description:"list of endpoints"`
}
// NodeStatus is information about the current status of a node.
type NodeStatus struct {
// Queried from cloud provider, if available.
Addresses []NodeAddress `json:"addresses,omitempty" description:"list of addresses reachable to the node; see http://releases.k8s.io/HEAD/docs/admin/node.md#node-addresses" patchStrategy:"merge" patchMergeKey:"type"`
}
type NodeAddressType string
// These are valid address type of node.
const (
NodeHostName NodeAddressType = "Hostname"
NodeExternalIP NodeAddressType = "ExternalIP"
NodeInternalIP NodeAddressType = "InternalIP"
)
type NodeAddress struct {
Type NodeAddressType `json:"type" description:"node address type, one of Hostname, ExternalIP or InternalIP"`
Address string `json:"address" description:"the node address"`
}
// Node is a worker node in Kubernetes, formerly known as minion.
// Each node will have a unique identifier in the cache (i.e. in etcd).
type Node struct {
ObjectMeta `json:"metadata,omitempty" description:"standard object metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
// Status describes the current status of a Node
Status NodeStatus `json:"status,omitempty" description:"most recently observed status of the node; populated by the system, read-only; http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#spec-and-status"`
}
// NodeList is the whole list of all Nodes which have been registered with master.
type NodeList struct {
ListMeta `json:"metadata,omitempty" description:"standard list metadata; see http://releases.k8s.io/HEAD/docs/devel/api-conventions.md#metadata"`
Items []Node `json:"items" description:"list of nodes"`
}