Browse Source

Merge pull request #8849 from gyuho/promhttp

*: deprecate prometheus.Handler, upgrade Prometheus dependencies
Gyu-Ho Lee 8 years ago
parent
commit
0dbcd7c1a7
46 changed files with 2920 additions and 902 deletions
  1. 1 0
      .words
  2. 4 4
      clientv3/example_metrics_test.go
  3. 3 3
      clientv3/integration/metrics_test.go
  4. 23 56
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client.go
  5. 165 0
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_metrics.go
  6. 13 78
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_reporter.go
  7. 29 55
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server.go
  8. 208 0
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_metrics.go
  9. 13 124
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_reporter.go
  10. 23 0
      cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/util.go
  11. 44 27
      cmd/vendor/github.com/prometheus/client_golang/prometheus/counter.go
  12. 10 26
      cmd/vendor/github.com/prometheus/client_golang/prometheus/desc.go
  13. 37 32
      cmd/vendor/github.com/prometheus/client_golang/prometheus/doc.go
  14. 51 18
      cmd/vendor/github.com/prometheus/client_golang/prometheus/gauge.go
  15. 36 15
      cmd/vendor/github.com/prometheus/client_golang/prometheus/go_collector.go
  16. 51 22
      cmd/vendor/github.com/prometheus/client_golang/prometheus/histogram.go
  17. 72 38
      cmd/vendor/github.com/prometheus/client_golang/prometheus/http.go
  18. 57 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/labels.go
  19. 50 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/observer.go
  20. 51 53
      cmd/vendor/github.com/prometheus/client_golang/prometheus/process_collector.go
  21. 199 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator.go
  22. 181 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator_1_8.go
  23. 44 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator_pre_1_8.go
  24. 204 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/http.go
  25. 98 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client.go
  26. 144 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client_1_8.go
  27. 440 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_server.go
  28. 12 56
      cmd/vendor/github.com/prometheus/client_golang/prometheus/registry.go
  29. 72 29
      cmd/vendor/github.com/prometheus/client_golang/prometheus/summary.go
  30. 51 0
      cmd/vendor/github.com/prometheus/client_golang/prometheus/timer.go
  31. 3 99
      cmd/vendor/github.com/prometheus/client_golang/prometheus/untyped.go
  32. 8 6
      cmd/vendor/github.com/prometheus/client_golang/prometheus/value.go
  33. 51 92
      cmd/vendor/github.com/prometheus/client_golang/prometheus/vec.go
  34. 4 0
      cmd/vendor/github.com/prometheus/common/expfmt/text_parse.go
  35. 13 1
      cmd/vendor/github.com/prometheus/common/model/time.go
  36. 34 12
      cmd/vendor/github.com/prometheus/procfs/ipvs.go
  37. 8 4
      cmd/vendor/github.com/prometheus/procfs/mountstats.go
  38. 19 19
      cmd/vendor/github.com/prometheus/procfs/proc_limits.go
  39. 178 15
      cmd/vendor/github.com/prometheus/procfs/stat.go
  40. 187 0
      cmd/vendor/github.com/prometheus/procfs/xfrm.go
  41. 0 2
      cmd/vendor/github.com/prometheus/procfs/xfs/parse.go
  42. 5 0
      cmd/vendor/github.com/prometheus/procfs/xfs/xfs.go
  43. 3 3
      etcdserver/api/etcdhttp/metrics.go
  44. 7 6
      glide.lock
  45. 12 5
      glide.yaml
  46. 2 2
      tools/functional-tester/etcd-tester/main.go

+ 1 - 0
.words

@@ -27,6 +27,7 @@ localhost
 mutex
 prefetching
 protobuf
+prometheus
 repin
 serializable
 teardown

+ 4 - 4
clientv3/example_metrics_test.go

@@ -26,7 +26,7 @@ import (
 	"github.com/coreos/etcd/clientv3"
 
 	grpcprom "github.com/grpc-ecosystem/go-grpc-prometheus"
-	"github.com/prometheus/client_golang/prometheus"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
 	"google.golang.org/grpc"
 )
 
@@ -46,7 +46,7 @@ func ExampleClient_metrics() {
 	// get a key so it shows up in the metrics as a range RPC
 	cli.Get(context.TODO(), "test_key")
 
-	// listen for all prometheus metrics
+	// listen for all Prometheus metrics
 	ln, err := net.Listen("tcp", ":0")
 	if err != nil {
 		log.Fatal(err)
@@ -54,14 +54,14 @@ func ExampleClient_metrics() {
 	donec := make(chan struct{})
 	go func() {
 		defer close(donec)
-		http.Serve(ln, prometheus.Handler())
+		http.Serve(ln, promhttp.Handler())
 	}()
 	defer func() {
 		ln.Close()
 		<-donec
 	}()
 
-	// make an http request to fetch all prometheus metrics
+	// make an http request to fetch all Prometheus metrics
 	url := "http://" + ln.Addr().String() + "/metrics"
 	resp, err := http.Get(url)
 	if err != nil {

+ 3 - 3
clientv3/integration/metrics_test.go

@@ -31,7 +31,7 @@ import (
 	"github.com/coreos/etcd/pkg/transport"
 
 	grpcprom "github.com/grpc-ecosystem/go-grpc-prometheus"
-	"github.com/prometheus/client_golang/prometheus"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
 	"google.golang.org/grpc"
 )
 
@@ -44,12 +44,12 @@ func TestV3ClientMetrics(t *testing.T) {
 		err  error
 	)
 
-	// listen for all prometheus metrics
+	// listen for all Prometheus metrics
 	donec := make(chan struct{})
 	go func() {
 		defer close(donec)
 
-		srv := &http.Server{Handler: prometheus.Handler()}
+		srv := &http.Server{Handler: promhttp.Handler()}
 		srv.SetKeepAlivesEnabled(false)
 
 		ln, err = transport.NewUnixListener(addr)

+ 23 - 56
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client.go

@@ -6,67 +6,34 @@
 package grpc_prometheus
 
 import (
-	"io"
-
-	"golang.org/x/net/context"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/codes"
+	prom "github.com/prometheus/client_golang/prometheus"
 )
 
-// UnaryClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Unary RPCs.
-func UnaryClientInterceptor(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
-	monitor := newClientReporter(Unary, method)
-	monitor.SentMessage()
-	err := invoker(ctx, method, req, reply, cc, opts...)
-	if err != nil {
-		monitor.ReceivedMessage()
-	}
-	monitor.Handled(grpc.Code(err))
-	return err
-}
+var (
+	// DefaultClientMetrics is the default instance of ClientMetrics. It is
+	// intended to be used in conjunction the default Prometheus metrics
+	// registry.
+	DefaultClientMetrics = NewClientMetrics()
 
-// StreamServerInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Streaming RPCs.
-func StreamClientInterceptor(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
-	monitor := newClientReporter(clientStreamType(desc), method)
-	clientStream, err := streamer(ctx, desc, cc, method, opts...)
-	if err != nil {
-		monitor.Handled(grpc.Code(err))
-		return nil, err
-	}
-	return &monitoredClientStream{clientStream, monitor}, nil
-}
+	// UnaryClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Unary RPCs.
+	UnaryClientInterceptor = DefaultClientMetrics.UnaryClientInterceptor()
 
-func clientStreamType(desc *grpc.StreamDesc) grpcType {
-	if desc.ClientStreams && !desc.ServerStreams {
-		return ClientStream
-	} else if !desc.ClientStreams && desc.ServerStreams {
-		return ServerStream
-	}
-	return BidiStream
-}
-
-// monitoredClientStream wraps grpc.ClientStream allowing each Sent/Recv of message to increment counters.
-type monitoredClientStream struct {
-	grpc.ClientStream
-	monitor *clientReporter
-}
+	// StreamClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+	StreamClientInterceptor = DefaultClientMetrics.StreamClientInterceptor()
+)
 
-func (s *monitoredClientStream) SendMsg(m interface{}) error {
-	err := s.ClientStream.SendMsg(m)
-	if err == nil {
-		s.monitor.SentMessage()
-	}
-	return err
+func init() {
+	prom.MustRegister(DefaultClientMetrics.clientStartedCounter)
+	prom.MustRegister(DefaultClientMetrics.clientHandledCounter)
+	prom.MustRegister(DefaultClientMetrics.clientStreamMsgReceived)
+	prom.MustRegister(DefaultClientMetrics.clientStreamMsgSent)
 }
 
-func (s *monitoredClientStream) RecvMsg(m interface{}) error {
-	err := s.ClientStream.RecvMsg(m)
-	if err == nil {
-		s.monitor.ReceivedMessage()
-	} else if err == io.EOF {
-		s.monitor.Handled(codes.OK)
-	} else {
-		s.monitor.Handled(grpc.Code(err))
-	}
-	return err
+// EnableClientHandlingTimeHistogram turns on recording of handling time of
+// RPCs. Histogram metrics can be very expensive for Prometheus to retain and
+// query. This function acts on the DefaultClientMetrics variable and the
+// default Prometheus metrics registry.
+func EnableClientHandlingTimeHistogram(opts ...HistogramOption) {
+	DefaultClientMetrics.EnableClientHandlingTimeHistogram(opts...)
+	prom.Register(DefaultClientMetrics.clientHandledHistogram)
 }

+ 165 - 0
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_metrics.go

@@ -0,0 +1,165 @@
+package grpc_prometheus
+
+import (
+	"io"
+
+	prom "github.com/prometheus/client_golang/prometheus"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+)
+
+// ClientMetrics represents a collection of metrics to be registered on a
+// Prometheus metrics registry for a gRPC client.
+type ClientMetrics struct {
+	clientStartedCounter          *prom.CounterVec
+	clientHandledCounter          *prom.CounterVec
+	clientStreamMsgReceived       *prom.CounterVec
+	clientStreamMsgSent           *prom.CounterVec
+	clientHandledHistogramEnabled bool
+	clientHandledHistogramOpts    prom.HistogramOpts
+	clientHandledHistogram        *prom.HistogramVec
+}
+
+// NewClientMetrics returns a ClientMetrics object. Use a new instance of
+// ClientMetrics when not using the default Prometheus metrics registry, for
+// example when wanting to control which metrics are added to a registry as
+// opposed to automatically adding metrics via init functions.
+func NewClientMetrics() *ClientMetrics {
+	return &ClientMetrics{
+		clientStartedCounter: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_client_started_total",
+				Help: "Total number of RPCs started on the client.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientHandledCounter: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_client_handled_total",
+				Help: "Total number of RPCs completed by the client, regardless of success or failure.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"}),
+
+		clientStreamMsgReceived: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_client_msg_received_total",
+				Help: "Total number of RPC stream messages received by the client.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientStreamMsgSent: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_client_msg_sent_total",
+				Help: "Total number of gRPC stream messages sent by the client.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientHandledHistogramEnabled: false,
+		clientHandledHistogramOpts: prom.HistogramOpts{
+			Name:    "grpc_client_handling_seconds",
+			Help:    "Histogram of response latency (seconds) of the gRPC until it is finished by the application.",
+			Buckets: prom.DefBuckets,
+		},
+		clientHandledHistogram: nil,
+	}
+}
+
+// Describe sends the super-set of all possible descriptors of metrics
+// collected by this Collector to the provided channel and returns once
+// the last descriptor has been sent.
+func (m *ClientMetrics) Describe(ch chan<- *prom.Desc) {
+	m.clientStartedCounter.Describe(ch)
+	m.clientHandledCounter.Describe(ch)
+	m.clientStreamMsgReceived.Describe(ch)
+	m.clientStreamMsgSent.Describe(ch)
+	if m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram.Describe(ch)
+	}
+}
+
+// Collect is called by the Prometheus registry when collecting
+// metrics. The implementation sends each collected metric via the
+// provided channel and returns once the last metric has been sent.
+func (m *ClientMetrics) Collect(ch chan<- prom.Metric) {
+	m.clientStartedCounter.Collect(ch)
+	m.clientHandledCounter.Collect(ch)
+	m.clientStreamMsgReceived.Collect(ch)
+	m.clientStreamMsgSent.Collect(ch)
+	if m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram.Collect(ch)
+	}
+}
+
+// EnableClientHandlingTimeHistogram turns on recording of handling time of RPCs.
+// Histogram metrics can be very expensive for Prometheus to retain and query.
+func (m *ClientMetrics) EnableClientHandlingTimeHistogram(opts ...HistogramOption) {
+	for _, o := range opts {
+		o(&m.clientHandledHistogramOpts)
+	}
+	if !m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram = prom.NewHistogramVec(
+			m.clientHandledHistogramOpts,
+			[]string{"grpc_type", "grpc_service", "grpc_method"},
+		)
+	}
+	m.clientHandledHistogramEnabled = true
+}
+
+// UnaryClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Unary RPCs.
+func (m *ClientMetrics) UnaryClientInterceptor() func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		monitor := newClientReporter(m, Unary, method)
+		monitor.SentMessage()
+		err := invoker(ctx, method, req, reply, cc, opts...)
+		if err != nil {
+			monitor.ReceivedMessage()
+		}
+		monitor.Handled(grpc.Code(err))
+		return err
+	}
+}
+
+// StreamServerInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+func (m *ClientMetrics) StreamClientInterceptor() func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+	return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		monitor := newClientReporter(m, clientStreamType(desc), method)
+		clientStream, err := streamer(ctx, desc, cc, method, opts...)
+		if err != nil {
+			monitor.Handled(grpc.Code(err))
+			return nil, err
+		}
+		return &monitoredClientStream{clientStream, monitor}, nil
+	}
+}
+
+func clientStreamType(desc *grpc.StreamDesc) grpcType {
+	if desc.ClientStreams && !desc.ServerStreams {
+		return ClientStream
+	} else if !desc.ClientStreams && desc.ServerStreams {
+		return ServerStream
+	}
+	return BidiStream
+}
+
+// monitoredClientStream wraps grpc.ClientStream allowing each Sent/Recv of message to increment counters.
+type monitoredClientStream struct {
+	grpc.ClientStream
+	monitor *clientReporter
+}
+
+func (s *monitoredClientStream) SendMsg(m interface{}) error {
+	err := s.ClientStream.SendMsg(m)
+	if err == nil {
+		s.monitor.SentMessage()
+	}
+	return err
+}
+
+func (s *monitoredClientStream) RecvMsg(m interface{}) error {
+	err := s.ClientStream.RecvMsg(m)
+	if err == nil {
+		s.monitor.ReceivedMessage()
+	} else if err == io.EOF {
+		s.monitor.Handled(codes.OK)
+	} else {
+		s.monitor.Handled(grpc.Code(err))
+	}
+	return err
+}

+ 13 - 78
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_reporter.go

@@ -7,105 +7,40 @@ import (
 	"time"
 
 	"google.golang.org/grpc/codes"
-
-	prom "github.com/prometheus/client_golang/prometheus"
-)
-
-var (
-	clientStartedCounter = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "client",
-			Name:      "started_total",
-			Help:      "Total number of RPCs started on the client.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	clientHandledCounter = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "client",
-			Name:      "handled_total",
-			Help:      "Total number of RPCs completed by the client, regardless of success or failure.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"})
-
-	clientStreamMsgReceived = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "client",
-			Name:      "msg_received_total",
-			Help:      "Total number of RPC stream messages received by the client.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	clientStreamMsgSent = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "client",
-			Name:      "msg_sent_total",
-			Help:      "Total number of gRPC stream messages sent by the client.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	clientHandledHistogramEnabled = false
-	clientHandledHistogramOpts    = prom.HistogramOpts{
-		Namespace: "grpc",
-		Subsystem: "client",
-		Name:      "handling_seconds",
-		Help:      "Histogram of response latency (seconds) of the gRPC until it is finished by the application.",
-		Buckets:   prom.DefBuckets,
-	}
-	clientHandledHistogram *prom.HistogramVec
 )
 
-func init() {
-	prom.MustRegister(clientStartedCounter)
-	prom.MustRegister(clientHandledCounter)
-	prom.MustRegister(clientStreamMsgReceived)
-	prom.MustRegister(clientStreamMsgSent)
-}
-
-// EnableClientHandlingTimeHistogram turns on recording of handling time of RPCs.
-// Histogram metrics can be very expensive for Prometheus to retain and query.
-func EnableClientHandlingTimeHistogram(opts ...HistogramOption) {
-	for _, o := range opts {
-		o(&clientHandledHistogramOpts)
-	}
-	if !clientHandledHistogramEnabled {
-		clientHandledHistogram = prom.NewHistogramVec(
-			clientHandledHistogramOpts,
-			[]string{"grpc_type", "grpc_service", "grpc_method"},
-		)
-		prom.Register(clientHandledHistogram)
-	}
-	clientHandledHistogramEnabled = true
-}
-
 type clientReporter struct {
+	metrics     *ClientMetrics
 	rpcType     grpcType
 	serviceName string
 	methodName  string
 	startTime   time.Time
 }
 
