diff --git a/cmd/cortex/main.go b/cmd/cortex/main.go index d84a3f8e9e..f210841496 100644 --- a/cmd/cortex/main.go +++ b/cmd/cortex/main.go @@ -3,6 +3,7 @@ package main import ( "flag" "fmt" + "net" "net/http" _ "net/http/pprof" "os" @@ -11,13 +12,16 @@ import ( "time" "github.com/gorilla/mux" + "github.com/mwitkow/go-grpc-middleware" + "github.com/weaveworks/scope/common/middleware" + "golang.org/x/net/context" + "google.golang.org/grpc" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/log" "github.com/prometheus/common/route" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/web/api/v1" - "github.com/weaveworks/scope/common/middleware" - "golang.org/x/net/context" "github.com/weaveworks/cortex" "github.com/weaveworks/cortex/chunk" @@ -28,6 +32,7 @@ import ( "github.com/weaveworks/cortex/ruler" "github.com/weaveworks/cortex/ui" "github.com/weaveworks/cortex/user" + cortex_grpc_middleware "github.com/weaveworks/cortex/util/middleware" ) const ( @@ -35,8 +40,7 @@ const ( modeIngester = "ingester" modeRuler = "ruler" - infName = "eth0" - userIDHeaderName = "X-Scope-OrgID" + infName = "eth0" ) var ( @@ -84,30 +88,38 @@ func main() { var cfg cfg flag.StringVar(&cfg.mode, "mode", modeDistributor, "Mode (distributor, ingester, ruler).") flag.IntVar(&cfg.listenPort, "web.listen-port", 9094, "HTTP server listen port.") + flag.BoolVar(&cfg.logSuccess, "log.success", false, "Log successful requests") + flag.StringVar(&cfg.consulHost, "consul.hostname", "localhost:8500", "Hostname and port of Consul.") flag.StringVar(&cfg.consulPrefix, "consul.prefix", "collectors/", "Prefix for keys in Consul.") + flag.StringVar(&cfg.s3URL, "s3.url", "localhost:4569", "S3 endpoint URL.") flag.StringVar(&cfg.dynamodbURL, "dynamodb.url", "localhost:8000", "DynamoDB endpoint URL.") - flag.BoolVar(&cfg.dynamodbCreateTables, "dynamodb.create-tables", false, "Create required DynamoDB tables on startup.") flag.DurationVar(&cfg.dynamodbPollInterval, "dynamodb.poll-interval", 2*time.Minute, "How frequently to poll DynamoDB to learn our capacity.") + flag.BoolVar(&cfg.dynamodbCreateTables, "dynamodb.create-tables", false, "Create required DynamoDB tables on startup.") + flag.BoolVar(&cfg.watchDynamo, "watch-dynamo", false, "Periodically collect DynamoDB provisioned throughput.") + flag.StringVar(&cfg.memcachedHostname, "memcached.hostname", "", "Hostname for memcached service to use when caching chunks. If empty, no memcached will be used.") + flag.StringVar(&cfg.memcachedService, "memcached.service", "memcached", "SRV service used to discover memcache servers.") flag.DurationVar(&cfg.memcachedTimeout, "memcached.timeout", 100*time.Millisecond, "Maximum time to wait before giving up on memcached requests.") flag.DurationVar(&cfg.memcachedExpiration, "memcached.expiration", 0, "How long chunks stay in the memcache.") - flag.StringVar(&cfg.memcachedService, "memcached.service", "memcached", "SRV service used to discover memcache servers.") - flag.DurationVar(&cfg.remoteTimeout, "remote.timeout", 5*time.Second, "Timeout for downstream ingesters.") + flag.DurationVar(&cfg.ingesterConfig.FlushCheckPeriod, "ingester.flush-period", 1*time.Minute, "Period with which to attempt to flush chunks.") flag.DurationVar(&cfg.ingesterConfig.RateUpdatePeriod, "ingester.rate-update-period", 15*time.Second, "Period with which to update the per-user ingestion rates.") flag.DurationVar(&cfg.ingesterConfig.MaxChunkIdle, "ingester.max-chunk-idle", 1*time.Hour, "Maximum chunk idle time before flushing.") flag.IntVar(&cfg.ingesterConfig.ConcurrentFlushes, "ingester.concurrent-flushes", 25, "Number of concurrent goroutines flushing to dynamodb.") flag.IntVar(&cfg.numTokens, "ingester.num-tokens", 128, "Number of tokens for each ingester.") + flag.IntVar(&cfg.ingesterConfig.GRPCListenPort, "ingester.grpc.listen-port", 9095, "gRPC server listen port.") + flag.IntVar(&cfg.distributorConfig.ReplicationFactor, "distributor.replication-factor", 3, "The number of ingesters to write to and read from.") flag.IntVar(&cfg.distributorConfig.MinReadSuccesses, "distributor.min-read-successes", 2, "The minimum number of ingesters from which a read must succeed.") flag.DurationVar(&cfg.distributorConfig.HeartbeatTimeout, "distributor.heartbeat-timeout", time.Minute, "The heartbeat timeout after which ingesters are skipped for reads/writes.") + flag.DurationVar(&cfg.distributorConfig.RemoteTimeout, "distributor.remote-timeout", 5*time.Second, "Timeout for downstream ingesters.") + flag.StringVar(&cfg.rulerConfig.ConfigsAPIURL, "ruler.configs.url", "", "URL of configs API server.") flag.StringVar(&cfg.rulerConfig.UserID, "ruler.userID", "", "Weave Cloud org to run rules for") flag.DurationVar(&cfg.rulerConfig.EvaluationInterval, "ruler.evaluation-interval", 15*time.Second, "How frequently to evaluate rules") - flag.BoolVar(&cfg.logSuccess, "log.success", false, "Log successful requests") - flag.BoolVar(&cfg.watchDynamo, "watch-dynamo", false, "Periodically collect DynamoDB provisioned throughput.") + flag.Parse() chunkStore, err := setupChunkStore(cfg) @@ -139,14 +151,11 @@ func main() { switch cfg.mode { case modeDistributor: cfg.distributorConfig.Ring = r - cfg.distributorConfig.ClientFactory = func(address string) (*distributor.IngesterClient, error) { - return distributor.NewIngesterClient(address, cfg.remoteTimeout) - } setupDistributor(cfg.distributorConfig, chunkStore, router.PathPrefix("/api/prom").Subrouter()) case modeIngester: cfg.ingesterConfig.Ring = r - registration, err := ring.RegisterIngester(consul, cfg.listenPort, cfg.numTokens) + registration, err := ring.RegisterIngester(consul, cfg.listenPort, cfg.ingesterConfig.GRPCListenPort, cfg.numTokens) if err != nil { // This only happens for errors in configuration & set-up, not for // network errors. @@ -154,6 +163,22 @@ func main() { } ing := setupIngester(chunkStore, cfg.ingesterConfig, router) + // Setup gRPC server + lis, err := net.Listen("tcp", fmt.Sprintf(":%d", cfg.ingesterConfig.GRPCListenPort)) + if err != nil { + log.Fatalf("failed to listen: %v", err) + } + grpcServer := grpc.NewServer( + grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer( + cortex_grpc_middleware.ServerInstrumentInterceptor(requestDuration), + cortex_grpc_middleware.ServerLoggingInterceptor(cfg.logSuccess), + cortex_grpc_middleware.ServerUserHeaderInterceptor, + )), + ) + cortex.RegisterIngesterServer(grpcServer, ing) + go grpcServer.Serve(lis) + defer grpcServer.Stop() + // Deferring a func to make ordering obvious defer func() { registration.ChangeState(ring.Leaving) @@ -166,9 +191,6 @@ func main() { case modeRuler: // XXX: Too much duplication w/ distributor set up. cfg.distributorConfig.Ring = r - cfg.distributorConfig.ClientFactory = func(address string) (*distributor.IngesterClient, error) { - return distributor.NewIngesterClient(address, cfg.remoteTimeout) - } cfg.rulerConfig.DistributorConfig = cfg.distributorConfig ruler, err := setupRuler(chunkStore, cfg.rulerConfig) if err != nil { @@ -242,26 +264,12 @@ func setupDistributor( } prometheus.MustRegister(dist) - router.Path("/push").Handler(cortex.AppenderHandler(dist, handleDistributorError)) + router.Path("/push").Handler(http.HandlerFunc(dist.PushHandler)) // TODO: Move querier to separate binary. setupQuerier(dist, chunkStore, router) } -func handleDistributorError(w http.ResponseWriter, err error) { - switch e := err.(type) { - case distributor.IngesterError: - switch { - case 400 <= e.StatusCode && e.StatusCode < 500: - log.Warnf("append err: %v", err) - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - } - log.Errorf("append err: %v", err) - http.Error(w, err.Error(), http.StatusInternalServerError) -} - // setupQuerier sets up a complete querying pipeline: // // PromQL -> MergeQuerier -> Distributor -> IngesterQuerier -> Ingester @@ -276,12 +284,15 @@ func setupQuerier( engine := promql.NewEngine(queryable, nil) api := v1.NewAPI(engine, querier.DummyStorage{Queryable: queryable}) promRouter := route.New(func(r *http.Request) (context.Context, error) { - userID := r.Header.Get(userIDHeaderName) + userID := r.Header.Get(user.UserIDHeaderName) + if userID == "" { + return nil, fmt.Errorf("no %s header", user.UserIDHeaderName) + } return user.WithID(context.Background(), userID), nil }).WithPrefix("/api/prom/api/v1") api.Register(promRouter) router.PathPrefix("/api/v1").Handler(promRouter) - router.Path("/user_stats").Handler(cortex.DistributorUserStatsHandler(distributor.UserStats)) + router.Path("/user_stats").Handler(http.HandlerFunc(distributor.UserStatsHandler)) router.Path("/graph").Handler(ui.GraphHandler()) router.PathPrefix("/static/").Handler(ui.StaticAssetsHandler("/api/prom/static/")) } @@ -297,25 +308,14 @@ func setupIngester( } prometheus.MustRegister(ingester) - router.Path("/push").Handler(cortex.AppenderHandler(ingester, handleIngesterError)) - router.Path("/query").Handler(cortex.QueryHandler(ingester)) - router.Path("/label_values").Handler(cortex.LabelValuesHandler(ingester)) - router.Path("/user_stats").Handler(cortex.IngesterUserStatsHandler(ingester.UserStats)) - router.Path("/ready").Handler(cortex.IngesterReadinessHandler(ingester)) + router.Path("/push").Handler(http.HandlerFunc(ingester.PushHandler)) + router.Path("/query").Handler(http.HandlerFunc(ingester.QueryHandler)) + router.Path("/label_values").Handler(http.HandlerFunc(ingester.LabelValuesHandler)) + router.Path("/user_stats").Handler(http.HandlerFunc(ingester.UserStatsHandler)) + router.Path("/ready").Handler(http.HandlerFunc(ingester.ReadinessHandler)) return ingester } -func handleIngesterError(w http.ResponseWriter, err error) { - switch err { - case ingester.ErrOutOfOrderSample, ingester.ErrDuplicateSampleForTimestamp: - log.Warnf("append err: %v", err) - http.Error(w, err.Error(), http.StatusBadRequest) - default: - log.Errorf("append err: %v", err) - http.Error(w, err.Error(), http.StatusInternalServerError) - } -} - // setupRuler sets up a ruler. func setupRuler(chunkStore chunk.Store, cfg ruler.Config) (*ruler.Ruler, error) { return ruler.New(chunkStore, cfg) diff --git a/cortex.proto b/cortex.proto index e5fb8e3446..47f1561bf7 100644 --- a/cortex.proto +++ b/cortex.proto @@ -1,44 +1,27 @@ syntax = "proto3"; -package cortex; - -message Sample { - double value = 1; - int64 timestamp_ms = 2; -} - -message LabelPair { - string name = 1; - string value = 2; -} +import "github.com/prometheus/prometheus/storage/remote/remote.proto"; -message TimeSeries { - repeated LabelPair labels = 1; - // Sorted by time, oldest sample first. - repeated Sample samples = 2; -} +package cortex; -enum MatchType { - EQUAL = 0; - NOT_EQUAL = 1; - REGEX_MATCH = 2; - REGEX_NO_MATCH = 3; +service Ingester { + rpc Push(remote.WriteRequest) returns (WriteResponse) {}; + rpc Query(QueryRequest) returns (QueryResponse) {}; + rpc LabelValues(LabelValuesRequest) returns (LabelValuesResponse) {}; + rpc UserStats(UserStatsRequest) returns (UserStatsResponse) {}; } -message LabelMatcher { - MatchType type = 1; - string name = 2; - string value = 3; +message WriteResponse { } -message ReadRequest { +message QueryRequest { int64 start_timestamp_ms = 1; int64 end_timestamp_ms = 2; repeated LabelMatcher matchers = 3; } -message ReadResponse { - repeated TimeSeries timeseries = 1; +message QueryResponse { + repeated remote.TimeSeries timeseries = 1; } message LabelValuesRequest { @@ -49,7 +32,23 @@ message LabelValuesResponse { repeated string label_values = 1; } +message UserStatsRequest { +} + message UserStatsResponse { double ingestion_rate = 1; uint64 num_series = 2; } + +enum MatchType { + EQUAL = 0; + NOT_EQUAL = 1; + REGEX_MATCH = 2; + REGEX_NO_MATCH = 3; +} + +message LabelMatcher { + MatchType type = 1; + string name = 2; + string value = 3; +} diff --git a/distributor/distributor.go b/distributor/distributor.go index 7a02d3d8c4..01a6c095d0 100644 --- a/distributor/distributor.go +++ b/distributor/distributor.go @@ -7,16 +7,20 @@ import ( "sync/atomic" "time" + "github.com/weaveworks/scope/common/instrument" + "golang.org/x/net/context" + "google.golang.org/grpc" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage/metric" - "github.com/weaveworks/scope/common/instrument" - "golang.org/x/net/context" + "github.com/prometheus/prometheus/storage/remote" - "github.com/weaveworks/cortex/ingester" + "github.com/weaveworks/cortex" "github.com/weaveworks/cortex/ring" "github.com/weaveworks/cortex/user" "github.com/weaveworks/cortex/util" + "github.com/weaveworks/cortex/util/middleware" ) var ( @@ -32,7 +36,7 @@ var ( type Distributor struct { cfg Config clientsMtx sync.RWMutex - clients map[string]*IngesterClient + clients map[string]cortex.IngesterClient queryDuration *prometheus.HistogramVec receivedSamples prometheus.Counter @@ -52,18 +56,15 @@ type ReadRing interface { GetAll() []ring.IngesterDesc } -// IngesterClientFactory creates ingester clients. -type IngesterClientFactory func(string) (*IngesterClient, error) - // Config contains the configuration require to // create a Distributor type Config struct { - Ring ReadRing - ClientFactory IngesterClientFactory + Ring ReadRing ReplicationFactor int MinReadSuccesses int HeartbeatTimeout time.Duration + RemoteTimeout time.Duration } // New constructs a new Distributor @@ -76,7 +77,7 @@ func New(cfg Config) (*Distributor, error) { } return &Distributor{ cfg: cfg, - clients: map[string]*IngesterClient{}, + clients: map[string]cortex.IngesterClient{}, queryDuration: prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "cortex", Name: "distributor_query_duration_seconds", @@ -117,9 +118,9 @@ func New(cfg Config) (*Distributor, error) { }, nil } -func (d *Distributor) getClientFor(hostname string) (*IngesterClient, error) { +func (d *Distributor) getClientFor(ingester ring.IngesterDesc) (cortex.IngesterClient, error) { d.clientsMtx.RLock() - client, ok := d.clients[hostname] + client, ok := d.clients[ingester.Hostname] d.clientsMtx.RUnlock() if ok { return client, nil @@ -127,16 +128,30 @@ func (d *Distributor) getClientFor(hostname string) (*IngesterClient, error) { d.clientsMtx.Lock() defer d.clientsMtx.Unlock() - client, ok = d.clients[hostname] + client, ok = d.clients[ingester.Hostname] if ok { return client, nil } - client, err := d.cfg.ClientFactory(hostname) - if err != nil { - return nil, err + if ingester.GRPCHostname != "" { + conn, err := grpc.Dial( + ingester.GRPCHostname, + grpc.WithInsecure(), + grpc.WithUnaryInterceptor(middleware.ClientUserHeaderInterceptor), + ) + if err != nil { + return nil, err + } + client = cortex.NewIngesterClient(conn) + } else { + var err error + client, err = NewHTTPIngesterClient(ingester.Hostname, d.cfg.RemoteTimeout) + if err != nil { + return nil, err + } } - d.clients[hostname] = client + + d.clients[ingester.Hostname] = client return client, nil } @@ -158,13 +173,14 @@ type sampleTracker struct { succeeded int32 } -// Append implements SampleAppender. -func (d *Distributor) Append(ctx context.Context, samples []*model.Sample) error { +// Push implements cortex.IngesterServer +func (d *Distributor) Push(ctx context.Context, req *remote.WriteRequest) (*cortex.WriteResponse, error) { userID, err := user.GetID(ctx) if err != nil { - return err + return nil, err } + samples := util.FromWriteRequest(req) d.receivedSamples.Add(float64(len(samples))) keys := make([]uint32, len(samples), len(samples)) @@ -174,11 +190,11 @@ func (d *Distributor) Append(ctx context.Context, samples []*model.Sample) error ingesters, err := d.cfg.Ring.BatchGet(keys, d.cfg.ReplicationFactor, ring.Write) if err != nil { - return err + return nil, err } sampleTrackers := make([]sampleTracker, len(samples), len(samples)) - samplesByIngester := map[string][]*sampleTracker{} + samplesByIngester := map[ring.IngesterDesc][]*sampleTracker{} for i := range samples { sampleTrackers[i] = sampleTracker{ sample: samples[i], @@ -190,17 +206,17 @@ func (d *Distributor) Append(ctx context.Context, samples []*model.Sample) error // Skip those that have not heartbeated in a while. NB these are still // included in the calculation of minSuccess, so if too many failed ingesters // will cause the whole write to fail. - liveIngesters := make([]string, 0, len(ingesters[i])) + liveIngesters := make([]ring.IngesterDesc, 0, len(ingesters[i])) for _, ingester := range ingesters[i] { if time.Now().Sub(ingester.Timestamp) <= d.cfg.HeartbeatTimeout { - liveIngesters = append(liveIngesters, ingester.Hostname) + liveIngesters = append(liveIngesters, ingester) } } // This is just a shortcut - if there are not minSuccess available ingesters, // after filtering out dead ones, don't even both trying. if len(liveIngesters) < sampleTrackers[i].minSuccess { - return fmt.Errorf("wanted at least %d live ingesters to process write, had %d", + return nil, fmt.Errorf("wanted at least %d live ingesters to process write, had %d", sampleTrackers[i].minSuccess, len(liveIngesters)) } @@ -212,8 +228,8 @@ func (d *Distributor) Append(ctx context.Context, samples []*model.Sample) error errs := make(chan error) for hostname, samples := range samplesByIngester { - go func(hostname string, samples []*sampleTracker) { - errs <- d.sendSamples(ctx, hostname, samples) + go func(ingester ring.IngesterDesc, samples []*sampleTracker) { + errs <- d.sendSamples(ctx, ingester, samples) }(hostname, samples) } var lastErr error @@ -225,15 +241,15 @@ func (d *Distributor) Append(ctx context.Context, samples []*model.Sample) error } for i := range sampleTrackers { if sampleTrackers[i].succeeded < int32(sampleTrackers[i].minSuccess) { - return fmt.Errorf("need %d successful writes, only got %d, last error was: %v", + return nil, fmt.Errorf("need %d successful writes, only got %d, last error was: %v", sampleTrackers[i].minSuccess, sampleTrackers[i].succeeded, lastErr) } } - return nil + return &cortex.WriteResponse{}, nil } -func (d *Distributor) sendSamples(ctx context.Context, hostname string, sampleTrackers []*sampleTracker) error { - client, err := d.getClientFor(hostname) +func (d *Distributor) sendSamples(ctx context.Context, ingester ring.IngesterDesc, sampleTrackers []*sampleTracker) error { + client, err := d.getClientFor(ingester) if err != nil { return err } @@ -242,12 +258,13 @@ func (d *Distributor) sendSamples(ctx context.Context, hostname string, sampleTr samples[i] = sampleTrackers[i].sample } err = instrument.TimeRequestHistogram("send", d.sendDuration, func() error { - return client.Append(ctx, samples) + _, err := client.Push(ctx, util.ToWriteRequest(samples)) + return err }) if err != nil { - d.ingesterAppendFailures.WithLabelValues(hostname).Inc() + d.ingesterAppendFailures.WithLabelValues(ingester.Hostname).Inc() } - d.ingesterAppends.WithLabelValues(hostname).Inc() + d.ingesterAppends.WithLabelValues(ingester.Hostname).Inc() for i := range sampleTrackers { atomic.AddInt32(&sampleTrackers[i].succeeded, 1) } @@ -296,11 +313,17 @@ func (d *Distributor) Query(ctx context.Context, from, to model.Time, matchers . successes := 0 var lastErr error for _, ing := range ingesters { - client, err := d.getClientFor(ing.Hostname) + client, err := d.getClientFor(ing) if err != nil { return err } - matrix, err := client.Query(ctx, from, to, matchers...) + + req, err := util.ToQueryRequest(from, to, matchers...) + if err != nil { + return err + } + + resp, err := client.Query(ctx, req) d.ingesterQueries.WithLabelValues(ing.Hostname).Inc() if err != nil { lastErr = err @@ -309,7 +332,7 @@ func (d *Distributor) Query(ctx context.Context, from, to model.Time, matchers . } successes++ - for _, ss := range matrix { + for _, ss := range util.FromQueryResponse(resp) { fp := ss.Metric.Fingerprint() if mss, ok := fpToSampleStream[fp]; !ok { fpToSampleStream[fp] = &model.SampleStream{ @@ -339,17 +362,19 @@ func (d *Distributor) Query(ctx context.Context, from, to model.Time, matchers . // LabelValuesForLabelName returns all of the label values that are associated with a given label name. func (d *Distributor) LabelValuesForLabelName(ctx context.Context, labelName model.LabelName) (model.LabelValues, error) { valueSet := map[model.LabelValue]struct{}{} - for _, c := range d.cfg.Ring.GetAll() { - client, err := d.getClientFor(c.Hostname) + for _, ingester := range d.cfg.Ring.GetAll() { + client, err := d.getClientFor(ingester) if err != nil { return nil, err } - vals, err := client.LabelValuesForLabelName(ctx, labelName) + resp, err := client.LabelValues(ctx, &cortex.LabelValuesRequest{ + LabelName: string(labelName), + }) if err != nil { return nil, err } - for _, v := range vals { - valueSet[v] = struct{}{} + for _, v := range resp.LabelValues { + valueSet[model.LabelValue(v)] = struct{}{} } } @@ -361,19 +386,19 @@ func (d *Distributor) LabelValuesForLabelName(ctx context.Context, labelName mod } // UserStats returns statistics about the current user. -func (d *Distributor) UserStats(ctx context.Context) (*ingester.UserStats, error) { - totalStats := &ingester.UserStats{} - for _, c := range d.cfg.Ring.GetAll() { - client, err := d.getClientFor(c.Hostname) +func (d *Distributor) UserStats(ctx context.Context) (*UserStats, error) { + totalStats := &UserStats{} + for _, ingester := range d.cfg.Ring.GetAll() { + client, err := d.getClientFor(ingester) if err != nil { return nil, err } - stats, err := client.UserStats(ctx) + resp, err := client.UserStats(ctx, &cortex.UserStatsRequest{}) if err != nil { return nil, err } - totalStats.IngestionRate += stats.IngestionRate - totalStats.NumSeries += stats.NumSeries + totalStats.IngestionRate += resp.IngestionRate + totalStats.NumSeries += resp.NumSeries } totalStats.IngestionRate /= float64(d.cfg.ReplicationFactor) @@ -382,11 +407,6 @@ func (d *Distributor) UserStats(ctx context.Context) (*ingester.UserStats, error return totalStats, nil } -// NeedsThrottling implements SampleAppender. -func (*Distributor) NeedsThrottling(_ context.Context) bool { - return false -} - // Describe implements prometheus.Collector. func (d *Distributor) Describe(ch chan<- *prometheus.Desc) { d.queryDuration.Describe(ch) diff --git a/distributor/http_ingester_client.go b/distributor/http_ingester_client.go new file mode 100644 index 0000000000..08b1067c4a --- /dev/null +++ b/distributor/http_ingester_client.go @@ -0,0 +1,149 @@ +package distributor + +import ( + "bytes" + "fmt" + "io" + "net/http" + "time" + + "github.com/golang/protobuf/proto" + "github.com/golang/snappy" + "golang.org/x/net/context" + "google.golang.org/grpc" + + "github.com/prometheus/common/expfmt" + "github.com/prometheus/prometheus/storage/remote" + + "github.com/weaveworks/cortex" + "github.com/weaveworks/cortex/user" +) + +// httpIngesterClient is a client library for the ingester +type httpIngesterClient struct { + address string + client http.Client + timeout time.Duration +} + +// IngesterError is an error we got from an ingester. +type IngesterError struct { + StatusCode int + err error +} + +func errStatusCode(code int, status string) IngesterError { + return IngesterError{ + StatusCode: code, + err: fmt.Errorf("server returned HTTP status %s", status), + } +} + +func (i IngesterError) Error() string { + return i.err.Error() +} + +// NewHTTPIngesterClient makes a new IngesterClient. This client is careful to +// propagate the user ID from Distributor -> Ingester. +func NewHTTPIngesterClient(address string, timeout time.Duration) (cortex.IngesterClient, error) { + return &httpIngesterClient{ + address: address, + client: http.Client{ + Timeout: timeout, + }, + timeout: timeout, + }, nil +} + +// Push adds new samples to the ingester +func (c *httpIngesterClient) Push(ctx context.Context, req *remote.WriteRequest, _ ...grpc.CallOption) (*cortex.WriteResponse, error) { + if err := c.doRequest(ctx, "/push", req, nil, true); err != nil { + return nil, err + } + return &cortex.WriteResponse{}, nil +} + +func (c *httpIngesterClient) Query(ctx context.Context, req *cortex.QueryRequest, _ ...grpc.CallOption) (*cortex.QueryResponse, error) { + resp := &cortex.QueryResponse{} + if err := c.doRequest(ctx, "/query", req, resp, false); err != nil { + return nil, err + } + return resp, nil +} + +// LabelValues returns all of the label values that are associated with a given label name. +func (c *httpIngesterClient) LabelValues(ctx context.Context, req *cortex.LabelValuesRequest, _ ...grpc.CallOption) (*cortex.LabelValuesResponse, error) { + resp := &cortex.LabelValuesResponse{} + err := c.doRequest(ctx, "/label_values", req, resp, false) + if err != nil { + return nil, err + } + return resp, nil +} + +// UserStats returns stats for the current user. +func (c *httpIngesterClient) UserStats(ctx context.Context, in *cortex.UserStatsRequest, _ ...grpc.CallOption) (*cortex.UserStatsResponse, error) { + resp := &cortex.UserStatsResponse{} + err := c.doRequest(ctx, "/user_stats", nil, resp, false) + if err != nil { + return nil, err + } + return resp, nil +} + +func (c *httpIngesterClient) doRequest(ctx context.Context, endpoint string, req proto.Message, resp proto.Message, compressed bool) error { + userID, err := user.GetID(ctx) + if err != nil { + return err + } + + var buf bytes.Buffer + if req != nil { + data, err := proto.Marshal(req) + if err != nil { + return fmt.Errorf("unable to marshal request: %v", err) + } + + var writer io.Writer = &buf + if compressed { + writer = snappy.NewWriter(writer) + } + if _, err := writer.Write(data); err != nil { + return err + } + } + + httpReq, err := http.NewRequest("POST", fmt.Sprintf("http://%s%s", c.address, endpoint), &buf) + if err != nil { + return fmt.Errorf("unable to create request: %v", err) + } + httpReq.Header.Add(user.UserIDHeaderName, userID) + // TODO: This isn't actually the correct Content-type. + httpReq.Header.Set("Content-Type", string(expfmt.FmtProtoDelim)) + httpResp, err := c.client.Do(httpReq) + if err != nil { + return fmt.Errorf("error sending request: %v", err) + } + defer httpResp.Body.Close() + if httpResp.StatusCode/100 != 2 { + return errStatusCode(httpResp.StatusCode, httpResp.Status) + } + if resp == nil { + return nil + } + + buf.Reset() + reader := httpResp.Body.(io.Reader) + if compressed { + reader = snappy.NewReader(reader) + } + if _, err = buf.ReadFrom(reader); err != nil { + return fmt.Errorf("unable to read response body: %v", err) + } + + err = proto.Unmarshal(buf.Bytes(), resp) + if err != nil { + return fmt.Errorf("unable to unmarshal response body: %v", err) + } + return nil +} diff --git a/distributor/http_server.go b/distributor/http_server.go new file mode 100644 index 0000000000..7592825e2e --- /dev/null +++ b/distributor/http_server.go @@ -0,0 +1,56 @@ +package distributor + +import ( + "net/http" + + "github.com/prometheus/common/log" + "github.com/prometheus/prometheus/storage/remote" + + "github.com/weaveworks/cortex/util" +) + +// PushHandler is a http.Handler which accepts WriteRequests. +func (d *Distributor) PushHandler(w http.ResponseWriter, r *http.Request) { + var req remote.WriteRequest + ctx, abort := util.ParseProtoRequest(w, r, &req, true) + if abort { + return + } + + _, err := d.Push(ctx, &req) + if err != nil { + switch e := err.(type) { + case IngesterError: + switch { + case 400 <= e.StatusCode && e.StatusCode < 500: + log.Warnf("push err: %v", err) + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + } + log.Errorf("append err: %v", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + } +} + +// UserStats models ingestion statistics for one user. +type UserStats struct { + IngestionRate float64 `json:"ingestionRate"` + NumSeries uint64 `json:"numSeries"` +} + +// UserStatsHandler handles user stats to the Distributor. +func (d *Distributor) UserStatsHandler(w http.ResponseWriter, r *http.Request) { + ctx, abort := util.ParseProtoRequest(w, r, nil, false) + if abort { + return + } + + stats, err := d.UserStats(ctx) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + util.WriteJSONResponse(w, stats) +} diff --git a/distributor/ingester_client.go b/distributor/ingester_client.go deleted file mode 100644 index 1be10b544b..0000000000 --- a/distributor/ingester_client.go +++ /dev/null @@ -1,230 +0,0 @@ -package distributor - -import ( - "bytes" - "fmt" - "io" - "net/http" - "time" - - "github.com/golang/protobuf/proto" - "github.com/golang/snappy" - "github.com/prometheus/common/expfmt" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/storage/remote" - "golang.org/x/net/context" - - "github.com/weaveworks/cortex" - "github.com/weaveworks/cortex/ingester" - "github.com/weaveworks/cortex/user" -) - -// IngesterClient is a client library for the ingester -type IngesterClient struct { - address string - client http.Client - timeout time.Duration -} - -// IngesterError is an error we got from an ingester. -type IngesterError struct { - StatusCode int - err error -} - -func errStatusCode(code int, status string) IngesterError { - return IngesterError{ - StatusCode: code, - err: fmt.Errorf("server returned HTTP status %s", status), - } -} - -func (i IngesterError) Error() string { - return i.err.Error() -} - -// NewIngesterClient makes a new IngesterClient. This client is careful to -// propagate the user ID from Distributor -> Ingester. -func NewIngesterClient(address string, timeout time.Duration) (*IngesterClient, error) { - client := http.Client{ - Timeout: timeout, - } - return &IngesterClient{ - address: address, - client: client, - timeout: timeout, - }, nil -} - -// Append adds new samples to the ingester -func (c *IngesterClient) Append(ctx context.Context, samples []*model.Sample) error { - req := &remote.WriteRequest{ - Timeseries: make([]*remote.TimeSeries, 0, len(samples)), - } - - for _, s := range samples { - ts := &remote.TimeSeries{ - Labels: make([]*remote.LabelPair, 0, len(s.Metric)), - } - for k, v := range s.Metric { - ts.Labels = append(ts.Labels, - &remote.LabelPair{ - Name: string(k), - Value: string(v), - }) - } - ts.Samples = []*remote.Sample{ - { - Value: float64(s.Value), - TimestampMs: int64(s.Timestamp), - }, - } - req.Timeseries = append(req.Timeseries, ts) - } - - return c.doRequest(ctx, "/push", req, nil, true) -} - -// Query implements Querier. -func (c *IngesterClient) Query(ctx context.Context, from, to model.Time, matchers ...*metric.LabelMatcher) (model.Matrix, error) { - req := &cortex.ReadRequest{ - StartTimestampMs: int64(from), - EndTimestampMs: int64(to), - } - for _, matcher := range matchers { - var mType cortex.MatchType - switch matcher.Type { - case metric.Equal: - mType = cortex.MatchType_EQUAL - case metric.NotEqual: - mType = cortex.MatchType_NOT_EQUAL - case metric.RegexMatch: - mType = cortex.MatchType_REGEX_MATCH - case metric.RegexNoMatch: - mType = cortex.MatchType_REGEX_NO_MATCH - default: - panic("invalid matcher type") - } - req.Matchers = append(req.Matchers, &cortex.LabelMatcher{ - Type: mType, - Name: string(matcher.Name), - Value: string(matcher.Value), - }) - } - - resp := &cortex.ReadResponse{} - err := c.doRequest(ctx, "/query", req, resp, false) - if err != nil { - return nil, err - } - - m := make(model.Matrix, 0, len(resp.Timeseries)) - for _, ts := range resp.Timeseries { - var ss model.SampleStream - ss.Metric = model.Metric{} - for _, l := range ts.Labels { - ss.Metric[model.LabelName(l.Name)] = model.LabelValue(l.Value) - } - - ss.Values = make([]model.SamplePair, 0, len(ts.Samples)) - for _, s := range ts.Samples { - ss.Values = append(ss.Values, model.SamplePair{ - Value: model.SampleValue(s.Value), - Timestamp: model.Time(s.TimestampMs), - }) - } - m = append(m, &ss) - } - - return m, nil -} - -// LabelValuesForLabelName returns all of the label values that are associated with a given label name. -func (c *IngesterClient) LabelValuesForLabelName(ctx context.Context, ln model.LabelName) (model.LabelValues, error) { - req := &cortex.LabelValuesRequest{ - LabelName: string(ln), - } - resp := &cortex.LabelValuesResponse{} - err := c.doRequest(ctx, "/label_values", req, resp, false) - if err != nil { - return nil, err - } - - values := make(model.LabelValues, 0, len(resp.LabelValues)) - for _, v := range resp.LabelValues { - values = append(values, model.LabelValue(v)) - } - return values, nil -} - -// UserStats returns stats for the current user. -func (c *IngesterClient) UserStats(ctx context.Context) (*ingester.UserStats, error) { - resp := &cortex.UserStatsResponse{} - err := c.doRequest(ctx, "/user_stats", nil, resp, false) - if err != nil { - return nil, err - } - - return &ingester.UserStats{ - IngestionRate: resp.IngestionRate, - NumSeries: resp.NumSeries, - }, nil -} - -func (c *IngesterClient) doRequest(ctx context.Context, endpoint string, req proto.Message, resp proto.Message, compressed bool) error { - userID, err := user.GetID(ctx) - if err != nil { - return err - } - - var buf bytes.Buffer - if req != nil { - data, err := proto.Marshal(req) - if err != nil { - return fmt.Errorf("unable to marshal request: %v", err) - } - - var writer io.Writer = &buf - if compressed { - writer = snappy.NewWriter(writer) - } - if _, err := writer.Write(data); err != nil { - return err - } - } - - httpReq, err := http.NewRequest("POST", fmt.Sprintf("http://%s%s", c.address, endpoint), &buf) - if err != nil { - return fmt.Errorf("unable to create request: %v", err) - } - httpReq.Header.Add(cortex.UserIDHeaderName, userID) - // TODO: This isn't actually the correct Content-type. - httpReq.Header.Set("Content-Type", string(expfmt.FmtProtoDelim)) - httpResp, err := c.client.Do(httpReq) - if err != nil { - return fmt.Errorf("error sending request: %v", err) - } - defer httpResp.Body.Close() - if httpResp.StatusCode/100 != 2 { - return errStatusCode(httpResp.StatusCode, httpResp.Status) - } - if resp == nil { - return nil - } - - buf.Reset() - reader := httpResp.Body.(io.Reader) - if compressed { - reader = snappy.NewReader(reader) - } - if _, err = buf.ReadFrom(reader); err != nil { - return fmt.Errorf("unable to read response body: %v", err) - } - - err = proto.Unmarshal(buf.Bytes(), resp) - if err != nil { - return fmt.Errorf("unable to unmarshal response body: %v", err) - } - return nil -} diff --git a/ingester/http_server.go b/ingester/http_server.go new file mode 100644 index 0000000000..2629da2f5c --- /dev/null +++ b/ingester/http_server.go @@ -0,0 +1,94 @@ +package ingester + +import ( + "net/http" + + "github.com/prometheus/common/log" + "github.com/prometheus/prometheus/storage/remote" + + "github.com/weaveworks/cortex" + "github.com/weaveworks/cortex/util" +) + +// PushHandler is a http.Handler that accepts proto encoded samples. +func (i *Ingester) PushHandler(w http.ResponseWriter, r *http.Request) { + var req remote.WriteRequest + ctx, abort := util.ParseProtoRequest(w, r, &req, true) + if abort { + return + } + + _, err := i.Push(ctx, &req) + if err != nil { + switch err { + case ErrOutOfOrderSample, ErrDuplicateSampleForTimestamp: + log.Warnf("append err: %v", err) + http.Error(w, err.Error(), http.StatusBadRequest) + default: + log.Errorf("append err: %v", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + } + } +} + +// QueryHandler is a http.Handler that accepts protobuf formatted +// query requests and serves them. +func (i *Ingester) QueryHandler(w http.ResponseWriter, r *http.Request) { + var req cortex.QueryRequest + ctx, abort := util.ParseProtoRequest(w, r, &req, false) + if abort { + return + } + + resp, err := i.Query(ctx, &req) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + util.WriteProtoResponse(w, resp) +} + +// LabelValuesHandler handles label values +func (i *Ingester) LabelValuesHandler(w http.ResponseWriter, r *http.Request) { + var req cortex.LabelValuesRequest + ctx, abort := util.ParseProtoRequest(w, r, &req, false) + if abort { + return + } + + resp, err := i.LabelValues(ctx, &req) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + util.WriteProtoResponse(w, resp) +} + +// UserStatsHandler handles user stats requests to the Ingester. +func (i *Ingester) UserStatsHandler(w http.ResponseWriter, r *http.Request) { + ctx, abort := util.ParseProtoRequest(w, r, nil, false) + if abort { + return + } + + resp, err := i.UserStats(ctx, &cortex.UserStatsRequest{}) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + util.WriteProtoResponse(w, resp) +} + +// ReadinessHandler returns 204 when the ingester is ready, +// 500 otherwise. It's used by kubernetes to indicate if the ingester +// pool is ready to have ingesters added / removed. +func (i *Ingester) ReadinessHandler(w http.ResponseWriter, r *http.Request) { + if i.Ready() { + w.WriteHeader(http.StatusNoContent) + } else { + w.WriteHeader(http.StatusInternalServerError) + } +} diff --git a/ingester/ingester.go b/ingester/ingester.go index b4a9b7e8d5..bf158b96d3 100644 --- a/ingester/ingester.go +++ b/ingester/ingester.go @@ -12,11 +12,14 @@ import ( "github.com/prometheus/common/model" prom_chunk "github.com/prometheus/prometheus/storage/local/chunk" "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/remote" "golang.org/x/net/context" - cortex "github.com/weaveworks/cortex/chunk" + "github.com/weaveworks/cortex" + cortex_chunk "github.com/weaveworks/cortex/chunk" "github.com/weaveworks/cortex/ring" "github.com/weaveworks/cortex/user" + "github.com/weaveworks/cortex/util" ) const ( @@ -69,7 +72,7 @@ var ( // Its like MemorySeriesStorage, but simpler. type Ingester struct { cfg Config - chunkStore cortex.Store + chunkStore cortex_chunk.Store stopLock sync.RWMutex stopped bool quit chan struct{} @@ -99,16 +102,11 @@ type Config struct { MaxChunkIdle time.Duration RateUpdatePeriod time.Duration ConcurrentFlushes int + GRPCListenPort int Ring *ring.Ring } -// UserStats models ingestion statistics for one user. -type UserStats struct { - IngestionRate float64 `json:"ingestionRate"` - NumSeries uint64 `json:"numSeries"` -} - type userState struct { userID string fpLocker *fingerprintLocker @@ -134,7 +132,7 @@ func (o *flushOp) Priority() int64 { } // New constructs a new Ingester. -func New(cfg Config, chunkStore cortex.Store) (*Ingester, error) { +func New(cfg Config, chunkStore cortex_chunk.Store) (*Ingester, error) { if cfg.FlushCheckPeriod == 0 { cfg.FlushCheckPeriod = 1 * time.Minute } @@ -243,19 +241,14 @@ func (i *Ingester) getStateFor(ctx context.Context) (*userState, error) { return state, nil } -// NeedsThrottling implements storage.SampleAppender. -func (*Ingester) NeedsThrottling(_ context.Context) bool { - return false -} - -// Append implements storage.SampleAppender. -func (i *Ingester) Append(ctx context.Context, samples []*model.Sample) error { - for _, sample := range samples { +// Push implements cortex.IngesterServer +func (i *Ingester) Push(ctx context.Context, req *remote.WriteRequest) (*cortex.WriteResponse, error) { + for _, sample := range util.FromWriteRequest(req) { if err := i.append(ctx, sample); err != nil { - return err + return nil, err } } - return nil + return &cortex.WriteResponse{}, nil } func (i *Ingester) append(ctx context.Context, sample *model.Sample) error { @@ -336,8 +329,22 @@ func (u *userState) getOrCreateSeries(metric model.Metric) (model.Fingerprint, * return fp, series, nil } -// Query implements cortex.Querier. -func (i *Ingester) Query(ctx context.Context, from, through model.Time, matchers ...*metric.LabelMatcher) (model.Matrix, error) { +// Query implements service.IngesterServer +func (i *Ingester) Query(ctx context.Context, req *cortex.QueryRequest) (*cortex.QueryResponse, error) { + start, end, matchers, err := util.FromQueryRequest(req) + if err != nil { + return nil, err + } + + matrix, err := i.query(ctx, start, end, matchers...) + if err != nil { + return nil, err + } + + return util.ToQueryResponse(matrix), nil +} + +func (i *Ingester) query(ctx context.Context, from, through model.Time, matchers ...*metric.LabelMatcher) (model.Matrix, error) { i.queries.Inc() state, err := i.getStateFor(ctx) @@ -418,23 +425,28 @@ func samplesForRange(s *memorySeries, from, through model.Time) ([]model.SampleP return values, nil } -// LabelValuesForLabelName returns all label values that are associated with a given label name. -func (i *Ingester) LabelValuesForLabelName(ctx context.Context, name model.LabelName) (model.LabelValues, error) { +// LabelValues returns all label values that are associated with a given label name. +func (i *Ingester) LabelValues(ctx context.Context, req *cortex.LabelValuesRequest) (*cortex.LabelValuesResponse, error) { state, err := i.getStateFor(ctx) if err != nil { return nil, err } - return state.index.lookupLabelValues(name), nil + resp := &cortex.LabelValuesResponse{} + for _, v := range state.index.lookupLabelValues(model.LabelName(req.LabelName)) { + resp.LabelValues = append(resp.LabelValues, string(v)) + } + + return resp, nil } // UserStats returns ingestion statistics for the current user. -func (i *Ingester) UserStats(ctx context.Context) (*UserStats, error) { +func (i *Ingester) UserStats(ctx context.Context, req *cortex.UserStatsRequest) (*cortex.UserStatsResponse, error) { state, err := i.getStateFor(ctx) if err != nil { return nil, err } - return &UserStats{ + return &cortex.UserStatsResponse{ IngestionRate: state.ingestedSamples.rate(), NumSeries: uint64(state.fpToSeries.length()), }, nil @@ -611,12 +623,12 @@ func (i *Ingester) flushLoop(j int) { } func (i *Ingester) flushChunks(ctx context.Context, fp model.Fingerprint, metric model.Metric, chunkDescs []*prom_chunk.Desc) error { - wireChunks := make([]cortex.Chunk, 0, len(chunkDescs)) + wireChunks := make([]cortex_chunk.Chunk, 0, len(chunkDescs)) for _, chunkDesc := range chunkDescs { i.chunkUtilization.Observe(chunkDesc.C.Utilization()) i.chunkLength.Observe(chunkDesc.C.Len()) i.chunkAge.Observe(model.Now().Sub(chunkDesc.ChunkFirstTime).Seconds()) - wireChunks = append(wireChunks, cortex.NewChunk(fp, metric, chunkDesc)) + wireChunks = append(wireChunks, cortex_chunk.NewChunk(fp, metric, chunkDesc)) } return i.chunkStore.Put(ctx, wireChunks) } diff --git a/ingester/ingester_test.go b/ingester/ingester_test.go index 129a57b4ac..66e746d981 100644 --- a/ingester/ingester_test.go +++ b/ingester/ingester_test.go @@ -14,6 +14,7 @@ import ( "github.com/weaveworks/cortex/chunk" "github.com/weaveworks/cortex/user" + "github.com/weaveworks/cortex/util" ) type testStore struct { @@ -98,7 +99,7 @@ func TestIngesterAppend(t *testing.T) { // Append samples. for _, userID := range userIDs { ctx := user.WithID(context.Background(), userID) - err = ing.Append(ctx, matrixToSamples(testData[userID])) + _, err = ing.Push(ctx, util.ToWriteRequest(matrixToSamples(testData[userID]))) if err != nil { t.Fatal(err) } @@ -111,12 +112,19 @@ func TestIngesterAppend(t *testing.T) { if err != nil { t.Fatal(err) } - res, err := ing.Query(ctx, model.Earliest, model.Latest, matcher) + + req, err := util.ToQueryRequest(model.Earliest, model.Latest, matcher) if err != nil { t.Fatal(err) } - sort.Sort(res) + resp, err := ing.Query(ctx, req) + if err != nil { + t.Fatal(err) + } + + res := util.FromQueryResponse(resp) + sort.Sort(res) if !reflect.DeepEqual(res, testData[userID]) { t.Fatalf("unexpected query result\n\nwant:\n\n%v\n\ngot:\n\n%v\n\n", testData[userID], res) } diff --git a/ring/ingester_lifecycle.go b/ring/ingester_lifecycle.go index 6cded5c5a3..91a8b1de9a 100644 --- a/ring/ingester_lifecycle.go +++ b/ring/ingester_lifecycle.go @@ -26,10 +26,11 @@ type IngesterRegistration struct { consul ConsulClient numTokens int - id string - hostname string - quit chan struct{} - wait sync.WaitGroup + id string + hostname string + grpcHostname string + quit chan struct{} + wait sync.WaitGroup // We need to remember the ingester state just in case consul goes away and comes // back empty. Channel is used to tell the actor to update consul on state changes. @@ -40,7 +41,7 @@ type IngesterRegistration struct { } // RegisterIngester registers an ingester with Consul. -func RegisterIngester(consulClient ConsulClient, listenPort, numTokens int) (*IngesterRegistration, error) { +func RegisterIngester(consulClient ConsulClient, listenPort, grpcPort, numTokens int) (*IngesterRegistration, error) { hostname, err := os.Hostname() if err != nil { return nil, err @@ -58,8 +59,9 @@ func RegisterIngester(consulClient ConsulClient, listenPort, numTokens int) (*In id: hostname, // hostname is the ip+port of this instance, written to consul so // the distributors know where to connect. - hostname: fmt.Sprintf("%s:%d", addr, listenPort), - quit: make(chan struct{}), + hostname: fmt.Sprintf("%s:%d", addr, listenPort), + grpcHostname: fmt.Sprintf("%s:%d", addr, grpcPort), + quit: make(chan struct{}), // Only read/written on actor goroutine. state: Active, @@ -123,8 +125,7 @@ func (r *IngesterRegistration) pickTokens() []uint32 { tokens = append(tokens, newTokens...) } sort.Sort(sortableUint32(tokens)) - - ringDesc.addIngester(r.id, r.hostname, tokens, r.state) + ringDesc.addIngester(r.id, r.hostname, r.grpcHostname, tokens, r.state) return ringDesc, true, nil } if err := r.consul.CAS(consulKey, descFactory, pickTokens); err != nil { @@ -146,7 +147,7 @@ func (r *IngesterRegistration) heartbeat(tokens []uint32) { if !ok { // consul must have restarted log.Infof("Found empty ring, inserting tokens!") - ringDesc.addIngester(r.id, r.hostname, tokens, r.state) + ringDesc.addIngester(r.id, r.hostname, r.grpcHostname, tokens, r.state) } else { ingesterDesc.Timestamp = time.Now() ingesterDesc.State = r.state diff --git a/ring/model.go b/ring/model.go index 768d1ea4cb..d6b866a5ad 100644 --- a/ring/model.go +++ b/ring/model.go @@ -38,6 +38,8 @@ type IngesterDesc struct { Hostname string `json:"hostname"` Timestamp time.Time `json:"timestamp"` State IngesterState `json:"state"` + + GRPCHostname string `json:"grpc_hostname"` } // TokenDescs is a sortable list of TokenDescs @@ -63,11 +65,12 @@ func newDesc() *Desc { } } -func (d *Desc) addIngester(id, hostname string, tokens []uint32, state IngesterState) { +func (d *Desc) addIngester(id, hostname, grpcHostname string, tokens []uint32, state IngesterState) { d.Ingesters[id] = IngesterDesc{ - Hostname: hostname, - Timestamp: time.Now(), - State: state, + Hostname: hostname, + GRPCHostname: grpcHostname, + Timestamp: time.Now(), + State: state, } for _, token := range tokens { diff --git a/ruler/ruler.go b/ruler/ruler.go index d9bce3a639..b86d3c448e 100644 --- a/ruler/ruler.go +++ b/ruler/ruler.go @@ -16,11 +16,11 @@ import ( "github.com/prometheus/prometheus/rules" "golang.org/x/net/context" - "github.com/weaveworks/cortex" "github.com/weaveworks/cortex/chunk" "github.com/weaveworks/cortex/distributor" "github.com/weaveworks/cortex/querier" "github.com/weaveworks/cortex/user" + "github.com/weaveworks/cortex/util" ) // Config is the configuration for the recording rules server. @@ -105,7 +105,7 @@ func (r *Ruler) getWorkerFor(userID string) (Worker, error) { func (r *Ruler) getManager(userID string) *rules.Manager { ctx := user.WithID(context.Background(), userID) - appender := appenderAdapter{appender: r.distributor, ctx: ctx} + appender := appenderAdapter{distributor: r.distributor, ctx: ctx} queryable := querier.NewQueryable(r.distributor, r.chunkStore) engine := promql.NewEngine(queryable, nil) return rules.NewManager(&rules.ManagerOptions{ @@ -149,14 +149,16 @@ func (r *Ruler) getConfig(userID string) (*config.Config, error) { }, nil } -// appenderAdapter adapts cortex.SampleAppender to prometheus.SampleAppender +// appenderAdapter adapts a distributor.Distributor to prometheus.SampleAppender type appenderAdapter struct { - appender cortex.SampleAppender - ctx context.Context + distributor *distributor.Distributor + ctx context.Context } func (a appenderAdapter) Append(sample *model.Sample) error { - return a.appender.Append(a.ctx, []*model.Sample{sample}) + req := util.ToWriteRequest([]*model.Sample{sample}) + _, err := a.distributor.Push(a.ctx, req) + return err } func (a appenderAdapter) NeedsThrottling() bool { diff --git a/server.go b/server.go deleted file mode 100644 index b52d7459d4..0000000000 --- a/server.go +++ /dev/null @@ -1,292 +0,0 @@ -// 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 cortex - -import ( - "bytes" - "encoding/json" - "fmt" - "io/ioutil" - "net/http" - - "github.com/golang/protobuf/proto" - "github.com/golang/snappy" - "github.com/prometheus/common/log" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/storage/remote" - "golang.org/x/net/context" - - "github.com/weaveworks/cortex/ingester" - "github.com/weaveworks/cortex/querier" - "github.com/weaveworks/cortex/user" -) - -// UserIDHeaderName is a legacy from scope as a service. -const UserIDHeaderName = "X-Scope-OrgID" - -// SampleAppender is the interface to append samples to both, local and remote -// storage. All methods are goroutine-safe. -type SampleAppender interface { - Append(context.Context, []*model.Sample) error -} - -func parseRequest(w http.ResponseWriter, r *http.Request, req proto.Message) (ctx context.Context, abort bool) { - userID := r.Header.Get(UserIDHeaderName) - if userID == "" { - http.Error(w, "", http.StatusUnauthorized) - return nil, true - } - - ctx = user.WithID(context.Background(), userID) - - if req == nil { - return ctx, false - } - - buf := bytes.Buffer{} - _, err := buf.ReadFrom(r.Body) - if err != nil { - log.Errorf(err.Error()) - http.Error(w, err.Error(), http.StatusBadRequest) - return nil, true - } - err = proto.Unmarshal(buf.Bytes(), req) - if err != nil { - log.Errorf(err.Error()) - http.Error(w, err.Error(), http.StatusBadRequest) - return nil, true - } - - return ctx, false -} - -func writeProtoResponse(w http.ResponseWriter, resp proto.Message) { - data, err := proto.Marshal(resp) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - if _, err = w.Write(data); err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - // TODO: set Content-type. -} - -func writeJSONResponse(w http.ResponseWriter, v interface{}) { - data, err := json.Marshal(v) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - if _, err = w.Write(data); err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - w.Header().Set("Content-Type", "application/json") -} - -func getSamples(req *remote.WriteRequest) []*model.Sample { - var samples []*model.Sample - for _, ts := range req.Timeseries { - metric := model.Metric{} - for _, l := range ts.Labels { - metric[model.LabelName(l.Name)] = model.LabelValue(l.Value) - } - - for _, s := range ts.Samples { - samples = append(samples, &model.Sample{ - Metric: metric, - Value: model.SampleValue(s.Value), - Timestamp: model.Time(s.TimestampMs), - }) - } - } - return samples -} - -// AppenderHandler returns a http.Handler that accepts proto encoded samples. -func AppenderHandler(appender SampleAppender, errorHandler func(http.ResponseWriter, error)) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - userID := r.Header.Get(UserIDHeaderName) - if userID == "" { - http.Error(w, "", http.StatusUnauthorized) - return - } - ctx := user.WithID(context.Background(), userID) - - reqBuf, err := ioutil.ReadAll(snappy.NewReader(r.Body)) - if err != nil { - log.Errorf("read err: %v", err) - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - var req remote.WriteRequest - if err := proto.Unmarshal(reqBuf, &req); err != nil { - log.Errorf("unmarshall err: %v", err) - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - err = appender.Append(ctx, getSamples(&req)) - if err != nil { - errorHandler(w, err) - } - }) -} - -// QueryHandler returns a http.Handler that accepts protobuf formatted -// query requests and serves them. -func QueryHandler(querier querier.Querier) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - req := &ReadRequest{} - ctx, abort := parseRequest(w, r, req) - if abort { - return - } - - matchers := make(metric.LabelMatchers, 0, len(req.Matchers)) - for _, matcher := range req.Matchers { - var mtype metric.MatchType - switch matcher.Type { - case MatchType_EQUAL: - mtype = metric.Equal - case MatchType_NOT_EQUAL: - mtype = metric.NotEqual - case MatchType_REGEX_MATCH: - mtype = metric.RegexMatch - case MatchType_REGEX_NO_MATCH: - mtype = metric.RegexNoMatch - default: - http.Error(w, "invalid matcher type", http.StatusBadRequest) - return - } - matcher, err := metric.NewLabelMatcher(mtype, model.LabelName(matcher.Name), model.LabelValue(matcher.Value)) - if err != nil { - http.Error(w, fmt.Sprintf("error creating matcher: %v", err), http.StatusBadRequest) - return - } - matchers = append(matchers, matcher) - } - - start := model.Time(req.StartTimestampMs) - end := model.Time(req.EndTimestampMs) - - res, err := querier.Query(ctx, start, end, matchers...) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - resp := &ReadResponse{} - for _, ss := range res { - ts := &TimeSeries{} - for k, v := range ss.Metric { - ts.Labels = append(ts.Labels, - &LabelPair{ - Name: string(k), - Value: string(v), - }) - } - ts.Samples = make([]*Sample, 0, len(ss.Values)) - for _, s := range ss.Values { - ts.Samples = append(ts.Samples, &Sample{ - Value: float64(s.Value), - TimestampMs: int64(s.Timestamp), - }) - } - resp.Timeseries = append(resp.Timeseries, ts) - } - - writeProtoResponse(w, resp) - }) -} - -// LabelValuesHandler handles label values -func LabelValuesHandler(querier querier.Querier) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - req := &LabelValuesRequest{} - ctx, abort := parseRequest(w, r, req) - if abort { - return - } - - values, err := querier.LabelValuesForLabelName(ctx, model.LabelName(req.LabelName)) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - resp := &LabelValuesResponse{} - for _, v := range values { - resp.LabelValues = append(resp.LabelValues, string(v)) - } - - writeProtoResponse(w, resp) - }) -} - -// IngesterUserStatsHandler handles user stats requests to the Ingester. -func IngesterUserStatsHandler(statsFn func(context.Context) (*ingester.UserStats, error)) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - ctx, abort := parseRequest(w, r, nil) - if abort { - return - } - - stats, err := statsFn(ctx) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - writeProtoResponse(w, &UserStatsResponse{ - IngestionRate: stats.IngestionRate, - NumSeries: stats.NumSeries, - }) - }) -} - -// IngesterReadinessHandler returns 204 when the ingester is ready, -// 500 otherwise. It's used by kubernetes to indicate if the ingester -// pool is ready to have ingesters added / removed. -func IngesterReadinessHandler(i *ingester.Ingester) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - if i.Ready() { - w.WriteHeader(http.StatusNoContent) - } else { - w.WriteHeader(http.StatusInternalServerError) - } - }) -} - -// DistributorUserStatsHandler handles user stats to the Distributor. -func DistributorUserStatsHandler(statsFn func(context.Context) (*ingester.UserStats, error)) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - ctx, abort := parseRequest(w, r, nil) - if abort { - return - } - - stats, err := statsFn(ctx) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - writeJSONResponse(w, stats) - }) -} diff --git a/user/id.go b/user/id.go index ee4e23ba42..cce2c00f52 100644 --- a/user/id.go +++ b/user/id.go @@ -13,6 +13,12 @@ import ( // UserIDContextKey is the key used in contexts to find the userid const userIDContextKey = "CortexUserID" // TODO dedupe with storage/local +// UserIDHeaderName is a legacy from scope as a service. +const UserIDHeaderName = "X-Scope-OrgID" + +// LowerUserIDHeaderName as gRPC / HTTP2.0 headers are lowercased. +const LowerUserIDHeaderName = "x-scope-orgid" + // GetID returns the user func GetID(ctx context.Context) (string, error) { userid, ok := ctx.Value(userIDContextKey).(string) diff --git a/util/compat.go b/util/compat.go new file mode 100644 index 0000000000..1e47a6ef27 --- /dev/null +++ b/util/compat.go @@ -0,0 +1,164 @@ +package util + +import ( + "fmt" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/remote" + + "github.com/weaveworks/cortex" +) + +// FromWriteRequest converts a WriteRequest proto into an array of samples. +func FromWriteRequest(req *remote.WriteRequest) []*model.Sample { + var samples []*model.Sample + for _, ts := range req.Timeseries { + metric := model.Metric{} + for _, l := range ts.Labels { + metric[model.LabelName(l.Name)] = model.LabelValue(l.Value) + } + + for _, s := range ts.Samples { + samples = append(samples, &model.Sample{ + Metric: metric, + Value: model.SampleValue(s.Value), + Timestamp: model.Time(s.TimestampMs), + }) + } + } + return samples +} + +// ToWriteRequest converts an array of samples into a WriteRequest proto. +func ToWriteRequest(samples []*model.Sample) *remote.WriteRequest { + req := &remote.WriteRequest{ + Timeseries: make([]*remote.TimeSeries, 0, len(samples)), + } + + for _, s := range samples { + ts := &remote.TimeSeries{ + Labels: make([]*remote.LabelPair, 0, len(s.Metric)), + } + for k, v := range s.Metric { + ts.Labels = append(ts.Labels, + &remote.LabelPair{ + Name: string(k), + Value: string(v), + }) + } + ts.Samples = []*remote.Sample{ + { + Value: float64(s.Value), + TimestampMs: int64(s.Timestamp), + }, + } + req.Timeseries = append(req.Timeseries, ts) + } + + return req +} + +// ToQueryRequest builds a QueryRequest proto. +func ToQueryRequest(from, to model.Time, matchers ...*metric.LabelMatcher) (*cortex.QueryRequest, error) { + req := &cortex.QueryRequest{ + StartTimestampMs: int64(from), + EndTimestampMs: int64(to), + } + for _, matcher := range matchers { + var mType cortex.MatchType + switch matcher.Type { + case metric.Equal: + mType = cortex.MatchType_EQUAL + case metric.NotEqual: + mType = cortex.MatchType_NOT_EQUAL + case metric.RegexMatch: + mType = cortex.MatchType_REGEX_MATCH + case metric.RegexNoMatch: + mType = cortex.MatchType_REGEX_NO_MATCH + default: + return nil, fmt.Errorf("invalid matcher type") + } + req.Matchers = append(req.Matchers, &cortex.LabelMatcher{ + Type: mType, + Name: string(matcher.Name), + Value: string(matcher.Value), + }) + } + return req, nil +} + +// FromQueryRequest unpacks a QueryRequest proto. +func FromQueryRequest(req *cortex.QueryRequest) (model.Time, model.Time, []*metric.LabelMatcher, error) { + matchers := make(metric.LabelMatchers, 0, len(req.Matchers)) + for _, matcher := range req.Matchers { + var mtype metric.MatchType + switch matcher.Type { + case cortex.MatchType_EQUAL: + mtype = metric.Equal + case cortex.MatchType_NOT_EQUAL: + mtype = metric.NotEqual + case cortex.MatchType_REGEX_MATCH: + mtype = metric.RegexMatch + case cortex.MatchType_REGEX_NO_MATCH: + mtype = metric.RegexNoMatch + default: + return 0, 0, nil, fmt.Errorf("invalid matcher type") + } + matcher, err := metric.NewLabelMatcher(mtype, model.LabelName(matcher.Name), model.LabelValue(matcher.Value)) + if err != nil { + return 0, 0, nil, err + } + matchers = append(matchers, matcher) + } + from := model.Time(req.StartTimestampMs) + to := model.Time(req.EndTimestampMs) + return from, to, matchers, nil +} + +// ToQueryResponse builds a QueryResponse proto. +func ToQueryResponse(matrix model.Matrix) *cortex.QueryResponse { + resp := &cortex.QueryResponse{} + for _, ss := range matrix { + ts := &remote.TimeSeries{} + for k, v := range ss.Metric { + ts.Labels = append(ts.Labels, + &remote.LabelPair{ + Name: string(k), + Value: string(v), + }) + } + ts.Samples = make([]*remote.Sample, 0, len(ss.Values)) + for _, s := range ss.Values { + ts.Samples = append(ts.Samples, &remote.Sample{ + Value: float64(s.Value), + TimestampMs: int64(s.Timestamp), + }) + } + resp.Timeseries = append(resp.Timeseries, ts) + } + return resp +} + +// FromQueryResponse unpacks a QueryResponse proto. +func FromQueryResponse(resp *cortex.QueryResponse) model.Matrix { + m := make(model.Matrix, 0, len(resp.Timeseries)) + for _, ts := range resp.Timeseries { + var ss model.SampleStream + ss.Metric = model.Metric{} + for _, l := range ts.Labels { + ss.Metric[model.LabelName(l.Name)] = model.LabelValue(l.Value) + } + + ss.Values = make([]model.SamplePair, 0, len(ts.Samples)) + for _, s := range ts.Samples { + ss.Values = append(ss.Values, model.SamplePair{ + Value: model.SampleValue(s.Value), + Timestamp: model.Time(s.TimestampMs), + }) + } + m = append(m, &ss) + } + + return m +} diff --git a/util/compat_test.go b/util/compat_test.go new file mode 100644 index 0000000000..fd9225d0e1 --- /dev/null +++ b/util/compat_test.go @@ -0,0 +1,110 @@ +package util + +import ( + "fmt" + "reflect" + "sort" + "testing" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/storage/metric" +) + +func TestWriteRequest(t *testing.T) { + want := []*model.Sample{} + for i := 0; i < 10; i++ { + want = append(want, &model.Sample{ + Metric: model.Metric{ + model.MetricNameLabel: model.LabelValue(fmt.Sprintf("testmetric_%d", i)), + model.JobLabel: "testjob", + }, + Timestamp: model.Time(i), + Value: model.SampleValue(float64(i)), + }) + } + + have := FromWriteRequest(ToWriteRequest(want)) + + if !reflect.DeepEqual(have, want) { + t.Fatalf("Bad FromWriteRequest(ToWriteRequest) round trip") + } +} + +func TestQueryRequest(t *testing.T) { + from, to := model.Time(int64(0)), model.Time(int64(10)) + matchers := []*metric.LabelMatcher{} + matcher1, err := metric.NewLabelMatcher(metric.Equal, model.LabelName("foo"), model.LabelValue("1")) + if err != nil { + t.Fatal(err) + } + matchers = append(matchers, matcher1) + + matcher2, err := metric.NewLabelMatcher(metric.NotEqual, model.LabelName("bar"), model.LabelValue("2")) + if err != nil { + t.Fatal(err) + } + matchers = append(matchers, matcher2) + + matcher3, err := metric.NewLabelMatcher(metric.RegexMatch, model.LabelName("baz"), model.LabelValue("3")) + if err != nil { + t.Fatal(err) + } + matchers = append(matchers, matcher3) + + matcher4, err := metric.NewLabelMatcher(metric.RegexNoMatch, model.LabelName("bop"), model.LabelValue("4")) + if err != nil { + t.Fatal(err) + } + matchers = append(matchers, matcher4) + + req, err := ToQueryRequest(from, to, matchers...) + if err != nil { + t.Fatal(err) + } + + haveFrom, haveTo, haveMatchers, err := FromQueryRequest(req) + if err != nil { + t.Fatal(err) + } + + if !reflect.DeepEqual(haveFrom, from) { + t.Fatalf("Bad from FromQueryRequest(ToQueryRequest) round trip") + } + if !reflect.DeepEqual(haveTo, to) { + t.Fatalf("Bad to FromQueryRequest(ToQueryRequest) round trip") + } + if !reflect.DeepEqual(haveMatchers, matchers) { + t.Fatalf("Bad have FromQueryRequest(ToQueryRequest) round trip - %v != %v", haveMatchers, matchers) + } +} + +func buildTestMatrix(numSeries int, samplesPerSeries int, offset int) model.Matrix { + m := make(model.Matrix, 0, numSeries) + for i := 0; i < numSeries; i++ { + ss := model.SampleStream{ + Metric: model.Metric{ + model.MetricNameLabel: model.LabelValue(fmt.Sprintf("testmetric_%d", i)), + model.JobLabel: "testjob", + }, + Values: make([]model.SamplePair, 0, samplesPerSeries), + } + for j := 0; j < samplesPerSeries; j++ { + ss.Values = append(ss.Values, model.SamplePair{ + Timestamp: model.Time(i + j + offset), + Value: model.SampleValue(i + j + offset), + }) + } + m = append(m, &ss) + } + sort.Sort(m) + return m +} + +func TestQueryResponse(t *testing.T) { + want := buildTestMatrix(10, 10, 10) + have := FromQueryResponse(ToQueryResponse(want)) + if !reflect.DeepEqual(have, want) { + t.Fatalf("Bad FromQueryResponse(ToQueryResponse) round trip") + } + +} diff --git a/util/http.go b/util/http.go new file mode 100644 index 0000000000..861112e44b --- /dev/null +++ b/util/http.go @@ -0,0 +1,77 @@ +package util + +import ( + "bytes" + "encoding/json" + "io" + "net/http" + + "github.com/golang/protobuf/proto" + "github.com/golang/snappy" + "github.com/prometheus/common/log" + "golang.org/x/net/context" + + "github.com/weaveworks/cortex/user" +) + +// ParseProtoRequest parses a proto from the body of a http request. +func ParseProtoRequest(w http.ResponseWriter, r *http.Request, req proto.Message, compressed bool) (ctx context.Context, abort bool) { + userID := r.Header.Get(user.UserIDHeaderName) + if userID == "" { + http.Error(w, "", http.StatusUnauthorized) + return nil, true + } + + ctx = user.WithID(context.Background(), userID) + if req == nil { + return ctx, false + } + + var reader io.Reader = r.Body + if compressed { + reader = snappy.NewReader(r.Body) + } + + buf := bytes.Buffer{} + if _, err := buf.ReadFrom(reader); err != nil { + log.Errorf(err.Error()) + http.Error(w, err.Error(), http.StatusBadRequest) + return nil, true + } + + if err := proto.Unmarshal(buf.Bytes(), req); err != nil { + log.Errorf(err.Error()) + http.Error(w, err.Error(), http.StatusBadRequest) + return nil, true + } + + return ctx, false +} + +// WriteJSONResponse writes some JSON as a HTTP response. +func WriteJSONResponse(w http.ResponseWriter, v interface{}) { + data, err := json.Marshal(v) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + if _, err = w.Write(data); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + w.Header().Set("Content-Type", "application/json") +} + +// WriteProtoResponse writes a proto as a HTTP response. +func WriteProtoResponse(w http.ResponseWriter, resp proto.Message) { + data, err := proto.Marshal(resp) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + if _, err = w.Write(data); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + // TODO: set Content-type. +} diff --git a/util/middleware/auth.go b/util/middleware/auth.go new file mode 100644 index 0000000000..e787bc6404 --- /dev/null +++ b/util/middleware/auth.go @@ -0,0 +1,38 @@ +package middleware + +import ( + "fmt" + + "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/metadata" + + "github.com/weaveworks/cortex/user" +) + +// ClientUserHeaderInterceptor propagates the user ID from the context to gRPC metadata, which eventually ends up as a HTTP2 header. +func ClientUserHeaderInterceptor(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + userID, err := user.GetID(ctx) + if err != nil { + return err + } + + newCtx := metadata.NewContext(ctx, metadata.New(map[string]string{user.UserIDHeaderName: userID})) + return invoker(newCtx, method, req, reply, cc, opts...) +} + +// ServerUserHeaderInterceptor propagates the user ID from the gRPC metadata back to our context. +func ServerUserHeaderInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + md, ok := metadata.FromContext(ctx) + if !ok { + return nil, fmt.Errorf("no metadata") + } + + userIDs, ok := md[user.LowerUserIDHeaderName] + if !ok || len(userIDs) != 1 { + return nil, fmt.Errorf("no user id") + } + + newCtx := user.WithID(ctx, userIDs[0]) + return handler(newCtx, req) +} diff --git a/util/middleware/instrumentation.go b/util/middleware/instrumentation.go new file mode 100644 index 0000000000..3d002be894 --- /dev/null +++ b/util/middleware/instrumentation.go @@ -0,0 +1,23 @@ +package middleware + +import ( + "time" + + "github.com/prometheus/client_golang/prometheus" + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// ServerInstrumentInterceptor instruments gRPC requests for errors and latency. +func ServerInstrumentInterceptor(duration *prometheus.HistogramVec) grpc.UnaryServerInterceptor { + return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + begin := time.Now() + resp, err := handler(ctx, req) + status := "success" + if err != nil { + status = "error" + } + duration.WithLabelValues(gRPC, info.FullMethod, status, "false").Observe(time.Since(begin).Seconds()) + return resp, err + } +} diff --git a/util/middleware/logging.go b/util/middleware/logging.go new file mode 100644 index 0000000000..986bf4de9d --- /dev/null +++ b/util/middleware/logging.go @@ -0,0 +1,25 @@ +package middleware + +import ( + "time" + + "github.com/prometheus/common/log" + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +const gRPC = "gRPC" + +// ServerLoggingInterceptor logs gRPC requests, errors and latency. +func ServerLoggingInterceptor(logSuccess bool) grpc.UnaryServerInterceptor { + return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + begin := time.Now() + resp, err := handler(ctx, req) + if err != nil { + log.Errorf("%s %s (%v) %s", gRPC, info.FullMethod, err, time.Since(begin)) + } else if logSuccess { + log.Infof("%s %s (success) %s", gRPC, info.FullMethod, time.Since(begin)) + } + return resp, err + } +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/LICENSE b/vendor/github.com/mwitkow/go-grpc-middleware/LICENSE new file mode 100644 index 0000000000..b2b065037f --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. \ No newline at end of file diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/chain.go b/vendor/github.com/mwitkow/go-grpc-middleware/chain.go new file mode 100644 index 0000000000..6c1dab6563 --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/chain.go @@ -0,0 +1,99 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +// gRPC Server Interceptor chaining middleware. + +package grpc_middleware + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// ChainUnaryServer creates a single interceptor out of a chain of many interceptors. +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryServer(one, two, three) will execute one before two before three, and three +// will see context changes of one and two. +func ChainUnaryServer(interceptors ...grpc.UnaryServerInterceptor) grpc.UnaryServerInterceptor { + return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + buildChain := func(current grpc.UnaryServerInterceptor, next grpc.UnaryHandler) grpc.UnaryHandler { + return func(currentCtx context.Context, currentReq interface{}) (interface{}, error) { + return current(currentCtx, currentReq, info, next) + } + } + chain := handler + for i := len(interceptors) - 1; i >= 0; i-- { + chain = buildChain(interceptors[i], chain) + } + return chain(ctx, req) + } +} + +// ChainStreamServer creates a single interceptor out of a chain of many interceptors. +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryServer(one, two, three) will execute one before two before three. +// If you want to pass context between interceptors, use WrapServerStream. +func ChainStreamServer(interceptors ...grpc.StreamServerInterceptor) grpc.StreamServerInterceptor { + return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { + buildChain := func(current grpc.StreamServerInterceptor, next grpc.StreamHandler) grpc.StreamHandler { + return func(currentSrv interface{}, currentStream grpc.ServerStream) error { + return current(currentSrv, currentStream, info, next) + } + } + chain := handler + for i := len(interceptors) - 1; i >= 0; i-- { + chain = buildChain(interceptors[i], chain) + } + return chain(srv, stream) + } +} + +// ChainUnaryClient creates a single interceptor out of a chain of many interceptors. +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryClient(one, two, three) will execute one before two before three. +func ChainUnaryClient(interceptors ...grpc.UnaryClientInterceptor) grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + buildChain := func(current grpc.UnaryClientInterceptor, next grpc.UnaryInvoker) grpc.UnaryInvoker { + return func(currentCtx context.Context, currentMethod string, currentReq, currentRepl interface{}, currentConn *grpc.ClientConn, currentOpts ...grpc.CallOption) error { + return current(currentCtx, currentMethod, currentReq, currentRepl, currentConn, next, currentOpts...) + } + } + chain := invoker + for i := len(interceptors) - 1; i >= 0; i-- { + chain = buildChain(interceptors[i], chain) + } + return chain(ctx, method, req, reply, cc, opts...) + } +} + +// ChainStreamClient creates a single interceptor out of a chain of many interceptors. +// Execution is done in left-to-right order, including passing of context. +// For example ChainStreamClient(one, two, three) will execute one before two before three. +func ChainStreamClient(interceptors ...grpc.StreamClientInterceptor) grpc.StreamClientInterceptor { + return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) { + buildChain := func(current grpc.StreamClientInterceptor, next grpc.Streamer) grpc.Streamer { + return func(currentCtx context.Context, currentDesc *grpc.StreamDesc, currentConn *grpc.ClientConn, currentMethod string, currentOpts ...grpc.CallOption) (grpc.ClientStream, error) { + return current(currentCtx, currentDesc, currentConn, currentMethod, next, currentOpts...) + } + } + chain := streamer + for i := len(interceptors) - 1; i >= 0; i-- { + chain = buildChain(interceptors[i], chain) + } + return chain(ctx, desc, cc, method, opts...) + } +} + +// Chain creates a single interceptor out of a chain of many interceptors. + +// WithUnaryServerChain is a grpc.Server config option that accepts multiple unary interceptors. +// Basically syntactic sugar. +func WithUnaryServerChain(interceptors ...grpc.UnaryServerInterceptor) grpc.ServerOption { + return grpc.UnaryInterceptor(ChainUnaryServer(interceptors...)) +} + +// WithStreamServerChain is a grpc.Server config option that accepts multiple stream interceptors. +// Basically syntactic sugar. +func WithStreamServerChain(interceptors ...grpc.StreamServerInterceptor) grpc.ServerOption { + return grpc.StreamInterceptor(ChainStreamServer(interceptors...)) +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/testing/interceptor_suite.go b/vendor/github.com/mwitkow/go-grpc-middleware/testing/interceptor_suite.go new file mode 100644 index 0000000000..504527ad11 --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/testing/interceptor_suite.go @@ -0,0 +1,69 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_testing + +import ( + "net" + "time" + + pb_testproto "github.com/mwitkow/go-grpc-middleware/testing/testproto" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// InterceptorTestSuite is a testify/Suite that starts a gRPC PingService server and a client. +type InterceptorTestSuite struct { + suite.Suite + + ServerOpts []grpc.ServerOption + ClientOpts []grpc.DialOption + + ServerListener net.Listener + Server *grpc.Server + clientConn *grpc.ClientConn + Client pb_testproto.TestServiceClient + ctx context.Context +} + +func (s *InterceptorTestSuite) SetupSuite() { + var err error + s.ServerListener, err = net.Listen("tcp", "127.0.0.1:0") + require.NoError(s.T(), err, "must be able to allocate a port for serverListener") + + // This is the point where we hook up the interceptor + s.Server = grpc.NewServer(s.ServerOpts...) + pb_testproto.RegisterTestServiceServer(s.Server, &TestPingService{T: s.T()}) + + go func() { + s.Server.Serve(s.ServerListener) + }() + clientOpts := append(s.ClientOpts, grpc.WithInsecure(), grpc.WithBlock(), grpc.WithTimeout(2*time.Second)) + s.clientConn, err = grpc.Dial(s.ServerAddr(), clientOpts...) + require.NoError(s.T(), err, "must not error on client Dial") + s.Client = pb_testproto.NewTestServiceClient(s.clientConn) + +} + +func (s *InterceptorTestSuite) ServerAddr() string { + return s.ServerListener.Addr().String() +} + +func (s *InterceptorTestSuite) SimpleCtx() context.Context { + ctx, _ := context.WithTimeout(context.TODO(), 2 * time.Second) + return ctx +} + +func (s *InterceptorTestSuite) TearDownSuite() { + if s.ServerListener != nil { + s.Server.Stop() + s.T().Logf("stopped grpc.Server at: %v", s.ServerAddr()) + s.ServerListener.Close() + + } + if s.clientConn != nil { + s.clientConn.Close() + } +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/testing/pingservice.go b/vendor/github.com/mwitkow/go-grpc-middleware/testing/pingservice.go new file mode 100644 index 0000000000..c52994c8ab --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/testing/pingservice.go @@ -0,0 +1,71 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +Package `grpc_testing` provides helper functions for testing validators in this package. +*/ + +package grpc_testing + +import ( + "testing" + "io" + + pb_testproto "github.com/mwitkow/go-grpc-middleware/testing/testproto" + + "golang.org/x/net/context" + "google.golang.org/grpc/codes" + "google.golang.org/grpc" +) + +const ( + // DefaultPongValue is the default value used. + DefaultResponseValue = "default_response_value" + // ListResponseCount is the expeted number of responses to PingList + ListResponseCount = 100 +) + +type TestPingService struct { + T *testing.T +} + +func (s *TestPingService) PingEmpty(ctx context.Context, _ *pb_testproto.Empty) (*pb_testproto.PingResponse, error) { + return &pb_testproto.PingResponse{Value: DefaultResponseValue, Counter: 42}, nil +} + +func (s *TestPingService) Ping(ctx context.Context, ping *pb_testproto.PingRequest) (*pb_testproto.PingResponse, error) { + // Send user trailers and headers. + return &pb_testproto.PingResponse{Value: ping.Value, Counter: 42}, nil +} + +func (s *TestPingService) PingError(ctx context.Context, ping *pb_testproto.PingRequest) (*pb_testproto.Empty, error) { + code := codes.Code(ping.ErrorCodeReturned) + return nil, grpc.Errorf(code, "Userspace error.") +} + +func (s *TestPingService) PingList(ping *pb_testproto.PingRequest, stream pb_testproto.TestService_PingListServer) error { + if ping.ErrorCodeReturned != 0 { + return grpc.Errorf(codes.Code(ping.ErrorCodeReturned), "foobar") + } + // Send user trailers and headers. + for i := 0; i < ListResponseCount; i++ { + stream.Send(&pb_testproto.PingResponse{Value: ping.Value, Counter: int32(i)}) + } + return nil +} + +func (s *TestPingService) PingStream(stream pb_testproto.TestService_PingStreamServer) error { + count := 0 + for true { + ping, err := stream.Recv() + if err == io.EOF { + break + } + if err != nil { + return err + } + stream.Send(&pb_testproto.PingResponse{Value: ping.Value, Counter: int32(count)}) + count += 1 + } + return nil +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.manual_validator.pb.go b/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.manual_validator.pb.go new file mode 100644 index 0000000000..e35ef37393 --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.manual_validator.pb.go @@ -0,0 +1,12 @@ +// Manual code for validation tests. + +package mwitkow_testproto + +import "errors" + +func (p *PingRequest) Validate() error { + if p.SleepTimeMs > 10000 { + return errors.New("cannot sleep for more than 10s") + } + return nil +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.pb.go b/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.pb.go new file mode 100644 index 0000000000..90dd985994 --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/testing/testproto/test.pb.go @@ -0,0 +1,359 @@ +// Code generated by protoc-gen-go. +// source: test.proto +// DO NOT EDIT! + +/* +Package mwitkow_testproto is a generated protocol buffer package. + +It is generated from these files: + test.proto + +It has these top-level messages: + Empty + PingRequest + PingResponse +*/ +package mwitkow_testproto + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" + +import ( + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +type Empty struct { +} + +func (m *Empty) Reset() { *m = Empty{} } +func (m *Empty) String() string { return proto.CompactTextString(m) } +func (*Empty) ProtoMessage() {} +func (*Empty) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +type PingRequest struct { + Value string `protobuf:"bytes,1,opt,name=value" json:"value,omitempty"` + SleepTimeMs int32 `protobuf:"varint,2,opt,name=sleep_time_ms,json=sleepTimeMs" json:"sleep_time_ms,omitempty"` + ErrorCodeReturned uint32 `protobuf:"varint,3,opt,name=error_code_returned,json=errorCodeReturned" json:"error_code_returned,omitempty"` +} + +func (m *PingRequest) Reset() { *m = PingRequest{} } +func (m *PingRequest) String() string { return proto.CompactTextString(m) } +func (*PingRequest) ProtoMessage() {} +func (*PingRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +type PingResponse struct { + Value string `protobuf:"bytes,1,opt,name=Value,json=value" json:"Value,omitempty"` + Counter int32 `protobuf:"varint,2,opt,name=counter" json:"counter,omitempty"` +} + +func (m *PingResponse) Reset() { *m = PingResponse{} } +func (m *PingResponse) String() string { return proto.CompactTextString(m) } +func (*PingResponse) ProtoMessage() {} +func (*PingResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +func init() { + proto.RegisterType((*Empty)(nil), "mwitkow.testproto.Empty") + proto.RegisterType((*PingRequest)(nil), "mwitkow.testproto.PingRequest") + proto.RegisterType((*PingResponse)(nil), "mwitkow.testproto.PingResponse") +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion3 + +// Client API for TestService service + +type TestServiceClient interface { + PingEmpty(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*PingResponse, error) + Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) + PingError(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*Empty, error) + PingList(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (TestService_PingListClient, error) + PingStream(ctx context.Context, opts ...grpc.CallOption) (TestService_PingStreamClient, error) +} + +type testServiceClient struct { + cc *grpc.ClientConn +} + +func NewTestServiceClient(cc *grpc.ClientConn) TestServiceClient { + return &testServiceClient{cc} +} + +func (c *testServiceClient) PingEmpty(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*PingResponse, error) { + out := new(PingResponse) + err := grpc.Invoke(ctx, "/mwitkow.testproto.TestService/PingEmpty", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *testServiceClient) Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) { + out := new(PingResponse) + err := grpc.Invoke(ctx, "/mwitkow.testproto.TestService/Ping", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *testServiceClient) PingError(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*Empty, error) { + out := new(Empty) + err := grpc.Invoke(ctx, "/mwitkow.testproto.TestService/PingError", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *testServiceClient) PingList(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (TestService_PingListClient, error) { + stream, err := grpc.NewClientStream(ctx, &_TestService_serviceDesc.Streams[0], c.cc, "/mwitkow.testproto.TestService/PingList", opts...) + if err != nil { + return nil, err + } + x := &testServicePingListClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type TestService_PingListClient interface { + Recv() (*PingResponse, error) + grpc.ClientStream +} + +type testServicePingListClient struct { + grpc.ClientStream +} + +func (x *testServicePingListClient) Recv() (*PingResponse, error) { + m := new(PingResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *testServiceClient) PingStream(ctx context.Context, opts ...grpc.CallOption) (TestService_PingStreamClient, error) { + stream, err := grpc.NewClientStream(ctx, &_TestService_serviceDesc.Streams[1], c.cc, "/mwitkow.testproto.TestService/PingStream", opts...) + if err != nil { + return nil, err + } + x := &testServicePingStreamClient{stream} + return x, nil +} + +type TestService_PingStreamClient interface { + Send(*PingRequest) error + Recv() (*PingResponse, error) + grpc.ClientStream +} + +type testServicePingStreamClient struct { + grpc.ClientStream +} + +func (x *testServicePingStreamClient) Send(m *PingRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *testServicePingStreamClient) Recv() (*PingResponse, error) { + m := new(PingResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// Server API for TestService service + +type TestServiceServer interface { + PingEmpty(context.Context, *Empty) (*PingResponse, error) + Ping(context.Context, *PingRequest) (*PingResponse, error) + PingError(context.Context, *PingRequest) (*Empty, error) + PingList(*PingRequest, TestService_PingListServer) error + PingStream(TestService_PingStreamServer) error +} + +func RegisterTestServiceServer(s *grpc.Server, srv TestServiceServer) { + s.RegisterService(&_TestService_serviceDesc, srv) +} + +func _TestService_PingEmpty_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(Empty) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TestServiceServer).PingEmpty(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/mwitkow.testproto.TestService/PingEmpty", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TestServiceServer).PingEmpty(ctx, req.(*Empty)) + } + return interceptor(ctx, in, info, handler) +} + +func _TestService_Ping_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PingRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TestServiceServer).Ping(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/mwitkow.testproto.TestService/Ping", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TestServiceServer).Ping(ctx, req.(*PingRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TestService_PingError_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PingRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TestServiceServer).PingError(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/mwitkow.testproto.TestService/PingError", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TestServiceServer).PingError(ctx, req.(*PingRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TestService_PingList_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(PingRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(TestServiceServer).PingList(m, &testServicePingListServer{stream}) +} + +type TestService_PingListServer interface { + Send(*PingResponse) error + grpc.ServerStream +} + +type testServicePingListServer struct { + grpc.ServerStream +} + +func (x *testServicePingListServer) Send(m *PingResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _TestService_PingStream_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(TestServiceServer).PingStream(&testServicePingStreamServer{stream}) +} + +type TestService_PingStreamServer interface { + Send(*PingResponse) error + Recv() (*PingRequest, error) + grpc.ServerStream +} + +type testServicePingStreamServer struct { + grpc.ServerStream +} + +func (x *testServicePingStreamServer) Send(m *PingResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *testServicePingStreamServer) Recv() (*PingRequest, error) { + m := new(PingRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _TestService_serviceDesc = grpc.ServiceDesc{ + ServiceName: "mwitkow.testproto.TestService", + HandlerType: (*TestServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "PingEmpty", + Handler: _TestService_PingEmpty_Handler, + }, + { + MethodName: "Ping", + Handler: _TestService_Ping_Handler, + }, + { + MethodName: "PingError", + Handler: _TestService_PingError_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "PingList", + Handler: _TestService_PingList_Handler, + ServerStreams: true, + }, + { + StreamName: "PingStream", + Handler: _TestService_PingStream_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: fileDescriptor0, +} + +func init() { proto.RegisterFile("test.proto", fileDescriptor0) } + +var fileDescriptor0 = []byte{ + // 287 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xac, 0x91, 0x41, 0x4b, 0x3b, 0x31, + 0x10, 0xc5, 0xff, 0xfb, 0xaf, 0x6b, 0xed, 0xac, 0x3d, 0x34, 0x7a, 0x08, 0x1e, 0x54, 0x72, 0xea, + 0x29, 0x14, 0xbd, 0x7b, 0x11, 0x51, 0x41, 0x51, 0xb2, 0xc5, 0xeb, 0x52, 0xb7, 0x83, 0x04, 0x9b, + 0xcd, 0x9a, 0x64, 0xbb, 0xf8, 0x31, 0xfc, 0xc6, 0x26, 0xe9, 0x0a, 0x42, 0x2d, 0x0a, 0x7a, 0x9c, + 0xf7, 0x1b, 0xde, 0x7b, 0x93, 0x00, 0x38, 0xb4, 0x8e, 0xd7, 0x46, 0x3b, 0x4d, 0x46, 0xaa, 0x95, + 0xee, 0x59, 0xb7, 0x3c, 0x68, 0x51, 0x62, 0x7d, 0x48, 0x2f, 0x54, 0xed, 0x5e, 0x59, 0x0b, 0xd9, + 0xbd, 0xac, 0x9e, 0x04, 0xbe, 0x34, 0x1e, 0x92, 0x7d, 0x48, 0x97, 0xb3, 0x45, 0x83, 0x34, 0x39, + 0x4e, 0xc6, 0x03, 0xb1, 0x1a, 0x08, 0x83, 0xa1, 0x5d, 0x20, 0xd6, 0x85, 0x93, 0x0a, 0x0b, 0x65, + 0xe9, 0x7f, 0x4f, 0x53, 0x91, 0x45, 0x71, 0xea, 0xb5, 0x5b, 0x4b, 0x38, 0xec, 0xa1, 0x31, 0xda, + 0x14, 0xa5, 0x9e, 0x63, 0x61, 0xd0, 0x35, 0xa6, 0xc2, 0x39, 0xed, 0xf9, 0xcd, 0xa1, 0x18, 0x45, + 0x74, 0xee, 0x89, 0xe8, 0x00, 0x3b, 0x83, 0xdd, 0x55, 0xb0, 0xad, 0x75, 0x65, 0x31, 0x24, 0x3f, + 0xac, 0x27, 0x53, 0xe8, 0x97, 0xba, 0xa9, 0x1c, 0x9a, 0x2e, 0xf3, 0x63, 0x3c, 0x79, 0xeb, 0x41, + 0x36, 0xf5, 0x95, 0x73, 0x34, 0x4b, 0x59, 0x22, 0xb9, 0x82, 0x41, 0xf0, 0x8b, 0x57, 0x11, 0xca, + 0xd7, 0x4e, 0xe6, 0x91, 0x1c, 0x1c, 0x7d, 0x41, 0x3e, 0xf7, 0x60, 0xff, 0xc8, 0x35, 0x6c, 0x05, + 0x85, 0x1c, 0x6e, 0x5c, 0x8d, 0x6f, 0xf5, 0x13, 0xab, 0xcb, 0xae, 0x54, 0xb8, 0xfe, 0x5b, 0xbf, + 0x8d, 0xa5, 0xbd, 0xd1, 0x1d, 0xec, 0x84, 0xd5, 0x1b, 0xe9, 0xff, 0xe8, 0xf7, 0xbd, 0x26, 0x09, + 0xc9, 0x01, 0x82, 0x96, 0x3b, 0x83, 0x33, 0xf5, 0x07, 0x96, 0xe3, 0x64, 0x92, 0x3c, 0x6e, 0x47, + 0x72, 0xfa, 0x1e, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x2a, 0x8a, 0x7b, 0x7d, 0x02, 0x00, 0x00, +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/validator/validator.go b/vendor/github.com/mwitkow/go-grpc-middleware/validator/validator.go new file mode 100644 index 0000000000..a001706e5b --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/validator/validator.go @@ -0,0 +1,62 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +Package `grpc_validator` provides an easy way to hook protobuf message validation as a gRPC +interceptor across all your APIs. + +It primarily meant to be used with https://github.com/mwitkow/go-proto-validators, which code-gen +assertions about allowed values from `.proto` files. + +Basically this will invoke a .Validate() method on incoming message of the stream, if such method is +defined. If that method returns an error, an `INVALID_ARGUMENT` gRPC status code is returned. +*/ + +package grpc_validator + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" +) + +type validator interface { + Validate() error +} + +// UnaryServerInterceptor returns a new unary server interceptors that validates incoming messages. +func UnaryServerInterceptor() grpc.UnaryServerInterceptor { + return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + if v, ok := req.(validator); ok { + if err := v.Validate(); err != nil { + return nil, grpc.Errorf(codes.InvalidArgument, err.Error()) + } + } + return handler(ctx, req) + } +} + +// StreamServerInterceptor returns a new streaming server interceptors that validates incoming messages. +// The validation happens on message receives. +func StreamServerInterceptor() grpc.StreamServerInterceptor { + return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { + wrapper := &recvWrapper{stream} + return handler(srv, wrapper) + } +} + +type recvWrapper struct { + grpc.ServerStream +} + +func (s *recvWrapper) RecvMsg(m interface{}) error { + if err := s.ServerStream.RecvMsg(m); err != nil { + return err + } + if v, ok := m.(validator); ok { + if err := v.Validate(); err != nil { + return grpc.Errorf(codes.InvalidArgument, err.Error()) + } + } + return nil +} diff --git a/vendor/github.com/mwitkow/go-grpc-middleware/wrappers.go b/vendor/github.com/mwitkow/go-grpc-middleware/wrappers.go new file mode 100644 index 0000000000..597b862445 --- /dev/null +++ b/vendor/github.com/mwitkow/go-grpc-middleware/wrappers.go @@ -0,0 +1,29 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_middleware + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// WrappedServerStream is a thin wrapper around grpc.ServerStream that allows modifying context. +type WrappedServerStream struct { + grpc.ServerStream + // WrappedContext is the wrapper's own Context. You can assign it. + WrappedContext context.Context +} + +// Context returns the wrapper's WrappedContext, overwriting the nested grpc.ServerStream.Context() +func (w *WrappedServerStream) Context() context.Context { + return w.WrappedContext +} + +// WrapServerStream returns a ServerStream that has the ability to overwrite context. +func WrapServerStream(stream grpc.ServerStream) *WrappedServerStream { + if existing, ok := stream.(*WrappedServerStream); ok { + return existing + } + return &WrappedServerStream{ServerStream: stream, WrappedContext: stream.Context()} +} diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/remote.proto b/vendor/github.com/prometheus/prometheus/storage/remote/remote.proto new file mode 100644 index 0000000000..88017c5168 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/storage/remote/remote.proto @@ -0,0 +1,36 @@ +// Copyright 2016 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. + +syntax = "proto3"; + +package remote; + +message Sample { + double value = 1; + int64 timestamp_ms = 2; +} + +message LabelPair { + string name = 1; + string value = 2; +} + +message TimeSeries { + repeated LabelPair labels = 1; + // Sorted by time, oldest sample first. + repeated Sample samples = 2; +} + +message WriteRequest { + repeated TimeSeries timeseries = 1; +} diff --git a/vendor/github.com/stretchr/testify/suite/LICENCE.txt b/vendor/github.com/stretchr/testify/suite/LICENCE.txt new file mode 100644 index 0000000000..473b670a7c --- /dev/null +++ b/vendor/github.com/stretchr/testify/suite/LICENCE.txt @@ -0,0 +1,22 @@ +Copyright (c) 2012 - 2013 Mat Ryer and Tyler Bunnell + +Please consider promoting this project if you find it useful. + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without restriction, +including without limitation the rights to use, copy, modify, merge, +publish, distribute, sublicense, and/or sell copies of the Software, +and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be included +in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT +OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE +OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/stretchr/testify/suite/LICENSE b/vendor/github.com/stretchr/testify/suite/LICENSE new file mode 100644 index 0000000000..473b670a7c --- /dev/null +++ b/vendor/github.com/stretchr/testify/suite/LICENSE @@ -0,0 +1,22 @@ +Copyright (c) 2012 - 2013 Mat Ryer and Tyler Bunnell + +Please consider promoting this project if you find it useful. + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without restriction, +including without limitation the rights to use, copy, modify, merge, +publish, distribute, sublicense, and/or sell copies of the Software, +and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be included +in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT +OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE +OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/stretchr/testify/suite/doc.go b/vendor/github.com/stretchr/testify/suite/doc.go new file mode 100644 index 0000000000..f91a245d3f --- /dev/null +++ b/vendor/github.com/stretchr/testify/suite/doc.go @@ -0,0 +1,65 @@ +// Package suite contains logic for creating testing suite structs +// and running the methods on those structs as tests. The most useful +// piece of this package is that you can create setup/teardown methods +// on your testing suites, which will run before/after the whole suite +// or individual tests (depending on which interface(s) you +// implement). +// +// A testing suite is usually built by first extending the built-in +// suite functionality from suite.Suite in testify. Alternatively, +// you could reproduce that logic on your own if you wanted (you +// just need to implement the TestingSuite interface from +// suite/interfaces.go). +// +// After that, you can implement any of the interfaces in +// suite/interfaces.go to add setup/teardown functionality to your +// suite, and add any methods that start with "Test" to add tests. +// Methods that do not match any suite interfaces and do not begin +// with "Test" will not be run by testify, and can safely be used as +// helper methods. +// +// Once you've built your testing suite, you need to run the suite +// (using suite.Run from testify) inside any function that matches the +// identity that "go test" is already looking for (i.e. +// func(*testing.T)). +// +// Regular expression to select test suites specified command-line +// argument "-run". Regular expression to select the methods +// of test suites specified command-line argument "-m". +// Suite object has assertion methods. +// +// A crude example: +// // Basic imports +// import ( +// "testing" +// "github.com/stretchr/testify/assert" +// "github.com/stretchr/testify/suite" +// ) +// +// // Define the suite, and absorb the built-in basic suite +// // functionality from testify - including a T() method which +// // returns the current testing context +// type ExampleTestSuite struct { +// suite.Suite +// VariableThatShouldStartAtFive int +// } +// +// // Make sure that VariableThatShouldStartAtFive is set to five +// // before each test +// func (suite *ExampleTestSuite) SetupTest() { +// suite.VariableThatShouldStartAtFive = 5 +// } +// +// // All methods that begin with "Test" are run as tests within a +// // suite. +// func (suite *ExampleTestSuite) TestExample() { +// assert.Equal(suite.T(), 5, suite.VariableThatShouldStartAtFive) +// suite.Equal(5, suite.VariableThatShouldStartAtFive) +// } +// +// // In order for 'go test' to run this suite, we need to create +// // a normal test function and pass our suite to suite.Run +// func TestExampleTestSuite(t *testing.T) { +// suite.Run(t, new(ExampleTestSuite)) +// } +package suite diff --git a/vendor/github.com/stretchr/testify/suite/interfaces.go b/vendor/github.com/stretchr/testify/suite/interfaces.go new file mode 100644 index 0000000000..20969472c7 --- /dev/null +++ b/vendor/github.com/stretchr/testify/suite/interfaces.go @@ -0,0 +1,34 @@ +package suite + +import "testing" + +// TestingSuite can store and return the current *testing.T context +// generated by 'go test'. +type TestingSuite interface { + T() *testing.T + SetT(*testing.T) +} + +// SetupAllSuite has a SetupSuite method, which will run before the +// tests in the suite are run. +type SetupAllSuite interface { + SetupSuite() +} + +// SetupTestSuite has a SetupTest method, which will run before each +// test in the suite. +type SetupTestSuite interface { + SetupTest() +} + +// TearDownAllSuite has a TearDownSuite method, which will run after +// all the tests in the suite have been run. +type TearDownAllSuite interface { + TearDownSuite() +} + +// TearDownTestSuite has a TearDownTest method, which will run after +// each test in the suite. +type TearDownTestSuite interface { + TearDownTest() +} diff --git a/vendor/github.com/stretchr/testify/suite/suite.go b/vendor/github.com/stretchr/testify/suite/suite.go new file mode 100644 index 0000000000..db74130009 --- /dev/null +++ b/vendor/github.com/stretchr/testify/suite/suite.go @@ -0,0 +1,115 @@ +package suite + +import ( + "flag" + "fmt" + "os" + "reflect" + "regexp" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var matchMethod = flag.String("testify.m", "", "regular expression to select tests of the testify suite to run") + +// Suite is a basic testing suite with methods for storing and +// retrieving the current *testing.T context. +type Suite struct { + *assert.Assertions + require *require.Assertions + t *testing.T +} + +// T retrieves the current *testing.T context. +func (suite *Suite) T() *testing.T { + return suite.t +} + +// SetT sets the current *testing.T context. +func (suite *Suite) SetT(t *testing.T) { + suite.t = t + suite.Assertions = assert.New(t) + suite.require = require.New(t) +} + +// Require returns a require context for suite. +func (suite *Suite) Require() *require.Assertions { + if suite.require == nil { + suite.require = require.New(suite.T()) + } + return suite.require +} + +// Assert returns an assert context for suite. Normally, you can call +// `suite.NoError(expected, actual)`, but for situations where the embedded +// methods are overridden (for example, you might want to override +// assert.Assertions with require.Assertions), this method is provided so you +// can call `suite.Assert().NoError()`. +func (suite *Suite) Assert() *assert.Assertions { + if suite.Assertions == nil { + suite.Assertions = assert.New(suite.T()) + } + return suite.Assertions +} + +// Run takes a testing suite and runs all of the tests attached +// to it. +func Run(t *testing.T, suite TestingSuite) { + suite.SetT(t) + + if setupAllSuite, ok := suite.(SetupAllSuite); ok { + setupAllSuite.SetupSuite() + } + defer func() { + if tearDownAllSuite, ok := suite.(TearDownAllSuite); ok { + tearDownAllSuite.TearDownSuite() + } + }() + + methodFinder := reflect.TypeOf(suite) + tests := []testing.InternalTest{} + for index := 0; index < methodFinder.NumMethod(); index++ { + method := methodFinder.Method(index) + ok, err := methodFilter(method.Name) + if err != nil { + fmt.Fprintf(os.Stderr, "testify: invalid regexp for -m: %s\n", err) + os.Exit(1) + } + if ok { + test := testing.InternalTest{ + Name: method.Name, + F: func(t *testing.T) { + parentT := suite.T() + suite.SetT(t) + if setupTestSuite, ok := suite.(SetupTestSuite); ok { + setupTestSuite.SetupTest() + } + defer func() { + if tearDownTestSuite, ok := suite.(TearDownTestSuite); ok { + tearDownTestSuite.TearDownTest() + } + suite.SetT(parentT) + }() + method.Func.Call([]reflect.Value{reflect.ValueOf(suite)}) + }, + } + tests = append(tests, test) + } + } + + if !testing.RunTests(func(_, _ string) (bool, error) { return true, nil }, + tests) { + t.Fail() + } +} + +// Filtering method according to set regular expression +// specified command-line argument -m +func methodFilter(name string) (bool, error) { + if ok, _ := regexp.MatchString("^Test", name); !ok { + return false, nil + } + return regexp.MatchString(*matchMethod, name) +} diff --git a/vendor/google.golang.org/grpc/backoff.go b/vendor/google.golang.org/grpc/backoff.go index 52f4f10fc2..c99024ee30 100644 --- a/vendor/google.golang.org/grpc/backoff.go +++ b/vendor/google.golang.org/grpc/backoff.go @@ -58,7 +58,7 @@ func setDefaults(bc *BackoffConfig) { } } -func (bc BackoffConfig) backoff(retries int) (t time.Duration) { +func (bc BackoffConfig) backoff(retries int) time.Duration { if retries == 0 { return bc.baseDelay } diff --git a/vendor/google.golang.org/grpc/balancer.go b/vendor/google.golang.org/grpc/balancer.go index 419e214611..9d943fbada 100644 --- a/vendor/google.golang.org/grpc/balancer.go +++ b/vendor/google.golang.org/grpc/balancer.go @@ -38,6 +38,8 @@ import ( "sync" "golang.org/x/net/context" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/naming" ) @@ -52,6 +54,14 @@ type Address struct { Metadata interface{} } +// BalancerConfig specifies the configurations for Balancer. +type BalancerConfig struct { + // DialCreds is the transport credential the Balancer implementation can + // use to dial to a remote load balancer server. The Balancer implementations + // can ignore this if it does not need to talk to another party securely. + DialCreds credentials.TransportCredentials +} + // BalancerGetOptions configures a Get call. // This is the EXPERIMENTAL API and may be changed or extended in the future. type BalancerGetOptions struct { @@ -66,11 +76,11 @@ type Balancer interface { // Start does the initialization work to bootstrap a Balancer. For example, // this function may start the name resolution and watch the updates. It will // be called when dialing. - Start(target string) error + Start(target string, config BalancerConfig) error // Up informs the Balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets // lost or closed. - // TODO: It is not clear how to construct and take advantage the meaningful error + // TODO: It is not clear how to construct and take advantage of the meaningful error // parameter for down. Need realistic demands to guide. Up(addr Address) (down func(error)) // Get gets the address of a server for the RPC corresponding to ctx. @@ -205,7 +215,12 @@ func (rr *roundRobin) watchAddrUpdates() error { return nil } -func (rr *roundRobin) Start(target string) error { +func (rr *roundRobin) Start(target string, config BalancerConfig) error { + rr.mu.Lock() + defer rr.mu.Unlock() + if rr.done { + return ErrClientConnClosing + } if rr.r == nil { // If there is no name resolver installed, it is not needed to // do name resolution. In this case, target is added into rr.addrs @@ -301,7 +316,7 @@ func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Ad if !opts.BlockingWait { if len(rr.addrs) == 0 { rr.mu.Unlock() - err = fmt.Errorf("there is no address available") + err = Errorf(codes.Unavailable, "there is no address available") return } // Returns the next addr on rr.addrs for failfast RPCs. diff --git a/vendor/google.golang.org/grpc/benchmark/grpc_testing/services.pb.go b/vendor/google.golang.org/grpc/benchmark/grpc_testing/services.pb.go index 15d0864487..2aae3179bc 100644 --- a/vendor/google.golang.org/grpc/benchmark/grpc_testing/services.pb.go +++ b/vendor/google.golang.org/grpc/benchmark/grpc_testing/services.pb.go @@ -24,7 +24,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for BenchmarkService service @@ -161,7 +161,7 @@ var _BenchmarkService_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor3, + Metadata: "services.proto", } // Client API for WorkerService service @@ -417,7 +417,7 @@ var _WorkerService_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor3, + Metadata: "services.proto", } func init() { proto.RegisterFile("services.proto", fileDescriptor3) } diff --git a/vendor/google.golang.org/grpc/benchmark/worker/benchmark_client.go b/vendor/google.golang.org/grpc/benchmark/worker/benchmark_client.go index 77e522f26b..199bbe1fa3 100644 --- a/vendor/google.golang.org/grpc/benchmark/worker/benchmark_client.go +++ b/vendor/google.golang.org/grpc/benchmark/worker/benchmark_client.go @@ -317,28 +317,17 @@ func (bc *benchmarkClient) doCloseLoopStreaming(conns []*grpc.ClientConn, rpcCou // Now relying on worker client to reserve time to do warm up. // The worker client needs to wait for some time after client is created, // before starting benchmark. - done := make(chan bool) for { - go func() { - start := time.Now() - if err := doRPC(stream, reqSize, respSize); err != nil { - select { - case <-bc.stop: - case done <- false: - } - return - } - elapse := time.Since(start) - bc.lockingHistograms[idx].add(int64(elapse)) - select { - case <-bc.stop: - case done <- true: - } - }() + start := time.Now() + if err := doRPC(stream, reqSize, respSize); err != nil { + return + } + elapse := time.Since(start) + bc.lockingHistograms[idx].add(int64(elapse)) select { case <-bc.stop: return - case <-done: + default: } } }(idx) diff --git a/vendor/google.golang.org/grpc/call.go b/vendor/google.golang.org/grpc/call.go index 788b3d9281..5d9214d153 100644 --- a/vendor/google.golang.org/grpc/call.go +++ b/vendor/google.golang.org/grpc/call.go @@ -42,6 +42,7 @@ import ( "golang.org/x/net/context" "golang.org/x/net/trace" "google.golang.org/grpc/codes" + "google.golang.org/grpc/stats" "google.golang.org/grpc/transport" ) @@ -49,9 +50,9 @@ import ( // On error, it returns the error and indicates whether the call should be retried. // // TODO(zhaoq): Check whether the received message sequence is valid. -func recvResponse(dopts dialOptions, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) error { +// TODO ctx is used for stats collection and processing. It is the context passed from the application. +func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) (err error) { // Try to acquire header metadata from the server if there is any. - var err error defer func() { if err != nil { if _, ok := err.(transport.ConnectionError); !ok { @@ -61,17 +62,28 @@ func recvResponse(dopts dialOptions, t transport.ClientTransport, c *callInfo, s }() c.headerMD, err = stream.Header() if err != nil { - return err + return } p := &parser{r: stream} + var inPayload *stats.InPayload + if stats.On() { + inPayload = &stats.InPayload{ + Client: true, + } + } for { - if err = recv(p, dopts.codec, stream, dopts.dc, reply, math.MaxInt32); err != nil { + if err = recv(p, dopts.codec, stream, dopts.dc, reply, math.MaxInt32, inPayload); err != nil { if err == io.EOF { break } - return err + return } } + if inPayload != nil && err == io.EOF && stream.StatusCode() == codes.OK { + // TODO in the current implementation, inTrailer may be handled before inPayload in some cases. + // Fix the order if necessary. + stats.Handle(ctx, inPayload) + } c.trailerMD = stream.Trailer() return nil } @@ -90,15 +102,27 @@ func sendRequest(ctx context.Context, codec Codec, compressor Compressor, callHd } } }() - var cbuf *bytes.Buffer + var ( + cbuf *bytes.Buffer + outPayload *stats.OutPayload + ) if compressor != nil { cbuf = new(bytes.Buffer) } - outBuf, err := encode(codec, args, compressor, cbuf) + if stats.On() { + outPayload = &stats.OutPayload{ + Client: true, + } + } + outBuf, err := encode(codec, args, compressor, cbuf, outPayload) if err != nil { return nil, Errorf(codes.Internal, "grpc: %v", err) } err = t.Write(stream, outBuf, opts) + if err == nil && outPayload != nil { + outPayload.SentTime = time.Now() + stats.Handle(ctx, outPayload) + } // t.NewStream(...) could lead to an early rejection of the RPC (e.g., the service/method // does not exist.) so that t.Write could get io.EOF from wait(...). Leave the following // recvResponse to get the final status. @@ -119,7 +143,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli return invoke(ctx, method, args, reply, cc, opts...) } -func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (err error) { +func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo for _, o := range opts { if err := o.before(&c); err != nil { @@ -141,12 +165,30 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli c.traceInfo.tr.LazyLog(&c.traceInfo.firstLine, false) // TODO(dsymonds): Arrange for c.traceInfo.firstLine.remoteAddr to be set. defer func() { - if err != nil { - c.traceInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{err}}, true) + if e != nil { + c.traceInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{e}}, true) c.traceInfo.tr.SetError() } }() } + if stats.On() { + begin := &stats.Begin{ + Client: true, + BeginTime: time.Now(), + FailFast: c.failFast, + } + stats.Handle(ctx, begin) + } + defer func() { + if stats.On() { + end := &stats.End{ + Client: true, + EndTime: time.Now(), + Error: e, + } + stats.Handle(ctx, end) + } + }() topts := &transport.Options{ Last: true, Delay: false, @@ -206,7 +248,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } return toRPCErr(err) } - err = recvResponse(cc.dopts, t, &c, stream, reply) + err = recvResponse(ctx, cc.dopts, t, &c, stream, reply) if err != nil { if put != nil { put() diff --git a/vendor/google.golang.org/grpc/clientconn.go b/vendor/google.golang.org/grpc/clientconn.go index 6428f8a23e..f6dab4b772 100644 --- a/vendor/google.golang.org/grpc/clientconn.go +++ b/vendor/google.golang.org/grpc/clientconn.go @@ -199,6 +199,8 @@ func WithTimeout(d time.Duration) DialOption { } // WithDialer returns a DialOption that specifies a function to use for dialing network addresses. +// If FailOnNonTempDialError() is set to true, and an error is returned by f, gRPC checks the error's +// Temporary() method to decide if it should try to reconnect to the network address. func WithDialer(f func(string, time.Duration) (net.Conn, error)) DialOption { return func(o *dialOptions) { o.copts.Dialer = func(ctx context.Context, addr string) (net.Conn, error) { @@ -210,6 +212,17 @@ func WithDialer(f func(string, time.Duration) (net.Conn, error)) DialOption { } } +// FailOnNonTempDialError returns a DialOption that specified if gRPC fails on non-temporary dial errors. +// If f is true, and dialer returns a non-temporary error, gRPC will fail the connection to the network +// address and won't try to reconnect. +// The default value of FailOnNonTempDialError is false. +// This is an EXPERIMENTAL API. +func FailOnNonTempDialError(f bool) DialOption { + return func(o *dialOptions) { + o.copts.FailOnNonTempDialError = f + } +} + // WithUserAgent returns a DialOption that specifies a user agent string for all the RPCs. func WithUserAgent(s string) DialOption { return func(o *dialOptions) { @@ -270,31 +283,47 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * if cc.dopts.bs == nil { cc.dopts.bs = DefaultBackoffConfig } - - var ( - ok bool - addrs []Address - ) - if cc.dopts.balancer == nil { - // Connect to target directly if balancer is nil. - addrs = append(addrs, Address{Addr: target}) + creds := cc.dopts.copts.TransportCredentials + if creds != nil && creds.Info().ServerName != "" { + cc.authority = creds.Info().ServerName } else { - if err := cc.dopts.balancer.Start(target); err != nil { - return nil, err + colonPos := strings.LastIndex(target, ":") + if colonPos == -1 { + colonPos = len(target) } - ch := cc.dopts.balancer.Notify() - if ch == nil { - // There is no name resolver installed. + cc.authority = target[:colonPos] + } + var ok bool + waitC := make(chan error, 1) + go func() { + var addrs []Address + if cc.dopts.balancer == nil { + // Connect to target directly if balancer is nil. addrs = append(addrs, Address{Addr: target}) } else { - addrs, ok = <-ch - if !ok || len(addrs) == 0 { - return nil, errNoAddr + var credsClone credentials.TransportCredentials + if creds != nil { + credsClone = creds.Clone() + } + config := BalancerConfig{ + DialCreds: credsClone, + } + if err := cc.dopts.balancer.Start(target, config); err != nil { + waitC <- err + return + } + ch := cc.dopts.balancer.Notify() + if ch == nil { + // There is no name resolver installed. + addrs = append(addrs, Address{Addr: target}) + } else { + addrs, ok = <-ch + if !ok || len(addrs) == 0 { + waitC <- errNoAddr + return + } } } - } - waitC := make(chan error, 1) - go func() { for _, a := range addrs { if err := cc.resetAddrConn(a, false, nil); err != nil { waitC <- err @@ -322,16 +351,6 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * if ok { go cc.lbWatcher() } - creds := cc.dopts.copts.TransportCredentials - if creds != nil && creds.Info().ServerName != "" { - cc.authority = creds.Info().ServerName - } else { - colonPos := strings.LastIndex(target, ":") - if colonPos == -1 { - colonPos = len(target) - } - cc.authority = target[:colonPos] - } return cc, nil } @@ -678,14 +697,18 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { } ctx, cancel := context.WithTimeout(ac.ctx, timeout) connectTime := time.Now() - newTransport, err := transport.NewClientTransport(ctx, ac.addr.Addr, ac.dopts.copts) + sinfo := transport.TargetInfo{ + Addr: ac.addr.Addr, + Metadata: ac.addr.Metadata, + } + newTransport, err := transport.NewClientTransport(ctx, sinfo, ac.dopts.copts) if err != nil { cancel() if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() { return err } - grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, ac.addr) + grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %v", err, ac.addr) ac.mu.Lock() if ac.state == Shutdown { // ac.tearDown(...) has been invoked. @@ -797,7 +820,7 @@ func (ac *addrConn) transportMonitor() { } // wait blocks until i) the new transport is up or ii) ctx is done or iii) ac is closed or -// iv) transport is in TransientFailure and there's no balancer/failfast is true. +// iv) transport is in TransientFailure and there is a balancer/failfast is true. func (ac *addrConn) wait(ctx context.Context, hasBalancer, failfast bool) (transport.ClientTransport, error) { for { ac.mu.Lock() diff --git a/vendor/google.golang.org/grpc/credentials/credentials.go b/vendor/google.golang.org/grpc/credentials/credentials.go index a6285e62e3..5555ef024f 100644 --- a/vendor/google.golang.org/grpc/credentials/credentials.go +++ b/vendor/google.golang.org/grpc/credentials/credentials.go @@ -109,6 +109,12 @@ type TransportCredentials interface { ServerHandshake(net.Conn) (net.Conn, AuthInfo, error) // Info provides the ProtocolInfo of this TransportCredentials. Info() ProtocolInfo + // Clone makes a copy of this TransportCredentials. + Clone() TransportCredentials + // OverrideServerName overrides the server name used to verify the hostname on the returned certificates from the server. + // gRPC internals also use it to override the virtual hosting name if it is set. + // It must be called before dialing. Currently, this is only used by grpclb. + OverrideServerName(string) error } // TLSInfo contains the auth information for a TLS authenticated connection. @@ -136,16 +142,6 @@ func (c tlsCreds) Info() ProtocolInfo { } } -// GetRequestMetadata returns nil, nil since TLS credentials does not have -// metadata. -func (c *tlsCreds) GetRequestMetadata(ctx context.Context, uri ...string) (map[string]string, error) { - return nil, nil -} - -func (c *tlsCreds) RequireTransportSecurity() bool { - return true -} - func (c *tlsCreds) ClientHandshake(ctx context.Context, addr string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) { // use local cfg to avoid clobbering ServerName if using multiple endpoints cfg := cloneTLSConfig(c.config) @@ -182,6 +178,15 @@ func (c *tlsCreds) ServerHandshake(rawConn net.Conn) (net.Conn, AuthInfo, error) return conn, TLSInfo{conn.ConnectionState()}, nil } +func (c *tlsCreds) Clone() TransportCredentials { + return NewTLS(c.config) +} + +func (c *tlsCreds) OverrideServerName(serverNameOverride string) error { + c.config.ServerName = serverNameOverride + return nil +} + // NewTLS uses c to construct a TransportCredentials based on TLS. func NewTLS(c *tls.Config) TransportCredentials { tc := &tlsCreds{cloneTLSConfig(c)} @@ -190,16 +195,16 @@ func NewTLS(c *tls.Config) TransportCredentials { } // NewClientTLSFromCert constructs a TLS from the input certificate for client. -// serverNameOverwrite is for testing only. If set to a non empty string, -// it will overwrite the virtual host name of authority (e.g. :authority header field) in requests. -func NewClientTLSFromCert(cp *x509.CertPool, serverNameOverwrite string) TransportCredentials { - return NewTLS(&tls.Config{ServerName: serverNameOverwrite, RootCAs: cp}) +// serverNameOverride is for testing only. If set to a non empty string, +// it will override the virtual host name of authority (e.g. :authority header field) in requests. +func NewClientTLSFromCert(cp *x509.CertPool, serverNameOverride string) TransportCredentials { + return NewTLS(&tls.Config{ServerName: serverNameOverride, RootCAs: cp}) } // NewClientTLSFromFile constructs a TLS from the input certificate file for client. -// serverNameOverwrite is for testing only. If set to a non empty string, -// it will overwrite the virtual host name of authority (e.g. :authority header field) in requests. -func NewClientTLSFromFile(certFile, serverNameOverwrite string) (TransportCredentials, error) { +// serverNameOverride is for testing only. If set to a non empty string, +// it will override the virtual host name of authority (e.g. :authority header field) in requests. +func NewClientTLSFromFile(certFile, serverNameOverride string) (TransportCredentials, error) { b, err := ioutil.ReadFile(certFile) if err != nil { return nil, err @@ -208,7 +213,7 @@ func NewClientTLSFromFile(certFile, serverNameOverwrite string) (TransportCreden if !cp.AppendCertsFromPEM(b) { return nil, fmt.Errorf("credentials: failed to append certificates") } - return NewTLS(&tls.Config{ServerName: serverNameOverwrite, RootCAs: cp}), nil + return NewTLS(&tls.Config{ServerName: serverNameOverride, RootCAs: cp}), nil } // NewServerTLSFromCert constructs a TLS from the input certificate for server. diff --git a/vendor/google.golang.org/grpc/credentials/oauth/oauth.go b/vendor/google.golang.org/grpc/credentials/oauth/oauth.go index 8e68c4d73b..25393cc641 100644 --- a/vendor/google.golang.org/grpc/credentials/oauth/oauth.go +++ b/vendor/google.golang.org/grpc/credentials/oauth/oauth.go @@ -61,7 +61,7 @@ func (ts TokenSource) GetRequestMetadata(ctx context.Context, uri ...string) (ma }, nil } -// RequireTransportSecurity indicates whether the credentails requires transport security. +// RequireTransportSecurity indicates whether the credentials requires transport security. func (ts TokenSource) RequireTransportSecurity() bool { return true } diff --git a/vendor/google.golang.org/grpc/examples/helloworld/greeter_server/main.go b/vendor/google.golang.org/grpc/examples/helloworld/greeter_server/main.go index 4dce9e0796..d572b221fe 100644 --- a/vendor/google.golang.org/grpc/examples/helloworld/greeter_server/main.go +++ b/vendor/google.golang.org/grpc/examples/helloworld/greeter_server/main.go @@ -61,5 +61,7 @@ func main() { } s := grpc.NewServer() pb.RegisterGreeterServer(s, &server{}) - s.Serve(lis) + if err := s.Serve(lis); err != nil { + log.Fatalf("failed to serve: %v", err) + } } diff --git a/vendor/google.golang.org/grpc/examples/helloworld/helloworld/helloworld.pb.go b/vendor/google.golang.org/grpc/examples/helloworld/helloworld/helloworld.pb.go index 0419f6af7c..c8c8942a12 100644 --- a/vendor/google.golang.org/grpc/examples/helloworld/helloworld/helloworld.pb.go +++ b/vendor/google.golang.org/grpc/examples/helloworld/helloworld/helloworld.pb.go @@ -65,7 +65,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for Greeter service @@ -130,7 +130,7 @@ var _Greeter_serviceDesc = grpc.ServiceDesc{ }, }, Streams: []grpc.StreamDesc{}, - Metadata: fileDescriptor0, + Metadata: "helloworld.proto", } func init() { proto.RegisterFile("helloworld.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/examples/route_guide/routeguide/route_guide.pb.go b/vendor/google.golang.org/grpc/examples/route_guide/routeguide/route_guide.pb.go index 9bb1d6052d..cbcf2f30ad 100644 --- a/vendor/google.golang.org/grpc/examples/route_guide/routeguide/route_guide.pb.go +++ b/vendor/google.golang.org/grpc/examples/route_guide/routeguide/route_guide.pb.go @@ -156,7 +156,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for RouteGuide service @@ -452,7 +452,7 @@ var _RouteGuide_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor0, + Metadata: "route_guide.proto", } func init() { proto.RegisterFile("route_guide.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/grpclb.pb.go b/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/grpclb.pb.go new file mode 100644 index 0000000000..7be89477db --- /dev/null +++ b/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/grpclb.pb.go @@ -0,0 +1,557 @@ +// Code generated by protoc-gen-go. +// source: grpclb.proto +// DO NOT EDIT! + +/* +Package grpc_lb_v1 is a generated protocol buffer package. + +It is generated from these files: + grpclb.proto + +It has these top-level messages: + Duration + LoadBalanceRequest + InitialLoadBalanceRequest + ClientStats + LoadBalanceResponse + InitialLoadBalanceResponse + ServerList + Server +*/ +package grpc_lb_v1 + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" + +import ( + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +type Duration struct { + // Signed seconds of the span of time. Must be from -315,576,000,000 + // to +315,576,000,000 inclusive. + Seconds int64 `protobuf:"varint,1,opt,name=seconds" json:"seconds,omitempty"` + // Signed fractions of a second at nanosecond resolution of the span + // of time. Durations less than one second are represented with a 0 + // `seconds` field and a positive or negative `nanos` field. For durations + // of one second or more, a non-zero value for the `nanos` field must be + // of the same sign as the `seconds` field. Must be from -999,999,999 + // to +999,999,999 inclusive. + Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"` +} + +func (m *Duration) Reset() { *m = Duration{} } +func (m *Duration) String() string { return proto.CompactTextString(m) } +func (*Duration) ProtoMessage() {} +func (*Duration) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +type LoadBalanceRequest struct { + // Types that are valid to be assigned to LoadBalanceRequestType: + // *LoadBalanceRequest_InitialRequest + // *LoadBalanceRequest_ClientStats + LoadBalanceRequestType isLoadBalanceRequest_LoadBalanceRequestType `protobuf_oneof:"load_balance_request_type"` +} + +func (m *LoadBalanceRequest) Reset() { *m = LoadBalanceRequest{} } +func (m *LoadBalanceRequest) String() string { return proto.CompactTextString(m) } +func (*LoadBalanceRequest) ProtoMessage() {} +func (*LoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +type isLoadBalanceRequest_LoadBalanceRequestType interface { + isLoadBalanceRequest_LoadBalanceRequestType() +} + +type LoadBalanceRequest_InitialRequest struct { + InitialRequest *InitialLoadBalanceRequest `protobuf:"bytes,1,opt,name=initial_request,oneof"` +} +type LoadBalanceRequest_ClientStats struct { + ClientStats *ClientStats `protobuf:"bytes,2,opt,name=client_stats,oneof"` +} + +func (*LoadBalanceRequest_InitialRequest) isLoadBalanceRequest_LoadBalanceRequestType() {} +func (*LoadBalanceRequest_ClientStats) isLoadBalanceRequest_LoadBalanceRequestType() {} + +func (m *LoadBalanceRequest) GetLoadBalanceRequestType() isLoadBalanceRequest_LoadBalanceRequestType { + if m != nil { + return m.LoadBalanceRequestType + } + return nil +} + +func (m *LoadBalanceRequest) GetInitialRequest() *InitialLoadBalanceRequest { + if x, ok := m.GetLoadBalanceRequestType().(*LoadBalanceRequest_InitialRequest); ok { + return x.InitialRequest + } + return nil +} + +func (m *LoadBalanceRequest) GetClientStats() *ClientStats { + if x, ok := m.GetLoadBalanceRequestType().(*LoadBalanceRequest_ClientStats); ok { + return x.ClientStats + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*LoadBalanceRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _LoadBalanceRequest_OneofMarshaler, _LoadBalanceRequest_OneofUnmarshaler, _LoadBalanceRequest_OneofSizer, []interface{}{ + (*LoadBalanceRequest_InitialRequest)(nil), + (*LoadBalanceRequest_ClientStats)(nil), + } +} + +func _LoadBalanceRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*LoadBalanceRequest) + // load_balance_request_type + switch x := m.LoadBalanceRequestType.(type) { + case *LoadBalanceRequest_InitialRequest: + b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.InitialRequest); err != nil { + return err + } + case *LoadBalanceRequest_ClientStats: + b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.ClientStats); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("LoadBalanceRequest.LoadBalanceRequestType has unexpected type %T", x) + } + return nil +} + +func _LoadBalanceRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*LoadBalanceRequest) + switch tag { + case 1: // load_balance_request_type.initial_request + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InitialLoadBalanceRequest) + err := b.DecodeMessage(msg) + m.LoadBalanceRequestType = &LoadBalanceRequest_InitialRequest{msg} + return true, err + case 2: // load_balance_request_type.client_stats + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ClientStats) + err := b.DecodeMessage(msg) + m.LoadBalanceRequestType = &LoadBalanceRequest_ClientStats{msg} + return true, err + default: + return false, nil + } +} + +func _LoadBalanceRequest_OneofSizer(msg proto.Message) (n int) { + m := msg.(*LoadBalanceRequest) + // load_balance_request_type + switch x := m.LoadBalanceRequestType.(type) { + case *LoadBalanceRequest_InitialRequest: + s := proto.Size(x.InitialRequest) + n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *LoadBalanceRequest_ClientStats: + s := proto.Size(x.ClientStats) + n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n +} + +type InitialLoadBalanceRequest struct { + // Name of load balanced service (IE, service.grpc.gslb.google.com) + Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` +} + +func (m *InitialLoadBalanceRequest) Reset() { *m = InitialLoadBalanceRequest{} } +func (m *InitialLoadBalanceRequest) String() string { return proto.CompactTextString(m) } +func (*InitialLoadBalanceRequest) ProtoMessage() {} +func (*InitialLoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +// Contains client level statistics that are useful to load balancing. Each +// count should be reset to zero after reporting the stats. +type ClientStats struct { + // The total number of requests sent by the client since the last report. + TotalRequests int64 `protobuf:"varint,1,opt,name=total_requests" json:"total_requests,omitempty"` + // The number of client rpc errors since the last report. + ClientRpcErrors int64 `protobuf:"varint,2,opt,name=client_rpc_errors" json:"client_rpc_errors,omitempty"` + // The number of dropped requests since the last report. + DroppedRequests int64 `protobuf:"varint,3,opt,name=dropped_requests" json:"dropped_requests,omitempty"` +} + +func (m *ClientStats) Reset() { *m = ClientStats{} } +func (m *ClientStats) String() string { return proto.CompactTextString(m) } +func (*ClientStats) ProtoMessage() {} +func (*ClientStats) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } + +type LoadBalanceResponse struct { + // Types that are valid to be assigned to LoadBalanceResponseType: + // *LoadBalanceResponse_InitialResponse + // *LoadBalanceResponse_ServerList + LoadBalanceResponseType isLoadBalanceResponse_LoadBalanceResponseType `protobuf_oneof:"load_balance_response_type"` +} + +func (m *LoadBalanceResponse) Reset() { *m = LoadBalanceResponse{} } +func (m *LoadBalanceResponse) String() string { return proto.CompactTextString(m) } +func (*LoadBalanceResponse) ProtoMessage() {} +func (*LoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } + +type isLoadBalanceResponse_LoadBalanceResponseType interface { + isLoadBalanceResponse_LoadBalanceResponseType() +} + +type LoadBalanceResponse_InitialResponse struct { + InitialResponse *InitialLoadBalanceResponse `protobuf:"bytes,1,opt,name=initial_response,oneof"` +} +type LoadBalanceResponse_ServerList struct { + ServerList *ServerList `protobuf:"bytes,2,opt,name=server_list,oneof"` +} + +func (*LoadBalanceResponse_InitialResponse) isLoadBalanceResponse_LoadBalanceResponseType() {} +func (*LoadBalanceResponse_ServerList) isLoadBalanceResponse_LoadBalanceResponseType() {} + +func (m *LoadBalanceResponse) GetLoadBalanceResponseType() isLoadBalanceResponse_LoadBalanceResponseType { + if m != nil { + return m.LoadBalanceResponseType + } + return nil +} + +func (m *LoadBalanceResponse) GetInitialResponse() *InitialLoadBalanceResponse { + if x, ok := m.GetLoadBalanceResponseType().(*LoadBalanceResponse_InitialResponse); ok { + return x.InitialResponse + } + return nil +} + +func (m *LoadBalanceResponse) GetServerList() *ServerList { + if x, ok := m.GetLoadBalanceResponseType().(*LoadBalanceResponse_ServerList); ok { + return x.ServerList + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*LoadBalanceResponse) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _LoadBalanceResponse_OneofMarshaler, _LoadBalanceResponse_OneofUnmarshaler, _LoadBalanceResponse_OneofSizer, []interface{}{ + (*LoadBalanceResponse_InitialResponse)(nil), + (*LoadBalanceResponse_ServerList)(nil), + } +} + +func _LoadBalanceResponse_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*LoadBalanceResponse) + // load_balance_response_type + switch x := m.LoadBalanceResponseType.(type) { + case *LoadBalanceResponse_InitialResponse: + b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.InitialResponse); err != nil { + return err + } + case *LoadBalanceResponse_ServerList: + b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.ServerList); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("LoadBalanceResponse.LoadBalanceResponseType has unexpected type %T", x) + } + return nil +} + +func _LoadBalanceResponse_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*LoadBalanceResponse) + switch tag { + case 1: // load_balance_response_type.initial_response + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InitialLoadBalanceResponse) + err := b.DecodeMessage(msg) + m.LoadBalanceResponseType = &LoadBalanceResponse_InitialResponse{msg} + return true, err + case 2: // load_balance_response_type.server_list + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ServerList) + err := b.DecodeMessage(msg) + m.LoadBalanceResponseType = &LoadBalanceResponse_ServerList{msg} + return true, err + default: + return false, nil + } +} + +func _LoadBalanceResponse_OneofSizer(msg proto.Message) (n int) { + m := msg.(*LoadBalanceResponse) + // load_balance_response_type + switch x := m.LoadBalanceResponseType.(type) { + case *LoadBalanceResponse_InitialResponse: + s := proto.Size(x.InitialResponse) + n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *LoadBalanceResponse_ServerList: + s := proto.Size(x.ServerList) + n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n +} + +type InitialLoadBalanceResponse struct { + // This is an application layer redirect that indicates the client should use + // the specified server for load balancing. When this field is non-empty in + // the response, the client should open a separate connection to the + // load_balancer_delegate and call the BalanceLoad method. + LoadBalancerDelegate string `protobuf:"bytes,1,opt,name=load_balancer_delegate" json:"load_balancer_delegate,omitempty"` + // This interval defines how often the client should send the client stats + // to the load balancer. Stats should only be reported when the duration is + // positive. + ClientStatsReportInterval *Duration `protobuf:"bytes,3,opt,name=client_stats_report_interval" json:"client_stats_report_interval,omitempty"` +} + +func (m *InitialLoadBalanceResponse) Reset() { *m = InitialLoadBalanceResponse{} } +func (m *InitialLoadBalanceResponse) String() string { return proto.CompactTextString(m) } +func (*InitialLoadBalanceResponse) ProtoMessage() {} +func (*InitialLoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } + +func (m *InitialLoadBalanceResponse) GetClientStatsReportInterval() *Duration { + if m != nil { + return m.ClientStatsReportInterval + } + return nil +} + +type ServerList struct { + // Contains a list of servers selected by the load balancer. The list will + // be updated when server resolutions change or as needed to balance load + // across more servers. The client should consume the server list in order + // unless instructed otherwise via the client_config. + Servers []*Server `protobuf:"bytes,1,rep,name=servers" json:"servers,omitempty"` + // Indicates the amount of time that the client should consider this server + // list as valid. It may be considered stale after waiting this interval of + // time after receiving the list. If the interval is not positive, the + // client can assume the list is valid until the next list is received. + ExpirationInterval *Duration `protobuf:"bytes,3,opt,name=expiration_interval" json:"expiration_interval,omitempty"` +} + +func (m *ServerList) Reset() { *m = ServerList{} } +func (m *ServerList) String() string { return proto.CompactTextString(m) } +func (*ServerList) ProtoMessage() {} +func (*ServerList) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } + +func (m *ServerList) GetServers() []*Server { + if m != nil { + return m.Servers + } + return nil +} + +func (m *ServerList) GetExpirationInterval() *Duration { + if m != nil { + return m.ExpirationInterval + } + return nil +} + +type Server struct { + // A resolved address for the server, serialized in network-byte-order. It may + // either be an IPv4 or IPv6 address. + IpAddress []byte `protobuf:"bytes,1,opt,name=ip_address,proto3" json:"ip_address,omitempty"` + // A resolved port number for the server. + Port int32 `protobuf:"varint,2,opt,name=port" json:"port,omitempty"` + // An opaque but printable token given to the frontend for each pick. All + // frontend requests for that pick must include the token in its initial + // metadata. The token is used by the backend to verify the request and to + // allow the backend to report load to the gRPC LB system. + LoadBalanceToken string `protobuf:"bytes,3,opt,name=load_balance_token" json:"load_balance_token,omitempty"` + // Indicates whether this particular request should be dropped by the client + // when this server is chosen from the list. + DropRequest bool `protobuf:"varint,4,opt,name=drop_request" json:"drop_request,omitempty"` +} + +func (m *Server) Reset() { *m = Server{} } +func (m *Server) String() string { return proto.CompactTextString(m) } +func (*Server) ProtoMessage() {} +func (*Server) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } + +func init() { + proto.RegisterType((*Duration)(nil), "grpc.lb.v1.Duration") + proto.RegisterType((*LoadBalanceRequest)(nil), "grpc.lb.v1.LoadBalanceRequest") + proto.RegisterType((*InitialLoadBalanceRequest)(nil), "grpc.lb.v1.InitialLoadBalanceRequest") + proto.RegisterType((*ClientStats)(nil), "grpc.lb.v1.ClientStats") + proto.RegisterType((*LoadBalanceResponse)(nil), "grpc.lb.v1.LoadBalanceResponse") + proto.RegisterType((*InitialLoadBalanceResponse)(nil), "grpc.lb.v1.InitialLoadBalanceResponse") + proto.RegisterType((*ServerList)(nil), "grpc.lb.v1.ServerList") + proto.RegisterType((*Server)(nil), "grpc.lb.v1.Server") +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// Client API for LoadBalancer service + +type LoadBalancerClient interface { + // Bidirectional rpc to get a list of servers. + BalanceLoad(ctx context.Context, opts ...grpc.CallOption) (LoadBalancer_BalanceLoadClient, error) +} + +type loadBalancerClient struct { + cc *grpc.ClientConn +} + +func NewLoadBalancerClient(cc *grpc.ClientConn) LoadBalancerClient { + return &loadBalancerClient{cc} +} + +func (c *loadBalancerClient) BalanceLoad(ctx context.Context, opts ...grpc.CallOption) (LoadBalancer_BalanceLoadClient, error) { + stream, err := grpc.NewClientStream(ctx, &_LoadBalancer_serviceDesc.Streams[0], c.cc, "/grpc.lb.v1.LoadBalancer/BalanceLoad", opts...) + if err != nil { + return nil, err + } + x := &loadBalancerBalanceLoadClient{stream} + return x, nil +} + +type LoadBalancer_BalanceLoadClient interface { + Send(*LoadBalanceRequest) error + Recv() (*LoadBalanceResponse, error) + grpc.ClientStream +} + +type loadBalancerBalanceLoadClient struct { + grpc.ClientStream +} + +func (x *loadBalancerBalanceLoadClient) Send(m *LoadBalanceRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *loadBalancerBalanceLoadClient) Recv() (*LoadBalanceResponse, error) { + m := new(LoadBalanceResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// Server API for LoadBalancer service + +type LoadBalancerServer interface { + // Bidirectional rpc to get a list of servers. + BalanceLoad(LoadBalancer_BalanceLoadServer) error +} + +func RegisterLoadBalancerServer(s *grpc.Server, srv LoadBalancerServer) { + s.RegisterService(&_LoadBalancer_serviceDesc, srv) +} + +func _LoadBalancer_BalanceLoad_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(LoadBalancerServer).BalanceLoad(&loadBalancerBalanceLoadServer{stream}) +} + +type LoadBalancer_BalanceLoadServer interface { + Send(*LoadBalanceResponse) error + Recv() (*LoadBalanceRequest, error) + grpc.ServerStream +} + +type loadBalancerBalanceLoadServer struct { + grpc.ServerStream +} + +func (x *loadBalancerBalanceLoadServer) Send(m *LoadBalanceResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *loadBalancerBalanceLoadServer) Recv() (*LoadBalanceRequest, error) { + m := new(LoadBalanceRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _LoadBalancer_serviceDesc = grpc.ServiceDesc{ + ServiceName: "grpc.lb.v1.LoadBalancer", + HandlerType: (*LoadBalancerServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "BalanceLoad", + Handler: _LoadBalancer_BalanceLoad_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "grpclb.proto", +} + +func init() { proto.RegisterFile("grpclb.proto", fileDescriptor0) } + +var fileDescriptor0 = []byte{ + // 471 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x8c, 0x93, 0x51, 0x6f, 0xd3, 0x3e, + 0x14, 0xc5, 0x9b, 0x7f, 0xb7, 0xfd, 0xb7, 0x9b, 0xc0, 0xc6, 0xdd, 0x54, 0xda, 0x32, 0x8d, 0x2a, + 0x08, 0x54, 0x90, 0x08, 0x2c, 0xbc, 0xf1, 0x84, 0x0a, 0x0f, 0x45, 0xda, 0xd3, 0xf6, 0x86, 0x90, + 0x2c, 0x27, 0xb9, 0x9a, 0x2c, 0x82, 0x6d, 0x6c, 0xaf, 0x1a, 0xdf, 0x07, 0xf1, 0x39, 0x91, 0xe3, + 0x94, 0x64, 0x54, 0x15, 0xbc, 0xc5, 0xbe, 0x3e, 0xf7, 0x1e, 0xff, 0x7c, 0x02, 0xc9, 0xb5, 0xd1, + 0x65, 0x5d, 0x64, 0xda, 0x28, 0xa7, 0x10, 0xfc, 0x2a, 0xab, 0x8b, 0x6c, 0x75, 0x9e, 0xbe, 0x80, + 0xfd, 0x0f, 0x37, 0x86, 0x3b, 0xa1, 0x24, 0x1e, 0xc2, 0xff, 0x96, 0x4a, 0x25, 0x2b, 0x3b, 0x8e, + 0x66, 0xd1, 0x7c, 0x88, 0xf7, 0x60, 0x57, 0x72, 0xa9, 0xec, 0xf8, 0xbf, 0x59, 0x34, 0xdf, 0x4d, + 0x7f, 0x44, 0x80, 0x17, 0x8a, 0x57, 0x0b, 0x5e, 0x73, 0x59, 0xd2, 0x25, 0x7d, 0xbb, 0x21, 0xeb, + 0xf0, 0x1d, 0x1c, 0x0a, 0x29, 0x9c, 0xe0, 0x35, 0x33, 0x61, 0xab, 0x91, 0xc7, 0xf9, 0xd3, 0xac, + 0x1b, 0x94, 0x7d, 0x0c, 0x47, 0x36, 0xf5, 0xcb, 0x01, 0xbe, 0x82, 0xa4, 0xac, 0x05, 0x49, 0xc7, + 0xac, 0xe3, 0x2e, 0x8c, 0x8b, 0xf3, 0x87, 0x7d, 0xf9, 0xfb, 0xa6, 0x7e, 0xe5, 0xcb, 0xcb, 0xc1, + 0xe2, 0x11, 0x4c, 0x6a, 0xc5, 0x2b, 0x56, 0x84, 0x4e, 0xeb, 0xb9, 0xcc, 0x7d, 0xd7, 0x94, 0x3e, + 0x87, 0xc9, 0xd6, 0x61, 0x98, 0xc0, 0x8e, 0xe4, 0x5f, 0xa9, 0x71, 0x78, 0x90, 0x7e, 0x82, 0xb8, + 0xd7, 0x18, 0x47, 0x70, 0xdf, 0x29, 0xd7, 0xdd, 0x63, 0xcd, 0x61, 0x02, 0x0f, 0x5a, 0x7f, 0x46, + 0x97, 0x8c, 0x8c, 0x51, 0x26, 0x98, 0x1c, 0xe2, 0x18, 0x8e, 0x2a, 0xa3, 0xb4, 0xa6, 0xaa, 0x13, + 0x0d, 0x7d, 0x25, 0xfd, 0x19, 0xc1, 0xf1, 0x1d, 0x03, 0x56, 0x2b, 0x69, 0x09, 0x17, 0x70, 0xd4, + 0xe1, 0x0a, 0x7b, 0x2d, 0xaf, 0x67, 0x7f, 0xe3, 0x15, 0x4e, 0x2f, 0x07, 0xf8, 0x12, 0x62, 0x4b, + 0x66, 0x45, 0x86, 0xd5, 0xc2, 0xba, 0x96, 0xd7, 0xa8, 0x2f, 0xbf, 0x6a, 0xca, 0x17, 0xc2, 0xf3, + 0x5d, 0x9c, 0xc2, 0xf4, 0x0f, 0x5c, 0xa1, 0x53, 0xe0, 0x75, 0x0b, 0xd3, 0xed, 0xc3, 0xf0, 0x0c, + 0x46, 0x7d, 0xad, 0x61, 0x15, 0xd5, 0x74, 0xcd, 0x5d, 0x8b, 0x10, 0xdf, 0xc2, 0x69, 0xff, 0xed, + 0x98, 0x21, 0xad, 0x8c, 0x63, 0x42, 0x3a, 0x32, 0x2b, 0x5e, 0x37, 0x30, 0xe2, 0xfc, 0xa4, 0xef, + 0x6d, 0x1d, 0xb8, 0xb4, 0x02, 0xe8, 0x7c, 0xe2, 0x13, 0x1f, 0x3f, 0xbf, 0xf2, 0xd8, 0x87, 0xf3, + 0x38, 0xc7, 0xcd, 0x0b, 0xe1, 0x39, 0x1c, 0xd3, 0xad, 0x16, 0xa1, 0xc1, 0xbf, 0x4d, 0xf9, 0x0c, + 0x7b, 0xad, 0x18, 0x01, 0x84, 0x66, 0xbc, 0xaa, 0x0c, 0xd9, 0xf0, 0xb6, 0x89, 0x0f, 0x84, 0x37, + 0x1c, 0x22, 0x8e, 0x53, 0xc0, 0x3b, 0xa4, 0x9c, 0xfa, 0x42, 0xb2, 0xe9, 0x7e, 0x80, 0x27, 0x90, + 0xf8, 0xa7, 0xfe, 0x1d, 0xf2, 0x9d, 0x59, 0x34, 0xdf, 0xcf, 0x0b, 0x48, 0x7a, 0xd8, 0x0c, 0x5e, + 0x42, 0xdc, 0x7e, 0xfb, 0x6d, 0x3c, 0xeb, 0x5b, 0xda, 0xcc, 0xe3, 0xf4, 0xf1, 0xd6, 0x7a, 0xe0, + 0x3f, 0x8f, 0x5e, 0x47, 0xc5, 0x5e, 0xf3, 0xdf, 0xbe, 0xf9, 0x15, 0x00, 0x00, 0xff, 0xff, 0x01, + 0x8b, 0xc9, 0x26, 0xc7, 0x03, 0x00, 0x00, +} diff --git a/vendor/google.golang.org/grpc/grpclb/grpclb.go b/vendor/google.golang.org/grpc/grpclb/grpclb.go new file mode 100644 index 0000000000..d9a1a8b6fb --- /dev/null +++ b/vendor/google.golang.org/grpc/grpclb/grpclb.go @@ -0,0 +1,560 @@ +/* + * + * Copyright 2016, Google Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are + * met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above + * copyright notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Google Inc. nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + */ + +// Package grpclb implements the load balancing protocol defined at +// https://github.com/grpc/grpc/blob/master/doc/load-balancing.md. +// The implementation is currently EXPERIMENTAL. +package grpclb + +import ( + "errors" + "fmt" + "sync" + "time" + + "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/naming" +) + +// AddressType indicates the address type returned by name resolution. +type AddressType uint8 + +const ( + // Backend indicates the server is a backend server. + Backend AddressType = iota + // GRPCLB indicates the server is a grpclb load balancer. + GRPCLB +) + +// Metadata contains the information the name resolution for grpclb should provide. The +// name resolver used by grpclb balancer is required to provide this type of metadata in +// its address updates. +type Metadata struct { + // AddrType is the type of server (grpc load balancer or backend). + AddrType AddressType + // ServerName is the name of the grpc load balancer. Used for authentication. + ServerName string +} + +// Balancer creates a grpclb load balancer. +func Balancer(r naming.Resolver) grpc.Balancer { + return &balancer{ + r: r, + } +} + +type remoteBalancerInfo struct { + addr string + // the server name used for authentication with the remote LB server. + name string +} + +// addrInfo consists of the information of a backend server. +type addrInfo struct { + addr grpc.Address + connected bool + // dropRequest indicates whether a particular RPC which chooses this address + // should be dropped. + dropRequest bool +} + +type balancer struct { + r naming.Resolver + mu sync.Mutex + seq int // a sequence number to make sure addrCh does not get stale addresses. + w naming.Watcher + addrCh chan []grpc.Address + rbs []remoteBalancerInfo + addrs []*addrInfo + next int + waitCh chan struct{} + done bool + expTimer *time.Timer +} + +func (b *balancer) watchAddrUpdates(w naming.Watcher, ch chan remoteBalancerInfo) error { + updates, err := w.Next() + if err != nil { + return err + } + b.mu.Lock() + defer b.mu.Unlock() + if b.done { + return grpc.ErrClientConnClosing + } + var bAddr remoteBalancerInfo + if len(b.rbs) > 0 { + bAddr = b.rbs[0] + } + for _, update := range updates { + switch update.Op { + case naming.Add: + var exist bool + for _, v := range b.rbs { + // TODO: Is the same addr with different server name a different balancer? + if update.Addr == v.addr { + exist = true + break + } + } + if exist { + continue + } + md, ok := update.Metadata.(*Metadata) + if !ok { + // TODO: Revisit the handling here and may introduce some fallback mechanism. + grpclog.Printf("The name resolution contains unexpected metadata %v", update.Metadata) + continue + } + switch md.AddrType { + case Backend: + // TODO: Revisit the handling here and may introduce some fallback mechanism. + grpclog.Printf("The name resolution does not give grpclb addresses") + continue + case GRPCLB: + b.rbs = append(b.rbs, remoteBalancerInfo{ + addr: update.Addr, + name: md.ServerName, + }) + default: + grpclog.Printf("Received unknow address type %d", md.AddrType) + continue + } + case naming.Delete: + for i, v := range b.rbs { + if update.Addr == v.addr { + copy(b.rbs[i:], b.rbs[i+1:]) + b.rbs = b.rbs[:len(b.rbs)-1] + break + } + } + default: + grpclog.Println("Unknown update.Op ", update.Op) + } + } + // TODO: Fall back to the basic round-robin load balancing if the resulting address is + // not a load balancer. + if len(b.rbs) > 0 { + // For simplicity, always use the first one now. May revisit this decision later. + if b.rbs[0] != bAddr { + select { + case <-ch: + default: + } + ch <- b.rbs[0] + } + } + return nil +} + +func (b *balancer) serverListExpire(seq int) { + b.mu.Lock() + defer b.mu.Unlock() + // TODO: gRPC interanls do not clear the connections when the server list is stale. + // This means RPCs will keep using the existing server list until b receives new + // server list even though the list is expired. Revisit this behavior later. + if b.done || seq < b.seq { + return + } + b.next = 0 + b.addrs = nil + // Ask grpc internals to close all the corresponding connections. + b.addrCh <- nil +} + +func convertDuration(d *lbpb.Duration) time.Duration { + if d == nil { + return 0 + } + return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond +} + +func (b *balancer) processServerList(l *lbpb.ServerList, seq int) { + if l == nil { + return + } + servers := l.GetServers() + expiration := convertDuration(l.GetExpirationInterval()) + var ( + sl []*addrInfo + addrs []grpc.Address + ) + for _, s := range servers { + md := metadata.Pairs("lb-token", s.LoadBalanceToken) + addr := grpc.Address{ + Addr: fmt.Sprintf("%s:%d", s.IpAddress, s.Port), + Metadata: &md, + } + sl = append(sl, &addrInfo{ + addr: addr, + dropRequest: s.DropRequest, + }) + addrs = append(addrs, addr) + } + b.mu.Lock() + defer b.mu.Unlock() + if b.done || seq < b.seq { + return + } + if len(sl) > 0 { + // reset b.next to 0 when replacing the server list. + b.next = 0 + b.addrs = sl + b.addrCh <- addrs + if b.expTimer != nil { + b.expTimer.Stop() + b.expTimer = nil + } + if expiration > 0 { + b.expTimer = time.AfterFunc(expiration, func() { + b.serverListExpire(seq) + }) + } + } + return +} + +func (b *balancer) callRemoteBalancer(lbc lbpb.LoadBalancerClient, seq int) (retry bool) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + stream, err := lbc.BalanceLoad(ctx, grpc.FailFast(false)) + if err != nil { + grpclog.Printf("Failed to perform RPC to the remote balancer %v", err) + return + } + b.mu.Lock() + if b.done { + b.mu.Unlock() + return + } + b.mu.Unlock() + initReq := &lbpb.LoadBalanceRequest{ + LoadBalanceRequestType: &lbpb.LoadBalanceRequest_InitialRequest{ + InitialRequest: new(lbpb.InitialLoadBalanceRequest), + }, + } + if err := stream.Send(initReq); err != nil { + // TODO: backoff on retry? + return true + } + reply, err := stream.Recv() + if err != nil { + // TODO: backoff on retry? + return true + } + initResp := reply.GetInitialResponse() + if initResp == nil { + grpclog.Println("Failed to receive the initial response from the remote balancer.") + return + } + // TODO: Support delegation. + if initResp.LoadBalancerDelegate != "" { + // delegation + grpclog.Println("TODO: Delegation is not supported yet.") + return + } + // Retrieve the server list. + for { + reply, err := stream.Recv() + if err != nil { + break + } + b.mu.Lock() + if b.done || seq < b.seq { + b.mu.Unlock() + return + } + b.seq++ // tick when receiving a new list of servers. + seq = b.seq + b.mu.Unlock() + if serverList := reply.GetServerList(); serverList != nil { + b.processServerList(serverList, seq) + } + } + return true +} + +func (b *balancer) Start(target string, config grpc.BalancerConfig) error { + // TODO: Fall back to the basic direct connection if there is no name resolver. + if b.r == nil { + return errors.New("there is no name resolver installed") + } + b.mu.Lock() + if b.done { + b.mu.Unlock() + return grpc.ErrClientConnClosing + } + b.addrCh = make(chan []grpc.Address) + w, err := b.r.Resolve(target) + if err != nil { + b.mu.Unlock() + return err + } + b.w = w + b.mu.Unlock() + balancerAddrCh := make(chan remoteBalancerInfo, 1) + // Spawn a goroutine to monitor the name resolution of remote load balancer. + go func() { + for { + if err := b.watchAddrUpdates(w, balancerAddrCh); err != nil { + grpclog.Printf("grpc: the naming watcher stops working due to %v.\n", err) + close(balancerAddrCh) + return + } + } + }() + // Spawn a goroutine to talk to the remote load balancer. + go func() { + var cc *grpc.ClientConn + for { + rb, ok := <-balancerAddrCh + if cc != nil { + cc.Close() + } + if !ok { + // b is closing. + return + } + // Talk to the remote load balancer to get the server list. + var err error + creds := config.DialCreds + if creds == nil { + cc, err = grpc.Dial(rb.addr, grpc.WithInsecure()) + } else { + if rb.name != "" { + if err := creds.OverrideServerName(rb.name); err != nil { + grpclog.Printf("Failed to override the server name in the credentials: %v", err) + continue + } + } + cc, err = grpc.Dial(rb.addr, grpc.WithTransportCredentials(creds)) + } + if err != nil { + grpclog.Printf("Failed to setup a connection to the remote balancer %v: %v", rb.addr, err) + return + } + b.mu.Lock() + b.seq++ // tick when getting a new balancer address + seq := b.seq + b.next = 0 + b.mu.Unlock() + go func(cc *grpc.ClientConn) { + lbc := lbpb.NewLoadBalancerClient(cc) + for { + if retry := b.callRemoteBalancer(lbc, seq); !retry { + cc.Close() + return + } + } + }(cc) + } + }() + return nil +} + +func (b *balancer) down(addr grpc.Address, err error) { + b.mu.Lock() + defer b.mu.Unlock() + for _, a := range b.addrs { + if addr == a.addr { + a.connected = false + break + } + } +} + +func (b *balancer) Up(addr grpc.Address) func(error) { + b.mu.Lock() + defer b.mu.Unlock() + if b.done { + return nil + } + var cnt int + for _, a := range b.addrs { + if a.addr == addr { + if a.connected { + return nil + } + a.connected = true + } + if a.connected && !a.dropRequest { + cnt++ + } + } + // addr is the only one which is connected. Notify the Get() callers who are blocking. + if cnt == 1 && b.waitCh != nil { + close(b.waitCh) + b.waitCh = nil + } + return func(err error) { + b.down(addr, err) + } +} + +func (b *balancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (addr grpc.Address, put func(), err error) { + var ch chan struct{} + b.mu.Lock() + if b.done { + b.mu.Unlock() + err = grpc.ErrClientConnClosing + return + } + + if len(b.addrs) > 0 { + if b.next >= len(b.addrs) { + b.next = 0 + } + next := b.next + for { + a := b.addrs[next] + next = (next + 1) % len(b.addrs) + if a.connected { + if !a.dropRequest { + addr = a.addr + b.next = next + b.mu.Unlock() + return + } + if !opts.BlockingWait { + b.next = next + b.mu.Unlock() + err = grpc.Errorf(codes.Unavailable, "%s drops requests", a.addr.Addr) + return + } + } + if next == b.next { + // Has iterated all the possible address but none is connected. + break + } + } + } + if !opts.BlockingWait { + if len(b.addrs) == 0 { + b.mu.Unlock() + err = grpc.Errorf(codes.Unavailable, "there is no address available") + return + } + // Returns the next addr on b.addrs for a failfast RPC. + addr = b.addrs[b.next].addr + b.next++ + b.mu.Unlock() + return + } + // Wait on b.waitCh for non-failfast RPCs. + if b.waitCh == nil { + ch = make(chan struct{}) + b.waitCh = ch + } else { + ch = b.waitCh + } + b.mu.Unlock() + for { + select { + case <-ctx.Done(): + err = ctx.Err() + return + case <-ch: + b.mu.Lock() + if b.done { + b.mu.Unlock() + err = grpc.ErrClientConnClosing + return + } + + if len(b.addrs) > 0 { + if b.next >= len(b.addrs) { + b.next = 0 + } + next := b.next + for { + a := b.addrs[next] + next = (next + 1) % len(b.addrs) + if a.connected { + if !a.dropRequest { + addr = a.addr + b.next = next + b.mu.Unlock() + return + } + if !opts.BlockingWait { + b.next = next + b.mu.Unlock() + err = grpc.Errorf(codes.Unavailable, "drop requests for the addreess %s", a.addr.Addr) + return + } + } + if next == b.next { + // Has iterated all the possible address but none is connected. + break + } + } + } + // The newly added addr got removed by Down() again. + if b.waitCh == nil { + ch = make(chan struct{}) + b.waitCh = ch + } else { + ch = b.waitCh + } + b.mu.Unlock() + } + } +} + +func (b *balancer) Notify() <-chan []grpc.Address { + return b.addrCh +} + +func (b *balancer) Close() error { + b.mu.Lock() + defer b.mu.Unlock() + b.done = true + if b.expTimer != nil { + b.expTimer.Stop() + } + if b.waitCh != nil { + close(b.waitCh) + } + if b.addrCh != nil { + close(b.addrCh) + } + if b.w != nil { + b.w.Close() + } + return nil +} diff --git a/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go index 0e6a9100af..89c4d459f0 100644 --- a/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go +++ b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go @@ -90,7 +90,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for Health service @@ -153,7 +153,7 @@ var _Health_serviceDesc = grpc.ServiceDesc{ }, }, Streams: []grpc.StreamDesc{}, - Metadata: fileDescriptor0, + Metadata: "health.proto", } func init() { proto.RegisterFile("health.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/interop/client/client.go b/vendor/google.golang.org/grpc/interop/client/client.go index 98f6cfec3a..7961752b6f 100644 --- a/vendor/google.golang.org/grpc/interop/client/client.go +++ b/vendor/google.golang.org/grpc/interop/client/client.go @@ -54,7 +54,7 @@ var ( defaultServiceAccount = flag.String("default_service_account", "", "Email of GCE default service account") serverHost = flag.String("server_host", "127.0.0.1", "The server host name") serverPort = flag.Int("server_port", 10000, "The server port number") - tlsServerName = flag.String("server_host_override", "x.test.youtube.com", "The server name use to verify the hostname returned by TLS handshake if it is not empty. Otherwise, --server_host is used.") + tlsServerName = flag.String("server_host_override", "", "The server name use to verify the hostname returned by TLS handshake if it is not empty. Otherwise, --server_host is used.") testCase = flag.String("test_case", "large_unary", `Configure different test cases. Valid options are: empty_unary : empty (zero bytes) request and response; @@ -70,7 +70,11 @@ var ( per_rpc_creds: large_unary with per rpc token; oauth2_auth_token: large_unary with oauth2 token auth; cancel_after_begin: cancellation after metadata has been sent but before payloads are sent; - cancel_after_first_response: cancellation after receiving 1st message from the server.`) + cancel_after_first_response: cancellation after receiving 1st message from the server; + status_code_and_message: status code propagated back to client; + custom_metadata: server will echo custom metadata; + unimplemented_method: client attempts to call unimplemented method; + unimplemented_service: client attempts to call unimplemented service.`) // The test CA root cert file testCAFile = "testdata/ca.pem" @@ -180,6 +184,18 @@ func main() { case "cancel_after_first_response": interop.DoCancelAfterFirstResponse(tc) grpclog.Println("CancelAfterFirstResponse done") + case "status_code_and_message": + interop.DoStatusCodeAndMessage(tc) + grpclog.Println("StatusCodeAndMessage done") + case "custom_metadata": + interop.DoCustomMetadata(tc) + grpclog.Println("CustomMetadata done") + case "unimplemented_method": + interop.DoUnimplementedMethod(conn) + grpclog.Println("UnimplementedMethod done") + case "unimplemented_service": + interop.DoUnimplementedService(testpb.NewUnimplementedServiceClient(conn)) + grpclog.Println("UnimplementedService done") default: grpclog.Fatal("Unsupported test case: ", *testCase) } diff --git a/vendor/google.golang.org/grpc/interop/grpc_testing/test.pb.go b/vendor/google.golang.org/grpc/interop/grpc_testing/test.pb.go index 0ceb12df56..76ae5643ab 100644 --- a/vendor/google.golang.org/grpc/interop/grpc_testing/test.pb.go +++ b/vendor/google.golang.org/grpc/interop/grpc_testing/test.pb.go @@ -11,6 +11,7 @@ It is generated from these files: It has these top-level messages: Empty Payload + EchoStatus SimpleRequest SimpleResponse StreamingInputCallRequest @@ -119,6 +120,33 @@ func (m *Payload) GetBody() []byte { return nil } +// A protobuf representation for grpc status. This is used by test +// clients to specify a status that the server should attempt to return. +type EchoStatus struct { + Code *int32 `protobuf:"varint,1,opt,name=code" json:"code,omitempty"` + Message *string `protobuf:"bytes,2,opt,name=message" json:"message,omitempty"` + XXX_unrecognized []byte `json:"-"` +} + +func (m *EchoStatus) Reset() { *m = EchoStatus{} } +func (m *EchoStatus) String() string { return proto.CompactTextString(m) } +func (*EchoStatus) ProtoMessage() {} +func (*EchoStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +func (m *EchoStatus) GetCode() int32 { + if m != nil && m.Code != nil { + return *m.Code + } + return 0 +} + +func (m *EchoStatus) GetMessage() string { + if m != nil && m.Message != nil { + return *m.Message + } + return "" +} + // Unary request. type SimpleRequest struct { // Desired payload type in the response from the server. @@ -132,14 +160,16 @@ type SimpleRequest struct { // Whether SimpleResponse should include username. FillUsername *bool `protobuf:"varint,4,opt,name=fill_username,json=fillUsername" json:"fill_username,omitempty"` // Whether SimpleResponse should include OAuth scope. - FillOauthScope *bool `protobuf:"varint,5,opt,name=fill_oauth_scope,json=fillOauthScope" json:"fill_oauth_scope,omitempty"` - XXX_unrecognized []byte `json:"-"` + FillOauthScope *bool `protobuf:"varint,5,opt,name=fill_oauth_scope,json=fillOauthScope" json:"fill_oauth_scope,omitempty"` + // Whether server should return a given status + ResponseStatus *EchoStatus `protobuf:"bytes,7,opt,name=response_status,json=responseStatus" json:"response_status,omitempty"` + XXX_unrecognized []byte `json:"-"` } func (m *SimpleRequest) Reset() { *m = SimpleRequest{} } func (m *SimpleRequest) String() string { return proto.CompactTextString(m) } func (*SimpleRequest) ProtoMessage() {} -func (*SimpleRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +func (*SimpleRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } func (m *SimpleRequest) GetResponseType() PayloadType { if m != nil && m.ResponseType != nil { @@ -176,6 +206,13 @@ func (m *SimpleRequest) GetFillOauthScope() bool { return false } +func (m *SimpleRequest) GetResponseStatus() *EchoStatus { + if m != nil { + return m.ResponseStatus + } + return nil +} + // Unary response, as configured by the request. type SimpleResponse struct { // Payload to increase message size. @@ -191,7 +228,7 @@ type SimpleResponse struct { func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } func (*SimpleResponse) ProtoMessage() {} -func (*SimpleResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } +func (*SimpleResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } func (m *SimpleResponse) GetPayload() *Payload { if m != nil { @@ -224,7 +261,7 @@ type StreamingInputCallRequest struct { func (m *StreamingInputCallRequest) Reset() { *m = StreamingInputCallRequest{} } func (m *StreamingInputCallRequest) String() string { return proto.CompactTextString(m) } func (*StreamingInputCallRequest) ProtoMessage() {} -func (*StreamingInputCallRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } +func (*StreamingInputCallRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } func (m *StreamingInputCallRequest) GetPayload() *Payload { if m != nil { @@ -243,7 +280,7 @@ type StreamingInputCallResponse struct { func (m *StreamingInputCallResponse) Reset() { *m = StreamingInputCallResponse{} } func (m *StreamingInputCallResponse) String() string { return proto.CompactTextString(m) } func (*StreamingInputCallResponse) ProtoMessage() {} -func (*StreamingInputCallResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +func (*StreamingInputCallResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } func (m *StreamingInputCallResponse) GetAggregatedPayloadSize() int32 { if m != nil && m.AggregatedPayloadSize != nil { @@ -266,7 +303,7 @@ type ResponseParameters struct { func (m *ResponseParameters) Reset() { *m = ResponseParameters{} } func (m *ResponseParameters) String() string { return proto.CompactTextString(m) } func (*ResponseParameters) ProtoMessage() {} -func (*ResponseParameters) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } +func (*ResponseParameters) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } func (m *ResponseParameters) GetSize() int32 { if m != nil && m.Size != nil { @@ -292,14 +329,16 @@ type StreamingOutputCallRequest struct { // Configuration for each expected response message. ResponseParameters []*ResponseParameters `protobuf:"bytes,2,rep,name=response_parameters,json=responseParameters" json:"response_parameters,omitempty"` // Optional input payload sent along with the request. - Payload *Payload `protobuf:"bytes,3,opt,name=payload" json:"payload,omitempty"` - XXX_unrecognized []byte `json:"-"` + Payload *Payload `protobuf:"bytes,3,opt,name=payload" json:"payload,omitempty"` + // Whether server should return a given status + ResponseStatus *EchoStatus `protobuf:"bytes,7,opt,name=response_status,json=responseStatus" json:"response_status,omitempty"` + XXX_unrecognized []byte `json:"-"` } func (m *StreamingOutputCallRequest) Reset() { *m = StreamingOutputCallRequest{} } func (m *StreamingOutputCallRequest) String() string { return proto.CompactTextString(m) } func (*StreamingOutputCallRequest) ProtoMessage() {} -func (*StreamingOutputCallRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } +func (*StreamingOutputCallRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } func (m *StreamingOutputCallRequest) GetResponseType() PayloadType { if m != nil && m.ResponseType != nil { @@ -322,6 +361,13 @@ func (m *StreamingOutputCallRequest) GetPayload() *Payload { return nil } +func (m *StreamingOutputCallRequest) GetResponseStatus() *EchoStatus { + if m != nil { + return m.ResponseStatus + } + return nil +} + // Server-streaming response, as configured by the request and parameters. type StreamingOutputCallResponse struct { // Payload to increase response size. @@ -332,7 +378,7 @@ type StreamingOutputCallResponse struct { func (m *StreamingOutputCallResponse) Reset() { *m = StreamingOutputCallResponse{} } func (m *StreamingOutputCallResponse) String() string { return proto.CompactTextString(m) } func (*StreamingOutputCallResponse) ProtoMessage() {} -func (*StreamingOutputCallResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } +func (*StreamingOutputCallResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } func (m *StreamingOutputCallResponse) GetPayload() *Payload { if m != nil { @@ -344,6 +390,7 @@ func (m *StreamingOutputCallResponse) GetPayload() *Payload { func init() { proto.RegisterType((*Empty)(nil), "grpc.testing.Empty") proto.RegisterType((*Payload)(nil), "grpc.testing.Payload") + proto.RegisterType((*EchoStatus)(nil), "grpc.testing.EchoStatus") proto.RegisterType((*SimpleRequest)(nil), "grpc.testing.SimpleRequest") proto.RegisterType((*SimpleResponse)(nil), "grpc.testing.SimpleResponse") proto.RegisterType((*StreamingInputCallRequest)(nil), "grpc.testing.StreamingInputCallRequest") @@ -360,7 +407,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for TestService service @@ -742,47 +789,118 @@ var _TestService_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, + Metadata: "test.proto", +} + +// Client API for UnimplementedService service + +type UnimplementedServiceClient interface { + // A call that no server should implement + UnimplementedCall(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) +} + +type unimplementedServiceClient struct { + cc *grpc.ClientConn +} + +func NewUnimplementedServiceClient(cc *grpc.ClientConn) UnimplementedServiceClient { + return &unimplementedServiceClient{cc} +} + +func (c *unimplementedServiceClient) UnimplementedCall(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) { + out := new(Empty) + err := grpc.Invoke(ctx, "/grpc.testing.UnimplementedService/UnimplementedCall", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// Server API for UnimplementedService service + +type UnimplementedServiceServer interface { + // A call that no server should implement + UnimplementedCall(context.Context, *Empty) (*Empty, error) +} + +func RegisterUnimplementedServiceServer(s *grpc.Server, srv UnimplementedServiceServer) { + s.RegisterService(&_UnimplementedService_serviceDesc, srv) +} + +func _UnimplementedService_UnimplementedCall_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(Empty) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(UnimplementedServiceServer).UnimplementedCall(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/grpc.testing.UnimplementedService/UnimplementedCall", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(UnimplementedServiceServer).UnimplementedCall(ctx, req.(*Empty)) + } + return interceptor(ctx, in, info, handler) +} + +var _UnimplementedService_serviceDesc = grpc.ServiceDesc{ + ServiceName: "grpc.testing.UnimplementedService", + HandlerType: (*UnimplementedServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "UnimplementedCall", + Handler: _UnimplementedService_UnimplementedCall_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, Metadata: fileDescriptor0, } func init() { proto.RegisterFile("test.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 567 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xbc, 0x54, 0x51, 0x6f, 0xd2, 0x50, - 0x14, 0xb6, 0x03, 0x64, 0x1c, 0x58, 0x43, 0x0e, 0x59, 0x64, 0x9d, 0x89, 0x4b, 0x7d, 0xb0, 0x9a, - 0x88, 0x86, 0x44, 0x1f, 0x35, 0x73, 0x63, 0x71, 0x09, 0x03, 0x6c, 0xe1, 0x99, 0x5c, 0xe1, 0x0e, - 0x9b, 0x94, 0xb6, 0xb6, 0xb7, 0x46, 0x7c, 0xf0, 0x8f, 0xf9, 0x67, 0xfc, 0x11, 0xfe, 0x00, 0xef, - 0xbd, 0x6d, 0xa1, 0x40, 0x17, 0x99, 0xc6, 0xbd, 0xb5, 0xdf, 0xf9, 0xce, 0x77, 0xbe, 0xef, 0x9e, - 0xdb, 0x02, 0x30, 0x1a, 0xb2, 0x96, 0x1f, 0x78, 0xcc, 0xc3, 0xda, 0x2c, 0xf0, 0x27, 0x2d, 0x01, - 0xd8, 0xee, 0x4c, 0x2f, 0x43, 0xa9, 0x33, 0xf7, 0xd9, 0x42, 0xef, 0x42, 0x79, 0x40, 0x16, 0x8e, - 0x47, 0xa6, 0xf8, 0x1c, 0x8a, 0x6c, 0xe1, 0xd3, 0xa6, 0x72, 0xa2, 0x18, 0x6a, 0xfb, 0xa8, 0x95, - 0x6d, 0x68, 0x25, 0xa4, 0x21, 0x27, 0x98, 0x92, 0x86, 0x08, 0xc5, 0x8f, 0xde, 0x74, 0xd1, 0xdc, - 0xe3, 0xf4, 0x9a, 0x29, 0x9f, 0xf5, 0x5f, 0x0a, 0x1c, 0x58, 0xf6, 0xdc, 0x77, 0xa8, 0x49, 0x3f, - 0x47, 0xbc, 0x15, 0xdf, 0xc0, 0x41, 0x40, 0x43, 0xdf, 0x73, 0x43, 0x3a, 0xde, 0x4d, 0xbd, 0x96, - 0xf2, 0xc5, 0x1b, 0x3e, 0xce, 0xf4, 0x87, 0xf6, 0x37, 0x2a, 0xc7, 0x95, 0x56, 0x24, 0x8b, 0x63, - 0xf8, 0x02, 0xca, 0x7e, 0xac, 0xd0, 0x2c, 0xf0, 0x72, 0xb5, 0x7d, 0x98, 0x2b, 0x6f, 0xa6, 0x2c, - 0xa1, 0x7a, 0x6d, 0x3b, 0xce, 0x38, 0x0a, 0x69, 0xe0, 0x92, 0x39, 0x6d, 0x16, 0x79, 0xdb, 0xbe, - 0x59, 0x13, 0xe0, 0x28, 0xc1, 0xd0, 0x80, 0xba, 0x24, 0x79, 0x24, 0x62, 0x9f, 0xc6, 0xe1, 0xc4, - 0xe3, 0xee, 0x4b, 0x92, 0xa7, 0x0a, 0xbc, 0x2f, 0x60, 0x4b, 0xa0, 0xfa, 0x77, 0x50, 0xd3, 0xd4, - 0xb1, 0xab, 0xac, 0x23, 0x65, 0x27, 0x47, 0x1a, 0xec, 0x2f, 0xcd, 0x88, 0x88, 0x15, 0x73, 0xf9, - 0x8e, 0x8f, 0xa0, 0x9a, 0xf5, 0x50, 0x90, 0x65, 0xf0, 0x56, 0xf3, 0xbb, 0x70, 0x64, 0xb1, 0x80, - 0x92, 0x39, 0x97, 0xbe, 0x74, 0xfd, 0x88, 0x9d, 0x11, 0xc7, 0x49, 0x37, 0x70, 0x5b, 0x2b, 0xfa, - 0x10, 0xb4, 0x3c, 0xb5, 0x24, 0xd9, 0x6b, 0x78, 0x40, 0x66, 0xb3, 0x80, 0xce, 0x08, 0xa3, 0xd3, - 0x71, 0xd2, 0x13, 0xaf, 0x46, 0x91, 0xab, 0x39, 0x5c, 0x95, 0x13, 0x69, 0xb1, 0x23, 0xfd, 0x12, - 0x30, 0xd5, 0x18, 0x90, 0x80, 0xc7, 0x62, 0x34, 0x08, 0xc5, 0x25, 0xca, 0xb4, 0xca, 0x67, 0x11, - 0xd7, 0x76, 0x79, 0xf5, 0x0b, 0x11, 0x0b, 0x4a, 0x16, 0x0e, 0x29, 0x34, 0x0a, 0xf5, 0x9f, 0x4a, - 0xc6, 0x61, 0x3f, 0x62, 0x1b, 0x81, 0xff, 0xf5, 0xca, 0x7d, 0x80, 0xc6, 0xb2, 0xdf, 0x5f, 0x5a, - 0xe5, 0x3e, 0x0a, 0xfc, 0xf0, 0x4e, 0xd6, 0x55, 0xb6, 0x23, 0x99, 0x18, 0x6c, 0xc7, 0xbc, 0xed, - 0x05, 0xd5, 0x7b, 0x70, 0x9c, 0x9b, 0xf0, 0x2f, 0xaf, 0xd7, 0xb3, 0xb7, 0x50, 0xcd, 0x04, 0xc6, - 0x3a, 0xd4, 0xce, 0xfa, 0x57, 0x03, 0xb3, 0x63, 0x59, 0xa7, 0xef, 0xba, 0x9d, 0xfa, 0x3d, 0xbe, - 0x08, 0x75, 0xd4, 0x5b, 0xc3, 0x14, 0x04, 0xb8, 0x6f, 0x9e, 0xf6, 0xce, 0xfb, 0x57, 0xf5, 0xbd, - 0xf6, 0x8f, 0x22, 0x54, 0x87, 0x5c, 0xdd, 0xe2, 0x4b, 0xb0, 0x27, 0x14, 0x5f, 0x41, 0x45, 0xfe, - 0x40, 0x84, 0x2d, 0x6c, 0xac, 0x4f, 0x97, 0x05, 0x2d, 0x0f, 0xc4, 0x0b, 0xa8, 0x8c, 0x5c, 0x12, - 0xc4, 0x6d, 0xc7, 0xeb, 0x8c, 0xb5, 0x1f, 0x87, 0xf6, 0x30, 0xbf, 0x98, 0x1c, 0x80, 0x03, 0x8d, - 0x9c, 0xf3, 0x41, 0x63, 0xa3, 0xe9, 0xc6, 0x4b, 0xa2, 0x3d, 0xdd, 0x81, 0x19, 0xcf, 0x7a, 0xa9, - 0xa0, 0x0d, 0xb8, 0xfd, 0x45, 0xe0, 0x93, 0x1b, 0x24, 0x36, 0xbf, 0x40, 0xcd, 0xf8, 0x33, 0x31, - 0x1e, 0x65, 0x88, 0x51, 0xea, 0x45, 0xe4, 0x38, 0xe7, 0x11, 0x4f, 0xfb, 0xf5, 0xbf, 0x65, 0x32, - 0x14, 0x99, 0x4a, 0x7d, 0x4f, 0x9c, 0xeb, 0x3b, 0x18, 0xf5, 0x3b, 0x00, 0x00, 0xff, 0xff, 0x4c, - 0x41, 0xfe, 0xb6, 0x89, 0x06, 0x00, 0x00, + // 649 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xbc, 0x54, 0x4d, 0x6f, 0xd3, 0x40, + 0x10, 0xc5, 0x69, 0x42, 0xda, 0x49, 0x6a, 0xc2, 0x94, 0x0a, 0x37, 0x45, 0x22, 0x32, 0x07, 0x0c, + 0x12, 0x01, 0x45, 0x82, 0x03, 0x12, 0xa0, 0xd2, 0xa6, 0xa2, 0x52, 0xdb, 0x14, 0xbb, 0x39, 0x47, + 0x4b, 0x32, 0x75, 0x2d, 0xf9, 0x0b, 0x7b, 0x5d, 0x91, 0x1e, 0xf8, 0x33, 0xfc, 0x08, 0x0e, 0xfc, + 0x39, 0xb4, 0x6b, 0x3b, 0x71, 0xd2, 0x54, 0x34, 0x7c, 0xdd, 0x76, 0xdf, 0xbe, 0xf9, 0x78, 0x33, + 0xcf, 0x06, 0xe0, 0x14, 0xf3, 0x76, 0x18, 0x05, 0x3c, 0xc0, 0xba, 0x1d, 0x85, 0xc3, 0xb6, 0x00, + 0x1c, 0xdf, 0xd6, 0xab, 0x50, 0xe9, 0x7a, 0x21, 0x1f, 0xeb, 0x87, 0x50, 0x3d, 0x61, 0x63, 0x37, + 0x60, 0x23, 0x7c, 0x06, 0x65, 0x3e, 0x0e, 0x49, 0x53, 0x5a, 0x8a, 0xa1, 0x76, 0xb6, 0xda, 0xc5, + 0x80, 0x76, 0x46, 0x3a, 0x1d, 0x87, 0x64, 0x4a, 0x1a, 0x22, 0x94, 0x3f, 0x05, 0xa3, 0xb1, 0x56, + 0x6a, 0x29, 0x46, 0xdd, 0x94, 0x67, 0xfd, 0x35, 0x40, 0x77, 0x78, 0x1e, 0x58, 0x9c, 0xf1, 0x24, + 0x16, 0x8c, 0x61, 0x30, 0x4a, 0x13, 0x56, 0x4c, 0x79, 0x46, 0x0d, 0xaa, 0x1e, 0xc5, 0x31, 0xb3, + 0x49, 0x06, 0xae, 0x99, 0xf9, 0x55, 0xff, 0x5e, 0x82, 0x75, 0xcb, 0xf1, 0x42, 0x97, 0x4c, 0xfa, + 0x9c, 0x50, 0xcc, 0xf1, 0x2d, 0xac, 0x47, 0x14, 0x87, 0x81, 0x1f, 0xd3, 0xe0, 0x66, 0x9d, 0xd5, + 0x73, 0xbe, 0xb8, 0xe1, 0xa3, 0x42, 0x7c, 0xec, 0x5c, 0xa6, 0x15, 0x2b, 0x53, 0x92, 0xe5, 0x5c, + 0x12, 0x3e, 0x87, 0x6a, 0x98, 0x66, 0xd0, 0x56, 0x5a, 0x8a, 0x51, 0xeb, 0x6c, 0x2e, 0x4c, 0x6f, + 0xe6, 0x2c, 0x91, 0xf5, 0xcc, 0x71, 0xdd, 0x41, 0x12, 0x53, 0xe4, 0x33, 0x8f, 0xb4, 0x72, 0x4b, + 0x31, 0x56, 0xcd, 0xba, 0x00, 0xfb, 0x19, 0x86, 0x06, 0x34, 0x24, 0x29, 0x60, 0x09, 0x3f, 0x1f, + 0xc4, 0xc3, 0x20, 0x24, 0xad, 0x22, 0x79, 0xaa, 0xc0, 0x7b, 0x02, 0xb6, 0x04, 0x8a, 0x3b, 0x70, + 0x67, 0xda, 0xa4, 0x9c, 0x9b, 0x56, 0x95, 0x7d, 0x68, 0xb3, 0x7d, 0x4c, 0xe7, 0x6a, 0xaa, 0x13, + 0x01, 0xf2, 0xae, 0x7f, 0x05, 0x35, 0x1f, 0x5c, 0x8a, 0x17, 0x45, 0x29, 0x37, 0x12, 0xd5, 0x84, + 0xd5, 0x89, 0x9e, 0x74, 0x2f, 0x93, 0x3b, 0x3e, 0x84, 0x5a, 0x51, 0xc6, 0x8a, 0x7c, 0x86, 0x60, + 0x22, 0x41, 0x3f, 0x84, 0x2d, 0x8b, 0x47, 0xc4, 0x3c, 0xc7, 0xb7, 0x0f, 0xfc, 0x30, 0xe1, 0xbb, + 0xcc, 0x75, 0xf3, 0x25, 0x2e, 0xdb, 0x8a, 0x7e, 0x0a, 0xcd, 0x45, 0xd9, 0x32, 0x65, 0xaf, 0xe0, + 0x3e, 0xb3, 0xed, 0x88, 0x6c, 0xc6, 0x69, 0x34, 0xc8, 0x62, 0xd2, 0xed, 0xa6, 0x36, 0xdb, 0x9c, + 0x3e, 0x67, 0xa9, 0xc5, 0x9a, 0xf5, 0x03, 0xc0, 0x3c, 0xc7, 0x09, 0x8b, 0x98, 0x47, 0x9c, 0x22, + 0xe9, 0xd0, 0x42, 0xa8, 0x3c, 0x0b, 0xb9, 0x8e, 0xcf, 0x29, 0xba, 0x60, 0x62, 0xc7, 0x99, 0x67, + 0x20, 0x87, 0xfa, 0xb1, 0xfe, 0xad, 0x54, 0xe8, 0xb0, 0x97, 0xf0, 0x39, 0xc1, 0x7f, 0xea, 0xda, + 0x8f, 0xb0, 0x31, 0x89, 0x0f, 0x27, 0xad, 0x6a, 0xa5, 0xd6, 0x8a, 0x51, 0xeb, 0xb4, 0x66, 0xb3, + 0x5c, 0x95, 0x64, 0x62, 0x74, 0x55, 0xe6, 0xd2, 0x1e, 0xff, 0x0b, 0xa6, 0x3c, 0x86, 0xed, 0x85, + 0x43, 0xfa, 0x4d, 0x87, 0x3e, 0x7d, 0x07, 0xb5, 0xc2, 0xcc, 0xb0, 0x01, 0xf5, 0xdd, 0xde, 0xd1, + 0x89, 0xd9, 0xb5, 0xac, 0x9d, 0xf7, 0x87, 0xdd, 0xc6, 0x2d, 0x44, 0x50, 0xfb, 0xc7, 0x33, 0x98, + 0x82, 0x00, 0xb7, 0xcd, 0x9d, 0xe3, 0xbd, 0xde, 0x51, 0xa3, 0xd4, 0xf9, 0x51, 0x86, 0xda, 0x29, + 0xc5, 0xdc, 0xa2, 0xe8, 0xc2, 0x19, 0x12, 0xbe, 0x84, 0x35, 0xf9, 0x0b, 0x14, 0x6d, 0xe1, 0xc6, + 0x9c, 0x2e, 0xf1, 0xd0, 0x5c, 0x04, 0xe2, 0x3e, 0xac, 0xf5, 0x7d, 0x16, 0xa5, 0x61, 0xdb, 0xb3, + 0x8c, 0x99, 0xdf, 0x57, 0xf3, 0xc1, 0xe2, 0xc7, 0x6c, 0x00, 0x2e, 0x6c, 0x2c, 0x98, 0x0f, 0x1a, + 0x73, 0x41, 0xd7, 0xfa, 0xac, 0xf9, 0xe4, 0x06, 0xcc, 0xb4, 0xd6, 0x0b, 0x05, 0x1d, 0xc0, 0xab, + 0x1f, 0x15, 0x3e, 0xbe, 0x26, 0xc5, 0xfc, 0x47, 0xdc, 0x34, 0x7e, 0x4d, 0x4c, 0x4b, 0x19, 0xa2, + 0x94, 0xba, 0x9f, 0xb8, 0xee, 0x5e, 0x12, 0xba, 0xf4, 0xe5, 0x9f, 0x69, 0x32, 0x14, 0xa9, 0x4a, + 0xfd, 0xc0, 0xdc, 0xb3, 0xff, 0x50, 0xaa, 0xd3, 0x87, 0x7b, 0x7d, 0x5f, 0x6e, 0xd0, 0x23, 0x9f, + 0xd3, 0x28, 0x77, 0xd1, 0x1b, 0xb8, 0x3b, 0x83, 0x2f, 0xe7, 0xa6, 0x9f, 0x01, 0x00, 0x00, 0xff, + 0xff, 0xdd, 0xb5, 0x50, 0x6f, 0xa2, 0x07, 0x00, 0x00, } diff --git a/vendor/google.golang.org/grpc/interop/test_utils.go b/vendor/google.golang.org/grpc/interop/test_utils.go index 23340ab814..e0f8bd8cc5 100644 --- a/vendor/google.golang.org/grpc/interop/test_utils.go +++ b/vendor/google.golang.org/grpc/interop/test_utils.go @@ -52,10 +52,12 @@ import ( ) var ( - reqSizes = []int{27182, 8, 1828, 45904} - respSizes = []int{31415, 9, 2653, 58979} - largeReqSize = 271828 - largeRespSize = 314159 + reqSizes = []int{27182, 8, 1828, 45904} + respSizes = []int{31415, 9, 2653, 58979} + largeReqSize = 271828 + largeRespSize = 314159 + initialMetadataKey = "x-grpc-test-echo-initial" + trailingMetadataKey = "x-grpc-test-echo-trailing-bin" ) func clientNewPayload(t testpb.PayloadType, size int) *testpb.Payload { @@ -454,6 +456,143 @@ func DoCancelAfterFirstResponse(tc testpb.TestServiceClient) { } } +var ( + initialMetadataValue = "test_initial_metadata_value" + trailingMetadataValue = "\x0a\x0b\x0a\x0b\x0a\x0b" + customMetadata = metadata.Pairs( + initialMetadataKey, initialMetadataValue, + trailingMetadataKey, trailingMetadataValue, + ) +) + +func validateMetadata(header, trailer metadata.MD) { + if len(header[initialMetadataKey]) != 1 { + grpclog.Fatalf("Expected exactly one header from server. Received %d", len(header[initialMetadataKey])) + } + if header[initialMetadataKey][0] != initialMetadataValue { + grpclog.Fatalf("Got header %s; want %s", header[initialMetadataKey][0], initialMetadataValue) + } + if len(trailer[trailingMetadataKey]) != 1 { + grpclog.Fatalf("Expected exactly one trailer from server. Received %d", len(trailer[trailingMetadataKey])) + } + if trailer[trailingMetadataKey][0] != trailingMetadataValue { + grpclog.Fatalf("Got trailer %s; want %s", trailer[trailingMetadataKey][0], trailingMetadataValue) + } +} + +// DoCustomMetadata checks that metadata is echoed back to the client. +func DoCustomMetadata(tc testpb.TestServiceClient) { + // Testing with UnaryCall. + pl := clientNewPayload(testpb.PayloadType_COMPRESSABLE, 1) + req := &testpb.SimpleRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseSize: proto.Int32(int32(1)), + Payload: pl, + } + ctx := metadata.NewContext(context.Background(), customMetadata) + var header, trailer metadata.MD + reply, err := tc.UnaryCall( + ctx, + req, + grpc.Header(&header), + grpc.Trailer(&trailer), + ) + if err != nil { + grpclog.Fatal("/TestService/UnaryCall RPC failed: ", err) + } + t := reply.GetPayload().GetType() + s := len(reply.GetPayload().GetBody()) + if t != testpb.PayloadType_COMPRESSABLE || s != 1 { + grpclog.Fatalf("Got the reply with type %d len %d; want %d, %d", t, s, testpb.PayloadType_COMPRESSABLE, 1) + } + validateMetadata(header, trailer) + + // Testing with FullDuplex. + stream, err := tc.FullDuplexCall(ctx) + if err != nil { + grpclog.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + respParam := []*testpb.ResponseParameters{ + { + Size: proto.Int32(1), + }, + } + streamReq := &testpb.StreamingOutputCallRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseParameters: respParam, + Payload: pl, + } + if err := stream.Send(streamReq); err != nil { + grpclog.Fatalf("%v.Send(%v) = %v", stream, streamReq, err) + } + streamHeader, err := stream.Header() + if err != nil { + grpclog.Fatalf("%v.Header() = %v", stream, err) + } + if _, err := stream.Recv(); err != nil { + grpclog.Fatalf("%v.Recv() = %v", stream, err) + } + if err := stream.CloseSend(); err != nil { + grpclog.Fatalf("%v.CloseSend() = %v, want ", stream, err) + } + if _, err := stream.Recv(); err != io.EOF { + grpclog.Fatalf("%v failed to complete the custom metadata test: %v", stream, err) + } + streamTrailer := stream.Trailer() + validateMetadata(streamHeader, streamTrailer) +} + +// DoStatusCodeAndMessage checks that the status code is propagated back to the client. +func DoStatusCodeAndMessage(tc testpb.TestServiceClient) { + var code int32 = 2 + msg := "test status message" + expectedErr := grpc.Errorf(codes.Code(code), msg) + respStatus := &testpb.EchoStatus{ + Code: proto.Int32(code), + Message: proto.String(msg), + } + // Test UnaryCall. + req := &testpb.SimpleRequest{ + ResponseStatus: respStatus, + } + if _, err := tc.UnaryCall(context.Background(), req); err == nil || err.Error() != expectedErr.Error() { + grpclog.Fatalf("%v.UnaryCall(_, %v) = _, %v, want _, %v", tc, req, err, expectedErr) + } + // Test FullDuplexCall. + stream, err := tc.FullDuplexCall(context.Background()) + if err != nil { + grpclog.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + streamReq := &testpb.StreamingOutputCallRequest{ + ResponseStatus: respStatus, + } + if err := stream.Send(streamReq); err != nil { + grpclog.Fatalf("%v.Send(%v) = %v, want ", stream, streamReq, err) + } + if err := stream.CloseSend(); err != nil { + grpclog.Fatalf("%v.CloseSend() = %v, want ", stream, err) + } + if _, err = stream.Recv(); err.Error() != expectedErr.Error() { + grpclog.Fatalf("%v.Recv() returned error %v, want %v", stream, err, expectedErr) + } +} + +// DoUnimplementedService attempts to call a method from an unimplemented service. +func DoUnimplementedService(tc testpb.UnimplementedServiceClient) { + _, err := tc.UnimplementedCall(context.Background(), &testpb.Empty{}) + if grpc.Code(err) != codes.Unimplemented { + grpclog.Fatalf("%v.UnimplementedCall() = _, %v, want _, %v", tc, grpc.Code(err), codes.Unimplemented) + } +} + +// DoUnimplementedMethod attempts to call an unimplemented method. +func DoUnimplementedMethod(cc *grpc.ClientConn) { + var req, reply proto.Message + if err := grpc.Invoke(context.Background(), "/grpc.testing.TestService/UnimplementedCall", req, reply, cc); err == nil || grpc.Code(err) != codes.Unimplemented { + grpclog.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want error code %s", err, codes.Unimplemented) + } +} + type testServer struct { } @@ -485,6 +624,20 @@ func serverNewPayload(t testpb.PayloadType, size int32) (*testpb.Payload, error) } func (s *testServer) UnaryCall(ctx context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + status := in.GetResponseStatus() + if md, ok := metadata.FromContext(ctx); ok { + if initialMetadata, ok := md[initialMetadataKey]; ok { + header := metadata.Pairs(initialMetadataKey, initialMetadata[0]) + grpc.SendHeader(ctx, header) + } + if trailingMetadata, ok := md[trailingMetadataKey]; ok { + trailer := metadata.Pairs(trailingMetadataKey, trailingMetadata[0]) + grpc.SetTrailer(ctx, trailer) + } + } + if status != nil && *status.Code != 0 { + return nil, grpc.Errorf(codes.Code(*status.Code), *status.Message) + } pl, err := serverNewPayload(in.GetResponseType(), in.GetResponseSize()) if err != nil { return nil, err @@ -531,6 +684,16 @@ func (s *testServer) StreamingInputCall(stream testpb.TestService_StreamingInput } func (s *testServer) FullDuplexCall(stream testpb.TestService_FullDuplexCallServer) error { + if md, ok := metadata.FromContext(stream.Context()); ok { + if initialMetadata, ok := md[initialMetadataKey]; ok { + header := metadata.Pairs(initialMetadataKey, initialMetadata[0]) + stream.SendHeader(header) + } + if trailingMetadata, ok := md[trailingMetadataKey]; ok { + trailer := metadata.Pairs(trailingMetadataKey, trailingMetadata[0]) + stream.SetTrailer(trailer) + } + } for { in, err := stream.Recv() if err == io.EOF { @@ -540,6 +703,10 @@ func (s *testServer) FullDuplexCall(stream testpb.TestService_FullDuplexCallServ if err != nil { return err } + status := in.GetResponseStatus() + if status != nil && *status.Code != 0 { + return grpc.Errorf(codes.Code(*status.Code), *status.Message) + } cs := in.GetResponseParameters() for _, c := range cs { if us := c.GetIntervalUs(); us > 0 { diff --git a/vendor/google.golang.org/grpc/metadata/metadata.go b/vendor/google.golang.org/grpc/metadata/metadata.go index 954c0f77c6..65dc5af57d 100644 --- a/vendor/google.golang.org/grpc/metadata/metadata.go +++ b/vendor/google.golang.org/grpc/metadata/metadata.go @@ -117,10 +117,17 @@ func (md MD) Len() int { // Copy returns a copy of md. func (md MD) Copy() MD { + return Join(md) +} + +// Join joins any number of MDs into a single MD. +// The order of values for each key is determined by the order in which +// the MDs containing those values are presented to Join. +func Join(mds ...MD) MD { out := MD{} - for k, v := range md { - for _, i := range v { - out[k] = append(out[k], i) + for _, md := range mds { + for k, v := range md { + out[k] = append(out[k], v...) } } return out @@ -134,6 +141,8 @@ func NewContext(ctx context.Context, md MD) context.Context { } // FromContext returns the MD in ctx if it exists. +// The returned md should be immutable, writing to it may cause races. +// Modification should be made to the copies of the returned md. func FromContext(ctx context.Context) (md MD, ok bool) { md, ok = ctx.Value(mdKey{}).(MD) return diff --git a/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go index da90479e71..76987a4209 100644 --- a/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go +++ b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go @@ -544,7 +544,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for ServerReflection service @@ -643,7 +643,7 @@ var _ServerReflection_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor0, + Metadata: "reflection.proto", } func init() { proto.RegisterFile("reflection.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/reflection/grpc_testing/test.pb.go b/vendor/google.golang.org/grpc/reflection/grpc_testing/test.pb.go index add7abd09d..607dfd3b97 100644 --- a/vendor/google.golang.org/grpc/reflection/grpc_testing/test.pb.go +++ b/vendor/google.golang.org/grpc/reflection/grpc_testing/test.pb.go @@ -66,7 +66,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for SearchService service @@ -195,7 +195,7 @@ var _SearchService_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor2, + Metadata: "test.proto", } func init() { proto.RegisterFile("test.proto", fileDescriptor2) } diff --git a/vendor/google.golang.org/grpc/reflection/serverreflection.go b/vendor/google.golang.org/grpc/reflection/serverreflection.go index 686090aa5d..d26eac3e50 100644 --- a/vendor/google.golang.org/grpc/reflection/serverreflection.go +++ b/vendor/google.golang.org/grpc/reflection/serverreflection.go @@ -119,11 +119,11 @@ func (s *serverReflectionServer) decodeFileDesc(enc []byte) (*dpb.FileDescriptor func decompress(b []byte) ([]byte, error) { r, err := gzip.NewReader(bytes.NewReader(b)) if err != nil { - return nil, fmt.Errorf("bad gzipped descriptor: %v\n", err) + return nil, fmt.Errorf("bad gzipped descriptor: %v", err) } out, err := ioutil.ReadAll(r) if err != nil { - return nil, fmt.Errorf("bad gzipped descriptor: %v\n", err) + return nil, fmt.Errorf("bad gzipped descriptor: %v", err) } return out, nil } @@ -251,11 +251,12 @@ func (s *serverReflectionServer) fileDescEncodingContainingSymbol(name string) ( } // Metadata not valid. - enc, ok := meta.([]byte) + fileNameForMeta, ok := meta.(string) if !ok { return nil, fmt.Errorf("invalid file descriptor for symbol: %v", name) } + enc := proto.FileDescriptor(fileNameForMeta) fd, err = s.decodeFileDesc(enc) if err != nil { return nil, err diff --git a/vendor/google.golang.org/grpc/rpc_util.go b/vendor/google.golang.org/grpc/rpc_util.go index 6b60095d56..66d08b5aa2 100644 --- a/vendor/google.golang.org/grpc/rpc_util.go +++ b/vendor/google.golang.org/grpc/rpc_util.go @@ -42,11 +42,13 @@ import ( "io/ioutil" "math" "os" + "time" "github.com/golang/protobuf/proto" "golang.org/x/net/context" "google.golang.org/grpc/codes" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/stats" "google.golang.org/grpc/transport" ) @@ -255,9 +257,11 @@ func (p *parser) recvMsg(maxMsgSize int) (pf payloadFormat, msg []byte, err erro // encode serializes msg and prepends the message header. If msg is nil, it // generates the message header of 0 message length. -func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer) ([]byte, error) { - var b []byte - var length uint +func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer, outPayload *stats.OutPayload) ([]byte, error) { + var ( + b []byte + length uint + ) if msg != nil { var err error // TODO(zhaoq): optimize to reduce memory alloc and copying. @@ -265,6 +269,12 @@ func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer) ([]byte if err != nil { return nil, err } + if outPayload != nil { + outPayload.Payload = msg + // TODO truncate large payload. + outPayload.Data = b + outPayload.Length = len(b) + } if cp != nil { if err := cp.Do(cbuf, b); err != nil { return nil, err @@ -295,6 +305,10 @@ func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer) ([]byte // Copy encoded msg to buf copy(buf[5:], b) + if outPayload != nil { + outPayload.WireLength = len(buf) + } + return buf, nil } @@ -311,11 +325,14 @@ func checkRecvPayload(pf payloadFormat, recvCompress string, dc Decompressor) er return nil } -func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxMsgSize int) error { +func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxMsgSize int, inPayload *stats.InPayload) error { pf, d, err := p.recvMsg(maxMsgSize) if err != nil { return err } + if inPayload != nil { + inPayload.WireLength = len(d) + } if err := checkRecvPayload(pf, s.RecvCompress(), dc); err != nil { return err } @@ -333,6 +350,13 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ if err := c.Unmarshal(d, m); err != nil { return Errorf(codes.Internal, "grpc: failed to unmarshal the received message %v", err) } + if inPayload != nil { + inPayload.RecvTime = time.Now() + inPayload.Payload = m + // TODO truncate large payload. + inPayload.Data = d + inPayload.Length = len(d) + } return nil } @@ -448,10 +472,10 @@ func convertCode(err error) codes.Code { return codes.Unknown } -// SupportPackageIsVersion3 is referenced from generated protocol buffer files +// SupportPackageIsVersion4 is referenced from generated protocol buffer files // to assert that that code is compatible with this version of the grpc package. // // This constant may be renamed in the future if a change in the generated code // requires a synchronised update of grpc-go and protoc-gen-go. This constant // should not be referenced from any other code. -const SupportPackageIsVersion3 = true +const SupportPackageIsVersion4 = true diff --git a/vendor/google.golang.org/grpc/server.go b/vendor/google.golang.org/grpc/server.go index 2524dd2297..3af001ac94 100644 --- a/vendor/google.golang.org/grpc/server.go +++ b/vendor/google.golang.org/grpc/server.go @@ -54,6 +54,8 @@ import ( "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/stats" + "google.golang.org/grpc/tap" "google.golang.org/grpc/transport" ) @@ -89,10 +91,12 @@ type service struct { type Server struct { opts options - mu sync.Mutex // guards following - lis map[net.Listener]bool - conns map[io.Closer]bool - drain bool + mu sync.Mutex // guards following + lis map[net.Listener]bool + conns map[io.Closer]bool + drain bool + ctx context.Context + cancel context.CancelFunc // A CondVar to let GracefulStop() blocks until all the pending RPCs are finished // and all the transport goes away. cv *sync.Cond @@ -108,6 +112,7 @@ type options struct { maxMsgSize int unaryInt UnaryServerInterceptor streamInt StreamServerInterceptor + inTapHandle tap.ServerInHandle maxConcurrentStreams uint32 useHandlerImpl bool // use http.Handler-based server } @@ -184,6 +189,17 @@ func StreamInterceptor(i StreamServerInterceptor) ServerOption { } } +// InTapHandle returns a ServerOption that sets the tap handle for all the server +// transport to be created. Only one can be installed. +func InTapHandle(h tap.ServerInHandle) ServerOption { + return func(o *options) { + if o.inTapHandle != nil { + panic("The tap handle has been set.") + } + o.inTapHandle = h + } +} + // NewServer creates a gRPC server which has no service registered and has not // started to accept requests yet. func NewServer(opt ...ServerOption) *Server { @@ -203,6 +219,7 @@ func NewServer(opt ...ServerOption) *Server { m: make(map[string]*service), } s.cv = sync.NewCond(&s.mu) + s.ctx, s.cancel = context.WithCancel(context.Background()) if EnableTracing { _, file, line, _ := runtime.Caller(1) s.events = trace.NewEventLog("grpc.Server", fmt.Sprintf("%s:%d", file, line)) @@ -324,7 +341,7 @@ func (s *Server) useTransportAuthenticator(rawConn net.Conn) (net.Conn, credenti // Serve accepts incoming connections on the listener lis, creating a new // ServerTransport and service goroutine for each. The service goroutines // read gRPC requests and then call the registered handlers to reply to them. -// Serve returns when lis.Accept fails. lis will be closed when +// Serve returns when lis.Accept fails with fatal errors. lis will be closed when // this method returns. func (s *Server) Serve(lis net.Listener) error { s.mu.Lock() @@ -344,14 +361,38 @@ func (s *Server) Serve(lis net.Listener) error { } s.mu.Unlock() }() + + var tempDelay time.Duration // how long to sleep on accept failure + for { rawConn, err := lis.Accept() if err != nil { + if ne, ok := err.(interface { + Temporary() bool + }); ok && ne.Temporary() { + if tempDelay == 0 { + tempDelay = 5 * time.Millisecond + } else { + tempDelay *= 2 + } + if max := 1 * time.Second; tempDelay > max { + tempDelay = max + } + s.mu.Lock() + s.printf("Accept error: %v; retrying in %v", err, tempDelay) + s.mu.Unlock() + select { + case <-time.After(tempDelay): + case <-s.ctx.Done(): + } + continue + } s.mu.Lock() s.printf("done serving; Accept = %v", err) s.mu.Unlock() return err } + tempDelay = 0 // Start a new goroutine to deal with rawConn // so we don't stall this Accept loop goroutine. go s.handleRawConn(rawConn) @@ -385,17 +426,22 @@ func (s *Server) handleRawConn(rawConn net.Conn) { if s.opts.useHandlerImpl { s.serveUsingHandler(conn) } else { - s.serveNewHTTP2Transport(conn, authInfo) + s.serveHTTP2Transport(conn, authInfo) } } -// serveNewHTTP2Transport sets up a new http/2 transport (using the +// serveHTTP2Transport sets up a http/2 transport (using the // gRPC http2 server transport in transport/http2_server.go) and // serves streams on it. // This is run in its own goroutine (it does network I/O in // transport.NewServerTransport). -func (s *Server) serveNewHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) { - st, err := transport.NewServerTransport("http2", c, s.opts.maxConcurrentStreams, authInfo) +func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) { + config := &transport.ServerConfig{ + MaxStreams: s.opts.maxConcurrentStreams, + AuthInfo: authInfo, + InTapHandle: s.opts.inTapHandle, + } + st, err := transport.NewServerTransport("http2", c, config) if err != nil { s.mu.Lock() s.errorf("NewServerTransport(%q) failed: %v", c.RemoteAddr(), err) @@ -421,6 +467,12 @@ func (s *Server) serveStreams(st transport.ServerTransport) { defer wg.Done() s.handleStream(st, stream, s.traceInfo(st, stream)) }() + }, func(ctx context.Context, method string) context.Context { + if !EnableTracing { + return ctx + } + tr := trace.New("grpc.Recv."+methodFamily(method), method) + return trace.NewContext(ctx, tr) }) wg.Wait() } @@ -470,15 +522,17 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { // traceInfo returns a traceInfo and associates it with stream, if tracing is enabled. // If tracing is not enabled, it returns nil. func (s *Server) traceInfo(st transport.ServerTransport, stream *transport.Stream) (trInfo *traceInfo) { - if !EnableTracing { + tr, ok := trace.FromContext(stream.Context()) + if !ok { return nil } + trInfo = &traceInfo{ - tr: trace.New("grpc.Recv."+methodFamily(stream.Method()), stream.Method()), + tr: tr, } trInfo.firstLine.client = false trInfo.firstLine.remoteAddr = st.RemoteAddr() - stream.TraceContext(trInfo.tr) + if dl, ok := stream.Context().Deadline(); ok { trInfo.firstLine.deadline = dl.Sub(time.Now()) } @@ -500,16 +554,22 @@ func (s *Server) removeConn(c io.Closer) { defer s.mu.Unlock() if s.conns != nil { delete(s.conns, c) - s.cv.Signal() + s.cv.Broadcast() } } func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Stream, msg interface{}, cp Compressor, opts *transport.Options) error { - var cbuf *bytes.Buffer + var ( + cbuf *bytes.Buffer + outPayload *stats.OutPayload + ) if cp != nil { cbuf = new(bytes.Buffer) } - p, err := encode(s.opts.codec, msg, cp, cbuf) + if stats.On() { + outPayload = &stats.OutPayload{} + } + p, err := encode(s.opts.codec, msg, cp, cbuf, outPayload) if err != nil { // This typically indicates a fatal issue (e.g., memory // corruption or hardware faults) the application program @@ -520,10 +580,32 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str // the optimal option. grpclog.Fatalf("grpc: Server failed to encode response %v", err) } - return t.Write(stream, p, opts) + err = t.Write(stream, p, opts) + if err == nil && outPayload != nil { + outPayload.SentTime = time.Now() + stats.Handle(stream.Context(), outPayload) + } + return err } func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.Stream, srv *service, md *MethodDesc, trInfo *traceInfo) (err error) { + if stats.On() { + begin := &stats.Begin{ + BeginTime: time.Now(), + } + stats.Handle(stream.Context(), begin) + } + defer func() { + if stats.On() { + end := &stats.End{ + EndTime: time.Now(), + } + if err != nil && err != io.EOF { + end.Error = toRPCErr(err) + } + stats.Handle(stream.Context(), end) + } + }() if trInfo != nil { defer trInfo.tr.Finish() trInfo.firstLine.client = false @@ -552,14 +634,14 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err != nil { switch err := err.(type) { case *rpcError: - if err := t.WriteStatus(stream, err.code, err.desc); err != nil { - grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err) + if e := t.WriteStatus(stream, err.code, err.desc); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", e) } case transport.ConnectionError: // Nothing to do here. case transport.StreamError: - if err := t.WriteStatus(stream, err.Code, err.Desc); err != nil { - grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err) + if e := t.WriteStatus(stream, err.Code, err.Desc); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", e) } default: panic(fmt.Sprintf("grpc: Unexpected error (%T) from recvMsg: %v", err, err)) @@ -570,20 +652,29 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err := checkRecvPayload(pf, stream.RecvCompress(), s.opts.dc); err != nil { switch err := err.(type) { case *rpcError: - if err := t.WriteStatus(stream, err.code, err.desc); err != nil { - grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err) + if e := t.WriteStatus(stream, err.code, err.desc); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", e) } + return err default: - if err := t.WriteStatus(stream, codes.Internal, err.Error()); err != nil { - grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err) + if e := t.WriteStatus(stream, codes.Internal, err.Error()); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", e) } - + // TODO checkRecvPayload always return RPC error. Add a return here if necessary. + } + } + var inPayload *stats.InPayload + if stats.On() { + inPayload = &stats.InPayload{ + RecvTime: time.Now(), } - return err } statusCode := codes.OK statusDesc := "" df := func(v interface{}) error { + if inPayload != nil { + inPayload.WireLength = len(req) + } if pf == compressionMade { var err error req, err = s.opts.dc.Do(bytes.NewReader(req)) @@ -591,7 +682,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err := t.WriteStatus(stream, codes.Internal, err.Error()); err != nil { grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err) } - return err + return Errorf(codes.Internal, err.Error()) } } if len(req) > s.opts.maxMsgSize { @@ -603,6 +694,12 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err := s.opts.codec.Unmarshal(req, v); err != nil { return err } + if inPayload != nil { + inPayload.Payload = v + inPayload.Data = req + inPayload.Length = len(req) + stats.Handle(stream.Context(), inPayload) + } if trInfo != nil { trInfo.tr.LazyLog(&payload{sent: false, msg: v}, true) } @@ -623,9 +720,8 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } if err := t.WriteStatus(stream, statusCode, statusDesc); err != nil { grpclog.Printf("grpc: Server.processUnaryRPC failed to write status: %v", err) - return err } - return nil + return Errorf(statusCode, statusDesc) } if trInfo != nil { trInfo.tr.LazyLog(stringer("OK"), false) @@ -650,11 +746,32 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if trInfo != nil { trInfo.tr.LazyLog(&payload{sent: true, msg: reply}, true) } - return t.WriteStatus(stream, statusCode, statusDesc) + errWrite := t.WriteStatus(stream, statusCode, statusDesc) + if statusCode != codes.OK { + return Errorf(statusCode, statusDesc) + } + return errWrite } } func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transport.Stream, srv *service, sd *StreamDesc, trInfo *traceInfo) (err error) { + if stats.On() { + begin := &stats.Begin{ + BeginTime: time.Now(), + } + stats.Handle(stream.Context(), begin) + } + defer func() { + if stats.On() { + end := &stats.End{ + EndTime: time.Now(), + } + if err != nil && err != io.EOF { + end.Error = toRPCErr(err) + } + stats.Handle(stream.Context(), end) + } + }() if s.opts.cp != nil { stream.SetSendCompress(s.opts.cp.Type()) } @@ -717,7 +834,11 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp } ss.mu.Unlock() } - return t.WriteStatus(ss.s, ss.statusCode, ss.statusDesc) + errWrite := t.WriteStatus(ss.s, ss.statusCode, ss.statusDesc) + if ss.statusCode != codes.OK { + return Errorf(ss.statusCode, ss.statusDesc) + } + return errWrite } @@ -732,7 +853,8 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Str trInfo.tr.LazyLog(&fmtStringer{"Malformed method name %q", []interface{}{sm}}, true) trInfo.tr.SetError() } - if err := t.WriteStatus(stream, codes.InvalidArgument, fmt.Sprintf("malformed method name: %q", stream.Method())); err != nil { + errDesc := fmt.Sprintf("malformed method name: %q", stream.Method()) + if err := t.WriteStatus(stream, codes.InvalidArgument, errDesc); err != nil { if trInfo != nil { trInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{err}}, true) trInfo.tr.SetError() @@ -752,7 +874,8 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Str trInfo.tr.LazyLog(&fmtStringer{"Unknown service %v", []interface{}{service}}, true) trInfo.tr.SetError() } - if err := t.WriteStatus(stream, codes.Unimplemented, fmt.Sprintf("unknown service %v", service)); err != nil { + errDesc := fmt.Sprintf("unknown service %v", service) + if err := t.WriteStatus(stream, codes.Unimplemented, errDesc); err != nil { if trInfo != nil { trInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{err}}, true) trInfo.tr.SetError() @@ -777,7 +900,8 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Str trInfo.tr.LazyLog(&fmtStringer{"Unknown method %v", []interface{}{method}}, true) trInfo.tr.SetError() } - if err := t.WriteStatus(stream, codes.Unimplemented, fmt.Sprintf("unknown method %v", method)); err != nil { + errDesc := fmt.Sprintf("unknown method %v", method) + if err := t.WriteStatus(stream, codes.Unimplemented, errDesc); err != nil { if trInfo != nil { trInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{err}}, true) trInfo.tr.SetError() @@ -801,7 +925,7 @@ func (s *Server) Stop() { st := s.conns s.conns = nil // interrupt GracefulStop if Stop and GracefulStop are called concurrently. - s.cv.Signal() + s.cv.Broadcast() s.mu.Unlock() for lis := range listeners { @@ -812,6 +936,7 @@ func (s *Server) Stop() { } s.mu.Lock() + s.cancel() if s.events != nil { s.events.Finish() s.events = nil @@ -824,16 +949,19 @@ func (s *Server) Stop() { func (s *Server) GracefulStop() { s.mu.Lock() defer s.mu.Unlock() - if s.drain == true || s.conns == nil { + if s.conns == nil { return } - s.drain = true for lis := range s.lis { lis.Close() } s.lis = nil - for c := range s.conns { - c.(transport.ServerTransport).Drain() + s.cancel() + if !s.drain { + for c := range s.conns { + c.(transport.ServerTransport).Drain() + } + s.drain = true } for len(s.conns) != 0 { s.cv.Wait() @@ -865,33 +993,49 @@ func (s *Server) testingCloseConns() { s.mu.Unlock() } -// SendHeader sends header metadata. It may be called at most once from a unary -// RPC handler. The ctx is the RPC handler's Context or one derived from it. -func SendHeader(ctx context.Context, md metadata.MD) error { +// SetHeader sets the header metadata. +// When called multiple times, all the provided metadata will be merged. +// All the metadata will be sent out when one of the following happens: +// - grpc.SendHeader() is called; +// - The first response is sent out; +// - An RPC status is sent out (error or success). +func SetHeader(ctx context.Context, md metadata.MD) error { if md.Len() == 0 { return nil } stream, ok := transport.StreamFromContext(ctx) if !ok { - return fmt.Errorf("grpc: failed to fetch the stream from the context %v", ctx) + return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) + } + return stream.SetHeader(md) +} + +// SendHeader sends header metadata. It may be called at most once. +// The provided md and headers set by SetHeader() will be sent. +func SendHeader(ctx context.Context, md metadata.MD) error { + stream, ok := transport.StreamFromContext(ctx) + if !ok { + return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) } t := stream.ServerTransport() if t == nil { grpclog.Fatalf("grpc: SendHeader: %v has no ServerTransport to send header metadata.", stream) } - return t.WriteHeader(stream, md) + if err := t.WriteHeader(stream, md); err != nil { + return toRPCErr(err) + } + return nil } // SetTrailer sets the trailer metadata that will be sent when an RPC returns. -// It may be called at most once from a unary RPC handler. The ctx is the RPC -// handler's Context or one derived from it. +// When called more than once, all the provided metadata will be merged. func SetTrailer(ctx context.Context, md metadata.MD) error { if md.Len() == 0 { return nil } stream, ok := transport.StreamFromContext(ctx) if !ok { - return fmt.Errorf("grpc: failed to fetch the stream from the context %v", ctx) + return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) } return stream.SetTrailer(md) } diff --git a/vendor/google.golang.org/grpc/stats/grpc_testing/test.pb.go b/vendor/google.golang.org/grpc/stats/grpc_testing/test.pb.go new file mode 100644 index 0000000000..b24dcd8d34 --- /dev/null +++ b/vendor/google.golang.org/grpc/stats/grpc_testing/test.pb.go @@ -0,0 +1,225 @@ +// Code generated by protoc-gen-go. +// source: test.proto +// DO NOT EDIT! + +/* +Package grpc_testing is a generated protocol buffer package. + +It is generated from these files: + test.proto + +It has these top-level messages: + SimpleRequest + SimpleResponse +*/ +package grpc_testing + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" + +import ( + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +// Unary request. +type SimpleRequest struct { + Id int32 `protobuf:"varint,2,opt,name=id" json:"id,omitempty"` +} + +func (m *SimpleRequest) Reset() { *m = SimpleRequest{} } +func (m *SimpleRequest) String() string { return proto.CompactTextString(m) } +func (*SimpleRequest) ProtoMessage() {} +func (*SimpleRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +// Unary response, as configured by the request. +type SimpleResponse struct { + Id int32 `protobuf:"varint,3,opt,name=id" json:"id,omitempty"` +} + +func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } +func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } +func (*SimpleResponse) ProtoMessage() {} +func (*SimpleResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +func init() { + proto.RegisterType((*SimpleRequest)(nil), "grpc.testing.SimpleRequest") + proto.RegisterType((*SimpleResponse)(nil), "grpc.testing.SimpleResponse") +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// Client API for TestService service + +type TestServiceClient interface { + // One request followed by one response. + // The server returns the client id as-is. + UnaryCall(ctx context.Context, in *SimpleRequest, opts ...grpc.CallOption) (*SimpleResponse, error) + // A sequence of requests with each request served by the server immediately. + // As one request could lead to multiple responses, this interface + // demonstrates the idea of full duplexing. + FullDuplexCall(ctx context.Context, opts ...grpc.CallOption) (TestService_FullDuplexCallClient, error) +} + +type testServiceClient struct { + cc *grpc.ClientConn +} + +func NewTestServiceClient(cc *grpc.ClientConn) TestServiceClient { + return &testServiceClient{cc} +} + +func (c *testServiceClient) UnaryCall(ctx context.Context, in *SimpleRequest, opts ...grpc.CallOption) (*SimpleResponse, error) { + out := new(SimpleResponse) + err := grpc.Invoke(ctx, "/grpc.testing.TestService/UnaryCall", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *testServiceClient) FullDuplexCall(ctx context.Context, opts ...grpc.CallOption) (TestService_FullDuplexCallClient, error) { + stream, err := grpc.NewClientStream(ctx, &_TestService_serviceDesc.Streams[0], c.cc, "/grpc.testing.TestService/FullDuplexCall", opts...) + if err != nil { + return nil, err + } + x := &testServiceFullDuplexCallClient{stream} + return x, nil +} + +type TestService_FullDuplexCallClient interface { + Send(*SimpleRequest) error + Recv() (*SimpleResponse, error) + grpc.ClientStream +} + +type testServiceFullDuplexCallClient struct { + grpc.ClientStream +} + +func (x *testServiceFullDuplexCallClient) Send(m *SimpleRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *testServiceFullDuplexCallClient) Recv() (*SimpleResponse, error) { + m := new(SimpleResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// Server API for TestService service + +type TestServiceServer interface { + // One request followed by one response. + // The server returns the client id as-is. + UnaryCall(context.Context, *SimpleRequest) (*SimpleResponse, error) + // A sequence of requests with each request served by the server immediately. + // As one request could lead to multiple responses, this interface + // demonstrates the idea of full duplexing. + FullDuplexCall(TestService_FullDuplexCallServer) error +} + +func RegisterTestServiceServer(s *grpc.Server, srv TestServiceServer) { + s.RegisterService(&_TestService_serviceDesc, srv) +} + +func _TestService_UnaryCall_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SimpleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TestServiceServer).UnaryCall(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/grpc.testing.TestService/UnaryCall", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TestServiceServer).UnaryCall(ctx, req.(*SimpleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TestService_FullDuplexCall_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(TestServiceServer).FullDuplexCall(&testServiceFullDuplexCallServer{stream}) +} + +type TestService_FullDuplexCallServer interface { + Send(*SimpleResponse) error + Recv() (*SimpleRequest, error) + grpc.ServerStream +} + +type testServiceFullDuplexCallServer struct { + grpc.ServerStream +} + +func (x *testServiceFullDuplexCallServer) Send(m *SimpleResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *testServiceFullDuplexCallServer) Recv() (*SimpleRequest, error) { + m := new(SimpleRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _TestService_serviceDesc = grpc.ServiceDesc{ + ServiceName: "grpc.testing.TestService", + HandlerType: (*TestServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "UnaryCall", + Handler: _TestService_UnaryCall_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "FullDuplexCall", + Handler: _TestService_FullDuplexCall_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "test.proto", +} + +func init() { proto.RegisterFile("test.proto", fileDescriptor0) } + +var fileDescriptor0 = []byte{ + // 167 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0x49, 0x2d, 0x2e, + 0xd1, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x49, 0x2f, 0x2a, 0x48, 0xd6, 0x03, 0x09, 0x64, + 0xe6, 0xa5, 0x2b, 0xc9, 0x73, 0xf1, 0x06, 0x67, 0xe6, 0x16, 0xe4, 0xa4, 0x06, 0xa5, 0x16, 0x96, + 0xa6, 0x16, 0x97, 0x08, 0xf1, 0x71, 0x31, 0x65, 0xa6, 0x48, 0x30, 0x29, 0x30, 0x6a, 0xb0, 0x06, + 0x31, 0x65, 0xa6, 0x28, 0x29, 0x70, 0xf1, 0xc1, 0x14, 0x14, 0x17, 0xe4, 0xe7, 0x15, 0xa7, 0x42, + 0x55, 0x30, 0xc3, 0x54, 0x18, 0x2d, 0x63, 0xe4, 0xe2, 0x0e, 0x49, 0x2d, 0x2e, 0x09, 0x4e, 0x2d, + 0x2a, 0xcb, 0x4c, 0x4e, 0x15, 0x72, 0xe3, 0xe2, 0x0c, 0xcd, 0x4b, 0x2c, 0xaa, 0x74, 0x4e, 0xcc, + 0xc9, 0x11, 0x92, 0xd6, 0x43, 0xb6, 0x4e, 0x0f, 0xc5, 0x2e, 0x29, 0x19, 0xec, 0x92, 0x50, 0x7b, + 0xfc, 0xb9, 0xf8, 0xdc, 0x4a, 0x73, 0x72, 0x5c, 0x4a, 0x0b, 0x72, 0x52, 0x2b, 0x28, 0x34, 0x4c, + 0x83, 0xd1, 0x80, 0x31, 0x89, 0x0d, 0x1c, 0x00, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8d, + 0x82, 0x5b, 0xdd, 0x0e, 0x01, 0x00, 0x00, +} diff --git a/vendor/google.golang.org/grpc/stats/stats.go b/vendor/google.golang.org/grpc/stats/stats.go new file mode 100644 index 0000000000..4b030d985d --- /dev/null +++ b/vendor/google.golang.org/grpc/stats/stats.go @@ -0,0 +1,219 @@ +/* + * + * Copyright 2016, Google Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are + * met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above + * copyright notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Google Inc. nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + */ + +// Package stats is for collecting and reporting various network and RPC stats. +// This package is for monitoring purpose only. All fields are read-only. +// All APIs are experimental. +package stats // import "google.golang.org/grpc/stats" + +import ( + "net" + "sync/atomic" + "time" + + "golang.org/x/net/context" + "google.golang.org/grpc/grpclog" +) + +// RPCStats contains stats information about RPCs. +// All stats types in this package implements this interface. +type RPCStats interface { + // IsClient returns true if this RPCStats is from client side. + IsClient() bool +} + +// Begin contains stats when an RPC begins. +// FailFast are only valid if Client is true. +type Begin struct { + // Client is true if this Begin is from client side. + Client bool + // BeginTime is the time when the RPC begins. + BeginTime time.Time + // FailFast indicates if this RPC is failfast. + FailFast bool +} + +// IsClient indicates if this is from client side. +func (s *Begin) IsClient() bool { return s.Client } + +// InPayload contains the information for an incoming payload. +type InPayload struct { + // Client is true if this InPayload is from client side. + Client bool + // Payload is the payload with original type. + Payload interface{} + // Data is the serialized message payload. + Data []byte + // Length is the length of uncompressed data. + Length int + // WireLength is the length of data on wire (compressed, signed, encrypted). + WireLength int + // RecvTime is the time when the payload is received. + RecvTime time.Time +} + +// IsClient indicates if this is from client side. +func (s *InPayload) IsClient() bool { return s.Client } + +// InHeader contains stats when a header is received. +// FullMethod, addresses and Compression are only valid if Client is false. +type InHeader struct { + // Client is true if this InHeader is from client side. + Client bool + // WireLength is the wire length of header. + WireLength int + + // FullMethod is the full RPC method string, i.e., /package.service/method. + FullMethod string + // RemoteAddr is the remote address of the corresponding connection. + RemoteAddr net.Addr + // LocalAddr is the local address of the corresponding connection. + LocalAddr net.Addr + // Compression is the compression algorithm used for the RPC. + Compression string +} + +// IsClient indicates if this is from client side. +func (s *InHeader) IsClient() bool { return s.Client } + +// InTrailer contains stats when a trailer is received. +type InTrailer struct { + // Client is true if this InTrailer is from client side. + Client bool + // WireLength is the wire length of trailer. + WireLength int +} + +// IsClient indicates if this is from client side. +func (s *InTrailer) IsClient() bool { return s.Client } + +// OutPayload contains the information for an outgoing payload. +type OutPayload struct { + // Client is true if this OutPayload is from client side. + Client bool + // Payload is the payload with original type. + Payload interface{} + // Data is the serialized message payload. + Data []byte + // Length is the length of uncompressed data. + Length int + // WireLength is the length of data on wire (compressed, signed, encrypted). + WireLength int + // SentTime is the time when the payload is sent. + SentTime time.Time +} + +// IsClient indicates if this is from client side. +func (s *OutPayload) IsClient() bool { return s.Client } + +// OutHeader contains stats when a header is sent. +// FullMethod, addresses and Compression are only valid if Client is true. +type OutHeader struct { + // Client is true if this OutHeader is from client side. + Client bool + // WireLength is the wire length of header. + WireLength int + + // FullMethod is the full RPC method string, i.e., /package.service/method. + FullMethod string + // RemoteAddr is the remote address of the corresponding connection. + RemoteAddr net.Addr + // LocalAddr is the local address of the corresponding connection. + LocalAddr net.Addr + // Compression is the compression algorithm used for the RPC. + Compression string +} + +// IsClient indicates if this is from client side. +func (s *OutHeader) IsClient() bool { return s.Client } + +// OutTrailer contains stats when a trailer is sent. +type OutTrailer struct { + // Client is true if this OutTrailer is from client side. + Client bool + // WireLength is the wire length of trailer. + WireLength int +} + +// IsClient indicates if this is from client side. +func (s *OutTrailer) IsClient() bool { return s.Client } + +// End contains stats when an RPC ends. +type End struct { + // Client is true if this End is from client side. + Client bool + // EndTime is the time when the RPC ends. + EndTime time.Time + // Error is the error just happened. Its type is gRPC error. + Error error +} + +// IsClient indicates if this is from client side. +func (s *End) IsClient() bool { return s.Client } + +var ( + on = new(int32) + handler func(context.Context, RPCStats) +) + +// On indicates whether stats is started. +func On() bool { + return atomic.CompareAndSwapInt32(on, 1, 1) +} + +// Handle processes the stats using the call back function registered by user. +func Handle(ctx context.Context, s RPCStats) { + handler(ctx, s) +} + +// RegisterHandler registers the user handler function. +// If another handler was registered before, this new handler will overwrite the old one. +// This handler function will be called to process the stats. +func RegisterHandler(f func(context.Context, RPCStats)) { + handler = f +} + +// Start starts the stats collection and reporting if there is a registered stats handle. +func Start() { + if handler == nil { + grpclog.Println("handler is nil when starting stats. Stats is not started") + return + } + atomic.StoreInt32(on, 1) +} + +// Stop stops the stats collection and processing. +// Stop does not unregister handler. +func Stop() { + atomic.StoreInt32(on, 0) +} diff --git a/vendor/google.golang.org/grpc/stream.go b/vendor/google.golang.org/grpc/stream.go index e1b4759e72..95c8acf8d1 100644 --- a/vendor/google.golang.org/grpc/stream.go +++ b/vendor/google.golang.org/grpc/stream.go @@ -45,6 +45,7 @@ import ( "golang.org/x/net/trace" "google.golang.org/grpc/codes" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/stats" "google.golang.org/grpc/transport" ) @@ -97,7 +98,7 @@ type ClientStream interface { // NewClientStream creates a new Stream for the client side. This is called // by generated code. -func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (ClientStream, error) { +func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (_ ClientStream, err error) { if cc.dopts.streamInt != nil { return cc.dopts.streamInt(ctx, desc, cc, method, newClientStream, opts...) } @@ -143,6 +144,24 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } }() } + if stats.On() { + begin := &stats.Begin{ + Client: true, + BeginTime: time.Now(), + FailFast: c.failFast, + } + stats.Handle(ctx, begin) + } + defer func() { + if err != nil && stats.On() { + // Only handle end stats if err != nil. + end := &stats.End{ + Client: true, + Error: err, + } + stats.Handle(ctx, end) + } + }() gopts := BalancerGetOptions{ BlockingWait: !c.failFast, } @@ -194,6 +213,8 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth tracing: EnableTracing, trInfo: trInfo, + + statsCtx: ctx, } if cc.dopts.cp != nil { cs.cbuf = new(bytes.Buffer) @@ -246,6 +267,11 @@ type clientStream struct { // trInfo.tr is set when the clientStream is created (if EnableTracing is true), // and is set to nil when the clientStream's finish method is called. trInfo traceInfo + + // statsCtx keeps the user context for stats handling. + // All stats collection should use the statsCtx (instead of the stream context) + // so that all the generated stats for a particular RPC can be associated in the processing phase. + statsCtx context.Context } func (cs *clientStream) Context() context.Context { @@ -274,6 +300,8 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { } cs.mu.Unlock() } + // TODO Investigate how to signal the stats handling party. + // generate error stats if err != nil && err != io.EOF? defer func() { if err != nil { cs.finish(err) @@ -296,7 +324,13 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { } err = toRPCErr(err) }() - out, err := encode(cs.codec, m, cs.cp, cs.cbuf) + var outPayload *stats.OutPayload + if stats.On() { + outPayload = &stats.OutPayload{ + Client: true, + } + } + out, err := encode(cs.codec, m, cs.cp, cs.cbuf, outPayload) defer func() { if cs.cbuf != nil { cs.cbuf.Reset() @@ -305,11 +339,37 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { if err != nil { return Errorf(codes.Internal, "grpc: %v", err) } - return cs.t.Write(cs.s, out, &transport.Options{Last: false}) + err = cs.t.Write(cs.s, out, &transport.Options{Last: false}) + if err == nil && outPayload != nil { + outPayload.SentTime = time.Now() + stats.Handle(cs.statsCtx, outPayload) + } + return err } func (cs *clientStream) RecvMsg(m interface{}) (err error) { - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32) + defer func() { + if err != nil && stats.On() { + // Only generate End if err != nil. + // If err == nil, it's not the last RecvMsg. + // The last RecvMsg gets either an RPC error or io.EOF. + end := &stats.End{ + Client: true, + EndTime: time.Now(), + } + if err != io.EOF { + end.Error = toRPCErr(err) + } + stats.Handle(cs.statsCtx, end) + } + }() + var inPayload *stats.InPayload + if stats.On() { + inPayload = &stats.InPayload{ + Client: true, + } + } + err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32, inPayload) defer func() { // err != nil indicates the termination of the stream. if err != nil { @@ -324,11 +384,15 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { } cs.mu.Unlock() } + if inPayload != nil { + stats.Handle(cs.statsCtx, inPayload) + } if !cs.desc.ClientStreams || cs.desc.ServerStreams { return } // Special handling for client streaming rpc. - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32) + // This recv expects EOF or errors, so we don't collect inPayload. + err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32, nil) cs.closeTransportStream(err) if err == nil { return toRPCErr(errors.New("grpc: client streaming protocol violation: get , want ")) @@ -410,12 +474,19 @@ func (cs *clientStream) finish(err error) { // ServerStream defines the interface a server stream has to satisfy. type ServerStream interface { - // SendHeader sends the header metadata. It should not be called - // after SendProto. It fails if called multiple times or if - // called after SendProto. + // SetHeader sets the header metadata. It may be called multiple times. + // When call multiple times, all the provided metadata will be merged. + // All the metadata will be sent out when one of the following happens: + // - ServerStream.SendHeader() is called; + // - The first response is sent out; + // - An RPC status is sent out (error or success). + SetHeader(metadata.MD) error + // SendHeader sends the header metadata. + // The provided md and headers set by SetHeader() will be sent. + // It fails if called multiple times. SendHeader(metadata.MD) error - // SetTrailer sets the trailer metadata which will be sent with the - // RPC status. + // SetTrailer sets the trailer metadata which will be sent with the RPC status. + // When called more than once, all the provided metadata will be merged. SetTrailer(metadata.MD) Stream } @@ -441,6 +512,13 @@ func (ss *serverStream) Context() context.Context { return ss.s.Context() } +func (ss *serverStream) SetHeader(md metadata.MD) error { + if md.Len() == 0 { + return nil + } + return ss.s.SetHeader(md) +} + func (ss *serverStream) SendHeader(md metadata.MD) error { return ss.t.WriteHeader(ss.s, md) } @@ -468,7 +546,11 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { ss.mu.Unlock() } }() - out, err := encode(ss.codec, m, ss.cp, ss.cbuf) + var outPayload *stats.OutPayload + if stats.On() { + outPayload = &stats.OutPayload{} + } + out, err := encode(ss.codec, m, ss.cp, ss.cbuf, outPayload) defer func() { if ss.cbuf != nil { ss.cbuf.Reset() @@ -481,6 +563,10 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { if err := ss.t.Write(ss.s, out, &transport.Options{Last: false}); err != nil { return toRPCErr(err) } + if outPayload != nil { + outPayload.SentTime = time.Now() + stats.Handle(ss.s.Context(), outPayload) + } return nil } @@ -499,7 +585,11 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) { ss.mu.Unlock() } }() - if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxMsgSize); err != nil { + var inPayload *stats.InPayload + if stats.On() { + inPayload = &stats.InPayload{} + } + if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxMsgSize, inPayload); err != nil { if err == io.EOF { return err } @@ -508,5 +598,8 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) { } return toRPCErr(err) } + if inPayload != nil { + stats.Handle(ss.s.Context(), inPayload) + } return nil } diff --git a/vendor/google.golang.org/grpc/stress/client/main.go b/vendor/google.golang.org/grpc/stress/client/main.go index 4579aab463..89df93d23a 100644 --- a/vendor/google.golang.org/grpc/stress/client/main.go +++ b/vendor/google.golang.org/grpc/stress/client/main.go @@ -180,7 +180,7 @@ func (s *server) GetGauge(ctx context.Context, in *metricspb.GaugeRequest) (*met return nil, grpc.Errorf(codes.InvalidArgument, "gauge with name %s not found", in.Name) } -// createGauge creates a guage using the given name in metrics server. +// createGauge creates a gauge using the given name in metrics server. func (s *server) createGauge(name string) *gauge { s.mutex.Lock() defer s.mutex.Unlock() diff --git a/vendor/google.golang.org/grpc/stress/grpc_testing/metrics.pb.go b/vendor/google.golang.org/grpc/stress/grpc_testing/metrics.pb.go index 4ad4ccdb29..a1310b5f58 100644 --- a/vendor/google.golang.org/grpc/stress/grpc_testing/metrics.pb.go +++ b/vendor/google.golang.org/grpc/stress/grpc_testing/metrics.pb.go @@ -205,7 +205,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for MetricsService service @@ -335,7 +335,7 @@ var _MetricsService_serviceDesc = grpc.ServiceDesc{ ServerStreams: true, }, }, - Metadata: fileDescriptor0, + Metadata: "metrics.proto", } func init() { proto.RegisterFile("metrics.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/tap/tap.go b/vendor/google.golang.org/grpc/tap/tap.go new file mode 100644 index 0000000000..0f36647674 --- /dev/null +++ b/vendor/google.golang.org/grpc/tap/tap.go @@ -0,0 +1,54 @@ +/* + * + * Copyright 2016, Google Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are + * met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above + * copyright notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Google Inc. nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + */ + +// Package tap defines the function handles which are executed on the transport +// layer of gRPC-Go and related information. Everything here is EXPERIMENTAL. +package tap + +import ( + "golang.org/x/net/context" +) + +// Info defines the relevant information needed by the handles. +type Info struct { + // FullMethodName is the string of grpc method (in the format of + // /package.service/method). + FullMethodName string + // TODO: More to be added. +} + +// ServerInHandle defines the function which runs when a new stream is created +// on the server side. Note that it is executed in the per-connection I/O goroutine(s) instead +// of per-RPC goroutine. Therefore, users should NOT have any blocking/time-consuming +// work in this handle. Otherwise all the RPCs would slow down. +type ServerInHandle func(ctx context.Context, info *Info) (context.Context, error) diff --git a/vendor/google.golang.org/grpc/test/grpc_testing/test.pb.go b/vendor/google.golang.org/grpc/test/grpc_testing/test.pb.go index 0ceb12df56..e584c4d6d6 100644 --- a/vendor/google.golang.org/grpc/test/grpc_testing/test.pb.go +++ b/vendor/google.golang.org/grpc/test/grpc_testing/test.pb.go @@ -360,7 +360,7 @@ var _ grpc.ClientConn // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion3 +const _ = grpc.SupportPackageIsVersion4 // Client API for TestService service @@ -742,7 +742,7 @@ var _TestService_serviceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: fileDescriptor0, + Metadata: "test.proto", } func init() { proto.RegisterFile("test.proto", fileDescriptor0) } diff --git a/vendor/google.golang.org/grpc/transport/handler_server.go b/vendor/google.golang.org/grpc/transport/handler_server.go index 114e34906a..10b6dc0b19 100644 --- a/vendor/google.golang.org/grpc/transport/handler_server.go +++ b/vendor/google.golang.org/grpc/transport/handler_server.go @@ -268,7 +268,7 @@ func (ht *serverHandlerTransport) WriteHeader(s *Stream, md metadata.MD) error { }) } -func (ht *serverHandlerTransport) HandleStreams(startStream func(*Stream)) { +func (ht *serverHandlerTransport) HandleStreams(startStream func(*Stream), traceCtx func(context.Context, string) context.Context) { // With this transport type there will be exactly 1 stream: this HTTP request. var ctx context.Context diff --git a/vendor/google.golang.org/grpc/transport/http2_client.go b/vendor/google.golang.org/grpc/transport/http2_client.go index e9cf6bec42..cbd9f32602 100644 --- a/vendor/google.golang.org/grpc/transport/http2_client.go +++ b/vendor/google.golang.org/grpc/transport/http2_client.go @@ -51,15 +51,19 @@ import ( "google.golang.org/grpc/grpclog" "google.golang.org/grpc/metadata" "google.golang.org/grpc/peer" + "google.golang.org/grpc/stats" ) // http2Client implements the ClientTransport interface with HTTP2. type http2Client struct { - target string // server name/addr - userAgent string - conn net.Conn // underlying communication channel - authInfo credentials.AuthInfo // auth info about the connection - nextID uint32 // the next stream ID to be used + target string // server name/addr + userAgent string + md interface{} + conn net.Conn // underlying communication channel + remoteAddr net.Addr + localAddr net.Addr + authInfo credentials.AuthInfo // auth info about the connection + nextID uint32 // the next stream ID to be used // writableChan synchronizes write access to the transport. // A writer acquires the write lock by sending a value on writableChan @@ -107,7 +111,7 @@ type http2Client struct { prevGoAwayID uint32 } -func dial(fn func(context.Context, string) (net.Conn, error), ctx context.Context, addr string) (net.Conn, error) { +func dial(ctx context.Context, fn func(context.Context, string) (net.Conn, error), addr string) (net.Conn, error) { if fn != nil { return fn(ctx, addr) } @@ -145,10 +149,13 @@ func isTemporary(err error) bool { // newHTTP2Client constructs a connected ClientTransport to addr based on HTTP2 // and starts to receive messages on it. Non-nil error returns if construction // fails. -func newHTTP2Client(ctx context.Context, addr string, opts ConnectOptions) (_ ClientTransport, err error) { +func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (_ ClientTransport, err error) { scheme := "http" - conn, err := dial(opts.Dialer, ctx, addr) + conn, err := dial(ctx, opts.Dialer, addr.Addr) if err != nil { + if opts.FailOnNonTempDialError { + return nil, connectionErrorf(isTemporary(err), err, "transport: %v", err) + } return nil, connectionErrorf(true, err, "transport: %v", err) } // Any further errors will close the underlying connection @@ -160,7 +167,7 @@ func newHTTP2Client(ctx context.Context, addr string, opts ConnectOptions) (_ Cl var authInfo credentials.AuthInfo if creds := opts.TransportCredentials; creds != nil { scheme = "https" - conn, authInfo, err = creds.ClientHandshake(ctx, addr, conn) + conn, authInfo, err = creds.ClientHandshake(ctx, addr.Addr, conn) if err != nil { // Credentials handshake errors are typically considered permanent // to avoid retrying on e.g. bad certificates. @@ -174,10 +181,13 @@ func newHTTP2Client(ctx context.Context, addr string, opts ConnectOptions) (_ Cl } var buf bytes.Buffer t := &http2Client{ - target: addr, - userAgent: ua, - conn: conn, - authInfo: authInfo, + target: addr.Addr, + userAgent: ua, + md: addr.Metadata, + conn: conn, + remoteAddr: conn.RemoteAddr(), + localAddr: conn.LocalAddr(), + authInfo: authInfo, // The client initiated stream id is odd starting from 1. nextID: 1, writableChan: make(chan int, 1), @@ -252,8 +262,10 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream { s.windowHandler = func(n int) { t.updateWindow(s, uint32(n)) } - // Make a stream be able to cancel the pending operations by itself. - s.ctx, s.cancel = context.WithCancel(ctx) + // The client side stream context should have exactly the same life cycle with the user provided context. + // That means, s.ctx should be read-only. And s.ctx is done iff ctx is done. + // So we use the original context here instead of creating a copy. + s.ctx = ctx s.dec = &recvBufferReader{ ctx: s.ctx, goAway: s.goAway, @@ -265,23 +277,14 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream { // NewStream creates a stream and register it into the transport as "active" // streams. func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Stream, err error) { - // Record the timeout value on the context. - var timeout time.Duration - if dl, ok := ctx.Deadline(); ok { - timeout = dl.Sub(time.Now()) - } - select { - case <-ctx.Done(): - return nil, ContextErr(ctx.Err()) - default: - } pr := &peer.Peer{ - Addr: t.conn.RemoteAddr(), + Addr: t.remoteAddr, } // Attach Auth info if there is any. if t.authInfo != nil { pr.AuthInfo = t.authInfo } + userCtx := ctx ctx = peer.NewContext(ctx, pr) authData := make(map[string]string) for _, c := range t.creds { @@ -353,6 +356,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea return nil, ErrConnClosing } s := t.newStream(ctx, callHdr) + s.clientStatsCtx = userCtx t.activeStreams[s.id] = s // This stream is not counted when applySetings(...) initialize t.streamsQuota. @@ -381,9 +385,12 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea if callHdr.SendCompress != "" { t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: callHdr.SendCompress}) } - if timeout > 0 { + if dl, ok := ctx.Deadline(); ok { + // Send out timeout regardless its value. The server can detect timeout context by itself. + timeout := dl.Sub(time.Now()) t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-timeout", Value: encodeTimeout(timeout)}) } + for k, v := range authData { // Capital header names are illegal in HTTP/2. k = strings.ToLower(k) @@ -405,7 +412,18 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea } } } + if md, ok := t.md.(*metadata.MD); ok { + for k, v := range *md { + if isReservedHeader(k) { + continue + } + for _, entry := range v { + t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry}) + } + } + } first := true + bufLen := t.hBuf.Len() // Sends the headers in a single batch even when they span multiple frames. for !endHeaders { size := t.hBuf.Len() @@ -440,6 +458,17 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea return nil, connectionErrorf(true, err, "transport: %v", err) } } + if stats.On() { + outHeader := &stats.OutHeader{ + Client: true, + WireLength: bufLen, + FullMethod: callHdr.Method, + RemoteAddr: t.remoteAddr, + LocalAddr: t.localAddr, + Compression: callHdr.SendCompress, + } + stats.Handle(s.clientStatsCtx, outHeader) + } t.writableChan <- 0 return s, nil } @@ -795,6 +824,9 @@ func (t *http2Client) handleSettings(f *http2.SettingsFrame) { } func (t *http2Client) handlePing(f *http2.PingFrame) { + if f.IsAck() { // Do nothing. + return + } pingAck := &ping{ack: true} copy(pingAck.data[:], f.Data[:]) t.controlBuf.put(pingAck) @@ -864,6 +896,24 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) { } endStream := frame.StreamEnded() + var isHeader bool + defer func() { + if stats.On() { + if isHeader { + inHeader := &stats.InHeader{ + Client: true, + WireLength: int(frame.Header().Length), + } + stats.Handle(s.clientStatsCtx, inHeader) + } else { + inTrailer := &stats.InTrailer{ + Client: true, + WireLength: int(frame.Header().Length), + } + stats.Handle(s.clientStatsCtx, inTrailer) + } + } + }() s.mu.Lock() if !endStream { @@ -875,6 +925,7 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) { } close(s.headerChan) s.headerDone = true + isHeader = true } if !endStream || s.state == streamDone { s.mu.Unlock() diff --git a/vendor/google.golang.org/grpc/transport/http2_server.go b/vendor/google.golang.org/grpc/transport/http2_server.go index f753c4f1ea..db9beb90a6 100644 --- a/vendor/google.golang.org/grpc/transport/http2_server.go +++ b/vendor/google.golang.org/grpc/transport/http2_server.go @@ -50,6 +50,8 @@ import ( "google.golang.org/grpc/grpclog" "google.golang.org/grpc/metadata" "google.golang.org/grpc/peer" + "google.golang.org/grpc/stats" + "google.golang.org/grpc/tap" ) // ErrIllegalHeaderWrite indicates that setting header is illegal because of @@ -59,8 +61,11 @@ var ErrIllegalHeaderWrite = errors.New("transport: the stream is done or WriteHe // http2Server implements the ServerTransport interface with HTTP2. type http2Server struct { conn net.Conn + remoteAddr net.Addr + localAddr net.Addr maxStreamID uint32 // max stream ID ever seen authInfo credentials.AuthInfo // auth info about the connection + inTapHandle tap.ServerInHandle // writableChan synchronizes write access to the transport. // A writer acquires the write lock by receiving a value on writableChan // and releases it by sending on writableChan. @@ -91,12 +96,13 @@ type http2Server struct { // newHTTP2Server constructs a ServerTransport based on HTTP2. ConnectionError is // returned if something goes wrong. -func newHTTP2Server(conn net.Conn, maxStreams uint32, authInfo credentials.AuthInfo) (_ ServerTransport, err error) { +func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err error) { framer := newFramer(conn) // Send initial settings as connection preface to client. var settings []http2.Setting // TODO(zhaoq): Have a better way to signal "no limit" because 0 is // permitted in the HTTP2 spec. + maxStreams := config.MaxStreams if maxStreams == 0 { maxStreams = math.MaxUint32 } else { @@ -122,11 +128,14 @@ func newHTTP2Server(conn net.Conn, maxStreams uint32, authInfo credentials.AuthI var buf bytes.Buffer t := &http2Server{ conn: conn, - authInfo: authInfo, + remoteAddr: conn.RemoteAddr(), + localAddr: conn.LocalAddr(), + authInfo: config.AuthInfo, framer: framer, hBuf: &buf, hEnc: hpack.NewEncoder(&buf), maxStreams: maxStreams, + inTapHandle: config.InTapHandle, controlBuf: newRecvBuffer(), fc: &inFlow{limit: initialConnWindowSize}, sendQuotaPool: newQuotaPool(defaultWindowSize), @@ -142,7 +151,7 @@ func newHTTP2Server(conn net.Conn, maxStreams uint32, authInfo credentials.AuthI } // operateHeader takes action on the decoded headers. -func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(*Stream)) (close bool) { +func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(*Stream), traceCtx func(context.Context, string) context.Context) (close bool) { buf := newRecvBuffer() s := &Stream{ id: frame.Header().StreamID, @@ -173,7 +182,7 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( s.ctx, s.cancel = context.WithCancel(context.TODO()) } pr := &peer.Peer{ - Addr: t.conn.RemoteAddr(), + Addr: t.remoteAddr, } // Attach Auth info if there is any. if t.authInfo != nil { @@ -195,6 +204,18 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( } s.recvCompress = state.encoding s.method = state.method + if t.inTapHandle != nil { + var err error + info := &tap.Info{ + FullMethodName: state.method, + } + s.ctx, err = t.inTapHandle(s.ctx, info) + if err != nil { + // TODO: Log the real error. + t.controlBuf.put(&resetStream{s.id, http2.ErrCodeRefusedStream}) + return + } + } t.mu.Lock() if t.state != reachable { t.mu.Unlock() @@ -218,13 +239,25 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( s.windowHandler = func(n int) { t.updateWindow(s, uint32(n)) } + s.ctx = traceCtx(s.ctx, s.method) + if stats.On() { + inHeader := &stats.InHeader{ + FullMethod: s.method, + RemoteAddr: t.remoteAddr, + LocalAddr: t.localAddr, + Compression: s.recvCompress, + WireLength: int(frame.Header().Length), + } + stats.Handle(s.ctx, inHeader) + } handle(s) return } // HandleStreams receives incoming streams using the given handler. This is // typically run in a separate goroutine. -func (t *http2Server) HandleStreams(handle func(*Stream)) { +// traceCtx attaches trace to ctx and returns the new context. +func (t *http2Server) HandleStreams(handle func(*Stream), traceCtx func(context.Context, string) context.Context) { // Check the validity of client preface. preface := make([]byte, len(clientPreface)) if _, err := io.ReadFull(t.conn, preface); err != nil { @@ -279,7 +312,7 @@ func (t *http2Server) HandleStreams(handle func(*Stream)) { } switch frame := frame.(type) { case *http2.MetaHeadersFrame: - if t.operateHeaders(frame, handle) { + if t.operateHeaders(frame, handle, traceCtx) { t.Close() break } @@ -405,6 +438,9 @@ func (t *http2Server) handleSettings(f *http2.SettingsFrame) { } func (t *http2Server) handlePing(f *http2.PingFrame) { + if f.IsAck() { // Do nothing. + return + } pingAck := &ping{ack: true} copy(pingAck.data[:], f.Data[:]) t.controlBuf.put(pingAck) @@ -462,6 +498,14 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error { return ErrIllegalHeaderWrite } s.headerOk = true + if md.Len() > 0 { + if s.header.Len() > 0 { + s.header = metadata.Join(s.header, md) + } else { + s.header = md + } + } + md = s.header s.mu.Unlock() if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil { return err @@ -481,9 +525,16 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error { t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry}) } } + bufLen := t.hBuf.Len() if err := t.writeHeaders(s, t.hBuf, false); err != nil { return err } + if stats.On() { + outHeader := &stats.OutHeader{ + WireLength: bufLen, + } + stats.Handle(s.Context(), outHeader) + } t.writableChan <- 0 return nil } @@ -493,7 +544,7 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error { // TODO(zhaoq): Now it indicates the end of entire stream. Revisit if early // OK is adopted. func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc string) error { - var headersSent bool + var headersSent, hasHeader bool s.mu.Lock() if s.state == streamDone { s.mu.Unlock() @@ -502,7 +553,16 @@ func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc s if s.headerOk { headersSent = true } + if s.header.Len() > 0 { + hasHeader = true + } s.mu.Unlock() + + if !headersSent && hasHeader { + t.WriteHeader(s, nil) + headersSent = true + } + if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil { return err } @@ -527,10 +587,17 @@ func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc s t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry}) } } + bufLen := t.hBuf.Len() if err := t.writeHeaders(s, t.hBuf, true); err != nil { t.Close() return err } + if stats.On() { + outTrailer := &stats.OutTrailer{ + WireLength: bufLen, + } + stats.Handle(s.Context(), outTrailer) + } t.closeStream(s) t.writableChan <- 0 return nil @@ -548,29 +615,10 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error { } if !s.headerOk { writeHeaderFrame = true - s.headerOk = true } s.mu.Unlock() if writeHeaderFrame { - if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil { - return err - } - t.hBuf.Reset() - t.hEnc.WriteField(hpack.HeaderField{Name: ":status", Value: "200"}) - t.hEnc.WriteField(hpack.HeaderField{Name: "content-type", Value: "application/grpc"}) - if s.sendCompress != "" { - t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress}) - } - p := http2.HeadersFrameParam{ - StreamID: s.id, - BlockFragment: t.hBuf.Bytes(), - EndHeaders: true, - } - if err := t.framer.writeHeaders(false, p); err != nil { - t.Close() - return connectionErrorf(true, err, "transport: %v", err) - } - t.writableChan <- 0 + t.WriteHeader(s, nil) } r := bytes.NewBuffer(data) for { @@ -766,7 +814,7 @@ func (t *http2Server) closeStream(s *Stream) { } func (t *http2Server) RemoteAddr() net.Addr { - return t.conn.RemoteAddr() + return t.remoteAddr } func (t *http2Server) Drain() { diff --git a/vendor/google.golang.org/grpc/transport/http_util.go b/vendor/google.golang.org/grpc/transport/http_util.go index b024594e7f..a3c68d4cac 100644 --- a/vendor/google.golang.org/grpc/transport/http_util.go +++ b/vendor/google.golang.org/grpc/transport/http_util.go @@ -253,6 +253,9 @@ func div(d, r time.Duration) int64 { // TODO(zhaoq): It is the simplistic and not bandwidth efficient. Improve it. func encodeTimeout(t time.Duration) string { + if t <= 0 { + return "0n" + } if d := div(t, time.Nanosecond); d <= maxTimeoutValue { return strconv.FormatInt(d, 10) + "n" } @@ -349,7 +352,7 @@ func decodeGrpcMessageUnchecked(msg string) string { for i := 0; i < lenMsg; i++ { c := msg[i] if c == percentByte && i+2 < lenMsg { - parsed, err := strconv.ParseInt(msg[i+1:i+3], 16, 8) + parsed, err := strconv.ParseUint(msg[i+1:i+3], 16, 8) if err != nil { buf.WriteByte(c) } else { diff --git a/vendor/google.golang.org/grpc/transport/transport.go b/vendor/google.golang.org/grpc/transport/transport.go index f4d8dafae2..4726bb2cea 100644 --- a/vendor/google.golang.org/grpc/transport/transport.go +++ b/vendor/google.golang.org/grpc/transport/transport.go @@ -39,17 +39,16 @@ package transport // import "google.golang.org/grpc/transport" import ( "bytes" - "errors" "fmt" "io" "net" "sync" "golang.org/x/net/context" - "golang.org/x/net/trace" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/tap" ) // recvMsg represents the received msg from the transport. All transport @@ -168,8 +167,14 @@ type Stream struct { id uint32 // nil for client side Stream. st ServerTransport + // clientStatsCtx keeps the user context for stats handling. + // It's only valid on client side. Server side stats context is same as s.ctx. + // All client side stats collection should use the clientStatsCtx (instead of the stream context) + // so that all the generated stats for a particular RPC can be associated in the processing phase. + clientStatsCtx context.Context // ctx is the associated context of the stream. - ctx context.Context + ctx context.Context + // cancel is always nil for client side Stream. cancel context.CancelFunc // done is closed when the final status arrives. done chan struct{} @@ -266,11 +271,6 @@ func (s *Stream) Context() context.Context { return s.ctx } -// TraceContext recreates the context of s with a trace.Trace. -func (s *Stream) TraceContext(tr trace.Trace) { - s.ctx = trace.NewContext(s.ctx, tr) -} - // Method returns the method for the stream. func (s *Stream) Method() string { return s.method @@ -286,19 +286,30 @@ func (s *Stream) StatusDesc() string { return s.statusDesc } -// ErrIllegalTrailerSet indicates that the trailer has already been set or it -// is too late to do so. -var ErrIllegalTrailerSet = errors.New("transport: trailer has been set") +// SetHeader sets the header metadata. This can be called multiple times. +// Server side only. +func (s *Stream) SetHeader(md metadata.MD) error { + s.mu.Lock() + defer s.mu.Unlock() + if s.headerOk || s.state == streamDone { + return ErrIllegalHeaderWrite + } + if md.Len() == 0 { + return nil + } + s.header = metadata.Join(s.header, md) + return nil +} // SetTrailer sets the trailer metadata which will be sent with the RPC status -// by the server. This can only be called at most once. Server side only. +// by the server. This can be called multiple times. Server side only. func (s *Stream) SetTrailer(md metadata.MD) error { + if md.Len() == 0 { + return nil + } s.mu.Lock() defer s.mu.Unlock() - if s.trailer != nil { - return ErrIllegalTrailerSet - } - s.trailer = md.Copy() + s.trailer = metadata.Join(s.trailer, md) return nil } @@ -344,27 +355,42 @@ const ( draining ) +// ServerConfig consists of all the configurations to establish a server transport. +type ServerConfig struct { + MaxStreams uint32 + AuthInfo credentials.AuthInfo + InTapHandle tap.ServerInHandle +} + // NewServerTransport creates a ServerTransport with conn or non-nil error // if it fails. -func NewServerTransport(protocol string, conn net.Conn, maxStreams uint32, authInfo credentials.AuthInfo) (ServerTransport, error) { - return newHTTP2Server(conn, maxStreams, authInfo) +func NewServerTransport(protocol string, conn net.Conn, config *ServerConfig) (ServerTransport, error) { + return newHTTP2Server(conn, config) } -// ConnectOptions covers all relevant options for dialing a server. +// ConnectOptions covers all relevant options for communicating with the server. type ConnectOptions struct { // UserAgent is the application user agent. UserAgent string // Dialer specifies how to dial a network address. Dialer func(context.Context, string) (net.Conn, error) + // FailOnNonTempDialError specifies if gRPC fails on non-temporary dial errors. + FailOnNonTempDialError bool // PerRPCCredentials stores the PerRPCCredentials required to issue RPCs. PerRPCCredentials []credentials.PerRPCCredentials // TransportCredentials stores the Authenticator required to setup a client connection. TransportCredentials credentials.TransportCredentials } +// TargetInfo contains the information of the target such as network address and metadata. +type TargetInfo struct { + Addr string + Metadata interface{} +} + // NewClientTransport establishes the transport with the required ConnectOptions // and returns it to the caller. -func NewClientTransport(ctx context.Context, target string, opts ConnectOptions) (ClientTransport, error) { +func NewClientTransport(ctx context.Context, target TargetInfo, opts ConnectOptions) (ClientTransport, error) { return newHTTP2Client(ctx, target, opts) } @@ -449,7 +475,7 @@ type ClientTransport interface { // Write methods for a given Stream will be called serially. type ServerTransport interface { // HandleStreams receives incoming streams using the given handler. - HandleStreams(func(*Stream)) + HandleStreams(func(*Stream), func(context.Context, string) context.Context) // WriteHeader sends the header metadata for the given stream. // WriteHeader may not be called on all streams. diff --git a/vendor/manifest b/vendor/manifest index 8b68a3c9d4..6538d86aea 100644 --- a/vendor/manifest +++ b/vendor/manifest @@ -513,6 +513,14 @@ "path": "testdata", "notests": true }, + { + "importpath": "github.com/mwitkow/go-grpc-middleware", + "repository": "https://github.com/mwitkow/go-grpc-middleware", + "vcs": "git", + "revision": "e32d89015bae0f74206a52a292b9da2407f51954", + "branch": "master", + "notests": true + }, { "importpath": "github.com/onsi/ginkgo", "repository": "https://github.com/onsi/ginkgo", @@ -631,6 +639,15 @@ "path": "/require", "notests": true }, + { + "importpath": "github.com/stretchr/testify/suite", + "repository": "https://github.com/stretchr/testify", + "vcs": "git", + "revision": "18a02ba4a312f95da08ff4cfc0055750ce50ae9e", + "branch": "master", + "path": "/suite", + "notests": true + }, { "importpath": "github.com/stretchr/testify/vendor/github.com/davecgh/go-spew/spew", "repository": "https://github.com/stretchr/testify", @@ -982,7 +999,7 @@ "importpath": "google.golang.org/grpc", "repository": "https://github.com/grpc/grpc-go", "vcs": "git", - "revision": "0e6ec3a4501ee9ee2d023abe92e436fd04ed4081", + "revision": "b13ef794997191ec801db90a42c50034df91850a", "branch": "master", "notests": true },