Skip to content

Commit

Permalink
Define QueryResponse and QueryRequest protobufs. (#10956)
Browse files Browse the repository at this point in the history
**What this PR does / why we need it**:
This will be a follow up to #10688
finally switching over to protos. Hang on tight!

Everything is behind the feature flag `-frontend.encoding=protobuf` which will disable the transcoding to HTTP and HTTPgRPC.


**Checklist**
- [ ] Reviewed the
[`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md)
guide (**required**)
- [ ] Documentation added
- [x] Tests updated
- [x] `CHANGELOG.md` updated
- [ ] If the change is worth mentioning in the release notes, add
`add-to-release-notes` label
- [ ] Changes that require user attention or interaction to upgrade are
documented in `docs/sources/setup/upgrade/_index.md`
- [ ] For Helm chart changes bump the Helm chart version in
`production/helm/loki/Chart.yaml` and update
`production/helm/loki/CHANGELOG.md` and
`production/helm/loki/README.md`. [Example
PR](d10549e)

---------

Signed-off-by: Callum Styan <[email protected]>
Co-authored-by: Callum Styan <[email protected]>
Co-authored-by: Danny Kopping <[email protected]>
  • Loading branch information
3 people authored Nov 10, 2023
1 parent 54edb21 commit 05c4b77
Show file tree
Hide file tree
Showing 48 changed files with 3,074 additions and 468 deletions.
2 changes: 2 additions & 0 deletions .golangci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@ issues:
- Error return value of .*.Log.* is not checked
- Error return value of `` is not checked
exclude-rules:
- path: pkg/scheduler/scheduler.go
text: 'SA1019: msg.GetHttpRequest is deprecated: Do not use'
- path: '(.+)_test\.go'
linters:
- goconst
Expand Down
2 changes: 2 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
* [10733](https://github.com/grafana/loki/pull/10733) **shantanualsi** Add support for case-insensitive logql funtions
* [10727](https://github.com/grafana/loki/pull/10727) **sandeepsukhani** Native otlp ingestion support
* [11051](https://github.com/grafana/loki/pull/11051) Refactor to not use global logger in modules
* [10956](https://github.com/grafana/loki/pull/10956) **jeschkies** do not wrap requests but send pure Protobuf from frontend v2 via scheduler to querier when `-frontend.encoding=protobuf`.

##### Fixes
* [11074](https://github.com/grafana/loki/pull/11074) **hainenber** Fix panic in lambda-promtail due to mishandling of empty DROP_LABELS env var.

Expand Down
10 changes: 5 additions & 5 deletions docs/sources/configure/_index.md
Original file line number Diff line number Diff line change
Expand Up @@ -773,6 +773,11 @@ The `frontend` block configures the Loki query-frontend.
# CLI flag: -frontend.instance-interface-names
[instance_interface_names: <list of strings> | default = [<private network interfaces>]]
# Defines the encoding for requests to and responses from the scheduler and
# querier. Can be 'json' or 'protobuf' (defaults to 'json').
# CLI flag: -frontend.encoding
[encoding: <string> | default = "json"]
# Compress HTTP responses.
# CLI flag: -querier.compress-http-responses
[compress_responses: <boolean> | default = true]
Expand Down Expand Up @@ -822,11 +827,6 @@ results_cache:
# CLI flag: -querier.parallelise-shardable-queries
[parallelise_shardable_queries: <boolean> | default = true]
# The downstream querier is required to answer in the accepted format. Can be
# 'json' or 'protobuf'. Note: Both will still be routed over GRPC.
# CLI flag: -frontend.required-query-response-format
[required_query_response_format: <string> | default = "json"]
# Cache index stats query results.
# CLI flag: -querier.cache-index-stats-results
[cache_index_stats_results: <boolean> | default = false]
Expand Down
4 changes: 2 additions & 2 deletions integration/loki_micro_services_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func TestMicroServicesIngestQuery(t *testing.T) {
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.required-query-response-format=protobuf",
"-frontend.encoding=protobuf",
)
_ = clu.AddComponent(
"querier",
Expand Down Expand Up @@ -634,7 +634,7 @@ func TestOTLPLogsIngestQuery(t *testing.T) {
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.required-query-response-format=protobuf",
"-frontend.encoding=protobuf",
)
_ = clu.AddComponent(
"querier",
Expand Down
3 changes: 2 additions & 1 deletion pkg/logql/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/grafana/loki/pkg/util/constants"
"github.com/grafana/loki/pkg/util/httpreq"
logutil "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/server"
"github.com/grafana/loki/pkg/util/spanlogger"
"github.com/grafana/loki/pkg/util/validation"
)
Expand Down Expand Up @@ -242,7 +243,7 @@ func (q *query) Exec(ctx context.Context) (logqlmodel.Result, error) {
statResult := statsCtx.Result(time.Since(start), queueTime, q.resultLength(data))
statResult.Log(level.Debug(spLogger))

status := logqlmodel.MapStatusCode(err)
status, _ := server.ClientHTTPStatusAndError(err)

if q.record {
RecordRangeAndInstantQueryMetrics(ctx, q.logger, q.params, strconv.Itoa(status), statResult, data)
Expand Down
16 changes: 0 additions & 16 deletions pkg/logqlmodel/error.go
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
package logqlmodel

import (
"context"
"errors"
"fmt"

Expand Down Expand Up @@ -95,18 +94,3 @@ func NewSeriesLimitError(limit int) *LimitError {
func (e LimitError) Is(target error) bool {
return target == ErrLimit
}

func MapStatusCode(err error) int {
status := 200
if err != nil {
status = 500
if errors.Is(err, ErrParse) ||
errors.Is(err, ErrPipeline) ||
errors.Is(err, ErrLimit) ||
errors.Is(err, ErrBlocked) ||
errors.Is(err, context.Canceled) {
status = 400
}
}
return status
}
2 changes: 1 addition & 1 deletion pkg/loki/loki.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ type Frontend interface {
// Codec defines methods to encode and decode requests from HTTP, httpgrpc and Protobuf.
type Codec interface {
transport.Codec
worker.GRPCCodec
worker.RequestCodec
}

// Loki is the root datastructure for Loki.
Expand Down
2 changes: 2 additions & 0 deletions pkg/lokifrontend/frontend/transport/roundtripper.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ import (

"github.com/grafana/dskit/httpgrpc"

"github.com/grafana/loki/pkg/querier/queryrange"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
)

Expand All @@ -16,4 +17,5 @@ type GrpcRoundTripper interface {
type Codec interface {
queryrangebase.Codec
DecodeHTTPGrpcResponse(r *httpgrpc.HTTPResponse, req queryrangebase.Request) (queryrangebase.Response, error)
QueryRequestWrap(context.Context, queryrangebase.Request) (*queryrange.QueryRequest, error)
}
96 changes: 67 additions & 29 deletions pkg/lokifrontend/frontend/v2/frontend.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"github.com/grafana/dskit/netutil"
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/services"
"github.com/grafana/dskit/user"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
Expand All @@ -29,13 +28,19 @@ import (

"github.com/grafana/loki/pkg/lokifrontend/frontend/transport"
"github.com/grafana/loki/pkg/lokifrontend/frontend/v2/frontendv2pb"
"github.com/grafana/loki/pkg/querier/queryrange"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/querier/stats"
lokigrpc "github.com/grafana/loki/pkg/util/httpgrpc"
"github.com/grafana/loki/pkg/util/httpreq"
util_log "github.com/grafana/loki/pkg/util/log"
)

const (
EncodingJSON = "json"
EncodingProtobuf = "protobuf"
)

// Config for a Frontend.
type Config struct {
SchedulerAddress string `yaml:"scheduler_address"`
Expand All @@ -50,6 +55,9 @@ type Config struct {
// If set, address is not computed from interfaces.
Addr string `yaml:"address" doc:"hidden"`
Port int `doc:"hidden"`

// Defines the encoding for requests to and responses from the scheduler and querier.
Encoding string `yaml:"encoding"`
}

func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
Expand All @@ -64,6 +72,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
f.IntVar(&cfg.Port, "frontend.instance-port", 0, "Port to advertise to querier (via scheduler) (defaults to server.grpc-listen-port).")

cfg.GRPCClientConfig.RegisterFlagsWithPrefix("frontend.grpc-client-config", f)

f.StringVar(&cfg.Encoding, "frontend.encoding", "json", "Defines the encoding for requests to and responses from the scheduler and querier. Can be 'json' or 'protobuf' (defaults to 'json').")
}

// Frontend implements GrpcRoundTripper. It queues HTTP requests,
Expand All @@ -88,17 +98,23 @@ type Frontend struct {
var _ queryrangebase.Handler = &Frontend{}
var _ transport.GrpcRoundTripper = &Frontend{}

type ResponseTuple = struct {
*frontendv2pb.QueryResultRequest
error
}

type frontendRequest struct {
queryID uint64
request *httpgrpc.HTTPRequest
queryRequest *queryrange.QueryRequest
tenantID string
actor []string
statsEnabled bool

cancel context.CancelFunc

enqueue chan enqueueResult
response chan *frontendv2pb.QueryResultRequest
response chan ResponseTuple
}

type enqueueStatus int
Expand Down Expand Up @@ -228,7 +244,7 @@ func (f *Frontend) RoundTripGRPC(ctx context.Context, req *httpgrpc.HTTPRequest)
// Buffer of 1 to ensure response or error can be written to the channel
// even if this goroutine goes away due to client context cancellation.
enqueue: make(chan enqueueResult, 1),
response: make(chan *frontendv2pb.QueryResultRequest, 1),
response: make(chan ResponseTuple, 1),
}

cancelCh, err := f.enqueue(ctx, freq)
Expand All @@ -251,12 +267,17 @@ func (f *Frontend) RoundTripGRPC(ctx context.Context, req *httpgrpc.HTTPRequest)
return nil, ctx.Err()

case resp := <-freq.response:
if stats.ShouldTrackHTTPGRPCResponse(resp.HttpResponse) {
stats := stats.FromContext(ctx)
stats.Merge(resp.Stats) // Safe if stats is nil.
}
switch concrete := resp.Response.(type) {
case *frontendv2pb.QueryResultRequest_HttpResponse:
if stats.ShouldTrackHTTPGRPCResponse(concrete.HttpResponse) {
stats := stats.FromContext(ctx)
stats.Merge(resp.Stats) // Safe if stats is nil.
}

return resp.HttpResponse, nil
return concrete.HttpResponse, nil
default:
return nil, fmt.Errorf("unsupported response type for roundtrip: %T", resp.Response)
}
}
}

Expand All @@ -271,23 +292,8 @@ func (f *Frontend) Do(ctx context.Context, req queryrangebase.Request) (queryran
ctx, cancel := context.WithCancel(ctx)
defer cancel()

// For backwards comaptibility we are sending both encodings
httpReq, err := f.codec.EncodeRequest(ctx, req)
if err != nil {
return nil, fmt.Errorf("connot convert request to HTTP request: %w", err)
}

if err := user.InjectOrgIDIntoHTTPRequest(ctx, httpReq); err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
httpgrpcReq, err := server.HTTPRequest(httpReq)
if err != nil {
return nil, fmt.Errorf("connot convert HTTP request to gRPC request: %w", err)
}

freq := &frontendRequest{
queryID: f.lastQueryID.Inc(),
request: httpgrpcReq,
tenantID: tenantID,
actor: httpreq.ExtractActorPath(ctx),
statsEnabled: stats.IsEnabled(ctx),
Expand All @@ -297,7 +303,24 @@ func (f *Frontend) Do(ctx context.Context, req queryrangebase.Request) (queryran
// Buffer of 1 to ensure response or error can be written to the channel
// even if this goroutine goes away due to client context cancellation.
enqueue: make(chan enqueueResult, 1),
response: make(chan *frontendv2pb.QueryResultRequest, 1),
response: make(chan ResponseTuple, 1),
}

if f.cfg.Encoding == EncodingProtobuf {
freq.queryRequest, err = f.codec.QueryRequestWrap(ctx, req)
if err != nil {
return nil, fmt.Errorf("cannot wrap request: %w", err)
}
} else {
httpReq, err := f.codec.EncodeRequest(ctx, req)
if err != nil {
return nil, fmt.Errorf("cannot convert request to HTTP request: %w", err)
}

freq.request, err = server.HTTPRequest(httpReq)
if err != nil {
return nil, fmt.Errorf("cannot convert HTTP request to gRPC request: %w", err)
}
}

cancelCh, err := f.enqueue(ctx, freq)
Expand All @@ -320,12 +343,27 @@ func (f *Frontend) Do(ctx context.Context, req queryrangebase.Request) (queryran
return nil, ctx.Err()

case resp := <-freq.response:
if stats.ShouldTrackHTTPGRPCResponse(resp.HttpResponse) {
stats := stats.FromContext(ctx)
stats.Merge(resp.Stats) // Safe if stats is nil.
if resp.error != nil {
return nil, resp.error
}
switch concrete := resp.Response.(type) {
case *frontendv2pb.QueryResultRequest_HttpResponse:
if stats.ShouldTrackHTTPGRPCResponse(concrete.HttpResponse) {
stats := stats.FromContext(ctx)
stats.Merge(resp.Stats) // Safe if stats is nil.
}

return f.codec.DecodeHTTPGrpcResponse(resp.HttpResponse, req)
return f.codec.DecodeHTTPGrpcResponse(concrete.HttpResponse, req)
case *frontendv2pb.QueryResultRequest_QueryResponse:
if stats.ShouldTrackQueryResponse(concrete.QueryResponse.Status) {
stats := stats.FromContext(ctx)
stats.Merge(resp.Stats) // Safe if stats is nil.
}

return queryrange.QueryResponseUnwrap(concrete.QueryResponse)
default:
return nil, fmt.Errorf("unexpected frontend v2 response type: %T", concrete)
}
}
}

Expand Down Expand Up @@ -373,7 +411,7 @@ func (f *Frontend) QueryResult(ctx context.Context, qrReq *frontendv2pb.QueryRes
// To avoid mixing results from different queries, we randomize queryID counter on start.
if req != nil && req.tenantID == userID {
select {
case req.response <- qrReq:
case req.response <- ResponseTuple{qrReq, nil}:
// Should always be possible, unless QueryResult is called multiple times with the same queryID.
default:
level.Warn(f.log).Log("msg", "failed to write query result to the response channel", "queryID", qrReq.QueryID, "user", userID)
Expand Down
40 changes: 19 additions & 21 deletions pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"github.com/pkg/errors"
"google.golang.org/grpc"

"github.com/grafana/loki/pkg/lokifrontend/frontend/v2/frontendv2pb"
"github.com/grafana/loki/pkg/scheduler/schedulerpb"
"github.com/grafana/loki/pkg/util"
lokiutil "github.com/grafana/loki/pkg/util"
Expand Down Expand Up @@ -283,15 +282,25 @@ func (w *frontendSchedulerWorker) schedulerLoop(loop schedulerpb.SchedulerForFro
return nil

case req := <-w.requestCh:
err := loop.Send(&schedulerpb.FrontendToScheduler{
Type: schedulerpb.ENQUEUE,
QueryID: req.queryID,
UserID: req.tenantID,
QueuePath: req.actor,
HttpRequest: req.request,
msg := &schedulerpb.FrontendToScheduler{
Type: schedulerpb.ENQUEUE,
QueryID: req.queryID,
UserID: req.tenantID,
QueuePath: req.actor,
Request: &schedulerpb.FrontendToScheduler_HttpRequest{
HttpRequest: req.request,
},
FrontendAddress: w.frontendAddr,
StatsEnabled: req.statsEnabled,
})
}

if req.queryRequest != nil {
msg.Request = &schedulerpb.FrontendToScheduler_QueryRequest{
QueryRequest: req.queryRequest,
}
}

err := loop.Send(msg)
if err != nil {
req.enqueue <- enqueueResult{status: failed}
return err
Expand All @@ -315,21 +324,10 @@ func (w *frontendSchedulerWorker) schedulerLoop(loop schedulerpb.SchedulerForFro

case schedulerpb.ERROR:
req.enqueue <- enqueueResult{status: waitForResponse}
req.response <- &frontendv2pb.QueryResultRequest{
HttpResponse: &httpgrpc.HTTPResponse{
Code: http.StatusInternalServerError,
Body: []byte(resp.Error),
},
}

req.response <- ResponseTuple{nil, httpgrpc.Errorf(http.StatusInternalServerError, resp.Error)}
case schedulerpb.TOO_MANY_REQUESTS_PER_TENANT:
req.enqueue <- enqueueResult{status: waitForResponse}
req.response <- &frontendv2pb.QueryResultRequest{
HttpResponse: &httpgrpc.HTTPResponse{
Code: http.StatusTooManyRequests,
Body: []byte("too many outstanding requests"),
},
}
req.response <- ResponseTuple{nil, httpgrpc.Errorf(http.StatusTooManyRequests, "too many outstanding requests")}
default:
level.Error(w.log).Log("msg", "unknown response status from the scheduler", "status", resp.Status, "queryID", req.queryID)
req.enqueue <- enqueueResult{status: failed}
Expand Down
Loading

0 comments on commit 05c4b77

Please sign in to comment.