-func newClientReporter(rpcType grpcType, fullMethod string) *clientReporter {
-	r := &clientReporter{rpcType: rpcType}
-	if clientHandledHistogramEnabled {
+func newClientReporter(m *ClientMetrics, rpcType grpcType, fullMethod string) *clientReporter {
+	r := &clientReporter{
+		metrics: m,
+		rpcType: rpcType,
+	}
+	if r.metrics.clientHandledHistogramEnabled {
 		r.startTime = time.Now()
 	}
 	r.serviceName, r.methodName = splitMethodName(fullMethod)
-	clientStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.clientStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 	return r
 }
 
 func (r *clientReporter) ReceivedMessage() {
-	clientStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.clientStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 }
 
 func (r *clientReporter) SentMessage() {
-	clientStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.clientStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 }
 
 func (r *clientReporter) Handled(code codes.Code) {
-	clientHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
-	if clientHandledHistogramEnabled {
-		clientHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
+	r.metrics.clientHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
+	if r.metrics.clientHandledHistogramEnabled {
+		r.metrics.clientHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
 	}
 }

+ 29 - 55
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server.go

@@ -6,69 +6,43 @@
 package grpc_prometheus
 
 import (
-	"golang.org/x/net/context"
+	prom "github.com/prometheus/client_golang/prometheus"
 	"google.golang.org/grpc"
 )
 
-// PreregisterServices takes a gRPC server and pre-initializes all counters to 0.
-// This allows for easier monitoring in Prometheus (no missing metrics), and should be called *after* all services have
-// been registered with the server.
-func Register(server *grpc.Server) {
-	serviceInfo := server.GetServiceInfo()
-	for serviceName, info := range serviceInfo {
-		for _, mInfo := range info.Methods {
-			preRegisterMethod(serviceName, &mInfo)
-		}
-	}
-}
-
-// UnaryServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Unary RPCs.
-func UnaryServerInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
-	monitor := newServerReporter(Unary, info.FullMethod)
-	monitor.ReceivedMessage()
-	resp, err := handler(ctx, req)
-	monitor.Handled(grpc.Code(err))
-	if err == nil {
-		monitor.SentMessage()
-	}
-	return resp, err
-}
+var (
+	// DefaultServerMetrics is the default instance of ServerMetrics. It is
+	// intended to be used in conjunction the default Prometheus metrics
+	// registry.
+	DefaultServerMetrics = NewServerMetrics()
 
-// StreamServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Streaming RPCs.
-func StreamServerInterceptor(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
-	monitor := newServerReporter(streamRpcType(info), info.FullMethod)
-	err := handler(srv, &monitoredServerStream{ss, monitor})
-	monitor.Handled(grpc.Code(err))
-	return err
-}
+	// UnaryServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Unary RPCs.
+	UnaryServerInterceptor = DefaultServerMetrics.UnaryServerInterceptor()
 
-func streamRpcType(info *grpc.StreamServerInfo) grpcType {
-	if info.IsClientStream && !info.IsServerStream {
-		return ClientStream
-	} else if !info.IsClientStream && info.IsServerStream {
-		return ServerStream
-	}
-	return BidiStream
-}
+	// StreamServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+	StreamServerInterceptor = DefaultServerMetrics.StreamServerInterceptor()
+)
 
-// monitoredStream wraps grpc.ServerStream allowing each Sent/Recv of message to increment counters.
-type monitoredServerStream struct {
-	grpc.ServerStream
-	monitor *serverReporter
+func init() {
+	prom.MustRegister(DefaultServerMetrics.serverStartedCounter)
+	prom.MustRegister(DefaultServerMetrics.serverHandledCounter)
+	prom.MustRegister(DefaultServerMetrics.serverStreamMsgReceived)
+	prom.MustRegister(DefaultServerMetrics.serverStreamMsgSent)
 }
 
-func (s *monitoredServerStream) SendMsg(m interface{}) error {
-	err := s.ServerStream.SendMsg(m)
-	if err == nil {
-		s.monitor.SentMessage()
-	}
-	return err
+// Register takes a gRPC server and pre-initializes all counters to 0. This
+// allows for easier monitoring in Prometheus (no missing metrics), and should
+// be called *after* all services have been registered with the server. This
+// function acts on the DefaultServerMetrics variable.
+func Register(server *grpc.Server) {
+	DefaultServerMetrics.InitializeMetrics(server)
 }
 
-func (s *monitoredServerStream) RecvMsg(m interface{}) error {
-	err := s.ServerStream.RecvMsg(m)
-	if err == nil {
-		s.monitor.ReceivedMessage()
-	}
-	return err
+// EnableHandlingTimeHistogram turns on recording of handling time
+// of RPCs. Histogram metrics can be very expensive for Prometheus
+// to retain and query. This function acts on the DefaultServerMetrics
+// variable and the default Prometheus metrics registry.
+func EnableHandlingTimeHistogram(opts ...HistogramOption) {
+	DefaultServerMetrics.EnableHandlingTimeHistogram(opts...)
+	prom.Register(DefaultServerMetrics.serverHandledHistogram)
 }

+ 208 - 0
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_metrics.go

@@ -0,0 +1,208 @@
+package grpc_prometheus
+
+import (
+	prom "github.com/prometheus/client_golang/prometheus"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+)
+
+// ServerMetrics represents a collection of metrics to be registered on a
+// Prometheus metrics registry for a gRPC server.
+type ServerMetrics struct {
+	serverStartedCounter          *prom.CounterVec
+	serverHandledCounter          *prom.CounterVec
+	serverStreamMsgReceived       *prom.CounterVec
+	serverStreamMsgSent           *prom.CounterVec
+	serverHandledHistogramEnabled bool
+	serverHandledHistogramOpts    prom.HistogramOpts
+	serverHandledHistogram        *prom.HistogramVec
+}
+
+// NewServerMetrics returns a ServerMetrics object. Use a new instance of
+// ServerMetrics when not using the default Prometheus metrics registry, for
+// example when wanting to control which metrics are added to a registry as
+// opposed to automatically adding metrics via init functions.
+func NewServerMetrics() *ServerMetrics {
+	return &ServerMetrics{
+		serverStartedCounter: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_server_started_total",
+				Help: "Total number of RPCs started on the server.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverHandledCounter: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_server_handled_total",
+				Help: "Total number of RPCs completed on the server, regardless of success or failure.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"}),
+		serverStreamMsgReceived: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_server_msg_received_total",
+				Help: "Total number of RPC stream messages received on the server.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverStreamMsgSent: prom.NewCounterVec(
+			prom.CounterOpts{
+				Name: "grpc_server_msg_sent_total",
+				Help: "Total number of gRPC stream messages sent by the server.",
+			}, []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverHandledHistogramEnabled: false,
+		serverHandledHistogramOpts: prom.HistogramOpts{
+			Name:    "grpc_server_handling_seconds",
+			Help:    "Histogram of response latency (seconds) of gRPC that had been application-level handled by the server.",
+			Buckets: prom.DefBuckets,
+		},
+		serverHandledHistogram: nil,
+	}
+}
+
+type HistogramOption func(*prom.HistogramOpts)
+
+// WithHistogramBuckets allows you to specify custom bucket ranges for histograms if EnableHandlingTimeHistogram is on.
+func WithHistogramBuckets(buckets []float64) HistogramOption {
+	return func(o *prom.HistogramOpts) { o.Buckets = buckets }
+}
+
+// EnableHandlingTimeHistogram enables histograms being registered when
+// registering the ServerMetrics on a Prometheus registry. Histograms can be
+// expensive on Prometheus servers. It takes options to configure histogram
+// options such as the defined buckets.
+func (m *ServerMetrics) EnableHandlingTimeHistogram(opts ...HistogramOption) {
+	for _, o := range opts {
+		o(&m.serverHandledHistogramOpts)
+	}
+	if !m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram = prom.NewHistogramVec(
+			m.serverHandledHistogramOpts,
+			[]string{"grpc_type", "grpc_service", "grpc_method"},
+		)
+	}
+	m.serverHandledHistogramEnabled = true
+}
+
+// Describe sends the super-set of all possible descriptors of metrics
+// collected by this Collector to the provided channel and returns once
+// the last descriptor has been sent.
+func (m *ServerMetrics) Describe(ch chan<- *prom.Desc) {
+	m.serverStartedCounter.Describe(ch)
+	m.serverHandledCounter.Describe(ch)
+	m.serverStreamMsgReceived.Describe(ch)
+	m.serverStreamMsgSent.Describe(ch)
+	if m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram.Describe(ch)
+	}
+}
+
+// Collect is called by the Prometheus registry when collecting
+// metrics. The implementation sends each collected metric via the
+// provided channel and returns once the last metric has been sent.
+func (m *ServerMetrics) Collect(ch chan<- prom.Metric) {
+	m.serverStartedCounter.Collect(ch)
+	m.serverHandledCounter.Collect(ch)
+	m.serverStreamMsgReceived.Collect(ch)
+	m.serverStreamMsgSent.Collect(ch)
+	if m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram.Collect(ch)
+	}
+}
+
+// UnaryServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Unary RPCs.
+func (m *ServerMetrics) UnaryServerInterceptor() func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		monitor := newServerReporter(m, Unary, info.FullMethod)
+		monitor.ReceivedMessage()
+		resp, err := handler(ctx, req)
+		monitor.Handled(grpc.Code(err))
+		if err == nil {
+			monitor.SentMessage()
+		}
+		return resp, err
+	}
+}
+
+// StreamServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+func (m *ServerMetrics) StreamServerInterceptor() func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+	return func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		monitor := newServerReporter(m, streamRpcType(info), info.FullMethod)
+		err := handler(srv, &monitoredServerStream{ss, monitor})
+		monitor.Handled(grpc.Code(err))
+		return err
+	}
+}
+
+// InitializeMetrics initializes all metrics, with their appropriate null
+// value, for all gRPC methods registered on a gRPC server. This is useful, to
+// ensure that all metrics exist when collecting and querying.
+func (m *ServerMetrics) InitializeMetrics(server *grpc.Server) {
+	serviceInfo := server.GetServiceInfo()
+	for serviceName, info := range serviceInfo {
+		for _, mInfo := range info.Methods {
+			preRegisterMethod(m, serviceName, &mInfo)
+		}
+	}
+}
+
+// Register registers all server metrics in a given metrics registry. Depending
+// on histogram options and whether they are enabled, histogram metrics are
+// also registered.
+//
+// Deprecated: ServerMetrics implements Prometheus Collector interface. You can
+// register an instance of ServerMetrics directly by using
+// prometheus.Register(m).
+func (m *ServerMetrics) Register(r prom.Registerer) error {
+	return r.Register(m)
+}
+
+// MustRegister tries to register all server metrics and panics on an error.
+//
+// Deprecated: ServerMetrics implements Prometheus Collector interface. You can
+// register an instance of ServerMetrics directly by using
+// prometheus.MustRegister(m).
+func (m *ServerMetrics) MustRegister(r prom.Registerer) {
+	r.MustRegister(m)
+}
+
+func streamRpcType(info *grpc.StreamServerInfo) grpcType {
+	if info.IsClientStream && !info.IsServerStream {
+		return ClientStream
+	} else if !info.IsClientStream && info.IsServerStream {
+		return ServerStream
+	}
+	return BidiStream
+}
+
+// monitoredStream wraps grpc.ServerStream allowing each Sent/Recv of message to increment counters.
+type monitoredServerStream struct {
+	grpc.ServerStream
+	monitor *serverReporter
+}
+
+func (s *monitoredServerStream) SendMsg(m interface{}) error {
+	err := s.ServerStream.SendMsg(m)
+	if err == nil {
+		s.monitor.SentMessage()
+	}
+	return err
+}
+
+func (s *monitoredServerStream) RecvMsg(m interface{}) error {
+	err := s.ServerStream.RecvMsg(m)
+	if err == nil {
+		s.monitor.ReceivedMessage()
+	}
+	return err
+}
+
+// preRegisterMethod is invoked on Register of a Server, allowing all gRPC services labels to be pre-populated.
+func preRegisterMethod(metrics *ServerMetrics, serviceName string, mInfo *grpc.MethodInfo) {
+	methodName := mInfo.Name
+	methodType := string(typeFromMethodInfo(mInfo))
+	// These are just references (no increments), as just referencing will create the labels but not set values.
+	metrics.serverStartedCounter.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	metrics.serverStreamMsgReceived.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	metrics.serverStreamMsgSent.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	if metrics.serverHandledHistogramEnabled {
+		metrics.serverHandledHistogram.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	}
+	for _, code := range allCodes {
+		metrics.serverHandledCounter.GetMetricWithLabelValues(methodType, serviceName, methodName, code.String())
+	}
+}

+ 13 - 124
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_reporter.go

@@ -7,151 +7,40 @@ import (
 	"time"
 
 	"google.golang.org/grpc/codes"
-
-	prom "github.com/prometheus/client_golang/prometheus"
-	"google.golang.org/grpc"
-)
-
-type grpcType string
-
-const (
-	Unary        grpcType = "unary"
-	ClientStream grpcType = "client_stream"
-	ServerStream grpcType = "server_stream"
-	BidiStream   grpcType = "bidi_stream"
 )
 
-var (
-	serverStartedCounter = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "server",
-			Name:      "started_total",
-			Help:      "Total number of RPCs started on the server.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	serverHandledCounter = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "server",
-			Name:      "handled_total",
-			Help:      "Total number of RPCs completed on the server, regardless of success or failure.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"})
-
-	serverStreamMsgReceived = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "server",
-			Name:      "msg_received_total",
-			Help:      "Total number of RPC stream messages received on the server.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	serverStreamMsgSent = prom.NewCounterVec(
-		prom.CounterOpts{
-			Namespace: "grpc",
-			Subsystem: "server",
-			Name:      "msg_sent_total",
-			Help:      "Total number of gRPC stream messages sent by the server.",
-		}, []string{"grpc_type", "grpc_service", "grpc_method"})
-
-	serverHandledHistogramEnabled = false
-	serverHandledHistogramOpts    = prom.HistogramOpts{
-		Namespace: "grpc",
-		Subsystem: "server",
-		Name:      "handling_seconds",
-		Help:      "Histogram of response latency (seconds) of gRPC that had been application-level handled by the server.",
-		Buckets:   prom.DefBuckets,
-	}
-	serverHandledHistogram *prom.HistogramVec
-)
-
-func init() {
-	prom.MustRegister(serverStartedCounter)
-	prom.MustRegister(serverHandledCounter)
-	prom.MustRegister(serverStreamMsgReceived)
-	prom.MustRegister(serverStreamMsgSent)
-}
-
-type HistogramOption func(*prom.HistogramOpts)
-
-// WithHistogramBuckets allows you to specify custom bucket ranges for histograms if EnableHandlingTimeHistogram is on.
-func WithHistogramBuckets(buckets []float64) HistogramOption {
-	return func(o *prom.HistogramOpts) { o.Buckets = buckets }
-}
-
-// EnableHandlingTimeHistogram turns on recording of handling time of RPCs for server-side interceptors.
-// Histogram metrics can be very expensive for Prometheus to retain and query.
-func EnableHandlingTimeHistogram(opts ...HistogramOption) {
-	for _, o := range opts {
-		o(&serverHandledHistogramOpts)
-	}
-	if !serverHandledHistogramEnabled {
-		serverHandledHistogram = prom.NewHistogramVec(
-			serverHandledHistogramOpts,
-			[]string{"grpc_type", "grpc_service", "grpc_method"},
-		)
-		prom.Register(serverHandledHistogram)
-	}
-	serverHandledHistogramEnabled = true
-}
-
 type serverReporter struct {
+	metrics     *ServerMetrics
 	rpcType     grpcType
 	serviceName string
 	methodName  string
 	startTime   time.Time
 }
 
