created docker-prometheus compose file and edited the Prometheus yml file
This commit is contained in:
293
prom/retrieval/discovery/consul.go
Normal file
293
prom/retrieval/discovery/consul.go
Normal file
@ -0,0 +1,293 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/log"
|
||||
|
||||
consul "github.com/hashicorp/consul/api"
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
const (
|
||||
consulWatchTimeout = 30 * time.Second
|
||||
consulRetryInterval = 15 * time.Second
|
||||
|
||||
// ConsuleAddressLabel is the name for the label containing a target's address.
|
||||
ConsulAddressLabel = clientmodel.MetaLabelPrefix + "consul_address"
|
||||
// ConsuleNodeLabel is the name for the label containing a target's node name.
|
||||
ConsulNodeLabel = clientmodel.MetaLabelPrefix + "consul_node"
|
||||
// ConsulTagsLabel is the name of the label containing the tags assigned to the target.
|
||||
ConsulTagsLabel = clientmodel.MetaLabelPrefix + "consul_tags"
|
||||
// ConsulServiceLabel is the name of the label containing the service name.
|
||||
ConsulServiceLabel = clientmodel.MetaLabelPrefix + "consul_service"
|
||||
// ConsulServiceAddressLabel is the name of the label containing the (optional) service address.
|
||||
ConsulServiceAddressLabel = clientmodel.MetaLabelPrefix + "consul_service_address"
|
||||
// ConsulServicePortLabel is the name of the label containing the service port.
|
||||
ConsulServicePortLabel = clientmodel.MetaLabelPrefix + "consul_service_port"
|
||||
// ConsulDCLabel is the name of the label containing the datacenter ID.
|
||||
ConsulDCLabel = clientmodel.MetaLabelPrefix + "consul_dc"
|
||||
)
|
||||
|
||||
// ConsulDiscovery retrieves target information from a Consul server
|
||||
// and updates them via watches.
|
||||
type ConsulDiscovery struct {
|
||||
client *consul.Client
|
||||
clientConf *consul.Config
|
||||
clientDatacenter string
|
||||
tagSeparator string
|
||||
scrapedServices map[string]struct{}
|
||||
|
||||
mu sync.RWMutex
|
||||
services map[string]*consulService
|
||||
}
|
||||
|
||||
// consulService contains data belonging to the same service.
|
||||
type consulService struct {
|
||||
name string
|
||||
tgroup *config.TargetGroup
|
||||
lastIndex uint64
|
||||
removed bool
|
||||
running bool
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
// NewConsulDiscovery returns a new ConsulDiscovery for the given config.
|
||||
func NewConsulDiscovery(conf *config.ConsulSDConfig) *ConsulDiscovery {
|
||||
clientConf := &consul.Config{
|
||||
Address: conf.Server,
|
||||
Scheme: conf.Scheme,
|
||||
Datacenter: conf.Datacenter,
|
||||
Token: conf.Token,
|
||||
HttpAuth: &consul.HttpBasicAuth{
|
||||
Username: conf.Username,
|
||||
Password: conf.Password,
|
||||
},
|
||||
}
|
||||
client, err := consul.NewClient(clientConf)
|
||||
if err != nil {
|
||||
// NewClient always returns a nil error.
|
||||
panic(fmt.Errorf("discovery.NewConsulDiscovery: %s", err))
|
||||
}
|
||||
cd := &ConsulDiscovery{
|
||||
client: client,
|
||||
clientConf: clientConf,
|
||||
tagSeparator: conf.TagSeparator,
|
||||
scrapedServices: map[string]struct{}{},
|
||||
services: map[string]*consulService{},
|
||||
}
|
||||
// If the datacenter isn't set in the clientConf, let's get it from the local Consul agent
|
||||
// (Consul default is to use local node's datacenter if one isn't given for a query).
|
||||
if clientConf.Datacenter == "" {
|
||||
info, err := client.Agent().Self()
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("discovery.NewConsulDiscovery: %s", err))
|
||||
}
|
||||
cd.clientDatacenter = info["Config"]["Datacenter"].(string)
|
||||
} else {
|
||||
cd.clientDatacenter = clientConf.Datacenter
|
||||
}
|
||||
for _, name := range conf.Services {
|
||||
cd.scrapedServices[name] = struct{}{}
|
||||
}
|
||||
return cd
|
||||
}
|
||||
|
||||
// Sources implements the TargetProvider interface.
|
||||
func (cd *ConsulDiscovery) Sources() []string {
|
||||
clientConf := *cd.clientConf
|
||||
clientConf.HttpClient = &http.Client{Timeout: 5 * time.Second}
|
||||
|
||||
client, err := consul.NewClient(&clientConf)
|
||||
if err != nil {
|
||||
// NewClient always returns a nil error.
|
||||
panic(fmt.Errorf("discovery.ConsulDiscovery.Sources: %s", err))
|
||||
}
|
||||
|
||||
srvs, _, err := client.Catalog().Services(nil)
|
||||
if err != nil {
|
||||
log.Errorf("Error refreshing service list: %s", err)
|
||||
return nil
|
||||
}
|
||||
cd.mu.Lock()
|
||||
defer cd.mu.Unlock()
|
||||
|
||||
srcs := make([]string, 0, len(srvs))
|
||||
for name := range srvs {
|
||||
if _, ok := cd.scrapedServices[name]; len(cd.scrapedServices) == 0 || ok {
|
||||
srcs = append(srcs, name)
|
||||
}
|
||||
}
|
||||
return srcs
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (cd *ConsulDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
defer cd.stop()
|
||||
|
||||
update := make(chan *consulService, 10)
|
||||
go cd.watchServices(update, done)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
case srv := <-update:
|
||||
if srv.removed {
|
||||
close(srv.done)
|
||||
|
||||
// Send clearing update.
|
||||
ch <- &config.TargetGroup{Source: srv.name}
|
||||
break
|
||||
}
|
||||
// Launch watcher for the service.
|
||||
if !srv.running {
|
||||
go cd.watchService(srv, ch)
|
||||
srv.running = true
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (cd *ConsulDiscovery) stop() {
|
||||
// The lock prevents Run from terminating while the watchers attempt
|
||||
// to send on their channels.
|
||||
cd.mu.Lock()
|
||||
defer cd.mu.Unlock()
|
||||
|
||||
for _, srv := range cd.services {
|
||||
close(srv.done)
|
||||
}
|
||||
}
|
||||
|
||||
// watchServices retrieves updates from Consul's services endpoint and sends
|
||||
// potential updates to the update channel.
|
||||
func (cd *ConsulDiscovery) watchServices(update chan<- *consulService, done <-chan struct{}) {
|
||||
var lastIndex uint64
|
||||
for {
|
||||
catalog := cd.client.Catalog()
|
||||
srvs, meta, err := catalog.Services(&consul.QueryOptions{
|
||||
WaitIndex: lastIndex,
|
||||
WaitTime: consulWatchTimeout,
|
||||
})
|
||||
if err != nil {
|
||||
log.Errorf("Error refreshing service list: %s", err)
|
||||
time.Sleep(consulRetryInterval)
|
||||
}
|
||||
// If the index equals the previous one, the watch timed out with no update.
|
||||
if meta.LastIndex == lastIndex {
|
||||
continue
|
||||
}
|
||||
lastIndex = meta.LastIndex
|
||||
|
||||
cd.mu.Lock()
|
||||
select {
|
||||
case <-done:
|
||||
cd.mu.Unlock()
|
||||
return
|
||||
default:
|
||||
// Continue.
|
||||
}
|
||||
// Check for new services.
|
||||
for name := range srvs {
|
||||
if _, ok := cd.scrapedServices[name]; len(cd.scrapedServices) > 0 && !ok {
|
||||
continue
|
||||
}
|
||||
srv, ok := cd.services[name]
|
||||
if !ok {
|
||||
srv = &consulService{
|
||||
name: name,
|
||||
tgroup: &config.TargetGroup{},
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
srv.tgroup.Source = name
|
||||
cd.services[name] = srv
|
||||
}
|
||||
srv.tgroup.Labels = clientmodel.LabelSet{
|
||||
ConsulServiceLabel: clientmodel.LabelValue(name),
|
||||
ConsulDCLabel: clientmodel.LabelValue(cd.clientDatacenter),
|
||||
}
|
||||
update <- srv
|
||||
}
|
||||
// Check for removed services.
|
||||
for name, srv := range cd.services {
|
||||
if _, ok := srvs[name]; !ok {
|
||||
srv.removed = true
|
||||
update <- srv
|
||||
delete(cd.services, name)
|
||||
}
|
||||
}
|
||||
cd.mu.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
// watchService retrieves updates about srv from Consul's service endpoint.
|
||||
// On a potential update the resulting target group is sent to ch.
|
||||
func (cd *ConsulDiscovery) watchService(srv *consulService, ch chan<- *config.TargetGroup) {
|
||||
catalog := cd.client.Catalog()
|
||||
for {
|
||||
nodes, meta, err := catalog.Service(srv.name, "", &consul.QueryOptions{
|
||||
WaitIndex: srv.lastIndex,
|
||||
WaitTime: consulWatchTimeout,
|
||||
})
|
||||
if err != nil {
|
||||
log.Errorf("Error refreshing service %s: %s", srv.name, err)
|
||||
time.Sleep(consulRetryInterval)
|
||||
continue
|
||||
}
|
||||
// If the index equals the previous one, the watch timed out with no update.
|
||||
if meta.LastIndex == srv.lastIndex {
|
||||
continue
|
||||
}
|
||||
srv.lastIndex = meta.LastIndex
|
||||
srv.tgroup.Targets = make([]clientmodel.LabelSet, 0, len(nodes))
|
||||
|
||||
for _, node := range nodes {
|
||||
addr := fmt.Sprintf("%s:%d", node.Address, node.ServicePort)
|
||||
// We surround the separated list with the separator as well. This way regular expressions
|
||||
// in relabeling rules don't have to consider tag positions.
|
||||
tags := cd.tagSeparator + strings.Join(node.ServiceTags, cd.tagSeparator) + cd.tagSeparator
|
||||
|
||||
srv.tgroup.Targets = append(srv.tgroup.Targets, clientmodel.LabelSet{
|
||||
clientmodel.AddressLabel: clientmodel.LabelValue(addr),
|
||||
ConsulAddressLabel: clientmodel.LabelValue(node.Address),
|
||||
ConsulNodeLabel: clientmodel.LabelValue(node.Node),
|
||||
ConsulTagsLabel: clientmodel.LabelValue(tags),
|
||||
ConsulServiceAddressLabel: clientmodel.LabelValue(node.ServiceAddress),
|
||||
ConsulServicePortLabel: clientmodel.LabelValue(strconv.Itoa(node.ServicePort)),
|
||||
})
|
||||
}
|
||||
|
||||
cd.mu.Lock()
|
||||
select {
|
||||
case <-srv.done:
|
||||
cd.mu.Unlock()
|
||||
return
|
||||
default:
|
||||
// Continue.
|
||||
}
|
||||
ch <- srv.tgroup
|
||||
cd.mu.Unlock()
|
||||
}
|
||||
}
|
239
prom/retrieval/discovery/dns.go
Normal file
239
prom/retrieval/discovery/dns.go
Normal file
@ -0,0 +1,239 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/miekg/dns"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/log"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
const (
|
||||
resolvConf = "/etc/resolv.conf"
|
||||
|
||||
DNSNameLabel = clientmodel.MetaLabelPrefix + "dns_name"
|
||||
|
||||
// Constants for instrumentation.
|
||||
namespace = "prometheus"
|
||||
interval = "interval"
|
||||
)
|
||||
|
||||
var (
|
||||
dnsSDLookupsCount = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: namespace,
|
||||
Name: "dns_sd_lookups_total",
|
||||
Help: "The number of DNS-SD lookups.",
|
||||
})
|
||||
dnsSDLookupFailuresCount = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: namespace,
|
||||
Name: "dns_sd_lookup_failures_total",
|
||||
Help: "The number of DNS-SD lookup failures.",
|
||||
})
|
||||
)
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(dnsSDLookupFailuresCount)
|
||||
prometheus.MustRegister(dnsSDLookupsCount)
|
||||
}
|
||||
|
||||
// DNSDiscovery periodically performs DNS-SD requests. It implements
|
||||
// the TargetProvider interface.
|
||||
type DNSDiscovery struct {
|
||||
names []string
|
||||
|
||||
done chan struct{}
|
||||
interval time.Duration
|
||||
m sync.RWMutex
|
||||
port int
|
||||
qtype uint16
|
||||
}
|
||||
|
||||
// NewDNSDiscovery returns a new DNSDiscovery which periodically refreshes its targets.
|
||||
func NewDNSDiscovery(conf *config.DNSSDConfig) *DNSDiscovery {
|
||||
qtype := dns.TypeSRV
|
||||
switch strings.ToUpper(conf.Type) {
|
||||
case "A":
|
||||
qtype = dns.TypeA
|
||||
case "AAAA":
|
||||
qtype = dns.TypeAAAA
|
||||
case "SRV":
|
||||
qtype = dns.TypeSRV
|
||||
}
|
||||
return &DNSDiscovery{
|
||||
names: conf.Names,
|
||||
done: make(chan struct{}),
|
||||
interval: time.Duration(conf.RefreshInterval),
|
||||
qtype: qtype,
|
||||
port: conf.Port,
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (dd *DNSDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
|
||||
ticker := time.NewTicker(dd.interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
// Get an initial set right away.
|
||||
dd.refreshAll(ch)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
dd.refreshAll(ch)
|
||||
case <-done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Sources implements the TargetProvider interface.
|
||||
func (dd *DNSDiscovery) Sources() []string {
|
||||
var srcs []string
|
||||
for _, name := range dd.names {
|
||||
srcs = append(srcs, name)
|
||||
}
|
||||
return srcs
|
||||
}
|
||||
|
||||
func (dd *DNSDiscovery) refreshAll(ch chan<- *config.TargetGroup) {
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(dd.names))
|
||||
for _, name := range dd.names {
|
||||
go func(n string) {
|
||||
if err := dd.refresh(n, ch); err != nil {
|
||||
log.Errorf("Error refreshing DNS targets: %s", err)
|
||||
}
|
||||
wg.Done()
|
||||
}(name)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
func (dd *DNSDiscovery) refresh(name string, ch chan<- *config.TargetGroup) error {
|
||||
response, err := lookupAll(name, dd.qtype)
|
||||
dnsSDLookupsCount.Inc()
|
||||
if err != nil {
|
||||
dnsSDLookupFailuresCount.Inc()
|
||||
return err
|
||||
}
|
||||
|
||||
tg := &config.TargetGroup{}
|
||||
for _, record := range response.Answer {
|
||||
target := clientmodel.LabelValue("")
|
||||
switch addr := record.(type) {
|
||||
case *dns.SRV:
|
||||
// Remove the final dot from rooted DNS names to make them look more usual.
|
||||
addr.Target = strings.TrimRight(addr.Target, ".")
|
||||
|
||||
target = clientmodel.LabelValue(fmt.Sprintf("%s:%d", addr.Target, addr.Port))
|
||||
case *dns.A:
|
||||
target = clientmodel.LabelValue(fmt.Sprintf("%s:%d", addr.A, dd.port))
|
||||
case *dns.AAAA:
|
||||
target = clientmodel.LabelValue(fmt.Sprintf("%s:%d", addr.AAAA, dd.port))
|
||||
default:
|
||||
log.Warnf("%q is not a valid SRV record", record)
|
||||
continue
|
||||
|
||||
}
|
||||
tg.Targets = append(tg.Targets, clientmodel.LabelSet{
|
||||
clientmodel.AddressLabel: target,
|
||||
DNSNameLabel: clientmodel.LabelValue(name),
|
||||
})
|
||||
}
|
||||
|
||||
tg.Source = name
|
||||
ch <- tg
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func lookupAll(name string, qtype uint16) (*dns.Msg, error) {
|
||||
conf, err := dns.ClientConfigFromFile(resolvConf)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not load resolv.conf: %s", err)
|
||||
}
|
||||
|
||||
client := &dns.Client{}
|
||||
response := &dns.Msg{}
|
||||
|
||||
for _, server := range conf.Servers {
|
||||
servAddr := net.JoinHostPort(server, conf.Port)
|
||||
for _, suffix := range conf.Search {
|
||||
response, err = lookup(name, qtype, client, servAddr, suffix, false)
|
||||
if err != nil {
|
||||
log.Warnf("resolving %s.%s failed: %s", name, suffix, err)
|
||||
continue
|
||||
}
|
||||
if len(response.Answer) > 0 {
|
||||
return response, nil
|
||||
}
|
||||
}
|
||||
response, err = lookup(name, qtype, client, servAddr, "", false)
|
||||
if err == nil {
|
||||
return response, nil
|
||||
}
|
||||
}
|
||||
return response, fmt.Errorf("could not resolve %s: No server responded", name)
|
||||
}
|
||||
|
||||
func lookup(name string, queryType uint16, client *dns.Client, servAddr string, suffix string, edns bool) (*dns.Msg, error) {
|
||||
msg := &dns.Msg{}
|
||||
lname := strings.Join([]string{name, suffix}, ".")
|
||||
msg.SetQuestion(dns.Fqdn(lname), queryType)
|
||||
|
||||
if edns {
|
||||
opt := &dns.OPT{
|
||||
Hdr: dns.RR_Header{
|
||||
Name: ".",
|
||||
Rrtype: dns.TypeOPT,
|
||||
},
|
||||
}
|
||||
opt.SetUDPSize(dns.DefaultMsgSize)
|
||||
msg.Extra = append(msg.Extra, opt)
|
||||
}
|
||||
|
||||
response, _, err := client.Exchange(msg, servAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if msg.Id != response.Id {
|
||||
return nil, fmt.Errorf("DNS ID mismatch, request: %d, response: %d", msg.Id, response.Id)
|
||||
}
|
||||
|
||||
if response.MsgHdr.Truncated {
|
||||
if client.Net == "tcp" {
|
||||
return nil, fmt.Errorf("got truncated message on tcp")
|
||||
}
|
||||
if edns { // Truncated even though EDNS is used
|
||||
client.Net = "tcp"
|
||||
}
|
||||
return lookup(name, queryType, client, servAddr, suffix, !edns)
|
||||
}
|
||||
|
||||
return response, nil
|
||||
}
|
252
prom/retrieval/discovery/file.go
Normal file
252
prom/retrieval/discovery/file.go
Normal file
@ -0,0 +1,252 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/log"
|
||||
"gopkg.in/fsnotify.v1"
|
||||
"gopkg.in/yaml.v2"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
const FileSDFilepathLabel = clientmodel.MetaLabelPrefix + "filepath"
|
||||
|
||||
// FileDiscovery provides service discovery functionality based
|
||||
// on files that contain target groups in JSON or YAML format. Refreshing
|
||||
// happens using file watches and periodic refreshes.
|
||||
type FileDiscovery struct {
|
||||
paths []string
|
||||
watcher *fsnotify.Watcher
|
||||
interval time.Duration
|
||||
|
||||
// lastRefresh stores which files were found during the last refresh
|
||||
// and how many target groups they contained.
|
||||
// This is used to detect deleted target groups.
|
||||
lastRefresh map[string]int
|
||||
}
|
||||
|
||||
// NewFileDiscovery returns a new file discovery for the given paths.
|
||||
func NewFileDiscovery(conf *config.FileSDConfig) *FileDiscovery {
|
||||
return &FileDiscovery{
|
||||
paths: conf.Names,
|
||||
interval: time.Duration(conf.RefreshInterval),
|
||||
}
|
||||
}
|
||||
|
||||
// Sources implements the TargetProvider interface.
|
||||
func (fd *FileDiscovery) Sources() []string {
|
||||
var srcs []string
|
||||
// As we allow multiple target groups per file we have no choice
|
||||
// but to parse them all.
|
||||
for _, p := range fd.listFiles() {
|
||||
tgroups, err := readFile(p)
|
||||
if err != nil {
|
||||
log.Errorf("Error reading file %q: ", p, err)
|
||||
}
|
||||
for _, tg := range tgroups {
|
||||
srcs = append(srcs, tg.Source)
|
||||
}
|
||||
}
|
||||
return srcs
|
||||
}
|
||||
|
||||
// listFiles returns a list of all files that match the configured patterns.
|
||||
func (fd *FileDiscovery) listFiles() []string {
|
||||
var paths []string
|
||||
for _, p := range fd.paths {
|
||||
files, err := filepath.Glob(p)
|
||||
if err != nil {
|
||||
log.Errorf("Error expanding glob %q: %s", p, err)
|
||||
continue
|
||||
}
|
||||
paths = append(paths, files...)
|
||||
}
|
||||
return paths
|
||||
}
|
||||
|
||||
// watchFiles sets watches on all full paths or directories that were configured for
|
||||
// this file discovery.
|
||||
func (fd *FileDiscovery) watchFiles() {
|
||||
if fd.watcher == nil {
|
||||
panic("no watcher configured")
|
||||
}
|
||||
for _, p := range fd.paths {
|
||||
if idx := strings.LastIndex(p, "/"); idx > -1 {
|
||||
p = p[:idx]
|
||||
} else {
|
||||
p = "./"
|
||||
}
|
||||
if err := fd.watcher.Add(p); err != nil {
|
||||
log.Errorf("Error adding file watch for %q: %s", p, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (fd *FileDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
defer fd.stop()
|
||||
|
||||
watcher, err := fsnotify.NewWatcher()
|
||||
if err != nil {
|
||||
log.Errorf("Error creating file watcher: %s", err)
|
||||
return
|
||||
}
|
||||
fd.watcher = watcher
|
||||
|
||||
fd.refresh(ch)
|
||||
|
||||
ticker := time.NewTicker(fd.interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
// Stopping has priority over refreshing. Thus we wrap the actual select
|
||||
// clause to always catch done signals.
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
default:
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
|
||||
case event := <-fd.watcher.Events:
|
||||
// fsnotify sometimes sends a bunch of events without name or operation.
|
||||
// It's unclear what they are and why they are sent - filter them out.
|
||||
if len(event.Name) == 0 {
|
||||
break
|
||||
}
|
||||
// Everything but a chmod requires rereading.
|
||||
if event.Op^fsnotify.Chmod == 0 {
|
||||
break
|
||||
}
|
||||
// Changes to a file can spawn various sequences of events with
|
||||
// different combinations of operations. For all practical purposes
|
||||
// this is inaccurate.
|
||||
// The most reliable solution is to reload everything if anything happens.
|
||||
fd.refresh(ch)
|
||||
|
||||
case <-ticker.C:
|
||||
// Setting a new watch after an update might fail. Make sure we don't lose
|
||||
// those files forever.
|
||||
fd.refresh(ch)
|
||||
|
||||
case err := <-fd.watcher.Errors:
|
||||
if err != nil {
|
||||
log.Errorf("Error on file watch: %s", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// refresh reads all files matching the discoveries patterns and sends the respective
|
||||
// updated target groups through the channel.
|
||||
func (fd *FileDiscovery) refresh(ch chan<- *config.TargetGroup) {
|
||||
ref := map[string]int{}
|
||||
for _, p := range fd.listFiles() {
|
||||
tgroups, err := readFile(p)
|
||||
if err != nil {
|
||||
log.Errorf("Error reading file %q: %s", p, err)
|
||||
// Prevent deletion down below.
|
||||
ref[p] = fd.lastRefresh[p]
|
||||
continue
|
||||
}
|
||||
for _, tg := range tgroups {
|
||||
ch <- tg
|
||||
}
|
||||
ref[p] = len(tgroups)
|
||||
}
|
||||
// Send empty updates for sources that disappeared.
|
||||
for f, n := range fd.lastRefresh {
|
||||
m, ok := ref[f]
|
||||
if !ok || n > m {
|
||||
for i := m; i < n; i++ {
|
||||
ch <- &config.TargetGroup{Source: fileSource(f, i)}
|
||||
}
|
||||
}
|
||||
}
|
||||
fd.lastRefresh = ref
|
||||
|
||||
fd.watchFiles()
|
||||
}
|
||||
|
||||
// fileSource returns a source ID for the i-th target group in the file.
|
||||
func fileSource(filename string, i int) string {
|
||||
return fmt.Sprintf("%s:%d", filename, i)
|
||||
}
|
||||
|
||||
// stop shuts down the file watcher.
|
||||
func (fd *FileDiscovery) stop() {
|
||||
log.Debugf("Stopping file discovery for %s...", fd.paths)
|
||||
|
||||
// Closing the watcher will deadlock unless all events and errors are drained.
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-fd.watcher.Errors:
|
||||
case <-fd.watcher.Events:
|
||||
// Drain all events and errors.
|
||||
default:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
fd.watcher.Close()
|
||||
|
||||
log.Debugf("File discovery for %s stopped.", fd.paths)
|
||||
}
|
||||
|
||||
// readFile reads a JSON or YAML list of targets groups from the file, depending on its
|
||||
// file extension. It returns full configuration target groups.
|
||||
func readFile(filename string) ([]*config.TargetGroup, error) {
|
||||
content, err := ioutil.ReadFile(filename)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var targetGroups []*config.TargetGroup
|
||||
|
||||
switch ext := filepath.Ext(filename); strings.ToLower(ext) {
|
||||
case ".json":
|
||||
if err := json.Unmarshal(content, &targetGroups); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case ".yml", ".yaml":
|
||||
if err := yaml.Unmarshal(content, &targetGroups); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
default:
|
||||
panic(fmt.Errorf("retrieval.FileDiscovery.readFile: unhandled file extension %q", ext))
|
||||
}
|
||||
|
||||
for i, tg := range targetGroups {
|
||||
tg.Source = fileSource(filename, i)
|
||||
if tg.Labels == nil {
|
||||
tg.Labels = clientmodel.LabelSet{}
|
||||
}
|
||||
tg.Labels[FileSDFilepathLabel] = clientmodel.LabelValue(filename)
|
||||
}
|
||||
return targetGroups, nil
|
||||
}
|
109
prom/retrieval/discovery/file_test.go
Normal file
109
prom/retrieval/discovery/file_test.go
Normal file
@ -0,0 +1,109 @@
|
||||
package discovery
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
func TestFileSD(t *testing.T) {
|
||||
defer os.Remove("fixtures/_test.yml")
|
||||
defer os.Remove("fixtures/_test.json")
|
||||
testFileSD(t, ".yml")
|
||||
testFileSD(t, ".json")
|
||||
}
|
||||
|
||||
func testFileSD(t *testing.T, ext string) {
|
||||
// As interval refreshing is more of a fallback, we only want to test
|
||||
// whether file watches work as expected.
|
||||
var conf config.FileSDConfig
|
||||
conf.Names = []string{"fixtures/_*" + ext}
|
||||
conf.RefreshInterval = config.Duration(1 * time.Hour)
|
||||
|
||||
var (
|
||||
fsd = NewFileDiscovery(&conf)
|
||||
ch = make(chan *config.TargetGroup)
|
||||
done = make(chan struct{})
|
||||
)
|
||||
go fsd.Run(ch, done)
|
||||
defer close(done)
|
||||
|
||||
select {
|
||||
case <-time.After(25 * time.Millisecond):
|
||||
// Expected.
|
||||
case tg := <-ch:
|
||||
t.Fatalf("Unexpected target group in file discovery: %s", tg)
|
||||
}
|
||||
|
||||
newf, err := os.Create("fixtures/_test" + ext)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer newf.Close()
|
||||
|
||||
f, err := os.Open("fixtures/target_groups" + ext)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
_, err = io.Copy(newf, f)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
newf.Close()
|
||||
|
||||
// The files contain two target groups which are read and sent in order.
|
||||
select {
|
||||
case <-time.After(15 * time.Second):
|
||||
t.Fatalf("Expected new target group but got none")
|
||||
case tg := <-ch:
|
||||
if _, ok := tg.Labels["foo"]; !ok {
|
||||
t.Fatalf("Label not parsed")
|
||||
}
|
||||
if tg.String() != fmt.Sprintf("fixtures/_test%s:0", ext) {
|
||||
t.Fatalf("Unexpected target group", tg)
|
||||
}
|
||||
}
|
||||
select {
|
||||
case <-time.After(15 * time.Second):
|
||||
t.Fatalf("Expected new target group but got none")
|
||||
case tg := <-ch:
|
||||
if tg.String() != fmt.Sprintf("fixtures/_test%s:1", ext) {
|
||||
t.Fatalf("Unexpected target group %s", tg)
|
||||
}
|
||||
}
|
||||
// Based on unknown circumstances, sometimes fsnotify will trigger more events in
|
||||
// some runs (which might be empty, chains of different operations etc.).
|
||||
// We have to drain those (as the target manager would) to avoid deadlocking and must
|
||||
// not try to make sense of it all...
|
||||
go func() {
|
||||
for tg := range ch {
|
||||
// Below we will change the file to a bad syntax. Previously extracted target
|
||||
// groups must not be deleted via sending an empty target group.
|
||||
if len(tg.Targets) == 0 {
|
||||
t.Errorf("Unexpected empty target group received: %s", tg)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
newf, err = os.Create("fixtures/_test.new")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.Remove(newf.Name())
|
||||
|
||||
if _, err := newf.Write([]byte("]gibberish\n][")); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
newf.Close()
|
||||
|
||||
os.Rename(newf.Name(), "fixtures/_test"+ext)
|
||||
|
||||
// Give notifcations some time to arrive.
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
}
|
11
prom/retrieval/discovery/fixtures/target_groups.json
Normal file
11
prom/retrieval/discovery/fixtures/target_groups.json
Normal file
@ -0,0 +1,11 @@
|
||||
[
|
||||
{
|
||||
"targets": ["localhost:9090", "example.org:443"],
|
||||
"labels": {
|
||||
"foo": "bar"
|
||||
}
|
||||
},
|
||||
{
|
||||
"targets": ["my.domain"]
|
||||
}
|
||||
]
|
5
prom/retrieval/discovery/fixtures/target_groups.yml
Normal file
5
prom/retrieval/discovery/fixtures/target_groups.yml
Normal file
@ -0,0 +1,5 @@
|
||||
- targets: ['localhost:9090', 'example.org:443']
|
||||
labels:
|
||||
foo: bar
|
||||
|
||||
- targets: ['my.domain']
|
30
prom/retrieval/discovery/kubernetes.go
Normal file
30
prom/retrieval/discovery/kubernetes.go
Normal file
@ -0,0 +1,30 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/retrieval/discovery/kubernetes"
|
||||
)
|
||||
|
||||
func NewKubernetesDiscovery(conf *config.KubernetesSDConfig) (*kubernetes.KubernetesDiscovery, error) {
|
||||
kd := &kubernetes.KubernetesDiscovery{
|
||||
Conf: conf,
|
||||
}
|
||||
err := kd.Initialize()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return kd, nil
|
||||
}
|
577
prom/retrieval/discovery/kubernetes/discovery.go
Normal file
577
prom/retrieval/discovery/kubernetes/discovery.go
Normal 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()
|
||||
}
|
||||
}
|
||||
}
|
210
prom/retrieval/discovery/kubernetes/types.go
Normal file
210
prom/retrieval/discovery/kubernetes/types.go
Normal 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"`
|
||||
}
|
105
prom/retrieval/discovery/marathon.go
Normal file
105
prom/retrieval/discovery/marathon.go
Normal file
@ -0,0 +1,105 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/log"
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/retrieval/discovery/marathon"
|
||||
)
|
||||
|
||||
// MarathonDiscovery provides service discovery based on a Marathon instance.
|
||||
type MarathonDiscovery struct {
|
||||
servers []string
|
||||
refreshInterval time.Duration
|
||||
done chan struct{}
|
||||
lastRefresh map[string]*config.TargetGroup
|
||||
client marathon.AppListClient
|
||||
}
|
||||
|
||||
// NewMarathonDiscovery creates a new Marathon based discovery.
|
||||
func NewMarathonDiscovery(conf *config.MarathonSDConfig) *MarathonDiscovery {
|
||||
return &MarathonDiscovery{
|
||||
servers: conf.Servers,
|
||||
refreshInterval: time.Duration(conf.RefreshInterval),
|
||||
done: make(chan struct{}),
|
||||
client: marathon.FetchMarathonApps,
|
||||
}
|
||||
}
|
||||
|
||||
// Sources implements the TargetProvider interface.
|
||||
func (md *MarathonDiscovery) Sources() []string {
|
||||
var sources []string
|
||||
tgroups, err := md.fetchTargetGroups()
|
||||
if err == nil {
|
||||
for source := range tgroups {
|
||||
sources = append(sources, source)
|
||||
}
|
||||
}
|
||||
return sources
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (md *MarathonDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
case <-time.After(md.refreshInterval):
|
||||
err := md.updateServices(ch)
|
||||
if err != nil {
|
||||
log.Errorf("Error while updating services: %s", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (md *MarathonDiscovery) updateServices(ch chan<- *config.TargetGroup) error {
|
||||
targetMap, err := md.fetchTargetGroups()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update services which are still present
|
||||
for _, tg := range targetMap {
|
||||
ch <- tg
|
||||
}
|
||||
|
||||
// Remove services which did disappear
|
||||
for source := range md.lastRefresh {
|
||||
_, ok := targetMap[source]
|
||||
if !ok {
|
||||
log.Debugf("Removing group for %s", source)
|
||||
ch <- &config.TargetGroup{Source: source}
|
||||
}
|
||||
}
|
||||
|
||||
md.lastRefresh = targetMap
|
||||
return nil
|
||||
}
|
||||
|
||||
func (md *MarathonDiscovery) fetchTargetGroups() (map[string]*config.TargetGroup, error) {
|
||||
url := marathon.RandomAppsURL(md.servers)
|
||||
apps, err := md.client(url)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
groups := marathon.AppsToTargetGroups(apps)
|
||||
return groups, nil
|
||||
}
|
47
prom/retrieval/discovery/marathon/client.go
Normal file
47
prom/retrieval/discovery/marathon/client.go
Normal file
@ -0,0 +1,47 @@
|
||||
// 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 marathon
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
)
|
||||
|
||||
// AppListClient defines a function that can be used to get an application list from marathon.
|
||||
type AppListClient func(url string) (*AppList, error)
|
||||
|
||||
// FetchMarathonApps requests a list of applications from a marathon server.
|
||||
func FetchMarathonApps(url string) (*AppList, error) {
|
||||
resp, err := http.Get(url)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
body, err := ioutil.ReadAll(resp.Body)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return parseAppJSON(body)
|
||||
}
|
||||
|
||||
func parseAppJSON(body []byte) (*AppList, error) {
|
||||
apps := &AppList{}
|
||||
err := json.Unmarshal(body, apps)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return apps, nil
|
||||
}
|
32
prom/retrieval/discovery/marathon/constants.go
Normal file
32
prom/retrieval/discovery/marathon/constants.go
Normal file
@ -0,0 +1,32 @@
|
||||
// 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 marathon
|
||||
|
||||
import (
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
)
|
||||
|
||||
const (
|
||||
// metaLabelPrefix is the meta prefix used for all meta labels in this discovery.
|
||||
metaLabelPrefix = clientmodel.MetaLabelPrefix + "marathon_"
|
||||
// appLabelPrefix is the prefix for the application labels.
|
||||
appLabelPrefix = metaLabelPrefix + "app_label_"
|
||||
|
||||
// appLabel is used for the name of the app in Marathon.
|
||||
appLabel clientmodel.LabelName = metaLabelPrefix + "app"
|
||||
// imageLabel is the label that is used for the docker image running the service.
|
||||
imageLabel clientmodel.LabelName = metaLabelPrefix + "image"
|
||||
// taskLabel contains the mesos task name of the app instance.
|
||||
taskLabel clientmodel.LabelName = metaLabelPrefix + "task"
|
||||
)
|
71
prom/retrieval/discovery/marathon/conversion.go
Normal file
71
prom/retrieval/discovery/marathon/conversion.go
Normal file
@ -0,0 +1,71 @@
|
||||
// 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 marathon
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
// AppsToTargetGroups takes an array of Marathon apps and converts them into target groups.
|
||||
func AppsToTargetGroups(apps *AppList) map[string]*config.TargetGroup {
|
||||
tgroups := map[string]*config.TargetGroup{}
|
||||
for _, a := range apps.Apps {
|
||||
group := createTargetGroup(&a)
|
||||
tgroups[group.Source] = group
|
||||
}
|
||||
return tgroups
|
||||
}
|
||||
|
||||
func createTargetGroup(app *App) *config.TargetGroup {
|
||||
var (
|
||||
targets = targetsForApp(app)
|
||||
appName = clientmodel.LabelValue(app.ID)
|
||||
image = clientmodel.LabelValue(app.Container.Docker.Image)
|
||||
)
|
||||
tg := &config.TargetGroup{
|
||||
Targets: targets,
|
||||
Labels: clientmodel.LabelSet{
|
||||
appLabel: appName,
|
||||
imageLabel: image,
|
||||
},
|
||||
Source: app.ID,
|
||||
}
|
||||
|
||||
for ln, lv := range app.Labels {
|
||||
ln = appLabelPrefix + ln
|
||||
tg.Labels[clientmodel.LabelName(ln)] = clientmodel.LabelValue(lv)
|
||||
}
|
||||
|
||||
return tg
|
||||
}
|
||||
|
||||
func targetsForApp(app *App) []clientmodel.LabelSet {
|
||||
targets := make([]clientmodel.LabelSet, 0, len(app.Tasks))
|
||||
for _, t := range app.Tasks {
|
||||
target := targetForTask(&t)
|
||||
targets = append(targets, clientmodel.LabelSet{
|
||||
clientmodel.AddressLabel: clientmodel.LabelValue(target),
|
||||
taskLabel: clientmodel.LabelValue(t.ID),
|
||||
})
|
||||
}
|
||||
return targets
|
||||
}
|
||||
|
||||
func targetForTask(task *Task) string {
|
||||
return fmt.Sprintf("%s:%d", task.Host, task.Ports[0])
|
||||
}
|
45
prom/retrieval/discovery/marathon/objects.go
Normal file
45
prom/retrieval/discovery/marathon/objects.go
Normal file
@ -0,0 +1,45 @@
|
||||
// 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 marathon
|
||||
|
||||
// Task describes one instance of a service running on Marathon.
|
||||
type Task struct {
|
||||
ID string `json:"id"`
|
||||
Host string `json:"host"`
|
||||
Ports []uint32 `json:"ports"`
|
||||
}
|
||||
|
||||
// DockerContainer describes a container which uses the docker runtime.
|
||||
type DockerContainer struct {
|
||||
Image string `json:"image"`
|
||||
}
|
||||
|
||||
// Container describes the runtime an app in running in.
|
||||
type Container struct {
|
||||
Docker DockerContainer `json:"docker"`
|
||||
}
|
||||
|
||||
// App describes a service running on Marathon.
|
||||
type App struct {
|
||||
ID string `json:"id"`
|
||||
Tasks []Task `json:"tasks"`
|
||||
RunningTasks int `json:"tasksRunning"`
|
||||
Labels map[string]string `json:"labels"`
|
||||
Container Container `json:"container"`
|
||||
}
|
||||
|
||||
// AppList is a list of Marathon apps.
|
||||
type AppList struct {
|
||||
Apps []App `json:"apps"`
|
||||
}
|
29
prom/retrieval/discovery/marathon/url.go
Normal file
29
prom/retrieval/discovery/marathon/url.go
Normal file
@ -0,0 +1,29 @@
|
||||
// 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 marathon
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
)
|
||||
|
||||
const appListPath string = "/v2/apps/?embed=apps.tasks"
|
||||
|
||||
// RandomAppsURL randomly selects a server from an array and creates
|
||||
// an URL pointing to the app list.
|
||||
func RandomAppsURL(servers []string) string {
|
||||
// TODO: If possible update server list from Marathon at some point.
|
||||
server := servers[rand.Intn(len(servers))]
|
||||
return fmt.Sprintf("%s%s", server, appListPath)
|
||||
}
|
180
prom/retrieval/discovery/marathon_test.go
Normal file
180
prom/retrieval/discovery/marathon_test.go
Normal file
@ -0,0 +1,180 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/retrieval/discovery/marathon"
|
||||
)
|
||||
|
||||
var marathonValidLabel = map[string]string{"prometheus": "yes"}
|
||||
|
||||
func newTestDiscovery(client marathon.AppListClient) (chan *config.TargetGroup, *MarathonDiscovery) {
|
||||
ch := make(chan *config.TargetGroup)
|
||||
md := NewMarathonDiscovery(&config.MarathonSDConfig{
|
||||
Servers: []string{"http://localhost:8080"},
|
||||
})
|
||||
md.client = client
|
||||
return ch, md
|
||||
}
|
||||
|
||||
func TestMarathonSDHandleError(t *testing.T) {
|
||||
var errTesting = errors.New("testing failure")
|
||||
ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return nil, errTesting
|
||||
})
|
||||
go func() {
|
||||
select {
|
||||
case tg := <-ch:
|
||||
t.Fatalf("Got group: %s", tg)
|
||||
default:
|
||||
}
|
||||
}()
|
||||
err := md.updateServices(ch)
|
||||
if err != errTesting {
|
||||
t.Fatalf("Expected error: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMarathonSDEmptyList(t *testing.T) {
|
||||
ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return &marathon.AppList{}, nil
|
||||
})
|
||||
go func() {
|
||||
select {
|
||||
case tg := <-ch:
|
||||
t.Fatalf("Got group: %v", tg)
|
||||
default:
|
||||
}
|
||||
}()
|
||||
err := md.updateServices(ch)
|
||||
if err != nil {
|
||||
t.Fatalf("Got error: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func marathonTestAppList(labels map[string]string, runningTasks int) *marathon.AppList {
|
||||
task := marathon.Task{
|
||||
ID: "test-task-1",
|
||||
Host: "mesos-slave1",
|
||||
Ports: []uint32{31000},
|
||||
}
|
||||
docker := marathon.DockerContainer{Image: "repo/image:tag"}
|
||||
container := marathon.Container{Docker: docker}
|
||||
app := marathon.App{
|
||||
ID: "test-service",
|
||||
Tasks: []marathon.Task{task},
|
||||
RunningTasks: runningTasks,
|
||||
Labels: labels,
|
||||
Container: container,
|
||||
}
|
||||
return &marathon.AppList{
|
||||
Apps: []marathon.App{app},
|
||||
}
|
||||
}
|
||||
|
||||
func TestMarathonSDSendGroup(t *testing.T) {
|
||||
ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return marathonTestAppList(marathonValidLabel, 1), nil
|
||||
})
|
||||
go func() {
|
||||
select {
|
||||
case tg := <-ch:
|
||||
if tg.Source != "test-service" {
|
||||
t.Fatalf("Wrong target group name: %s", tg.Source)
|
||||
}
|
||||
if len(tg.Targets) != 1 {
|
||||
t.Fatalf("Wrong number of targets: %v", tg.Targets)
|
||||
}
|
||||
tgt := tg.Targets[0]
|
||||
if tgt[clientmodel.AddressLabel] != "mesos-slave1:31000" {
|
||||
t.Fatalf("Wrong target address: %s", tgt[clientmodel.AddressLabel])
|
||||
}
|
||||
default:
|
||||
t.Fatal("Did not get a target group.")
|
||||
}
|
||||
}()
|
||||
err := md.updateServices(ch)
|
||||
if err != nil {
|
||||
t.Fatalf("Got error: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMarathonSDRemoveApp(t *testing.T) {
|
||||
ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return marathonTestAppList(marathonValidLabel, 1), nil
|
||||
})
|
||||
go func() {
|
||||
up1 := <-ch
|
||||
up2 := <-ch
|
||||
if up2.Source != up1.Source {
|
||||
t.Fatalf("Source is different: %s", up2)
|
||||
if len(up2.Targets) > 0 {
|
||||
t.Fatalf("Got a non-empty target set: %s", up2.Targets)
|
||||
}
|
||||
}
|
||||
}()
|
||||
err := md.updateServices(ch)
|
||||
if err != nil {
|
||||
t.Fatalf("Got error on first update: %s", err)
|
||||
}
|
||||
|
||||
md.client = func(url string) (*marathon.AppList, error) {
|
||||
return marathonTestAppList(marathonValidLabel, 0), nil
|
||||
}
|
||||
err = md.updateServices(ch)
|
||||
if err != nil {
|
||||
t.Fatalf("Got error on second update: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMarathonSDSources(t *testing.T) {
|
||||
_, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return marathonTestAppList(marathonValidLabel, 1), nil
|
||||
})
|
||||
sources := md.Sources()
|
||||
if len(sources) != 1 {
|
||||
t.Fatalf("Wrong number of sources: %s", sources)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMarathonSDRunAndStop(t *testing.T) {
|
||||
ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) {
|
||||
return marathonTestAppList(marathonValidLabel, 1), nil
|
||||
})
|
||||
md.refreshInterval = time.Millisecond * 10
|
||||
done := make(chan struct{})
|
||||
|
||||
go func() {
|
||||
select {
|
||||
case <-ch:
|
||||
close(done)
|
||||
case <-time.After(md.refreshInterval * 3):
|
||||
close(done)
|
||||
t.Fatalf("Update took too long.")
|
||||
}
|
||||
}()
|
||||
md.Run(ch, done)
|
||||
select {
|
||||
case <-ch:
|
||||
default:
|
||||
t.Fatalf("Channel not closed.")
|
||||
}
|
||||
}
|
362
prom/retrieval/discovery/serverset.go
Normal file
362
prom/retrieval/discovery/serverset.go
Normal file
@ -0,0 +1,362 @@
|
||||
// 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 discovery
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/log"
|
||||
"github.com/samuel/go-zookeeper/zk"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/util/strutil"
|
||||
)
|
||||
|
||||
const (
|
||||
serversetNodePrefix = "member_"
|
||||
|
||||
serversetLabelPrefix = clientmodel.MetaLabelPrefix + "serverset_"
|
||||
serversetStatusLabel = serversetLabelPrefix + "status"
|
||||
serversetPathLabel = serversetLabelPrefix + "path"
|
||||
serversetEndpointLabelPrefix = serversetLabelPrefix + "endpoint"
|
||||
)
|
||||
|
||||
type serversetMember struct {
|
||||
ServiceEndpoint serversetEndpoint
|
||||
AdditionalEndpoints map[string]serversetEndpoint
|
||||
Status string `json:"status"`
|
||||
}
|
||||
|
||||
type serversetEndpoint struct {
|
||||
Host string
|
||||
Port int
|
||||
}
|
||||
|
||||
type ZookeeperLogger struct {
|
||||
}
|
||||
|
||||
// Implements zk.Logger
|
||||
func (zl ZookeeperLogger) Printf(s string, i ...interface{}) {
|
||||
log.Infof(s, i...)
|
||||
}
|
||||
|
||||
// ServersetDiscovery retrieves target information from a Serverset server
|
||||
// and updates them via watches.
|
||||
type ServersetDiscovery struct {
|
||||
conf *config.ServersetSDConfig
|
||||
conn *zk.Conn
|
||||
mu sync.RWMutex
|
||||
sources map[string]*config.TargetGroup
|
||||
sdUpdates *chan<- *config.TargetGroup
|
||||
updates chan zookeeperTreeCacheEvent
|
||||
treeCache *zookeeperTreeCache
|
||||
}
|
||||
|
||||
// NewServersetDiscovery returns a new ServersetDiscovery for the given config.
|
||||
func NewServersetDiscovery(conf *config.ServersetSDConfig) *ServersetDiscovery {
|
||||
conn, _, err := zk.Connect(conf.Servers, time.Duration(conf.Timeout))
|
||||
conn.SetLogger(ZookeeperLogger{})
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
updates := make(chan zookeeperTreeCacheEvent)
|
||||
sd := &ServersetDiscovery{
|
||||
conf: conf,
|
||||
conn: conn,
|
||||
updates: updates,
|
||||
sources: map[string]*config.TargetGroup{},
|
||||
}
|
||||
go sd.processUpdates()
|
||||
sd.treeCache = NewZookeeperTreeCache(conn, conf.Paths[0], updates)
|
||||
return sd
|
||||
}
|
||||
|
||||
// Sources implements the TargetProvider interface.
|
||||
func (sd *ServersetDiscovery) Sources() []string {
|
||||
sd.mu.RLock()
|
||||
defer sd.mu.RUnlock()
|
||||
srcs := []string{}
|
||||
for t := range sd.sources {
|
||||
srcs = append(srcs, t)
|
||||
}
|
||||
return srcs
|
||||
}
|
||||
|
||||
func (sd *ServersetDiscovery) processUpdates() {
|
||||
defer sd.conn.Close()
|
||||
for event := range sd.updates {
|
||||
tg := &config.TargetGroup{
|
||||
Source: event.Path,
|
||||
}
|
||||
sd.mu.Lock()
|
||||
if event.Data != nil {
|
||||
labelSet, err := parseServersetMember(*event.Data, event.Path)
|
||||
if err == nil {
|
||||
tg.Targets = []clientmodel.LabelSet{*labelSet}
|
||||
sd.sources[event.Path] = tg
|
||||
} else {
|
||||
delete(sd.sources, event.Path)
|
||||
}
|
||||
} else {
|
||||
delete(sd.sources, event.Path)
|
||||
}
|
||||
sd.mu.Unlock()
|
||||
if sd.sdUpdates != nil {
|
||||
*sd.sdUpdates <- tg
|
||||
}
|
||||
}
|
||||
|
||||
if sd.sdUpdates != nil {
|
||||
close(*sd.sdUpdates)
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (sd *ServersetDiscovery) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
// Send on everything we have seen so far.
|
||||
sd.mu.Lock()
|
||||
for _, targetGroup := range sd.sources {
|
||||
ch <- targetGroup
|
||||
}
|
||||
// Tell processUpdates to send future updates.
|
||||
sd.sdUpdates = &ch
|
||||
sd.mu.Unlock()
|
||||
|
||||
<-done
|
||||
sd.treeCache.Stop()
|
||||
}
|
||||
|
||||
func parseServersetMember(data []byte, path string) (*clientmodel.LabelSet, error) {
|
||||
member := serversetMember{}
|
||||
err := json.Unmarshal(data, &member)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error unmarshaling serverset member %q: %s", path, err)
|
||||
}
|
||||
|
||||
labels := clientmodel.LabelSet{}
|
||||
labels[serversetPathLabel] = clientmodel.LabelValue(path)
|
||||
labels[clientmodel.AddressLabel] = clientmodel.LabelValue(
|
||||
fmt.Sprintf("%s:%d", member.ServiceEndpoint.Host, member.ServiceEndpoint.Port))
|
||||
|
||||
labels[serversetEndpointLabelPrefix+"_host"] = clientmodel.LabelValue(member.ServiceEndpoint.Host)
|
||||
labels[serversetEndpointLabelPrefix+"_port"] = clientmodel.LabelValue(fmt.Sprintf("%d", member.ServiceEndpoint.Port))
|
||||
|
||||
for name, endpoint := range member.AdditionalEndpoints {
|
||||
cleanName := clientmodel.LabelName(strutil.SanitizeLabelName(name))
|
||||
labels[serversetEndpointLabelPrefix+"_host_"+cleanName] = clientmodel.LabelValue(
|
||||
endpoint.Host)
|
||||
labels[serversetEndpointLabelPrefix+"_port_"+cleanName] = clientmodel.LabelValue(
|
||||
fmt.Sprintf("%d", endpoint.Port))
|
||||
|
||||
}
|
||||
|
||||
labels[serversetStatusLabel] = clientmodel.LabelValue(member.Status)
|
||||
|
||||
return &labels, nil
|
||||
}
|
||||
|
||||
type zookeeperTreeCache struct {
|
||||
conn *zk.Conn
|
||||
prefix string
|
||||
events chan zookeeperTreeCacheEvent
|
||||
zkEvents chan zk.Event
|
||||
stop chan struct{}
|
||||
head *zookeeperTreeCacheNode
|
||||
}
|
||||
|
||||
type zookeeperTreeCacheEvent struct {
|
||||
Path string
|
||||
Data *[]byte
|
||||
}
|
||||
|
||||
type zookeeperTreeCacheNode struct {
|
||||
data *[]byte
|
||||
events chan zk.Event
|
||||
done chan struct{}
|
||||
stopped bool
|
||||
children map[string]*zookeeperTreeCacheNode
|
||||
}
|
||||
|
||||
func NewZookeeperTreeCache(conn *zk.Conn, path string, events chan zookeeperTreeCacheEvent) *zookeeperTreeCache {
|
||||
tc := &zookeeperTreeCache{
|
||||
conn: conn,
|
||||
prefix: path,
|
||||
events: events,
|
||||
stop: make(chan struct{}),
|
||||
}
|
||||
tc.head = &zookeeperTreeCacheNode{
|
||||
events: make(chan zk.Event),
|
||||
children: map[string]*zookeeperTreeCacheNode{},
|
||||
stopped: true,
|
||||
}
|
||||
err := tc.recursiveNodeUpdate(path, tc.head)
|
||||
if err != nil {
|
||||
log.Errorf("Error during initial read of Zookeeper: %s", err)
|
||||
}
|
||||
go tc.loop(err != nil)
|
||||
return tc
|
||||
}
|
||||
|
||||
func (tc *zookeeperTreeCache) Stop() {
|
||||
tc.stop <- struct{}{}
|
||||
}
|
||||
|
||||
func (tc *zookeeperTreeCache) loop(failureMode bool) {
|
||||
retryChan := make(chan struct{})
|
||||
|
||||
failure := func() {
|
||||
failureMode = true
|
||||
time.AfterFunc(time.Second*10, func() {
|
||||
retryChan <- struct{}{}
|
||||
})
|
||||
}
|
||||
if failureMode {
|
||||
failure()
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case ev := <-tc.head.events:
|
||||
log.Debugf("Received Zookeeper event: %s", ev)
|
||||
if failureMode {
|
||||
continue
|
||||
}
|
||||
if ev.Type == zk.EventNotWatching {
|
||||
log.Infof("Lost connection to Zookeeper.")
|
||||
failure()
|
||||
} else {
|
||||
path := strings.TrimPrefix(ev.Path, tc.prefix)
|
||||
parts := strings.Split(path, "/")
|
||||
node := tc.head
|
||||
for _, part := range parts[1:] {
|
||||
childNode := node.children[part]
|
||||
if childNode == nil {
|
||||
childNode = &zookeeperTreeCacheNode{
|
||||
events: tc.head.events,
|
||||
children: map[string]*zookeeperTreeCacheNode{},
|
||||
done: make(chan struct{}, 1),
|
||||
}
|
||||
node.children[part] = childNode
|
||||
}
|
||||
node = childNode
|
||||
}
|
||||
err := tc.recursiveNodeUpdate(ev.Path, node)
|
||||
if err != nil {
|
||||
log.Errorf("Error during processing of Zookeeper event: %s", err)
|
||||
failure()
|
||||
} else if tc.head.data == nil {
|
||||
log.Errorf("Error during processing of Zookeeper event: path %s no longer exists", tc.prefix)
|
||||
failure()
|
||||
}
|
||||
}
|
||||
case <-retryChan:
|
||||
log.Infof("Attempting to resync state with Zookeeper")
|
||||
err := tc.recursiveNodeUpdate(tc.prefix, tc.head)
|
||||
if err != nil {
|
||||
log.Errorf("Error during Zookeeper resync: %s", err)
|
||||
failure()
|
||||
} else {
|
||||
log.Infof("Zookeeper resync successful")
|
||||
failureMode = false
|
||||
}
|
||||
case <-tc.stop:
|
||||
close(tc.events)
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (tc *zookeeperTreeCache) recursiveNodeUpdate(path string, node *zookeeperTreeCacheNode) error {
|
||||
data, _, dataWatcher, err := tc.conn.GetW(path)
|
||||
if err == zk.ErrNoNode {
|
||||
tc.recursiveDelete(path, node)
|
||||
if node == tc.head {
|
||||
return fmt.Errorf("path %s does not exist", path)
|
||||
}
|
||||
return nil
|
||||
} else if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if node.data == nil || !bytes.Equal(*node.data, data) {
|
||||
node.data = &data
|
||||
tc.events <- zookeeperTreeCacheEvent{Path: path, Data: node.data}
|
||||
}
|
||||
|
||||
children, _, childWatcher, err := tc.conn.ChildrenW(path)
|
||||
if err == zk.ErrNoNode {
|
||||
tc.recursiveDelete(path, node)
|
||||
return nil
|
||||
} else if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
currentChildren := map[string]struct{}{}
|
||||
for _, child := range children {
|
||||
currentChildren[child] = struct{}{}
|
||||
childNode := node.children[child]
|
||||
// Does not already exists, create it.
|
||||
if childNode == nil {
|
||||
node.children[child] = &zookeeperTreeCacheNode{
|
||||
events: node.events,
|
||||
children: map[string]*zookeeperTreeCacheNode{},
|
||||
done: make(chan struct{}, 1),
|
||||
}
|
||||
}
|
||||
err = tc.recursiveNodeUpdate(path+"/"+child, node.children[child])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// Remove nodes that no longer exist
|
||||
for name, childNode := range node.children {
|
||||
if _, ok := currentChildren[name]; !ok || node.data == nil {
|
||||
tc.recursiveDelete(path+"/"+name, childNode)
|
||||
delete(node.children, name)
|
||||
}
|
||||
}
|
||||
|
||||
go func() {
|
||||
// Pass up zookeeper events, until the node is deleted.
|
||||
select {
|
||||
case event := <-dataWatcher:
|
||||
node.events <- event
|
||||
case event := <-childWatcher:
|
||||
node.events <- event
|
||||
case <-node.done:
|
||||
}
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (tc *zookeeperTreeCache) recursiveDelete(path string, node *zookeeperTreeCacheNode) {
|
||||
if !node.stopped {
|
||||
node.done <- struct{}{}
|
||||
node.stopped = true
|
||||
}
|
||||
if node.data != nil {
|
||||
tc.events <- zookeeperTreeCacheEvent{Path: path, Data: nil}
|
||||
node.data = nil
|
||||
}
|
||||
for name, childNode := range node.children {
|
||||
tc.recursiveDelete(path+"/"+name, childNode)
|
||||
}
|
||||
}
|
70
prom/retrieval/helpers_test.go
Normal file
70
prom/retrieval/helpers_test.go
Normal file
@ -0,0 +1,70 @@
|
||||
// Copyright 2013 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 retrieval
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
type nopAppender struct{}
|
||||
|
||||
func (a nopAppender) Append(*clientmodel.Sample) {
|
||||
}
|
||||
|
||||
type slowAppender struct{}
|
||||
|
||||
func (a slowAppender) Append(*clientmodel.Sample) {
|
||||
time.Sleep(time.Millisecond)
|
||||
return
|
||||
}
|
||||
|
||||
type collectResultAppender struct {
|
||||
result clientmodel.Samples
|
||||
}
|
||||
|
||||
func (a *collectResultAppender) Append(s *clientmodel.Sample) {
|
||||
for ln, lv := range s.Metric {
|
||||
if len(lv) == 0 {
|
||||
delete(s.Metric, ln)
|
||||
}
|
||||
}
|
||||
a.result = append(a.result, s)
|
||||
}
|
||||
|
||||
// fakeTargetProvider implements a TargetProvider and allows manual injection
|
||||
// of TargetGroups through the update channel.
|
||||
type fakeTargetProvider struct {
|
||||
sources []string
|
||||
update chan *config.TargetGroup
|
||||
}
|
||||
|
||||
func (tp *fakeTargetProvider) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
for {
|
||||
select {
|
||||
case tg := <-tp.update:
|
||||
ch <- tg
|
||||
case <-done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (tp *fakeTargetProvider) Sources() []string {
|
||||
return tp.sources
|
||||
}
|
93
prom/retrieval/relabel.go
Normal file
93
prom/retrieval/relabel.go
Normal file
@ -0,0 +1,93 @@
|
||||
package retrieval
|
||||
|
||||
import (
|
||||
"crypto/md5"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
// Relabel returns a relabeled copy of the given label set. The relabel configurations
|
||||
// are applied in order of input.
|
||||
// If a label set is dropped, nil is returned.
|
||||
func Relabel(labels clientmodel.LabelSet, cfgs ...*config.RelabelConfig) (clientmodel.LabelSet, error) {
|
||||
out := clientmodel.LabelSet{}
|
||||
for ln, lv := range labels {
|
||||
out[ln] = lv
|
||||
}
|
||||
var err error
|
||||
for _, cfg := range cfgs {
|
||||
if out, err = relabel(out, cfg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if out == nil {
|
||||
return nil, nil
|
||||
}
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func relabel(labels clientmodel.LabelSet, cfg *config.RelabelConfig) (clientmodel.LabelSet, error) {
|
||||
values := make([]string, 0, len(cfg.SourceLabels))
|
||||
for _, ln := range cfg.SourceLabels {
|
||||
values = append(values, string(labels[ln]))
|
||||
}
|
||||
val := strings.Join(values, cfg.Separator)
|
||||
|
||||
switch cfg.Action {
|
||||
case config.RelabelDrop:
|
||||
if cfg.Regex.MatchString(val) {
|
||||
return nil, nil
|
||||
}
|
||||
case config.RelabelKeep:
|
||||
if !cfg.Regex.MatchString(val) {
|
||||
return nil, nil
|
||||
}
|
||||
case config.RelabelReplace:
|
||||
indexes := cfg.Regex.FindStringSubmatchIndex(val)
|
||||
// If there is no match no replacement must take place.
|
||||
if indexes == nil {
|
||||
break
|
||||
}
|
||||
res := cfg.Regex.ExpandString([]byte{}, cfg.Replacement, val, indexes)
|
||||
if len(res) == 0 {
|
||||
delete(labels, cfg.TargetLabel)
|
||||
} else {
|
||||
labels[cfg.TargetLabel] = clientmodel.LabelValue(res)
|
||||
}
|
||||
case config.RelabelHashMod:
|
||||
mod := sum64(md5.Sum([]byte(val))) % cfg.Modulus
|
||||
labels[cfg.TargetLabel] = clientmodel.LabelValue(fmt.Sprintf("%d", mod))
|
||||
case config.RelabelLabelMap:
|
||||
out := make(clientmodel.LabelSet, len(labels))
|
||||
// Take a copy to avoid infinite loops.
|
||||
for ln, lv := range labels {
|
||||
out[ln] = lv
|
||||
}
|
||||
for ln, lv := range labels {
|
||||
if cfg.Regex.MatchString(string(ln)) {
|
||||
res := cfg.Regex.ReplaceAllString(string(ln), cfg.Replacement)
|
||||
out[clientmodel.LabelName(res)] = lv
|
||||
}
|
||||
}
|
||||
labels = out
|
||||
default:
|
||||
panic(fmt.Errorf("retrieval.relabel: unknown relabel action type %q", cfg.Action))
|
||||
}
|
||||
return labels, nil
|
||||
}
|
||||
|
||||
// sum64 sums the md5 hash to an uint64.
|
||||
func sum64(hash [md5.Size]byte) uint64 {
|
||||
var s uint64
|
||||
|
||||
for i, b := range hash {
|
||||
shift := uint64((md5.Size - i - 1) * 8)
|
||||
|
||||
s |= uint64(b) << shift
|
||||
}
|
||||
return s
|
||||
}
|
251
prom/retrieval/relabel_test.go
Normal file
251
prom/retrieval/relabel_test.go
Normal file
@ -0,0 +1,251 @@
|
||||
package retrieval
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"regexp"
|
||||
"testing"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
func TestRelabel(t *testing.T) {
|
||||
tests := []struct {
|
||||
input clientmodel.LabelSet
|
||||
relabel []*config.RelabelConfig
|
||||
output clientmodel.LabelSet
|
||||
}{
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("f(.*)")},
|
||||
TargetLabel: clientmodel.LabelName("d"),
|
||||
Separator: ";",
|
||||
Replacement: "ch${1}-ch${1}",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
"d": "choo-choo",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a", "b"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^f(.*);(.*)r$")},
|
||||
TargetLabel: clientmodel.LabelName("a"),
|
||||
Separator: ";",
|
||||
Replacement: "b${1}${2}m", // boobam
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"c", "a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("(b).*b(.*)ba(.*)")},
|
||||
TargetLabel: clientmodel.LabelName("d"),
|
||||
Separator: ";",
|
||||
Replacement: "$1$2$2$3",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "boobam",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
"d": "boooom",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("o$")},
|
||||
Action: config.RelabelDrop,
|
||||
}, {
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("f(.*)")},
|
||||
TargetLabel: clientmodel.LabelName("d"),
|
||||
Separator: ";",
|
||||
Replacement: "ch$1-ch$1",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
output: nil,
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "abc",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("(b)")},
|
||||
TargetLabel: clientmodel.LabelName("d"),
|
||||
Separator: ";",
|
||||
Replacement: "$1",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "abc",
|
||||
"d": "b",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("no-match")},
|
||||
Action: config.RelabelDrop,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("no-match")},
|
||||
Action: config.RelabelKeep,
|
||||
},
|
||||
},
|
||||
output: nil,
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^f")},
|
||||
Action: config.RelabelKeep,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
},
|
||||
},
|
||||
{
|
||||
// No replacement must be applied if there is no match.
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "boo",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"a"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^f")},
|
||||
TargetLabel: clientmodel.LabelName("b"),
|
||||
Replacement: "bar",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "boo",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"c"},
|
||||
TargetLabel: clientmodel.LabelName("d"),
|
||||
Separator: ";",
|
||||
Action: config.RelabelHashMod,
|
||||
Modulus: 1000,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b": "bar",
|
||||
"c": "baz",
|
||||
"d": "976",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b1": "bar",
|
||||
"b2": "baz",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^(b.*)")},
|
||||
Replacement: "bar_${1}",
|
||||
Action: config.RelabelLabelMap,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"b1": "bar",
|
||||
"b2": "baz",
|
||||
"bar_b1": "bar",
|
||||
"bar_b2": "baz",
|
||||
},
|
||||
},
|
||||
{
|
||||
input: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"__meta_my_bar": "aaa",
|
||||
"__meta_my_baz": "bbb",
|
||||
"__meta_other": "ccc",
|
||||
},
|
||||
relabel: []*config.RelabelConfig{
|
||||
{
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^__meta_(my.*)")},
|
||||
Replacement: "${1}",
|
||||
Action: config.RelabelLabelMap,
|
||||
},
|
||||
},
|
||||
output: clientmodel.LabelSet{
|
||||
"a": "foo",
|
||||
"__meta_my_bar": "aaa",
|
||||
"__meta_my_baz": "bbb",
|
||||
"__meta_other": "ccc",
|
||||
"my_bar": "aaa",
|
||||
"my_baz": "bbb",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for i, test := range tests {
|
||||
res, err := Relabel(test.input, test.relabel...)
|
||||
if err != nil {
|
||||
t.Errorf("Test %d: error relabeling: %s", i+1, err)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(res, test.output) {
|
||||
t.Errorf("Test %d: relabel output mismatch: expected %#v, got %#v", i+1, test.output, res)
|
||||
}
|
||||
}
|
||||
}
|
577
prom/retrieval/target.go
Normal file
577
prom/retrieval/target.go
Normal file
@ -0,0 +1,577 @@
|
||||
// Copyright 2013 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 retrieval
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"math/rand"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/client_golang/extraction"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/log"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/util/httputil"
|
||||
)
|
||||
|
||||
const (
|
||||
// ScrapeHealthMetricName is the metric name for the synthetic health
|
||||
// variable.
|
||||
scrapeHealthMetricName clientmodel.LabelValue = "up"
|
||||
// ScrapeTimeMetricName is the metric name for the synthetic scrape duration
|
||||
// variable.
|
||||
scrapeDurationMetricName clientmodel.LabelValue = "scrape_duration_seconds"
|
||||
// Capacity of the channel to buffer samples during ingestion.
|
||||
ingestedSamplesCap = 256
|
||||
|
||||
// Constants for instrumentation.
|
||||
namespace = "prometheus"
|
||||
interval = "interval"
|
||||
)
|
||||
|
||||
var (
|
||||
errIngestChannelFull = errors.New("ingestion channel full")
|
||||
|
||||
targetIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Namespace: namespace,
|
||||
Name: "target_interval_length_seconds",
|
||||
Help: "Actual intervals between scrapes.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{interval},
|
||||
)
|
||||
)
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(targetIntervalLength)
|
||||
}
|
||||
|
||||
// TargetHealth describes the health state of a target.
|
||||
type TargetHealth int
|
||||
|
||||
func (t TargetHealth) String() string {
|
||||
switch t {
|
||||
case HealthUnknown:
|
||||
return "unknown"
|
||||
case HealthGood:
|
||||
return "healthy"
|
||||
case HealthBad:
|
||||
return "unhealthy"
|
||||
}
|
||||
panic("unknown state")
|
||||
}
|
||||
|
||||
const (
|
||||
// Unknown is the state of a Target before it is first scraped.
|
||||
HealthUnknown TargetHealth = iota
|
||||
// Healthy is the state of a Target that has been successfully scraped.
|
||||
HealthGood
|
||||
// Unhealthy is the state of a Target that was scraped unsuccessfully.
|
||||
HealthBad
|
||||
)
|
||||
|
||||
// TargetStatus contains information about the current status of a scrape target.
|
||||
type TargetStatus struct {
|
||||
lastError error
|
||||
lastScrape time.Time
|
||||
health TargetHealth
|
||||
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
// LastError returns the error encountered during the last scrape.
|
||||
func (ts *TargetStatus) LastError() error {
|
||||
ts.mu.RLock()
|
||||
defer ts.mu.RUnlock()
|
||||
return ts.lastError
|
||||
}
|
||||
|
||||
// LastScrape returns the time of the last scrape.
|
||||
func (ts *TargetStatus) LastScrape() time.Time {
|
||||
ts.mu.RLock()
|
||||
defer ts.mu.RUnlock()
|
||||
return ts.lastScrape
|
||||
}
|
||||
|
||||
// Health returns the last known health state of the target.
|
||||
func (ts *TargetStatus) Health() TargetHealth {
|
||||
ts.mu.RLock()
|
||||
defer ts.mu.RUnlock()
|
||||
return ts.health
|
||||
}
|
||||
|
||||
func (ts *TargetStatus) setLastScrape(t time.Time) {
|
||||
ts.mu.Lock()
|
||||
defer ts.mu.Unlock()
|
||||
ts.lastScrape = t
|
||||
}
|
||||
|
||||
func (ts *TargetStatus) setLastError(err error) {
|
||||
ts.mu.Lock()
|
||||
defer ts.mu.Unlock()
|
||||
if err == nil {
|
||||
ts.health = HealthGood
|
||||
} else {
|
||||
ts.health = HealthBad
|
||||
}
|
||||
ts.lastError = err
|
||||
}
|
||||
|
||||
// Target refers to a singular HTTP or HTTPS endpoint.
|
||||
type Target struct {
|
||||
// The status object for the target. It is only set once on initialization.
|
||||
status *TargetStatus
|
||||
// Closing scraperStopping signals that scraping should stop.
|
||||
scraperStopping chan struct{}
|
||||
// Closing scraperStopped signals that scraping has been stopped.
|
||||
scraperStopped chan struct{}
|
||||
// Channel to buffer ingested samples.
|
||||
ingestedSamples chan clientmodel.Samples
|
||||
|
||||
// Mutex protects the members below.
|
||||
sync.RWMutex
|
||||
// The HTTP client used to scrape the target's endpoint.
|
||||
httpClient *http.Client
|
||||
// url is the URL to be scraped. Its host is immutable.
|
||||
url *url.URL
|
||||
// Labels before any processing.
|
||||
metaLabels clientmodel.LabelSet
|
||||
// Any base labels that are added to this target and its metrics.
|
||||
baseLabels clientmodel.LabelSet
|
||||
// What is the deadline for the HTTP or HTTPS against this endpoint.
|
||||
deadline time.Duration
|
||||
// The time between two scrapes.
|
||||
scrapeInterval time.Duration
|
||||
// Whether the target's labels have precedence over the base labels
|
||||
// assigned by the scraping instance.
|
||||
honorLabels bool
|
||||
// Metric relabel configuration.
|
||||
metricRelabelConfigs []*config.RelabelConfig
|
||||
}
|
||||
|
||||
// NewTarget creates a reasonably configured target for querying.
|
||||
func NewTarget(cfg *config.ScrapeConfig, baseLabels, metaLabels clientmodel.LabelSet) *Target {
|
||||
t := &Target{
|
||||
url: &url.URL{
|
||||
Scheme: string(baseLabels[clientmodel.SchemeLabel]),
|
||||
Host: string(baseLabels[clientmodel.AddressLabel]),
|
||||
},
|
||||
status: &TargetStatus{},
|
||||
scraperStopping: make(chan struct{}),
|
||||
scraperStopped: make(chan struct{}),
|
||||
}
|
||||
t.Update(cfg, baseLabels, metaLabels)
|
||||
return t
|
||||
}
|
||||
|
||||
// Status returns the status of the target.
|
||||
func (t *Target) Status() *TargetStatus {
|
||||
return t.status
|
||||
}
|
||||
|
||||
// Update overwrites settings in the target that are derived from the job config
|
||||
// it belongs to.
|
||||
func (t *Target) Update(cfg *config.ScrapeConfig, baseLabels, metaLabels clientmodel.LabelSet) {
|
||||
t.Lock()
|
||||
defer t.Unlock()
|
||||
|
||||
httpClient, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
log.Errorf("cannot create HTTP client: %v", err)
|
||||
return
|
||||
}
|
||||
t.httpClient = httpClient
|
||||
|
||||
t.url.Scheme = string(baseLabels[clientmodel.SchemeLabel])
|
||||
t.url.Path = string(baseLabels[clientmodel.MetricsPathLabel])
|
||||
params := url.Values{}
|
||||
for k, v := range cfg.Params {
|
||||
params[k] = make([]string, len(v))
|
||||
copy(params[k], v)
|
||||
}
|
||||
for k, v := range baseLabels {
|
||||
if strings.HasPrefix(string(k), clientmodel.ParamLabelPrefix) {
|
||||
if len(params[string(k[len(clientmodel.ParamLabelPrefix):])]) > 0 {
|
||||
params[string(k[len(clientmodel.ParamLabelPrefix):])][0] = string(v)
|
||||
} else {
|
||||
params[string(k[len(clientmodel.ParamLabelPrefix):])] = []string{string(v)}
|
||||
}
|
||||
}
|
||||
}
|
||||
t.url.RawQuery = params.Encode()
|
||||
|
||||
t.scrapeInterval = time.Duration(cfg.ScrapeInterval)
|
||||
t.deadline = time.Duration(cfg.ScrapeTimeout)
|
||||
|
||||
t.honorLabels = cfg.HonorLabels
|
||||
t.metaLabels = metaLabels
|
||||
t.baseLabels = clientmodel.LabelSet{}
|
||||
// All remaining internal labels will not be part of the label set.
|
||||
for name, val := range baseLabels {
|
||||
if !strings.HasPrefix(string(name), clientmodel.ReservedLabelPrefix) {
|
||||
t.baseLabels[name] = val
|
||||
}
|
||||
}
|
||||
if _, ok := t.baseLabels[clientmodel.InstanceLabel]; !ok {
|
||||
t.baseLabels[clientmodel.InstanceLabel] = clientmodel.LabelValue(t.InstanceIdentifier())
|
||||
}
|
||||
t.metricRelabelConfigs = cfg.MetricRelabelConfigs
|
||||
}
|
||||
|
||||
func newHTTPClient(cfg *config.ScrapeConfig) (*http.Client, error) {
|
||||
tlsConfig := &tls.Config{}
|
||||
|
||||
// If a CA cert is provided then let's read it in so we can validate the
|
||||
// scrape target's certificate properly.
|
||||
if len(cfg.CACert) > 0 {
|
||||
caCertPool := x509.NewCertPool()
|
||||
// Load CA cert.
|
||||
caCert, err := ioutil.ReadFile(cfg.CACert)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Unable to use specified CA cert %s: %s", cfg.CACert, err)
|
||||
}
|
||||
caCertPool.AppendCertsFromPEM(caCert)
|
||||
tlsConfig.RootCAs = caCertPool
|
||||
}
|
||||
|
||||
// If a client cert & key is provided then configure TLS config accordingly.
|
||||
if cfg.ClientCert != nil && len(cfg.ClientCert.Cert) > 0 && len(cfg.ClientCert.Key) > 0 {
|
||||
cert, err := tls.LoadX509KeyPair(cfg.ClientCert.Cert, cfg.ClientCert.Key)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Unable to use specified client cert (%s) & key (%s): %s", cfg.ClientCert.Cert, cfg.ClientCert.Key, err)
|
||||
}
|
||||
tlsConfig.Certificates = []tls.Certificate{cert}
|
||||
}
|
||||
tlsConfig.BuildNameToCertificate()
|
||||
|
||||
// Get a default roundtripper with the scrape timeout.
|
||||
rt := httputil.NewDeadlineRoundTripper(time.Duration(cfg.ScrapeTimeout), cfg.ProxyURL.URL)
|
||||
tr := rt.(*http.Transport)
|
||||
// Set the TLS config from above
|
||||
tr.TLSClientConfig = tlsConfig
|
||||
rt = tr
|
||||
|
||||
// If a bearer token is provided, create a round tripper that will set the
|
||||
// Authorization header correctly on each request.
|
||||
bearerToken := cfg.BearerToken
|
||||
if len(bearerToken) == 0 && len(cfg.BearerTokenFile) > 0 {
|
||||
if b, err := ioutil.ReadFile(cfg.BearerTokenFile); err != nil {
|
||||
return nil, fmt.Errorf("Unable to read bearer token file %s: %s", cfg.BearerTokenFile, err)
|
||||
} else {
|
||||
bearerToken = string(b)
|
||||
}
|
||||
}
|
||||
if len(bearerToken) > 0 {
|
||||
rt = httputil.NewBearerAuthRoundTripper(bearerToken, rt)
|
||||
}
|
||||
|
||||
if cfg.BasicAuth != nil {
|
||||
rt = httputil.NewBasicAuthRoundTripper(cfg.BasicAuth.Username, cfg.BasicAuth.Password, rt)
|
||||
}
|
||||
|
||||
// Return a new client with the configured round tripper.
|
||||
return httputil.NewClient(rt), nil
|
||||
}
|
||||
|
||||
func (t *Target) String() string {
|
||||
return t.url.Host
|
||||
}
|
||||
|
||||
// Ingest implements an extraction.Ingester.
|
||||
func (t *Target) Ingest(s clientmodel.Samples) error {
|
||||
t.RLock()
|
||||
deadline := t.deadline
|
||||
t.RUnlock()
|
||||
// Since the regular case is that ingestedSamples is ready to receive,
|
||||
// first try without setting a timeout so that we don't need to allocate
|
||||
// a timer most of the time.
|
||||
select {
|
||||
case t.ingestedSamples <- s:
|
||||
return nil
|
||||
default:
|
||||
select {
|
||||
case t.ingestedSamples <- s:
|
||||
return nil
|
||||
case <-time.After(deadline / 10):
|
||||
return errIngestChannelFull
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure that Target implements extraction.Ingester at compile time.
|
||||
var _ extraction.Ingester = (*Target)(nil)
|
||||
|
||||
// RunScraper implements Target.
|
||||
func (t *Target) RunScraper(sampleAppender storage.SampleAppender) {
|
||||
defer close(t.scraperStopped)
|
||||
|
||||
t.RLock()
|
||||
lastScrapeInterval := t.scrapeInterval
|
||||
t.RUnlock()
|
||||
|
||||
log.Debugf("Starting scraper for target %v...", t)
|
||||
|
||||
jitterTimer := time.NewTimer(time.Duration(float64(lastScrapeInterval) * rand.Float64()))
|
||||
select {
|
||||
case <-jitterTimer.C:
|
||||
case <-t.scraperStopping:
|
||||
jitterTimer.Stop()
|
||||
return
|
||||
}
|
||||
jitterTimer.Stop()
|
||||
|
||||
ticker := time.NewTicker(lastScrapeInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
t.status.setLastScrape(time.Now())
|
||||
t.scrape(sampleAppender)
|
||||
|
||||
// Explanation of the contraption below:
|
||||
//
|
||||
// In case t.scraperStopping has something to receive, we want to read
|
||||
// from that channel rather than starting a new scrape (which might take very
|
||||
// long). That's why the outer select has no ticker.C. Should t.scraperStopping
|
||||
// not have anything to receive, we go into the inner select, where ticker.C
|
||||
// is in the mix.
|
||||
for {
|
||||
select {
|
||||
case <-t.scraperStopping:
|
||||
return
|
||||
default:
|
||||
select {
|
||||
case <-t.scraperStopping:
|
||||
return
|
||||
case <-ticker.C:
|
||||
took := time.Since(t.status.LastScrape())
|
||||
t.status.setLastScrape(time.Now())
|
||||
|
||||
intervalStr := lastScrapeInterval.String()
|
||||
|
||||
t.RLock()
|
||||
// On changed scrape interval the new interval becomes effective
|
||||
// after the next scrape.
|
||||
if lastScrapeInterval != t.scrapeInterval {
|
||||
ticker.Stop()
|
||||
ticker = time.NewTicker(t.scrapeInterval)
|
||||
lastScrapeInterval = t.scrapeInterval
|
||||
}
|
||||
t.RUnlock()
|
||||
|
||||
targetIntervalLength.WithLabelValues(intervalStr).Observe(
|
||||
float64(took) / float64(time.Second), // Sub-second precision.
|
||||
)
|
||||
t.scrape(sampleAppender)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// StopScraper implements Target.
|
||||
func (t *Target) StopScraper() {
|
||||
log.Debugf("Stopping scraper for target %v...", t)
|
||||
|
||||
close(t.scraperStopping)
|
||||
<-t.scraperStopped
|
||||
|
||||
log.Debugf("Scraper for target %v stopped.", t)
|
||||
}
|
||||
|
||||
const acceptHeader = `application/vnd.google.protobuf;proto=io.prometheus.client.MetricFamily;encoding=delimited;q=0.7,text/plain;version=0.0.4;q=0.3,application/json;schema="prometheus/telemetry";version=0.0.2;q=0.2,*/*;q=0.1`
|
||||
|
||||
func (t *Target) scrape(sampleAppender storage.SampleAppender) (err error) {
|
||||
start := time.Now()
|
||||
baseLabels := t.BaseLabels()
|
||||
|
||||
t.RLock()
|
||||
var (
|
||||
honorLabels = t.honorLabels
|
||||
httpClient = t.httpClient
|
||||
metricRelabelConfigs = t.metricRelabelConfigs
|
||||
)
|
||||
t.RUnlock()
|
||||
|
||||
defer func() {
|
||||
t.status.setLastError(err)
|
||||
recordScrapeHealth(sampleAppender, clientmodel.TimestampFromTime(start), baseLabels, t.status.Health(), time.Since(start))
|
||||
}()
|
||||
|
||||
req, err := http.NewRequest("GET", t.URL().String(), nil)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
req.Header.Add("Accept", acceptHeader)
|
||||
|
||||
resp, err := httpClient.Do(req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
if resp.StatusCode != http.StatusOK {
|
||||
return fmt.Errorf("server returned HTTP status %s", resp.Status)
|
||||
}
|
||||
|
||||
processor, err := extraction.ProcessorForRequestHeader(resp.Header)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
t.ingestedSamples = make(chan clientmodel.Samples, ingestedSamplesCap)
|
||||
|
||||
processOptions := &extraction.ProcessOptions{
|
||||
Timestamp: clientmodel.TimestampFromTime(start),
|
||||
}
|
||||
go func() {
|
||||
err = processor.ProcessSingle(resp.Body, t, processOptions)
|
||||
close(t.ingestedSamples)
|
||||
}()
|
||||
|
||||
for samples := range t.ingestedSamples {
|
||||
for _, s := range samples {
|
||||
if honorLabels {
|
||||
// Merge the metric with the baseLabels for labels not already set in the
|
||||
// metric. This also considers labels explicitly set to the empty string.
|
||||
for ln, lv := range baseLabels {
|
||||
if _, ok := s.Metric[ln]; !ok {
|
||||
s.Metric[ln] = lv
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Merge the ingested metric with the base label set. On a collision the
|
||||
// value of the label is stored in a label prefixed with the exported prefix.
|
||||
for ln, lv := range baseLabels {
|
||||
if v, ok := s.Metric[ln]; ok && v != "" {
|
||||
s.Metric[clientmodel.ExportedLabelPrefix+ln] = v
|
||||
}
|
||||
s.Metric[ln] = lv
|
||||
}
|
||||
}
|
||||
// Avoid the copy in Relabel if there are no configs.
|
||||
if len(metricRelabelConfigs) > 0 {
|
||||
labels, err := Relabel(clientmodel.LabelSet(s.Metric), metricRelabelConfigs...)
|
||||
if err != nil {
|
||||
log.Errorf("Error while relabeling metric %s of instance %s: %s", s.Metric, req.URL, err)
|
||||
continue
|
||||
}
|
||||
// Check if the timeseries was dropped.
|
||||
if labels == nil {
|
||||
continue
|
||||
}
|
||||
s.Metric = clientmodel.Metric(labels)
|
||||
}
|
||||
sampleAppender.Append(s)
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// URL returns a copy of the target's URL.
|
||||
func (t *Target) URL() *url.URL {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
u := &url.URL{}
|
||||
*u = *t.url
|
||||
return u
|
||||
}
|
||||
|
||||
// InstanceIdentifier returns the identifier for the target.
|
||||
func (t *Target) InstanceIdentifier() string {
|
||||
return t.url.Host
|
||||
}
|
||||
|
||||
// fullLabels returns the base labels plus internal labels defining the target.
|
||||
func (t *Target) fullLabels() clientmodel.LabelSet {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
lset := make(clientmodel.LabelSet, len(t.baseLabels)+2)
|
||||
for ln, lv := range t.baseLabels {
|
||||
lset[ln] = lv
|
||||
}
|
||||
lset[clientmodel.MetricsPathLabel] = clientmodel.LabelValue(t.url.Path)
|
||||
lset[clientmodel.AddressLabel] = clientmodel.LabelValue(t.url.Host)
|
||||
lset[clientmodel.SchemeLabel] = clientmodel.LabelValue(t.url.Scheme)
|
||||
return lset
|
||||
}
|
||||
|
||||
// BaseLabels returns a copy of the target's base labels.
|
||||
func (t *Target) BaseLabels() clientmodel.LabelSet {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
lset := make(clientmodel.LabelSet, len(t.baseLabels))
|
||||
for ln, lv := range t.baseLabels {
|
||||
lset[ln] = lv
|
||||
}
|
||||
return lset
|
||||
}
|
||||
|
||||
// MetaLabels returns a copy of the target's labels before any processing.
|
||||
func (t *Target) MetaLabels() clientmodel.LabelSet {
|
||||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
lset := make(clientmodel.LabelSet, len(t.metaLabels))
|
||||
for ln, lv := range t.metaLabels {
|
||||
lset[ln] = lv
|
||||
}
|
||||
return lset
|
||||
}
|
||||
|
||||
func recordScrapeHealth(
|
||||
sampleAppender storage.SampleAppender,
|
||||
timestamp clientmodel.Timestamp,
|
||||
baseLabels clientmodel.LabelSet,
|
||||
health TargetHealth,
|
||||
scrapeDuration time.Duration,
|
||||
) {
|
||||
healthMetric := make(clientmodel.Metric, len(baseLabels)+1)
|
||||
durationMetric := make(clientmodel.Metric, len(baseLabels)+1)
|
||||
|
||||
healthMetric[clientmodel.MetricNameLabel] = clientmodel.LabelValue(scrapeHealthMetricName)
|
||||
durationMetric[clientmodel.MetricNameLabel] = clientmodel.LabelValue(scrapeDurationMetricName)
|
||||
|
||||
for label, value := range baseLabels {
|
||||
healthMetric[label] = value
|
||||
durationMetric[label] = value
|
||||
}
|
||||
|
||||
healthValue := clientmodel.SampleValue(0)
|
||||
if health == HealthGood {
|
||||
healthValue = clientmodel.SampleValue(1)
|
||||
}
|
||||
|
||||
healthSample := &clientmodel.Sample{
|
||||
Metric: healthMetric,
|
||||
Timestamp: timestamp,
|
||||
Value: healthValue,
|
||||
}
|
||||
durationSample := &clientmodel.Sample{
|
||||
Metric: durationMetric,
|
||||
Timestamp: timestamp,
|
||||
Value: clientmodel.SampleValue(float64(scrapeDuration) / float64(time.Second)),
|
||||
}
|
||||
|
||||
sampleAppender.Append(healthSample)
|
||||
sampleAppender.Append(durationSample)
|
||||
}
|
631
prom/retrieval/target_test.go
Normal file
631
prom/retrieval/target_test.go
Normal file
@ -0,0 +1,631 @@
|
||||
// Copyright 2013 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 retrieval
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"net/url"
|
||||
"reflect"
|
||||
"regexp"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
func TestBaseLabels(t *testing.T) {
|
||||
target := newTestTarget("example.com:80", 0, clientmodel.LabelSet{"job": "some_job", "foo": "bar"})
|
||||
want := clientmodel.LabelSet{
|
||||
clientmodel.JobLabel: "some_job",
|
||||
clientmodel.InstanceLabel: "example.com:80",
|
||||
"foo": "bar",
|
||||
}
|
||||
got := target.BaseLabels()
|
||||
if !reflect.DeepEqual(want, got) {
|
||||
t.Errorf("want base labels %v, got %v", want, got)
|
||||
}
|
||||
}
|
||||
|
||||
func TestOverwriteLabels(t *testing.T) {
|
||||
type test struct {
|
||||
metric string
|
||||
resultNormal clientmodel.Metric
|
||||
resultHonor clientmodel.Metric
|
||||
}
|
||||
var tests []test
|
||||
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
for _, test := range tests {
|
||||
w.Write([]byte(test.metric))
|
||||
w.Write([]byte(" 1\n"))
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
addr := clientmodel.LabelValue(strings.Split(server.URL, "://")[1])
|
||||
|
||||
tests = []test{
|
||||
{
|
||||
metric: `foo{}`,
|
||||
resultNormal: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
clientmodel.InstanceLabel: addr,
|
||||
},
|
||||
resultHonor: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
clientmodel.InstanceLabel: addr,
|
||||
},
|
||||
},
|
||||
{
|
||||
metric: `foo{instance=""}`,
|
||||
resultNormal: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
clientmodel.InstanceLabel: addr,
|
||||
},
|
||||
resultHonor: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
},
|
||||
},
|
||||
{
|
||||
metric: `foo{instance="other_instance"}`,
|
||||
resultNormal: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
clientmodel.InstanceLabel: addr,
|
||||
clientmodel.ExportedLabelPrefix + clientmodel.InstanceLabel: "other_instance",
|
||||
},
|
||||
resultHonor: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "foo",
|
||||
clientmodel.InstanceLabel: "other_instance",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
target := newTestTarget(server.URL, 10*time.Millisecond, nil)
|
||||
|
||||
target.honorLabels = false
|
||||
app := &collectResultAppender{}
|
||||
if err := target.scrape(app); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
for i, test := range tests {
|
||||
if !reflect.DeepEqual(app.result[i].Metric, test.resultNormal) {
|
||||
t.Errorf("Error comparing %q:\nExpected:\n%s\nGot:\n%s\n", test.metric, test.resultNormal, app.result[i].Metric)
|
||||
}
|
||||
}
|
||||
|
||||
target.honorLabels = true
|
||||
app = &collectResultAppender{}
|
||||
if err := target.scrape(app); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
for i, test := range tests {
|
||||
if !reflect.DeepEqual(app.result[i].Metric, test.resultHonor) {
|
||||
t.Errorf("Error comparing %q:\nExpected:\n%s\nGot:\n%s\n", test.metric, test.resultHonor, app.result[i].Metric)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
func TestTargetScrapeUpdatesState(t *testing.T) {
|
||||
testTarget := newTestTarget("bad schema", 0, nil)
|
||||
|
||||
testTarget.scrape(nopAppender{})
|
||||
if testTarget.status.Health() != HealthBad {
|
||||
t.Errorf("Expected target state %v, actual: %v", HealthBad, testTarget.status.Health())
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetScrapeWithFullChannel(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
for i := 0; i < 2*ingestedSamplesCap; i++ {
|
||||
w.Write([]byte(
|
||||
fmt.Sprintf("test_metric_%d{foo=\"bar\"} 123.456\n", i),
|
||||
))
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
testTarget := newTestTarget(server.URL, 10*time.Millisecond, clientmodel.LabelSet{"dings": "bums"})
|
||||
|
||||
testTarget.scrape(slowAppender{})
|
||||
if testTarget.status.Health() != HealthBad {
|
||||
t.Errorf("Expected target state %v, actual: %v", HealthBad, testTarget.status.Health())
|
||||
}
|
||||
if testTarget.status.LastError() != errIngestChannelFull {
|
||||
t.Errorf("Expected target error %q, actual: %q", errIngestChannelFull, testTarget.status.LastError())
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetScrapeMetricRelabelConfigs(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte("test_metric_drop 0\n"))
|
||||
w.Write([]byte("test_metric_relabel 1\n"))
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
testTarget := newTestTarget(server.URL, 10*time.Millisecond, clientmodel.LabelSet{})
|
||||
testTarget.metricRelabelConfigs = []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"__name__"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile(".*drop.*")},
|
||||
Action: config.RelabelDrop,
|
||||
},
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{"__name__"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile(".*(relabel|up).*")},
|
||||
TargetLabel: "foo",
|
||||
Replacement: "bar",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
}
|
||||
|
||||
appender := &collectResultAppender{}
|
||||
testTarget.scrape(appender)
|
||||
|
||||
// Remove variables part of result.
|
||||
for _, sample := range appender.result {
|
||||
sample.Timestamp = 0
|
||||
sample.Value = 0
|
||||
}
|
||||
|
||||
expected := []*clientmodel.Sample{
|
||||
{
|
||||
Metric: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: "test_metric_relabel",
|
||||
"foo": "bar",
|
||||
clientmodel.InstanceLabel: clientmodel.LabelValue(testTarget.url.Host),
|
||||
},
|
||||
Timestamp: 0,
|
||||
Value: 0,
|
||||
},
|
||||
// The metrics about the scrape are not affected.
|
||||
{
|
||||
Metric: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: scrapeHealthMetricName,
|
||||
clientmodel.InstanceLabel: clientmodel.LabelValue(testTarget.url.Host),
|
||||
},
|
||||
Timestamp: 0,
|
||||
Value: 0,
|
||||
},
|
||||
{
|
||||
Metric: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: scrapeDurationMetricName,
|
||||
clientmodel.InstanceLabel: clientmodel.LabelValue(testTarget.url.Host),
|
||||
},
|
||||
Timestamp: 0,
|
||||
Value: 0,
|
||||
},
|
||||
}
|
||||
|
||||
if !appender.result.Equal(expected) {
|
||||
t.Fatalf("Expected and actual samples not equal. Expected: %s, actual: %s", expected, appender.result)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func TestTargetRecordScrapeHealth(t *testing.T) {
|
||||
testTarget := newTestTarget("example.url:80", 0, clientmodel.LabelSet{clientmodel.JobLabel: "testjob"})
|
||||
|
||||
now := clientmodel.Now()
|
||||
appender := &collectResultAppender{}
|
||||
testTarget.status.setLastError(nil)
|
||||
recordScrapeHealth(appender, now, testTarget.BaseLabels(), testTarget.status.Health(), 2*time.Second)
|
||||
|
||||
result := appender.result
|
||||
|
||||
if len(result) != 2 {
|
||||
t.Fatalf("Expected two samples, got %d", len(result))
|
||||
}
|
||||
|
||||
actual := result[0]
|
||||
expected := &clientmodel.Sample{
|
||||
Metric: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: scrapeHealthMetricName,
|
||||
clientmodel.InstanceLabel: "example.url:80",
|
||||
clientmodel.JobLabel: "testjob",
|
||||
},
|
||||
Timestamp: now,
|
||||
Value: 1,
|
||||
}
|
||||
|
||||
if !actual.Equal(expected) {
|
||||
t.Fatalf("Expected and actual samples not equal. Expected: %v, actual: %v", expected, actual)
|
||||
}
|
||||
|
||||
actual = result[1]
|
||||
expected = &clientmodel.Sample{
|
||||
Metric: clientmodel.Metric{
|
||||
clientmodel.MetricNameLabel: scrapeDurationMetricName,
|
||||
clientmodel.InstanceLabel: "example.url:80",
|
||||
clientmodel.JobLabel: "testjob",
|
||||
},
|
||||
Timestamp: now,
|
||||
Value: 2.0,
|
||||
}
|
||||
|
||||
if !actual.Equal(expected) {
|
||||
t.Fatalf("Expected and actual samples not equal. Expected: %v, actual: %v", expected, actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetScrapeTimeout(t *testing.T) {
|
||||
signal := make(chan bool, 1)
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
<-signal
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte{})
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
testTarget := newTestTarget(server.URL, 50*time.Millisecond, clientmodel.LabelSet{})
|
||||
|
||||
appender := nopAppender{}
|
||||
|
||||
// scrape once without timeout
|
||||
signal <- true
|
||||
if err := testTarget.scrape(appender); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// let the deadline lapse
|
||||
time.Sleep(55 * time.Millisecond)
|
||||
|
||||
// now scrape again
|
||||
signal <- true
|
||||
if err := testTarget.scrape(appender); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// now timeout
|
||||
if err := testTarget.scrape(appender); err == nil {
|
||||
t.Fatal("expected scrape to timeout")
|
||||
} else {
|
||||
signal <- true // let handler continue
|
||||
}
|
||||
|
||||
// now scrape again without timeout
|
||||
signal <- true
|
||||
if err := testTarget.scrape(appender); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetScrape404(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusNotFound)
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
testTarget := newTestTarget(server.URL, 10*time.Millisecond, clientmodel.LabelSet{})
|
||||
appender := nopAppender{}
|
||||
|
||||
want := errors.New("server returned HTTP status 404 Not Found")
|
||||
got := testTarget.scrape(appender)
|
||||
if got == nil || want.Error() != got.Error() {
|
||||
t.Fatalf("want err %q, got %q", want, got)
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetRunScraperScrapes(t *testing.T) {
|
||||
testTarget := newTestTarget("bad schema", 0, nil)
|
||||
|
||||
go testTarget.RunScraper(nopAppender{})
|
||||
|
||||
// Enough time for a scrape to happen.
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
if testTarget.status.LastScrape().IsZero() {
|
||||
t.Errorf("Scrape hasn't occured.")
|
||||
}
|
||||
|
||||
testTarget.StopScraper()
|
||||
// Wait for it to take effect.
|
||||
time.Sleep(5 * time.Millisecond)
|
||||
last := testTarget.status.LastScrape()
|
||||
// Enough time for a scrape to happen.
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
if testTarget.status.LastScrape() != last {
|
||||
t.Errorf("Scrape occured after it was stopped.")
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkScrape(b *testing.B) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte("test_metric{foo=\"bar\"} 123.456\n"))
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
testTarget := newTestTarget(server.URL, 100*time.Millisecond, clientmodel.LabelSet{"dings": "bums"})
|
||||
appender := nopAppender{}
|
||||
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
if err := testTarget.scrape(appender); err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestURLParams(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte{})
|
||||
r.ParseForm()
|
||||
if r.Form["foo"][0] != "bar" {
|
||||
t.Fatalf("URL parameter 'foo' had unexpected first value '%v'", r.Form["foo"][0])
|
||||
}
|
||||
if r.Form["foo"][1] != "baz" {
|
||||
t.Fatalf("URL parameter 'foo' had unexpected second value '%v'", r.Form["foo"][1])
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
serverURL, err := url.Parse(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
target := NewTarget(
|
||||
&config.ScrapeConfig{
|
||||
JobName: "test_job1",
|
||||
ScrapeInterval: config.Duration(1 * time.Minute),
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
Scheme: serverURL.Scheme,
|
||||
Params: url.Values{
|
||||
"foo": []string{"bar", "baz"},
|
||||
},
|
||||
},
|
||||
clientmodel.LabelSet{
|
||||
clientmodel.SchemeLabel: clientmodel.LabelValue(serverURL.Scheme),
|
||||
clientmodel.AddressLabel: clientmodel.LabelValue(serverURL.Host),
|
||||
"__param_foo": "bar",
|
||||
},
|
||||
nil)
|
||||
app := &collectResultAppender{}
|
||||
if err = target.scrape(app); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func newTestTarget(targetURL string, deadline time.Duration, baseLabels clientmodel.LabelSet) *Target {
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(deadline),
|
||||
}
|
||||
c, _ := newHTTPClient(cfg)
|
||||
t := &Target{
|
||||
url: &url.URL{
|
||||
Scheme: "http",
|
||||
Host: strings.TrimLeft(targetURL, "http://"),
|
||||
Path: "/metrics",
|
||||
},
|
||||
deadline: deadline,
|
||||
status: &TargetStatus{},
|
||||
scrapeInterval: 1 * time.Millisecond,
|
||||
httpClient: c,
|
||||
scraperStopping: make(chan struct{}),
|
||||
scraperStopped: make(chan struct{}),
|
||||
}
|
||||
t.baseLabels = clientmodel.LabelSet{
|
||||
clientmodel.InstanceLabel: clientmodel.LabelValue(t.InstanceIdentifier()),
|
||||
}
|
||||
for baseLabel, baseValue := range baseLabels {
|
||||
t.baseLabels[baseLabel] = baseValue
|
||||
}
|
||||
return t
|
||||
}
|
||||
|
||||
func TestNewHTTPBearerToken(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
expected := "Bearer 1234"
|
||||
received := r.Header.Get("Authorization")
|
||||
if expected != received {
|
||||
t.Fatalf("Authorization header was not set correctly: expected '%v', got '%v'", expected, received)
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
BearerToken: "1234",
|
||||
}
|
||||
c, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, err = c.Get(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewHTTPBearerTokenFile(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
expected := "Bearer 12345"
|
||||
received := r.Header.Get("Authorization")
|
||||
if expected != received {
|
||||
t.Fatalf("Authorization header was not set correctly: expected '%v', got '%v'", expected, received)
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
BearerTokenFile: "testdata/bearertoken.txt",
|
||||
}
|
||||
c, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, err = c.Get(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewHTTPBasicAuth(t *testing.T) {
|
||||
server := httptest.NewServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
username, password, ok := r.BasicAuth()
|
||||
if !(ok && username == "user" && password == "password123") {
|
||||
t.Fatalf("Basic authorization header was not set correctly: expected '%v:%v', got '%v:%v'", "user", "password123", username, password)
|
||||
}
|
||||
},
|
||||
),
|
||||
)
|
||||
defer server.Close()
|
||||
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
BasicAuth: &config.BasicAuth{
|
||||
Username: "user",
|
||||
Password: "password123",
|
||||
},
|
||||
}
|
||||
c, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, err = c.Get(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewHTTPCACert(t *testing.T) {
|
||||
server := httptest.NewUnstartedServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte{})
|
||||
},
|
||||
),
|
||||
)
|
||||
server.TLS = newTLSConfig(t)
|
||||
server.StartTLS()
|
||||
defer server.Close()
|
||||
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
CACert: "testdata/ca.cer",
|
||||
}
|
||||
c, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, err = c.Get(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewHTTPClientCert(t *testing.T) {
|
||||
server := httptest.NewUnstartedServer(
|
||||
http.HandlerFunc(
|
||||
func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Content-Type", `text/plain; version=0.0.4`)
|
||||
w.Write([]byte{})
|
||||
},
|
||||
),
|
||||
)
|
||||
tlsConfig := newTLSConfig(t)
|
||||
tlsConfig.ClientAuth = tls.RequireAndVerifyClientCert
|
||||
tlsConfig.ClientCAs = tlsConfig.RootCAs
|
||||
tlsConfig.BuildNameToCertificate()
|
||||
server.TLS = tlsConfig
|
||||
server.StartTLS()
|
||||
defer server.Close()
|
||||
|
||||
cfg := &config.ScrapeConfig{
|
||||
ScrapeTimeout: config.Duration(1 * time.Second),
|
||||
CACert: "testdata/ca.cer",
|
||||
ClientCert: &config.ClientCert{
|
||||
Cert: "testdata/client.cer",
|
||||
Key: "testdata/client.key",
|
||||
},
|
||||
}
|
||||
c, err := newHTTPClient(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
_, err = c.Get(server.URL)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func newTLSConfig(t *testing.T) *tls.Config {
|
||||
tlsConfig := &tls.Config{}
|
||||
caCertPool := x509.NewCertPool()
|
||||
caCert, err := ioutil.ReadFile("testdata/ca.cer")
|
||||
if err != nil {
|
||||
t.Fatalf("Couldn't set up TLS server: %v", err)
|
||||
}
|
||||
caCertPool.AppendCertsFromPEM(caCert)
|
||||
tlsConfig.RootCAs = caCertPool
|
||||
tlsConfig.ServerName = "127.0.0.1"
|
||||
cert, err := tls.LoadX509KeyPair("testdata/server.cer", "testdata/server.key")
|
||||
if err != nil {
|
||||
t.Errorf("Unable to use specified server cert (%s) & key (%v): %s", "testdata/server.cer", "testdata/server.key")
|
||||
}
|
||||
tlsConfig.Certificates = []tls.Certificate{cert}
|
||||
tlsConfig.BuildNameToCertificate()
|
||||
return tlsConfig
|
||||
}
|
557
prom/retrieval/targetmanager.go
Normal file
557
prom/retrieval/targetmanager.go
Normal file
@ -0,0 +1,557 @@
|
||||
// Copyright 2013 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 retrieval
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/prometheus/log"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/retrieval/discovery"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
)
|
||||
|
||||
// A TargetProvider provides information about target groups. It maintains a set
|
||||
// of sources from which TargetGroups can originate. Whenever a target provider
|
||||
// detects a potential change, it sends the TargetGroup through its provided channel.
|
||||
//
|
||||
// The TargetProvider does not have to guarantee that an actual change happened.
|
||||
// It does guarantee that it sends the new TargetGroup whenever a change happens.
|
||||
//
|
||||
// Sources() is guaranteed to be called exactly once before each call to Run().
|
||||
// On a call to Run() implementing types must send a valid target group for each of
|
||||
// the sources they declared in the last call to Sources().
|
||||
type TargetProvider interface {
|
||||
// Sources returns the source identifiers the provider is currently aware of.
|
||||
Sources() []string
|
||||
// Run hands a channel to the target provider through which it can send
|
||||
// updated target groups. The channel must be closed by the target provider
|
||||
// if no more updates will be sent.
|
||||
// On receiving from done Run must return.
|
||||
Run(up chan<- *config.TargetGroup, done <-chan struct{})
|
||||
}
|
||||
|
||||
// TargetManager maintains a set of targets, starts and stops their scraping and
|
||||
// creates the new targets based on the target groups it receives from various
|
||||
// target providers.
|
||||
type TargetManager struct {
|
||||
mtx sync.RWMutex
|
||||
globalLabels clientmodel.LabelSet
|
||||
sampleAppender storage.SampleAppender
|
||||
running bool
|
||||
done chan struct{}
|
||||
|
||||
// Targets by their source ID.
|
||||
targets map[string][]*Target
|
||||
// Providers by the scrape configs they are derived from.
|
||||
providers map[*config.ScrapeConfig][]TargetProvider
|
||||
}
|
||||
|
||||
// NewTargetManager creates a new TargetManager.
|
||||
func NewTargetManager(sampleAppender storage.SampleAppender) *TargetManager {
|
||||
tm := &TargetManager{
|
||||
sampleAppender: sampleAppender,
|
||||
targets: make(map[string][]*Target),
|
||||
}
|
||||
return tm
|
||||
}
|
||||
|
||||
// merge multiple target group channels into a single output channel.
|
||||
func merge(done <-chan struct{}, cs ...<-chan targetGroupUpdate) <-chan targetGroupUpdate {
|
||||
var wg sync.WaitGroup
|
||||
out := make(chan targetGroupUpdate)
|
||||
|
||||
// Start an output goroutine for each input channel in cs. output
|
||||
// copies values from c to out until c or done is closed, then calls
|
||||
// wg.Done.
|
||||
redir := func(c <-chan targetGroupUpdate) {
|
||||
defer wg.Done()
|
||||
for n := range c {
|
||||
select {
|
||||
case out <- n:
|
||||
case <-done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
wg.Add(len(cs))
|
||||
for _, c := range cs {
|
||||
go redir(c)
|
||||
}
|
||||
|
||||
// Close the out channel if all inbound channels are closed.
|
||||
go func() {
|
||||
wg.Wait()
|
||||
close(out)
|
||||
}()
|
||||
return out
|
||||
}
|
||||
|
||||
// targetGroupUpdate is a potentially changed/new target group
|
||||
// for the given scrape configuration.
|
||||
type targetGroupUpdate struct {
|
||||
tg *config.TargetGroup
|
||||
scfg *config.ScrapeConfig
|
||||
}
|
||||
|
||||
// Run starts background processing to handle target updates.
|
||||
func (tm *TargetManager) Run() {
|
||||
log.Info("Starting target manager...")
|
||||
|
||||
tm.done = make(chan struct{})
|
||||
|
||||
sources := map[string]struct{}{}
|
||||
updates := []<-chan targetGroupUpdate{}
|
||||
|
||||
for scfg, provs := range tm.providers {
|
||||
for _, prov := range provs {
|
||||
// Get an initial set of available sources so we don't remove
|
||||
// target groups from the last run that are still available.
|
||||
for _, src := range prov.Sources() {
|
||||
sources[src] = struct{}{}
|
||||
}
|
||||
|
||||
tgc := make(chan *config.TargetGroup)
|
||||
// Run the target provider after cleanup of the stale targets is done.
|
||||
defer func(prov TargetProvider, tgc chan *config.TargetGroup) {
|
||||
go prov.Run(tgc, tm.done)
|
||||
}(prov, tgc)
|
||||
|
||||
tgupc := make(chan targetGroupUpdate)
|
||||
updates = append(updates, tgupc)
|
||||
|
||||
go func(scfg *config.ScrapeConfig) {
|
||||
defer close(tgupc)
|
||||
for {
|
||||
select {
|
||||
case tg := <-tgc:
|
||||
if tg == nil {
|
||||
break
|
||||
}
|
||||
tgupc <- targetGroupUpdate{tg: tg, scfg: scfg}
|
||||
case <-tm.done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}(scfg)
|
||||
}
|
||||
}
|
||||
|
||||
// Merge all channels of incoming target group updates into a single
|
||||
// one and keep applying the updates.
|
||||
go tm.handleUpdates(merge(tm.done, updates...), tm.done)
|
||||
|
||||
tm.mtx.Lock()
|
||||
defer tm.mtx.Unlock()
|
||||
|
||||
// Remove old target groups that are no longer in the set of sources.
|
||||
tm.removeTargets(func(src string) bool {
|
||||
if _, ok := sources[src]; ok {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
})
|
||||
|
||||
tm.running = true
|
||||
}
|
||||
|
||||
// handleTargetUpdates receives target group updates and handles them in the
|
||||
// context of the given job config.
|
||||
func (tm *TargetManager) handleUpdates(ch <-chan targetGroupUpdate, done <-chan struct{}) {
|
||||
for {
|
||||
select {
|
||||
case update := <-ch:
|
||||
if update.tg == nil {
|
||||
break
|
||||
}
|
||||
log.Debugf("Received potential update for target group %q", update.tg.Source)
|
||||
|
||||
if err := tm.updateTargetGroup(update.tg, update.scfg); err != nil {
|
||||
log.Errorf("Error updating targets: %s", err)
|
||||
}
|
||||
case <-done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stop all background processing.
|
||||
func (tm *TargetManager) Stop() {
|
||||
tm.mtx.RLock()
|
||||
if tm.running {
|
||||
defer tm.stop(true)
|
||||
}
|
||||
// Return the lock before calling tm.stop().
|
||||
defer tm.mtx.RUnlock()
|
||||
}
|
||||
|
||||
// stop background processing of the target manager. If removeTargets is true,
|
||||
// existing targets will be stopped and removed.
|
||||
func (tm *TargetManager) stop(removeTargets bool) {
|
||||
log.Info("Stopping target manager...")
|
||||
defer log.Info("Target manager stopped.")
|
||||
|
||||
close(tm.done)
|
||||
|
||||
tm.mtx.Lock()
|
||||
defer tm.mtx.Unlock()
|
||||
|
||||
if removeTargets {
|
||||
tm.removeTargets(nil)
|
||||
}
|
||||
|
||||
tm.running = false
|
||||
}
|
||||
|
||||
// removeTargets stops and removes targets for sources where f(source) is true
|
||||
// or if f is nil. This method is not thread-safe.
|
||||
func (tm *TargetManager) removeTargets(f func(string) bool) {
|
||||
if f == nil {
|
||||
f = func(string) bool { return true }
|
||||
}
|
||||
var wg sync.WaitGroup
|
||||
for src, targets := range tm.targets {
|
||||
if !f(src) {
|
||||
continue
|
||||
}
|
||||
wg.Add(len(targets))
|
||||
for _, target := range targets {
|
||||
go func(t *Target) {
|
||||
t.StopScraper()
|
||||
wg.Done()
|
||||
}(target)
|
||||
}
|
||||
delete(tm.targets, src)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
// updateTargetGroup creates new targets for the group and replaces the old targets
|
||||
// for the source ID.
|
||||
func (tm *TargetManager) updateTargetGroup(tgroup *config.TargetGroup, cfg *config.ScrapeConfig) error {
|
||||
newTargets, err := tm.targetsFromGroup(tgroup, cfg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
tm.mtx.Lock()
|
||||
defer tm.mtx.Unlock()
|
||||
|
||||
if !tm.running {
|
||||
return nil
|
||||
}
|
||||
|
||||
oldTargets, ok := tm.targets[tgroup.Source]
|
||||
if ok {
|
||||
var wg sync.WaitGroup
|
||||
// Replace the old targets with the new ones while keeping the state
|
||||
// of intersecting targets.
|
||||
for i, tnew := range newTargets {
|
||||
var match *Target
|
||||
for j, told := range oldTargets {
|
||||
if told == nil {
|
||||
continue
|
||||
}
|
||||
if tnew.InstanceIdentifier() == told.InstanceIdentifier() {
|
||||
match = told
|
||||
oldTargets[j] = nil
|
||||
break
|
||||
}
|
||||
}
|
||||
// Update the existing target and discard the new equivalent.
|
||||
// Otherwise start scraping the new target.
|
||||
if match != nil {
|
||||
// Updating is blocked during a scrape. We don't want those wait times
|
||||
// to build up.
|
||||
wg.Add(1)
|
||||
go func(t *Target) {
|
||||
match.Update(cfg, t.fullLabels(), t.metaLabels)
|
||||
wg.Done()
|
||||
}(tnew)
|
||||
newTargets[i] = match
|
||||
} else {
|
||||
go tnew.RunScraper(tm.sampleAppender)
|
||||
}
|
||||
}
|
||||
// Remove all old targets that disappeared.
|
||||
for _, told := range oldTargets {
|
||||
if told != nil {
|
||||
wg.Add(1)
|
||||
go func(t *Target) {
|
||||
t.StopScraper()
|
||||
wg.Done()
|
||||
}(told)
|
||||
}
|
||||
}
|
||||
wg.Wait()
|
||||
} else {
|
||||
// The source ID is new, start all target scrapers.
|
||||
for _, tnew := range newTargets {
|
||||
go tnew.RunScraper(tm.sampleAppender)
|
||||
}
|
||||
}
|
||||
|
||||
if len(newTargets) > 0 {
|
||||
tm.targets[tgroup.Source] = newTargets
|
||||
} else {
|
||||
delete(tm.targets, tgroup.Source)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Pools returns the targets currently being scraped bucketed by their job name.
|
||||
func (tm *TargetManager) Pools() map[string][]*Target {
|
||||
tm.mtx.RLock()
|
||||
defer tm.mtx.RUnlock()
|
||||
|
||||
pools := map[string][]*Target{}
|
||||
|
||||
for _, ts := range tm.targets {
|
||||
for _, t := range ts {
|
||||
job := string(t.BaseLabels()[clientmodel.JobLabel])
|
||||
pools[job] = append(pools[job], t)
|
||||
}
|
||||
}
|
||||
return pools
|
||||
}
|
||||
|
||||
// ApplyConfig resets the manager's target providers and job configurations as defined
|
||||
// by the new cfg. The state of targets that are valid in the new configuration remains unchanged.
|
||||
// Returns true on success.
|
||||
func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool {
|
||||
tm.mtx.RLock()
|
||||
running := tm.running
|
||||
tm.mtx.RUnlock()
|
||||
|
||||
if running {
|
||||
tm.stop(false)
|
||||
// Even if updating the config failed, we want to continue rather than stop scraping anything.
|
||||
defer tm.Run()
|
||||
}
|
||||
providers := map[*config.ScrapeConfig][]TargetProvider{}
|
||||
|
||||
for _, scfg := range cfg.ScrapeConfigs {
|
||||
providers[scfg] = providersFromConfig(scfg)
|
||||
}
|
||||
|
||||
tm.mtx.Lock()
|
||||
defer tm.mtx.Unlock()
|
||||
|
||||
tm.globalLabels = cfg.GlobalConfig.Labels
|
||||
tm.providers = providers
|
||||
return true
|
||||
}
|
||||
|
||||
// prefixedTargetProvider wraps TargetProvider and prefixes source strings
|
||||
// to make the sources unique across a configuration.
|
||||
type prefixedTargetProvider struct {
|
||||
TargetProvider
|
||||
|
||||
job string
|
||||
mechanism string
|
||||
idx int
|
||||
}
|
||||
|
||||
func (tp *prefixedTargetProvider) prefix(src string) string {
|
||||
return fmt.Sprintf("%s:%s:%d:%s", tp.job, tp.mechanism, tp.idx, src)
|
||||
}
|
||||
|
||||
func (tp *prefixedTargetProvider) Sources() []string {
|
||||
srcs := tp.TargetProvider.Sources()
|
||||
for i, src := range srcs {
|
||||
srcs[i] = tp.prefix(src)
|
||||
}
|
||||
|
||||
return srcs
|
||||
}
|
||||
|
||||
func (tp *prefixedTargetProvider) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
|
||||
ch2 := make(chan *config.TargetGroup)
|
||||
go tp.TargetProvider.Run(ch2, done)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
case tg := <-ch2:
|
||||
if tg == nil {
|
||||
break
|
||||
}
|
||||
tg.Source = tp.prefix(tg.Source)
|
||||
ch <- tg
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// providersFromConfig returns all TargetProviders configured in cfg.
|
||||
func providersFromConfig(cfg *config.ScrapeConfig) []TargetProvider {
|
||||
var providers []TargetProvider
|
||||
|
||||
app := func(mech string, i int, tp TargetProvider) {
|
||||
providers = append(providers, &prefixedTargetProvider{
|
||||
job: cfg.JobName,
|
||||
mechanism: mech,
|
||||
idx: i,
|
||||
TargetProvider: tp,
|
||||
})
|
||||
}
|
||||
|
||||
for i, c := range cfg.DNSSDConfigs {
|
||||
app("dns", i, discovery.NewDNSDiscovery(c))
|
||||
}
|
||||
for i, c := range cfg.FileSDConfigs {
|
||||
app("file", i, discovery.NewFileDiscovery(c))
|
||||
}
|
||||
for i, c := range cfg.ConsulSDConfigs {
|
||||
app("consul", i, discovery.NewConsulDiscovery(c))
|
||||
}
|
||||
for i, c := range cfg.MarathonSDConfigs {
|
||||
app("marathon", i, discovery.NewMarathonDiscovery(c))
|
||||
}
|
||||
for i, c := range cfg.KubernetesSDConfigs {
|
||||
k, err := discovery.NewKubernetesDiscovery(c)
|
||||
if err != nil {
|
||||
log.Errorf("Cannot create Kubernetes discovery: %s", err)
|
||||
continue
|
||||
}
|
||||
app("kubernetes", i, k)
|
||||
}
|
||||
for i, c := range cfg.ServersetSDConfigs {
|
||||
app("serverset", i, discovery.NewServersetDiscovery(c))
|
||||
}
|
||||
if len(cfg.TargetGroups) > 0 {
|
||||
app("static", 0, NewStaticProvider(cfg.TargetGroups))
|
||||
}
|
||||
|
||||
return providers
|
||||
}
|
||||
|
||||
// targetsFromGroup builds targets based on the given TargetGroup and config.
|
||||
func (tm *TargetManager) targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) ([]*Target, error) {
|
||||
tm.mtx.RLock()
|
||||
defer tm.mtx.RUnlock()
|
||||
|
||||
targets := make([]*Target, 0, len(tg.Targets))
|
||||
for i, labels := range tg.Targets {
|
||||
addr := string(labels[clientmodel.AddressLabel])
|
||||
// If no port was provided, infer it based on the used scheme.
|
||||
if !strings.Contains(addr, ":") {
|
||||
switch cfg.Scheme {
|
||||
case "http":
|
||||
addr = fmt.Sprintf("%s:80", addr)
|
||||
case "https":
|
||||
addr = fmt.Sprintf("%s:443", addr)
|
||||
default:
|
||||
panic(fmt.Errorf("targetsFromGroup: invalid scheme %q", cfg.Scheme))
|
||||
}
|
||||
labels[clientmodel.AddressLabel] = clientmodel.LabelValue(addr)
|
||||
}
|
||||
for k, v := range cfg.Params {
|
||||
if len(v) > 0 {
|
||||
labels[clientmodel.LabelName(clientmodel.ParamLabelPrefix+k)] = clientmodel.LabelValue(v[0])
|
||||
}
|
||||
}
|
||||
// Copy labels into the labelset for the target if they are not
|
||||
// set already. Apply the labelsets in order of decreasing precedence.
|
||||
labelsets := []clientmodel.LabelSet{
|
||||
tg.Labels,
|
||||
{
|
||||
clientmodel.SchemeLabel: clientmodel.LabelValue(cfg.Scheme),
|
||||
clientmodel.MetricsPathLabel: clientmodel.LabelValue(cfg.MetricsPath),
|
||||
clientmodel.JobLabel: clientmodel.LabelValue(cfg.JobName),
|
||||
},
|
||||
tm.globalLabels,
|
||||
}
|
||||
for _, lset := range labelsets {
|
||||
for ln, lv := range lset {
|
||||
if _, ok := labels[ln]; !ok {
|
||||
labels[ln] = lv
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if _, ok := labels[clientmodel.AddressLabel]; !ok {
|
||||
return nil, fmt.Errorf("instance %d in target group %s has no address", i, tg)
|
||||
}
|
||||
|
||||
preRelabelLabels := labels
|
||||
|
||||
labels, err := Relabel(labels, cfg.RelabelConfigs...)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error while relabeling instance %d in target group %s: %s", i, tg, err)
|
||||
}
|
||||
// Check if the target was dropped.
|
||||
if labels == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
for ln := range labels {
|
||||
// Meta labels are deleted after relabelling. Other internal labels propagate to
|
||||
// the target which decides whether they will be part of their label set.
|
||||
if strings.HasPrefix(string(ln), clientmodel.MetaLabelPrefix) {
|
||||
delete(labels, ln)
|
||||
}
|
||||
}
|
||||
tr := NewTarget(cfg, labels, preRelabelLabels)
|
||||
targets = append(targets, tr)
|
||||
}
|
||||
|
||||
return targets, nil
|
||||
}
|
||||
|
||||
// StaticProvider holds a list of target groups that never change.
|
||||
type StaticProvider struct {
|
||||
TargetGroups []*config.TargetGroup
|
||||
}
|
||||
|
||||
// NewStaticProvider returns a StaticProvider configured with the given
|
||||
// target groups.
|
||||
func NewStaticProvider(groups []*config.TargetGroup) *StaticProvider {
|
||||
for i, tg := range groups {
|
||||
tg.Source = fmt.Sprintf("%d", i)
|
||||
}
|
||||
return &StaticProvider{
|
||||
TargetGroups: groups,
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements the TargetProvider interface.
|
||||
func (sd *StaticProvider) Run(ch chan<- *config.TargetGroup, done <-chan struct{}) {
|
||||
defer close(ch)
|
||||
|
||||
for _, tg := range sd.TargetGroups {
|
||||
select {
|
||||
case <-done:
|
||||
return
|
||||
case ch <- tg:
|
||||
}
|
||||
}
|
||||
<-done
|
||||
}
|
||||
|
||||
// TargetGroups returns the provider's target groups.
|
||||
func (sd *StaticProvider) Sources() (srcs []string) {
|
||||
for _, tg := range sd.TargetGroups {
|
||||
srcs = append(srcs, tg.Source)
|
||||
}
|
||||
return srcs
|
||||
}
|
378
prom/retrieval/targetmanager_test.go
Normal file
378
prom/retrieval/targetmanager_test.go
Normal file
@ -0,0 +1,378 @@
|
||||
// Copyright 2013 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 retrieval
|
||||
|
||||
import (
|
||||
"net/url"
|
||||
"reflect"
|
||||
"regexp"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
)
|
||||
|
||||
func TestPrefixedTargetProvider(t *testing.T) {
|
||||
targetGroups := []*config.TargetGroup{
|
||||
{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test-1:1234"},
|
||||
},
|
||||
}, {
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test-1:1235"},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
tp := &prefixedTargetProvider{
|
||||
job: "job-x",
|
||||
mechanism: "static",
|
||||
idx: 123,
|
||||
TargetProvider: NewStaticProvider(targetGroups),
|
||||
}
|
||||
|
||||
expSources := []string{
|
||||
"job-x:static:123:0",
|
||||
"job-x:static:123:1",
|
||||
}
|
||||
if !reflect.DeepEqual(tp.Sources(), expSources) {
|
||||
t.Fatalf("expected sources %v, got %v", expSources, tp.Sources())
|
||||
}
|
||||
|
||||
ch := make(chan *config.TargetGroup)
|
||||
done := make(chan struct{})
|
||||
|
||||
defer close(done)
|
||||
go tp.Run(ch, done)
|
||||
|
||||
expGroup1 := *targetGroups[0]
|
||||
expGroup2 := *targetGroups[1]
|
||||
expGroup1.Source = "job-x:static:123:0"
|
||||
expGroup2.Source = "job-x:static:123:1"
|
||||
|
||||
// The static target provider sends on the channel once per target group.
|
||||
if tg := <-ch; !reflect.DeepEqual(tg, &expGroup1) {
|
||||
t.Fatalf("expected target group %v, got %v", expGroup1, tg)
|
||||
}
|
||||
if tg := <-ch; !reflect.DeepEqual(tg, &expGroup2) {
|
||||
t.Fatalf("expected target group %v, got %v", expGroup2, tg)
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetManagerChan(t *testing.T) {
|
||||
testJob1 := &config.ScrapeConfig{
|
||||
JobName: "test_job1",
|
||||
ScrapeInterval: config.Duration(1 * time.Minute),
|
||||
TargetGroups: []*config.TargetGroup{{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "example.org:80"},
|
||||
{clientmodel.AddressLabel: "example.com:80"},
|
||||
},
|
||||
}},
|
||||
}
|
||||
prov1 := &fakeTargetProvider{
|
||||
sources: []string{"src1", "src2"},
|
||||
update: make(chan *config.TargetGroup),
|
||||
}
|
||||
|
||||
targetManager := &TargetManager{
|
||||
sampleAppender: nopAppender{},
|
||||
providers: map[*config.ScrapeConfig][]TargetProvider{
|
||||
testJob1: {prov1},
|
||||
},
|
||||
targets: make(map[string][]*Target),
|
||||
}
|
||||
go targetManager.Run()
|
||||
defer targetManager.Stop()
|
||||
|
||||
sequence := []struct {
|
||||
tgroup *config.TargetGroup
|
||||
expected map[string][]clientmodel.LabelSet
|
||||
}{
|
||||
{
|
||||
tgroup: &config.TargetGroup{
|
||||
Source: "src1",
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test-1:1234"},
|
||||
{clientmodel.AddressLabel: "test-2:1234", "label": "set"},
|
||||
{clientmodel.AddressLabel: "test-3:1234"},
|
||||
},
|
||||
},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"src1": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-1:1234"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-2:1234", "label": "set"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-3:1234"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
tgroup: &config.TargetGroup{
|
||||
Source: "src2",
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test-1:1235"},
|
||||
{clientmodel.AddressLabel: "test-2:1235"},
|
||||
{clientmodel.AddressLabel: "test-3:1235"},
|
||||
},
|
||||
Labels: clientmodel.LabelSet{"group": "label"},
|
||||
},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"src1": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-1:1234"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-2:1234", "label": "set"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-3:1234"},
|
||||
},
|
||||
"src2": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-1:1235", "group": "label"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-2:1235", "group": "label"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-3:1235", "group": "label"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
tgroup: &config.TargetGroup{
|
||||
Source: "src2",
|
||||
Targets: []clientmodel.LabelSet{},
|
||||
},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"src1": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-1:1234"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-2:1234", "label": "set"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-3:1234"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
tgroup: &config.TargetGroup{
|
||||
Source: "src1",
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test-1:1234", "added": "label"},
|
||||
{clientmodel.AddressLabel: "test-3:1234"},
|
||||
{clientmodel.AddressLabel: "test-4:1234", "fancy": "label"},
|
||||
},
|
||||
},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"src1": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-1:1234", "added": "label"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-3:1234"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "test-4:1234", "fancy": "label"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for i, step := range sequence {
|
||||
prov1.update <- step.tgroup
|
||||
|
||||
<-time.After(1 * time.Millisecond)
|
||||
|
||||
if len(targetManager.targets) != len(step.expected) {
|
||||
t.Fatalf("step %d: sources mismatch %v, %v", targetManager.targets, step.expected)
|
||||
}
|
||||
|
||||
for source, actTargets := range targetManager.targets {
|
||||
expTargets, ok := step.expected[source]
|
||||
if !ok {
|
||||
t.Fatalf("step %d: unexpected source %q: %v", i, source, actTargets)
|
||||
}
|
||||
for _, expt := range expTargets {
|
||||
found := false
|
||||
for _, actt := range actTargets {
|
||||
if reflect.DeepEqual(expt, actt.BaseLabels()) {
|
||||
found = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !found {
|
||||
t.Errorf("step %d: expected target %v not found in actual targets", i, expt)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestTargetManagerConfigUpdate(t *testing.T) {
|
||||
testJob1 := &config.ScrapeConfig{
|
||||
JobName: "test_job1",
|
||||
ScrapeInterval: config.Duration(1 * time.Minute),
|
||||
Params: url.Values{
|
||||
"testParam": []string{"paramValue", "secondValue"},
|
||||
},
|
||||
TargetGroups: []*config.TargetGroup{{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "example.org:80"},
|
||||
{clientmodel.AddressLabel: "example.com:80"},
|
||||
},
|
||||
}},
|
||||
RelabelConfigs: []*config.RelabelConfig{
|
||||
{
|
||||
// Copy out the URL parameter.
|
||||
SourceLabels: clientmodel.LabelNames{"__param_testParam"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^(.*)$")},
|
||||
TargetLabel: "testParam",
|
||||
Replacement: "$1",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
}
|
||||
testJob2 := &config.ScrapeConfig{
|
||||
JobName: "test_job2",
|
||||
ScrapeInterval: config.Duration(1 * time.Minute),
|
||||
TargetGroups: []*config.TargetGroup{
|
||||
{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "example.org:8080"},
|
||||
{clientmodel.AddressLabel: "example.com:8081"},
|
||||
},
|
||||
Labels: clientmodel.LabelSet{
|
||||
"foo": "bar",
|
||||
"boom": "box",
|
||||
},
|
||||
},
|
||||
{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test.com:1234"},
|
||||
},
|
||||
},
|
||||
{
|
||||
Targets: []clientmodel.LabelSet{
|
||||
{clientmodel.AddressLabel: "test.com:1235"},
|
||||
},
|
||||
Labels: clientmodel.LabelSet{"instance": "fixed"},
|
||||
},
|
||||
},
|
||||
RelabelConfigs: []*config.RelabelConfig{
|
||||
{
|
||||
SourceLabels: clientmodel.LabelNames{clientmodel.AddressLabel},
|
||||
Regex: &config.Regexp{*regexp.MustCompile(`^test\.(.*?):(.*)`)},
|
||||
Replacement: "foo.${1}:${2}",
|
||||
TargetLabel: clientmodel.AddressLabel,
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
{
|
||||
// Add a new label for example.* targets.
|
||||
SourceLabels: clientmodel.LabelNames{clientmodel.AddressLabel, "boom", "foo"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile("^example.*?-b([a-z-]+)r$")},
|
||||
TargetLabel: "new",
|
||||
Replacement: "$1",
|
||||
Separator: "-",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
{
|
||||
// Drop an existing label.
|
||||
SourceLabels: clientmodel.LabelNames{"boom"},
|
||||
Regex: &config.Regexp{*regexp.MustCompile(".*")},
|
||||
TargetLabel: "boom",
|
||||
Replacement: "",
|
||||
Action: config.RelabelReplace,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
sequence := []struct {
|
||||
scrapeConfigs []*config.ScrapeConfig
|
||||
expected map[string][]clientmodel.LabelSet
|
||||
}{
|
||||
{
|
||||
scrapeConfigs: []*config.ScrapeConfig{testJob1},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"test_job1:static:0:0": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.org:80", "testParam": "paramValue"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.com:80", "testParam": "paramValue"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
scrapeConfigs: []*config.ScrapeConfig{testJob1},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"test_job1:static:0:0": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.org:80", "testParam": "paramValue"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.com:80", "testParam": "paramValue"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
scrapeConfigs: []*config.ScrapeConfig{testJob1, testJob2},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"test_job1:static:0:0": {
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.org:80", "testParam": "paramValue"},
|
||||
{clientmodel.JobLabel: "test_job1", clientmodel.InstanceLabel: "example.com:80", "testParam": "paramValue"},
|
||||
},
|
||||
"test_job2:static:0:0": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "example.org:8080", "foo": "bar", "new": "ox-ba"},
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "example.com:8081", "foo": "bar", "new": "ox-ba"},
|
||||
},
|
||||
"test_job2:static:0:1": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "foo.com:1234"},
|
||||
},
|
||||
"test_job2:static:0:2": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "fixed"},
|
||||
},
|
||||
},
|
||||
}, {
|
||||
scrapeConfigs: []*config.ScrapeConfig{},
|
||||
expected: map[string][]clientmodel.LabelSet{},
|
||||
}, {
|
||||
scrapeConfigs: []*config.ScrapeConfig{testJob2},
|
||||
expected: map[string][]clientmodel.LabelSet{
|
||||
"test_job2:static:0:0": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "example.org:8080", "foo": "bar", "new": "ox-ba"},
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "example.com:8081", "foo": "bar", "new": "ox-ba"},
|
||||
},
|
||||
"test_job2:static:0:1": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "foo.com:1234"},
|
||||
},
|
||||
"test_job2:static:0:2": {
|
||||
{clientmodel.JobLabel: "test_job2", clientmodel.InstanceLabel: "fixed"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
conf := &config.Config{}
|
||||
*conf = config.DefaultConfig
|
||||
|
||||
targetManager := NewTargetManager(nopAppender{})
|
||||
targetManager.ApplyConfig(conf)
|
||||
|
||||
targetManager.Run()
|
||||
defer targetManager.Stop()
|
||||
|
||||
for i, step := range sequence {
|
||||
conf.ScrapeConfigs = step.scrapeConfigs
|
||||
targetManager.ApplyConfig(conf)
|
||||
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
|
||||
if len(targetManager.targets) != len(step.expected) {
|
||||
t.Fatalf("step %d: sources mismatch: expected %v, got %v", i, step.expected, targetManager.targets)
|
||||
}
|
||||
|
||||
for source, actTargets := range targetManager.targets {
|
||||
expTargets, ok := step.expected[source]
|
||||
if !ok {
|
||||
t.Fatalf("step %d: unexpected source %q: %v", i, source, actTargets)
|
||||
}
|
||||
for _, expt := range expTargets {
|
||||
found := false
|
||||
for _, actt := range actTargets {
|
||||
if reflect.DeepEqual(expt, actt.BaseLabels()) {
|
||||
found = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !found {
|
||||
t.Errorf("step %d: expected target %v for %q not found in actual targets", i, expt, source)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
1
prom/retrieval/testdata/bearertoken.txt
vendored
Normal file
1
prom/retrieval/testdata/bearertoken.txt
vendored
Normal file
@ -0,0 +1 @@
|
||||
12345
|
22
prom/retrieval/testdata/ca.cer
vendored
Normal file
22
prom/retrieval/testdata/ca.cer
vendored
Normal file
@ -0,0 +1,22 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDkTCCAnmgAwIBAgIJAJNsnimNN3tmMA0GCSqGSIb3DQEBCwUAMF8xCzAJBgNV
|
||||
BAYTAlhYMRUwEwYDVQQHDAxEZWZhdWx0IENpdHkxHDAaBgNVBAoME0RlZmF1bHQg
|
||||
Q29tcGFueSBMdGQxGzAZBgNVBAMMElByb21ldGhldXMgVGVzdCBDQTAeFw0xNTA4
|
||||
MDQxNDA5MjFaFw0yNTA4MDExNDA5MjFaMF8xCzAJBgNVBAYTAlhYMRUwEwYDVQQH
|
||||
DAxEZWZhdWx0IENpdHkxHDAaBgNVBAoME0RlZmF1bHQgQ29tcGFueSBMdGQxGzAZ
|
||||
BgNVBAMMElByb21ldGhldXMgVGVzdCBDQTCCASIwDQYJKoZIhvcNAQEBBQADggEP
|
||||
ADCCAQoCggEBAOlSBU3yWpUELbhzizznR0hnAL7dbEHzfEtEc6N3PoSvMNcqrUVq
|
||||
t4kjBRWzqkZ5uJVkzBPERKEBoOI9pWcrqtMTBkMzHJY2Ep7GHTab10e9KC2IFQT6
|
||||
FKP/jCYixaIVx3azEfajRJooD8r79FGoagWUfHdHyCFWJb/iLt8z8+S91kelSRMS
|
||||
yB9M1ypWomzBz1UFXZp1oiNO5o7/dgXW4MgLUfC2obJ9j5xqpc6GkhWMW4ZFwEr/
|
||||
VLjuzxG9B8tLfQuhnXKGn1W8+WzZVWCWMD/sLfZfmjKaWlwcXzL51g8E+IEIBJqV
|
||||
w51aMI6lDkcvAM7gLq1auLZMVXyKWSKw7XMCAwEAAaNQME4wHQYDVR0OBBYEFMz1
|
||||
BZnlqxJp2HiJSjHK8IsLrWYbMB8GA1UdIwQYMBaAFMz1BZnlqxJp2HiJSjHK8IsL
|
||||
rWYbMAwGA1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAI2iA3w3TK5J15Pu
|
||||
e4fPFB4jxQqsbUwuyXbCCv/jKLeFNCD4BjM181WZEYjPMumeTBVzU3aF45LWQIG1
|
||||
0DJcrCL4mjMz9qgAoGqA7aDDXiJGbukMgYYsn7vrnVmrZH8T3E8ySlltr7+W578k
|
||||
pJ5FxnbCroQwn0zLyVB3sFbS8E3vpBr3L8oy8PwPHhIScexcNVc3V6/m4vTZsXTH
|
||||
U+vUm1XhDgpDcFMTg2QQiJbfpOYUkwIgnRDAT7t282t2KQWtnlqc3zwPQ1F/6Cpx
|
||||
j19JeNsaF1DArkD7YlyKj/GhZLtHwFHG5cxznH0mLDJTW7bQvqqh2iQTeXmBk1lU
|
||||
mM5lH/s=
|
||||
-----END CERTIFICATE-----
|
27
prom/retrieval/testdata/ca.key
vendored
Normal file
27
prom/retrieval/testdata/ca.key
vendored
Normal file
@ -0,0 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpgIBAAKCAQEA6VIFTfJalQQtuHOLPOdHSGcAvt1sQfN8S0Rzo3c+hK8w1yqt
|
||||
RWq3iSMFFbOqRnm4lWTME8REoQGg4j2lZyuq0xMGQzMcljYSnsYdNpvXR70oLYgV
|
||||
BPoUo/+MJiLFohXHdrMR9qNEmigPyvv0UahqBZR8d0fIIVYlv+Iu3zPz5L3WR6VJ
|
||||
ExLIH0zXKlaibMHPVQVdmnWiI07mjv92BdbgyAtR8Lahsn2PnGqlzoaSFYxbhkXA
|
||||
Sv9UuO7PEb0Hy0t9C6GdcoafVbz5bNlVYJYwP+wt9l+aMppaXBxfMvnWDwT4gQgE
|
||||
mpXDnVowjqUORy8AzuAurVq4tkxVfIpZIrDtcwIDAQABAoIBAQCcVDd3pYWpyLX1
|
||||
m31UnkX1rgYi3Gs3uTOznra4dSIvds6LrG2SUFGPEibLBql1NQNHHdVa/StakaPB
|
||||
UrqraOe5K0sL5Ygm4S4Ssf1K5JoW2Be+gipLPmBsDcJSnwO6eUs/LfZAQd6qR2Nl
|
||||
hvGJcQUwne/TYAYox/bdHWh4Zu/odz4NrZKZLbnXkdLLDEhZbjA0HpwJZ7NpMcB7
|
||||
Z6NayOm5dAZncfqBjY+3GNL0VjvDjwwYbESM8GkAbojMgcpODGk0h9arRWCP2RqT
|
||||
SVgmiFI2mVT7sW1XLdVXmyCL2jzak7sktpbLbVgngwOrBmLO/m4NBftzcZrgvxj3
|
||||
YakCPH/hAoGBAP1v85pIxqWr5dFdRlOW0MG35ifL+YXpavcs233jGDHYNZefrR5q
|
||||
Mw8eA20zwj41OdryqGh58nLYm3zYM0vPFrRJrzWYQfcWDmQELAylr9z9vsMj8gRq
|
||||
IZQD6wzFmLi1PN2QDmovF+2y/CLAq03XK6FQlNsVQxubfjh4hcX5+nXDAoGBAOut
|
||||
/pQaIBbIhaI8y3KjpizU24jxIkV8R/q1yE5V01YCl2OC5hEd4iZP14YLDRXLSHKT
|
||||
e/dyJ/OEyTIzUeDg0ZF3ao9ugbWuASgrnrrdPEooi7C9n9PeaLFTK5oVZoVP2A7E
|
||||
BwhSFW3VdEzQkdJczVE2jOY6JdBKMndjoDQnhT6RAoGBAL4WMO1gdnYeZ0JQJoZd
|
||||
kPgrOZpR2DaDa3I3F+3k3enM0+2EmzE70E4fYcyPTLqh62H4LS4ngRx4sK7D7j2G
|
||||
9u2EcsDNEXUE+wgzROK7hxtGysTMeiKrg8Hj6nFq53Bqp1s7SESGS/lCDPD398Rr
|
||||
hdL5gJyN5waW6uXqJ9Pk+eFHAoGBAKV/YGcV1XTKSPT9ZgxRmM6ghq0qT1umA1Gt
|
||||
t0QzBp2+Yhqx/+cDKhynMnxhZEXqoyw6HvJLSny5wSMsYJHeratNxRmFizZOQ2e3
|
||||
AdbMppqY0EdDUWnRI4lqExM3de+let4bj6irI3smSm3qhIvJOTCPcu/04zrZ74hh
|
||||
AE2/dtTRAoGBAO6bENEqLgxZOvX5NnbytTuuoEnbceUPiIvc6S/nWJPEoGXVN2EJ
|
||||
a3OaIOQmknE6bjXIWrHTaXJhwejvPUz9DVa4GxU5aJhs4gpocVGf+owQFvk4nJO8
|
||||
JL+QVVdXp3XdrXIGyvXJfy0fXXgJg5czrnDHjSTE8/2POtyuZ6VyBtQc
|
||||
-----END RSA PRIVATE KEY-----
|
25
prom/retrieval/testdata/client.cer
vendored
Normal file
25
prom/retrieval/testdata/client.cer
vendored
Normal file
@ -0,0 +1,25 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIERjCCAy6gAwIBAgIBZDANBgkqhkiG9w0BAQUFADBfMQswCQYDVQQGEwJYWDEV
|
||||
MBMGA1UEBwwMRGVmYXVsdCBDaXR5MRwwGgYDVQQKDBNEZWZhdWx0IENvbXBhbnkg
|
||||
THRkMRswGQYDVQQDDBJQcm9tZXRoZXVzIFRlc3QgQ0EwHhcNMTUwODA0MTQ0MTE2
|
||||
WhcNNDIxMjIwMTQ0MTE2WjBVMQswCQYDVQQGEwJYWDEVMBMGA1UEBwwMRGVmYXVs
|
||||
dCBDaXR5MRwwGgYDVQQKDBNEZWZhdWx0IENvbXBhbnkgTHRkMREwDwYDVQQDDAh0
|
||||
ZXN0dXNlcjCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAOKBBXx35X9+
|
||||
BLGqY/cC2+lQYZzn13Z8ZEDrUKpv5n91QA0B/YZE3gDSnk2yry8dxmp1NJtXm8Wr
|
||||
rIQSBnsTGOKwyIwR1gcggUYPD9fCyy7T7y7YbzBG8drEcxiK/YIWyio0fpRCfT9b
|
||||
2+fOEeY+0+tgFV++XjbXVzXRCBMmsZ22cOm4t2t7GHKBZhYoUoPgKjDn+4t/rr0r
|
||||
1od6yVOocYCo6RruQHsWPHj6QlU8VGutkD7PpvLS+w2l/6JqmZDHlY6o6pDidC8a
|
||||
kp8i/t3pNBlexk6st/8YZ5S9j6LjqC6bUnerUZB40b6L8OXXwWS3S5y6t07A1QIn
|
||||
Pv2DZKGbn8Uuj7RvS5OAZdDn1P+M5aVlRLoYbdTHJILrLg+bxyDIokqONbLgj78A
|
||||
FT6a013eJAZJBkeoaN7Djbf/d5FjRDadH2bX0Uur3APh4cbv+0Fo13CPPSckA9EU
|
||||
o42qBmKLWys858D8vRKyS/mq/IeRL0AIwKuaEIJtPtiwCTnk6PvFfQvO80z/Eyq+
|
||||
uvRBoZbrWHb+3GR8rNzu8Gc1UbTC+jnGYtbQhxx1/7nae52XGRpplnwPO9cb+px2
|
||||
Zf802h+lP3SMY/XS+nyTAp/jcy/jOAwrZKY4rgz+5ZmKCI61NZ0iovaK7Jqo9qTM
|
||||
iSjykZCamFhm4pg8itECD5FhnUetJ6axAgMBAAGjFzAVMBMGA1UdJQQMMAoGCCsG
|
||||
AQUFBwMCMA0GCSqGSIb3DQEBBQUAA4IBAQDEQyFfY9WAkdnzb+vIlICgfkydceYx
|
||||
KVJZ2WRMvrn2ZoRoSaK3CfGlz4nrCOgDjQxfX8OpKzudr/ghuBQCbDHHzxRrOen5
|
||||
0Zig9Q+pxTZNrtds/SwX2dHJ7PVEwGxXXaKl8S19bNEdO0syFrRJU6I50ZbeEkJe
|
||||
RI9IEFvBHcuG/GnEfqWj2ozI/+VhIOb4cTItg67ClmIPe8lteT2wj+/aydF9PKgF
|
||||
QhooCe/G1nok1uiaGjo1HzFEn4HzI3s4mrolc8PpBBVsS+HckCOrHpRPWnYuCFEm
|
||||
0yzS6tGaMrnITywwB2/uJ2aBAZIx2Go1zFhPf0YvFJc3e2x8cAuqBRLu
|
||||
-----END CERTIFICATE-----
|
51
prom/retrieval/testdata/client.key
vendored
Normal file
51
prom/retrieval/testdata/client.key
vendored
Normal file
@ -0,0 +1,51 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIJKAIBAAKCAgEA4oEFfHflf34Esapj9wLb6VBhnOfXdnxkQOtQqm/mf3VADQH9
|
||||
hkTeANKeTbKvLx3GanU0m1ebxaushBIGexMY4rDIjBHWByCBRg8P18LLLtPvLthv
|
||||
MEbx2sRzGIr9ghbKKjR+lEJ9P1vb584R5j7T62AVX75eNtdXNdEIEyaxnbZw6bi3
|
||||
a3sYcoFmFihSg+AqMOf7i3+uvSvWh3rJU6hxgKjpGu5AexY8ePpCVTxUa62QPs+m
|
||||
8tL7DaX/omqZkMeVjqjqkOJ0LxqSnyL+3ek0GV7GTqy3/xhnlL2PouOoLptSd6tR
|
||||
kHjRvovw5dfBZLdLnLq3TsDVAic+/YNkoZufxS6PtG9Lk4Bl0OfU/4zlpWVEuhht
|
||||
1MckgusuD5vHIMiiSo41suCPvwAVPprTXd4kBkkGR6ho3sONt/93kWNENp0fZtfR
|
||||
S6vcA+Hhxu/7QWjXcI89JyQD0RSjjaoGYotbKzznwPy9ErJL+ar8h5EvQAjAq5oQ
|
||||
gm0+2LAJOeTo+8V9C87zTP8TKr669EGhlutYdv7cZHys3O7wZzVRtML6OcZi1tCH
|
||||
HHX/udp7nZcZGmmWfA871xv6nHZl/zTaH6U/dIxj9dL6fJMCn+NzL+M4DCtkpjiu
|
||||
DP7lmYoIjrU1nSKi9orsmqj2pMyJKPKRkJqYWGbimDyK0QIPkWGdR60nprECAwEA
|
||||
AQKCAgEA18az1ERf9Fm33Q0GmE039IdnxlMy9qQ/2XyS5xsdCXVIZFvuClhW6Y+7
|
||||
0ScVLpx95fLr/8SxF9mYymRlmh+ySFrDYnSnYTi9DmHQ5OmkKGMr64OyQNqFErSt
|
||||
NMdMA/7z7sr9fv3sVUyMLMMqWB6oQgXRttki5bm1UgZlW+EzuZwQ6wbWbWTiAEt3
|
||||
VkppeUo2x0poXxdu/rXhdEUrwC+qmTfQgaBQ+zFOwK0gPhTwE3hP/xZQ4+jL08+8
|
||||
vRwyWTNZLYOLmiSxLCJzZXiwNfUwda7M2iw+SJ0WKCOBz1pzYJsFMA2b8Ta4EX89
|
||||
Kailiu328UMK19Jp2dhLcLUYS8B2rVVAK5b/O6iKV8UpKTriXDiCKSpcugpsQ1ML
|
||||
zq/6vR0SQXD+/W0MesGaNa33votBXJSsf9kZnYJw43n+W4Z/XFUE5pyNM/+TGAqw
|
||||
yuF4FX2sJL1uP5VMOh2HdthTr+/ewx/Trn9/re0p54z83plVlp4qbcORLiQ2uDf6
|
||||
ZZ0/gHzNTp4Fzz81ZvHLm9smpe8cLvojrKLvCl0hv5zAf3QtsajpTN9uM7AsshV1
|
||||
QVZSuAxb5n9bcij5F2za1/dd7WLlvsSzgNJ4Td/gEDI8qepB0+7PGlJ17sMg0nWP
|
||||
nFxUfGIsCF1KOoPwLyaNHHrRGjJigFUufqkbmSWkOzgC6pZVUXECggEBAP81To16
|
||||
O5BlGDahcQkjKkqUwUtkhjE9/KQBh3zHqxsitI8f0U7eL3Ge1qhbgEgvHwHOjWSV
|
||||
pcG9atE55b7qlqqGQboiO1jfyLfIVLfamj0fHLinO/pV/wcBNy6Hz4rP7DNJDCMz
|
||||
0agz/Ys3VXrZIk5sO0sUBYMBxho1x0n65Z06iK1SwD/x4Xg3/Psyx+ujEEkSsv5I
|
||||
Gg7aOTHLRSIPUx/OK+4M3sp58PeMGfEYNYxNiEoMiUQgu/srKRjs+pUKXCkEraNW
|
||||
8s/ODYJ7iso6Z1z4NxfBH+hh+UrxTffh7t0Sz5gdUwUnBNb2I4EdeCcCTOnWYkut
|
||||
/GKW8oHD7f9VDS0CggEBAOM06rrp9rSsl6UhTu8LS5bjBeyUxab4HLZKP5YBitQO
|
||||
ltcPS05MxQ3UQ1BAMDRjXE2nrKlWMOAybrffEXBi4U1jYt7CiuCwwsPyaYNWT5qO
|
||||
Iwdjebkeq3+Mh8c48swhOwRLWSGg6mtRoR/c5cthYU62+s2zdxc/yhVTQ0WNFabT
|
||||
23PYtjjW41WuR6K7Dhrdcw0MwIs1arZHTsDdU6Hln9raTSNwlHMBWVz/tzuwLieQ
|
||||
WEUXvsQvPtgPyohmDd0ueXiuS2FiYaXKFIMFj5/JyyJc1OCr1vIQN8mMcUjNbk2I
|
||||
VaeeSPawgKIiYARhbjJtjwjY6D59gOZrNGYASQOTGhUCggEAJPOB8SgekbShgd90
|
||||
L1+BExVgu1rNtzmDZ/e0t1Ntqdsni4WO172B3xChgfTlqQ3xjmBqxoKIYnnbinm4
|
||||
kyECOaSAxcOJFkAonruJ0Kj9JhZoITBNldx3tXruk3UkjrO2PmK4OCybkaAdeNfF
|
||||
L6lat0Iif6dheOt71HWu6j5CmrZL7dSKc3fBLpfksDZVDgApLntfoUOtSjM8jsIg
|
||||
u2K+pV9Dqw7//w8S3bTSWL8pmavsLNSN12hp7177b1l4mrXKTEIaJglD1OS/vgHH
|
||||
QaqdJq/lwjG7PflZkAlKQbbbz/SWTC8Kwzc4EyvGTj6HFBbYLg9VYiHJ5jh22mUV
|
||||
A6A77QKCAQAM6DWpdp8QNnnK5LCCPecGZFEy1mTADno7FM6169KCJ24EO5cwlIXh
|
||||
Ojy0s2DJqRdWRf82A3J1WggWI/Luqn9YERxNwUl4aDI4RW4fCuksw4RT6B/DF23w
|
||||
qgAQnjiUxhJ/NPSUR3rpq9J2Z+sZ+ac4fIaU5uwOAw6s1XUN32zqdECUPSxk4Dg7
|
||||
5tGk+fFcL1ZY2G+buOYeAsEDjc8xdET3fs1BBSU5v0rfUJuNJX4Ju1Z4Xlf09yYf
|
||||
yg3cX8fL19cItwYLOzaG34r4wnkdP65tfk6NkNV+HNO+fF73Hsx0VRlgk0pb0T0N
|
||||
eNxxg0NqU/T7MK9I1YJcFJz+ame7b0DdAoIBAFw3Sf9LbVVNh8ef4OqjBZR8RCYq
|
||||
4HeG0FPYvMLzUtFi7j4uBfiL4+pNpSFvecSuLRKE8Pr5dPRJNPNgJud5gvuykBZX
|
||||
Q9ktQJTAPZK8Q5neLeXfAdoF3szJuEZbDdGSps4JFokVIX+h3c+uFRD9QMSh+bz1
|
||||
nEXCdYvmTs+bsTL+l7cbXq2iIKk1QnEcL+cRYr3VjP5xxZ/hGnuYqe9wmyo2MVkS
|
||||
NVUmCifIvE34TO072HH49gVPrhj9qIZsfBh4LBpl75eKwXTXx+HFqHhP8OfzuK6U
|
||||
v/JQn9JUGGzkmoMazQ9o5D5h/o0t/OGOPnQeqWL4BIPXdHv/dua6jLnAoU8=
|
||||
-----END RSA PRIVATE KEY-----
|
20
prom/retrieval/testdata/server.cer
vendored
Normal file
20
prom/retrieval/testdata/server.cer
vendored
Normal file
@ -0,0 +1,20 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDSzCCAjOgAwIBAgIJAPn0lI/95RQVMA0GCSqGSIb3DQEBBQUAMF8xCzAJBgNV
|
||||
BAYTAlhYMRUwEwYDVQQHDAxEZWZhdWx0IENpdHkxHDAaBgNVBAoME0RlZmF1bHQg
|
||||
Q29tcGFueSBMdGQxGzAZBgNVBAMMElByb21ldGhldXMgVGVzdCBDQTAeFw0xNTA4
|
||||
MDQxNDE5MjRaFw00MjEyMjAxNDE5MjRaMFYxCzAJBgNVBAYTAlhYMRUwEwYDVQQH
|
||||
DAxEZWZhdWx0IENpdHkxHDAaBgNVBAoME0RlZmF1bHQgQ29tcGFueSBMdGQxEjAQ
|
||||
BgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB
|
||||
AMQhH0walZlA+Gy5ZB3YzzxZta7mhTX3P+yBeQ6G6yrei4H7gv+MTCJj5qUBc+BS
|
||||
cta8loKKUQWjoppjyh4tz8awkTD5sEyedE7/G3DS7mLgmx0PslwqrkXFBQhm/C2f
|
||||
aZfSO69TZ8uu1dgCmmGe9K2XqPnR6fu9egtLpK8RT0s/Cx04bFnaPS0ecyj+3q7A
|
||||
xzDsH84Z1KPo4LHgqNWlHqFsQPqH+7W9ajhF6lnO4ArEDJ3KuLDlgrENzCsDabls
|
||||
0U2XsccBJzP+Ls+iQwMfKpx2ISQDHqniopSICw+sPufiAv+OGnnG6rGGWQjUstqf
|
||||
w4DnU4DZvkrcEWoGa6fq26kCAwEAAaMTMBEwDwYDVR0RBAgwBocEfwAAATANBgkq
|
||||
hkiG9w0BAQUFAAOCAQEAVPs8IZffawWuRqbXJSvFz7a1q95febWQFjvvMe8ZJeCZ
|
||||
y1k9laQ5ZLHYuQ6NUWn09UbQNtK3fCLF4sJx5PCPCp1vZWx4nJs8N5mNyqdQ1Zfk
|
||||
oyoYTOR2izNcIj6ZUFRoOR/7B9hl2JouCXrbExr96oO13xIfsdslScINz1X68oyW
|
||||
KjU0yUrY+lWG1zEkUGXti9K6ujtXa7YY2n3nK/CvIqny5nVToYUgEMpjUR9S+KgN
|
||||
JUtawY3VQKyp6ZXlHqa0ihsuvY9Hrlh14h0AsZchPAHUtDFv2nEQob/Kf1XynKw6
|
||||
itVKcj/UFpkhsnc/19aP1gWje76fejXl0tzyPXDXFg==
|
||||
-----END CERTIFICATE-----
|
27
prom/retrieval/testdata/server.key
vendored
Normal file
27
prom/retrieval/testdata/server.key
vendored
Normal file
@ -0,0 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpAIBAAKCAQEAxCEfTBqVmUD4bLlkHdjPPFm1ruaFNfc/7IF5DobrKt6LgfuC
|
||||
/4xMImPmpQFz4FJy1ryWgopRBaOimmPKHi3PxrCRMPmwTJ50Tv8bcNLuYuCbHQ+y
|
||||
XCquRcUFCGb8LZ9pl9I7r1Nny67V2AKaYZ70rZeo+dHp+716C0ukrxFPSz8LHThs
|
||||
Wdo9LR5zKP7ersDHMOwfzhnUo+jgseCo1aUeoWxA+of7tb1qOEXqWc7gCsQMncq4
|
||||
sOWCsQ3MKwNpuWzRTZexxwEnM/4uz6JDAx8qnHYhJAMeqeKilIgLD6w+5+IC/44a
|
||||
ecbqsYZZCNSy2p/DgOdTgNm+StwRagZrp+rbqQIDAQABAoIBACeOjqNo0TdhtTko
|
||||
gxrJ+bIwXcZy0/c4cPogeuwFJjU1QWnr8lXcVBazk3dAPcDGoEbTLoARqZm7kTYW
|
||||
XlOL5dYrEn2QPpCVfNvZ9AzjXhUvO9m2qsCQEyobPJKfQslo14E5c7Q+3DZmgtbY
|
||||
X47E4pCIgBoyzkBpzM2uaf6tPRLtv8QcLklcf7lP5rd0Zypc325RR6+J5nxfCoFp
|
||||
fD3sj7t/lJLS8Xb6m4/YFjsVJ2qEAelZ086v8unMBEj324Vv/VqrkPFtFNJKI+Az
|
||||
Pd9xFDBdsKijBn1Yam9/dj7CiyZYKaVZ9p/w7Oqkpbrt8J8S8OtNHZ4fz9FJgRu9
|
||||
uu+VTikCgYEA5ZkDmozDseA/c9JTUGAiPfAt5OrnqlKQNzp2m19GKh+Mlwg4k6O5
|
||||
uE+0vaQEfc0cX3o8qntWNsb63XC9h6oHewrdyVFMZNS4nzzmKEvGWt9ON6qfQDUs
|
||||
1cgZ0Y/uKydDX/3hk/hnJbeRW429rk0/GTuSHHilBzhE0uXJ11xPG48CgYEA2q7a
|
||||
yqTdqPmZFIAYT9ny099PhnGYE6cJljTUMX9Xhk4POqcigcq9kvNNsly2O1t0Eq0H
|
||||
2tYo91xTCZc3Cb0N+Vx3meLIljnzhEtwzU9w6W5VGJHWiqovjGwtCdm/W28OlMzY
|
||||
zM+0gVCJzZLhL0vOwBLwGUJvjgfpvgIb/W+C2UcCgYB5TJ3ayQOath7P0g6yKBfv
|
||||
ITUd+/zovzXx97Ex5OPs3T4pjO5XEejMt0+F4WF+FR8oUiw65W5nAjkHRMjdI7dQ
|
||||
Ci2ibpEttDTV7Bass1vYJqHsRvhbs7w8NbtuO9xYcCXoUPkcc+AKzTC+beQIckcj
|
||||
zZUj9Zk6dz/lLAG3Bc3FgQKBgQC+MmZI6auAU9Y4ZlC+4qi4bfkUzaefMCC+a6RC
|
||||
iKbvQOUt9j+k81h+fu6MuuYkKh6CP8wdITbwLXRrWwGbjrqgrzO2u/AJ+M07uwGZ
|
||||
EAb8f+GzROR8JhjE4TEq6B/uvmDIOoI1YFF2Rz4TdjQ0lpJzrAT3czjjJy68+8is
|
||||
XFhJ8QKBgQCMPpB7taMLQzuilEGabL6Xas9UxryiGoBHk4Umb107GVWgwXxWT6fk
|
||||
YSlvbMQHCgVeaJe374Bghyw33Z3WilWM1fCWya/CxXlw9wakjQHiqFCIOCxdgosX
|
||||
Sr35bRFWJMnHXD+jD0Vr8WrtbGzFSZb3ZrjT6WhWRIGCHcaMANN9ew==
|
||||
-----END RSA PRIVATE KEY-----
|
Reference in New Issue
Block a user