-func newServerReporter(rpcType grpcType, fullMethod string) *serverReporter {
-	r := &serverReporter{rpcType: rpcType}
-	if serverHandledHistogramEnabled {
+func newServerReporter(m *ServerMetrics, rpcType grpcType, fullMethod string) *serverReporter {
+	r := &serverReporter{
+		metrics: m,
+		rpcType: rpcType,
+	}
+	if r.metrics.serverHandledHistogramEnabled {
 		r.startTime = time.Now()
 	}
 	r.serviceName, r.methodName = splitMethodName(fullMethod)
-	serverStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.serverStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 	return r
 }
 
 func (r *serverReporter) ReceivedMessage() {
-	serverStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.serverStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 }
 
 func (r *serverReporter) SentMessage() {
-	serverStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	r.metrics.serverStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
 }
 
 func (r *serverReporter) Handled(code codes.Code) {
-	serverHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
-	if serverHandledHistogramEnabled {
-		serverHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
-	}
-}
-
-// preRegisterMethod is invoked on Register of a Server, allowing all gRPC services labels to be pre-populated.
-func preRegisterMethod(serviceName string, mInfo *grpc.MethodInfo) {
-	methodName := mInfo.Name
-	methodType := string(typeFromMethodInfo(mInfo))
-	// These are just references (no increments), as just referencing will create the labels but not set values.
-	serverStartedCounter.GetMetricWithLabelValues(methodType, serviceName, methodName)
-	serverStreamMsgReceived.GetMetricWithLabelValues(methodType, serviceName, methodName)
-	serverStreamMsgSent.GetMetricWithLabelValues(methodType, serviceName, methodName)
-	if serverHandledHistogramEnabled {
-		serverHandledHistogram.GetMetricWithLabelValues(methodType, serviceName, methodName)
-	}
-	for _, code := range allCodes {
-		serverHandledCounter.GetMetricWithLabelValues(methodType, serviceName, methodName, code.String())
-	}
-}
-
-func typeFromMethodInfo(mInfo *grpc.MethodInfo) grpcType {
-	if mInfo.IsClientStream == false && mInfo.IsServerStream == false {
-		return Unary
-	}
-	if mInfo.IsClientStream == true && mInfo.IsServerStream == false {
-		return ClientStream
-	}
-	if mInfo.IsClientStream == false && mInfo.IsServerStream == true {
-		return ServerStream
+	r.metrics.serverHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
+	if r.metrics.serverHandledHistogramEnabled {
+		r.metrics.serverHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
 	}
-	return BidiStream
 }

+ 23 - 0
cmd/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/util.go

@@ -6,9 +6,19 @@ package grpc_prometheus
 import (
 	"strings"
 
+	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 )
 
+type grpcType string
+
+const (
+	Unary        grpcType = "unary"
+	ClientStream grpcType = "client_stream"
+	ServerStream grpcType = "server_stream"
+	BidiStream   grpcType = "bidi_stream"
+)
+
 var (
 	allCodes = []codes.Code{
 		codes.OK, codes.Canceled, codes.Unknown, codes.InvalidArgument, codes.DeadlineExceeded, codes.NotFound,
@@ -25,3 +35,16 @@ func splitMethodName(fullMethodName string) (string, string) {
 	}
 	return "unknown", "unknown"
 }
+
+func typeFromMethodInfo(mInfo *grpc.MethodInfo) grpcType {
+	if mInfo.IsClientStream == false && mInfo.IsServerStream == false {
+		return Unary
+	}
+	if mInfo.IsClientStream == true && mInfo.IsServerStream == false {
+		return ClientStream
+	}
+	if mInfo.IsClientStream == false && mInfo.IsServerStream == true {
+		return ServerStream
+	}
+	return BidiStream
+}

+ 44 - 27
cmd/vendor/github.com/prometheus/client_golang/prometheus/counter.go

@@ -30,16 +30,8 @@ type Counter interface {
 	Metric
 	Collector
 
-	// Set is used to set the Counter to an arbitrary value. It is only used
-	// if you have to transfer a value from an external counter into this
-	// Prometheus metric. Do not use it for regular handling of a
-	// Prometheus counter (as it can be used to break the contract of
-	// monotonically increasing values).
-	//
-	// Deprecated: Use NewConstMetric to create a counter for an external
-	// value. A Counter should never be set.
-	Set(float64)
-	// Inc increments the counter by 1.
+	// Inc increments the counter by 1. Use Add to increment it by arbitrary
+	// non-negative values.
 	Inc()
 	// Add adds the given value to the counter. It panics if the value is <
 	// 0.
@@ -78,16 +70,12 @@ func (c *counter) Add(v float64) {
 // if you want to count the same thing partitioned by various dimensions
 // (e.g. number of HTTP requests, partitioned by response code and
 // method). Create instances with NewCounterVec.
-//
-// CounterVec embeds MetricVec. See there for a full list of methods with
-// detailed documentation.
 type CounterVec struct {
-	*MetricVec
+	*metricVec
 }
 
 // NewCounterVec creates a new CounterVec based on the provided CounterOpts and
-// partitioned by the given label names. At least one label name must be
-// provided.
+// partitioned by the given label names.
 func NewCounterVec(opts CounterOpts, labelNames []string) *CounterVec {
 	desc := NewDesc(
 		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
@@ -96,7 +84,7 @@ func NewCounterVec(opts CounterOpts, labelNames []string) *CounterVec {
 		opts.ConstLabels,
 	)
 	return &CounterVec{
-		MetricVec: newMetricVec(desc, func(lvs ...string) Metric {
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
 			result := &counter{value: value{
 				desc:       desc,
 				valType:    CounterValue,
@@ -108,22 +96,51 @@ func NewCounterVec(opts CounterOpts, labelNames []string) *CounterVec {
 	}
 }
 
-// GetMetricWithLabelValues replaces the method of the same name in
-// MetricVec. The difference is that this method returns a Counter and not a
-// Metric so that no type conversion is required.
+// GetMetricWithLabelValues returns the Counter for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Counter is created.
+//
+// It is possible to call this method without using the returned Counter to only
+// create the new Counter but leave it at its starting value 0. See also the
+// SummaryVec example.
+//
+// Keeping the Counter for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Counter from the CounterVec. In that case,
+// the Counter will still exist, but it will not be exported anymore, even if a
+// Counter with the same label values is created later.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc.
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
 func (m *CounterVec) GetMetricWithLabelValues(lvs ...string) (Counter, error) {
-	metric, err := m.MetricVec.GetMetricWithLabelValues(lvs...)
+	metric, err := m.metricVec.getMetricWithLabelValues(lvs...)
 	if metric != nil {
 		return metric.(Counter), err
 	}
 	return nil, err
 }
 
-// GetMetricWith replaces the method of the same name in MetricVec. The
-// difference is that this method returns a Counter and not a Metric so that no
-// type conversion is required.
+// GetMetricWith returns the Counter for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Counter is created. Implications of
+// creating a Counter without using it and keeping the Counter for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc.
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
 func (m *CounterVec) GetMetricWith(labels Labels) (Counter, error) {
-	metric, err := m.MetricVec.GetMetricWith(labels)
+	metric, err := m.metricVec.getMetricWith(labels)
 	if metric != nil {
 		return metric.(Counter), err
 	}
@@ -135,14 +152,14 @@ func (m *CounterVec) GetMetricWith(labels Labels) (Counter, error) {
 // error, WithLabelValues allows shortcuts like
 //     myVec.WithLabelValues("404", "GET").Add(42)
 func (m *CounterVec) WithLabelValues(lvs ...string) Counter {
-	return m.MetricVec.WithLabelValues(lvs...).(Counter)
+	return m.metricVec.withLabelValues(lvs...).(Counter)
 }
 
 // With works as GetMetricWith, but panics where GetMetricWithLabels would have
 // returned an error. By not returning an error, With allows shortcuts like
 //     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
 func (m *CounterVec) With(labels Labels) Counter {
-	return m.MetricVec.With(labels).(Counter)
+	return m.metricVec.with(labels).(Counter)
 }
 
 // CounterFunc is a Counter whose value is determined at collect time by calling a

+ 10 - 26
cmd/vendor/github.com/prometheus/client_golang/prometheus/desc.go

@@ -16,33 +16,15 @@ package prometheus
 import (
 	"errors"
 	"fmt"
-	"regexp"
 	"sort"
 	"strings"
 
 	"github.com/golang/protobuf/proto"
+	"github.com/prometheus/common/model"
 
 	dto "github.com/prometheus/client_model/go"
 )
 
-var (
-	metricNameRE = regexp.MustCompile(`^[a-zA-Z_][a-zA-Z0-9_:]*$`)
-	labelNameRE  = regexp.MustCompile("^[a-zA-Z_][a-zA-Z0-9_]*$")
-)
-
-// reservedLabelPrefix is a prefix which is not legal in user-supplied
-// label names.
-const reservedLabelPrefix = "__"
-
-// Labels represents a collection of label name -> value mappings. This type is
-// commonly used with the With(Labels) and GetMetricWith(Labels) methods of
-// metric vector Collectors, e.g.:
-//     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
-//
-// The other use-case is the specification of constant label pairs in Opts or to
-// create a Desc.
-type Labels map[string]string
-
 // Desc is the descriptor used by every Prometheus Metric. It is essentially
 // the immutable meta-data of a Metric. The normal Metric implementations
 // included in this package manage their Desc under the hood. Users only have to
@@ -78,7 +60,7 @@ type Desc struct {
 	// Help string. Each Desc with the same fqName must have the same
 	// dimHash.
 	dimHash uint64
-	// err is an error that occured during construction. It is reported on
+	// err is an error that occurred during construction. It is reported on
 	// registration time.
 	err error
 }
@@ -103,7 +85,7 @@ func NewDesc(fqName, help string, variableLabels []string, constLabels Labels) *
 		d.err = errors.New("empty help string")
 		return d
 	}
-	if !metricNameRE.MatchString(fqName) {
+	if !model.IsValidMetricName(model.LabelValue(fqName)) {
 		d.err = fmt.Errorf("%q is not a valid metric name", fqName)
 		return d
 	}
@@ -127,6 +109,12 @@ func NewDesc(fqName, help string, variableLabels []string, constLabels Labels) *
 	for _, labelName := range labelNames {
 		labelValues = append(labelValues, constLabels[labelName])
 	}
+	// Validate the const label values. They can't have a wrong cardinality, so
+	// use in len(labelValues) as expectedNumberOfValues.
+	if err := validateLabelValues(labelValues, len(labelValues)); err != nil {
+		d.err = err
+		return d
+	}
 	// Now add the variable label names, but prefix them with something that
 	// cannot be in a regular label name. That prevents matching the label
 	// dimension with a different mix between preset and variable labels.
@@ -142,6 +130,7 @@ func NewDesc(fqName, help string, variableLabels []string, constLabels Labels) *
 		d.err = errors.New("duplicate label names")
 		return d
 	}
+
 	vh := hashNew()
 	for _, val := range labelValues {
 		vh = hashAdd(vh, val)
@@ -198,8 +187,3 @@ func (d *Desc) String() string {
 		d.variableLabels,
 	)
 }
-
-func checkLabelName(l string) bool {
-	return labelNameRE.MatchString(l) &&
-		!strings.HasPrefix(l, reservedLabelPrefix)
-}

+ 37 - 32
cmd/vendor/github.com/prometheus/client_golang/prometheus/doc.go

@@ -17,7 +17,7 @@
 // Pushgateway (package push).
 //
 // All exported functions and methods are safe to be used concurrently unless
-//specified otherwise.
+// specified otherwise.
 //
 // A Basic Example
 //
@@ -26,6 +26,7 @@
 //    package main
 //
 //    import (
+//    	"log"
 //    	"net/http"
 //
 //    	"github.com/prometheus/client_golang/prometheus"
@@ -59,7 +60,7 @@
 //    	// The Handler function provides a default handler to expose metrics
 //    	// via an HTTP server. "/metrics" is the usual endpoint for that.
 //    	http.Handle("/metrics", promhttp.Handler())
-//    	http.ListenAndServe(":8080", nil)
+//    	log.Fatal(http.ListenAndServe(":8080", nil))
 //    }
 //
 //
@@ -69,7 +70,7 @@
 // Metrics
 //
 // The number of exported identifiers in this package might appear a bit
-// overwhelming. Hovever, in addition to the basic plumbing shown in the example
+// overwhelming. However, in addition to the basic plumbing shown in the example
 // above, you only need to understand the different metric types and their
 // vector versions for basic usage.
 //
@@ -95,8 +96,8 @@
 // SummaryVec, HistogramVec, and UntypedVec are not.
 //
 // To create instances of Metrics and their vector versions, you need a suitable
-// …Opts struct, i.e. GaugeOpts, CounterOpts, SummaryOpts,
-// HistogramOpts, or UntypedOpts.
+// …Opts struct, i.e. GaugeOpts, CounterOpts, SummaryOpts, HistogramOpts, or
+// UntypedOpts.
 //
 // Custom Collectors and constant Metrics
 //
@@ -114,8 +115,8 @@
 // Metric instances “on the fly” using NewConstMetric, NewConstHistogram, and
 // NewConstSummary (and their respective Must… versions). That will happen in
 // the Collect method. The Describe method has to return separate Desc
-// instances, representative of the “throw-away” metrics to be created
-// later. NewDesc comes in handy to create those Desc instances.
+// instances, representative of the “throw-away” metrics to be created later.
+// NewDesc comes in handy to create those Desc instances.
 //
 // The Collector example illustrates the use case. You can also look at the
 // source code of the processCollector (mirroring process metrics), the
@@ -129,34 +130,34 @@
 // Advanced Uses of the Registry
 //
 // While MustRegister is the by far most common way of registering a Collector,
-// sometimes you might want to handle the errors the registration might
-// cause. As suggested by the name, MustRegister panics if an error occurs. With
-// the Register function, the error is returned and can be handled.
+// sometimes you might want to handle the errors the registration might cause.
+// As suggested by the name, MustRegister panics if an error occurs. With the
+// Register function, the error is returned and can be handled.
 //
 // An error is returned if the registered Collector is incompatible or
 // inconsistent with already registered metrics. The registry aims for
-// consistency of the collected metrics according to the Prometheus data
-// model. Inconsistencies are ideally detected at registration time, not at
-// collect time. The former will usually be detected at start-up time of a
-// program, while the latter will only happen at scrape time, possibly not even
-// on the first scrape if the inconsistency only becomes relevant later. That is
-// the main reason why a Collector and a Metric have to describe themselves to
-// the registry.
+// consistency of the collected metrics according to the Prometheus data model.
+// Inconsistencies are ideally detected at registration time, not at collect
+// time. The former will usually be detected at start-up time of a program,
+// while the latter will only happen at scrape time, possibly not even on the
+// first scrape if the inconsistency only becomes relevant later. That is the
+// main reason why a Collector and a Metric have to describe themselves to the
+// registry.
 //
 // So far, everything we did operated on the so-called default registry, as it
-// can be found in the global DefaultRegistry variable. With NewRegistry, you
+// can be found in the global DefaultRegisterer variable. With NewRegistry, you
 // can create a custom registry, or you can even implement the Registerer or
-// Gatherer interfaces yourself. The methods Register and Unregister work in
-// the same way on a custom registry as the global functions Register and
-// Unregister on the default registry.
-//
-// There are a number of uses for custom registries: You can use registries
-// with special properties, see NewPedanticRegistry. You can avoid global state,
-// as it is imposed by the DefaultRegistry. You can use multiple registries at
-// the same time to expose different metrics in different ways. You can use
+// Gatherer interfaces yourself. The methods Register and Unregister work in the
+// same way on a custom registry as the global functions Register and Unregister
+// on the default registry.
+//
+// There are a number of uses for custom registries: You can use registries with
+// special properties, see NewPedanticRegistry. You can avoid global state, as
+// it is imposed by the DefaultRegisterer. You can use multiple registries at
+// the same time to expose different metrics in different ways.  You can use
 // separate registries for testing purposes.
 //
-// Also note that the DefaultRegistry comes registered with a Collector for Go
+// Also note that the DefaultRegisterer comes registered with a Collector for Go
 // runtime metrics (via NewGoCollector) and a Collector for process metrics (via
 // NewProcessCollector). With a custom registry, you are in control and decide
 // yourself about the Collectors to register.
@@ -166,16 +167,20 @@
 // The Registry implements the Gatherer interface. The caller of the Gather
 // method can then expose the gathered metrics in some way. Usually, the metrics
 // are served via HTTP on the /metrics endpoint. That's happening in the example
-// above. The tools to expose metrics via HTTP are in the promhttp
-// sub-package. (The top-level functions in the prometheus package are
-// deprecated.)
+// above. The tools to expose metrics via HTTP are in the promhttp sub-package.
+// (The top-level functions in the prometheus package are deprecated.)
 //
 // Pushing to the Pushgateway
 //
 // Function for pushing to the Pushgateway can be found in the push sub-package.
 //
+// Graphite Bridge
+//
+// Functions and examples to push metrics from a Gatherer to Graphite can be
+// found in the graphite sub-package.
+//
 // Other Means of Exposition
 //
-// More ways of exposing metrics can easily be added. Sending metrics to
-// Graphite would be an example that will soon be implemented.
+// More ways of exposing metrics can easily be added by following the approaches
+// of the existing implementations.
 package prometheus

+ 51 - 18
cmd/vendor/github.com/prometheus/client_golang/prometheus/gauge.go

@@ -27,16 +27,21 @@ type Gauge interface {
 
 	// Set sets the Gauge to an arbitrary value.
 	Set(float64)
-	// Inc increments the Gauge by 1.
+	// Inc increments the Gauge by 1. Use Add to increment it by arbitrary
+	// values.
 	Inc()
-	// Dec decrements the Gauge by 1.
+	// Dec decrements the Gauge by 1. Use Sub to decrement it by arbitrary
+	// values.
 	Dec()
-	// Add adds the given value to the Gauge. (The value can be
-	// negative, resulting in a decrease of the Gauge.)
+	// Add adds the given value to the Gauge. (The value can be negative,
+	// resulting in a decrease of the Gauge.)
 	Add(float64)
 	// Sub subtracts the given value from the Gauge. (The value can be
 	// negative, resulting in an increase of the Gauge.)
 	Sub(float64)
+
+	// SetToCurrentTime sets the Gauge to the current Unix time in seconds.
+	SetToCurrentTime()
 }
 
 // GaugeOpts is an alias for Opts. See there for doc comments.
@@ -58,12 +63,11 @@ func NewGauge(opts GaugeOpts) Gauge {
 // (e.g. number of operations queued, partitioned by user and operation
 // type). Create instances with NewGaugeVec.
 type GaugeVec struct {
-	*MetricVec
+	*metricVec
 }
 
 // NewGaugeVec creates a new GaugeVec based on the provided GaugeOpts and
-// partitioned by the given label names. At least one label name must be
-// provided.
+// partitioned by the given label names.
 func NewGaugeVec(opts GaugeOpts, labelNames []string) *GaugeVec {
 	desc := NewDesc(
 		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
@@ -72,28 +76,57 @@ func NewGaugeVec(opts GaugeOpts, labelNames []string) *GaugeVec {
 		opts.ConstLabels,
 	)
 	return &GaugeVec{
-		MetricVec: newMetricVec(desc, func(lvs ...string) Metric {
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
 			return newValue(desc, GaugeValue, 0, lvs...)
 		}),
 	}
 }
 
-// GetMetricWithLabelValues replaces the method of the same name in
-// MetricVec. The difference is that this method returns a Gauge and not a
-// Metric so that no type conversion is required.
+// GetMetricWithLabelValues returns the Gauge for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Gauge is created.
+//
+// It is possible to call this method without using the returned Gauge to only
+// create the new Gauge but leave it at its starting value 0. See also the
+// SummaryVec example.
+//
+// Keeping the Gauge for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Gauge from the GaugeVec. In that case, the
+// Gauge will still exist, but it will not be exported anymore, even if a
+// Gauge with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc.
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
 func (m *GaugeVec) GetMetricWithLabelValues(lvs ...string) (Gauge, error) {
-	metric, err := m.MetricVec.GetMetricWithLabelValues(lvs...)
+	metric, err := m.metricVec.getMetricWithLabelValues(lvs...)
 	if metric != nil {
 		return metric.(Gauge), err
 	}
 	return nil, err
 }
 
-// GetMetricWith replaces the method of the same name in MetricVec. The
-// difference is that this method returns a Gauge and not a Metric so that no
-// type conversion is required.
+// GetMetricWith returns the Gauge for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Gauge is created. Implications of
+// creating a Gauge without using it and keeping the Gauge for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc.
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
 func (m *GaugeVec) GetMetricWith(labels Labels) (Gauge, error) {
-	metric, err := m.MetricVec.GetMetricWith(labels)
+	metric, err := m.metricVec.getMetricWith(labels)
 	if metric != nil {
 		return metric.(Gauge), err
 	}
@@ -105,14 +138,14 @@ func (m *GaugeVec) GetMetricWith(labels Labels) (Gauge, error) {
 // error, WithLabelValues allows shortcuts like
 //     myVec.WithLabelValues("404", "GET").Add(42)
 func (m *GaugeVec) WithLabelValues(lvs ...string) Gauge {
-	return m.MetricVec.WithLabelValues(lvs...).(Gauge)
+	return m.metricVec.withLabelValues(lvs...).(Gauge)
 }
 
 // With works as GetMetricWith, but panics where GetMetricWithLabels would have
 // returned an error. By not returning an error, With allows shortcuts like
 //     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
 func (m *GaugeVec) With(labels Labels) Gauge {
-	return m.MetricVec.With(labels).(Gauge)
+	return m.metricVec.with(labels).(Gauge)
 }
 
 // GaugeFunc is a Gauge whose value is determined at collect time by calling a

+ 36 - 15
cmd/vendor/github.com/prometheus/client_golang/prometheus/go_collector.go

@@ -8,8 +8,10 @@ import (
 )
 
 type goCollector struct {
-	goroutines Gauge
-	gcDesc     *Desc
+	goroutinesDesc *Desc
+	threadsDesc    *Desc
+	gcDesc         *Desc
+	goInfoDesc     *Desc
 
 	// metrics to describe and collect
 	metrics memStatsMetrics
@@ -19,15 +21,22 @@ type goCollector struct {
 // go process.
 func NewGoCollector() Collector {
 	return &goCollector{
-		goroutines: NewGauge(GaugeOpts{
-			Namespace: "go",
-			Name:      "goroutines",
-			Help:      "Number of goroutines that currently exist.",
-		}),
+		goroutinesDesc: NewDesc(
+			"go_goroutines",
+			"Number of goroutines that currently exist.",
+			nil, nil),
+		threadsDesc: NewDesc(
+			"go_threads",
+			"Number of OS threads created.",
+			nil, nil),
 		gcDesc: NewDesc(
 			"go_gc_duration_seconds",
 			"A summary of the GC invocation durations.",
 			nil, nil),
+		goInfoDesc: NewDesc(
+			"go_info",
+			"Information about the Go environment.",
+			nil, Labels{"version": runtime.Version()}),
 		metrics: memStatsMetrics{
 			{
 				desc: NewDesc(
@@ -48,7 +57,7 @@ func NewGoCollector() Collector {
 			}, {
 				desc: NewDesc(
 					memstatNamespace("sys_bytes"),
-					"Number of bytes obtained by system. Sum of all system allocations.",
+					"Number of bytes obtained from system.",
 					nil, nil,
 				),
 				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Sys) },
@@ -111,12 +120,12 @@ func NewGoCollector() Collector {
 				valType: GaugeValue,
 			}, {
 				desc: NewDesc(
-					memstatNamespace("heap_released_bytes_total"),
-					"Total number of heap bytes released to OS.",
+					memstatNamespace("heap_released_bytes"),
+					"Number of heap bytes released to OS.",
 					nil, nil,
 				),
 				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapReleased) },
-				valType: CounterValue,
+				valType: GaugeValue,
 			}, {
 				desc: NewDesc(
 					memstatNamespace("heap_objects"),
@@ -213,6 +222,14 @@ func NewGoCollector() Collector {
 				),
 				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.LastGC) / 1e9 },
 				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("gc_cpu_fraction"),
+					"The fraction of this program's available CPU time used by the GC since the program started.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return ms.GCCPUFraction },
+				valType: GaugeValue,
 			},
 		},
 	}
@@ -224,9 +241,10 @@ func memstatNamespace(s string) string {
 
 // Describe returns all descriptions of the collector.
 func (c *goCollector) Describe(ch chan<- *Desc) {
-	ch <- c.goroutines.Desc()
+	ch <- c.goroutinesDesc
+	ch <- c.threadsDesc
 	ch <- c.gcDesc
-
+	ch <- c.goInfoDesc
 	for _, i := range c.metrics {
 		ch <- i.desc
 	}
@@ -234,8 +252,9 @@ func (c *goCollector) Describe(ch chan<- *Desc) {
 
 // Collect returns the current state of all metrics of the collector.
 func (c *goCollector) Collect(ch chan<- Metric) {
-	c.goroutines.Set(float64(runtime.NumGoroutine()))
-	ch <- c.goroutines
+	ch <- MustNewConstMetric(c.goroutinesDesc, GaugeValue, float64(runtime.NumGoroutine()))
+	n, _ := runtime.ThreadCreateProfile(nil)
+	ch <- MustNewConstMetric(c.threadsDesc, GaugeValue, float64(n))
 
 	var stats debug.GCStats
 	stats.PauseQuantiles = make([]time.Duration, 5)
@@ -248,6 +267,8 @@ func (c *goCollector) Collect(ch chan<- Metric) {
 	quantiles[0.0] = stats.PauseQuantiles[0].Seconds()
 	ch <- MustNewConstSummary(c.gcDesc, uint64(stats.NumGC), float64(stats.PauseTotal.Seconds()), quantiles)
 
+	ch <- MustNewConstMetric(c.goInfoDesc, GaugeValue, 1)
+
 	ms := &runtime.MemStats{}
 	runtime.ReadMemStats(ms)
 	for _, i := range c.metrics {

+ 51 - 22
cmd/vendor/github.com/prometheus/client_golang/prometheus/histogram.go

@@ -287,12 +287,11 @@ func (h *histogram) Write(out *dto.Metric) error {
 // (e.g. HTTP request latencies, partitioned by status code and method). Create
 // instances with NewHistogramVec.
 type HistogramVec struct {
-	*MetricVec
+	*metricVec
 }
 
 // NewHistogramVec creates a new HistogramVec based on the provided HistogramOpts and
-// partitioned by the given label names. At least one label name must be
-// provided.
+// partitioned by the given label names.
 func NewHistogramVec(opts HistogramOpts, labelNames []string) *HistogramVec {
 	desc := NewDesc(
 		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
@@ -301,30 +300,60 @@ func NewHistogramVec(opts HistogramOpts, labelNames []string) *HistogramVec {
 		opts.ConstLabels,
 	)
 	return &HistogramVec{
-		MetricVec: newMetricVec(desc, func(lvs ...string) Metric {
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
 			return newHistogram(desc, opts, lvs...)
 		}),
 	}
 }
 
-// GetMetricWithLabelValues replaces the method of the same name in
-// MetricVec. The difference is that this method returns a Histogram and not a
-// Metric so that no type conversion is required.
-func (m *HistogramVec) GetMetricWithLabelValues(lvs ...string) (Histogram, error) {
-	metric, err := m.MetricVec.GetMetricWithLabelValues(lvs...)
+// GetMetricWithLabelValues returns the Histogram for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Histogram is created.
+//
+// It is possible to call this method without using the returned Histogram to only
+// create the new Histogram but leave it at its starting value, a Histogram without
+// any observations.
+//
+// Keeping the Histogram for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Histogram from the HistogramVec. In that case, the
+// Histogram will still exist, but it will not be exported anymore, even if a
+// Histogram with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc.
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
+func (m *HistogramVec) GetMetricWithLabelValues(lvs ...string) (Observer, error) {
+	metric, err := m.metricVec.getMetricWithLabelValues(lvs...)
 	if metric != nil {
-		return metric.(Histogram), err
+		return metric.(Observer), err
 	}
 	return nil, err
 }
 
-// GetMetricWith replaces the method of the same name in MetricVec. The
-// difference is that this method returns a Histogram and not a Metric so that no
-// type conversion is required.
-func (m *HistogramVec) GetMetricWith(labels Labels) (Histogram, error) {
-	metric, err := m.MetricVec.GetMetricWith(labels)
+// GetMetricWith returns the Histogram for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Histogram is created. Implications of
+// creating a Histogram without using it and keeping the Histogram for later use
+// are the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc.
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (m *HistogramVec) GetMetricWith(labels Labels) (Observer, error) {
+	metric, err := m.metricVec.getMetricWith(labels)
 	if metric != nil {
-		return metric.(Histogram), err
+		return metric.(Observer), err
 	}
 	return nil, err
 }
@@ -333,15 +362,15 @@ func (m *HistogramVec) GetMetricWith(labels Labels) (Histogram, error) {
 // GetMetricWithLabelValues would have returned an error. By not returning an
 // error, WithLabelValues allows shortcuts like
 //     myVec.WithLabelValues("404", "GET").Observe(42.21)
-func (m *HistogramVec) WithLabelValues(lvs ...string) Histogram {
-	return m.MetricVec.WithLabelValues(lvs...).(Histogram)
+func (m *HistogramVec) WithLabelValues(lvs ...string) Observer {
+	return m.metricVec.withLabelValues(lvs...).(Observer)
 }
 
 // With works as GetMetricWith, but panics where GetMetricWithLabels would have
 // returned an error. By not returning an error, With allows shortcuts like
 //     myVec.With(Labels{"code": "404", "method": "GET"}).Observe(42.21)
-func (m *HistogramVec) With(labels Labels) Histogram {
-	return m.MetricVec.With(labels).(Histogram)
+func (m *HistogramVec) With(labels Labels) Observer {
+	return m.metricVec.with(labels).(Observer)
 }
 
 type constHistogram struct {
@@ -401,8 +430,8 @@ func NewConstHistogram(
 	buckets map[float64]uint64,
 	labelValues ...string,
 ) (Metric, error) {
-	if len(desc.variableLabels) != len(labelValues) {
-		return nil, errInconsistentCardinality
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
 	}
 	return &constHistogram{
 		desc:       desc,

+ 72 - 38
cmd/vendor/github.com/prometheus/client_golang/prometheus/http.go

@@ -62,7 +62,8 @@ func giveBuf(buf *bytes.Buffer) {
 //
 // Deprecated: Please note the issues described in the doc comment of
 // InstrumentHandler. You might want to consider using promhttp.Handler instead
-// (which is non instrumented).
+// (which is not instrumented, but can be instrumented with the tooling provided
+// in package promhttp).
 func Handler() http.Handler {
 	return InstrumentHandler("prometheus", UninstrumentedHandler())
 }
@@ -95,7 +96,7 @@ func UninstrumentedHandler() http.Handler {
 			closer.Close()
 		}
 		if lastErr != nil && buf.Len() == 0 {
-			http.Error(w, "No metrics encoded, last error:\n\n"+err.Error(), http.StatusInternalServerError)
+			http.Error(w, "No metrics encoded, last error:\n\n"+lastErr.Error(), http.StatusInternalServerError)
 			return
 		}
 		header := w.Header()
@@ -158,7 +159,8 @@ func nowSeries(t ...time.Time) nower {
 // value. http_requests_total is a metric vector partitioned by HTTP method
 // (label name "method") and HTTP status code (label name "code").
 //
-// Deprecated: InstrumentHandler has several issues:
+// Deprecated: InstrumentHandler has several issues. Use the tooling provided in
+// package promhttp instead. The issues are the following:
 //
 // - It uses Summaries rather than Histograms. Summaries are not useful if
 // aggregation across multiple instances is required.
@@ -172,9 +174,8 @@ func nowSeries(t ...time.Time) nower {
 // httputil.ReverseProxy is a prominent example for a handler
 // performing such writes.
 //
-// Upcoming versions of this package will provide ways of instrumenting HTTP
-// handlers that are more flexible and have fewer issues. Please prefer direct
-// instrumentation in the meantime.
+// - It has additional issues with HTTP/2, cf.
+// https://github.com/prometheus/client_golang/issues/272.
 func InstrumentHandler(handlerName string, handler http.Handler) http.HandlerFunc {
 	return InstrumentHandlerFunc(handlerName, handler.ServeHTTP)
 }
@@ -184,12 +185,13 @@ func InstrumentHandler(handlerName string, handler http.Handler) http.HandlerFun
 // issues).
 //
 // Deprecated: InstrumentHandlerFunc is deprecated for the same reasons as
-// InstrumentHandler is.
+// InstrumentHandler is. Use the tooling provided in package promhttp instead.
 func InstrumentHandlerFunc(handlerName string, handlerFunc func(http.ResponseWriter, *http.Request)) http.HandlerFunc {
 	return InstrumentHandlerFuncWithOpts(
 		SummaryOpts{
 			Subsystem:   "http",
 			ConstLabels: Labels{"handler": handlerName},
+			Objectives:  map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
 		},
 		handlerFunc,
 	)
@@ -222,7 +224,7 @@ func InstrumentHandlerFunc(handlerName string, handlerFunc func(http.ResponseWri
 // SummaryOpts.
 //
 // Deprecated: InstrumentHandlerWithOpts is deprecated for the same reasons as
-// InstrumentHandler is.
+// InstrumentHandler is. Use the tooling provided in package promhttp instead.
 func InstrumentHandlerWithOpts(opts SummaryOpts, handler http.Handler) http.HandlerFunc {
 	return InstrumentHandlerFuncWithOpts(opts, handler.ServeHTTP)
 }
@@ -233,7 +235,7 @@ func InstrumentHandlerWithOpts(opts SummaryOpts, handler http.Handler) http.Hand
 // SummaryOpts are used.
 //
 // Deprecated: InstrumentHandlerFuncWithOpts is deprecated for the same reasons
-// as InstrumentHandler is.
+// as InstrumentHandler is. Use the tooling provided in package promhttp instead.
 func InstrumentHandlerFuncWithOpts(opts SummaryOpts, handlerFunc func(http.ResponseWriter, *http.Request)) http.HandlerFunc {
 	reqCnt := NewCounterVec(
 		CounterOpts{
@@ -245,34 +247,52 @@ func InstrumentHandlerFuncWithOpts(opts SummaryOpts, handlerFunc func(http.Respo
 		},
 		instLabels,
 	)
+	if err := Register(reqCnt); err != nil {
+		if are, ok := err.(AlreadyRegisteredError); ok {
+			reqCnt = are.ExistingCollector.(*CounterVec)
+		} else {
+			panic(err)
+		}
+	}
 
 	opts.Name = "request_duration_microseconds"
 	opts.Help = "The HTTP request latencies in microseconds."
 	reqDur := NewSummary(opts)
+	if err := Register(reqDur); err != nil {
+		if are, ok := err.(AlreadyRegisteredError); ok {
+			reqDur = are.ExistingCollector.(Summary)
+		} else {
+			panic(err)
+		}
+	}
 
 	opts.Name = "request_size_bytes"
 	opts.Help = "The HTTP request sizes in bytes."
 	reqSz := NewSummary(opts)
+	if err := Register(reqSz); err != nil {
+		if are, ok := err.(AlreadyRegisteredError); ok {
+			reqSz = are.ExistingCollector.(Summary)
+		} else {
+			panic(err)
+		}
+	}
 
 	opts.Name = "response_size_bytes"
 	opts.Help = "The HTTP response sizes in bytes."
 	resSz := NewSummary(opts)
-
-	regReqCnt := MustRegisterOrGet(reqCnt).(*CounterVec)
-	regReqDur := MustRegisterOrGet(reqDur).(Summary)
-	regReqSz := MustRegisterOrGet(reqSz).(Summary)
-	regResSz := MustRegisterOrGet(resSz).(Summary)
+	if err := Register(resSz); err != nil {
+		if are, ok := err.(AlreadyRegisteredError); ok {
+			resSz = are.ExistingCollector.(Summary)
+		} else {
+			panic(err)
+		}
+	}
 
 	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
 		now := time.Now()
 
 		delegate := &responseWriterDelegator{ResponseWriter: w}
-		out := make(chan int)
-		urlLen := 0
-		if r.URL != nil {
-			urlLen = len(r.URL.String())
-		}
-		go computeApproximateRequestSize(r, out, urlLen)
+		out := computeApproximateRequestSize(r)
 
 		_, cn := w.(http.CloseNotifier)
 		_, fl := w.(http.Flusher)
@@ -290,30 +310,44 @@ func InstrumentHandlerFuncWithOpts(opts SummaryOpts, handlerFunc func(http.Respo
 
 		method := sanitizeMethod(r.Method)
 		code := sanitizeCode(delegate.status)
-		regReqCnt.WithLabelValues(method, code).Inc()
-		regReqDur.Observe(elapsed)
-		regResSz.Observe(float64(delegate.written))
-		regReqSz.Observe(float64(<-out))
+		reqCnt.WithLabelValues(method, code).Inc()
+		reqDur.Observe(elapsed)
+		resSz.Observe(float64(delegate.written))
+		reqSz.Observe(float64(<-out))
 	})
 }
 
-func computeApproximateRequestSize(r *http.Request, out chan int, s int) {
-	s += len(r.Method)
-	s += len(r.Proto)
-	for name, values := range r.Header {
-		s += len(name)
-		for _, value := range values {
-			s += len(value)
-		}
+func computeApproximateRequestSize(r *http.Request) <-chan int {
+	// Get URL length in current go routine for avoiding a race condition.
+	// HandlerFunc that runs in parallel may modify the URL.
+	s := 0
+	if r.URL != nil {
+		s += len(r.URL.String())
 	}
-	s += len(r.Host)
 
-	// N.B. r.Form and r.MultipartForm are assumed to be included in r.URL.
+	out := make(chan int, 1)
 
-	if r.ContentLength != -1 {
-		s += int(r.ContentLength)
-	}
-	out <- s
+	go func() {
+		s += len(r.Method)
+		s += len(r.Proto)
+		for name, values := range r.Header {
+			s += len(name)
+			for _, value := range values {
+				s += len(value)
+			}
+		}
+		s += len(r.Host)
+
+		// N.B. r.Form and r.MultipartForm are assumed to be included in r.URL.
+
+		if r.ContentLength != -1 {
+			s += int(r.ContentLength)
+		}
+		out <- s
+		close(out)
+	}()
+
+	return out
 }
 
 type responseWriterDelegator struct {

+ 57 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/labels.go

@@ -0,0 +1,57 @@
+package prometheus
+
+import (
+	"errors"
+	"fmt"
+	"strings"
+	"unicode/utf8"
+
+	"github.com/prometheus/common/model"
+)
+
+// Labels represents a collection of label name -> value mappings. This type is
+// commonly used with the With(Labels) and GetMetricWith(Labels) methods of
+// metric vector Collectors, e.g.:
+//     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
+//
+// The other use-case is the specification of constant label pairs in Opts or to
+// create a Desc.
+type Labels map[string]string
+
+// reservedLabelPrefix is a prefix which is not legal in user-supplied
+// label names.
+const reservedLabelPrefix = "__"
+
+var errInconsistentCardinality = errors.New("inconsistent label cardinality")
+
+func validateValuesInLabels(labels Labels, expectedNumberOfValues int) error {
+	if len(labels) != expectedNumberOfValues {
+		return errInconsistentCardinality
+	}
+
+	for name, val := range labels {
+		if !utf8.ValidString(val) {
+			return fmt.Errorf("label %s: value %q is not valid UTF-8", name, val)
+		}
+	}
+
+	return nil
+}
+
+func validateLabelValues(vals []string, expectedNumberOfValues int) error {
+	if len(vals) != expectedNumberOfValues {
+		return errInconsistentCardinality
+	}
+
+	for _, val := range vals {
+		if !utf8.ValidString(val) {
+			return fmt.Errorf("label value %q is not valid UTF-8", val)
+		}
+	}
+
+	return nil
+}
+
+func checkLabelName(l string) bool {
+	return model.LabelName(l).IsValid() && !strings.HasPrefix(l, reservedLabelPrefix)
+}

+ 50 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/observer.go

@@ -0,0 +1,50 @@
+// Copyright 2017 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 prometheus
+
+// Observer is the interface that wraps the Observe method, which is used by
+// Histogram and Summary to add observations.
+type Observer interface {
+	Observe(float64)
+}
+
+// The ObserverFunc type is an adapter to allow the use of ordinary
+// functions as Observers. If f is a function with the appropriate
+// signature, ObserverFunc(f) is an Observer that calls f.
+//
+// This adapter is usually used in connection with the Timer type, and there are
+// two general use cases:
+//
+// The most common one is to use a Gauge as the Observer for a Timer.
+// See the "Gauge" Timer example.
+//
+// The more advanced use case is to create a function that dynamically decides
+// which Observer to use for observing the duration. See the "Complex" Timer
+// example.
+type ObserverFunc func(float64)
+
+// Observe calls f(value). It implements Observer.
+func (f ObserverFunc) Observe(value float64) {
+	f(value)
+}
+
+// ObserverVec is an interface implemented by `HistogramVec` and `SummaryVec`.
+type ObserverVec interface {
+	GetMetricWith(Labels) (Observer, error)
+	GetMetricWithLabelValues(lvs ...string) (Observer, error)
+	With(Labels) Observer
+	WithLabelValues(...string) Observer
+
+	Collector
+}

+ 51 - 53
cmd/vendor/github.com/prometheus/client_golang/prometheus/process_collector.go

@@ -19,10 +19,10 @@ type processCollector struct {
 	pid             int
 	collectFn       func(chan<- Metric)
 	pidFn           func() (int, error)
-	cpuTotal        Counter
-	openFDs, maxFDs Gauge
-	vsize, rss      Gauge
-	startTime       Gauge
+	cpuTotal        *Desc
+	openFDs, maxFDs *Desc
+	vsize, rss      *Desc
+	startTime       *Desc
 }
 
 // NewProcessCollector returns a collector which exports the current state of
@@ -44,40 +44,45 @@ func NewProcessCollectorPIDFn(
 	pidFn func() (int, error),
 	namespace string,
 ) Collector {
+	ns := ""
+	if len(namespace) > 0 {
+		ns = namespace + "_"
+	}
+
 	c := processCollector{
 		pidFn:     pidFn,
 		collectFn: func(chan<- Metric) {},
 
-		cpuTotal: NewCounter(CounterOpts{
-			Namespace: namespace,
-			Name:      "process_cpu_seconds_total",
-			Help:      "Total user and system CPU time spent in seconds.",
-		}),
-		openFDs: NewGauge(GaugeOpts{
-			Namespace: namespace,
-			Name:      "process_open_fds",
-			Help:      "Number of open file descriptors.",
-		}),
-		maxFDs: NewGauge(GaugeOpts{
-			Namespace: namespace,
-			Name:      "process_max_fds",
-			Help:      "Maximum number of open file descriptors.",
-		}),
-		vsize: NewGauge(GaugeOpts{
-			Namespace: namespace,
-			Name:      "process_virtual_memory_bytes",
-			Help:      "Virtual memory size in bytes.",
-		}),
-		rss: NewGauge(GaugeOpts{
-			Namespace: namespace,
-			Name:      "process_resident_memory_bytes",
-			Help:      "Resident memory size in bytes.",
-		}),
-		startTime: NewGauge(GaugeOpts{
-			Namespace: namespace,
-			Name:      "process_start_time_seconds",
-			Help:      "Start time of the process since unix epoch in seconds.",
-		}),
+		cpuTotal: NewDesc(
+			ns+"process_cpu_seconds_total",
+			"Total user and system CPU time spent in seconds.",
+			nil, nil,
+		),
+		openFDs: NewDesc(
+			ns+"process_open_fds",
+			"Number of open file descriptors.",
+			nil, nil,
+		),
+		maxFDs: NewDesc(
+			ns+"process_max_fds",
+			"Maximum number of open file descriptors.",
+			nil, nil,
+		),
+		vsize: NewDesc(
+			ns+"process_virtual_memory_bytes",
+			"Virtual memory size in bytes.",
+			nil, nil,
+		),
+		rss: NewDesc(
+			ns+"process_resident_memory_bytes",
+			"Resident memory size in bytes.",
+			nil, nil,
+		),
+		startTime: NewDesc(
+			ns+"process_start_time_seconds",
+			"Start time of the process since unix epoch in seconds.",
+			nil, nil,
+		),
 	}
 
 	// Set up process metric collection if supported by the runtime.
@@ -90,12 +95,12 @@ func NewProcessCollectorPIDFn(
 
 // Describe returns all descriptions of the collector.
 func (c *processCollector) Describe(ch chan<- *Desc) {
-	ch <- c.cpuTotal.Desc()
-	ch <- c.openFDs.Desc()
-	ch <- c.maxFDs.Desc()
-	ch <- c.vsize.Desc()
-	ch <- c.rss.Desc()
-	ch <- c.startTime.Desc()
+	ch <- c.cpuTotal
+	ch <- c.openFDs
+	ch <- c.maxFDs
+	ch <- c.vsize
+	ch <- c.rss
+	ch <- c.startTime
 }
 
 // Collect returns the current state of all metrics of the collector.
@@ -117,26 +122,19 @@ func (c *processCollector) processCollect(ch chan<- Metric) {
 	}
 
 	if stat, err := p.NewStat(); err == nil {
-		c.cpuTotal.Set(stat.CPUTime())
-		ch <- c.cpuTotal
-		c.vsize.Set(float64(stat.VirtualMemory()))
-		ch <- c.vsize
-		c.rss.Set(float64(stat.ResidentMemory()))
-		ch <- c.rss
-
+		ch <- MustNewConstMetric(c.cpuTotal, CounterValue, stat.CPUTime())
+		ch <- MustNewConstMetric(c.vsize, GaugeValue, float64(stat.VirtualMemory()))
+		ch <- MustNewConstMetric(c.rss, GaugeValue, float64(stat.ResidentMemory()))
 		if startTime, err := stat.StartTime(); err == nil {
-			c.startTime.Set(startTime)
-			ch <- c.startTime
+			ch <- MustNewConstMetric(c.startTime, GaugeValue, startTime)
 		}
 	}
 
 	if fds, err := p.FileDescriptorsLen(); err == nil {
-		c.openFDs.Set(float64(fds))
-		ch <- c.openFDs
+		ch <- MustNewConstMetric(c.openFDs, GaugeValue, float64(fds))
 	}
 
 	if limits, err := p.NewLimits(); err == nil {
-		c.maxFDs.Set(float64(limits.OpenFiles))
-		ch <- c.maxFDs
+		ch <- MustNewConstMetric(c.maxFDs, GaugeValue, float64(limits.OpenFiles))
 	}
 }

+ 199 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator.go

@@ -0,0 +1,199 @@
+// Copyright 2017 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 promhttp
+
+import (
+	"bufio"
+	"io"
+	"net"
+	"net/http"
+)
+
+const (
+	closeNotifier = 1 << iota
+	flusher
+	hijacker
+	readerFrom
+	pusher
+)
+
+type delegator interface {
+	http.ResponseWriter
+
+	Status() int
+	Written() int64
+}
+
+type responseWriterDelegator struct {
+	http.ResponseWriter
+
+	handler, method    string
+	status             int
+	written            int64
+	wroteHeader        bool
+	observeWriteHeader func(int)
+}
+
+func (r *responseWriterDelegator) Status() int {
+	return r.status
+}
+
+func (r *responseWriterDelegator) Written() int64 {
+	return r.written
+}
+
+func (r *responseWriterDelegator) WriteHeader(code int) {
+	r.status = code
+	r.wroteHeader = true
+	r.ResponseWriter.WriteHeader(code)
+	if r.observeWriteHeader != nil {
+		r.observeWriteHeader(code)
+	}
+}
+
+func (r *responseWriterDelegator) Write(b []byte) (int, error) {
+	if !r.wroteHeader {
+		r.WriteHeader(http.StatusOK)
+	}
+	n, err := r.ResponseWriter.Write(b)
+	r.written += int64(n)
+	return n, err
+}
+
+type closeNotifierDelegator struct{ *responseWriterDelegator }
+type flusherDelegator struct{ *responseWriterDelegator }
+type hijackerDelegator struct{ *responseWriterDelegator }
+type readerFromDelegator struct{ *responseWriterDelegator }
+
+func (d *closeNotifierDelegator) CloseNotify() <-chan bool {
+	return d.ResponseWriter.(http.CloseNotifier).CloseNotify()
+}
+func (d *flusherDelegator) Flush() {
+	d.ResponseWriter.(http.Flusher).Flush()
+}
+func (d *hijackerDelegator) Hijack() (net.Conn, *bufio.ReadWriter, error) {
+	return d.ResponseWriter.(http.Hijacker).Hijack()
+}
+func (d *readerFromDelegator) ReadFrom(re io.Reader) (int64, error) {
+	if !d.wroteHeader {
+		d.WriteHeader(http.StatusOK)
+	}
+	n, err := d.ResponseWriter.(io.ReaderFrom).ReadFrom(re)
+	d.written += n
+	return n, err
+}
+
+var pickDelegator = make([]func(*responseWriterDelegator) delegator, 32)
+
+func init() {
+	// TODO(beorn7): Code generation would help here.
+	pickDelegator[0] = func(d *responseWriterDelegator) delegator { // 0
+		return d
+	}
+	pickDelegator[closeNotifier] = func(d *responseWriterDelegator) delegator { // 1
+		return closeNotifierDelegator{d}
+	}
+	pickDelegator[flusher] = func(d *responseWriterDelegator) delegator { // 2
+		return flusherDelegator{d}
+	}
+	pickDelegator[flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 3
+		return struct {
+			*responseWriterDelegator
+			http.Flusher
+			http.CloseNotifier
+		}{d, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[hijacker] = func(d *responseWriterDelegator) delegator { // 4
+		return hijackerDelegator{d}
+	}
+	pickDelegator[hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 5
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.CloseNotifier
+		}{d, &hijackerDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 6
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.Flusher
+		}{d, &hijackerDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 7
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, &hijackerDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom] = func(d *responseWriterDelegator) delegator { // 8
+		return readerFromDelegator{d}
+	}
+	pickDelegator[readerFrom+closeNotifier] = func(d *responseWriterDelegator) delegator { // 9
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.CloseNotifier
+		}{d, &readerFromDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+flusher] = func(d *responseWriterDelegator) delegator { // 10
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Flusher
+		}{d, &readerFromDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[readerFrom+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 11
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Flusher
+			http.CloseNotifier
+		}{d, &readerFromDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker] = func(d *responseWriterDelegator) delegator { // 12
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+		}{d, &readerFromDelegator{d}, &hijackerDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 13
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.CloseNotifier
+		}{d, &readerFromDelegator{d}, &hijackerDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 14
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+		}{d, &readerFromDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 15
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, &readerFromDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+}

+ 181 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator_1_8.go

@@ -0,0 +1,181 @@
+// Copyright 2017 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.
+
+// +build go1.8
+
+package promhttp
+
+import (
+	"io"
+	"net/http"
+)
+
+type pusherDelegator struct{ *responseWriterDelegator }
+
+func (d *pusherDelegator) Push(target string, opts *http.PushOptions) error {
+	return d.ResponseWriter.(http.Pusher).Push(target, opts)
+}
+
+func init() {
+	pickDelegator[pusher] = func(d *responseWriterDelegator) delegator { // 16
+		return pusherDelegator{d}
+	}
+	pickDelegator[pusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 17
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+flusher] = func(d *responseWriterDelegator) delegator { // 18
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Flusher
+		}{d, &pusherDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[pusher+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 19
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Flusher
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker] = func(d *responseWriterDelegator) delegator { // 20
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+		}{d, &pusherDelegator{d}, &hijackerDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 21
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &hijackerDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 22
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.Flusher
+		}{d, &pusherDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { //23
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom] = func(d *responseWriterDelegator) delegator { // 24
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+closeNotifier] = func(d *responseWriterDelegator) delegator { // 25
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+flusher] = func(d *responseWriterDelegator) delegator { // 26
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Flusher
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 27
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Flusher
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker] = func(d *responseWriterDelegator) delegator { // 28
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &hijackerDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 29
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &hijackerDelegator{d}, &closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 30
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 31
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, &pusherDelegator{d}, &readerFromDelegator{d}, &hijackerDelegator{d}, &flusherDelegator{d}, &closeNotifierDelegator{d}}
+	}
+}
+
+func newDelegator(w http.ResponseWriter, observeWriteHeaderFunc func(int)) delegator {
+	d := &responseWriterDelegator{
+		ResponseWriter:     w,
+		observeWriteHeader: observeWriteHeaderFunc,
+	}
+
+	id := 0
+	if _, ok := w.(http.CloseNotifier); ok {
+		id += closeNotifier
+	}
+	if _, ok := w.(http.Flusher); ok {
+		id += flusher
+	}
+	if _, ok := w.(http.Hijacker); ok {
+		id += hijacker
+	}
+	if _, ok := w.(io.ReaderFrom); ok {
+		id += readerFrom
+	}
+	if _, ok := w.(http.Pusher); ok {
+		id += pusher
+	}
+
+	return pickDelegator[id](d)
+}

+ 44 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator_pre_1_8.go

@@ -0,0 +1,44 @@
+// Copyright 2017 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.
+
+// +build !go1.8
+
+package promhttp
+
+import (
+	"io"
+	"net/http"
+)
+
+func newDelegator(w http.ResponseWriter, observeWriteHeaderFunc func(int)) delegator {
+	d := &responseWriterDelegator{
+		ResponseWriter:     w,
+		observeWriteHeader: observeWriteHeaderFunc,
+	}
+
+	id := 0
+	if _, ok := w.(http.CloseNotifier); ok {
+		id += closeNotifier
+	}
+	if _, ok := w.(http.Flusher); ok {
+		id += flusher
+	}
+	if _, ok := w.(http.Hijacker); ok {
+		id += hijacker
+	}
+	if _, ok := w.(io.ReaderFrom); ok {
+		id += readerFrom
+	}
+
+	return pickDelegator[id](d)
+}

+ 204 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/http.go

@@ -0,0 +1,204 @@
+// Copyright 2016 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 promhttp provides tooling around HTTP servers and clients.
+//
+// First, the package allows the creation of http.Handler instances to expose
+// Prometheus metrics via HTTP. promhttp.Handler acts on the
+// prometheus.DefaultGatherer. With HandlerFor, you can create a handler for a
+// custom registry or anything that implements the Gatherer interface. It also
+// allows the creation of handlers that act differently on errors or allow to
+// log errors.
+//
+// Second, the package provides tooling to instrument instances of http.Handler
+// via middleware. Middleware wrappers follow the naming scheme
+// InstrumentHandlerX, where X describes the intended use of the middleware.
+// See each function's doc comment for specific details.
+//
+// Finally, the package allows for an http.RoundTripper to be instrumented via
+// middleware. Middleware wrappers follow the naming scheme
+// InstrumentRoundTripperX, where X describes the intended use of the
+// middleware. See each function's doc comment for specific details.
+package promhttp
+
+import (
+	"bytes"
+	"compress/gzip"
+	"fmt"
+	"io"
+	"net/http"
+	"strings"
+	"sync"
+
+	"github.com/prometheus/common/expfmt"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+const (
+	contentTypeHeader     = "Content-Type"
+	contentLengthHeader   = "Content-Length"
+	contentEncodingHeader = "Content-Encoding"
+	acceptEncodingHeader  = "Accept-Encoding"
+)
+
+var bufPool sync.Pool
+
+func getBuf() *bytes.Buffer {
+	buf := bufPool.Get()
+	if buf == nil {
+		return &bytes.Buffer{}
+	}
+	return buf.(*bytes.Buffer)
+}
+
+func giveBuf(buf *bytes.Buffer) {
+	buf.Reset()
+	bufPool.Put(buf)
+}
+
+// Handler returns an HTTP handler for the prometheus.DefaultGatherer. The
+// Handler uses the default HandlerOpts, i.e. report the first error as an HTTP
+// error, no error logging, and compression if requested by the client.
+//
+// If you want to create a Handler for the DefaultGatherer with different
+// HandlerOpts, create it with HandlerFor with prometheus.DefaultGatherer and
+// your desired HandlerOpts.
+func Handler() http.Handler {
+	return HandlerFor(prometheus.DefaultGatherer, HandlerOpts{})
+}
+
+// HandlerFor returns an http.Handler for the provided Gatherer. The behavior
+// of the Handler is defined by the provided HandlerOpts.
+func HandlerFor(reg prometheus.Gatherer, opts HandlerOpts) http.Handler {
+	return http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) {
+		mfs, err := reg.Gather()
+		if err != nil {
+			if opts.ErrorLog != nil {
+				opts.ErrorLog.Println("error gathering metrics:", err)
+			}
+			switch opts.ErrorHandling {
+			case PanicOnError:
+				panic(err)
+			case ContinueOnError:
+				if len(mfs) == 0 {
+					http.Error(w, "No metrics gathered, last error:\n\n"+err.Error(), http.StatusInternalServerError)
+					return
+				}
+			case HTTPErrorOnError:
+				http.Error(w, "An error has occurred during metrics gathering:\n\n"+err.Error(), http.StatusInternalServerError)
+				return
+			}
+		}
+
+		contentType := expfmt.Negotiate(req.Header)
+		buf := getBuf()
+		defer giveBuf(buf)
+		writer, encoding := decorateWriter(req, buf, opts.DisableCompression)
+		enc := expfmt.NewEncoder(writer, contentType)
+		var lastErr error
+		for _, mf := range mfs {
+			if err := enc.Encode(mf); err != nil {
+				lastErr = err
+				if opts.ErrorLog != nil {
+					opts.ErrorLog.Println("error encoding metric family:", err)
+				}
+				switch opts.ErrorHandling {
+				case PanicOnError:
+					panic(err)
+				case ContinueOnError:
+					// Handled later.
+				case HTTPErrorOnError:
+					http.Error(w, "An error has occurred during metrics encoding:\n\n"+err.Error(), http.StatusInternalServerError)
+					return
+				}
+			}
+		}
+		if closer, ok := writer.(io.Closer); ok {
+			closer.Close()
+		}
+		if lastErr != nil && buf.Len() == 0 {
+			http.Error(w, "No metrics encoded, last error:\n\n"+lastErr.Error(), http.StatusInternalServerError)
+			return
+		}
+		header := w.Header()
+		header.Set(contentTypeHeader, string(contentType))
+		header.Set(contentLengthHeader, fmt.Sprint(buf.Len()))
+		if encoding != "" {
+			header.Set(contentEncodingHeader, encoding)
+		}
+		w.Write(buf.Bytes())
+		// TODO(beorn7): Consider streaming serving of metrics.
+	})
+}
+
+// HandlerErrorHandling defines how a Handler serving metrics will handle
+// errors.
+type HandlerErrorHandling int
+
+// These constants cause handlers serving metrics to behave as described if
+// errors are encountered.
+const (
+	// Serve an HTTP status code 500 upon the first error
+	// encountered. Report the error message in the body.
+	HTTPErrorOnError HandlerErrorHandling = iota
+	// Ignore errors and try to serve as many metrics as possible.  However,
+	// if no metrics can be served, serve an HTTP status code 500 and the
+	// last error message in the body. Only use this in deliberate "best
+	// effort" metrics collection scenarios. It is recommended to at least
+	// log errors (by providing an ErrorLog in HandlerOpts) to not mask
+	// errors completely.
+	ContinueOnError
+	// Panic upon the first error encountered (useful for "crash only" apps).
+	PanicOnError
+)
+
+// Logger is the minimal interface HandlerOpts needs for logging. Note that
+// log.Logger from the standard library implements this interface, and it is
+// easy to implement by custom loggers, if they don't do so already anyway.
+type Logger interface {
+	Println(v ...interface{})
+}
+
+// HandlerOpts specifies options how to serve metrics via an http.Handler. The
+// zero value of HandlerOpts is a reasonable default.
+type HandlerOpts struct {
+	// ErrorLog specifies an optional logger for errors collecting and
+	// serving metrics. If nil, errors are not logged at all.
+	ErrorLog Logger
+	// ErrorHandling defines how errors are handled. Note that errors are
+	// logged regardless of the configured ErrorHandling provided ErrorLog
+	// is not nil.
+	ErrorHandling HandlerErrorHandling
+	// If DisableCompression is true, the handler will never compress the
+	// response, even if requested by the client.
+	DisableCompression bool
+}
+
+// decorateWriter wraps a writer to handle gzip compression if requested.  It
+// returns the decorated writer and the appropriate "Content-Encoding" header
+// (which is empty if no compression is enabled).
+func decorateWriter(request *http.Request, writer io.Writer, compressionDisabled bool) (io.Writer, string) {
+	if compressionDisabled {
+		return writer, ""
+	}
+	header := request.Header.Get(acceptEncodingHeader)
+	parts := strings.Split(header, ",")
+	for _, part := range parts {
+		part := strings.TrimSpace(part)
+		if part == "gzip" || strings.HasPrefix(part, "gzip;") {
+			return gzip.NewWriter(writer), "gzip"
+		}
+	}
+	return writer, ""
+}

+ 98 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client.go

@@ -0,0 +1,98 @@
+// Copyright 2017 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 promhttp
+
+import (
+	"net/http"
+	"time"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+// The RoundTripperFunc type is an adapter to allow the use of ordinary
+// functions as RoundTrippers. If f is a function with the appropriate
+// signature, RountTripperFunc(f) is a RoundTripper that calls f.
+type RoundTripperFunc func(req *http.Request) (*http.Response, error)
+
+// RoundTrip implements the RoundTripper interface.
+func (rt RoundTripperFunc) RoundTrip(r *http.Request) (*http.Response, error) {
+	return rt(r)
+}
+
+// InstrumentRoundTripperInFlight is a middleware that wraps the provided
+// http.RoundTripper. It sets the provided prometheus.Gauge to the number of
+// requests currently handled by the wrapped http.RoundTripper.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperInFlight(gauge prometheus.Gauge, next http.RoundTripper) RoundTripperFunc {
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		gauge.Inc()
+		defer gauge.Dec()
+		return next.RoundTrip(r)
+	})
+}
+
+// InstrumentRoundTripperCounter is a middleware that wraps the provided
+// http.RoundTripper to observe the request result with the provided CounterVec.
+// The CounterVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. Partitioning of the CounterVec happens by HTTP status
+// code and/or HTTP method if the respective instance label names are present
+// in the CounterVec. For unpartitioned counting, use a CounterVec with
+// zero labels.
+//
+// If the wrapped RoundTripper panics or returns a non-nil error, the Counter
+// is not incremented.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperCounter(counter *prometheus.CounterVec, next http.RoundTripper) RoundTripperFunc {
+	code, method := checkLabels(counter)
+
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		resp, err := next.RoundTrip(r)
+		if err == nil {
+			counter.With(labels(code, method, r.Method, resp.StatusCode)).Inc()
+		}
+		return resp, err
+	})
+}
+
+// InstrumentRoundTripperDuration is a middleware that wraps the provided
+// http.RoundTripper to observe the request duration with the provided ObserverVec.
+// The ObserverVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. The Observe method of the Observer in the ObserverVec
+// is called with the request duration in seconds. Partitioning happens by HTTP
+// status code and/or HTTP method if the respective instance label names are
+// present in the ObserverVec. For unpartitioned observations, use an
+// ObserverVec with zero labels. Note that partitioning of Histograms is
+// expensive and should be used judiciously.
+//
+// If the wrapped RoundTripper panics or returns a non-nil error, no values are
+// reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func InstrumentRoundTripperDuration(obs prometheus.ObserverVec, next http.RoundTripper) RoundTripperFunc {
+	code, method := checkLabels(obs)
+
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		start := time.Now()
+		resp, err := next.RoundTrip(r)
+		if err == nil {
+			obs.With(labels(code, method, r.Method, resp.StatusCode)).Observe(time.Since(start).Seconds())
+		}
+		return resp, err
+	})
+}

+ 144 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client_1_8.go

@@ -0,0 +1,144 @@
+// Copyright 2017 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.
+
+// +build go1.8
+
+package promhttp
+
+import (
+	"context"
+	"crypto/tls"
+	"net/http"
+	"net/http/httptrace"
+	"time"
+)
+
+// InstrumentTrace is used to offer flexibility in instrumenting the available
+// httptrace.ClientTrace hook functions. Each function is passed a float64
+// representing the time in seconds since the start of the http request. A user
+// may choose to use separately buckets Histograms, or implement custom
+// instance labels on a per function basis.
+type InstrumentTrace struct {
+	GotConn              func(float64)
+	PutIdleConn          func(float64)
+	GotFirstResponseByte func(float64)
+	Got100Continue       func(float64)
+	DNSStart             func(float64)
+	DNSDone              func(float64)
+	ConnectStart         func(float64)
+	ConnectDone          func(float64)
+	TLSHandshakeStart    func(float64)
+	TLSHandshakeDone     func(float64)
+	WroteHeaders         func(float64)
+	Wait100Continue      func(float64)
+	WroteRequest         func(float64)
+}
+
+// InstrumentRoundTripperTrace is a middleware that wraps the provided
+// RoundTripper and reports times to hook functions provided in the
+// InstrumentTrace struct. Hook functions that are not present in the provided
+// InstrumentTrace struct are ignored. Times reported to the hook functions are
+// time since the start of the request. Only with Go1.9+, those times are
+// guaranteed to never be negative. (Earlier Go versions are not using a
+// monotonic clock.) Note that partitioning of Histograms is expensive and
+// should be used judiciously.
+//
+// For hook functions that receive an error as an argument, no observations are
+// made in the event of a non-nil error value.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperTrace(it *InstrumentTrace, next http.RoundTripper) RoundTripperFunc {
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		start := time.Now()
+
+		trace := &httptrace.ClientTrace{
+			GotConn: func(_ httptrace.GotConnInfo) {
+				if it.GotConn != nil {
+					it.GotConn(time.Since(start).Seconds())
+				}
+			},
+			PutIdleConn: func(err error) {
+				if err != nil {
+					return
+				}
+				if it.PutIdleConn != nil {
+					it.PutIdleConn(time.Since(start).Seconds())
+				}
+			},
+			DNSStart: func(_ httptrace.DNSStartInfo) {
+				if it.DNSStart != nil {
+					it.DNSStart(time.Since(start).Seconds())
+				}
+			},
+			DNSDone: func(_ httptrace.DNSDoneInfo) {
+				if it.DNSStart != nil {
+					it.DNSStart(time.Since(start).Seconds())
+				}
+			},
+			ConnectStart: func(_, _ string) {
+				if it.ConnectStart != nil {
+					it.ConnectStart(time.Since(start).Seconds())
+				}
+			},
+			ConnectDone: func(_, _ string, err error) {
+				if err != nil {
+					return
+				}
+				if it.ConnectDone != nil {
+					it.ConnectDone(time.Since(start).Seconds())
+				}
+			},
+			GotFirstResponseByte: func() {
+				if it.GotFirstResponseByte != nil {
+					it.GotFirstResponseByte(time.Since(start).Seconds())
+				}
+			},
+			Got100Continue: func() {
+				if it.Got100Continue != nil {
+					it.Got100Continue(time.Since(start).Seconds())
+				}
+			},
+			TLSHandshakeStart: func() {
+				if it.TLSHandshakeStart != nil {
+					it.TLSHandshakeStart(time.Since(start).Seconds())
+				}
+			},
+			TLSHandshakeDone: func(_ tls.ConnectionState, err error) {
+				if err != nil {
+					return
+				}
+				if it.TLSHandshakeDone != nil {
+					it.TLSHandshakeDone(time.Since(start).Seconds())
+				}
+			},
+			WroteHeaders: func() {
+				if it.WroteHeaders != nil {
+					it.WroteHeaders(time.Since(start).Seconds())
+				}
+			},
+			Wait100Continue: func() {
+				if it.Wait100Continue != nil {
+					it.Wait100Continue(time.Since(start).Seconds())
+				}
+			},
+			WroteRequest: func(_ httptrace.WroteRequestInfo) {
+				if it.WroteRequest != nil {
+					it.WroteRequest(time.Since(start).Seconds())
+				}
+			},
+		}
+		r = r.WithContext(httptrace.WithClientTrace(context.Background(), trace))
+
+		return next.RoundTrip(r)
+	})
+}

+ 440 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_server.go

@@ -0,0 +1,440 @@
+// Copyright 2017 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 promhttp
+
+import (
+	"net/http"
+	"strconv"
+	"strings"
+	"time"
+
+	dto "github.com/prometheus/client_model/go"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+// magicString is used for the hacky label test in checkLabels. Remove once fixed.
+const magicString = "zZgWfBxLqvG8kc8IMv3POi2Bb0tZI3vAnBx+gBaFi9FyPzB/CzKUer1yufDa"
+
+// InstrumentHandlerInFlight is a middleware that wraps the provided
+// http.Handler. It sets the provided prometheus.Gauge to the number of
+// requests currently handled by the wrapped http.Handler.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerInFlight(g prometheus.Gauge, next http.Handler) http.Handler {
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		g.Inc()
+		defer g.Dec()
+		next.ServeHTTP(w, r)
+	})
+}
+
+// InstrumentHandlerDuration is a middleware that wraps the provided
+// http.Handler to observe the request duration with the provided ObserverVec.
+// The ObserverVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. The Observe method of the Observer in the ObserverVec
+// is called with the request duration in seconds. Partitioning happens by HTTP
+// status code and/or HTTP method if the respective instance label names are
+// present in the ObserverVec. For unpartitioned observations, use an
+// ObserverVec with zero labels. Note that partitioning of Histograms is
+// expensive and should be used judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func InstrumentHandlerDuration(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			now := time.Now()
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+
+			obs.With(labels(code, method, r.Method, d.Status())).Observe(time.Since(now).Seconds())
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		now := time.Now()
+		next.ServeHTTP(w, r)
+		obs.With(labels(code, method, r.Method, 0)).Observe(time.Since(now).Seconds())
+	})
+}
+
+// InstrumentHandlerCounter is a middleware that wraps the provided
+// http.Handler to observe the request result with the provided CounterVec.
+// The CounterVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. Partitioning of the CounterVec happens by HTTP status
+// code and/or HTTP method if the respective instance label names are present
+// in the CounterVec. For unpartitioned counting, use a CounterVec with
+// zero labels.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, the Counter is not incremented.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerCounter(counter *prometheus.CounterVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(counter)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+			counter.With(labels(code, method, r.Method, d.Status())).Inc()
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		next.ServeHTTP(w, r)
+		counter.With(labels(code, method, r.Method, 0)).Inc()
+	})
+}
+
+// InstrumentHandlerTimeToWriteHeader is a middleware that wraps the provided
+// http.Handler to observe with the provided ObserverVec the request duration
+// until the response headers are written. The ObserverVec must have zero, one,
+// or two labels. The only allowed label names are "code" and "method". The
+// function panics if any other instance labels are provided. The Observe
+// method of the Observer in the ObserverVec is called with the request
+// duration in seconds. Partitioning happens by HTTP status code and/or HTTP
+// method if the respective instance label names are present in the
+// ObserverVec. For unpartitioned observations, use an ObserverVec with zero
+// labels. Note that partitioning of Histograms is expensive and should be used
+// judiciously.
+//
+// If the wrapped Handler panics before calling WriteHeader, no value is
+// reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerTimeToWriteHeader(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		now := time.Now()
+		d := newDelegator(w, func(status int) {
+			obs.With(labels(code, method, r.Method, status)).Observe(time.Since(now).Seconds())
+		})
+		next.ServeHTTP(d, r)
+	})
+}
+
+// InstrumentHandlerRequestSize is a middleware that wraps the provided
+// http.Handler to observe the request size with the provided ObserverVec.
+// The ObserverVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. The Observe method of the Observer in the ObserverVec
+// is called with the request size in bytes. Partitioning happens by HTTP
+// status code and/or HTTP method if the respective instance label names are
+// present in the ObserverVec. For unpartitioned observations, use an
+// ObserverVec with zero labels. Note that partitioning of Histograms is
+// expensive and should be used judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerRequestSize(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+			size := computeApproximateRequestSize(r)
+			obs.With(labels(code, method, r.Method, d.Status())).Observe(float64(size))
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		next.ServeHTTP(w, r)
+		size := computeApproximateRequestSize(r)
+		obs.With(labels(code, method, r.Method, 0)).Observe(float64(size))
+	})
+}
+
+// InstrumentHandlerResponseSize is a middleware that wraps the provided
+// http.Handler to observe the response size with the provided ObserverVec.
+// The ObserverVec must have zero, one, or two labels. The only allowed label
+// names are "code" and "method". The function panics if any other instance
+// labels are provided. The Observe method of the Observer in the ObserverVec
+// is called with the response size in bytes. Partitioning happens by HTTP
+// status code and/or HTTP method if the respective instance label names are
+// present in the ObserverVec. For unpartitioned observations, use an
+// ObserverVec with zero labels. Note that partitioning of Histograms is
+// expensive and should be used judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerResponseSize(obs prometheus.ObserverVec, next http.Handler) http.Handler {
+	code, method := checkLabels(obs)
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		d := newDelegator(w, nil)
+		next.ServeHTTP(d, r)
+		obs.With(labels(code, method, r.Method, d.Status())).Observe(float64(d.Written()))
+	})
+}
+
+func checkLabels(c prometheus.Collector) (code bool, method bool) {
+	// TODO(beorn7): Remove this hacky way to check for instance labels
+	// once Descriptors can have their dimensionality queried.
+	var (
+		desc *prometheus.Desc
+		pm   dto.Metric
+	)
+
+	descc := make(chan *prometheus.Desc, 1)
+	c.Describe(descc)
+
+	select {
+	case desc = <-descc:
+	default:
+		panic("no description provided by collector")
+	}
+	select {
+	case <-descc:
+		panic("more than one description provided by collector")
+	default:
+	}
+
+	close(descc)
+
+	if _, err := prometheus.NewConstMetric(desc, prometheus.UntypedValue, 0); err == nil {
+		return
+	}
+	if m, err := prometheus.NewConstMetric(desc, prometheus.UntypedValue, 0, magicString); err == nil {
+		if err := m.Write(&pm); err != nil {
+			panic("error checking metric for labels")
+		}
+		for _, label := range pm.Label {
+			name, value := label.GetName(), label.GetValue()
+			if value != magicString {
+				continue
+			}
+			switch name {
+			case "code":
+				code = true
+			case "method":
+				method = true
+			default:
+				panic("metric partitioned with non-supported labels")
+			}
+			return
+		}
+		panic("previously set label not found – this must never happen")
+	}
+	if m, err := prometheus.NewConstMetric(desc, prometheus.UntypedValue, 0, magicString, magicString); err == nil {
+		if err := m.Write(&pm); err != nil {
+			panic("error checking metric for labels")
+		}
+		for _, label := range pm.Label {
+			name, value := label.GetName(), label.GetValue()
+			if value != magicString {
+				continue
+			}
+			if name == "code" || name == "method" {
+				continue
+			}
+			panic("metric partitioned with non-supported labels")
+		}
+		code = true
+		method = true
+		return
+	}
+	panic("metric partitioned with non-supported labels")
+}
+
+// emptyLabels is a one-time allocation for non-partitioned metrics to avoid
+// unnecessary allocations on each request.
+var emptyLabels = prometheus.Labels{}
+
+func labels(code, method bool, reqMethod string, status int) prometheus.Labels {
+	if !(code || method) {
+		return emptyLabels
+	}
+	labels := prometheus.Labels{}
+
+	if code {
+		labels["code"] = sanitizeCode(status)
+	}
+	if method {
+		labels["method"] = sanitizeMethod(reqMethod)
+	}
+
+	return labels
+}
+
+func computeApproximateRequestSize(r *http.Request) int {
+	s := 0
+	if r.URL != nil {
+		s += len(r.URL.String())
+	}
+
+	s += len(r.Method)
+	s += len(r.Proto)
+	for name, values := range r.Header {
+		s += len(name)
+		for _, value := range values {
+			s += len(value)
+		}
+	}
+	s += len(r.Host)
+
+	// N.B. r.Form and r.MultipartForm are assumed to be included in r.URL.
+
+	if r.ContentLength != -1 {
+		s += int(r.ContentLength)
+	}
+	return s
+}
+
+func sanitizeMethod(m string) string {
+	switch m {
+	case "GET", "get":
+		return "get"
+	case "PUT", "put":
+		return "put"
+	case "HEAD", "head":
+		return "head"
+	case "POST", "post":
+		return "post"
+	case "DELETE", "delete":
+		return "delete"
+	case "CONNECT", "connect":
+		return "connect"
+	case "OPTIONS", "options":
+		return "options"
+	case "NOTIFY", "notify":
+		return "notify"
+	default:
+		return strings.ToLower(m)
+	}
+}
+
+// If the wrapped http.Handler has not set a status code, i.e. the value is
+// currently 0, santizeCode will return 200, for consistency with behavior in
+// the stdlib.
+func sanitizeCode(s int) string {
+	switch s {
+	case 100:
+		return "100"
+	case 101:
+		return "101"
+
+	case 200, 0:
+		return "200"
+	case 201:
+		return "201"
+	case 202:
+		return "202"
+	case 203:
+		return "203"
+	case 204:
+		return "204"
+	case 205:
+		return "205"
+	case 206:
+		return "206"
+
+	case 300:
+		return "300"
+	case 301:
+		return "301"
+	case 302:
+		return "302"
+	case 304:
+		return "304"
+	case 305:
+		return "305"
+	case 307:
+		return "307"
+
+	case 400:
+		return "400"
+	case 401:
+		return "401"
+	case 402:
+		return "402"
+	case 403:
+		return "403"
+	case 404:
+		return "404"
+	case 405:
+		return "405"
+	case 406:
+		return "406"
+	case 407:
+		return "407"
+	case 408:
+		return "408"
+	case 409:
+		return "409"
+	case 410:
+		return "410"
+	case 411:
+		return "411"
+	case 412:
+		return "412"
+	case 413:
+		return "413"
+	case 414:
+		return "414"
+	case 415:
+		return "415"
+	case 416:
+		return "416"
+	case 417:
+		return "417"
+	case 418:
+		return "418"
+
+	case 500:
+		return "500"
+	case 501:
+		return "501"
+	case 502:
+		return "502"
+	case 503:
+		return "503"
+	case 504:
+		return "504"
+	case 505:
+		return "505"
+
+	case 428:
+		return "428"
+	case 429:
+		return "429"
+	case 431:
+		return "431"
+	case 511:
+		return "511"
+
+	default:
+		return strconv.Itoa(s)
+	}
+}

+ 12 - 56
cmd/vendor/github.com/prometheus/client_golang/prometheus/registry.go

@@ -20,6 +20,7 @@ import (
 	"os"
 	"sort"
 	"sync"
+	"unicode/utf8"
 
 	"github.com/golang/protobuf/proto"
 
@@ -80,7 +81,7 @@ func NewPedanticRegistry() *Registry {
 
 // Registerer is the interface for the part of a registry in charge of
 // registering and unregistering. Users of custom registries should use
-// Registerer as type for registration purposes (rather then the Registry type
+// Registerer as type for registration purposes (rather than the Registry type
 // directly). In that way, they are free to use custom Registerer implementation
 // (e.g. for testing purposes).
 type Registerer interface {
@@ -152,38 +153,6 @@ func MustRegister(cs ...Collector) {
 	DefaultRegisterer.MustRegister(cs...)
 }
 
-// RegisterOrGet registers the provided Collector with the DefaultRegisterer and
-// returns the Collector, unless an equal Collector was registered before, in
-// which case that Collector is returned.
-//
-// Deprecated: RegisterOrGet is merely a convenience function for the
-// implementation as described in the documentation for
-// AlreadyRegisteredError. As the use case is relatively rare, this function
-// will be removed in a future version of this package to clean up the
-// namespace.
-func RegisterOrGet(c Collector) (Collector, error) {
-	if err := Register(c); err != nil {
-		if are, ok := err.(AlreadyRegisteredError); ok {
-			return are.ExistingCollector, nil
-		}
-		return nil, err
-	}
-	return c, nil
-}
-
-// MustRegisterOrGet behaves like RegisterOrGet but panics instead of returning
-// an error.
-//
-// Deprecated: This is deprecated for the same reason RegisterOrGet is. See
-// there for details.
-func MustRegisterOrGet(c Collector) Collector {
-	c, err := RegisterOrGet(c)
-	if err != nil {
-		panic(err)
-	}
-	return c
-}
-
 // Unregister removes the registration of the provided Collector from the
 // DefaultRegisterer.
 //
@@ -201,25 +170,6 @@ func (gf GathererFunc) Gather() ([]*dto.MetricFamily, error) {
 	return gf()
 }
 
-// SetMetricFamilyInjectionHook replaces the DefaultGatherer with one that
-// gathers from the previous DefaultGatherers but then merges the MetricFamily
-// protobufs returned from the provided hook function with the MetricFamily
-// protobufs returned from the original DefaultGatherer.
-//
-// Deprecated: This function manipulates the DefaultGatherer variable. Consider
-// the implications, i.e. don't do this concurrently with any uses of the
-// DefaultGatherer. In the rare cases where you need to inject MetricFamily
-// protobufs directly, it is recommended to use a custom Registry and combine it
-// with a custom Gatherer using the Gatherers type (see
-// there). SetMetricFamilyInjectionHook only exists for compatibility reasons
-// with previous versions of this package.
-func SetMetricFamilyInjectionHook(hook func() []*dto.MetricFamily) {
-	DefaultGatherer = Gatherers{
-		DefaultGatherer,
-		GathererFunc(func() ([]*dto.MetricFamily, error) { return hook(), nil }),
-	}
-}
-
 // AlreadyRegisteredError is returned by the Register method if the Collector to
 // be registered has already been registered before, or a different Collector
 // that collects the same metrics has been registered before. Registration fails
@@ -294,7 +244,7 @@ func (r *Registry) Register(c Collector) error {
 	}()
 	r.mtx.Lock()
 	defer r.mtx.Unlock()
-	// Coduct various tests...
+	// Conduct various tests...
 	for desc := range descChan {
 
 		// Is the descriptor valid at all?
@@ -447,7 +397,7 @@ func (r *Registry) Gather() ([]*dto.MetricFamily, error) {
 
 	// Drain metricChan in case of premature return.
 	defer func() {
-		for _ = range metricChan {
+		for range metricChan {
 		}
 	}()
 
@@ -683,7 +633,7 @@ func (s metricSorter) Less(i, j int) bool {
 	return s[i].GetTimestampMs() < s[j].GetTimestampMs()
 }
 
-// normalizeMetricFamilies returns a MetricFamily slice whith empty
+// normalizeMetricFamilies returns a MetricFamily slice with empty
 // MetricFamilies pruned and the remaining MetricFamilies sorted by name within
 // the slice, with the contained Metrics sorted within each MetricFamily.
 func normalizeMetricFamilies(metricFamiliesByName map[string]*dto.MetricFamily) []*dto.MetricFamily {
@@ -706,7 +656,7 @@ func normalizeMetricFamilies(metricFamiliesByName map[string]*dto.MetricFamily)
 
 // checkMetricConsistency checks if the provided Metric is consistent with the
 // provided MetricFamily. It also hashed the Metric labels and the MetricFamily
-// name. If the resulting hash is alread in the provided metricHashes, an error
+// name. If the resulting hash is already in the provided metricHashes, an error
 // is returned. If not, it is added to metricHashes. The provided dimHashes maps
 // MetricFamily names to their dimHash (hashed sorted label names). If dimHashes
 // doesn't yet contain a hash for the provided MetricFamily, it is
@@ -730,6 +680,12 @@ func checkMetricConsistency(
 		)
 	}
 
+	for _, labelPair := range dtoMetric.GetLabel() {
+		if !utf8.ValidString(*labelPair.Value) {
+			return fmt.Errorf("collected metric's label %s is not utf8: %#v", *labelPair.Name, *labelPair.Value)
+		}
+	}
+
 	// Is the metric unique (i.e. no other metric with the same name and the same label values)?
 	h := hashNew()
 	h = hashAdd(h, metricFamily.GetName())

+ 72 - 29
cmd/vendor/github.com/prometheus/client_golang/prometheus/summary.go

@@ -36,7 +36,10 @@ const quantileLabel = "quantile"
 //
 // A typical use-case is the observation of request latencies. By default, a
 // Summary provides the median, the 90th and the 99th percentile of the latency
-// as rank estimations.
+// as rank estimations. However, the default behavior will change in the
+// upcoming v0.10 of the library. There will be no rank estiamtions at all by
+// default. For a sane transition, it is recommended to set the desired rank
+// estimations explicitly.
 //
 // Note that the rank estimations cannot be aggregated in a meaningful way with
 // the Prometheus query language (i.e. you cannot average or add them). If you
@@ -54,6 +57,9 @@ type Summary interface {
 }
 
 // DefObjectives are the default Summary quantile values.
+//
+// Deprecated: DefObjectives will not be used as the default objectives in
+// v0.10 of the library. The default Summary will have no quantiles then.
 var (
 	DefObjectives = map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001}
 
@@ -75,8 +81,10 @@ const (
 )
 
 // SummaryOpts bundles the options for creating a Summary metric. It is
-// mandatory to set Name and Help to a non-empty string. All other fields are
-// optional and can safely be left at their zero value.
+// mandatory to set Name and Help to a non-empty string. While all other fields
+// are optional and can safely be left at their zero value, it is recommended to
+// explicitly set the Objectives field to the desired value as the default value
+// will change in the upcoming v0.10 of the library.
 type SummaryOpts struct {
 	// Namespace, Subsystem, and Name are components of the fully-qualified
 	// name of the Summary (created by joining these components with
@@ -113,9 +121,15 @@ type SummaryOpts struct {
 	ConstLabels Labels
 
 	// Objectives defines the quantile rank estimates with their respective
-	// absolute error. If Objectives[q] = e, then the value reported
-	// for q will be the φ-quantile value for some φ between q-e and q+e.
-	// The default value is DefObjectives.
+	// absolute error. If Objectives[q] = e, then the value reported for q
+	// will be the φ-quantile value for some φ between q-e and q+e.  The
+	// default value is DefObjectives. It is used if Objectives is left at
+	// its zero value (i.e. nil). To create a Summary without Objectives,
+	// set it to an empty map (i.e. map[float64]float64{}).
+	//
+	// Deprecated: Note that the current value of DefObjectives is
+	// deprecated. It will be replaced by an empty map in v0.10 of the
+	// library. Please explicitly set Objectives to the desired value.
 	Objectives map[float64]float64
 
 	// MaxAge defines the duration for which an observation stays relevant
@@ -183,7 +197,7 @@ func newSummary(desc *Desc, opts SummaryOpts, labelValues ...string) Summary {
 		}
 	}
 
-	if len(opts.Objectives) == 0 {
+	if opts.Objectives == nil {
 		opts.Objectives = DefObjectives
 	}
 
@@ -390,12 +404,11 @@ func (s quantSort) Less(i, j int) bool {
 // (e.g. HTTP request latencies, partitioned by status code and method). Create
 // instances with NewSummaryVec.
 type SummaryVec struct {
-	*MetricVec
+	*metricVec
 }
 
 // NewSummaryVec creates a new SummaryVec based on the provided SummaryOpts and
-// partitioned by the given label names. At least one label name must be
-// provided.
+// partitioned by the given label names.
 func NewSummaryVec(opts SummaryOpts, labelNames []string) *SummaryVec {
 	desc := NewDesc(
 		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
@@ -404,30 +417,60 @@ func NewSummaryVec(opts SummaryOpts, labelNames []string) *SummaryVec {
 		opts.ConstLabels,
 	)
 	return &SummaryVec{
-		MetricVec: newMetricVec(desc, func(lvs ...string) Metric {
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
 			return newSummary(desc, opts, lvs...)
 		}),
 	}
 }
 
-// GetMetricWithLabelValues replaces the method of the same name in
-// MetricVec. The difference is that this method returns a Summary and not a
-// Metric so that no type conversion is required.
-func (m *SummaryVec) GetMetricWithLabelValues(lvs ...string) (Summary, error) {
-	metric, err := m.MetricVec.GetMetricWithLabelValues(lvs...)
+// GetMetricWithLabelValues returns the Summary for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Summary is created.
+//
+// It is possible to call this method without using the returned Summary to only
+// create the new Summary but leave it at its starting value, a Summary without
+// any observations.
+//
+// Keeping the Summary for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Summary from the SummaryVec. In that case, the
+// Summary will still exist, but it will not be exported anymore, even if a
+// Summary with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc.
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
+func (m *SummaryVec) GetMetricWithLabelValues(lvs ...string) (Observer, error) {
+	metric, err := m.metricVec.getMetricWithLabelValues(lvs...)
 	if metric != nil {
-		return metric.(Summary), err
+		return metric.(Observer), err
 	}
 	return nil, err
 }
 
-// GetMetricWith replaces the method of the same name in MetricVec. The
-// difference is that this method returns a Summary and not a Metric so that no
-// type conversion is required.
-func (m *SummaryVec) GetMetricWith(labels Labels) (Summary, error) {
-	metric, err := m.MetricVec.GetMetricWith(labels)
+// GetMetricWith returns the Summary for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Summary is created. Implications of
+// creating a Summary without using it and keeping the Summary for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc.
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (m *SummaryVec) GetMetricWith(labels Labels) (Observer, error) {
+	metric, err := m.metricVec.getMetricWith(labels)
 	if metric != nil {
-		return metric.(Summary), err
+		return metric.(Observer), err
 	}
 	return nil, err
 }
@@ -436,15 +479,15 @@ func (m *SummaryVec) GetMetricWith(labels Labels) (Summary, error) {
 // GetMetricWithLabelValues would have returned an error. By not returning an
 // error, WithLabelValues allows shortcuts like
 //     myVec.WithLabelValues("404", "GET").Observe(42.21)
-func (m *SummaryVec) WithLabelValues(lvs ...string) Summary {
-	return m.MetricVec.WithLabelValues(lvs...).(Summary)
+func (m *SummaryVec) WithLabelValues(lvs ...string) Observer {
+	return m.metricVec.withLabelValues(lvs...).(Observer)
 }
 
 // With works as GetMetricWith, but panics where GetMetricWithLabels would have
 // returned an error. By not returning an error, With allows shortcuts like
 //     myVec.With(Labels{"code": "404", "method": "GET"}).Observe(42.21)
-func (m *SummaryVec) With(labels Labels) Summary {
-	return m.MetricVec.With(labels).(Summary)
+func (m *SummaryVec) With(labels Labels) Observer {
+	return m.metricVec.with(labels).(Observer)
 }
 
 type constSummary struct {
@@ -505,8 +548,8 @@ func NewConstSummary(
 	quantiles map[float64]float64,
 	labelValues ...string,
 ) (Metric, error) {
-	if len(desc.variableLabels) != len(labelValues) {
-		return nil, errInconsistentCardinality
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
 	}
 	return &constSummary{
 		desc:       desc,

+ 51 - 0
cmd/vendor/github.com/prometheus/client_golang/prometheus/timer.go

@@ -0,0 +1,51 @@
+// Copyright 2016 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 prometheus
+
+import "time"
+
+// Timer is a helper type to time functions. Use NewTimer to create new
+// instances.
+type Timer struct {
+	begin    time.Time
+	observer Observer
+}
+
+// NewTimer creates a new Timer. The provided Observer is used to observe a
+// duration in seconds. Timer is usually used to time a function call in the
+// following way:
+//    func TimeMe() {
+//        timer := NewTimer(myHistogram)
+//        defer timer.ObserveDuration()
+//        // Do actual work.
+//    }
+func NewTimer(o Observer) *Timer {
+	return &Timer{
+		begin:    time.Now(),
+		observer: o,
+	}
+}
+
+// ObserveDuration records the duration passed since the Timer was created with
+// NewTimer. It calls the Observe method of the Observer provided during
+// construction with the duration in seconds as an argument. ObserveDuration is
+// usually called with a defer statement.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func (t *Timer) ObserveDuration() {
+	if t.observer != nil {
+		t.observer.Observe(time.Since(t.begin).Seconds())
+	}
+}

+ 3 - 99
cmd/vendor/github.com/prometheus/client_golang/prometheus/untyped.go

@@ -13,108 +13,12 @@
 
 package prometheus
 
-// Untyped is a Metric that represents a single numerical value that can
-// arbitrarily go up and down.
-//
-// An Untyped metric works the same as a Gauge. The only difference is that to
-// no type information is implied.
-//
-// To create Untyped instances, use NewUntyped.
-type Untyped interface {
-	Metric
-	Collector
-
-	// Set sets the Untyped metric to an arbitrary value.
-	Set(float64)
-	// Inc increments the Untyped metric by 1.
-	Inc()
-	// Dec decrements the Untyped metric by 1.
-	Dec()
-	// Add adds the given value to the Untyped metric. (The value can be
-	// negative, resulting in a decrease.)
-	Add(float64)
-	// Sub subtracts the given value from the Untyped metric. (The value can
-	// be negative, resulting in an increase.)
-	Sub(float64)
-}
-
 // UntypedOpts is an alias for Opts. See there for doc comments.
 type UntypedOpts Opts
 
-// NewUntyped creates a new Untyped metric from the provided UntypedOpts.
-func NewUntyped(opts UntypedOpts) Untyped {
-	return newValue(NewDesc(
-		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
-		opts.Help,
-		nil,
-		opts.ConstLabels,
-	), UntypedValue, 0)
-}
-
-// UntypedVec is a Collector that bundles a set of Untyped metrics that all
-// share the same Desc, but have different values for their variable
-// labels. This is used if you want to count the same thing partitioned by
-// various dimensions. Create instances with NewUntypedVec.
-type UntypedVec struct {
-	*MetricVec
-}
-
-// NewUntypedVec creates a new UntypedVec based on the provided UntypedOpts and
-// partitioned by the given label names. At least one label name must be
-// provided.
-func NewUntypedVec(opts UntypedOpts, labelNames []string) *UntypedVec {
-	desc := NewDesc(
-		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
-		opts.Help,
-		labelNames,
-		opts.ConstLabels,
-	)
-	return &UntypedVec{
-		MetricVec: newMetricVec(desc, func(lvs ...string) Metric {
-			return newValue(desc, UntypedValue, 0, lvs...)
-		}),
-	}
-}
-
-// GetMetricWithLabelValues replaces the method of the same name in
-// MetricVec. The difference is that this method returns an Untyped and not a
-// Metric so that no type conversion is required.
-func (m *UntypedVec) GetMetricWithLabelValues(lvs ...string) (Untyped, error) {
-	metric, err := m.MetricVec.GetMetricWithLabelValues(lvs...)
-	if metric != nil {
-		return metric.(Untyped), err
-	}
-	return nil, err
-}
-
-// GetMetricWith replaces the method of the same name in MetricVec. The
-// difference is that this method returns an Untyped and not a Metric so that no
-// type conversion is required.
-func (m *UntypedVec) GetMetricWith(labels Labels) (Untyped, error) {
-	metric, err := m.MetricVec.GetMetricWith(labels)
-	if metric != nil {
-		return metric.(Untyped), err
-	}
-	return nil, err
-}
-
-// WithLabelValues works as GetMetricWithLabelValues, but panics where
-// GetMetricWithLabelValues would have returned an error. By not returning an
-// error, WithLabelValues allows shortcuts like
-//     myVec.WithLabelValues("404", "GET").Add(42)
-func (m *UntypedVec) WithLabelValues(lvs ...string) Untyped {
-	return m.MetricVec.WithLabelValues(lvs...).(Untyped)
-}
-
-// With works as GetMetricWith, but panics where GetMetricWithLabels would have
-// returned an error. By not returning an error, With allows shortcuts like
-//     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
-func (m *UntypedVec) With(labels Labels) Untyped {
-	return m.MetricVec.With(labels).(Untyped)
-}
-
-// UntypedFunc is an Untyped whose value is determined at collect time by
-// calling a provided function.
+// UntypedFunc works like GaugeFunc but the collected metric is of type
+// "Untyped". UntypedFunc is useful to mirror an external metric of unknown
+// type.
 //
 // To create UntypedFunc instances, use NewUntypedFunc.
 type UntypedFunc interface {

+ 8 - 6
cmd/vendor/github.com/prometheus/client_golang/prometheus/value.go

@@ -14,11 +14,11 @@
 package prometheus
 
 import (
-	"errors"
 	"fmt"
 	"math"
 	"sort"
 	"sync/atomic"
+	"time"
 
 	dto "github.com/prometheus/client_model/go"
 
@@ -36,14 +36,12 @@ const (
 	UntypedValue
 )
 
-var errInconsistentCardinality = errors.New("inconsistent label cardinality")
-
 // value is a generic metric for simple values. It implements Metric, Collector,
 // Counter, Gauge, and Untyped. Its effective type is determined by
 // ValueType. This is a low-level building block used by the library to back the
 // implementations of Counter, Gauge, and Untyped.
 type value struct {
-	// valBits containst the bits of the represented float64 value. It has
+	// valBits contains the bits of the represented float64 value. It has
 	// to go first in the struct to guarantee alignment for atomic
 	// operations.  http://golang.org/pkg/sync/atomic/#pkg-note-BUG
 	valBits uint64
@@ -80,6 +78,10 @@ func (v *value) Set(val float64) {
 	atomic.StoreUint64(&v.valBits, math.Float64bits(val))
 }
 
+func (v *value) SetToCurrentTime() {
+	v.Set(float64(time.Now().UnixNano()) / 1e9)
+}
+
 func (v *value) Inc() {
 	v.Add(1)
 }
@@ -153,8 +155,8 @@ func (v *valueFunc) Write(out *dto.Metric) error {
 // the Collect method. NewConstMetric returns an error if the length of
 // labelValues is not consistent with the variable labels in Desc.
 func NewConstMetric(desc *Desc, valueType ValueType, value float64, labelValues ...string) (Metric, error) {
-	if len(desc.variableLabels) != len(labelValues) {
-		return nil, errInconsistentCardinality
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
 	}
 	return &constMetric{
 		desc:       desc,

+ 51 - 92
cmd/vendor/github.com/prometheus/client_golang/prometheus/vec.go

@@ -20,12 +20,12 @@ import (
 	"github.com/prometheus/common/model"
 )
 
-// MetricVec is a Collector to bundle metrics of the same name that
-// differ in their label values. MetricVec is usually not used directly but as a
-// building block for implementations of vectors of a given metric
-// type. GaugeVec, CounterVec, SummaryVec, and UntypedVec are examples already
-// provided in this package.
-type MetricVec struct {
+// metricVec is a Collector to bundle metrics of the same name that differ in
+// their label values. metricVec is not used directly (and therefore
+// unexported). It is used as a building block for implementations of vectors of
+// a given metric type, like GaugeVec, CounterVec, SummaryVec, HistogramVec, and
+// UntypedVec.
+type metricVec struct {
 	mtx      sync.RWMutex // Protects the children.
 	children map[uint64][]metricWithLabelValues
 	desc     *Desc
@@ -35,10 +35,9 @@ type MetricVec struct {
 	hashAddByte func(h uint64, b byte) uint64
 }
 
-// newMetricVec returns an initialized MetricVec. The concrete value is
-// returned for embedding into another struct.
-func newMetricVec(desc *Desc, newMetric func(lvs ...string) Metric) *MetricVec {
-	return &MetricVec{
+// newMetricVec returns an initialized metricVec.
+func newMetricVec(desc *Desc, newMetric func(lvs ...string) Metric) *metricVec {
+	return &metricVec{
 		children:    map[uint64][]metricWithLabelValues{},
 		desc:        desc,
 		newMetric:   newMetric,
@@ -56,12 +55,12 @@ type metricWithLabelValues struct {
 
 // Describe implements Collector. The length of the returned slice
 // is always one.
-func (m *MetricVec) Describe(ch chan<- *Desc) {
+func (m *metricVec) Describe(ch chan<- *Desc) {
 	ch <- m.desc
 }
 
 // Collect implements Collector.
-func (m *MetricVec) Collect(ch chan<- Metric) {
+func (m *metricVec) Collect(ch chan<- Metric) {
 	m.mtx.RLock()
 	defer m.mtx.RUnlock()
 
@@ -72,31 +71,7 @@ func (m *MetricVec) Collect(ch chan<- Metric) {
 	}
 }
 
-// GetMetricWithLabelValues returns the Metric for the given slice of label
-// values (same order as the VariableLabels in Desc). If that combination of
-// label values is accessed for the first time, a new Metric is created.
-//
-// It is possible to call this method without using the returned Metric to only
-// create the new Metric but leave it at its start value (e.g. a Summary or
-// Histogram without any observations). See also the SummaryVec example.
-//
-// Keeping the Metric for later use is possible (and should be considered if
-// performance is critical), but keep in mind that Reset, DeleteLabelValues and
-// Delete can be used to delete the Metric from the MetricVec. In that case, the
-// Metric will still exist, but it will not be exported anymore, even if a
-// Metric with the same label values is created later. See also the CounterVec
-// example.
-//
-// An error is returned if the number of label values is not the same as the
-// number of VariableLabels in Desc.
-//
-// Note that for more than one label value, this method is prone to mistakes
-// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
-// an alternative to avoid that type of mistake. For higher label numbers, the
-// latter has a much more readable (albeit more verbose) syntax, but it comes
-// with a performance overhead (for creating and processing the Labels map).
-// See also the GaugeVec example.
-func (m *MetricVec) GetMetricWithLabelValues(lvs ...string) (Metric, error) {
+func (m *metricVec) getMetricWithLabelValues(lvs ...string) (Metric, error) {
 	h, err := m.hashLabelValues(lvs)
 	if err != nil {
 		return nil, err
@@ -105,19 +80,7 @@ func (m *MetricVec) GetMetricWithLabelValues(lvs ...string) (Metric, error) {
 	return m.getOrCreateMetricWithLabelValues(h, lvs), nil
 }
 
-// GetMetricWith returns the Metric for the given Labels map (the label names
-// must match those of the VariableLabels in Desc). If that label map is
-// accessed for the first time, a new Metric is created. Implications of
-// creating a Metric without using it and keeping the Metric for later use are
-// the same as for GetMetricWithLabelValues.
-//
-// An error is returned if the number and names of the Labels are inconsistent
-// with those of the VariableLabels in Desc.
-//
-// This method is used for the same purpose as
-// GetMetricWithLabelValues(...string). See there for pros and cons of the two
-// methods.
-func (m *MetricVec) GetMetricWith(labels Labels) (Metric, error) {
+func (m *metricVec) getMetricWith(labels Labels) (Metric, error) {
 	h, err := m.hashLabels(labels)
 	if err != nil {
 		return nil, err
@@ -126,22 +89,16 @@ func (m *MetricVec) GetMetricWith(labels Labels) (Metric, error) {
 	return m.getOrCreateMetricWithLabels(h, labels), nil
 }
 
-// WithLabelValues works as GetMetricWithLabelValues, but panics if an error
-// occurs. The method allows neat syntax like:
-//     httpReqs.WithLabelValues("404", "POST").Inc()
-func (m *MetricVec) WithLabelValues(lvs ...string) Metric {
-	metric, err := m.GetMetricWithLabelValues(lvs...)
+func (m *metricVec) withLabelValues(lvs ...string) Metric {
+	metric, err := m.getMetricWithLabelValues(lvs...)
 	if err != nil {
 		panic(err)
 	}
 	return metric
 }
 
-// With works as GetMetricWith, but panics if an error occurs. The method allows
-// neat syntax like:
-//     httpReqs.With(Labels{"status":"404", "method":"POST"}).Inc()
-func (m *MetricVec) With(labels Labels) Metric {
-	metric, err := m.GetMetricWith(labels)
+func (m *metricVec) with(labels Labels) Metric {
+	metric, err := m.getMetricWith(labels)
 	if err != nil {
 		panic(err)
 	}
@@ -153,8 +110,8 @@ func (m *MetricVec) With(labels Labels) Metric {
 // returns true if a metric was deleted.
 //
 // It is not an error if the number of label values is not the same as the
-// number of VariableLabels in Desc.  However, such inconsistent label count can
-// never match an actual Metric, so the method will always return false in that
+// number of VariableLabels in Desc. However, such inconsistent label count can
+// never match an actual metric, so the method will always return false in that
 // case.
 //
 // Note that for more than one label value, this method is prone to mistakes
@@ -163,7 +120,7 @@ func (m *MetricVec) With(labels Labels) Metric {
 // latter has a much more readable (albeit more verbose) syntax, but it comes
 // with a performance overhead (for creating and processing the Labels map).
 // See also the CounterVec example.
-func (m *MetricVec) DeleteLabelValues(lvs ...string) bool {
+func (m *metricVec) DeleteLabelValues(lvs ...string) bool {
 	m.mtx.Lock()
 	defer m.mtx.Unlock()
 
@@ -178,13 +135,13 @@ func (m *MetricVec) DeleteLabelValues(lvs ...string) bool {
 // passed in as labels. It returns true if a metric was deleted.
 //
 // It is not an error if the number and names of the Labels are inconsistent
-// with those of the VariableLabels in the Desc of the MetricVec. However, such
-// inconsistent Labels can never match an actual Metric, so the method will
-// always return false in that case.
+// with those of the VariableLabels in Desc. However, such inconsistent Labels
+// can never match an actual metric, so the method will always return false in
+// that case.
 //
 // This method is used for the same purpose as DeleteLabelValues(...string). See
 // there for pros and cons of the two methods.
-func (m *MetricVec) Delete(labels Labels) bool {
+func (m *metricVec) Delete(labels Labels) bool {
 	m.mtx.Lock()
 	defer m.mtx.Unlock()
 
@@ -199,7 +156,7 @@ func (m *MetricVec) Delete(labels Labels) bool {
 // deleteByHashWithLabelValues removes the metric from the hash bucket h. If
 // there are multiple matches in the bucket, use lvs to select a metric and
 // remove only that metric.
-func (m *MetricVec) deleteByHashWithLabelValues(h uint64, lvs []string) bool {
+func (m *metricVec) deleteByHashWithLabelValues(h uint64, lvs []string) bool {
 	metrics, ok := m.children[h]
 	if !ok {
 		return false
@@ -221,7 +178,7 @@ func (m *MetricVec) deleteByHashWithLabelValues(h uint64, lvs []string) bool {
 // deleteByHashWithLabels removes the metric from the hash bucket h. If there
 // are multiple matches in the bucket, use lvs to select a metric and remove
 // only that metric.
-func (m *MetricVec) deleteByHashWithLabels(h uint64, labels Labels) bool {
+func (m *metricVec) deleteByHashWithLabels(h uint64, labels Labels) bool {
 	metrics, ok := m.children[h]
 	if !ok {
 		return false
@@ -240,7 +197,7 @@ func (m *MetricVec) deleteByHashWithLabels(h uint64, labels Labels) bool {
 }
 
 // Reset deletes all metrics in this vector.
-func (m *MetricVec) Reset() {
+func (m *metricVec) Reset() {
 	m.mtx.Lock()
 	defer m.mtx.Unlock()
 
@@ -249,10 +206,11 @@ func (m *MetricVec) Reset() {
 	}
 }
 
-func (m *MetricVec) hashLabelValues(vals []string) (uint64, error) {
-	if len(vals) != len(m.desc.variableLabels) {
-		return 0, errInconsistentCardinality
+func (m *metricVec) hashLabelValues(vals []string) (uint64, error) {
+	if err := validateLabelValues(vals, len(m.desc.variableLabels)); err != nil {
+		return 0, err
 	}
+
 	h := hashNew()
 	for _, val := range vals {
 		h = m.hashAdd(h, val)
@@ -261,10 +219,11 @@ func (m *MetricVec) hashLabelValues(vals []string) (uint64, error) {
 	return h, nil
 }
 
-func (m *MetricVec) hashLabels(labels Labels) (uint64, error) {
-	if len(labels) != len(m.desc.variableLabels) {
-		return 0, errInconsistentCardinality
+func (m *metricVec) hashLabels(labels Labels) (uint64, error) {
+	if err := validateValuesInLabels(labels, len(m.desc.variableLabels)); err != nil {
+		return 0, err
 	}
+
 	h := hashNew()
 	for _, label := range m.desc.variableLabels {
 		val, ok := labels[label]
@@ -281,9 +240,9 @@ func (m *MetricVec) hashLabels(labels Labels) (uint64, error) {
 // or creates it and returns the new one.
 //
 // This function holds the mutex.
-func (m *MetricVec) getOrCreateMetricWithLabelValues(hash uint64, lvs []string) Metric {
+func (m *metricVec) getOrCreateMetricWithLabelValues(hash uint64, lvs []string) Metric {
 	m.mtx.RLock()
-	metric, ok := m.getMetricWithLabelValues(hash, lvs)
+	metric, ok := m.getMetricWithHashAndLabelValues(hash, lvs)
 	m.mtx.RUnlock()
 	if ok {
 		return metric
@@ -291,7 +250,7 @@ func (m *MetricVec) getOrCreateMetricWithLabelValues(hash uint64, lvs []string)
 
 	m.mtx.Lock()
 	defer m.mtx.Unlock()
-	metric, ok = m.getMetricWithLabelValues(hash, lvs)
+	metric, ok = m.getMetricWithHashAndLabelValues(hash, lvs)
 	if !ok {
 		// Copy to avoid allocation in case wo don't go down this code path.
 		copiedLVs := make([]string, len(lvs))
@@ -306,9 +265,9 @@ func (m *MetricVec) getOrCreateMetricWithLabelValues(hash uint64, lvs []string)
 // or creates it and returns the new one.
 //
 // This function holds the mutex.
-func (m *MetricVec) getOrCreateMetricWithLabels(hash uint64, labels Labels) Metric {
+func (m *metricVec) getOrCreateMetricWithLabels(hash uint64, labels Labels) Metric {
 	m.mtx.RLock()
-	metric, ok := m.getMetricWithLabels(hash, labels)
+	metric, ok := m.getMetricWithHashAndLabels(hash, labels)
 	m.mtx.RUnlock()
 	if ok {
 		return metric
@@ -316,7 +275,7 @@ func (m *MetricVec) getOrCreateMetricWithLabels(hash uint64, labels Labels) Metr
 
 	m.mtx.Lock()
 	defer m.mtx.Unlock()
-	metric, ok = m.getMetricWithLabels(hash, labels)
+	metric, ok = m.getMetricWithHashAndLabels(hash, labels)
 	if !ok {
 		lvs := m.extractLabelValues(labels)
 		metric = m.newMetric(lvs...)
@@ -325,9 +284,9 @@ func (m *MetricVec) getOrCreateMetricWithLabels(hash uint64, labels Labels) Metr
 	return metric
 }
 
-// getMetricWithLabelValues gets a metric while handling possible collisions in
-// the hash space. Must be called while holding read mutex.
-func (m *MetricVec) getMetricWithLabelValues(h uint64, lvs []string) (Metric, bool) {
+// getMetricWithHashAndLabelValues gets a metric while handling possible
+// collisions in the hash space. Must be called while holding the read mutex.
+func (m *metricVec) getMetricWithHashAndLabelValues(h uint64, lvs []string) (Metric, bool) {
 	metrics, ok := m.children[h]
 	if ok {
 		if i := m.findMetricWithLabelValues(metrics, lvs); i < len(metrics) {
@@ -337,9 +296,9 @@ func (m *MetricVec) getMetricWithLabelValues(h uint64, lvs []string) (Metric, bo
 	return nil, false
 }
 
-// getMetricWithLabels gets a metric while handling possible collisions in
+// getMetricWithHashAndLabels gets a metric while handling possible collisions in
 // the hash space. Must be called while holding read mutex.
-func (m *MetricVec) getMetricWithLabels(h uint64, labels Labels) (Metric, bool) {
+func (m *metricVec) getMetricWithHashAndLabels(h uint64, labels Labels) (Metric, bool) {
 	metrics, ok := m.children[h]
 	if ok {
 		if i := m.findMetricWithLabels(metrics, labels); i < len(metrics) {
@@ -351,7 +310,7 @@ func (m *MetricVec) getMetricWithLabels(h uint64, labels Labels) (Metric, bool)
 
 // findMetricWithLabelValues returns the index of the matching metric or
 // len(metrics) if not found.
-func (m *MetricVec) findMetricWithLabelValues(metrics []metricWithLabelValues, lvs []string) int {
+func (m *metricVec) findMetricWithLabelValues(metrics []metricWithLabelValues, lvs []string) int {
 	for i, metric := range metrics {
 		if m.matchLabelValues(metric.values, lvs) {
 			return i
@@ -362,7 +321,7 @@ func (m *MetricVec) findMetricWithLabelValues(metrics []metricWithLabelValues, l
 
 // findMetricWithLabels returns the index of the matching metric or len(metrics)
 // if not found.
-func (m *MetricVec) findMetricWithLabels(metrics []metricWithLabelValues, labels Labels) int {
+func (m *metricVec) findMetricWithLabels(metrics []metricWithLabelValues, labels Labels) int {
 	for i, metric := range metrics {
 		if m.matchLabels(metric.values, labels) {
 			return i
@@ -371,7 +330,7 @@ func (m *MetricVec) findMetricWithLabels(metrics []metricWithLabelValues, labels
 	return len(metrics)
 }
 
-func (m *MetricVec) matchLabelValues(values []string, lvs []string) bool {
+func (m *metricVec) matchLabelValues(values []string, lvs []string) bool {
 	if len(values) != len(lvs) {
 		return false
 	}
@@ -383,7 +342,7 @@ func (m *MetricVec) matchLabelValues(values []string, lvs []string) bool {
 	return true
 }
 
-func (m *MetricVec) matchLabels(values []string, labels Labels) bool {
+func (m *metricVec) matchLabels(values []string, labels Labels) bool {
 	if len(labels) != len(values) {
 		return false
 	}
@@ -395,7 +354,7 @@ func (m *MetricVec) matchLabels(values []string, labels Labels) bool {
 	return true
 }
 
-func (m *MetricVec) extractLabelValues(labels Labels) []string {
+func (m *metricVec) extractLabelValues(labels Labels) []string {
 	labelValues := make([]string, len(labels))
 	for i, k := range m.desc.variableLabels {
 		labelValues[i] = labels[k]

+ 4 - 0
cmd/vendor/github.com/prometheus/common/expfmt/text_parse.go

@@ -315,6 +315,10 @@ func (p *TextParser) startLabelValue() stateFn {
 	if p.readTokenAsLabelValue(); p.err != nil {
 		return nil
 	}
+	if !model.LabelValue(p.currentToken.String()).IsValid() {
+		p.parseError(fmt.Sprintf("invalid label value %q", p.currentToken.String()))
+		return nil
+	}
 	p.currentLabelPair.Value = proto.String(p.currentToken.String())
 	// Special treatment of summaries:
 	// - Quantile labels are special, will result in dto.Quantile later.

+ 13 - 1
cmd/vendor/github.com/prometheus/common/model/time.go

@@ -163,9 +163,21 @@ func (t *Time) UnmarshalJSON(b []byte) error {
 // This type should not propagate beyond the scope of input/output processing.
 type Duration time.Duration
 
+// Set implements pflag/flag.Value
+func (d *Duration) Set(s string) error {
+	var err error
+	*d, err = ParseDuration(s)
+	return err
+}
+
+// Type implements pflag.Value
+func (d *Duration) Type() string {
+	return "duration"
+}
+
 var durationRE = regexp.MustCompile("^([0-9]+)(y|w|d|h|m|s|ms)$")
 
-// StringToDuration parses a string into a time.Duration, assuming that a year
+// ParseDuration parses a string into a time.Duration, assuming that a year
 // always has 365d, a week always has 7d, and a day always has 24h.
 func ParseDuration(durationStr string) (Duration, error) {
 	matches := durationRE.FindStringSubmatch(durationStr)

+ 34 - 12
cmd/vendor/github.com/prometheus/procfs/ipvs.go

@@ -33,6 +33,8 @@ type IPVSBackendStatus struct {
 	LocalAddress net.IP
 	// The local (virtual) port.
 	LocalPort uint16
+	// The local firewall mark
+	LocalMark string
 	// The transport protocol (TCP, UDP).
 	Proto string
 	// The remote (real) IP address.
@@ -142,6 +144,7 @@ func parseIPVSBackendStatus(file io.Reader) ([]IPVSBackendStatus, error) {
 		status       []IPVSBackendStatus
 		scanner      = bufio.NewScanner(file)
 		proto        string
+		localMark    string
 		localAddress net.IP
 		localPort    uint16
 		err          error
@@ -160,10 +163,19 @@ func parseIPVSBackendStatus(file io.Reader) ([]IPVSBackendStatus, error) {
 				continue
 			}
 			proto = fields[0]
+			localMark = ""
 			localAddress, localPort, err = parseIPPort(fields[1])
 			if err != nil {
 				return nil, err
 			}
+		case fields[0] == "FWM":
+			if len(fields) < 2 {
+				continue
+			}
+			proto = fields[0]
+			localMark = fields[1]
+			localAddress = nil
+			localPort = 0
 		case fields[0] == "->":
 			if len(fields) < 6 {
 				continue
@@ -187,6 +199,7 @@ func parseIPVSBackendStatus(file io.Reader) ([]IPVSBackendStatus, error) {
 			status = append(status, IPVSBackendStatus{
 				LocalAddress:  localAddress,
 				LocalPort:     localPort,
+				LocalMark:     localMark,
 				RemoteAddress: remoteAddress,
 				RemotePort:    remotePort,
 				Proto:         proto,
@@ -200,22 +213,31 @@ func parseIPVSBackendStatus(file io.Reader) ([]IPVSBackendStatus, error) {
 }
 
 func parseIPPort(s string) (net.IP, uint16, error) {
-	tmp := strings.SplitN(s, ":", 2)
-
-	if len(tmp) != 2 {
-		return nil, 0, fmt.Errorf("invalid IP:Port: %s", s)
-	}
+	var (
+		ip  net.IP
+		err error
+	)
 
-	if len(tmp[0]) != 8 && len(tmp[0]) != 32 {
-		return nil, 0, fmt.Errorf("invalid IP: %s", tmp[0])
+	switch len(s) {
+	case 13:
+		ip, err = hex.DecodeString(s[0:8])
+		if err != nil {
+			return nil, 0, err
+		}
+	case 46:
+		ip = net.ParseIP(s[1:40])
+		if ip == nil {
+			return nil, 0, fmt.Errorf("invalid IPv6 address: %s", s[1:40])
+		}
+	default:
+		return nil, 0, fmt.Errorf("unexpected IP:Port: %s", s)
 	}
 
-	ip, err := hex.DecodeString(tmp[0])
-	if err != nil {
-		return nil, 0, err
+	portString := s[len(s)-4:]
+	if len(portString) != 4 {
+		return nil, 0, fmt.Errorf("unexpected port string format: %s", portString)
 	}
-
-	port, err := strconv.ParseUint(tmp[1], 16, 16)
+	port, err := strconv.ParseUint(portString, 16, 16)
 	if err != nil {
 		return nil, 0, err
 	}

+ 8 - 4
cmd/vendor/github.com/prometheus/procfs/mountstats.go

@@ -523,15 +523,19 @@ func parseNFSTransportStats(ss []string, statVersion string) (*NFSTransportStats
 	}
 
 	// Allocate enough for v1.1 stats since zero value for v1.1 stats will be okay
-	// in a v1.0 response
-	ns := make([]uint64, 0, fieldTransport11Len)
-	for _, s := range ss {
+	// in a v1.0 response.
+	//
+	// Note: slice length must be set to length of v1.1 stats to avoid a panic when
+	// only v1.0 stats are present.
+	// See: https://github.com/prometheus/node_exporter/issues/571.
+	ns := make([]uint64, fieldTransport11Len)
+	for i, s := range ss {
 		n, err := strconv.ParseUint(s, 10, 64)
 		if err != nil {
 			return nil, err
 		}
 
-		ns = append(ns, n)
+		ns[i] = n
 	}
 
 	return &NFSTransportStats{

+ 19 - 19
cmd/vendor/github.com/prometheus/procfs/proc_limits.go

@@ -13,46 +13,46 @@ import (
 // http://man7.org/linux/man-pages/man2/getrlimit.2.html.
 type ProcLimits struct {
 	// CPU time limit in seconds.
-	CPUTime int
+	CPUTime int64
 	// Maximum size of files that the process may create.
-	FileSize int
+	FileSize int64
 	// Maximum size of the process's data segment (initialized data,
 	// uninitialized data, and heap).
-	DataSize int
+	DataSize int64
 	// Maximum size of the process stack in bytes.
-	StackSize int
+	StackSize int64
 	// Maximum size of a core file.
-	CoreFileSize int
+	CoreFileSize int64
 	// Limit of the process's resident set in pages.
-	ResidentSet int
+	ResidentSet int64
 	// Maximum number of processes that can be created for the real user ID of
 	// the calling process.
-	Processes int
+	Processes int64
 	// Value one greater than the maximum file descriptor number that can be
 	// opened by this process.
-	OpenFiles int
+	OpenFiles int64
 	// Maximum number of bytes of memory that may be locked into RAM.
-	LockedMemory int
+	LockedMemory int64
 	// Maximum size of the process's virtual memory address space in bytes.
-	AddressSpace int
+	AddressSpace int64
 	// Limit on the combined number of flock(2) locks and fcntl(2) leases that
 	// this process may establish.
-	FileLocks int
+	FileLocks int64
 	// Limit of signals that may be queued for the real user ID of the calling
 	// process.
-	PendingSignals int
+	PendingSignals int64
 	// Limit on the number of bytes that can be allocated for POSIX message
 	// queues for the real user ID of the calling process.
-	MsqqueueSize int
+	MsqqueueSize int64
 	// Limit of the nice priority set using setpriority(2) or nice(2).
-	NicePriority int
+	NicePriority int64
 	// Limit of the real-time priority set using sched_setscheduler(2) or
 	// sched_setparam(2).
-	RealtimePriority int
+	RealtimePriority int64
 	// Limit (in microseconds) on the amount of CPU time that a process
 	// scheduled under a real-time scheduling policy may consume without making
 	// a blocking system call.
-	RealtimeTimeout int
+	RealtimeTimeout int64
 }
 
 const (
@@ -125,13 +125,13 @@ func (p Proc) NewLimits() (ProcLimits, error) {
 	return l, s.Err()
 }
 
-func parseInt(s string) (int, error) {
+func parseInt(s string) (int64, error) {
 	if s == limitsUnlimited {
 		return -1, nil
 	}
-	i, err := strconv.ParseInt(s, 10, 32)
+	i, err := strconv.ParseInt(s, 10, 64)
 	if err != nil {
 		return 0, fmt.Errorf("couldn't parse value %s: %s", s, err)
 	}
-	return int(i), nil
+	return i, nil
 }

+ 178 - 15
cmd/vendor/github.com/prometheus/procfs/stat.go

@@ -3,15 +3,66 @@ package procfs
 import (
 	"bufio"
 	"fmt"
+	"io"
 	"os"
 	"strconv"
 	"strings"
 )
 
+// CPUStat shows how much time the cpu spend in various stages.
+type CPUStat struct {
+	User      float64
+	Nice      float64
+	System    float64
+	Idle      float64
+	Iowait    float64
+	IRQ       float64
+	SoftIRQ   float64
+	Steal     float64
+	Guest     float64
+	GuestNice float64
+}
+
+// SoftIRQStat represent the softirq statistics as exported in the procfs stat file.
+// A nice introduction can be found at https://0xax.gitbooks.io/linux-insides/content/interrupts/interrupts-9.html
+// It is possible to get per-cpu stats by reading /proc/softirqs
+type SoftIRQStat struct {
+	Hi          uint64
+	Timer       uint64
+	NetTx       uint64
+	NetRx       uint64
+	Block       uint64
+	BlockIoPoll uint64
+	Tasklet     uint64
+	Sched       uint64
+	Hrtimer     uint64
+	Rcu         uint64
+}
+
 // Stat represents kernel/system statistics.
 type Stat struct {
 	// Boot time in seconds since the Epoch.
-	BootTime int64
+	BootTime uint64
+	// Summed up cpu statistics.
+	CPUTotal CPUStat
+	// Per-CPU statistics.
+	CPU []CPUStat
+	// Number of times interrupts were handled, which contains numbered and unnumbered IRQs.
+	IRQTotal uint64
+	// Number of times a numbered IRQ was triggered.
+	IRQ []uint64
+	// Number of times a context switch happened.
+	ContextSwitches uint64
+	// Number of times a process was created.
+	ProcessCreated uint64
+	// Number of processes currently running.
+	ProcessesRunning uint64
+	// Number of processes currently blocked (waiting for IO).
+	ProcessesBlocked uint64
+	// Number of times a softirq was scheduled.
+	SoftIRQTotal uint64
+	// Detailed softirq statistics.
+	SoftIRQ SoftIRQStat
 }
 
 // NewStat returns kernel/system statistics read from /proc/stat.
@@ -24,33 +75,145 @@ func NewStat() (Stat, error) {
 	return fs.NewStat()
 }
 
+// Parse a cpu statistics line and returns the CPUStat struct plus the cpu id (or -1 for the overall sum).
+func parseCPUStat(line string) (CPUStat, int64, error) {
+	cpuStat := CPUStat{}
+	var cpu string
+
+	count, err := fmt.Sscanf(line, "%s %f %f %f %f %f %f %f %f %f %f",
+		&cpu,
+		&cpuStat.User, &cpuStat.Nice, &cpuStat.System, &cpuStat.Idle,
+		&cpuStat.Iowait, &cpuStat.IRQ, &cpuStat.SoftIRQ, &cpuStat.Steal,
+		&cpuStat.Guest, &cpuStat.GuestNice)
+
+	if err != nil && err != io.EOF {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu): %s", line, err)
+	}
+	if count == 0 {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu): 0 elements parsed", line)
+	}
+
+	cpuStat.User /= userHZ
+	cpuStat.Nice /= userHZ
+	cpuStat.System /= userHZ
+	cpuStat.Idle /= userHZ
+	cpuStat.Iowait /= userHZ
+	cpuStat.IRQ /= userHZ
+	cpuStat.SoftIRQ /= userHZ
+	cpuStat.Steal /= userHZ
+	cpuStat.Guest /= userHZ
+	cpuStat.GuestNice /= userHZ
+
+	if cpu == "cpu" {
+		return cpuStat, -1, nil
+	}
+
+	cpuID, err := strconv.ParseInt(cpu[3:], 10, 64)
+	if err != nil {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu/cpuid): %s", line, err)
+	}
+
+	return cpuStat, cpuID, nil
+}
+
+// Parse a softirq line.
+func parseSoftIRQStat(line string) (SoftIRQStat, uint64, error) {
+	softIRQStat := SoftIRQStat{}
+	var total uint64
+	var prefix string
+
+	_, err := fmt.Sscanf(line, "%s %d %d %d %d %d %d %d %d %d %d %d",
+		&prefix, &total,
+		&softIRQStat.Hi, &softIRQStat.Timer, &softIRQStat.NetTx, &softIRQStat.NetRx,
+		&softIRQStat.Block, &softIRQStat.BlockIoPoll,
+		&softIRQStat.Tasklet, &softIRQStat.Sched,
+		&softIRQStat.Hrtimer, &softIRQStat.Rcu)
+
+	if err != nil {
+		return SoftIRQStat{}, 0, fmt.Errorf("couldn't parse %s (softirq): %s", line, err)
+	}
+
+	return softIRQStat, total, nil
+}
+
 // NewStat returns an information about current kernel/system statistics.
 func (fs FS) NewStat() (Stat, error) {
+	// See https://www.kernel.org/doc/Documentation/filesystems/proc.txt
+
 	f, err := os.Open(fs.Path("stat"))
 	if err != nil {
 		return Stat{}, err
 	}
 	defer f.Close()
 
-	s := bufio.NewScanner(f)
-	for s.Scan() {
-		line := s.Text()
-		if !strings.HasPrefix(line, "btime") {
+	stat := Stat{}
+
+	scanner := bufio.NewScanner(f)
+	for scanner.Scan() {
+		line := scanner.Text()
+		parts := strings.Fields(scanner.Text())
+		// require at least <key> <value>
+		if len(parts) < 2 {
 			continue
 		}
-		fields := strings.Fields(line)
-		if len(fields) != 2 {
-			return Stat{}, fmt.Errorf("couldn't parse %s line %s", f.Name(), line)
-		}
-		i, err := strconv.ParseInt(fields[1], 10, 32)
-		if err != nil {
-			return Stat{}, fmt.Errorf("couldn't parse %s: %s", fields[1], err)
+		switch {
+		case parts[0] == "btime":
+			if stat.BootTime, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (btime): %s", parts[1], err)
+			}
+		case parts[0] == "intr":
+			if stat.IRQTotal, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (intr): %s", parts[1], err)
+			}
+			numberedIRQs := parts[2:]
+			stat.IRQ = make([]uint64, len(numberedIRQs))
+			for i, count := range numberedIRQs {
+				if stat.IRQ[i], err = strconv.ParseUint(count, 10, 64); err != nil {
+					return Stat{}, fmt.Errorf("couldn't parse %s (intr%d): %s", count, i, err)
+				}
+			}
+		case parts[0] == "ctxt":
+			if stat.ContextSwitches, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (ctxt): %s", parts[1], err)
+			}
+		case parts[0] == "processes":
+			if stat.ProcessCreated, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (processes): %s", parts[1], err)
+			}
+		case parts[0] == "procs_running":
+			if stat.ProcessesRunning, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (procs_running): %s", parts[1], err)
+			}
+		case parts[0] == "procs_blocked":
+			if stat.ProcessesBlocked, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (procs_blocked): %s", parts[1], err)
+			}
+		case parts[0] == "softirq":
+			softIRQStats, total, err := parseSoftIRQStat(line)
+			if err != nil {
+				return Stat{}, err
+			}
+			stat.SoftIRQTotal = total
+			stat.SoftIRQ = softIRQStats
+		case strings.HasPrefix(parts[0], "cpu"):
+			cpuStat, cpuID, err := parseCPUStat(line)
+			if err != nil {
+				return Stat{}, err
+			}
+			if cpuID == -1 {
+				stat.CPUTotal = cpuStat
+			} else {
+				for int64(len(stat.CPU)) <= cpuID {
+					stat.CPU = append(stat.CPU, CPUStat{})
+				}
+				stat.CPU[cpuID] = cpuStat
+			}
 		}
-		return Stat{BootTime: i}, nil
 	}
-	if err := s.Err(); err != nil {
+
+	if err := scanner.Err(); err != nil {
 		return Stat{}, fmt.Errorf("couldn't parse %s: %s", f.Name(), err)
 	}
 
-	return Stat{}, fmt.Errorf("couldn't parse %s, missing btime", f.Name())
+	return stat, nil
 }

+ 187 - 0
cmd/vendor/github.com/prometheus/procfs/xfrm.go

@@ -0,0 +1,187 @@
+// Copyright 2017 Prometheus Team
+// 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 procfs
+
+import (
+	"bufio"
+	"fmt"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// XfrmStat models the contents of /proc/net/xfrm_stat.
+type XfrmStat struct {
+	// All errors which are not matched by other
+	XfrmInError int
+	// No buffer is left
+	XfrmInBufferError int
+	// Header Error
+	XfrmInHdrError int
+	// No state found
+	// i.e. either inbound SPI, address, or IPSEC protocol at SA is wrong
+	XfrmInNoStates int
+	// Transformation protocol specific error
+	// e.g. SA Key is wrong
+	XfrmInStateProtoError int
+	// Transformation mode specific error
+	XfrmInStateModeError int
+	// Sequence error
+	// e.g. sequence number is out of window
+	XfrmInStateSeqError int
+	// State is expired
+	XfrmInStateExpired int
+	// State has mismatch option
+	// e.g. UDP encapsulation type is mismatched
+	XfrmInStateMismatch int
+	// State is invalid
+	XfrmInStateInvalid int
+	// No matching template for states
+	// e.g. Inbound SAs are correct but SP rule is wrong
+	XfrmInTmplMismatch int
+	// No policy is found for states
+	// e.g. Inbound SAs are correct but no SP is found
+	XfrmInNoPols int
+	// Policy discards
+	XfrmInPolBlock int
+	// Policy error
+	XfrmInPolError int
+	// All errors which are not matched by others
+	XfrmOutError int
+	// Bundle generation error
+	XfrmOutBundleGenError int
+	// Bundle check error
+	XfrmOutBundleCheckError int
+	// No state was found
+	XfrmOutNoStates int
+	// Transformation protocol specific error
+	XfrmOutStateProtoError int
+	// Transportation mode specific error
+	XfrmOutStateModeError int
+	// Sequence error
+	// i.e sequence number overflow
+	XfrmOutStateSeqError int
+	// State is expired
+	XfrmOutStateExpired int
+	// Policy discads
+	XfrmOutPolBlock int
+	// Policy is dead
+	XfrmOutPolDead int
+	// Policy Error
+	XfrmOutPolError     int
+	XfrmFwdHdrError     int
+	XfrmOutStateInvalid int
+	XfrmAcquireError    int
+}
+
+// NewXfrmStat reads the xfrm_stat statistics.
+func NewXfrmStat() (XfrmStat, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return XfrmStat{}, err
+	}
+
+	return fs.NewXfrmStat()
+}
+
+// NewXfrmStat reads the xfrm_stat statistics from the 'proc' filesystem.
+func (fs FS) NewXfrmStat() (XfrmStat, error) {
+	file, err := os.Open(fs.Path("net/xfrm_stat"))
+	if err != nil {
+		return XfrmStat{}, err
+	}
+	defer file.Close()
+
+	var (
+		x = XfrmStat{}
+		s = bufio.NewScanner(file)
+	)
+
+	for s.Scan() {
+		fields := strings.Fields(s.Text())
+
+		if len(fields) != 2 {
+			return XfrmStat{}, fmt.Errorf(
+				"couldnt parse %s line %s", file.Name(), s.Text())
+		}
+
+		name := fields[0]
+		value, err := strconv.Atoi(fields[1])
+		if err != nil {
+			return XfrmStat{}, err
+		}
+
+		switch name {
+		case "XfrmInError":
+			x.XfrmInError = value
+		case "XfrmInBufferError":
+			x.XfrmInBufferError = value
+		case "XfrmInHdrError":
+			x.XfrmInHdrError = value
+		case "XfrmInNoStates":
+			x.XfrmInNoStates = value
+		case "XfrmInStateProtoError":
+			x.XfrmInStateProtoError = value
+		case "XfrmInStateModeError":
+			x.XfrmInStateModeError = value
+		case "XfrmInStateSeqError":
+			x.XfrmInStateSeqError = value
+		case "XfrmInStateExpired":
+			x.XfrmInStateExpired = value
+		case "XfrmInStateInvalid":
+			x.XfrmInStateInvalid = value
+		case "XfrmInTmplMismatch":
+			x.XfrmInTmplMismatch = value
+		case "XfrmInNoPols":
+			x.XfrmInNoPols = value
+		case "XfrmInPolBlock":
+			x.XfrmInPolBlock = value
+		case "XfrmInPolError":
+			x.XfrmInPolError = value
+		case "XfrmOutError":
+			x.XfrmOutError = value
+		case "XfrmInStateMismatch":
+			x.XfrmInStateMismatch = value
+		case "XfrmOutBundleGenError":
+			x.XfrmOutBundleGenError = value
+		case "XfrmOutBundleCheckError":
+			x.XfrmOutBundleCheckError = value
+		case "XfrmOutNoStates":
+			x.XfrmOutNoStates = value
+		case "XfrmOutStateProtoError":
+			x.XfrmOutStateProtoError = value
+		case "XfrmOutStateModeError":
+			x.XfrmOutStateModeError = value
+		case "XfrmOutStateSeqError":
+			x.XfrmOutStateSeqError = value
+		case "XfrmOutStateExpired":
+			x.XfrmOutStateExpired = value
+		case "XfrmOutPolBlock":
+			x.XfrmOutPolBlock = value
+		case "XfrmOutPolDead":
+			x.XfrmOutPolDead = value
+		case "XfrmOutPolError":
+			x.XfrmOutPolError = value
+		case "XfrmFwdHdrError":
+			x.XfrmFwdHdrError = value
+		case "XfrmOutStateInvalid":
+			x.XfrmOutStateInvalid = value
+		case "XfrmAcquireError":
+			x.XfrmAcquireError = value
+		}
+
+	}
+
+	return x, s.Err()
+}

+ 0 - 2
cmd/vendor/github.com/prometheus/procfs/xfs/parse.go

@@ -17,7 +17,6 @@ import (
 	"bufio"
 	"fmt"
 	"io"
-	"log"
 	"strconv"
 	"strings"
 )
@@ -273,7 +272,6 @@ func vnodeStats(us []uint32) (VnodeStats, error) {
 	// stats versions.  Therefore, 7 or 8 elements may appear in
 	// this slice.
 	l := len(us)
-	log.Println(l)
 	if l != 7 && l != 8 {
 		return VnodeStats{}, fmt.Errorf("incorrect number of values for XFS vnode stats: %d", l)
 	}

+ 5 - 0
cmd/vendor/github.com/prometheus/procfs/xfs/xfs.go

@@ -22,6 +22,11 @@ package xfs
 // kernel source. Most counters are uint32s (same data types used in
 // xfs_stats.h), but some of the "extended precision stats" are uint64s.
 type Stats struct {
+	// The name of the filesystem used to source these statistics.
+	// If empty, this indicates aggregated statistics for all XFS
+	// filesystems on the host.
+	Name string
+
 	ExtentAllocation   ExtentAllocationStats
 	AllocationBTree    BTreeStats
 	BlockMapping       BlockMappingStats

+ 3 - 3
etcdserver/api/etcdhttp/metrics.go

@@ -24,7 +24,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/raft"
 
-	"github.com/prometheus/client_golang/prometheus"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
 )
 
 const (
@@ -34,13 +34,13 @@ const (
 
 // HandleMetricsHealth registers metrics and health handlers.
 func HandleMetricsHealth(mux *http.ServeMux, srv etcdserver.ServerV2) {
-	mux.Handle(pathMetrics, prometheus.Handler())
+	mux.Handle(pathMetrics, promhttp.Handler())
 	mux.Handle(PathHealth, NewHealthHandler(func() Health { return checkHealth(srv) }))
 }
 
 // HandlePrometheus registers prometheus handler on '/metrics'.
 func HandlePrometheus(mux *http.ServeMux) {
-	mux.Handle(pathMetrics, prometheus.Handler())
+	mux.Handle(pathMetrics, promhttp.Handler())
 }
 
 // HandleHealth registers health handler on '/health'.

+ 7 - 6
glide.lock

@@ -1,5 +1,5 @@
-hash: 7e13ef2acb8e54723a9530aadee7f8ef83d3ac58a2d902fbc1a7f80ee36c61f7
-updated: 2017-11-08T18:45:57.327801-08:00
+hash: cff74aae5a6b8c11816c9994dedfdfdcd9f4137d61d8ed8ba0bf623f0ff21d50
+updated: 2017-11-10T09:46:28.3753-08:00
 imports:
 - name: github.com/beorn7/perks
   version: 4c0e84591b9aa9e6dcfdf3e020114cd81f89d5f9
@@ -57,7 +57,7 @@ imports:
 - name: github.com/gorilla/websocket
   version: 4201258b820c74ac8e6922fc9e6b52f71fe46f8d
 - name: github.com/grpc-ecosystem/go-grpc-prometheus
-  version: 6b7015e65d366bf3f19b2b2a000a831940f0f7e0
+  version: 0dafe0d496ea71181bf2dd039e7e3f44b6bd11a7
 - name: github.com/grpc-ecosystem/grpc-gateway
   version: 8cc3a55af3bcf171a1c23a90c4df9cf591706104
   subpackages:
@@ -81,21 +81,22 @@ imports:
 - name: github.com/olekukonko/tablewriter
   version: a0225b3f23b5ce0cbec6d7a66a968f8a59eca9c4
 - name: github.com/prometheus/client_golang
-  version: c5b7fccd204277076155f10851dad72b76a49317
+  version: 5cec1d0429b02e4323e042eb04dafdb079ddf568
   subpackages:
   - prometheus
+  - prometheus/promhttp
 - name: github.com/prometheus/client_model
   version: 6f3806018612930941127f2a7c6c453ba2c527d2
   subpackages:
   - go
 - name: github.com/prometheus/common
-  version: 49fee292b27bfff7f354ee0f64e1bc4850462edf
+  version: e3fb1a1acd7605367a2b378bc2e2f893c05174b7
   subpackages:
   - expfmt
   - internal/bitbucket.org/ww/goautoneg
   - model
 - name: github.com/prometheus/procfs
-  version: a1dba9ce8baed984a2495b658c82687f8157b98f
+  version: a6e9df898b1336106c743392c48ee0b71f5c4efa
   subpackages:
   - xfs
 - name: github.com/russross/blackfriday

+ 12 - 5
glide.yaml

@@ -60,13 +60,22 @@ import:
   subpackages:
   - runewidth.go
 - package: github.com/prometheus/client_golang
-  version: v0.8.0
+  version: 5cec1d0429b02e4323e042eb04dafdb079ddf568
   subpackages:
   - prometheus
+  - prometheus/promhttp
+- package: github.com/prometheus/client_model
+  version: 6f3806018612930941127f2a7c6c453ba2c527d2
+  subpackages:
+  - go
 - package: github.com/prometheus/common
-  version: 49fee292b27bfff7f354ee0f64e1bc4850462edf
+  version: e3fb1a1acd7605367a2b378bc2e2f893c05174b7
 - package: github.com/prometheus/procfs
-  version: a1dba9ce8baed984a2495b658c82687f8157b98f
+  version: a6e9df898b1336106c743392c48ee0b71f5c4efa
+  subpackages:
+  - xfs
+- package: github.com/grpc-ecosystem/go-grpc-prometheus
+  version: 0dafe0d496ea71181bf2dd039e7e3f44b6bd11a7
 - package: github.com/spf13/cobra
   version: 1c44ec8d3f1552cac48999f9306da23c4d8a288b
 - package: github.com/spf13/pflag
@@ -79,8 +88,6 @@ import:
   version: v1.18.0
 - package: github.com/xiang90/probing
   version: 0.0.1
-- package: github.com/grpc-ecosystem/go-grpc-prometheus
-  version: v1.1
 - package: golang.org/x/crypto
   version: 9419663f5a44be8b34ca85f08abc5fe1be11f8a3
   subpackages:

+ 2 - 2
tools/functional-tester/etcd-tester/main.go

@@ -24,7 +24,7 @@ import (
 	"github.com/coreos/etcd/pkg/debugutil"
 
 	"github.com/coreos/pkg/capnslog"
-	"github.com/prometheus/client_golang/prometheus"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
 	"golang.org/x/time/rate"
 )
 
@@ -138,7 +138,7 @@ func main() {
 
 	sh := statusHandler{status: &t.status}
 	http.Handle("/status", sh)
-	http.Handle("/metrics", prometheus.Handler())
+	http.Handle("/metrics", promhttp.Handler())
 
 	if *enablePprof {
 		for p, h := range debugutil.PProfHandlers() {