From fa0ff913e1e2daa9a5fd7ae8c00ac88cf4d17ea4 Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Sat, 14 Oct 2023 19:48:22 +0200 Subject: [PATCH 1/6] Get rid of httpgrpc on distributor's write path Signed-off-by: Yuri Nikolic --- pkg/distributor/distributor.go | 21 +- pkg/distributor/distributor_test.go | 302 ++++++++++------ pkg/distributor/errors.go | 46 ++- pkg/distributor/errors_test.go | 159 +++++---- pkg/distributor/push.go | 46 ++- pkg/distributor/push_test.go | 49 ++- pkg/mimirpb/mimir.pb.go | 520 ++++++++++++++++++++-------- pkg/mimirpb/mimir.proto | 12 + 8 files changed, 810 insertions(+), 345 deletions(-) diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index eaaf9d296a5..5791c0ee0c1 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -17,6 +17,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/gogo/status" "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/instrument" "github.com/grafana/dskit/kv" @@ -38,6 +39,7 @@ import ( "go.uber.org/atomic" "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" + "google.golang.org/grpc/codes" "github.com/grafana/mimir/pkg/cardinality" ingester_client "github.com/grafana/mimir/pkg/ingester/client" @@ -1089,15 +1091,24 @@ func (d *Distributor) handlePushError(ctx context.Context, pushErr error) error return pushErr } + if errors.Is(pushErr, context.DeadlineExceeded) { + return status.Error(codes.DeadlineExceeded, pushErr.Error()) + } + + // TODO This code is needed for backwards compatibility, since ingesters may still return + // errors created by httpgrpc.Errorf(). If pushErr is one of those errors, we just propagate + // it. This code should be removed once that creation is removed from the ingesters. + _, ok := httpgrpc.HTTPResponseFromError(pushErr) + if ok { + return pushErr + } + serviceOverloadErrorEnabled := false userID, err := tenant.TenantID(ctx) if err == nil { serviceOverloadErrorEnabled = d.limits.ServiceOverloadStatusCodeOnRateLimitEnabled(userID) } - if httpStatus, ok := toHTTPStatus(pushErr, serviceOverloadErrorEnabled); ok { - return httpgrpc.Errorf(httpStatus, pushErr.Error()) - } - return pushErr + return toGRPCError(pushErr, serviceOverloadErrorEnabled) } // push takes a write request and distributes it to ingesters using the ring. @@ -1191,7 +1202,7 @@ func (d *Distributor) push(ctx context.Context, pushReq *Request) error { err := d.send(localCtx, ingester, timeseries, metadata, req.Source) if errors.Is(err, context.DeadlineExceeded) { - return httpgrpc.Errorf(500, "exceeded configured distributor remote timeout: %s", err.Error()) + return errors.Wrap(err, deadlineExceededWrapMessage) } return err }, func() { pushReq.CleanUp(); cancel() }) diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index ff9eb1ae41a..6e0abad33e7 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/go-kit/log" + "github.com/gogo/status" "github.com/grafana/dskit/flagext" "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/kv" @@ -46,7 +47,6 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" "github.com/grafana/mimir/pkg/cardinality" "github.com/grafana/mimir/pkg/ingester" @@ -122,14 +122,16 @@ func TestDistributor_Push(t *testing.T) { startTimestampMs int64 } for name, tc := range map[string]struct { - metricNames []string - numIngesters int - happyIngesters int - samples samplesIn - metadata int - expectedError error - expectedMetrics string - timeOut bool + metricNames []string + numIngesters int + happyIngesters int + samples samplesIn + metadata int + expectedError error + expectedGRPCError *status.Status + expectedErrorDetails *mimirpb.WriteErrorDetails + expectedMetrics string + timeOut bool }{ "A push of no samples shouldn't block or return error, even if ingesters are sad": { numIngesters: 3, @@ -184,12 +186,13 @@ func TestDistributor_Push(t *testing.T) { `, }, "A push exceeding burst size should fail": { - numIngesters: 3, - happyIngesters: 3, - samples: samplesIn{num: 25, startTimestampMs: 123456789000}, - metadata: 5, - expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(20, 20).Error()), - metricNames: []string{lastSeenTimestamp}, + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 25, startTimestampMs: 123456789000}, + metadata: 5, + expectedGRPCError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(20, 20).Error()), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED}, + metricNames: []string{lastSeenTimestamp}, expectedMetrics: ` # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge @@ -275,13 +278,12 @@ func TestDistributor_Push(t *testing.T) { `, }, "A timed out push should fail": { - numIngesters: 3, - happyIngesters: 3, - samples: samplesIn{num: 10, startTimestampMs: 123456789000}, - timeOut: true, - expectedError: httpgrpc.Errorf(http.StatusInternalServerError, - "exceeded configured distributor remote timeout: failed pushing to ingester: context deadline exceeded"), - metricNames: []string{lastSeenTimestamp}, + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 10, startTimestampMs: 123456789000}, + timeOut: true, + expectedGRPCError: status.New(codes.DeadlineExceeded, "exceeded configured distributor remote timeout: failed pushing to ingester: context deadline exceeded"), + metricNames: []string{lastSeenTimestamp}, expectedMetrics: ` # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge @@ -306,16 +308,20 @@ func TestDistributor_Push(t *testing.T) { request := makeWriteRequest(tc.samples.startTimestampMs, tc.samples.num, tc.metadata, false, true) response, err := ds[0].Push(ctx, request) - if tc.expectedError == nil { + if tc.expectedError == nil && tc.expectedGRPCError == nil { require.NoError(t, err) assert.Equal(t, emptyResponse, response) } else { assert.Nil(t, response) - assert.EqualError(t, err, tc.expectedError.Error()) - // Assert that downstream gRPC statuses are passed back upstream - _, ok := httpgrpc.HTTPResponseFromError(err) - assert.True(t, ok, fmt.Sprintf("expected error to be an httpgrpc error, but got: %T", err)) + if tc.expectedGRPCError == nil { + // Assert that downstream gRPC statuses are passed back upstream + _, ok := httpgrpc.HTTPResponseFromError(err) + assert.True(t, ok, fmt.Sprintf("expected error to be an httpgrpc error, but got: %T", err)) + assert.EqualError(t, err, tc.expectedError.Error()) + } else { + checkGRPCError(t, tc.expectedGRPCError, tc.expectedErrorDetails, err) + } } // Check tracked Prometheus metrics. Since the Push() response is sent as soon as the quorum @@ -469,7 +475,7 @@ func TestDistributor_MetricsCleanup(t *testing.T) { func TestDistributor_PushRequestRateLimiter(t *testing.T) { type testPush struct { - expectedError error + expectedError *status.Status } ctx := user.InjectOrgID(context.Background(), "user") tests := map[string]struct { @@ -486,7 +492,7 @@ func TestDistributor_PushRequestRateLimiter(t *testing.T) { pushes: []testPush{ {expectedError: nil}, {expectedError: nil}, - {expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newRequestRateLimitedError(4, 2).Error())}, + {expectedError: status.New(codes.ResourceExhausted, newRequestRateLimitedError(4, 2).Error())}, }, }, "request limit is disabled when set to 0": { @@ -507,7 +513,7 @@ func TestDistributor_PushRequestRateLimiter(t *testing.T) { {expectedError: nil}, {expectedError: nil}, {expectedError: nil}, - {expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newRequestRateLimitedError(2, 3).Error())}, + {expectedError: status.New(codes.ResourceExhausted, newRequestRateLimitedError(2, 3).Error())}, }, }, "request limit is reached return 529 when enable service overload error set to true": { @@ -518,11 +524,13 @@ func TestDistributor_PushRequestRateLimiter(t *testing.T) { pushes: []testPush{ {expectedError: nil}, {expectedError: nil}, - {expectedError: httpgrpc.Errorf(StatusServiceOverloaded, newRequestRateLimitedError(4, 2).Error())}, + {expectedError: status.New(codes.Unavailable, newRequestRateLimitedError(4, 2).Error())}, }, }, } + expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED} + for testName, testData := range tests { testData := testData @@ -551,7 +559,7 @@ func TestDistributor_PushRequestRateLimiter(t *testing.T) { assert.Nil(t, err) } else { assert.Nil(t, response) - assert.EqualError(t, err, push.expectedError.Error()) + checkGRPCError(t, push.expectedError, expectedDetails, err) } } }) @@ -562,7 +570,7 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) { type testPush struct { samples int metadata int - expectedError error + expectedError *status.Status } ctx := user.InjectOrgID(context.Background(), "user") @@ -579,10 +587,10 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) { pushes: []testPush{ {samples: 2, expectedError: nil}, {samples: 1, expectedError: nil}, - {samples: 2, metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 5).Error())}, + {samples: 2, metadata: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 5).Error())}, {samples: 2, expectedError: nil}, - {samples: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 5).Error())}, - {metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 5).Error())}, + {samples: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 5).Error())}, + {metadata: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 5).Error())}, }, }, "for each distributor, set an ingestion burst limit.": { @@ -592,14 +600,16 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) { pushes: []testPush{ {samples: 10, expectedError: nil}, {samples: 5, expectedError: nil}, - {samples: 5, metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 20).Error())}, + {samples: 5, metadata: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 20).Error())}, {samples: 5, expectedError: nil}, - {samples: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 20).Error())}, - {metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, newIngestionRateLimitedError(10, 20).Error())}, + {samples: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 20).Error())}, + {metadata: 1, expectedError: status.New(codes.ResourceExhausted, newIngestionRateLimitedError(10, 20).Error())}, }, }, } + expectedErrorDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED} + for testName, testData := range tests { testData := testData @@ -627,7 +637,7 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) { assert.Nil(t, err) } else { assert.Nil(t, response) - assert.Equal(t, push.expectedError, err) + checkGRPCError(t, push.expectedError, expectedErrorDetails, err) } } }) @@ -798,12 +808,15 @@ func TestDistributor_PushInstanceLimits(t *testing.T) { for _, push := range testData.pushes { request := makeWriteRequest(0, push.samples, push.metadata, false, false) - _, err := d.Push(ctx, request) + resp, err := d.Push(ctx, request) if push.expectedError == nil { - assert.Nil(t, err) + assert.NoError(t, err) + assert.Equal(t, emptyResponse, resp) } else { - assert.ErrorIs(t, err, push.expectedError) + assert.Error(t, err) + assert.Nil(t, resp) + checkGRPCError(t, status.New(codes.Internal, push.expectedError.Error()), nil, err) } d.ingestionRate.Tick() @@ -831,7 +844,8 @@ func TestDistributor_PushHAInstances(t *testing.T) { cluster string samples int expectedResponse *mimirpb.WriteResponse - expectedCode int32 + expectedError *status.Status + expectedDetails *mimirpb.WriteErrorDetails }{ { enableTracker: true, @@ -848,7 +862,8 @@ func TestDistributor_PushHAInstances(t *testing.T) { testReplica: "instance0", cluster: "cluster0", samples: 5, - expectedCode: 202, + expectedError: status.New(codes.AlreadyExists, newReplicasDidNotMatchError("instance0", "instance2").Error()), + expectedDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH}, }, // If the HA tracker is disabled we should still accept samples that have both labels. { @@ -866,7 +881,8 @@ func TestDistributor_PushHAInstances(t *testing.T) { testReplica: "instance1234567890123456789012345678901234567890", cluster: "cluster0", samples: 5, - expectedCode: 400, + expectedError: status.New(codes.FailedPrecondition, fmt.Sprintf(labelValueTooLongMsgFormat, "instance1234567890123456789012345678901234567890", formatLabelSet(labelSetGenWithReplicaAndCluster("instance1234567890123456789012345678901234567890", "cluster0")(0)))), + expectedDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, }, } { t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -894,11 +910,8 @@ func TestDistributor_PushHAInstances(t *testing.T) { response, err := d.Push(ctx, request) assert.Equal(t, tc.expectedResponse, response) - httpResp, ok := httpgrpc.HTTPResponseFromError(err) - if ok { - assert.Equal(t, tc.expectedCode, httpResp.Code) - } else if tc.expectedCode != 0 { - assert.Fail(t, "expected HTTP status code", tc.expectedCode) + if tc.expectedError != nil { + checkGRPCError(t, tc.expectedError, tc.expectedDetails, err) } }) } @@ -1344,6 +1357,7 @@ func TestDistributor_Push_HistogramValidation(t *testing.T) { tests := map[string]struct { req *mimirpb.WriteRequest + expectedErr *status.Status errMsg string errID globalerror.ID bucketLimit int @@ -1352,17 +1366,15 @@ func TestDistributor_Push_HistogramValidation(t *testing.T) { req: makeWriteRequestHistogram([]string{model.MetricNameLabel, "test"}, 1000, generateTestHistogram(0)), }, "too new histogram": { - req: makeWriteRequestHistogram([]string{model.MetricNameLabel, "test"}, math.MaxInt64, generateTestHistogram(0)), - errMsg: "received a sample whose timestamp is too far in the future", - errID: globalerror.SampleTooFarInFuture, + req: makeWriteRequestHistogram([]string{model.MetricNameLabel, "test"}, math.MaxInt64, generateTestHistogram(0)), + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(sampleTimestampTooNewMsgFormat, math.MaxInt64, "test")), }, "valid float histogram": { req: makeWriteRequestFloatHistogram([]string{model.MetricNameLabel, "test"}, 1000, generateTestFloatHistogram(0)), }, "too new float histogram": { - req: makeWriteRequestFloatHistogram([]string{model.MetricNameLabel, "test"}, math.MaxInt64, generateTestFloatHistogram(0)), - errMsg: "received a sample whose timestamp is too far in the future", - errID: globalerror.SampleTooFarInFuture, + req: makeWriteRequestFloatHistogram([]string{model.MetricNameLabel, "test"}, math.MaxInt64, generateTestFloatHistogram(0)), + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(sampleTimestampTooNewMsgFormat, math.MaxInt64, "test")), }, "buckets at limit": { req: makeWriteRequestFloatHistogram([]string{model.MetricNameLabel, "test"}, 1000, testHistogram), @@ -1373,9 +1385,12 @@ func TestDistributor_Push_HistogramValidation(t *testing.T) { bucketLimit: 7, errMsg: "received a native histogram sample with too many buckets, timestamp", errID: globalerror.MaxNativeHistogramBuckets, + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(maxNativeHistogramBucketsMsgFormat, 1000, "{__name__=\"test\"}", 8, 7)), }, } + expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} + for testName, tc := range tests { t.Run(testName, func(t *testing.T) { limits := &validation.Limits{} @@ -1391,14 +1406,14 @@ func TestDistributor_Push_HistogramValidation(t *testing.T) { limits: limits, }) - _, err := ds[0].Push(ctx, tc.req) - if tc.errMsg != "" { - fromError, _ := status.FromError(err) - require.Equal(t, int32(400), fromError.Proto().Code) - assert.Contains(t, fromError.Message(), tc.errMsg) - assert.Contains(t, fromError.Message(), tc.errID) + resp, err := ds[0].Push(ctx, tc.req) + if tc.expectedErr == nil { + assert.NoError(t, err) + assert.Equal(t, emptyResponse, resp) } else { - assert.Nil(t, err) + assert.Error(t, err) + assert.Nil(t, resp) + checkGRPCError(t, tc.expectedErr, expectedDetails, err) } t.Cleanup(func() { @@ -2616,6 +2631,8 @@ func TestHaDedupeMiddleware(t *testing.T) { const replica2 = "replicaB" const cluster1 = "clusterA" const cluster2 = "clusterB" + replicasDidNotMatchDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH} + tooManyClusterDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.TOO_MANY_CLUSTERS} type testCase struct { name string @@ -2625,7 +2642,8 @@ func TestHaDedupeMiddleware(t *testing.T) { reqs []*mimirpb.WriteRequest expectedReqs []*mimirpb.WriteRequest expectedNextCalls int - expectErrs []int + expectErrs []*status.Status + expectDetails []*mimirpb.WriteErrorDetails } testCases := []testCase{ { @@ -2636,7 +2654,7 @@ func TestHaDedupeMiddleware(t *testing.T) { reqs: []*mimirpb.WriteRequest{{}}, expectedReqs: []*mimirpb.WriteRequest{{}}, expectedNextCalls: 1, - expectErrs: []int{0}, + expectErrs: []*status.Status{nil}, }, { name: "no changes if accept HA samples is false", ctx: ctxWithUser, @@ -2645,7 +2663,7 @@ func TestHaDedupeMiddleware(t *testing.T) { reqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithReplicaAndCluster(replica1, cluster1), nil, nil)}, expectedReqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithReplicaAndCluster(replica1, cluster1), nil, nil)}, expectedNextCalls: 1, - expectErrs: []int{0}, + expectErrs: []*status.Status{nil}, }, { name: "remove replica label with HA tracker disabled", ctx: ctxWithUser, @@ -2654,7 +2672,7 @@ func TestHaDedupeMiddleware(t *testing.T) { reqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithReplicaAndCluster(replica1, cluster1), nil, nil)}, expectedReqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithCluster(cluster1), nil, nil)}, expectedNextCalls: 1, - expectErrs: []int{0}, + expectErrs: []*status.Status{nil}, }, { name: "do nothing without user in context, don't even call next", ctx: context.Background(), @@ -2663,7 +2681,8 @@ func TestHaDedupeMiddleware(t *testing.T) { reqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithReplicaAndCluster(replica1, cluster1), nil, nil)}, expectedReqs: nil, expectedNextCalls: 0, - expectErrs: []int{-1}, // Special value because this is not an httpgrpc error. + expectErrs: []*status.Status{status.New(codes.Internal, "no org id")}, + expectDetails: []*mimirpb.WriteErrorDetails{nil}, }, { name: "perform HA deduplication", ctx: ctxWithUser, @@ -2675,7 +2694,8 @@ func TestHaDedupeMiddleware(t *testing.T) { }, expectedReqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithCluster(cluster1), nil, nil)}, expectedNextCalls: 1, - expectErrs: []int{0, 202}, + expectErrs: []*status.Status{nil, status.New(codes.AlreadyExists, newReplicasDidNotMatchError(replica2, replica1).Error())}, + expectDetails: []*mimirpb.WriteErrorDetails{nil, replicasDidNotMatchDetails}, }, { name: "exceed max ha clusters limit", ctx: ctxWithUser, @@ -2689,7 +2709,13 @@ func TestHaDedupeMiddleware(t *testing.T) { }, expectedReqs: []*mimirpb.WriteRequest{makeWriteRequestForGenerators(5, labelSetGenWithCluster(cluster1), nil, nil)}, expectedNextCalls: 1, - expectErrs: []int{0, 202, 400, 400}, + expectErrs: []*status.Status{ + nil, + status.New(codes.AlreadyExists, newReplicasDidNotMatchError(replica2, replica1).Error()), + status.New(codes.FailedPrecondition, newTooManyClustersError(1).Error()), + status.New(codes.FailedPrecondition, newTooManyClustersError(1).Error()), + }, + expectDetails: []*mimirpb.WriteErrorDetails{nil, replicasDidNotMatchDetails, tooManyClusterDetails, tooManyClusterDetails}, }, } @@ -2722,6 +2748,7 @@ func TestHaDedupeMiddleware(t *testing.T) { limits: &limits, enableTracker: tc.enableHaTracker, }) + middleware := ds[0].prePushHaDedupeMiddleware(next) var gotErrs []error @@ -2729,24 +2756,20 @@ func TestHaDedupeMiddleware(t *testing.T) { pushReq := NewParsedRequest(req) pushReq.AddCleanup(cleanup) err := middleware(tc.ctx, pushReq) - handledErr := ds[0].handlePushError(tc.ctx, err) + handledErr := err + if handledErr != nil { + handledErr = ds[0].handlePushError(tc.ctx, err) + } gotErrs = append(gotErrs, handledErr) } assert.Equal(t, tc.expectedReqs, gotReqs) assert.Len(t, gotErrs, len(tc.expectErrs)) for errIdx, expectErr := range tc.expectErrs { - if expectErr > 0 { - // Expect an httpgrpc error with specific status code. - resp, ok := httpgrpc.HTTPResponseFromError(gotErrs[errIdx]) - assert.True(t, ok) - assert.Equal(t, expectErr, int(resp.Code)) - } else if expectErr == 0 { - // Expect no error. - assert.Nil(t, gotErrs[errIdx]) + if expectErr == nil { + assert.NoError(t, gotErrs[errIdx]) } else { - // Expect an error which is not an httpgrpc error. - assert.NotNil(t, gotErrs[errIdx]) + checkGRPCError(t, expectErr, tc.expectDetails[errIdx], gotErrs[errIdx]) } } @@ -4049,14 +4072,14 @@ func TestDistributorValidation(t *testing.T) { ctx := user.InjectOrgID(context.Background(), "1") now := model.Now() future, past := now.Add(5*time.Hour), now.Add(-25*time.Hour) + expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} for name, tc := range map[string]struct { - metadata []*mimirpb.MetricMetadata - labels [][]mimirpb.LabelAdapter - samples []mimirpb.Sample - exemplars []*mimirpb.Exemplar - expectedStatusCode int32 - expectedErr string + metadata []*mimirpb.MetricMetadata + labels [][]mimirpb.LabelAdapter + samples []mimirpb.Sample + exemplars []*mimirpb.Exemplar + expectedErr *status.Status }{ "validation passes": { metadata: []*mimirpb.MetricMetadata{{MetricFamilyName: "testmetric", Help: "a test metric.", Unit: "", Type: mimirpb.COUNTER}}, @@ -4090,8 +4113,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(future), Value: 4, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: fmt.Sprintf(`received a sample whose timestamp is too far in the future, timestamp: %d series: 'testmetric' (err-mimir-too-far-in-future)`, future), + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(sampleTimestampTooNewMsgFormat, future, "testmetric")), }, "exceeds maximum labels per series": { @@ -4100,8 +4122,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(now), Value: 2, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: `received a series whose number of labels exceeds the limit (actual: 3, limit: 2) series: 'testmetric{foo2="bar2", foo="bar"}'`, + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(tooManyLabelsMsgFormat, 3, 2, `testmetric{foo2="bar2", foo="bar"}`, "")), }, "exceeds maximum labels per series with a metric that exceeds 200 characters when formatted": { labels: [][]mimirpb.LabelAdapter{{ @@ -4115,8 +4136,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(now), Value: 2, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: `received a series whose number of labels exceeds the limit (actual: 5, limit: 2) series: 'testmetric{foo-with-a-long-long-label="bar-with-a-long-long-value", foo2-with-a-long-long-label="bar2-with-a-long-long-value", foo3-with-a-long-long-label="bar3-with-a-long-long-value", foo4-with-a-lo…'`, + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(tooManyLabelsMsgFormat, 5, 2, `testmetric{foo-with-a-long-long-label="bar-with-a-long-long-value", foo2-with-a-long-long-label="bar2-with-a-long-long-value", foo3-with-a-long-long-label="bar3-with-a-long-long-value", foo4-with-a-lo`, "…")), }, "exceeds maximum labels per series with a metric that exceeds 200 bytes when formatted": { labels: [][]mimirpb.LabelAdapter{{ @@ -4128,8 +4148,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(now), Value: 2, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: `received a series whose number of labels exceeds the limit (actual: 3, limit: 2) series: 'testmetric{families="👩\u200d👦👨\u200d👧👨\u200d👩\u200d👧👩\u200d👧👩\u200d👩\u200d👦\u200d👦👨\u200d👩\u200d👧\u200d👦👨\u200d👧\u200d👦👨\u200d👩\u200d👦👪👨\u200d👦👨\u200d👦\u200d👦👨\u200d👨\u200d👧👨\u200d👧\u200d👧", foo="b"}'`, + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(tooManyLabelsMsgFormat, 3, 2, `testmetric{families="👩\u200d👦👨\u200d👧👨\u200d👩\u200d👧👩\u200d👧👩\u200d👩\u200d👦\u200d👦👨\u200d👩\u200d👧\u200d👦👨\u200d👧\u200d👦👨\u200d👩\u200d👦👪👨\u200d👦👨\u200d👦\u200d👦👨\u200d👨\u200d👧👨\u200d👧\u200d👧", foo="b"}`, "")), }, "multiple validation failures should return the first failure": { labels: [][]mimirpb.LabelAdapter{ @@ -4140,8 +4159,7 @@ func TestDistributorValidation(t *testing.T) { {TimestampMs: int64(now), Value: 2}, {TimestampMs: int64(past), Value: 2}, }, - expectedStatusCode: http.StatusBadRequest, - expectedErr: `received a series whose number of labels exceeds the limit (actual: 3, limit: 2) series: 'testmetric{foo2="bar2", foo="bar"}'`, + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(tooManyLabelsMsgFormat, 3, 2, `testmetric{foo2="bar2", foo="bar"}`, "")), }, "metadata validation failure": { metadata: []*mimirpb.MetricMetadata{{MetricFamilyName: "", Help: "a test metric.", Unit: "", Type: mimirpb.COUNTER}}, @@ -4150,8 +4168,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(now), Value: 1, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: `received a metric metadata with no metric name`, + expectedErr: status.New(codes.FailedPrecondition, metadataMetricNameMissingMsgFormat), }, "empty exemplar labels": { metadata: []*mimirpb.MetricMetadata{{MetricFamilyName: "testmetric", Help: "a test metric.", Unit: "", Type: mimirpb.COUNTER}}, @@ -4165,8 +4182,7 @@ func TestDistributorValidation(t *testing.T) { TimestampMs: int64(now), Value: 1, }}, - expectedStatusCode: http.StatusBadRequest, - expectedErr: fmt.Sprintf("received an exemplar with no valid labels, timestamp: %d series: %+v labels: {}", now, labels.FromStrings(labels.MetricName, "testmetric", "foo", "bar")), + expectedErr: status.New(codes.FailedPrecondition, fmt.Sprintf(exemplarEmptyLabelsMsgFormat, now, labels.FromStrings(labels.MetricName, "testmetric", "foo", "bar"), "{}")), }, } { t.Run(name, func(t *testing.T) { @@ -4184,14 +4200,14 @@ func TestDistributorValidation(t *testing.T) { limits: &limits, }) - _, err := ds[0].Push(ctx, mimirpb.ToWriteRequest(tc.labels, tc.samples, tc.exemplars, tc.metadata, mimirpb.API)) - if tc.expectedErr == "" { + resp, err := ds[0].Push(ctx, mimirpb.ToWriteRequest(tc.labels, tc.samples, tc.exemplars, tc.metadata, mimirpb.API)) + if tc.expectedErr == nil { require.NoError(t, err) + require.Equal(t, emptyResponse, resp) } else { - res, ok := httpgrpc.HTTPResponseFromError(err) - require.True(t, ok) - require.Equal(t, tc.expectedStatusCode, res.Code) - require.Contains(t, string(res.GetBody()), tc.expectedErr) + require.Error(t, err) + require.Nil(t, resp) + checkGRPCError(t, tc.expectedErr, expectedDetails, err) } }) } @@ -4692,7 +4708,7 @@ func TestDistributor_CleanupIsDoneAfterLastIngesterReturns(t *testing.T) { // First push request returned, but there's still an ingester call inflight. // This means that the push request is counted as inflight, so another incoming request should be rejected. _, err = distributors[0].Push(ctx, mockWriteRequest(labels.EmptyLabels(), 1, 1)) - assert.ErrorIs(t, err, errMaxInflightRequestsReached) + checkGRPCError(t, status.New(codes.Internal, errMaxInflightRequestsReached.Error()), nil, err) } func TestSeriesAreShardedToCorrectIngesters(t *testing.T) { @@ -4813,6 +4829,58 @@ func TestHandleIngesterPushError(t *testing.T) { } } +func TestHandlePushError(t *testing.T) { + testErrorMsg := "this is a test error message" + userID := "test" + errWithUserID := fmt.Errorf("user=%s: %s", userID, testErrorMsg) + httpGrpc4xxErr := httpgrpc.Errorf(http.StatusBadRequest, testErrorMsg) + httpGrpc5xxErr := httpgrpc.Errorf(http.StatusServiceUnavailable, testErrorMsg) + test := map[string]struct { + pushError error + expectedGRPCError *status.Status + expectedOtherError error + }{ + "a context.Canceled error gives a gRPC Canceled error": { + pushError: context.Canceled, + expectedOtherError: context.Canceled, + }, + "a context.DeadlineExceeded error gives a gRPC DeadlineExceeded error": { + pushError: context.DeadlineExceeded, + expectedGRPCError: status.New(codes.DeadlineExceeded, context.DeadlineExceeded.Error()), + }, + "a 4xx HTTP gRPC error gives the same 4xx HTTP gRPC error": { + pushError: httpGrpc4xxErr, + expectedOtherError: httpGrpc4xxErr, + }, + "a 5xx HTTP gRPC error gives the same 5xx HTTP gRPC error": { + pushError: httpGrpc5xxErr, + expectedOtherError: httpGrpc5xxErr, + }, + "a random ingester error without status gives an Internal gRPC error": { + pushError: errWithUserID, + expectedGRPCError: status.New(codes.Internal, errWithUserID.Error()), + }, + } + + config := prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + replicationFactor: 1, // push each series to single ingester only + } + d, _, _ := prepare(t, config) + ctx := context.Background() + + for _, testData := range test { + err := d[0].handlePushError(ctx, testData.pushError) + if testData.expectedGRPCError == nil { + require.Equal(t, testData.expectedOtherError, err) + } else { + checkGRPCError(t, testData.expectedGRPCError, nil, err) + } + } +} + func getIngesterIndexForToken(key uint32, ings []mockIngester) int { tokens := []uint32{} tokensMap := map[uint32]int{} @@ -4839,3 +4907,19 @@ func searchToken(tokens []uint32, key uint32) int { } return i } + +func checkGRPCError(t *testing.T, expectedStatus *status.Status, expectedDetails *mimirpb.WriteErrorDetails, err error) { + stat, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, expectedStatus.Code(), stat.Code()) + require.Equal(t, expectedStatus.Message(), stat.Message()) + if expectedDetails == nil { + require.Len(t, stat.Details(), 0) + } else { + details := stat.Details() + require.Len(t, details, 1) + errorDetails, ok := details[0].(*mimirpb.WriteErrorDetails) + require.True(t, ok) + require.Equal(t, expectedDetails, errorDetails) + } +} diff --git a/pkg/distributor/errors.go b/pkg/distributor/errors.go index 636eea0b982..41ca23f25d3 100644 --- a/pkg/distributor/errors.go +++ b/pkg/distributor/errors.go @@ -3,15 +3,21 @@ package distributor import ( + "errors" "fmt" + "github.com/gogo/status" + "google.golang.org/grpc/codes" + + "github.com/grafana/mimir/pkg/mimirpb" "github.com/grafana/mimir/pkg/util/globalerror" "github.com/grafana/mimir/pkg/util/validation" ) const ( // 529 is non-standard status code used by some services to signal that "The service is overloaded". - StatusServiceOverloaded = 529 + StatusServiceOverloaded = 529 + deadlineExceededWrapMessage = "exceeded configured distributor remote timeout" ) var ( @@ -111,3 +117,41 @@ func newRequestRateLimitedError(limit float64, burst int) requestRateLimitedErro func (e requestRateLimitedError) Error() string { return fmt.Sprintf(requestRateLimitedMsgFormat, e.limit, e.burst) } + +// toGRPCError converts the given error into an appropriate gRPC error. +func toGRPCError(pushErr error, serviceOverloadErrorEnabled bool) error { + var ( + errDetails *mimirpb.WriteErrorDetails + errCode = codes.Internal + ) + + switch { + case errors.As(pushErr, &validationError{}): + errCode = codes.FailedPrecondition + errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} + case errors.As(pushErr, &ingestionRateLimitedError{}): + errCode = codes.ResourceExhausted + errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED} + case errors.As(pushErr, &requestRateLimitedError{}): + if serviceOverloadErrorEnabled { + errCode = codes.Unavailable + } else { + errCode = codes.ResourceExhausted + } + errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED} + case errors.As(pushErr, &replicasDidNotMatchError{}): + errCode = codes.AlreadyExists + errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH} + case errors.As(pushErr, &tooManyClustersError{}): + errCode = codes.FailedPrecondition + errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.TOO_MANY_CLUSTERS} + } + stat := status.New(errCode, pushErr.Error()) + if errDetails != nil { + statWithDetails, err := stat.WithDetails(errDetails) + if err == nil { + return statWithDetails.Err() + } + } + return stat.Err() +} diff --git a/pkg/distributor/errors_test.go b/pkg/distributor/errors_test.go index b465116e6ac..673771c4908 100644 --- a/pkg/distributor/errors_test.go +++ b/pkg/distributor/errors_test.go @@ -4,13 +4,15 @@ package distributor import ( "fmt" - "net/http" "testing" + "github.com/gogo/status" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "github.com/grafana/mimir/pkg/mimirpb" "github.com/grafana/mimir/pkg/util/log" ) @@ -108,109 +110,136 @@ func TestNewRequestRateError(t *testing.T) { assert.True(t, errors.As(wrappedErr, &requestRateLimitedError{})) } -func TestToHTTPStatusHandler(t *testing.T) { +func TestToGRPCError(t *testing.T) { originalMsg := "this is an error" originalErr := errors.New(originalMsg) + replicasDidNotMatchErr := newReplicasDidNotMatchError("a", "b") + tooManyClustersErr := newTooManyClustersError(10) + ingestionRateLimitedErr := newIngestionRateLimitedError(10, 10) + requestRateLimitedErr := newRequestRateLimitedError(10, 10) testCases := []struct { name string err error serviceOverloadErrorEnabled bool - expectedHTTPStatus int - expectedOutcome bool + expectedGRPCCode codes.Code + expectedErrorMsg string + expectedErrorDetails *mimirpb.WriteErrorDetails }{ { - name: "a generic error gets translated into -1, false", - err: originalErr, - expectedHTTPStatus: -1, - expectedOutcome: false, + name: "a generic error gets translated into an Internal error with no details", + err: originalErr, + expectedGRPCCode: codes.Internal, + expectedErrorMsg: originalMsg, }, { - name: "a DoNotLog error of a generic error gets translated into a -1, false", - err: log.DoNotLogError{Err: originalErr}, - expectedHTTPStatus: -1, - expectedOutcome: false, + name: "a DoNotLog error of a generic error gets translated into an Internal error with no details", + err: log.DoNotLogError{Err: originalErr}, + expectedGRPCCode: codes.Internal, + expectedErrorMsg: originalMsg, }, { - name: "a replicasDidNotMatchError gets translated into 202, true", - err: newReplicasDidNotMatchError("a", "b"), - expectedHTTPStatus: http.StatusAccepted, - expectedOutcome: true, + name: "a replicasDidNotMatchError gets translated into an AlreadyExists error with REPLICASE_DID_NOT_MATCH cause", + err: replicasDidNotMatchErr, + expectedGRPCCode: codes.AlreadyExists, + expectedErrorMsg: replicasDidNotMatchErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH}, }, { - name: "a DoNotLog error of a replicasDidNotMatchError gets translated into 202, true", - err: log.DoNotLogError{Err: newReplicasDidNotMatchError("a", "b")}, - expectedHTTPStatus: http.StatusAccepted, - expectedOutcome: true, + name: "a DoNotLotError of a replicasDidNotMatchError gets translated into an AlreadyExists error with REPLICASE_DID_NOT_MATCH cause", + err: log.DoNotLogError{Err: replicasDidNotMatchErr}, + expectedGRPCCode: codes.AlreadyExists, + expectedErrorMsg: replicasDidNotMatchErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH}, }, { - name: "a tooManyClustersError gets translated into 400, true", - err: newTooManyClustersError(10), - expectedHTTPStatus: http.StatusBadRequest, - expectedOutcome: true, + name: "a tooManyClustersError gets translated into a FailedPrecondition error with TOO_MANY_CLUSTERS cause", + err: tooManyClustersErr, + expectedGRPCCode: codes.FailedPrecondition, + expectedErrorMsg: tooManyClustersErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.TOO_MANY_CLUSTERS}, }, { - name: "a DoNotLog error of a tooManyClustersError gets translated into 400, true", - err: log.DoNotLogError{Err: newTooManyClustersError(10)}, - expectedHTTPStatus: http.StatusBadRequest, - expectedOutcome: true, + name: "a DoNotLogError of a tooManyClustersError gets translated into a FailedPrecondition error with TOO_MANY_CLUSTERS cause", + err: log.DoNotLogError{Err: tooManyClustersErr}, + expectedGRPCCode: codes.FailedPrecondition, + expectedErrorMsg: tooManyClustersErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.TOO_MANY_CLUSTERS}, }, { - name: "a validationError gets translated into 400, true", - err: newValidationError(originalErr), - expectedHTTPStatus: http.StatusBadRequest, - expectedOutcome: true, + name: "a validationError gets translated into gets translated into a FailedPrecondition error with VALIDATION cause", + err: newValidationError(originalErr), + expectedGRPCCode: codes.FailedPrecondition, + expectedErrorMsg: originalMsg, + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, }, { - name: "a DoNotLog error of a validationError gets translated into 400, true", - err: log.DoNotLogError{Err: newValidationError(originalErr)}, - expectedHTTPStatus: http.StatusBadRequest, - expectedOutcome: true, + name: "a DoNotLogError of a validationError gets translated into gets translated into a FailedPrecondition error with VALIDATION cause", + err: log.DoNotLogError{Err: newValidationError(originalErr)}, + expectedGRPCCode: codes.FailedPrecondition, + expectedErrorMsg: originalMsg, + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, }, { - name: "an ingestionRateLimitedError gets translated into an HTTP 429", - err: newIngestionRateLimitedError(10, 10), - expectedHTTPStatus: http.StatusTooManyRequests, - expectedOutcome: true, + name: "an ingestionRateLimitedError gets translated into gets translated into a ResourceExhausted error with INGESTION_RATE_LIMITED cause", + err: ingestionRateLimitedErr, + expectedGRPCCode: codes.ResourceExhausted, + expectedErrorMsg: ingestionRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED}, }, { - name: "a DoNotLog error of an ingestionRateLimitedError gets translated into an HTTP 429", - err: log.DoNotLogError{Err: newIngestionRateLimitedError(10, 10)}, - expectedHTTPStatus: http.StatusTooManyRequests, - expectedOutcome: true, + name: "a DoNotLogError of an ingestionRateLimitedError gets translated into gets translated into a ResourceExhausted error with INGESTION_RATE_LIMITED cause", + err: log.DoNotLogError{Err: ingestionRateLimitedErr}, + expectedGRPCCode: codes.ResourceExhausted, + expectedErrorMsg: ingestionRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED}, }, { - name: "a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an HTTP 529", - err: newRequestRateLimitedError(10, 10), + name: "a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an Unavailable error with REQUEST_RATE_LIMITED cause", + err: requestRateLimitedErr, serviceOverloadErrorEnabled: true, - expectedHTTPStatus: StatusServiceOverloaded, - expectedOutcome: true, + expectedGRPCCode: codes.Unavailable, + expectedErrorMsg: requestRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED}, }, { - name: "a DoNotLog error of a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an HTTP 529", - err: log.DoNotLogError{Err: newRequestRateLimitedError(10, 10)}, + name: "a DoNotLogError of a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an Unavailable error with REQUEST_RATE_LIMITED cause", + err: log.DoNotLogError{Err: requestRateLimitedErr}, serviceOverloadErrorEnabled: true, - expectedHTTPStatus: StatusServiceOverloaded, - expectedOutcome: true, + expectedGRPCCode: codes.Unavailable, + expectedErrorMsg: requestRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED}, }, { - name: "a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an HTTP 429", - err: newRequestRateLimitedError(10, 10), - serviceOverloadErrorEnabled: false, - expectedHTTPStatus: http.StatusTooManyRequests, - expectedOutcome: true, + name: "a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an ResourceExhausted error with REQUEST_RATE_LIMITED cause", + err: requestRateLimitedErr, + expectedGRPCCode: codes.ResourceExhausted, + expectedErrorMsg: requestRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED}, }, { - name: "a DoNotLog error of a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an HTTP 429", - err: log.DoNotLogError{Err: newRequestRateLimitedError(10, 10)}, - serviceOverloadErrorEnabled: false, - expectedHTTPStatus: http.StatusTooManyRequests, - expectedOutcome: true, + name: "a DoNotLogError of a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an ResourceExhausted error with REQUEST_RATE_LIMITED cause", + err: log.DoNotLogError{Err: requestRateLimitedErr}, + expectedGRPCCode: codes.ResourceExhausted, + expectedErrorMsg: requestRateLimitedErr.Error(), + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED}, }, } for _, tc := range testCases { - httpStatus, outcome := toHTTPStatus(tc.err, tc.serviceOverloadErrorEnabled) - require.Equal(t, tc.expectedHTTPStatus, httpStatus) - require.Equal(t, tc.expectedOutcome, outcome) + err := toGRPCError(tc.err, tc.serviceOverloadErrorEnabled) + + stat, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, tc.expectedGRPCCode, stat.Code()) + require.Equal(t, tc.expectedErrorMsg, stat.Message()) + if tc.expectedErrorDetails == nil { + require.Len(t, stat.Details(), 0) + } else { + details := stat.Details() + require.Len(t, details, 1) + errDetails, ok := details[0].(*mimirpb.WriteErrorDetails) + require.True(t, ok) + require.Equal(t, tc.expectedErrorDetails, errDetails) + } } } diff --git a/pkg/distributor/push.go b/pkg/distributor/push.go index 0c344216f6f..3d568c00fee 100644 --- a/pkg/distributor/push.go +++ b/pkg/distributor/push.go @@ -135,7 +135,7 @@ func handler( if resp, ok := httpgrpc.HTTPResponseFromError(err); ok { code, msg = int(resp.Code), string(resp.Body) } else { - code, msg = distributorPushErrorHTTPStatus(ctx, err, limits), err.Error() + code, msg = toHTTPStatus(ctx, err, limits), err.Error() } if code != 202 { level.Error(logger).Log("msg", "push error", "err", err) @@ -145,44 +145,40 @@ func handler( }) } -func distributorPushErrorHTTPStatus(ctx context.Context, pushErr error, limits *validation.Overrides) int { - serviceOverloadErrorEnabled := false - userID, err := tenant.TenantID(ctx) - if err == nil { - serviceOverloadErrorEnabled = limits.ServiceOverloadStatusCodeOnRateLimitEnabled(userID) - } - if httpStatus, ok := toHTTPStatus(pushErr, serviceOverloadErrorEnabled); ok { - return httpStatus +// toHTTPStatus converts the given error into an appropriate HTTP status corresponding +// to that error, if the error is one of the errors from this package. Otherwise, an +// http.StatusInternalServerError is returned. +func toHTTPStatus(ctx context.Context, pushErr error, limits *validation.Overrides) int { + if errors.Is(pushErr, context.DeadlineExceeded) { + return http.StatusInternalServerError } - return http.StatusInternalServerError -} -// toHTTPStatus converts the given error into an appropriate HTTP status corresponding -// to that error, if the error is one of the errors from this package. In that case, -// the resulting HTTP status is returned with status true. Otherwise, -1 and the status -// false are returned. -func toHTTPStatus(pushErr error, serviceOverloadErrorEnabled bool) (int, bool) { switch { - case errors.As(pushErr, &replicasDidNotMatchError{}): - return http.StatusAccepted, true - case errors.As(pushErr, &tooManyClustersError{}): - return http.StatusBadRequest, true case errors.As(pushErr, &validationError{}): - return http.StatusBadRequest, true + return http.StatusBadRequest case errors.As(pushErr, &ingestionRateLimitedError{}): // Return a 429 here to tell the client it is going too fast. // Client may discard the data or slow down and re-send. // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. - return http.StatusTooManyRequests, true + return http.StatusTooManyRequests case errors.As(pushErr, &requestRateLimitedError{}): + serviceOverloadErrorEnabled := false + userID, err := tenant.TenantID(ctx) + if err == nil { + serviceOverloadErrorEnabled = limits.ServiceOverloadStatusCodeOnRateLimitEnabled(userID) + } // Return a 429 or a 529 here depending on configuration to tell the client it is going too fast. // Client may discard the data or slow down and re-send. // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. if serviceOverloadErrorEnabled { - return StatusServiceOverloaded, true + return StatusServiceOverloaded } - return http.StatusTooManyRequests, true + return http.StatusTooManyRequests + case errors.As(pushErr, &replicasDidNotMatchError{}): + return http.StatusAccepted + case errors.As(pushErr, &tooManyClustersError{}): + return http.StatusBadRequest default: - return -1, false + return http.StatusInternalServerError } } diff --git a/pkg/distributor/push_test.go b/pkg/distributor/push_test.go index 8fe190faaf5..386f5bf77c6 100644 --- a/pkg/distributor/push_test.go +++ b/pkg/distributor/push_test.go @@ -798,7 +798,7 @@ func TestHandler_ErrorTranslation(t *testing.T) { } } -func TestHandler_DistributorPushErrorHTTPStatus(t *testing.T) { +func TestHandler_ToHTTPStatus(t *testing.T) { userID := "user" originalMsg := "this is an error" originalErr := errors.New(originalMsg) @@ -819,33 +819,62 @@ func TestHandler_DistributorPushErrorHTTPStatus(t *testing.T) { expectedHTTPStatus: http.StatusInternalServerError, }, { - name: "a DoNotLog error gets translated into a HTTP 500", + name: "a DoNotLog of a generic error gets translated into a HTTP 500", err: log.DoNotLogError{Err: originalErr}, expectedHTTPStatus: http.StatusInternalServerError, }, + { + name: "a context.DeadlineExceeded gets translated into a HTTP 500", + err: context.DeadlineExceeded, + expectedHTTPStatus: http.StatusInternalServerError, + expectedErrorMsg: context.DeadlineExceeded.Error(), + }, { name: "a replicasDidNotMatchError gets translated into an HTTP 202", err: replicasNotMatchErr, expectedHTTPStatus: http.StatusAccepted, expectedErrorMsg: replicasNotMatchErr.Error(), }, + { + name: "a DoNotLogError of a replicasDidNotMatchError gets translated into an HTTP 202", + err: log.DoNotLogError{Err: replicasNotMatchErr}, + expectedHTTPStatus: http.StatusAccepted, + expectedErrorMsg: replicasNotMatchErr.Error(), + }, { name: "a tooManyClustersError gets translated into an HTTP 400", err: tooManyClustersErr, expectedHTTPStatus: http.StatusBadRequest, expectedErrorMsg: tooManyClustersErr.Error(), }, + { + name: "a DoNotLogError of a tooManyClustersError gets translated into an HTTP 400", + err: log.DoNotLogError{Err: tooManyClustersErr}, + expectedHTTPStatus: http.StatusBadRequest, + expectedErrorMsg: tooManyClustersErr.Error(), + }, { name: "a validationError gets translated into an HTTP 400", err: newValidationError(originalErr), expectedHTTPStatus: http.StatusBadRequest, }, + { + name: "a DoNotLogError of a validationError gets translated into an HTTP 400", + err: log.DoNotLogError{Err: newValidationError(originalErr)}, + expectedHTTPStatus: http.StatusBadRequest, + }, { name: "an ingestionRateLimitedError gets translated into an HTTP 429", err: ingestionRateLimitedErr, expectedHTTPStatus: http.StatusTooManyRequests, expectedErrorMsg: ingestionRateLimitedErr.Error(), }, + { + name: "a DoNotLogError of an ingestionRateLimitedError gets translated into an HTTP 429", + err: log.DoNotLogError{Err: ingestionRateLimitedErr}, + expectedHTTPStatus: http.StatusTooManyRequests, + expectedErrorMsg: ingestionRateLimitedErr.Error(), + }, { name: "a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an HTTP 529", err: requestRateLimitedErr, @@ -853,6 +882,13 @@ func TestHandler_DistributorPushErrorHTTPStatus(t *testing.T) { expectedHTTPStatus: StatusServiceOverloaded, expectedErrorMsg: requestRateLimitedErr.Error(), }, + { + name: "a DoNotLogError of a requestRateLimitedError with serviceOverloadErrorEnabled gets translated into an HTTP 529", + err: log.DoNotLogError{Err: requestRateLimitedErr}, + serviceOverloadErrorEnabled: true, + expectedHTTPStatus: StatusServiceOverloaded, + expectedErrorMsg: requestRateLimitedErr.Error(), + }, { name: "a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an HTTP 429", err: requestRateLimitedErr, @@ -860,6 +896,13 @@ func TestHandler_DistributorPushErrorHTTPStatus(t *testing.T) { expectedHTTPStatus: http.StatusTooManyRequests, expectedErrorMsg: requestRateLimitedErr.Error(), }, + { + name: "a DoNotLogError of a requestRateLimitedError without serviceOverloadErrorEnabled gets translated into an HTTP 429", + err: log.DoNotLogError{Err: requestRateLimitedErr}, + serviceOverloadErrorEnabled: false, + expectedHTTPStatus: http.StatusTooManyRequests, + expectedErrorMsg: requestRateLimitedErr.Error(), + }, } for _, tc := range testCases { @@ -876,7 +919,7 @@ func TestHandler_DistributorPushErrorHTTPStatus(t *testing.T) { ) require.NoError(t, err) - status := distributorPushErrorHTTPStatus(ctx, tc.err, limits) + status := toHTTPStatus(ctx, tc.err, limits) msg := tc.err.Error() assert.Equal(t, tc.expectedHTTPStatus, status) expectedErrMsg := tc.expectedErrorMsg diff --git a/pkg/mimirpb/mimir.pb.go b/pkg/mimirpb/mimir.pb.go index 9454abf39db..6ef3ea63e6f 100644 --- a/pkg/mimirpb/mimir.pb.go +++ b/pkg/mimirpb/mimir.pb.go @@ -29,6 +29,36 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type ErrorCause int32 + +const ( + REPLICAS_DID_NOT_MATCH ErrorCause = 0 + TOO_MANY_CLUSTERS ErrorCause = 1 + VALIDATION ErrorCause = 2 + INGESTION_RATE_LIMITED ErrorCause = 3 + REQUEST_RATE_LIMITED ErrorCause = 4 +) + +var ErrorCause_name = map[int32]string{ + 0: "REPLICAS_DID_NOT_MATCH", + 1: "TOO_MANY_CLUSTERS", + 2: "VALIDATION", + 3: "INGESTION_RATE_LIMITED", + 4: "REQUEST_RATE_LIMITED", +} + +var ErrorCause_value = map[string]int32{ + "REPLICAS_DID_NOT_MATCH": 0, + "TOO_MANY_CLUSTERS": 1, + "VALIDATION": 2, + "INGESTION_RATE_LIMITED": 3, + "REQUEST_RATE_LIMITED": 4, +} + +func (ErrorCause) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{0} +} + type WriteRequest_SourceEnum int32 const ( @@ -86,7 +116,7 @@ var MetricMetadata_MetricType_value = map[string]int32{ } func (MetricMetadata_MetricType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{5, 0} + return fileDescriptor_86d4d7485f544059, []int{6, 0} } type Histogram_ResetHint int32 @@ -113,7 +143,7 @@ var Histogram_ResetHint_value = map[string]int32{ } func (Histogram_ResetHint) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{8, 0} + return fileDescriptor_86d4d7485f544059, []int{9, 0} } // These values correspond to the possible status values defined in https://github.com/prometheus/prometheus/blob/main/web/api/v1/api.go. @@ -135,7 +165,7 @@ var QueryResponse_Status_value = map[string]int32{ } func (QueryResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{15, 0} + return fileDescriptor_86d4d7485f544059, []int{16, 0} } // These values correspond to the possible error type values defined in https://github.com/prometheus/prometheus/blob/main/web/api/v1/api.go. @@ -178,7 +208,7 @@ var QueryResponse_ErrorType_value = map[string]int32{ } func (QueryResponse_ErrorType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{15, 1} + return fileDescriptor_86d4d7485f544059, []int{16, 1} } type WriteRequest struct { @@ -277,6 +307,49 @@ func (m *WriteResponse) XXX_DiscardUnknown() { var xxx_messageInfo_WriteResponse proto.InternalMessageInfo +type WriteErrorDetails struct { + Cause ErrorCause `protobuf:"varint,1,opt,name=Cause,proto3,enum=cortexpb.ErrorCause" json:"Cause,omitempty"` +} + +func (m *WriteErrorDetails) Reset() { *m = WriteErrorDetails{} } +func (*WriteErrorDetails) ProtoMessage() {} +func (*WriteErrorDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{2} +} +func (m *WriteErrorDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteErrorDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_WriteErrorDetails.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *WriteErrorDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteErrorDetails.Merge(m, src) +} +func (m *WriteErrorDetails) XXX_Size() int { + return m.Size() +} +func (m *WriteErrorDetails) XXX_DiscardUnknown() { + xxx_messageInfo_WriteErrorDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteErrorDetails proto.InternalMessageInfo + +func (m *WriteErrorDetails) GetCause() ErrorCause { + if m != nil { + return m.Cause + } + return REPLICAS_DID_NOT_MATCH +} + type TimeSeries struct { Labels []LabelAdapter `protobuf:"bytes,1,rep,name=labels,proto3,customtype=LabelAdapter" json:"labels"` // Sorted by time, oldest sample first. @@ -288,7 +361,7 @@ type TimeSeries struct { func (m *TimeSeries) Reset() { *m = TimeSeries{} } func (*TimeSeries) ProtoMessage() {} func (*TimeSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{2} + return fileDescriptor_86d4d7485f544059, []int{3} } func (m *TimeSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +419,7 @@ type LabelPair struct { func (m *LabelPair) Reset() { *m = LabelPair{} } func (*LabelPair) ProtoMessage() {} func (*LabelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{3} + return fileDescriptor_86d4d7485f544059, []int{4} } func (m *LabelPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -398,7 +471,7 @@ type Sample struct { func (m *Sample) Reset() { *m = Sample{} } func (*Sample) ProtoMessage() {} func (*Sample) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{4} + return fileDescriptor_86d4d7485f544059, []int{5} } func (m *Sample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -451,7 +524,7 @@ type MetricMetadata struct { func (m *MetricMetadata) Reset() { *m = MetricMetadata{} } func (*MetricMetadata) ProtoMessage() {} func (*MetricMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{5} + return fileDescriptor_86d4d7485f544059, []int{6} } func (m *MetricMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -515,7 +588,7 @@ type Metric struct { func (m *Metric) Reset() { *m = Metric{} } func (*Metric) ProtoMessage() {} func (*Metric) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{6} + return fileDescriptor_86d4d7485f544059, []int{7} } func (m *Metric) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -554,7 +627,7 @@ type Exemplar struct { func (m *Exemplar) Reset() { *m = Exemplar{} } func (*Exemplar) ProtoMessage() {} func (*Exemplar) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{7} + return fileDescriptor_86d4d7485f544059, []int{8} } func (m *Exemplar) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -641,7 +714,7 @@ type Histogram struct { func (m *Histogram) Reset() { *m = Histogram{} } func (*Histogram) ProtoMessage() {} func (*Histogram) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{8} + return fileDescriptor_86d4d7485f544059, []int{9} } func (m *Histogram) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -848,7 +921,7 @@ type FloatHistogram struct { func (m *FloatHistogram) Reset() { *m = FloatHistogram{} } func (*FloatHistogram) ProtoMessage() {} func (*FloatHistogram) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{9} + return fileDescriptor_86d4d7485f544059, []int{10} } func (m *FloatHistogram) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -964,7 +1037,7 @@ type BucketSpan struct { func (m *BucketSpan) Reset() { *m = BucketSpan{} } func (*BucketSpan) ProtoMessage() {} func (*BucketSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{10} + return fileDescriptor_86d4d7485f544059, []int{11} } func (m *BucketSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1016,7 +1089,7 @@ type FloatHistogramPair struct { func (m *FloatHistogramPair) Reset() { *m = FloatHistogramPair{} } func (*FloatHistogramPair) ProtoMessage() {} func (*FloatHistogramPair) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{11} + return fileDescriptor_86d4d7485f544059, []int{12} } func (m *FloatHistogramPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1071,7 +1144,7 @@ type SampleHistogram struct { func (m *SampleHistogram) Reset() { *m = SampleHistogram{} } func (*SampleHistogram) ProtoMessage() {} func (*SampleHistogram) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{12} + return fileDescriptor_86d4d7485f544059, []int{13} } func (m *SampleHistogram) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1132,7 +1205,7 @@ type HistogramBucket struct { func (m *HistogramBucket) Reset() { *m = HistogramBucket{} } func (*HistogramBucket) ProtoMessage() {} func (*HistogramBucket) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{13} + return fileDescriptor_86d4d7485f544059, []int{14} } func (m *HistogramBucket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1198,7 +1271,7 @@ type SampleHistogramPair struct { func (m *SampleHistogramPair) Reset() { *m = SampleHistogramPair{} } func (*SampleHistogramPair) ProtoMessage() {} func (*SampleHistogramPair) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{14} + return fileDescriptor_86d4d7485f544059, []int{15} } func (m *SampleHistogramPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1257,7 +1330,7 @@ type QueryResponse struct { func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{15} + return fileDescriptor_86d4d7485f544059, []int{16} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1385,7 +1458,7 @@ type StringData struct { func (m *StringData) Reset() { *m = StringData{} } func (*StringData) ProtoMessage() {} func (*StringData) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{16} + return fileDescriptor_86d4d7485f544059, []int{17} } func (m *StringData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1436,7 +1509,7 @@ type VectorData struct { func (m *VectorData) Reset() { *m = VectorData{} } func (*VectorData) ProtoMessage() {} func (*VectorData) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{17} + return fileDescriptor_86d4d7485f544059, []int{18} } func (m *VectorData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1489,7 +1562,7 @@ type VectorSample struct { func (m *VectorSample) Reset() { *m = VectorSample{} } func (*VectorSample) ProtoMessage() {} func (*VectorSample) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{18} + return fileDescriptor_86d4d7485f544059, []int{19} } func (m *VectorSample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1549,7 +1622,7 @@ type VectorHistogram struct { func (m *VectorHistogram) Reset() { *m = VectorHistogram{} } func (*VectorHistogram) ProtoMessage() {} func (*VectorHistogram) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{19} + return fileDescriptor_86d4d7485f544059, []int{20} } func (m *VectorHistogram) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1607,7 +1680,7 @@ type ScalarData struct { func (m *ScalarData) Reset() { *m = ScalarData{} } func (*ScalarData) ProtoMessage() {} func (*ScalarData) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{20} + return fileDescriptor_86d4d7485f544059, []int{21} } func (m *ScalarData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1657,7 +1730,7 @@ type MatrixData struct { func (m *MatrixData) Reset() { *m = MatrixData{} } func (*MatrixData) ProtoMessage() {} func (*MatrixData) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{21} + return fileDescriptor_86d4d7485f544059, []int{22} } func (m *MatrixData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1703,7 +1776,7 @@ type MatrixSeries struct { func (m *MatrixSeries) Reset() { *m = MatrixSeries{} } func (*MatrixSeries) ProtoMessage() {} func (*MatrixSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_86d4d7485f544059, []int{22} + return fileDescriptor_86d4d7485f544059, []int{23} } func (m *MatrixSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1754,6 +1827,7 @@ func (m *MatrixSeries) GetHistograms() []FloatHistogramPair { } func init() { + proto.RegisterEnum("cortexpb.ErrorCause", ErrorCause_name, ErrorCause_value) proto.RegisterEnum("cortexpb.WriteRequest_SourceEnum", WriteRequest_SourceEnum_name, WriteRequest_SourceEnum_value) proto.RegisterEnum("cortexpb.MetricMetadata_MetricType", MetricMetadata_MetricType_name, MetricMetadata_MetricType_value) proto.RegisterEnum("cortexpb.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) @@ -1761,6 +1835,7 @@ func init() { proto.RegisterEnum("cortexpb.QueryResponse_ErrorType", QueryResponse_ErrorType_name, QueryResponse_ErrorType_value) proto.RegisterType((*WriteRequest)(nil), "cortexpb.WriteRequest") proto.RegisterType((*WriteResponse)(nil), "cortexpb.WriteResponse") + proto.RegisterType((*WriteErrorDetails)(nil), "cortexpb.WriteErrorDetails") proto.RegisterType((*TimeSeries)(nil), "cortexpb.TimeSeries") proto.RegisterType((*LabelPair)(nil), "cortexpb.LabelPair") proto.RegisterType((*Sample)(nil), "cortexpb.Sample") @@ -1787,119 +1862,134 @@ func init() { func init() { proto.RegisterFile("mimir.proto", fileDescriptor_86d4d7485f544059) } var fileDescriptor_86d4d7485f544059 = []byte{ - // 1758 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcf, 0x73, 0x1b, 0x49, - 0x15, 0x56, 0x4b, 0x63, 0x49, 0xf3, 0x2c, 0xc9, 0xb3, 0xbd, 0xa9, 0xa0, 0x4d, 0x6d, 0x64, 0x67, - 0x28, 0x16, 0x43, 0x81, 0x42, 0x65, 0x21, 0x5b, 0xbb, 0x15, 0x0a, 0x46, 0xf2, 0x24, 0xb6, 0xd7, - 0x96, 0x4c, 0x4b, 0xca, 0xb2, 0x5c, 0x54, 0x23, 0xb9, 0x6d, 0x4d, 0xed, 0x8c, 0x66, 0x98, 0x19, - 0x85, 0x98, 0x13, 0x17, 0x28, 0x8a, 0x13, 0x17, 0x2e, 0x14, 0x37, 0x0e, 0xf0, 0x17, 0xf0, 0x37, - 0xa4, 0x8a, 0xa2, 0x2a, 0xc7, 0x2d, 0x0e, 0x29, 0xe2, 0x5c, 0xf6, 0x98, 0x03, 0x27, 0x4e, 0x54, - 0xbf, 0x9e, 0x1f, 0x1a, 0xd9, 0x86, 0xc0, 0xfa, 0x36, 0xfd, 0xfa, 0x7b, 0xaf, 0xbf, 0x7e, 0xfd, - 0xf5, 0xd3, 0x6b, 0xc1, 0xba, 0x6b, 0xbb, 0x76, 0xd0, 0xf6, 0x03, 0x2f, 0xf2, 0x68, 0x75, 0xea, - 0x05, 0x11, 0x7f, 0xea, 0x4f, 0x6e, 0x7d, 0xfb, 0xd4, 0x8e, 0x66, 0x8b, 0x49, 0x7b, 0xea, 0xb9, - 0x77, 0x4f, 0xbd, 0x53, 0xef, 0x2e, 0x02, 0x26, 0x8b, 0x13, 0x1c, 0xe1, 0x00, 0xbf, 0xa4, 0xa3, - 0xfe, 0x97, 0x22, 0xd4, 0x3e, 0x09, 0xec, 0x88, 0x33, 0xfe, 0xd3, 0x05, 0x0f, 0x23, 0x7a, 0x04, - 0x10, 0xd9, 0x2e, 0x0f, 0x79, 0x60, 0xf3, 0xb0, 0x49, 0xb6, 0x4a, 0xdb, 0xeb, 0xf7, 0x6e, 0xb4, - 0x93, 0xf0, 0xed, 0xa1, 0xed, 0xf2, 0x01, 0xce, 0x75, 0x6e, 0x3d, 0x7b, 0xb1, 0x59, 0xf8, 0xfb, - 0x8b, 0x4d, 0x7a, 0x14, 0x70, 0xcb, 0x71, 0xbc, 0xe9, 0x30, 0xf5, 0x63, 0x4b, 0x31, 0xe8, 0x87, - 0x50, 0x1e, 0x78, 0x8b, 0x60, 0xca, 0x9b, 0xc5, 0x2d, 0xb2, 0xdd, 0xb8, 0x77, 0x27, 0x8b, 0xb6, - 0xbc, 0x72, 0x5b, 0x82, 0xcc, 0xf9, 0xc2, 0x65, 0xb1, 0x03, 0xfd, 0x08, 0xaa, 0x2e, 0x8f, 0xac, - 0x63, 0x2b, 0xb2, 0x9a, 0x25, 0xa4, 0xd2, 0xcc, 0x9c, 0x0f, 0x79, 0x14, 0xd8, 0xd3, 0xc3, 0x78, - 0xbe, 0xa3, 0x3c, 0x7b, 0xb1, 0x49, 0x58, 0x8a, 0xa7, 0x0f, 0xe0, 0x56, 0xf8, 0x99, 0xed, 0x8f, - 0x1d, 0x6b, 0xc2, 0x9d, 0xf1, 0xdc, 0x72, 0xf9, 0xf8, 0x89, 0xe5, 0xd8, 0xc7, 0x56, 0x64, 0x7b, - 0xf3, 0xe6, 0x17, 0x95, 0x2d, 0xb2, 0x5d, 0x65, 0x5f, 0x11, 0x90, 0x03, 0x81, 0xe8, 0x59, 0x2e, - 0x7f, 0x9c, 0xce, 0xeb, 0x9b, 0x00, 0x19, 0x1f, 0x5a, 0x81, 0x92, 0x71, 0xb4, 0xa7, 0x15, 0x68, - 0x15, 0x14, 0x36, 0x3a, 0x30, 0x35, 0xa2, 0x6f, 0x40, 0x3d, 0x66, 0x1f, 0xfa, 0xde, 0x3c, 0xe4, - 0xfa, 0x3f, 0x09, 0x40, 0x96, 0x1d, 0x6a, 0x40, 0x19, 0x57, 0x4e, 0x72, 0xf8, 0x76, 0x46, 0x1c, - 0xd7, 0x3b, 0xb2, 0xec, 0xa0, 0x73, 0x23, 0x4e, 0x61, 0x0d, 0x4d, 0xc6, 0xb1, 0xe5, 0x47, 0x3c, - 0x60, 0xb1, 0x23, 0xfd, 0x0e, 0x54, 0x42, 0xcb, 0xf5, 0x1d, 0x1e, 0x36, 0x8b, 0x18, 0x43, 0xcb, - 0x62, 0x0c, 0x70, 0x02, 0x37, 0x5d, 0x60, 0x09, 0x8c, 0xde, 0x07, 0x95, 0x3f, 0xe5, 0xae, 0xef, - 0x58, 0x41, 0x18, 0x27, 0x8c, 0x66, 0x3e, 0x66, 0x3c, 0x15, 0x7b, 0x65, 0x50, 0xfa, 0x21, 0xc0, - 0xcc, 0x0e, 0x23, 0xef, 0x34, 0xb0, 0xdc, 0xb0, 0xa9, 0xac, 0x12, 0xde, 0x4d, 0xe6, 0x62, 0xcf, - 0x25, 0xb0, 0xfe, 0x3d, 0x50, 0xd3, 0xfd, 0x50, 0x0a, 0x8a, 0x48, 0x74, 0x93, 0x6c, 0x91, 0xed, - 0x1a, 0xc3, 0x6f, 0x7a, 0x03, 0xd6, 0x9e, 0x58, 0xce, 0x42, 0x9e, 0x7e, 0x8d, 0xc9, 0x81, 0x6e, - 0x40, 0x59, 0x6e, 0x81, 0xde, 0x81, 0x1a, 0x8a, 0x25, 0xb2, 0x5c, 0x7f, 0xec, 0x86, 0x08, 0x2b, - 0xb1, 0xf5, 0xd4, 0x76, 0x18, 0x66, 0x21, 0x44, 0x5c, 0x92, 0x84, 0xf8, 0x7d, 0x11, 0x1a, 0x79, - 0x0d, 0xd0, 0x0f, 0x40, 0x89, 0xce, 0x7c, 0x89, 0x6b, 0xdc, 0xfb, 0xea, 0x55, 0x5a, 0x89, 0x87, - 0xc3, 0x33, 0x9f, 0x33, 0x74, 0xa0, 0xdf, 0x02, 0xea, 0xa2, 0x6d, 0x7c, 0x62, 0xb9, 0xb6, 0x73, - 0x86, 0x7a, 0x41, 0x2a, 0x2a, 0xd3, 0xe4, 0xcc, 0x43, 0x9c, 0x10, 0x32, 0x11, 0xdb, 0x9c, 0x71, - 0xc7, 0x6f, 0x2a, 0x38, 0x8f, 0xdf, 0xc2, 0xb6, 0x98, 0xdb, 0x51, 0x73, 0x4d, 0xda, 0xc4, 0xb7, - 0x7e, 0x06, 0x90, 0xad, 0x44, 0xd7, 0xa1, 0x32, 0xea, 0x7d, 0xdc, 0xeb, 0x7f, 0xd2, 0xd3, 0x0a, - 0x62, 0xd0, 0xed, 0x8f, 0x7a, 0x43, 0x93, 0x69, 0x84, 0xaa, 0xb0, 0xf6, 0xc8, 0x18, 0x3d, 0x32, - 0xb5, 0x22, 0xad, 0x83, 0xba, 0xbb, 0x37, 0x18, 0xf6, 0x1f, 0x31, 0xe3, 0x50, 0x2b, 0x51, 0x0a, - 0x0d, 0x9c, 0xc9, 0x6c, 0x8a, 0x70, 0x1d, 0x8c, 0x0e, 0x0f, 0x0d, 0xf6, 0xa9, 0xb6, 0x26, 0x04, - 0xb9, 0xd7, 0x7b, 0xd8, 0xd7, 0xca, 0xb4, 0x06, 0xd5, 0xc1, 0xd0, 0x18, 0x9a, 0x03, 0x73, 0xa8, - 0x55, 0xf4, 0x8f, 0xa1, 0x2c, 0x97, 0xbe, 0x06, 0x21, 0xea, 0xbf, 0x22, 0x50, 0x4d, 0xc4, 0x73, - 0x1d, 0xc2, 0xce, 0x49, 0x22, 0x39, 0xcf, 0x0b, 0x42, 0x28, 0x5d, 0x10, 0x82, 0xfe, 0xd7, 0x35, - 0x50, 0x53, 0x31, 0xd2, 0xdb, 0xa0, 0x4e, 0xbd, 0xc5, 0x3c, 0x1a, 0xdb, 0xf3, 0x08, 0x8f, 0x5c, - 0xd9, 0x2d, 0xb0, 0x2a, 0x9a, 0xf6, 0xe6, 0x11, 0xbd, 0x03, 0xeb, 0x72, 0xfa, 0xc4, 0xf1, 0xac, - 0x48, 0xae, 0xb5, 0x5b, 0x60, 0x80, 0xc6, 0x87, 0xc2, 0x46, 0x35, 0x28, 0x85, 0x0b, 0x17, 0x57, - 0x22, 0x4c, 0x7c, 0xd2, 0x9b, 0x50, 0x0e, 0xa7, 0x33, 0xee, 0x5a, 0x78, 0xb8, 0x6f, 0xb1, 0x78, - 0x44, 0xbf, 0x06, 0x8d, 0x9f, 0xf3, 0xc0, 0x1b, 0x47, 0xb3, 0x80, 0x87, 0x33, 0xcf, 0x39, 0xc6, - 0x83, 0x26, 0xac, 0x2e, 0xac, 0xc3, 0xc4, 0x48, 0xdf, 0x8b, 0x61, 0x19, 0xaf, 0x32, 0xf2, 0x22, - 0xac, 0x26, 0xec, 0xdd, 0x84, 0xdb, 0x37, 0x41, 0x5b, 0xc2, 0x49, 0x82, 0x15, 0x24, 0x48, 0x58, - 0x23, 0x45, 0x4a, 0x92, 0x06, 0x34, 0xe6, 0xfc, 0xd4, 0x8a, 0xec, 0x27, 0x7c, 0x1c, 0xfa, 0xd6, - 0x3c, 0x6c, 0x56, 0x57, 0xab, 0x72, 0x67, 0x31, 0xfd, 0x8c, 0x47, 0x03, 0xdf, 0x9a, 0xc7, 0x37, - 0xb4, 0x9e, 0x78, 0x08, 0x5b, 0x48, 0xbf, 0x0e, 0x1b, 0x69, 0x88, 0x63, 0xee, 0x44, 0x56, 0xd8, - 0x54, 0xb7, 0x4a, 0xdb, 0x94, 0xa5, 0x91, 0x77, 0xd0, 0x9a, 0x03, 0x22, 0xb7, 0xb0, 0x09, 0x5b, - 0xa5, 0x6d, 0x92, 0x01, 0x91, 0x98, 0x28, 0x6f, 0x0d, 0xdf, 0x0b, 0xed, 0x25, 0x52, 0xeb, 0xff, - 0x9d, 0x54, 0xe2, 0x91, 0x92, 0x4a, 0x43, 0xc4, 0xa4, 0x6a, 0x92, 0x54, 0x62, 0xce, 0x48, 0xa5, - 0xc0, 0x98, 0x54, 0x5d, 0x92, 0x4a, 0xcc, 0x31, 0xa9, 0x07, 0x00, 0x01, 0x0f, 0x79, 0x34, 0x9e, - 0x89, 0xcc, 0x37, 0xb0, 0x08, 0xdc, 0xbe, 0xa4, 0x8c, 0xb5, 0x99, 0x40, 0xed, 0xda, 0xf3, 0x88, - 0xa9, 0x41, 0xf2, 0x49, 0xdf, 0x05, 0x35, 0xd5, 0x5a, 0x73, 0x03, 0xc5, 0x97, 0x19, 0xf4, 0x8f, - 0x40, 0x4d, 0xbd, 0xf2, 0x57, 0xb9, 0x02, 0xa5, 0x4f, 0xcd, 0x81, 0x46, 0x68, 0x19, 0x8a, 0xbd, - 0xbe, 0x56, 0xcc, 0xae, 0x73, 0xe9, 0x96, 0xf2, 0xeb, 0x3f, 0xb6, 0x48, 0xa7, 0x02, 0x6b, 0xc8, - 0xbb, 0x53, 0x03, 0xc8, 0x8e, 0x5d, 0xff, 0x9b, 0x02, 0x0d, 0x3c, 0xe2, 0x4c, 0xd2, 0x21, 0x50, - 0x9c, 0xe3, 0xc1, 0x78, 0x65, 0x27, 0xf5, 0x8e, 0xf9, 0xaf, 0x17, 0x9b, 0xc6, 0xd2, 0xaf, 0xbb, - 0x1f, 0x78, 0x2e, 0x8f, 0x66, 0x7c, 0x11, 0x2e, 0x7f, 0xba, 0xde, 0x31, 0x77, 0xee, 0xa6, 0x05, - 0xba, 0xdd, 0x95, 0xe1, 0xb2, 0x1d, 0x6b, 0xd3, 0x15, 0xcb, 0x97, 0xd5, 0xfc, 0xed, 0xe5, 0x4d, - 0x49, 0x15, 0x33, 0x35, 0xd5, 0xb0, 0xb8, 0xec, 0x72, 0x26, 0xbe, 0xec, 0x38, 0xb8, 0xe4, 0xe6, - 0x5d, 0x83, 0xa2, 0xae, 0xe1, 0xa6, 0x7c, 0x03, 0xb4, 0x94, 0xc5, 0x04, 0xb1, 0x89, 0xd8, 0x52, - 0x0d, 0xca, 0x10, 0x08, 0x4d, 0x57, 0x4b, 0xa0, 0xf2, 0xb2, 0xa4, 0x77, 0x28, 0x86, 0xee, 0x2b, - 0x55, 0xa2, 0x15, 0xf7, 0x95, 0x6a, 0x59, 0xab, 0xec, 0x2b, 0x55, 0x55, 0x83, 0x7d, 0xa5, 0x5a, - 0xd3, 0xea, 0xfb, 0x4a, 0x75, 0x43, 0xd3, 0x58, 0x56, 0xc5, 0xd8, 0x4a, 0xf5, 0x60, 0xab, 0xd7, - 0x96, 0xad, 0x5e, 0x99, 0x65, 0x89, 0x3e, 0x00, 0xc8, 0xb6, 0x27, 0x4e, 0xd5, 0x3b, 0x39, 0x09, - 0xb9, 0x2c, 0x8d, 0x6f, 0xb1, 0x78, 0x24, 0xec, 0x0e, 0x9f, 0x9f, 0x46, 0x33, 0x3c, 0x90, 0x3a, - 0x8b, 0x47, 0xfa, 0x02, 0x68, 0x5e, 0x8c, 0xf8, 0x8b, 0xfe, 0x06, 0xbf, 0xce, 0x0f, 0x40, 0x4d, - 0xe5, 0x86, 0x6b, 0xe5, 0xba, 0xb4, 0x7c, 0xcc, 0xb8, 0x4b, 0xcb, 0x1c, 0xf4, 0x39, 0x6c, 0xc8, - 0x46, 0x20, 0xbb, 0x04, 0xa9, 0x62, 0xc8, 0x25, 0x8a, 0x29, 0x66, 0x8a, 0x79, 0x1f, 0x2a, 0x49, - 0xde, 0x65, 0xaf, 0xf3, 0xce, 0x65, 0x2d, 0x0b, 0x22, 0x58, 0x82, 0xd4, 0x43, 0xd8, 0x58, 0x99, - 0xa3, 0x2d, 0x80, 0x89, 0xb7, 0x98, 0x1f, 0x5b, 0x71, 0xcb, 0x4b, 0xb6, 0xd7, 0xd8, 0x92, 0x45, - 0xf0, 0x71, 0xbc, 0x9f, 0xf1, 0x20, 0x51, 0x30, 0x0e, 0x84, 0x75, 0xe1, 0xfb, 0x3c, 0x88, 0x35, - 0x2c, 0x07, 0x19, 0x77, 0x65, 0x89, 0xbb, 0xee, 0xc0, 0xdb, 0x2b, 0x9b, 0xc4, 0xe4, 0xe6, 0x2a, - 0x4e, 0x71, 0xa5, 0xe2, 0xd0, 0x0f, 0x2e, 0xe6, 0xf5, 0x9d, 0xd5, 0x06, 0x30, 0x8d, 0xb7, 0x9c, - 0xd2, 0x3f, 0x29, 0x50, 0xff, 0xd1, 0x82, 0x07, 0x67, 0x49, 0x6f, 0x4a, 0xef, 0x43, 0x39, 0x8c, - 0xac, 0x68, 0x11, 0xc6, 0x9d, 0x51, 0x2b, 0x8b, 0x93, 0x03, 0xb6, 0x07, 0x88, 0x62, 0x31, 0x9a, - 0xfe, 0x10, 0x80, 0x07, 0x81, 0x17, 0x8c, 0xb1, 0xab, 0xba, 0xd0, 0xbe, 0xe7, 0x7d, 0x4d, 0x81, - 0xc4, 0x9e, 0x4a, 0xe5, 0xc9, 0xa7, 0xc8, 0x07, 0x0e, 0x30, 0x4b, 0x2a, 0x93, 0x03, 0xda, 0x16, - 0x7c, 0x02, 0x7b, 0x7e, 0x8a, 0x69, 0xca, 0x5d, 0xd0, 0x01, 0xda, 0x77, 0xac, 0xc8, 0xda, 0x2d, - 0xb0, 0x18, 0x25, 0xf0, 0x4f, 0xf8, 0x34, 0xf2, 0x02, 0xac, 0x40, 0x39, 0xfc, 0x63, 0xb4, 0x27, - 0x78, 0x89, 0xc2, 0xf8, 0x53, 0xcb, 0xb1, 0x02, 0xfc, 0xf9, 0xcd, 0xc7, 0x47, 0x7b, 0x1a, 0x1f, - 0x47, 0x02, 0xef, 0x5a, 0x51, 0x60, 0x3f, 0xc5, 0xf2, 0x95, 0xc3, 0x1f, 0xa2, 0x3d, 0xc1, 0x4b, - 0x94, 0xfe, 0x1e, 0x94, 0x65, 0xa6, 0x44, 0xad, 0x37, 0x19, 0xeb, 0x33, 0xd9, 0xd2, 0x0d, 0x46, - 0xdd, 0xae, 0x39, 0x18, 0x68, 0x44, 0x16, 0x7e, 0xfd, 0x77, 0x04, 0xd4, 0x34, 0x2d, 0xa2, 0x57, - 0xeb, 0xf5, 0x7b, 0xa6, 0x84, 0x0e, 0xf7, 0x0e, 0xcd, 0xfe, 0x68, 0xa8, 0x11, 0xd1, 0xb8, 0x75, - 0x8d, 0x5e, 0xd7, 0x3c, 0x30, 0x77, 0x64, 0x03, 0x68, 0xfe, 0xd8, 0xec, 0x8e, 0x86, 0x7b, 0xfd, - 0x9e, 0x56, 0x12, 0x93, 0x1d, 0x63, 0x67, 0xbc, 0x63, 0x0c, 0x0d, 0x4d, 0x11, 0xa3, 0x3d, 0xd1, - 0x33, 0xf6, 0x8c, 0x03, 0x6d, 0x8d, 0x6e, 0xc0, 0xfa, 0xa8, 0x67, 0x3c, 0x36, 0xf6, 0x0e, 0x8c, - 0xce, 0x81, 0xa9, 0x95, 0x85, 0x6f, 0xaf, 0x3f, 0x1c, 0x3f, 0xec, 0x8f, 0x7a, 0x3b, 0x5a, 0x45, - 0x34, 0x8f, 0x62, 0x68, 0x74, 0xbb, 0xe6, 0xd1, 0x10, 0x21, 0xd5, 0xf8, 0x07, 0xa9, 0x0c, 0x8a, - 0xe8, 0x83, 0x75, 0x13, 0x20, 0xcb, 0x77, 0xbe, 0xcd, 0x56, 0xaf, 0x6a, 0xcb, 0x2e, 0x56, 0x00, - 0xfd, 0x97, 0x04, 0x20, 0x3b, 0x07, 0x7a, 0x3f, 0x7b, 0xb7, 0xc8, 0x16, 0xf1, 0xe6, 0xea, 0x71, - 0x5d, 0xfe, 0x7a, 0xf9, 0x41, 0xee, 0x15, 0x52, 0x5c, 0xbd, 0xd2, 0xd2, 0xf5, 0x3f, 0xbd, 0x45, - 0xc6, 0x50, 0x5b, 0x8e, 0x2f, 0x4a, 0x9d, 0xec, 0xdd, 0x91, 0x87, 0xca, 0xe2, 0xd1, 0xff, 0xdf, - 0x7f, 0xfe, 0x86, 0xc0, 0xc6, 0x0a, 0x8d, 0x2b, 0x17, 0xc9, 0x95, 0xc5, 0xe2, 0x1b, 0x94, 0xc5, - 0xc2, 0xd2, 0x1d, 0x7e, 0x13, 0x32, 0xe2, 0xf0, 0x52, 0x31, 0x5f, 0xfe, 0x46, 0x7a, 0x93, 0xc3, - 0xeb, 0x00, 0x64, 0x1a, 0xa7, 0xdf, 0x85, 0x72, 0xee, 0xe9, 0x7f, 0x73, 0xf5, 0x26, 0xc4, 0x8f, - 0x7f, 0x49, 0x38, 0xc6, 0xea, 0x7f, 0x20, 0x50, 0x5b, 0x9e, 0xbe, 0x32, 0x29, 0xff, 0xfb, 0x93, - 0xb6, 0x93, 0x13, 0x85, 0xac, 0xf3, 0xef, 0x5e, 0x95, 0x47, 0x7c, 0x7b, 0x5c, 0xd0, 0x45, 0xe7, - 0xfb, 0xcf, 0x5f, 0xb6, 0x0a, 0x9f, 0xbf, 0x6c, 0x15, 0x5e, 0xbf, 0x6c, 0x91, 0x5f, 0x9c, 0xb7, - 0xc8, 0x9f, 0xcf, 0x5b, 0xe4, 0xd9, 0x79, 0x8b, 0x3c, 0x3f, 0x6f, 0x91, 0x7f, 0x9c, 0xb7, 0xc8, - 0x17, 0xe7, 0xad, 0xc2, 0xeb, 0xf3, 0x16, 0xf9, 0xed, 0xab, 0x56, 0xe1, 0xf9, 0xab, 0x56, 0xe1, - 0xf3, 0x57, 0xad, 0xc2, 0x4f, 0x2a, 0xf8, 0x07, 0x8b, 0x3f, 0x99, 0x94, 0xf1, 0xaf, 0x92, 0xf7, - 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x65, 0xcf, 0xac, 0x82, 0x72, 0x11, 0x00, 0x00, + // 1876 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcf, 0x93, 0xdb, 0x48, + 0x15, 0xb6, 0x6c, 0x8d, 0x6d, 0xbd, 0xf1, 0x78, 0x94, 0xde, 0x10, 0xbc, 0x53, 0x1b, 0xcf, 0x44, + 0x14, 0xcb, 0x90, 0x82, 0x09, 0xb5, 0x0b, 0xd9, 0xda, 0xad, 0x50, 0x8b, 0x6c, 0x2b, 0x19, 0xcf, + 0xfa, 0xc7, 0x6c, 0x5b, 0xce, 0x12, 0x2e, 0x2a, 0x8d, 0xa7, 0x67, 0xac, 0x5a, 0xc9, 0x32, 0x92, + 0x1c, 0x32, 0x9c, 0xb8, 0x40, 0x51, 0x9c, 0xb8, 0x70, 0xa1, 0xb8, 0x71, 0x80, 0xbf, 0x80, 0xbf, + 0x21, 0x55, 0x14, 0x55, 0x39, 0x6e, 0x71, 0x48, 0x91, 0xc9, 0x65, 0x8f, 0x39, 0x70, 0xe2, 0x44, + 0xf5, 0x6b, 0xfd, 0xb0, 0x3c, 0x33, 0x10, 0xd8, 0xdc, 0xd4, 0xef, 0x7d, 0xfd, 0xf4, 0xf5, 0xeb, + 0xef, 0x3d, 0x3f, 0x19, 0xd6, 0x3d, 0xc7, 0x73, 0x82, 0xbd, 0x79, 0xe0, 0x47, 0x3e, 0xa9, 0x4e, + 0xfc, 0x20, 0x62, 0x4f, 0xe6, 0x47, 0x5b, 0xdf, 0x3d, 0x75, 0xa2, 0xe9, 0xe2, 0x68, 0x6f, 0xe2, + 0x7b, 0x77, 0x4e, 0xfd, 0x53, 0xff, 0x0e, 0x02, 0x8e, 0x16, 0x27, 0xb8, 0xc2, 0x05, 0x3e, 0x89, + 0x8d, 0xda, 0x5f, 0x8a, 0x50, 0xfb, 0x2c, 0x70, 0x22, 0x46, 0xd9, 0x4f, 0x17, 0x2c, 0x8c, 0xc8, + 0x21, 0x40, 0xe4, 0x78, 0x2c, 0x64, 0x81, 0xc3, 0xc2, 0x86, 0xb4, 0x53, 0xda, 0x5d, 0x7f, 0xef, + 0xfa, 0x5e, 0x12, 0x7e, 0xcf, 0x74, 0x3c, 0x36, 0x42, 0x5f, 0x6b, 0xeb, 0xe9, 0xf3, 0xed, 0xc2, + 0xdf, 0x9f, 0x6f, 0x93, 0xc3, 0x80, 0xd9, 0xae, 0xeb, 0x4f, 0xcc, 0x74, 0x1f, 0x5d, 0x8a, 0x41, + 0x3e, 0x84, 0xf2, 0xc8, 0x5f, 0x04, 0x13, 0xd6, 0x28, 0xee, 0x48, 0xbb, 0xf5, 0xf7, 0x6e, 0x65, + 0xd1, 0x96, 0xdf, 0xbc, 0x27, 0x40, 0xc6, 0x6c, 0xe1, 0xd1, 0x78, 0x03, 0xf9, 0x08, 0xaa, 0x1e, + 0x8b, 0xec, 0x63, 0x3b, 0xb2, 0x1b, 0x25, 0xa4, 0xd2, 0xc8, 0x36, 0xf7, 0x59, 0x14, 0x38, 0x93, + 0x7e, 0xec, 0x6f, 0xc9, 0x4f, 0x9f, 0x6f, 0x4b, 0x34, 0xc5, 0x93, 0x7b, 0xb0, 0x15, 0x7e, 0xee, + 0xcc, 0x2d, 0xd7, 0x3e, 0x62, 0xae, 0x35, 0xb3, 0x3d, 0x66, 0x3d, 0xb6, 0x5d, 0xe7, 0xd8, 0x8e, + 0x1c, 0x7f, 0xd6, 0xf8, 0xb2, 0xb2, 0x23, 0xed, 0x56, 0xe9, 0xd7, 0x39, 0xa4, 0xc7, 0x11, 0x03, + 0xdb, 0x63, 0x0f, 0x53, 0xbf, 0xb6, 0x0d, 0x90, 0xf1, 0x21, 0x15, 0x28, 0xe9, 0x87, 0x5d, 0xb5, + 0x40, 0xaa, 0x20, 0xd3, 0x71, 0xcf, 0x50, 0x25, 0x6d, 0x13, 0x36, 0x62, 0xf6, 0xe1, 0xdc, 0x9f, + 0x85, 0x4c, 0xfb, 0x18, 0xae, 0xa1, 0xc1, 0x08, 0x02, 0x3f, 0xe8, 0xb0, 0xc8, 0x76, 0xdc, 0x90, + 0xdc, 0x86, 0xb5, 0xb6, 0xbd, 0x08, 0x59, 0x43, 0xc2, 0xa3, 0x2f, 0x25, 0x12, 0x61, 0xe8, 0xa3, + 0x02, 0xa2, 0xfd, 0x53, 0x02, 0xc8, 0xd2, 0x4b, 0x74, 0x28, 0x23, 0xf5, 0xe4, 0x12, 0xde, 0xca, + 0xf6, 0x22, 0xe1, 0x43, 0xdb, 0x09, 0x5a, 0xd7, 0xe3, 0x3b, 0xa8, 0xa1, 0x49, 0x3f, 0xb6, 0xe7, + 0x11, 0x0b, 0x68, 0xbc, 0x91, 0x7c, 0x0f, 0x2a, 0xa1, 0xed, 0xcd, 0x5d, 0x16, 0x36, 0x8a, 0x18, + 0x43, 0xcd, 0x62, 0x8c, 0xd0, 0x81, 0x59, 0x2b, 0xd0, 0x04, 0x46, 0xee, 0x82, 0xc2, 0x9e, 0x30, + 0x6f, 0xee, 0xda, 0x41, 0x18, 0x67, 0x9c, 0x2c, 0x71, 0x8e, 0x5d, 0xf1, 0xae, 0x0c, 0x4a, 0x3e, + 0x04, 0x98, 0x3a, 0x61, 0xe4, 0x9f, 0x06, 0xb6, 0x17, 0x36, 0xe4, 0x55, 0xc2, 0xfb, 0x89, 0x2f, + 0xde, 0xb9, 0x04, 0xd6, 0x7e, 0x00, 0x4a, 0x7a, 0x1e, 0x42, 0x40, 0xe6, 0x37, 0x85, 0xe9, 0xaa, + 0x51, 0x7c, 0x26, 0xd7, 0x61, 0xed, 0xb1, 0xed, 0x2e, 0x84, 0x7c, 0x6a, 0x54, 0x2c, 0x34, 0x1d, + 0xca, 0xe2, 0x08, 0xe4, 0x16, 0xd4, 0x50, 0x6d, 0x91, 0xed, 0xcd, 0x2d, 0x2f, 0x44, 0x58, 0x89, + 0xae, 0xa7, 0xb6, 0x7e, 0x98, 0x85, 0xe0, 0x71, 0xa5, 0x24, 0xc4, 0xef, 0x8b, 0x50, 0xcf, 0x8b, + 0x88, 0x7c, 0x00, 0x72, 0x74, 0x36, 0x4f, 0xae, 0xeb, 0x1b, 0x57, 0x89, 0x2d, 0x5e, 0x9a, 0x67, + 0x73, 0x46, 0x71, 0x03, 0xf9, 0x0e, 0x10, 0x0f, 0x6d, 0xd6, 0x89, 0xed, 0x39, 0xee, 0x19, 0x0a, + 0x0e, 0xa9, 0x28, 0x54, 0x15, 0x9e, 0xfb, 0xe8, 0xe0, 0x3a, 0xe3, 0xc7, 0x9c, 0x32, 0x77, 0xde, + 0x90, 0xd1, 0x8f, 0xcf, 0xdc, 0xb6, 0x98, 0x39, 0x51, 0x63, 0x4d, 0xd8, 0xf8, 0xb3, 0x76, 0x06, + 0x90, 0xbd, 0x89, 0xac, 0x43, 0x65, 0x3c, 0xf8, 0x64, 0x30, 0xfc, 0x6c, 0xa0, 0x16, 0xf8, 0xa2, + 0x3d, 0x1c, 0x0f, 0x4c, 0x83, 0xaa, 0x12, 0x51, 0x60, 0xed, 0x81, 0x3e, 0x7e, 0x60, 0xa8, 0x45, + 0xb2, 0x01, 0xca, 0x7e, 0x77, 0x64, 0x0e, 0x1f, 0x50, 0xbd, 0xaf, 0x96, 0x08, 0x81, 0x3a, 0x7a, + 0x32, 0x9b, 0xcc, 0xb7, 0x8e, 0xc6, 0xfd, 0xbe, 0x4e, 0x1f, 0xa9, 0x6b, 0x5c, 0xd1, 0xdd, 0xc1, + 0xfd, 0xa1, 0x5a, 0x26, 0x35, 0xa8, 0x8e, 0x4c, 0xdd, 0x34, 0x46, 0x86, 0xa9, 0x56, 0xb4, 0x4f, + 0xa0, 0x2c, 0x5e, 0xfd, 0x06, 0x84, 0xa8, 0xfd, 0x4a, 0x82, 0x6a, 0x22, 0x9e, 0x37, 0x21, 0xec, + 0x9c, 0x24, 0x92, 0xfb, 0xbc, 0x20, 0x84, 0xd2, 0x05, 0x21, 0x68, 0x7f, 0x5d, 0x03, 0x25, 0x15, + 0x23, 0xb9, 0x09, 0xca, 0xc4, 0x5f, 0xcc, 0x22, 0xcb, 0x99, 0x45, 0x78, 0xe5, 0xf2, 0x7e, 0x81, + 0x56, 0xd1, 0xd4, 0x9d, 0x45, 0xe4, 0x16, 0xac, 0x0b, 0xf7, 0x89, 0xeb, 0xdb, 0x91, 0x78, 0xd7, + 0x7e, 0x81, 0x02, 0x1a, 0xef, 0x73, 0x1b, 0x51, 0xa1, 0x14, 0x2e, 0x3c, 0x7c, 0x93, 0x44, 0xf9, + 0x23, 0xb9, 0x01, 0xe5, 0x70, 0x32, 0x65, 0x9e, 0x8d, 0x97, 0x7b, 0x8d, 0xc6, 0x2b, 0xf2, 0x4d, + 0xa8, 0xff, 0x9c, 0x05, 0xbe, 0x15, 0x4d, 0x03, 0x16, 0x4e, 0x7d, 0xf7, 0x18, 0x2f, 0x5a, 0xa2, + 0x1b, 0xdc, 0x6a, 0x26, 0x46, 0xf2, 0x6e, 0x0c, 0xcb, 0x78, 0x95, 0x91, 0x97, 0x44, 0x6b, 0xdc, + 0xde, 0x4e, 0xb8, 0xdd, 0x06, 0x75, 0x09, 0x27, 0x08, 0x56, 0x90, 0xa0, 0x44, 0xeb, 0x29, 0x52, + 0x90, 0xd4, 0xa1, 0x3e, 0x63, 0xa7, 0x76, 0xe4, 0x3c, 0x66, 0x56, 0x38, 0xb7, 0x67, 0x61, 0xa3, + 0xba, 0xda, 0xd6, 0x5b, 0x8b, 0xc9, 0xe7, 0x2c, 0x1a, 0xcd, 0xed, 0x59, 0x5c, 0xa1, 0x1b, 0xc9, + 0x0e, 0x6e, 0x0b, 0xc9, 0xb7, 0x60, 0x33, 0x0d, 0x71, 0xcc, 0xdc, 0xc8, 0x0e, 0x1b, 0xca, 0x4e, + 0x69, 0x97, 0xd0, 0x34, 0x72, 0x07, 0xad, 0x39, 0x20, 0x72, 0x0b, 0x1b, 0xb0, 0x53, 0xda, 0x95, + 0x32, 0x20, 0x12, 0xe3, 0xed, 0xad, 0x3e, 0xf7, 0x43, 0x67, 0x89, 0xd4, 0xfa, 0x7f, 0x27, 0x95, + 0xec, 0x48, 0x49, 0xa5, 0x21, 0x62, 0x52, 0x35, 0x41, 0x2a, 0x31, 0x67, 0xa4, 0x52, 0x60, 0x4c, + 0x6a, 0x43, 0x90, 0x4a, 0xcc, 0x31, 0xa9, 0x7b, 0x00, 0x01, 0x0b, 0x59, 0x64, 0x4d, 0x79, 0xe6, + 0xeb, 0xd8, 0x04, 0x6e, 0x5e, 0xd2, 0xc6, 0xf6, 0x28, 0x47, 0xed, 0x3b, 0xb3, 0x88, 0x2a, 0x41, + 0xf2, 0x48, 0xde, 0x01, 0x25, 0xd5, 0x5a, 0x63, 0x13, 0xc5, 0x97, 0x19, 0xb4, 0x8f, 0x40, 0x49, + 0x77, 0xe5, 0x4b, 0xb9, 0x02, 0xa5, 0x47, 0xc6, 0x48, 0x95, 0x48, 0x19, 0x8a, 0x83, 0xa1, 0x5a, + 0xcc, 0xca, 0xb9, 0xb4, 0x25, 0xff, 0xfa, 0x8f, 0x4d, 0xa9, 0x55, 0x81, 0x35, 0xe4, 0xdd, 0xaa, + 0x01, 0x64, 0xd7, 0xae, 0xfd, 0x4d, 0x86, 0x3a, 0x5e, 0x71, 0x26, 0xe9, 0x10, 0x08, 0xfa, 0x58, + 0x60, 0xad, 0x9c, 0x64, 0xa3, 0x65, 0xfc, 0xeb, 0xf9, 0xb6, 0xbe, 0x34, 0x1e, 0xcc, 0x03, 0xdf, + 0x63, 0xd1, 0x94, 0x2d, 0xc2, 0xe5, 0x47, 0xcf, 0x3f, 0x66, 0xee, 0x9d, 0xb4, 0x41, 0xef, 0xb5, + 0x45, 0xb8, 0xec, 0xc4, 0xea, 0x64, 0xc5, 0xf2, 0x55, 0x35, 0x7f, 0x73, 0xf9, 0x50, 0x42, 0xc5, + 0x54, 0x49, 0x35, 0xcc, 0x8b, 0x5d, 0x78, 0xe2, 0x62, 0xc7, 0xc5, 0x25, 0x95, 0xf7, 0x06, 0x14, + 0xf5, 0x06, 0x2a, 0xe5, 0xdb, 0xa0, 0xa6, 0x2c, 0x8e, 0x10, 0x9b, 0x88, 0x2d, 0xd5, 0xa0, 0x08, + 0x81, 0xd0, 0xf4, 0x6d, 0x09, 0x54, 0x14, 0x4b, 0x5a, 0x43, 0x31, 0xf4, 0x40, 0xae, 0x4a, 0x6a, + 0xf1, 0x40, 0xae, 0x96, 0xd5, 0xca, 0x81, 0x5c, 0x55, 0x54, 0x38, 0x90, 0xab, 0x35, 0x75, 0xe3, + 0x40, 0xae, 0x6e, 0xaa, 0x2a, 0xcd, 0xba, 0x18, 0x5d, 0xe9, 0x1e, 0x74, 0xb5, 0x6c, 0xe9, 0x6a, + 0xc9, 0x2c, 0x4b, 0xf4, 0x1e, 0x40, 0x76, 0x3c, 0x7e, 0xab, 0xfe, 0xc9, 0x49, 0xc8, 0x44, 0x6b, + 0xbc, 0x46, 0xe3, 0x15, 0xb7, 0xbb, 0x6c, 0x76, 0x1a, 0x4d, 0xf1, 0x42, 0x36, 0x68, 0xbc, 0xd2, + 0x16, 0x40, 0xf2, 0x62, 0xc4, 0x5f, 0xf4, 0xd7, 0xf8, 0x75, 0xbe, 0x07, 0x4a, 0x2a, 0x37, 0x7c, + 0x57, 0x6e, 0xcc, 0xcb, 0xc7, 0x8c, 0xc7, 0xbc, 0x6c, 0x83, 0x36, 0x83, 0x4d, 0x31, 0x08, 0x64, + 0x45, 0x90, 0x2a, 0x46, 0xba, 0x44, 0x31, 0xc5, 0x4c, 0x31, 0xef, 0x43, 0x25, 0xc9, 0xbb, 0x98, + 0x75, 0xde, 0xbe, 0x6c, 0x64, 0x41, 0x04, 0x4d, 0x90, 0x5a, 0x08, 0x9b, 0x2b, 0x3e, 0xd2, 0x04, + 0x38, 0xf2, 0x17, 0xb3, 0x63, 0x3b, 0x9e, 0x99, 0xa5, 0xdd, 0x35, 0xba, 0x64, 0xe1, 0x7c, 0x5c, + 0xff, 0x67, 0x2c, 0x48, 0x14, 0x8c, 0x0b, 0x6e, 0x5d, 0xcc, 0xe7, 0x2c, 0x88, 0x35, 0x2c, 0x16, + 0x19, 0x77, 0x79, 0x89, 0xbb, 0xe6, 0xc2, 0x5b, 0x2b, 0x87, 0xc4, 0xe4, 0xe6, 0x3a, 0x4e, 0x71, + 0xa5, 0xe3, 0x90, 0x0f, 0x2e, 0xe6, 0xf5, 0xed, 0xd5, 0x01, 0x30, 0x8d, 0xb7, 0x9c, 0xd2, 0x3f, + 0xc9, 0xb0, 0xf1, 0xe9, 0x82, 0x05, 0x67, 0xc9, 0x70, 0x4b, 0xee, 0x42, 0x39, 0x8c, 0xec, 0x68, + 0x11, 0xc6, 0x93, 0x51, 0x33, 0x8b, 0x93, 0x03, 0xee, 0x8d, 0x10, 0x45, 0x63, 0x34, 0xf9, 0x11, + 0x00, 0xe3, 0x83, 0xae, 0x85, 0x53, 0xd5, 0x85, 0xf9, 0x3f, 0xbf, 0x17, 0x47, 0x62, 0x9c, 0xa9, + 0x14, 0x96, 0x3c, 0xf2, 0x7c, 0xe0, 0x02, 0xb3, 0xa4, 0x50, 0xb1, 0x20, 0x7b, 0x9c, 0x4f, 0xe0, + 0xcc, 0x4e, 0x31, 0x4d, 0xb9, 0x02, 0x1d, 0xa1, 0xbd, 0x63, 0x47, 0xf6, 0x7e, 0x81, 0xc6, 0x28, + 0x8e, 0x7f, 0xcc, 0x26, 0x91, 0x1f, 0x60, 0x07, 0xca, 0xe1, 0x1f, 0xa2, 0x3d, 0xc1, 0x0b, 0x14, + 0xc6, 0x9f, 0xd8, 0xae, 0x1d, 0xe0, 0xcf, 0x6f, 0x3e, 0x3e, 0xda, 0xd3, 0xf8, 0xb8, 0xe2, 0x78, + 0xcf, 0x8e, 0x02, 0xe7, 0x09, 0xb6, 0xaf, 0x1c, 0xbe, 0x8f, 0xf6, 0x04, 0x2f, 0x50, 0xda, 0xbb, + 0x50, 0x16, 0x99, 0xe2, 0xbd, 0xde, 0xa0, 0x74, 0x48, 0xc5, 0x48, 0x37, 0x1a, 0xb7, 0xdb, 0xc6, + 0x68, 0xa4, 0x4a, 0xa2, 0xf1, 0x6b, 0xbf, 0x93, 0x40, 0x49, 0xd3, 0xc2, 0x67, 0xb5, 0xc1, 0x70, + 0x60, 0x08, 0xa8, 0xd9, 0xed, 0x1b, 0xc3, 0xb1, 0xa9, 0x4a, 0x7c, 0x70, 0x6b, 0xeb, 0x83, 0xb6, + 0xd1, 0x33, 0x3a, 0x62, 0x00, 0x34, 0x7e, 0x6c, 0xb4, 0xc7, 0x66, 0x77, 0x38, 0x50, 0x4b, 0xdc, + 0xd9, 0xd2, 0x3b, 0x56, 0x47, 0x37, 0x75, 0x55, 0xe6, 0xab, 0x2e, 0x9f, 0x19, 0x07, 0x7a, 0x4f, + 0x5d, 0x23, 0x9b, 0xb0, 0x3e, 0x1e, 0xe8, 0x0f, 0xf5, 0x6e, 0x4f, 0x6f, 0xf5, 0x0c, 0xb5, 0xcc, + 0xf7, 0x0e, 0x86, 0xa6, 0x75, 0x7f, 0x38, 0x1e, 0x74, 0xd4, 0x0a, 0x1f, 0x1e, 0xf9, 0x52, 0x6f, + 0xb7, 0x8d, 0x43, 0x13, 0x21, 0xd5, 0xf8, 0x07, 0xa9, 0x0c, 0x32, 0x9f, 0x83, 0x35, 0x03, 0x20, + 0xcb, 0x77, 0x7e, 0xcc, 0x56, 0xae, 0x1a, 0xcb, 0x2e, 0x76, 0x00, 0xed, 0x97, 0x12, 0x40, 0x76, + 0x0f, 0xe4, 0x6e, 0xf6, 0xdd, 0x22, 0x46, 0xc4, 0x1b, 0xab, 0xd7, 0x75, 0xf9, 0xd7, 0xcb, 0xc7, + 0xb9, 0xaf, 0x90, 0xe2, 0x6a, 0x49, 0x8b, 0xad, 0xff, 0xe9, 0x5b, 0xc4, 0x82, 0xda, 0x72, 0x7c, + 0xde, 0xea, 0xc4, 0xec, 0x8e, 0x3c, 0x14, 0x1a, 0xaf, 0xfe, 0xff, 0xf9, 0xf3, 0x37, 0x12, 0x6c, + 0xae, 0xd0, 0xb8, 0xf2, 0x25, 0xb9, 0xb6, 0x58, 0x7c, 0x8d, 0xb6, 0x58, 0x58, 0xaa, 0xe1, 0xd7, + 0x21, 0xc3, 0x2f, 0x2f, 0x15, 0xf3, 0xe5, 0xdf, 0x48, 0xaf, 0x73, 0x79, 0x2d, 0x80, 0x4c, 0xe3, + 0xe4, 0xfb, 0x50, 0xce, 0xfd, 0x77, 0x70, 0x63, 0xb5, 0x12, 0xe2, 0x7f, 0x0f, 0x04, 0xe1, 0x18, + 0xab, 0xfd, 0x41, 0x82, 0xda, 0xb2, 0xfb, 0xca, 0xa4, 0xfc, 0xef, 0x9f, 0xb4, 0xad, 0x9c, 0x28, + 0x44, 0x9f, 0x7f, 0xe7, 0xaa, 0x3c, 0xe2, 0xb7, 0xc7, 0x05, 0x5d, 0xdc, 0xe6, 0xfa, 0xcc, 0x3e, + 0xd8, 0xc9, 0x16, 0xdc, 0xa0, 0xc6, 0x61, 0xaf, 0xdb, 0xd6, 0x47, 0x56, 0xa7, 0xdb, 0xb1, 0x78, + 0x91, 0xf4, 0x75, 0xb3, 0xbd, 0xaf, 0x16, 0xc8, 0xd7, 0xe0, 0x9a, 0x39, 0x1c, 0x5a, 0x7d, 0x7d, + 0xf0, 0xc8, 0x6a, 0xf7, 0xc6, 0x23, 0xd3, 0xa0, 0x7c, 0xb4, 0xab, 0x03, 0x3c, 0xd4, 0x7b, 0xdd, + 0x8e, 0x8e, 0x65, 0x59, 0xe4, 0x21, 0xba, 0x83, 0x07, 0xc6, 0x88, 0x2f, 0x2d, 0xaa, 0x9b, 0x86, + 0xd5, 0xeb, 0xf6, 0xbb, 0xa6, 0xd1, 0x51, 0x4b, 0xa4, 0x01, 0xd7, 0xa9, 0xf1, 0xe9, 0xd8, 0x18, + 0x99, 0x79, 0x8f, 0xdc, 0xfa, 0xe1, 0xb3, 0x17, 0xcd, 0xc2, 0x17, 0x2f, 0x9a, 0x85, 0x57, 0x2f, + 0x9a, 0xd2, 0x2f, 0xce, 0x9b, 0xd2, 0x9f, 0xcf, 0x9b, 0xd2, 0xd3, 0xf3, 0xa6, 0xf4, 0xec, 0xbc, + 0x29, 0xfd, 0xe3, 0xbc, 0x29, 0x7d, 0x79, 0xde, 0x2c, 0xbc, 0x3a, 0x6f, 0x4a, 0xbf, 0x7d, 0xd9, + 0x2c, 0x3c, 0x7b, 0xd9, 0x2c, 0x7c, 0xf1, 0xb2, 0x59, 0xf8, 0x49, 0x05, 0xff, 0x29, 0x9a, 0x1f, + 0x1d, 0x95, 0xf1, 0x3f, 0x9f, 0xf7, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x6e, 0x9d, 0x53, 0x64, + 0x3b, 0x12, 0x00, 0x00, +} + +func (x ErrorCause) String() string { + s, ok := ErrorCause_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) } - func (x WriteRequest_SourceEnum) String() string { s, ok := WriteRequest_SourceEnum_name[int32(x)] if ok { @@ -1999,6 +2089,30 @@ func (this *WriteResponse) Equal(that interface{}) bool { } return true } +func (this *WriteErrorDetails) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*WriteErrorDetails) + if !ok { + that2, ok := that.(WriteErrorDetails) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Cause != that1.Cause { + return false + } + return true +} func (this *TimeSeries) Equal(that interface{}) bool { if that == nil { return this == nil @@ -3015,6 +3129,16 @@ func (this *WriteResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *WriteErrorDetails) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&mimirpb.WriteErrorDetails{") + s = append(s, "Cause: "+fmt.Sprintf("%#v", this.Cause)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *TimeSeries) GoString() string { if this == nil { return "nil" @@ -3523,6 +3647,34 @@ func (m *WriteResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *WriteErrorDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *WriteErrorDetails) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *WriteErrorDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Cause != 0 { + i = encodeVarintMimir(dAtA, i, uint64(m.Cause)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *TimeSeries) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4803,6 +4955,18 @@ func (m *WriteResponse) Size() (n int) { return n } +func (m *WriteErrorDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Cause != 0 { + n += 1 + sovMimir(uint64(m.Cause)) + } + return n +} + func (m *TimeSeries) Size() (n int) { if m == nil { return 0 @@ -5398,6 +5562,16 @@ func (this *WriteResponse) String() string { }, "") return s } +func (this *WriteErrorDetails) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&WriteErrorDetails{`, + `Cause:` + fmt.Sprintf("%v", this.Cause) + `,`, + `}`, + }, "") + return s +} func (this *TimeSeries) String() string { if this == nil { return "nil" @@ -6025,6 +6199,78 @@ func (m *WriteResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *WriteErrorDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: WriteErrorDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WriteErrorDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Cause", wireType) + } + m.Cause = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Cause |= ErrorCause(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipMimir(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *TimeSeries) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/mimirpb/mimir.proto b/pkg/mimirpb/mimir.proto index 31d7021e9ba..6184fca969a 100644 --- a/pkg/mimirpb/mimir.proto +++ b/pkg/mimirpb/mimir.proto @@ -34,6 +34,18 @@ message WriteRequest { message WriteResponse {} +enum ErrorCause { + REPLICAS_DID_NOT_MATCH = 0; + TOO_MANY_CLUSTERS = 1; + VALIDATION = 2; + INGESTION_RATE_LIMITED = 3; + REQUEST_RATE_LIMITED = 4; +} + +message WriteErrorDetails { + ErrorCause Cause = 1; +} + message TimeSeries { repeated LabelPair labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "LabelAdapter"]; // Sorted by time, oldest sample first. From 28cf65fa4a724a5bafb6f1249f523d126e554788 Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Mon, 16 Oct 2023 11:15:24 +0200 Subject: [PATCH 2/6] Introduce distributorError interface Signed-off-by: Yuri Nikolic --- pkg/distributor/errors.go | 70 ++++++++++++++++++++++++---------- pkg/distributor/errors_test.go | 16 ++++++++ pkg/distributor/push.go | 55 +++++++++++++------------- 3 files changed, 94 insertions(+), 47 deletions(-) diff --git a/pkg/distributor/errors.go b/pkg/distributor/errors.go index 41ca23f25d3..ed68ac21520 100644 --- a/pkg/distributor/errors.go +++ b/pkg/distributor/errors.go @@ -39,6 +39,11 @@ var ( ) ) +// distributorError is a marker interface for the errors returned by distributor. +type distributorError interface { + errorCause() mimirpb.ErrorCause +} + // replicasDidNotMatchError is an error stating that replicas do not match. type replicasDidNotMatchError struct { replica, elected string @@ -56,6 +61,11 @@ func (e replicasDidNotMatchError) Error() string { return fmt.Sprintf("replicas did not match, rejecting sample: replica=%s, elected=%s", e.replica, e.elected) } +// replicasDidNotMatchError implements the distributorError interface. +func (e replicasDidNotMatchError) errorCause() mimirpb.ErrorCause { + return mimirpb.REPLICAS_DID_NOT_MATCH +} + // tooManyClustersError is an error stating that there are too many HA clusters. type tooManyClustersError struct { limit int @@ -72,6 +82,11 @@ func (e tooManyClustersError) Error() string { return fmt.Sprintf(tooManyClustersMsgFormat, e.limit) } +// tooManyClustersError implements the distributorError interface. +func (e tooManyClustersError) errorCause() mimirpb.ErrorCause { + return mimirpb.TOO_MANY_CLUSTERS +} + // validationError is an error, used to represent all validation errors from the validation package. type validationError struct { error @@ -82,6 +97,11 @@ func newValidationError(err error) validationError { return validationError{error: err} } +// validationError implements the distributorError interface. +func (e validationError) errorCause() mimirpb.ErrorCause { + return mimirpb.VALIDATION +} + // ingestionRateLimitedError is an error used to represent the ingestion rate limited error. type ingestionRateLimitedError struct { limit float64 @@ -100,6 +120,11 @@ func (e ingestionRateLimitedError) Error() string { return fmt.Sprintf(ingestionRateLimitedMsgFormat, e.limit, e.burst) } +// ingestionRateLimitedError implements the distributorError interface. +func (e ingestionRateLimitedError) errorCause() mimirpb.ErrorCause { + return mimirpb.INGESTION_RATE_LIMITED +} + // requestRateLimitedError is an error used to represent the request rate limited error. type requestRateLimitedError struct { limit float64 @@ -118,33 +143,36 @@ func (e requestRateLimitedError) Error() string { return fmt.Sprintf(requestRateLimitedMsgFormat, e.limit, e.burst) } +// requestRateLimitedError implements the distributorError interface. +func (e requestRateLimitedError) errorCause() mimirpb.ErrorCause { + return mimirpb.REQUEST_RATE_LIMITED +} + // toGRPCError converts the given error into an appropriate gRPC error. func toGRPCError(pushErr error, serviceOverloadErrorEnabled bool) error { var ( - errDetails *mimirpb.WriteErrorDetails - errCode = codes.Internal + distributorErr distributorError + errDetails *mimirpb.WriteErrorDetails + errCode = codes.Internal ) - - switch { - case errors.As(pushErr, &validationError{}): - errCode = codes.FailedPrecondition - errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} - case errors.As(pushErr, &ingestionRateLimitedError{}): - errCode = codes.ResourceExhausted - errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.INGESTION_RATE_LIMITED} - case errors.As(pushErr, &requestRateLimitedError{}): - if serviceOverloadErrorEnabled { - errCode = codes.Unavailable - } else { + if errors.As(pushErr, &distributorErr) { + errDetails = &mimirpb.WriteErrorDetails{Cause: distributorErr.errorCause()} + switch distributorErr.errorCause() { + case mimirpb.VALIDATION: + errCode = codes.FailedPrecondition + case mimirpb.INGESTION_RATE_LIMITED: errCode = codes.ResourceExhausted + case mimirpb.REQUEST_RATE_LIMITED: + if serviceOverloadErrorEnabled { + errCode = codes.Unavailable + } else { + errCode = codes.ResourceExhausted + } + case mimirpb.REPLICAS_DID_NOT_MATCH: + errCode = codes.AlreadyExists + case mimirpb.TOO_MANY_CLUSTERS: + errCode = codes.FailedPrecondition } - errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.REQUEST_RATE_LIMITED} - case errors.As(pushErr, &replicasDidNotMatchError{}): - errCode = codes.AlreadyExists - errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.REPLICAS_DID_NOT_MATCH} - case errors.As(pushErr, &tooManyClustersError{}): - errCode = codes.FailedPrecondition - errDetails = &mimirpb.WriteErrorDetails{Cause: mimirpb.TOO_MANY_CLUSTERS} } stat := status.New(errCode, pushErr.Error()) if errDetails != nil { diff --git a/pkg/distributor/errors_test.go b/pkg/distributor/errors_test.go index 673771c4908..04720fe439e 100644 --- a/pkg/distributor/errors_test.go +++ b/pkg/distributor/errors_test.go @@ -23,6 +23,7 @@ func TestNewReplicasNotMatchError(t *testing.T) { assert.Error(t, err) expectedMsg := fmt.Sprintf("replicas did not match, rejecting sample: replica=%s, elected=%s", replica, elected) assert.EqualError(t, err, expectedMsg) + checkDistributorError(t, err, mimirpb.REPLICAS_DID_NOT_MATCH) anotherErr := newReplicasDidNotMatchError("c", "d") assert.NotErrorIs(t, err, anotherErr) @@ -33,6 +34,7 @@ func TestNewReplicasNotMatchError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &replicasDidNotMatchError{})) + checkDistributorError(t, wrappedErr, mimirpb.REPLICAS_DID_NOT_MATCH) } func TestNewTooManyClustersError(t *testing.T) { @@ -41,6 +43,7 @@ func TestNewTooManyClustersError(t *testing.T) { expectedErrorMsg := fmt.Sprintf(tooManyClustersMsgFormat, limit) assert.Error(t, err) assert.EqualError(t, err, expectedErrorMsg) + checkDistributorError(t, err, mimirpb.TOO_MANY_CLUSTERS) anotherErr := newTooManyClustersError(20) assert.NotErrorIs(t, err, anotherErr) @@ -51,6 +54,7 @@ func TestNewTooManyClustersError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &tooManyClustersError{})) + checkDistributorError(t, wrappedErr, mimirpb.TOO_MANY_CLUSTERS) } func TestNewValidationError(t *testing.T) { @@ -60,6 +64,7 @@ func TestNewValidationError(t *testing.T) { err := newValidationError(firstErr) assert.Error(t, err) assert.EqualError(t, err, validationMsg) + checkDistributorError(t, err, mimirpb.VALIDATION) anotherErr := newValidationError(errors.New("this is another validation error")) assert.NotErrorIs(t, err, anotherErr) @@ -70,6 +75,7 @@ func TestNewValidationError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &validationError{})) + checkDistributorError(t, wrappedErr, mimirpb.VALIDATION) } func TestNewIngestionRateError(t *testing.T) { @@ -79,6 +85,7 @@ func TestNewIngestionRateError(t *testing.T) { expectedErrorMsg := fmt.Sprintf(ingestionRateLimitedMsgFormat, limit, burst) assert.Error(t, err) assert.EqualError(t, err, expectedErrorMsg) + checkDistributorError(t, err, mimirpb.INGESTION_RATE_LIMITED) anotherErr := newIngestionRateLimitedError(20, 20) assert.NotErrorIs(t, err, anotherErr) @@ -89,6 +96,7 @@ func TestNewIngestionRateError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &ingestionRateLimitedError{})) + checkDistributorError(t, wrappedErr, mimirpb.INGESTION_RATE_LIMITED) } func TestNewRequestRateError(t *testing.T) { @@ -98,6 +106,7 @@ func TestNewRequestRateError(t *testing.T) { expectedErrorMsg := fmt.Sprintf(requestRateLimitedMsgFormat, limit, burst) assert.Error(t, err) assert.EqualError(t, err, expectedErrorMsg) + checkDistributorError(t, err, mimirpb.REQUEST_RATE_LIMITED) anotherErr := newRequestRateLimitedError(20, 20) assert.NotErrorIs(t, err, anotherErr) @@ -108,6 +117,7 @@ func TestNewRequestRateError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &requestRateLimitedError{})) + checkDistributorError(t, wrappedErr, mimirpb.REQUEST_RATE_LIMITED) } func TestToGRPCError(t *testing.T) { @@ -243,3 +253,9 @@ func TestToGRPCError(t *testing.T) { } } } + +func checkDistributorError(t *testing.T, err error, expectedCause mimirpb.ErrorCause) { + var distributorErr distributorError + require.ErrorAs(t, err, &distributorErr) + require.Equal(t, expectedCause, distributorErr.errorCause()) +} diff --git a/pkg/distributor/push.go b/pkg/distributor/push.go index 3d568c00fee..f8030b43cde 100644 --- a/pkg/distributor/push.go +++ b/pkg/distributor/push.go @@ -153,32 +153,35 @@ func toHTTPStatus(ctx context.Context, pushErr error, limits *validation.Overrid return http.StatusInternalServerError } - switch { - case errors.As(pushErr, &validationError{}): - return http.StatusBadRequest - case errors.As(pushErr, &ingestionRateLimitedError{}): - // Return a 429 here to tell the client it is going too fast. - // Client may discard the data or slow down and re-send. - // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. - return http.StatusTooManyRequests - case errors.As(pushErr, &requestRateLimitedError{}): - serviceOverloadErrorEnabled := false - userID, err := tenant.TenantID(ctx) - if err == nil { - serviceOverloadErrorEnabled = limits.ServiceOverloadStatusCodeOnRateLimitEnabled(userID) - } - // Return a 429 or a 529 here depending on configuration to tell the client it is going too fast. - // Client may discard the data or slow down and re-send. - // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. - if serviceOverloadErrorEnabled { - return StatusServiceOverloaded + var distributorErr distributorError + if errors.As(pushErr, &distributorErr) { + switch distributorErr.errorCause() { + case mimirpb.VALIDATION: + return http.StatusBadRequest + case mimirpb.INGESTION_RATE_LIMITED: + // Return a 429 here to tell the client it is going too fast. + // Client may discard the data or slow down and re-send. + // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. + return http.StatusTooManyRequests + case mimirpb.REQUEST_RATE_LIMITED: + serviceOverloadErrorEnabled := false + userID, err := tenant.TenantID(ctx) + if err == nil { + serviceOverloadErrorEnabled = limits.ServiceOverloadStatusCodeOnRateLimitEnabled(userID) + } + // Return a 429 or a 529 here depending on configuration to tell the client it is going too fast. + // Client may discard the data or slow down and re-send. + // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. + if serviceOverloadErrorEnabled { + return StatusServiceOverloaded + } + return http.StatusTooManyRequests + case mimirpb.REPLICAS_DID_NOT_MATCH: + return http.StatusAccepted + case mimirpb.TOO_MANY_CLUSTERS: + return http.StatusBadRequest } - return http.StatusTooManyRequests - case errors.As(pushErr, &replicasDidNotMatchError{}): - return http.StatusAccepted - case errors.As(pushErr, &tooManyClustersError{}): - return http.StatusBadRequest - default: - return http.StatusInternalServerError } + + return http.StatusInternalServerError } From 8c12c0737ccf9f34d4739238869c90bd4d5c64aa Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Mon, 16 Oct 2023 12:12:10 +0200 Subject: [PATCH 3/6] Ingester gRPC errors should be propagated as they are Signed-off-by: Yuri Nikolic --- pkg/distributor/distributor.go | 9 +++++++-- pkg/distributor/distributor_test.go | 27 ++++++++++++++------------- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 5791c0ee0c1..844656ebd78 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -39,7 +39,6 @@ import ( "go.uber.org/atomic" "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" - "google.golang.org/grpc/codes" "github.com/grafana/mimir/pkg/cardinality" ingester_client "github.com/grafana/mimir/pkg/ingester/client" @@ -1092,7 +1091,7 @@ func (d *Distributor) handlePushError(ctx context.Context, pushErr error) error } if errors.Is(pushErr, context.DeadlineExceeded) { - return status.Error(codes.DeadlineExceeded, pushErr.Error()) + return pushErr } // TODO This code is needed for backwards compatibility, since ingesters may still return @@ -1102,6 +1101,12 @@ func (d *Distributor) handlePushError(ctx context.Context, pushErr error) error if ok { return pushErr } + // If pushErr is already a gRPC (for example returned by the ingester), we just propagate it. + // TODO this should be updated once the ingester error handling is improved. + _, ok = status.FromError(pushErr) + if ok { + return pushErr + } serviceOverloadErrorEnabled := false userID, err := tenant.TenantID(ctx) diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index 6e0abad33e7..b04fa7c5c8f 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -278,12 +278,12 @@ func TestDistributor_Push(t *testing.T) { `, }, "A timed out push should fail": { - numIngesters: 3, - happyIngesters: 3, - samples: samplesIn{num: 10, startTimestampMs: 123456789000}, - timeOut: true, - expectedGRPCError: status.New(codes.DeadlineExceeded, "exceeded configured distributor remote timeout: failed pushing to ingester: context deadline exceeded"), - metricNames: []string{lastSeenTimestamp}, + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 10, startTimestampMs: 123456789000}, + timeOut: true, + expectedError: errors.New("exceeded configured distributor remote timeout: failed pushing to ingester: context deadline exceeded"), + metricNames: []string{lastSeenTimestamp}, expectedMetrics: ` # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge @@ -315,9 +315,6 @@ func TestDistributor_Push(t *testing.T) { assert.Nil(t, response) if tc.expectedGRPCError == nil { - // Assert that downstream gRPC statuses are passed back upstream - _, ok := httpgrpc.HTTPResponseFromError(err) - assert.True(t, ok, fmt.Sprintf("expected error to be an httpgrpc error, but got: %T", err)) assert.EqualError(t, err, tc.expectedError.Error()) } else { checkGRPCError(t, tc.expectedGRPCError, tc.expectedErrorDetails, err) @@ -4840,13 +4837,13 @@ func TestHandlePushError(t *testing.T) { expectedGRPCError *status.Status expectedOtherError error }{ - "a context.Canceled error gives a gRPC Canceled error": { + "a context.Canceled error gives context.Canceled": { pushError: context.Canceled, expectedOtherError: context.Canceled, }, - "a context.DeadlineExceeded error gives a gRPC DeadlineExceeded error": { - pushError: context.DeadlineExceeded, - expectedGRPCError: status.New(codes.DeadlineExceeded, context.DeadlineExceeded.Error()), + "a context.DeadlineExceeded error gives context.DeadlineExceeded": { + pushError: context.DeadlineExceeded, + expectedOtherError: context.DeadlineExceeded, }, "a 4xx HTTP gRPC error gives the same 4xx HTTP gRPC error": { pushError: httpGrpc4xxErr, @@ -4860,6 +4857,10 @@ func TestHandlePushError(t *testing.T) { pushError: errWithUserID, expectedGRPCError: status.New(codes.Internal, errWithUserID.Error()), }, + "a random ingester gRPC error gives the same gRPC errpr": { + pushError: status.Error(codes.Unavailable, testErrorMsg), + expectedGRPCError: status.New(codes.Unavailable, testErrorMsg), + }, } config := prepConfig{ From 7e6e38da81e173eb5cbf5bdc6d889fa6ac451277 Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Mon, 16 Oct 2023 12:27:42 +0200 Subject: [PATCH 4/6] Fixing review findings Signed-off-by: Yuri Nikolic --- pkg/distributor/errors.go | 15 ++- pkg/mimirpb/mimir.pb.go | 273 +++++++++++++++++++------------------- pkg/mimirpb/mimir.proto | 11 +- 3 files changed, 154 insertions(+), 145 deletions(-) diff --git a/pkg/distributor/errors.go b/pkg/distributor/errors.go index ed68ac21520..fb464b95ca0 100644 --- a/pkg/distributor/errors.go +++ b/pkg/distributor/errors.go @@ -61,11 +61,12 @@ func (e replicasDidNotMatchError) Error() string { return fmt.Sprintf("replicas did not match, rejecting sample: replica=%s, elected=%s", e.replica, e.elected) } -// replicasDidNotMatchError implements the distributorError interface. func (e replicasDidNotMatchError) errorCause() mimirpb.ErrorCause { return mimirpb.REPLICAS_DID_NOT_MATCH } +var _ distributorError = replicasDidNotMatchError{} + // tooManyClustersError is an error stating that there are too many HA clusters. type tooManyClustersError struct { limit int @@ -82,11 +83,12 @@ func (e tooManyClustersError) Error() string { return fmt.Sprintf(tooManyClustersMsgFormat, e.limit) } -// tooManyClustersError implements the distributorError interface. func (e tooManyClustersError) errorCause() mimirpb.ErrorCause { return mimirpb.TOO_MANY_CLUSTERS } +var _ distributorError = tooManyClustersError{} + // validationError is an error, used to represent all validation errors from the validation package. type validationError struct { error @@ -97,11 +99,12 @@ func newValidationError(err error) validationError { return validationError{error: err} } -// validationError implements the distributorError interface. func (e validationError) errorCause() mimirpb.ErrorCause { return mimirpb.VALIDATION } +var _ distributorError = validationError{} + // ingestionRateLimitedError is an error used to represent the ingestion rate limited error. type ingestionRateLimitedError struct { limit float64 @@ -120,11 +123,12 @@ func (e ingestionRateLimitedError) Error() string { return fmt.Sprintf(ingestionRateLimitedMsgFormat, e.limit, e.burst) } -// ingestionRateLimitedError implements the distributorError interface. func (e ingestionRateLimitedError) errorCause() mimirpb.ErrorCause { return mimirpb.INGESTION_RATE_LIMITED } +var _ distributorError = ingestionRateLimitedError{} + // requestRateLimitedError is an error used to represent the request rate limited error. type requestRateLimitedError struct { limit float64 @@ -143,11 +147,12 @@ func (e requestRateLimitedError) Error() string { return fmt.Sprintf(requestRateLimitedMsgFormat, e.limit, e.burst) } -// requestRateLimitedError implements the distributorError interface. func (e requestRateLimitedError) errorCause() mimirpb.ErrorCause { return mimirpb.REQUEST_RATE_LIMITED } +var _ distributorError = requestRateLimitedError{} + // toGRPCError converts the given error into an appropriate gRPC error. func toGRPCError(pushErr error, serviceOverloadErrorEnabled bool) error { var ( diff --git a/pkg/mimirpb/mimir.pb.go b/pkg/mimirpb/mimir.pb.go index 6ef3ea63e6f..333bb74112e 100644 --- a/pkg/mimirpb/mimir.pb.go +++ b/pkg/mimirpb/mimir.pb.go @@ -32,27 +32,30 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type ErrorCause int32 const ( - REPLICAS_DID_NOT_MATCH ErrorCause = 0 - TOO_MANY_CLUSTERS ErrorCause = 1 - VALIDATION ErrorCause = 2 - INGESTION_RATE_LIMITED ErrorCause = 3 - REQUEST_RATE_LIMITED ErrorCause = 4 + INVALID ErrorCause = 0 + REPLICAS_DID_NOT_MATCH ErrorCause = 1 + TOO_MANY_CLUSTERS ErrorCause = 2 + VALIDATION ErrorCause = 3 + INGESTION_RATE_LIMITED ErrorCause = 4 + REQUEST_RATE_LIMITED ErrorCause = 5 ) var ErrorCause_name = map[int32]string{ - 0: "REPLICAS_DID_NOT_MATCH", - 1: "TOO_MANY_CLUSTERS", - 2: "VALIDATION", - 3: "INGESTION_RATE_LIMITED", - 4: "REQUEST_RATE_LIMITED", + 0: "INVALID", + 1: "REPLICAS_DID_NOT_MATCH", + 2: "TOO_MANY_CLUSTERS", + 3: "VALIDATION", + 4: "INGESTION_RATE_LIMITED", + 5: "REQUEST_RATE_LIMITED", } var ErrorCause_value = map[string]int32{ - "REPLICAS_DID_NOT_MATCH": 0, - "TOO_MANY_CLUSTERS": 1, - "VALIDATION": 2, - "INGESTION_RATE_LIMITED": 3, - "REQUEST_RATE_LIMITED": 4, + "INVALID": 0, + "REPLICAS_DID_NOT_MATCH": 1, + "TOO_MANY_CLUSTERS": 2, + "VALIDATION": 3, + "INGESTION_RATE_LIMITED": 4, + "REQUEST_RATE_LIMITED": 5, } func (ErrorCause) EnumDescriptor() ([]byte, []int) { @@ -347,7 +350,7 @@ func (m *WriteErrorDetails) GetCause() ErrorCause { if m != nil { return m.Cause } - return REPLICAS_DID_NOT_MATCH + return INVALID } type TimeSeries struct { @@ -1862,125 +1865,125 @@ func init() { func init() { proto.RegisterFile("mimir.proto", fileDescriptor_86d4d7485f544059) } var fileDescriptor_86d4d7485f544059 = []byte{ - // 1876 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcf, 0x93, 0xdb, 0x48, - 0x15, 0xb6, 0x6c, 0x8d, 0x6d, 0xbd, 0xf1, 0x78, 0x94, 0xde, 0x10, 0xbc, 0x53, 0x1b, 0xcf, 0x44, - 0x14, 0xcb, 0x90, 0x82, 0x09, 0xb5, 0x0b, 0xd9, 0xda, 0xad, 0x50, 0x8b, 0x6c, 0x2b, 0x19, 0xcf, - 0xfa, 0xc7, 0x6c, 0x5b, 0xce, 0x12, 0x2e, 0x2a, 0x8d, 0xa7, 0x67, 0xac, 0x5a, 0xc9, 0x32, 0x92, - 0x1c, 0x32, 0x9c, 0xb8, 0x40, 0x51, 0x9c, 0xb8, 0x70, 0xa1, 0xb8, 0x71, 0x80, 0xbf, 0x80, 0xbf, - 0x21, 0x55, 0x14, 0x55, 0x39, 0x6e, 0x71, 0x48, 0x91, 0xc9, 0x65, 0x8f, 0x39, 0x70, 0xe2, 0x44, - 0xf5, 0x6b, 0xfd, 0xb0, 0x3c, 0x33, 0x10, 0xd8, 0xdc, 0xd4, 0xef, 0x7d, 0xfd, 0xf4, 0xf5, 0xeb, - 0xef, 0x3d, 0x3f, 0x19, 0xd6, 0x3d, 0xc7, 0x73, 0x82, 0xbd, 0x79, 0xe0, 0x47, 0x3e, 0xa9, 0x4e, - 0xfc, 0x20, 0x62, 0x4f, 0xe6, 0x47, 0x5b, 0xdf, 0x3d, 0x75, 0xa2, 0xe9, 0xe2, 0x68, 0x6f, 0xe2, - 0x7b, 0x77, 0x4e, 0xfd, 0x53, 0xff, 0x0e, 0x02, 0x8e, 0x16, 0x27, 0xb8, 0xc2, 0x05, 0x3e, 0x89, - 0x8d, 0xda, 0x5f, 0x8a, 0x50, 0xfb, 0x2c, 0x70, 0x22, 0x46, 0xd9, 0x4f, 0x17, 0x2c, 0x8c, 0xc8, - 0x21, 0x40, 0xe4, 0x78, 0x2c, 0x64, 0x81, 0xc3, 0xc2, 0x86, 0xb4, 0x53, 0xda, 0x5d, 0x7f, 0xef, - 0xfa, 0x5e, 0x12, 0x7e, 0xcf, 0x74, 0x3c, 0x36, 0x42, 0x5f, 0x6b, 0xeb, 0xe9, 0xf3, 0xed, 0xc2, - 0xdf, 0x9f, 0x6f, 0x93, 0xc3, 0x80, 0xd9, 0xae, 0xeb, 0x4f, 0xcc, 0x74, 0x1f, 0x5d, 0x8a, 0x41, - 0x3e, 0x84, 0xf2, 0xc8, 0x5f, 0x04, 0x13, 0xd6, 0x28, 0xee, 0x48, 0xbb, 0xf5, 0xf7, 0x6e, 0x65, - 0xd1, 0x96, 0xdf, 0xbc, 0x27, 0x40, 0xc6, 0x6c, 0xe1, 0xd1, 0x78, 0x03, 0xf9, 0x08, 0xaa, 0x1e, - 0x8b, 0xec, 0x63, 0x3b, 0xb2, 0x1b, 0x25, 0xa4, 0xd2, 0xc8, 0x36, 0xf7, 0x59, 0x14, 0x38, 0x93, - 0x7e, 0xec, 0x6f, 0xc9, 0x4f, 0x9f, 0x6f, 0x4b, 0x34, 0xc5, 0x93, 0x7b, 0xb0, 0x15, 0x7e, 0xee, - 0xcc, 0x2d, 0xd7, 0x3e, 0x62, 0xae, 0x35, 0xb3, 0x3d, 0x66, 0x3d, 0xb6, 0x5d, 0xe7, 0xd8, 0x8e, - 0x1c, 0x7f, 0xd6, 0xf8, 0xb2, 0xb2, 0x23, 0xed, 0x56, 0xe9, 0xd7, 0x39, 0xa4, 0xc7, 0x11, 0x03, - 0xdb, 0x63, 0x0f, 0x53, 0xbf, 0xb6, 0x0d, 0x90, 0xf1, 0x21, 0x15, 0x28, 0xe9, 0x87, 0x5d, 0xb5, - 0x40, 0xaa, 0x20, 0xd3, 0x71, 0xcf, 0x50, 0x25, 0x6d, 0x13, 0x36, 0x62, 0xf6, 0xe1, 0xdc, 0x9f, - 0x85, 0x4c, 0xfb, 0x18, 0xae, 0xa1, 0xc1, 0x08, 0x02, 0x3f, 0xe8, 0xb0, 0xc8, 0x76, 0xdc, 0x90, - 0xdc, 0x86, 0xb5, 0xb6, 0xbd, 0x08, 0x59, 0x43, 0xc2, 0xa3, 0x2f, 0x25, 0x12, 0x61, 0xe8, 0xa3, - 0x02, 0xa2, 0xfd, 0x53, 0x02, 0xc8, 0xd2, 0x4b, 0x74, 0x28, 0x23, 0xf5, 0xe4, 0x12, 0xde, 0xca, - 0xf6, 0x22, 0xe1, 0x43, 0xdb, 0x09, 0x5a, 0xd7, 0xe3, 0x3b, 0xa8, 0xa1, 0x49, 0x3f, 0xb6, 0xe7, - 0x11, 0x0b, 0x68, 0xbc, 0x91, 0x7c, 0x0f, 0x2a, 0xa1, 0xed, 0xcd, 0x5d, 0x16, 0x36, 0x8a, 0x18, - 0x43, 0xcd, 0x62, 0x8c, 0xd0, 0x81, 0x59, 0x2b, 0xd0, 0x04, 0x46, 0xee, 0x82, 0xc2, 0x9e, 0x30, - 0x6f, 0xee, 0xda, 0x41, 0x18, 0x67, 0x9c, 0x2c, 0x71, 0x8e, 0x5d, 0xf1, 0xae, 0x0c, 0x4a, 0x3e, - 0x04, 0x98, 0x3a, 0x61, 0xe4, 0x9f, 0x06, 0xb6, 0x17, 0x36, 0xe4, 0x55, 0xc2, 0xfb, 0x89, 0x2f, - 0xde, 0xb9, 0x04, 0xd6, 0x7e, 0x00, 0x4a, 0x7a, 0x1e, 0x42, 0x40, 0xe6, 0x37, 0x85, 0xe9, 0xaa, - 0x51, 0x7c, 0x26, 0xd7, 0x61, 0xed, 0xb1, 0xed, 0x2e, 0x84, 0x7c, 0x6a, 0x54, 0x2c, 0x34, 0x1d, - 0xca, 0xe2, 0x08, 0xe4, 0x16, 0xd4, 0x50, 0x6d, 0x91, 0xed, 0xcd, 0x2d, 0x2f, 0x44, 0x58, 0x89, - 0xae, 0xa7, 0xb6, 0x7e, 0x98, 0x85, 0xe0, 0x71, 0xa5, 0x24, 0xc4, 0xef, 0x8b, 0x50, 0xcf, 0x8b, - 0x88, 0x7c, 0x00, 0x72, 0x74, 0x36, 0x4f, 0xae, 0xeb, 0x1b, 0x57, 0x89, 0x2d, 0x5e, 0x9a, 0x67, - 0x73, 0x46, 0x71, 0x03, 0xf9, 0x0e, 0x10, 0x0f, 0x6d, 0xd6, 0x89, 0xed, 0x39, 0xee, 0x19, 0x0a, - 0x0e, 0xa9, 0x28, 0x54, 0x15, 0x9e, 0xfb, 0xe8, 0xe0, 0x3a, 0xe3, 0xc7, 0x9c, 0x32, 0x77, 0xde, - 0x90, 0xd1, 0x8f, 0xcf, 0xdc, 0xb6, 0x98, 0x39, 0x51, 0x63, 0x4d, 0xd8, 0xf8, 0xb3, 0x76, 0x06, - 0x90, 0xbd, 0x89, 0xac, 0x43, 0x65, 0x3c, 0xf8, 0x64, 0x30, 0xfc, 0x6c, 0xa0, 0x16, 0xf8, 0xa2, - 0x3d, 0x1c, 0x0f, 0x4c, 0x83, 0xaa, 0x12, 0x51, 0x60, 0xed, 0x81, 0x3e, 0x7e, 0x60, 0xa8, 0x45, - 0xb2, 0x01, 0xca, 0x7e, 0x77, 0x64, 0x0e, 0x1f, 0x50, 0xbd, 0xaf, 0x96, 0x08, 0x81, 0x3a, 0x7a, - 0x32, 0x9b, 0xcc, 0xb7, 0x8e, 0xc6, 0xfd, 0xbe, 0x4e, 0x1f, 0xa9, 0x6b, 0x5c, 0xd1, 0xdd, 0xc1, - 0xfd, 0xa1, 0x5a, 0x26, 0x35, 0xa8, 0x8e, 0x4c, 0xdd, 0x34, 0x46, 0x86, 0xa9, 0x56, 0xb4, 0x4f, - 0xa0, 0x2c, 0x5e, 0xfd, 0x06, 0x84, 0xa8, 0xfd, 0x4a, 0x82, 0x6a, 0x22, 0x9e, 0x37, 0x21, 0xec, - 0x9c, 0x24, 0x92, 0xfb, 0xbc, 0x20, 0x84, 0xd2, 0x05, 0x21, 0x68, 0x7f, 0x5d, 0x03, 0x25, 0x15, - 0x23, 0xb9, 0x09, 0xca, 0xc4, 0x5f, 0xcc, 0x22, 0xcb, 0x99, 0x45, 0x78, 0xe5, 0xf2, 0x7e, 0x81, - 0x56, 0xd1, 0xd4, 0x9d, 0x45, 0xe4, 0x16, 0xac, 0x0b, 0xf7, 0x89, 0xeb, 0xdb, 0x91, 0x78, 0xd7, - 0x7e, 0x81, 0x02, 0x1a, 0xef, 0x73, 0x1b, 0x51, 0xa1, 0x14, 0x2e, 0x3c, 0x7c, 0x93, 0x44, 0xf9, - 0x23, 0xb9, 0x01, 0xe5, 0x70, 0x32, 0x65, 0x9e, 0x8d, 0x97, 0x7b, 0x8d, 0xc6, 0x2b, 0xf2, 0x4d, - 0xa8, 0xff, 0x9c, 0x05, 0xbe, 0x15, 0x4d, 0x03, 0x16, 0x4e, 0x7d, 0xf7, 0x18, 0x2f, 0x5a, 0xa2, - 0x1b, 0xdc, 0x6a, 0x26, 0x46, 0xf2, 0x6e, 0x0c, 0xcb, 0x78, 0x95, 0x91, 0x97, 0x44, 0x6b, 0xdc, - 0xde, 0x4e, 0xb8, 0xdd, 0x06, 0x75, 0x09, 0x27, 0x08, 0x56, 0x90, 0xa0, 0x44, 0xeb, 0x29, 0x52, - 0x90, 0xd4, 0xa1, 0x3e, 0x63, 0xa7, 0x76, 0xe4, 0x3c, 0x66, 0x56, 0x38, 0xb7, 0x67, 0x61, 0xa3, - 0xba, 0xda, 0xd6, 0x5b, 0x8b, 0xc9, 0xe7, 0x2c, 0x1a, 0xcd, 0xed, 0x59, 0x5c, 0xa1, 0x1b, 0xc9, - 0x0e, 0x6e, 0x0b, 0xc9, 0xb7, 0x60, 0x33, 0x0d, 0x71, 0xcc, 0xdc, 0xc8, 0x0e, 0x1b, 0xca, 0x4e, - 0x69, 0x97, 0xd0, 0x34, 0x72, 0x07, 0xad, 0x39, 0x20, 0x72, 0x0b, 0x1b, 0xb0, 0x53, 0xda, 0x95, - 0x32, 0x20, 0x12, 0xe3, 0xed, 0xad, 0x3e, 0xf7, 0x43, 0x67, 0x89, 0xd4, 0xfa, 0x7f, 0x27, 0x95, - 0xec, 0x48, 0x49, 0xa5, 0x21, 0x62, 0x52, 0x35, 0x41, 0x2a, 0x31, 0x67, 0xa4, 0x52, 0x60, 0x4c, - 0x6a, 0x43, 0x90, 0x4a, 0xcc, 0x31, 0xa9, 0x7b, 0x00, 0x01, 0x0b, 0x59, 0x64, 0x4d, 0x79, 0xe6, - 0xeb, 0xd8, 0x04, 0x6e, 0x5e, 0xd2, 0xc6, 0xf6, 0x28, 0x47, 0xed, 0x3b, 0xb3, 0x88, 0x2a, 0x41, - 0xf2, 0x48, 0xde, 0x01, 0x25, 0xd5, 0x5a, 0x63, 0x13, 0xc5, 0x97, 0x19, 0xb4, 0x8f, 0x40, 0x49, - 0x77, 0xe5, 0x4b, 0xb9, 0x02, 0xa5, 0x47, 0xc6, 0x48, 0x95, 0x48, 0x19, 0x8a, 0x83, 0xa1, 0x5a, - 0xcc, 0xca, 0xb9, 0xb4, 0x25, 0xff, 0xfa, 0x8f, 0x4d, 0xa9, 0x55, 0x81, 0x35, 0xe4, 0xdd, 0xaa, - 0x01, 0x64, 0xd7, 0xae, 0xfd, 0x4d, 0x86, 0x3a, 0x5e, 0x71, 0x26, 0xe9, 0x10, 0x08, 0xfa, 0x58, - 0x60, 0xad, 0x9c, 0x64, 0xa3, 0x65, 0xfc, 0xeb, 0xf9, 0xb6, 0xbe, 0x34, 0x1e, 0xcc, 0x03, 0xdf, - 0x63, 0xd1, 0x94, 0x2d, 0xc2, 0xe5, 0x47, 0xcf, 0x3f, 0x66, 0xee, 0x9d, 0xb4, 0x41, 0xef, 0xb5, - 0x45, 0xb8, 0xec, 0xc4, 0xea, 0x64, 0xc5, 0xf2, 0x55, 0x35, 0x7f, 0x73, 0xf9, 0x50, 0x42, 0xc5, - 0x54, 0x49, 0x35, 0xcc, 0x8b, 0x5d, 0x78, 0xe2, 0x62, 0xc7, 0xc5, 0x25, 0x95, 0xf7, 0x06, 0x14, - 0xf5, 0x06, 0x2a, 0xe5, 0xdb, 0xa0, 0xa6, 0x2c, 0x8e, 0x10, 0x9b, 0x88, 0x2d, 0xd5, 0xa0, 0x08, - 0x81, 0xd0, 0xf4, 0x6d, 0x09, 0x54, 0x14, 0x4b, 0x5a, 0x43, 0x31, 0xf4, 0x40, 0xae, 0x4a, 0x6a, - 0xf1, 0x40, 0xae, 0x96, 0xd5, 0xca, 0x81, 0x5c, 0x55, 0x54, 0x38, 0x90, 0xab, 0x35, 0x75, 0xe3, - 0x40, 0xae, 0x6e, 0xaa, 0x2a, 0xcd, 0xba, 0x18, 0x5d, 0xe9, 0x1e, 0x74, 0xb5, 0x6c, 0xe9, 0x6a, - 0xc9, 0x2c, 0x4b, 0xf4, 0x1e, 0x40, 0x76, 0x3c, 0x7e, 0xab, 0xfe, 0xc9, 0x49, 0xc8, 0x44, 0x6b, - 0xbc, 0x46, 0xe3, 0x15, 0xb7, 0xbb, 0x6c, 0x76, 0x1a, 0x4d, 0xf1, 0x42, 0x36, 0x68, 0xbc, 0xd2, - 0x16, 0x40, 0xf2, 0x62, 0xc4, 0x5f, 0xf4, 0xd7, 0xf8, 0x75, 0xbe, 0x07, 0x4a, 0x2a, 0x37, 0x7c, - 0x57, 0x6e, 0xcc, 0xcb, 0xc7, 0x8c, 0xc7, 0xbc, 0x6c, 0x83, 0x36, 0x83, 0x4d, 0x31, 0x08, 0x64, - 0x45, 0x90, 0x2a, 0x46, 0xba, 0x44, 0x31, 0xc5, 0x4c, 0x31, 0xef, 0x43, 0x25, 0xc9, 0xbb, 0x98, - 0x75, 0xde, 0xbe, 0x6c, 0x64, 0x41, 0x04, 0x4d, 0x90, 0x5a, 0x08, 0x9b, 0x2b, 0x3e, 0xd2, 0x04, - 0x38, 0xf2, 0x17, 0xb3, 0x63, 0x3b, 0x9e, 0x99, 0xa5, 0xdd, 0x35, 0xba, 0x64, 0xe1, 0x7c, 0x5c, - 0xff, 0x67, 0x2c, 0x48, 0x14, 0x8c, 0x0b, 0x6e, 0x5d, 0xcc, 0xe7, 0x2c, 0x88, 0x35, 0x2c, 0x16, - 0x19, 0x77, 0x79, 0x89, 0xbb, 0xe6, 0xc2, 0x5b, 0x2b, 0x87, 0xc4, 0xe4, 0xe6, 0x3a, 0x4e, 0x71, - 0xa5, 0xe3, 0x90, 0x0f, 0x2e, 0xe6, 0xf5, 0xed, 0xd5, 0x01, 0x30, 0x8d, 0xb7, 0x9c, 0xd2, 0x3f, - 0xc9, 0xb0, 0xf1, 0xe9, 0x82, 0x05, 0x67, 0xc9, 0x70, 0x4b, 0xee, 0x42, 0x39, 0x8c, 0xec, 0x68, - 0x11, 0xc6, 0x93, 0x51, 0x33, 0x8b, 0x93, 0x03, 0xee, 0x8d, 0x10, 0x45, 0x63, 0x34, 0xf9, 0x11, - 0x00, 0xe3, 0x83, 0xae, 0x85, 0x53, 0xd5, 0x85, 0xf9, 0x3f, 0xbf, 0x17, 0x47, 0x62, 0x9c, 0xa9, - 0x14, 0x96, 0x3c, 0xf2, 0x7c, 0xe0, 0x02, 0xb3, 0xa4, 0x50, 0xb1, 0x20, 0x7b, 0x9c, 0x4f, 0xe0, - 0xcc, 0x4e, 0x31, 0x4d, 0xb9, 0x02, 0x1d, 0xa1, 0xbd, 0x63, 0x47, 0xf6, 0x7e, 0x81, 0xc6, 0x28, - 0x8e, 0x7f, 0xcc, 0x26, 0x91, 0x1f, 0x60, 0x07, 0xca, 0xe1, 0x1f, 0xa2, 0x3d, 0xc1, 0x0b, 0x14, - 0xc6, 0x9f, 0xd8, 0xae, 0x1d, 0xe0, 0xcf, 0x6f, 0x3e, 0x3e, 0xda, 0xd3, 0xf8, 0xb8, 0xe2, 0x78, - 0xcf, 0x8e, 0x02, 0xe7, 0x09, 0xb6, 0xaf, 0x1c, 0xbe, 0x8f, 0xf6, 0x04, 0x2f, 0x50, 0xda, 0xbb, - 0x50, 0x16, 0x99, 0xe2, 0xbd, 0xde, 0xa0, 0x74, 0x48, 0xc5, 0x48, 0x37, 0x1a, 0xb7, 0xdb, 0xc6, - 0x68, 0xa4, 0x4a, 0xa2, 0xf1, 0x6b, 0xbf, 0x93, 0x40, 0x49, 0xd3, 0xc2, 0x67, 0xb5, 0xc1, 0x70, - 0x60, 0x08, 0xa8, 0xd9, 0xed, 0x1b, 0xc3, 0xb1, 0xa9, 0x4a, 0x7c, 0x70, 0x6b, 0xeb, 0x83, 0xb6, - 0xd1, 0x33, 0x3a, 0x62, 0x00, 0x34, 0x7e, 0x6c, 0xb4, 0xc7, 0x66, 0x77, 0x38, 0x50, 0x4b, 0xdc, - 0xd9, 0xd2, 0x3b, 0x56, 0x47, 0x37, 0x75, 0x55, 0xe6, 0xab, 0x2e, 0x9f, 0x19, 0x07, 0x7a, 0x4f, - 0x5d, 0x23, 0x9b, 0xb0, 0x3e, 0x1e, 0xe8, 0x0f, 0xf5, 0x6e, 0x4f, 0x6f, 0xf5, 0x0c, 0xb5, 0xcc, - 0xf7, 0x0e, 0x86, 0xa6, 0x75, 0x7f, 0x38, 0x1e, 0x74, 0xd4, 0x0a, 0x1f, 0x1e, 0xf9, 0x52, 0x6f, - 0xb7, 0x8d, 0x43, 0x13, 0x21, 0xd5, 0xf8, 0x07, 0xa9, 0x0c, 0x32, 0x9f, 0x83, 0x35, 0x03, 0x20, - 0xcb, 0x77, 0x7e, 0xcc, 0x56, 0xae, 0x1a, 0xcb, 0x2e, 0x76, 0x00, 0xed, 0x97, 0x12, 0x40, 0x76, - 0x0f, 0xe4, 0x6e, 0xf6, 0xdd, 0x22, 0x46, 0xc4, 0x1b, 0xab, 0xd7, 0x75, 0xf9, 0xd7, 0xcb, 0xc7, - 0xb9, 0xaf, 0x90, 0xe2, 0x6a, 0x49, 0x8b, 0xad, 0xff, 0xe9, 0x5b, 0xc4, 0x82, 0xda, 0x72, 0x7c, - 0xde, 0xea, 0xc4, 0xec, 0x8e, 0x3c, 0x14, 0x1a, 0xaf, 0xfe, 0xff, 0xf9, 0xf3, 0x37, 0x12, 0x6c, - 0xae, 0xd0, 0xb8, 0xf2, 0x25, 0xb9, 0xb6, 0x58, 0x7c, 0x8d, 0xb6, 0x58, 0x58, 0xaa, 0xe1, 0xd7, - 0x21, 0xc3, 0x2f, 0x2f, 0x15, 0xf3, 0xe5, 0xdf, 0x48, 0xaf, 0x73, 0x79, 0x2d, 0x80, 0x4c, 0xe3, - 0xe4, 0xfb, 0x50, 0xce, 0xfd, 0x77, 0x70, 0x63, 0xb5, 0x12, 0xe2, 0x7f, 0x0f, 0x04, 0xe1, 0x18, - 0xab, 0xfd, 0x41, 0x82, 0xda, 0xb2, 0xfb, 0xca, 0xa4, 0xfc, 0xef, 0x9f, 0xb4, 0xad, 0x9c, 0x28, - 0x44, 0x9f, 0x7f, 0xe7, 0xaa, 0x3c, 0xe2, 0xb7, 0xc7, 0x05, 0x5d, 0xdc, 0xe6, 0xfa, 0xcc, 0x3e, - 0xd8, 0xc9, 0x16, 0xdc, 0xa0, 0xc6, 0x61, 0xaf, 0xdb, 0xd6, 0x47, 0x56, 0xa7, 0xdb, 0xb1, 0x78, - 0x91, 0xf4, 0x75, 0xb3, 0xbd, 0xaf, 0x16, 0xc8, 0xd7, 0xe0, 0x9a, 0x39, 0x1c, 0x5a, 0x7d, 0x7d, - 0xf0, 0xc8, 0x6a, 0xf7, 0xc6, 0x23, 0xd3, 0xa0, 0x7c, 0xb4, 0xab, 0x03, 0x3c, 0xd4, 0x7b, 0xdd, - 0x8e, 0x8e, 0x65, 0x59, 0xe4, 0x21, 0xba, 0x83, 0x07, 0xc6, 0x88, 0x2f, 0x2d, 0xaa, 0x9b, 0x86, - 0xd5, 0xeb, 0xf6, 0xbb, 0xa6, 0xd1, 0x51, 0x4b, 0xa4, 0x01, 0xd7, 0xa9, 0xf1, 0xe9, 0xd8, 0x18, - 0x99, 0x79, 0x8f, 0xdc, 0xfa, 0xe1, 0xb3, 0x17, 0xcd, 0xc2, 0x17, 0x2f, 0x9a, 0x85, 0x57, 0x2f, - 0x9a, 0xd2, 0x2f, 0xce, 0x9b, 0xd2, 0x9f, 0xcf, 0x9b, 0xd2, 0xd3, 0xf3, 0xa6, 0xf4, 0xec, 0xbc, - 0x29, 0xfd, 0xe3, 0xbc, 0x29, 0x7d, 0x79, 0xde, 0x2c, 0xbc, 0x3a, 0x6f, 0x4a, 0xbf, 0x7d, 0xd9, - 0x2c, 0x3c, 0x7b, 0xd9, 0x2c, 0x7c, 0xf1, 0xb2, 0x59, 0xf8, 0x49, 0x05, 0xff, 0x29, 0x9a, 0x1f, - 0x1d, 0x95, 0xf1, 0x3f, 0x9f, 0xf7, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x6e, 0x9d, 0x53, 0x64, - 0x3b, 0x12, 0x00, 0x00, + // 1885 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcd, 0x6f, 0x23, 0x59, + 0x11, 0x77, 0xdb, 0xed, 0xaf, 0x8a, 0xe3, 0xf4, 0xbc, 0x1d, 0x82, 0x37, 0xda, 0x75, 0x32, 0x8d, + 0x58, 0xc2, 0x08, 0x32, 0x68, 0x17, 0x66, 0xb5, 0xab, 0x41, 0x4b, 0xdb, 0xee, 0x99, 0x38, 0xeb, + 0x8f, 0xec, 0xeb, 0xf6, 0x2c, 0xc3, 0xa5, 0xd5, 0x71, 0x5e, 0x92, 0xd6, 0x76, 0xbb, 0x4d, 0x77, + 0x7b, 0x98, 0x70, 0xe2, 0x02, 0x42, 0x9c, 0x10, 0x12, 0x17, 0xc4, 0x8d, 0x03, 0xfc, 0x05, 0xfc, + 0x0d, 0x23, 0x21, 0xa4, 0x39, 0xae, 0x38, 0x8c, 0x98, 0xcc, 0x65, 0x8f, 0x7b, 0xe0, 0xc4, 0x09, + 0xbd, 0x7a, 0xfd, 0xe1, 0x76, 0x12, 0x18, 0x60, 0x6e, 0x5d, 0x55, 0xbf, 0xaa, 0xae, 0x57, 0xef, + 0x57, 0xe5, 0x6a, 0xc3, 0x9a, 0xe7, 0x78, 0x4e, 0xb0, 0x37, 0x0f, 0xfc, 0xc8, 0x27, 0xb5, 0xa9, + 0x1f, 0x44, 0xec, 0xc9, 0xfc, 0x68, 0xeb, 0xdb, 0xa7, 0x4e, 0x74, 0xb6, 0x38, 0xda, 0x9b, 0xfa, + 0xde, 0x9d, 0x53, 0xff, 0xd4, 0xbf, 0x83, 0x80, 0xa3, 0xc5, 0x09, 0x4a, 0x28, 0xe0, 0x93, 0x70, + 0x54, 0xff, 0x5c, 0x84, 0xc6, 0xa7, 0x81, 0x13, 0x31, 0xca, 0x7e, 0xbc, 0x60, 0x61, 0x44, 0x0e, + 0x01, 0x22, 0xc7, 0x63, 0x21, 0x0b, 0x1c, 0x16, 0xb6, 0xa4, 0x9d, 0xd2, 0xee, 0xda, 0xbb, 0x37, + 0xf7, 0x92, 0xf0, 0x7b, 0xa6, 0xe3, 0x31, 0x03, 0x6d, 0x9d, 0xad, 0xa7, 0xcf, 0xb7, 0x0b, 0x7f, + 0x7b, 0xbe, 0x4d, 0x0e, 0x03, 0x66, 0xbb, 0xae, 0x3f, 0x35, 0x53, 0x3f, 0xba, 0x14, 0x83, 0x7c, + 0x00, 0x15, 0xc3, 0x5f, 0x04, 0x53, 0xd6, 0x2a, 0xee, 0x48, 0xbb, 0xcd, 0x77, 0x6f, 0x65, 0xd1, + 0x96, 0xdf, 0xbc, 0x27, 0x40, 0xfa, 0x6c, 0xe1, 0xd1, 0xd8, 0x81, 0x7c, 0x08, 0x35, 0x8f, 0x45, + 0xf6, 0xb1, 0x1d, 0xd9, 0xad, 0x12, 0xa6, 0xd2, 0xca, 0x9c, 0x87, 0x2c, 0x0a, 0x9c, 0xe9, 0x30, + 0xb6, 0x77, 0xe4, 0xa7, 0xcf, 0xb7, 0x25, 0x9a, 0xe2, 0xc9, 0x3d, 0xd8, 0x0a, 0x3f, 0x73, 0xe6, + 0x96, 0x6b, 0x1f, 0x31, 0xd7, 0x9a, 0xd9, 0x1e, 0xb3, 0x1e, 0xdb, 0xae, 0x73, 0x6c, 0x47, 0x8e, + 0x3f, 0x6b, 0x7d, 0x51, 0xdd, 0x91, 0x76, 0x6b, 0xf4, 0xab, 0x1c, 0x32, 0xe0, 0x88, 0x91, 0xed, + 0xb1, 0x87, 0xa9, 0x5d, 0xdd, 0x06, 0xc8, 0xf2, 0x21, 0x55, 0x28, 0x69, 0x87, 0x7d, 0xa5, 0x40, + 0x6a, 0x20, 0xd3, 0xc9, 0x40, 0x57, 0x24, 0x75, 0x03, 0xd6, 0xe3, 0xec, 0xc3, 0xb9, 0x3f, 0x0b, + 0x99, 0xfa, 0x11, 0xdc, 0x40, 0x85, 0x1e, 0x04, 0x7e, 0xd0, 0x63, 0x91, 0xed, 0xb8, 0x21, 0xb9, + 0x0d, 0xe5, 0xae, 0xbd, 0x08, 0x59, 0x4b, 0xc2, 0xa3, 0x2f, 0x15, 0x12, 0x61, 0x68, 0xa3, 0x02, + 0xa2, 0xfe, 0x43, 0x02, 0xc8, 0xca, 0x4b, 0x34, 0xa8, 0x60, 0xea, 0xc9, 0x25, 0xbc, 0x91, 0xf9, + 0x62, 0xc2, 0x87, 0xb6, 0x13, 0x74, 0x6e, 0xc6, 0x77, 0xd0, 0x40, 0x95, 0x76, 0x6c, 0xcf, 0x23, + 0x16, 0xd0, 0xd8, 0x91, 0x7c, 0x07, 0xaa, 0xa1, 0xed, 0xcd, 0x5d, 0x16, 0xb6, 0x8a, 0x18, 0x43, + 0xc9, 0x62, 0x18, 0x68, 0xc0, 0xaa, 0x15, 0x68, 0x02, 0x23, 0x77, 0xa1, 0xce, 0x9e, 0x30, 0x6f, + 0xee, 0xda, 0x41, 0x18, 0x57, 0x9c, 0x2c, 0xe5, 0x1c, 0x9b, 0x62, 0xaf, 0x0c, 0x4a, 0x3e, 0x00, + 0x38, 0x73, 0xc2, 0xc8, 0x3f, 0x0d, 0x6c, 0x2f, 0x6c, 0xc9, 0xab, 0x09, 0xef, 0x27, 0xb6, 0xd8, + 0x73, 0x09, 0xac, 0x7e, 0x0f, 0xea, 0xe9, 0x79, 0x08, 0x01, 0x99, 0xdf, 0x14, 0x96, 0xab, 0x41, + 0xf1, 0x99, 0xdc, 0x84, 0xf2, 0x63, 0xdb, 0x5d, 0x08, 0xfa, 0x34, 0xa8, 0x10, 0x54, 0x0d, 0x2a, + 0xe2, 0x08, 0xe4, 0x16, 0x34, 0x90, 0x6d, 0x91, 0xed, 0xcd, 0x2d, 0x2f, 0x44, 0x58, 0x89, 0xae, + 0xa5, 0xba, 0x61, 0x98, 0x85, 0xe0, 0x71, 0xa5, 0x24, 0xc4, 0xef, 0x8a, 0xd0, 0xcc, 0x93, 0x88, + 0xbc, 0x0f, 0x72, 0x74, 0x3e, 0x4f, 0xae, 0xeb, 0x6b, 0xd7, 0x91, 0x2d, 0x16, 0xcd, 0xf3, 0x39, + 0xa3, 0xe8, 0x40, 0xbe, 0x05, 0xc4, 0x43, 0x9d, 0x75, 0x62, 0x7b, 0x8e, 0x7b, 0x8e, 0x84, 0xc3, + 0x54, 0xea, 0x54, 0x11, 0x96, 0xfb, 0x68, 0xe0, 0x3c, 0xe3, 0xc7, 0x3c, 0x63, 0xee, 0xbc, 0x25, + 0xa3, 0x1d, 0x9f, 0xb9, 0x6e, 0x31, 0x73, 0xa2, 0x56, 0x59, 0xe8, 0xf8, 0xb3, 0x7a, 0x0e, 0x90, + 0xbd, 0x89, 0xac, 0x41, 0x75, 0x32, 0xfa, 0x78, 0x34, 0xfe, 0x74, 0xa4, 0x14, 0xb8, 0xd0, 0x1d, + 0x4f, 0x46, 0xa6, 0x4e, 0x15, 0x89, 0xd4, 0xa1, 0xfc, 0x40, 0x9b, 0x3c, 0xd0, 0x95, 0x22, 0x59, + 0x87, 0xfa, 0x7e, 0xdf, 0x30, 0xc7, 0x0f, 0xa8, 0x36, 0x54, 0x4a, 0x84, 0x40, 0x13, 0x2d, 0x99, + 0x4e, 0xe6, 0xae, 0xc6, 0x64, 0x38, 0xd4, 0xe8, 0x23, 0xa5, 0xcc, 0x19, 0xdd, 0x1f, 0xdd, 0x1f, + 0x2b, 0x15, 0xd2, 0x80, 0x9a, 0x61, 0x6a, 0xa6, 0x6e, 0xe8, 0xa6, 0x52, 0x55, 0x3f, 0x86, 0x8a, + 0x78, 0xf5, 0x6b, 0x20, 0xa2, 0xfa, 0x0b, 0x09, 0x6a, 0x09, 0x79, 0x5e, 0x07, 0xb1, 0x73, 0x94, + 0x48, 0xee, 0xf3, 0x12, 0x11, 0x4a, 0x97, 0x88, 0xa0, 0xfe, 0xa5, 0x0c, 0xf5, 0x94, 0x8c, 0xe4, + 0x6d, 0xa8, 0x4f, 0xfd, 0xc5, 0x2c, 0xb2, 0x9c, 0x59, 0x84, 0x57, 0x2e, 0xef, 0x17, 0x68, 0x0d, + 0x55, 0xfd, 0x59, 0x44, 0x6e, 0xc1, 0x9a, 0x30, 0x9f, 0xb8, 0xbe, 0x1d, 0x89, 0x77, 0xed, 0x17, + 0x28, 0xa0, 0xf2, 0x3e, 0xd7, 0x11, 0x05, 0x4a, 0xe1, 0xc2, 0xc3, 0x37, 0x49, 0x94, 0x3f, 0x92, + 0x4d, 0xa8, 0x84, 0xd3, 0x33, 0xe6, 0xd9, 0x78, 0xb9, 0x37, 0x68, 0x2c, 0x91, 0xaf, 0x43, 0xf3, + 0xa7, 0x2c, 0xf0, 0xad, 0xe8, 0x2c, 0x60, 0xe1, 0x99, 0xef, 0x1e, 0xe3, 0x45, 0x4b, 0x74, 0x9d, + 0x6b, 0xcd, 0x44, 0x49, 0xde, 0x89, 0x61, 0x59, 0x5e, 0x15, 0xcc, 0x4b, 0xa2, 0x0d, 0xae, 0xef, + 0x26, 0xb9, 0xdd, 0x06, 0x65, 0x09, 0x27, 0x12, 0xac, 0x62, 0x82, 0x12, 0x6d, 0xa6, 0x48, 0x91, + 0xa4, 0x06, 0xcd, 0x19, 0x3b, 0xb5, 0x23, 0xe7, 0x31, 0xb3, 0xc2, 0xb9, 0x3d, 0x0b, 0x5b, 0xb5, + 0xd5, 0xb1, 0xde, 0x59, 0x4c, 0x3f, 0x63, 0x91, 0x31, 0xb7, 0x67, 0x71, 0x87, 0xae, 0x27, 0x1e, + 0x5c, 0x17, 0x92, 0x6f, 0xc0, 0x46, 0x1a, 0xe2, 0x98, 0xb9, 0x91, 0x1d, 0xb6, 0xea, 0x3b, 0xa5, + 0x5d, 0x42, 0xd3, 0xc8, 0x3d, 0xd4, 0xe6, 0x80, 0x98, 0x5b, 0xd8, 0x82, 0x9d, 0xd2, 0xae, 0x94, + 0x01, 0x31, 0x31, 0x3e, 0xde, 0x9a, 0x73, 0x3f, 0x74, 0x96, 0x92, 0x5a, 0xfb, 0xcf, 0x49, 0x25, + 0x1e, 0x69, 0x52, 0x69, 0x88, 0x38, 0xa9, 0x86, 0x48, 0x2a, 0x51, 0x67, 0x49, 0xa5, 0xc0, 0x38, + 0xa9, 0x75, 0x91, 0x54, 0xa2, 0x8e, 0x93, 0xba, 0x07, 0x10, 0xb0, 0x90, 0x45, 0xd6, 0x19, 0xaf, + 0x7c, 0x13, 0x87, 0xc0, 0xdb, 0x57, 0x8c, 0xb1, 0x3d, 0xca, 0x51, 0xfb, 0xce, 0x2c, 0xa2, 0xf5, + 0x20, 0x79, 0x24, 0x6f, 0x41, 0x3d, 0xe5, 0x5a, 0x6b, 0x03, 0xc9, 0x97, 0x29, 0xd4, 0x0f, 0xa1, + 0x9e, 0x7a, 0xe5, 0x5b, 0xb9, 0x0a, 0xa5, 0x47, 0xba, 0xa1, 0x48, 0xa4, 0x02, 0xc5, 0xd1, 0x58, + 0x29, 0x66, 0xed, 0x5c, 0xda, 0x92, 0x7f, 0xf9, 0x87, 0xb6, 0xd4, 0xa9, 0x42, 0x19, 0xf3, 0xee, + 0x34, 0x00, 0xb2, 0x6b, 0x57, 0xff, 0x2a, 0x43, 0x13, 0xaf, 0x38, 0xa3, 0x74, 0x08, 0x04, 0x6d, + 0x2c, 0xb0, 0x56, 0x4e, 0xb2, 0xde, 0xd1, 0xff, 0xf9, 0x7c, 0x5b, 0x5b, 0x5a, 0x0f, 0xe6, 0x81, + 0xef, 0xb1, 0xe8, 0x8c, 0x2d, 0xc2, 0xe5, 0x47, 0xcf, 0x3f, 0x66, 0xee, 0x9d, 0x74, 0x40, 0xef, + 0x75, 0x45, 0xb8, 0xec, 0xc4, 0xca, 0x74, 0x45, 0xf3, 0xff, 0x72, 0xfe, 0xed, 0xe5, 0x43, 0x09, + 0x16, 0xd3, 0x7a, 0xca, 0x61, 0xde, 0xec, 0xc2, 0x12, 0x37, 0x3b, 0x0a, 0x57, 0x74, 0xde, 0x6b, + 0x60, 0xd4, 0x6b, 0xe8, 0x94, 0x6f, 0x82, 0x92, 0x66, 0x71, 0x84, 0xd8, 0x84, 0x6c, 0x29, 0x07, + 0x45, 0x08, 0x84, 0xa6, 0x6f, 0x4b, 0xa0, 0xa2, 0x59, 0xd2, 0x1e, 0x8a, 0xa1, 0x07, 0x72, 0x4d, + 0x52, 0x8a, 0x07, 0x72, 0xad, 0xa2, 0x54, 0x0f, 0xe4, 0x5a, 0x5d, 0x81, 0x03, 0xb9, 0xd6, 0x50, + 0xd6, 0x0f, 0xe4, 0xda, 0x86, 0xa2, 0xd0, 0x6c, 0x8a, 0xd1, 0x95, 0xe9, 0x41, 0x57, 0xdb, 0x96, + 0xae, 0xb6, 0xcc, 0x32, 0x45, 0xef, 0x01, 0x64, 0xc7, 0xe3, 0xb7, 0xea, 0x9f, 0x9c, 0x84, 0x4c, + 0x8c, 0xc6, 0x1b, 0x34, 0x96, 0xb8, 0xde, 0x65, 0xb3, 0xd3, 0xe8, 0x0c, 0x2f, 0x64, 0x9d, 0xc6, + 0x92, 0xba, 0x00, 0x92, 0x27, 0x23, 0xfe, 0xa2, 0xbf, 0xc2, 0xaf, 0xf3, 0x3d, 0xa8, 0xa7, 0x74, + 0xc3, 0x77, 0xe5, 0xd6, 0xbc, 0x7c, 0xcc, 0x78, 0xcd, 0xcb, 0x1c, 0xd4, 0x19, 0x6c, 0x88, 0x45, + 0x20, 0x6b, 0x82, 0x94, 0x31, 0xd2, 0x15, 0x8c, 0x29, 0x66, 0x8c, 0x79, 0x0f, 0xaa, 0x49, 0xdd, + 0xc5, 0xae, 0xf3, 0xe6, 0x55, 0x2b, 0x0b, 0x22, 0x68, 0x82, 0x54, 0x43, 0xd8, 0x58, 0xb1, 0x91, + 0x36, 0xc0, 0x91, 0xbf, 0x98, 0x1d, 0xdb, 0xf1, 0xce, 0x2c, 0xed, 0x96, 0xe9, 0x92, 0x86, 0xe7, + 0xe3, 0xfa, 0x3f, 0x61, 0x41, 0xc2, 0x60, 0x14, 0xb8, 0x76, 0x31, 0x9f, 0xb3, 0x20, 0xe6, 0xb0, + 0x10, 0xb2, 0xdc, 0xe5, 0xa5, 0xdc, 0x55, 0x17, 0xde, 0x58, 0x39, 0x24, 0x16, 0x37, 0x37, 0x71, + 0x8a, 0x2b, 0x13, 0x87, 0xbc, 0x7f, 0xb9, 0xae, 0x6f, 0xae, 0x2e, 0x80, 0x69, 0xbc, 0xe5, 0x92, + 0xfe, 0x51, 0x86, 0xf5, 0x4f, 0x16, 0x2c, 0x38, 0x4f, 0x96, 0x5b, 0x72, 0x17, 0x2a, 0x61, 0x64, + 0x47, 0x8b, 0x30, 0xde, 0x8c, 0xda, 0x59, 0x9c, 0x1c, 0x70, 0xcf, 0x40, 0x14, 0x8d, 0xd1, 0xe4, + 0x07, 0x00, 0x8c, 0x2f, 0xba, 0x16, 0x6e, 0x55, 0x97, 0xf6, 0xff, 0xbc, 0x2f, 0xae, 0xc4, 0xb8, + 0x53, 0xd5, 0x59, 0xf2, 0xc8, 0xeb, 0x81, 0x02, 0x56, 0xa9, 0x4e, 0x85, 0x40, 0xf6, 0x78, 0x3e, + 0x81, 0x33, 0x3b, 0xc5, 0x32, 0xe5, 0x1a, 0xd4, 0x40, 0x7d, 0xcf, 0x8e, 0xec, 0xfd, 0x02, 0x8d, + 0x51, 0x1c, 0xff, 0x98, 0x4d, 0x23, 0x3f, 0xc0, 0x09, 0x94, 0xc3, 0x3f, 0x44, 0x7d, 0x82, 0x17, + 0x28, 0x8c, 0x3f, 0xb5, 0x5d, 0x3b, 0xc0, 0x9f, 0xdf, 0x7c, 0x7c, 0xd4, 0xa7, 0xf1, 0x51, 0xe2, + 0x78, 0xcf, 0x8e, 0x02, 0xe7, 0x09, 0x8e, 0xaf, 0x1c, 0x7e, 0x88, 0xfa, 0x04, 0x2f, 0x50, 0xea, + 0x3b, 0x50, 0x11, 0x95, 0xe2, 0xb3, 0x5e, 0xa7, 0x74, 0x4c, 0xc5, 0x4a, 0x67, 0x4c, 0xba, 0x5d, + 0xdd, 0x30, 0x14, 0x49, 0x0c, 0x7e, 0xf5, 0xb7, 0x12, 0xd4, 0xd3, 0xb2, 0xf0, 0x5d, 0x6d, 0x34, + 0x1e, 0xe9, 0x02, 0x6a, 0xf6, 0x87, 0xfa, 0x78, 0x62, 0x2a, 0x12, 0x5f, 0xdc, 0xba, 0xda, 0xa8, + 0xab, 0x0f, 0xf4, 0x9e, 0x58, 0x00, 0xf5, 0x1f, 0xea, 0xdd, 0x89, 0xd9, 0x1f, 0x8f, 0x94, 0x12, + 0x37, 0x76, 0xb4, 0x9e, 0xd5, 0xd3, 0x4c, 0x4d, 0x91, 0xb9, 0xd4, 0xe7, 0x3b, 0xe3, 0x48, 0x1b, + 0x28, 0x65, 0xb2, 0x01, 0x6b, 0x93, 0x91, 0xf6, 0x50, 0xeb, 0x0f, 0xb4, 0xce, 0x40, 0x57, 0x2a, + 0xdc, 0x77, 0x34, 0x36, 0xad, 0xfb, 0xe3, 0xc9, 0xa8, 0xa7, 0x54, 0xf9, 0xf2, 0xc8, 0x45, 0xad, + 0xdb, 0xd5, 0x0f, 0x4d, 0x84, 0xd4, 0xe2, 0x1f, 0xa4, 0x0a, 0xc8, 0x7c, 0x0f, 0x56, 0x75, 0x80, + 0xac, 0xde, 0xf9, 0x35, 0xbb, 0x7e, 0xdd, 0x5a, 0x76, 0x79, 0x02, 0xa8, 0x3f, 0x97, 0x00, 0xb2, + 0x7b, 0x20, 0x77, 0xb3, 0xef, 0x16, 0xb1, 0x22, 0x6e, 0xae, 0x5e, 0xd7, 0xd5, 0x5f, 0x2f, 0x1f, + 0xe5, 0xbe, 0x42, 0x8a, 0xab, 0x2d, 0x2d, 0x5c, 0xff, 0xdd, 0xb7, 0x88, 0x05, 0x8d, 0xe5, 0xf8, + 0x7c, 0xd4, 0x89, 0xdd, 0x1d, 0xf3, 0xa8, 0xd3, 0x58, 0xfa, 0xdf, 0xf7, 0xcf, 0x5f, 0x49, 0xb0, + 0xb1, 0x92, 0xc6, 0xb5, 0x2f, 0xc9, 0x8d, 0xc5, 0xe2, 0x2b, 0x8c, 0xc5, 0xc2, 0x52, 0x0f, 0xbf, + 0x4a, 0x32, 0xfc, 0xf2, 0x52, 0x32, 0x5f, 0xfd, 0x8d, 0xf4, 0x2a, 0x97, 0xd7, 0x01, 0xc8, 0x38, + 0x4e, 0xbe, 0x0b, 0x95, 0xdc, 0x7f, 0x07, 0x9b, 0xab, 0x9d, 0x10, 0xff, 0x7b, 0x20, 0x12, 0x8e, + 0xb1, 0xea, 0xef, 0x25, 0x68, 0x2c, 0x9b, 0xaf, 0x2d, 0xca, 0x7f, 0xff, 0x49, 0xdb, 0xc9, 0x91, + 0x42, 0xcc, 0xf9, 0xb7, 0xae, 0xab, 0x23, 0x7e, 0x7b, 0x5c, 0xe2, 0xc5, 0xed, 0xdf, 0x48, 0x00, + 0xd9, 0x07, 0x3b, 0xef, 0xbe, 0xfe, 0xe8, 0xa1, 0x36, 0xe8, 0xf7, 0x94, 0x02, 0xd9, 0x82, 0x4d, + 0xaa, 0x1f, 0x0e, 0xfa, 0x5d, 0xcd, 0xb0, 0x7a, 0xfd, 0x9e, 0xc5, 0x3b, 0x66, 0xa8, 0x99, 0xdd, + 0x7d, 0x45, 0x22, 0x5f, 0x81, 0x1b, 0xe6, 0x78, 0x6c, 0x0d, 0xb5, 0xd1, 0x23, 0xab, 0x3b, 0x98, + 0x18, 0xa6, 0x4e, 0x0d, 0xa5, 0x48, 0x9a, 0x00, 0xe8, 0xad, 0xc5, 0x3d, 0xba, 0x05, 0x9b, 0xfd, + 0xd1, 0x03, 0xdd, 0xe0, 0xa2, 0x45, 0x35, 0x53, 0xb7, 0x06, 0xfd, 0x61, 0xdf, 0xd4, 0x7b, 0x8a, + 0x4c, 0x5a, 0x70, 0x93, 0xea, 0x9f, 0x4c, 0x74, 0xc3, 0xcc, 0x5b, 0xca, 0x9d, 0xef, 0x3f, 0x7b, + 0xd1, 0x2e, 0x7c, 0xfe, 0xa2, 0x5d, 0xf8, 0xf2, 0x45, 0x5b, 0xfa, 0xd9, 0x45, 0x5b, 0xfa, 0xd3, + 0x45, 0x5b, 0x7a, 0x7a, 0xd1, 0x96, 0x9e, 0x5d, 0xb4, 0xa5, 0xbf, 0x5f, 0xb4, 0xa5, 0x2f, 0x2e, + 0xda, 0x85, 0x2f, 0x2f, 0xda, 0xd2, 0xaf, 0x5f, 0xb6, 0x0b, 0xcf, 0x5e, 0xb6, 0x0b, 0x9f, 0xbf, + 0x6c, 0x17, 0x7e, 0x54, 0xc5, 0xbf, 0x8d, 0xe6, 0x47, 0x47, 0x15, 0xfc, 0x03, 0xe8, 0xbd, 0x7f, + 0x05, 0x00, 0x00, 0xff, 0xff, 0xab, 0x91, 0xcb, 0x7a, 0x48, 0x12, 0x00, 0x00, } func (x ErrorCause) String() string { diff --git a/pkg/mimirpb/mimir.proto b/pkg/mimirpb/mimir.proto index 6184fca969a..467a06d292b 100644 --- a/pkg/mimirpb/mimir.proto +++ b/pkg/mimirpb/mimir.proto @@ -35,11 +35,12 @@ message WriteRequest { message WriteResponse {} enum ErrorCause { - REPLICAS_DID_NOT_MATCH = 0; - TOO_MANY_CLUSTERS = 1; - VALIDATION = 2; - INGESTION_RATE_LIMITED = 3; - REQUEST_RATE_LIMITED = 4; + INVALID = 0; + REPLICAS_DID_NOT_MATCH = 1; + TOO_MANY_CLUSTERS = 2; + VALIDATION = 3; + INGESTION_RATE_LIMITED = 4; + REQUEST_RATE_LIMITED = 5; } message WriteErrorDetails { From c2f0350d71177e9af450b1ae797cb55f3e13bfff Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Mon, 16 Oct 2023 22:57:20 +0200 Subject: [PATCH 5/6] Rename mimirpb.VALIDATION in mimirpb.BAD_DATA Signed-off-by: Yuri Nikolic --- pkg/distributor/distributor_test.go | 6 +- pkg/distributor/errors.go | 4 +- pkg/distributor/errors_test.go | 8 +- pkg/distributor/push.go | 2 +- pkg/mimirpb/mimir.pb.go | 244 ++++++++++++++-------------- pkg/mimirpb/mimir.proto | 2 +- 6 files changed, 133 insertions(+), 133 deletions(-) diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index b04fa7c5c8f..b6192ca2714 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -879,7 +879,7 @@ func TestDistributor_PushHAInstances(t *testing.T) { cluster: "cluster0", samples: 5, expectedError: status.New(codes.FailedPrecondition, fmt.Sprintf(labelValueTooLongMsgFormat, "instance1234567890123456789012345678901234567890", formatLabelSet(labelSetGenWithReplicaAndCluster("instance1234567890123456789012345678901234567890", "cluster0")(0)))), - expectedDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, + expectedDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.BAD_DATA}, }, } { t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -1386,7 +1386,7 @@ func TestDistributor_Push_HistogramValidation(t *testing.T) { }, } - expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} + expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.BAD_DATA} for testName, tc := range tests { t.Run(testName, func(t *testing.T) { @@ -4069,7 +4069,7 @@ func TestDistributorValidation(t *testing.T) { ctx := user.InjectOrgID(context.Background(), "1") now := model.Now() future, past := now.Add(5*time.Hour), now.Add(-25*time.Hour) - expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION} + expectedDetails := &mimirpb.WriteErrorDetails{Cause: mimirpb.BAD_DATA} for name, tc := range map[string]struct { metadata []*mimirpb.MetricMetadata diff --git a/pkg/distributor/errors.go b/pkg/distributor/errors.go index fb464b95ca0..dd2c0db225b 100644 --- a/pkg/distributor/errors.go +++ b/pkg/distributor/errors.go @@ -100,7 +100,7 @@ func newValidationError(err error) validationError { } func (e validationError) errorCause() mimirpb.ErrorCause { - return mimirpb.VALIDATION + return mimirpb.BAD_DATA } var _ distributorError = validationError{} @@ -163,7 +163,7 @@ func toGRPCError(pushErr error, serviceOverloadErrorEnabled bool) error { if errors.As(pushErr, &distributorErr) { errDetails = &mimirpb.WriteErrorDetails{Cause: distributorErr.errorCause()} switch distributorErr.errorCause() { - case mimirpb.VALIDATION: + case mimirpb.BAD_DATA: errCode = codes.FailedPrecondition case mimirpb.INGESTION_RATE_LIMITED: errCode = codes.ResourceExhausted diff --git a/pkg/distributor/errors_test.go b/pkg/distributor/errors_test.go index 04720fe439e..72e1503ddbe 100644 --- a/pkg/distributor/errors_test.go +++ b/pkg/distributor/errors_test.go @@ -64,7 +64,7 @@ func TestNewValidationError(t *testing.T) { err := newValidationError(firstErr) assert.Error(t, err) assert.EqualError(t, err, validationMsg) - checkDistributorError(t, err, mimirpb.VALIDATION) + checkDistributorError(t, err, mimirpb.BAD_DATA) anotherErr := newValidationError(errors.New("this is another validation error")) assert.NotErrorIs(t, err, anotherErr) @@ -75,7 +75,7 @@ func TestNewValidationError(t *testing.T) { wrappedErr := fmt.Errorf("wrapped %w", err) assert.ErrorIs(t, wrappedErr, err) assert.True(t, errors.As(wrappedErr, &validationError{})) - checkDistributorError(t, wrappedErr, mimirpb.VALIDATION) + checkDistributorError(t, wrappedErr, mimirpb.BAD_DATA) } func TestNewIngestionRateError(t *testing.T) { @@ -180,14 +180,14 @@ func TestToGRPCError(t *testing.T) { err: newValidationError(originalErr), expectedGRPCCode: codes.FailedPrecondition, expectedErrorMsg: originalMsg, - expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.BAD_DATA}, }, { name: "a DoNotLogError of a validationError gets translated into gets translated into a FailedPrecondition error with VALIDATION cause", err: log.DoNotLogError{Err: newValidationError(originalErr)}, expectedGRPCCode: codes.FailedPrecondition, expectedErrorMsg: originalMsg, - expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.VALIDATION}, + expectedErrorDetails: &mimirpb.WriteErrorDetails{Cause: mimirpb.BAD_DATA}, }, { name: "an ingestionRateLimitedError gets translated into gets translated into a ResourceExhausted error with INGESTION_RATE_LIMITED cause", diff --git a/pkg/distributor/push.go b/pkg/distributor/push.go index f8030b43cde..01dc6bf3a99 100644 --- a/pkg/distributor/push.go +++ b/pkg/distributor/push.go @@ -156,7 +156,7 @@ func toHTTPStatus(ctx context.Context, pushErr error, limits *validation.Overrid var distributorErr distributorError if errors.As(pushErr, &distributorErr) { switch distributorErr.errorCause() { - case mimirpb.VALIDATION: + case mimirpb.BAD_DATA: return http.StatusBadRequest case mimirpb.INGESTION_RATE_LIMITED: // Return a 429 here to tell the client it is going too fast. diff --git a/pkg/mimirpb/mimir.pb.go b/pkg/mimirpb/mimir.pb.go index 333bb74112e..27104efd74e 100644 --- a/pkg/mimirpb/mimir.pb.go +++ b/pkg/mimirpb/mimir.pb.go @@ -35,7 +35,7 @@ const ( INVALID ErrorCause = 0 REPLICAS_DID_NOT_MATCH ErrorCause = 1 TOO_MANY_CLUSTERS ErrorCause = 2 - VALIDATION ErrorCause = 3 + BAD_DATA ErrorCause = 3 INGESTION_RATE_LIMITED ErrorCause = 4 REQUEST_RATE_LIMITED ErrorCause = 5 ) @@ -44,7 +44,7 @@ var ErrorCause_name = map[int32]string{ 0: "INVALID", 1: "REPLICAS_DID_NOT_MATCH", 2: "TOO_MANY_CLUSTERS", - 3: "VALIDATION", + 3: "BAD_DATA", 4: "INGESTION_RATE_LIMITED", 5: "REQUEST_RATE_LIMITED", } @@ -53,7 +53,7 @@ var ErrorCause_value = map[string]int32{ "INVALID": 0, "REPLICAS_DID_NOT_MATCH": 1, "TOO_MANY_CLUSTERS": 2, - "VALIDATION": 3, + "BAD_DATA": 3, "INGESTION_RATE_LIMITED": 4, "REQUEST_RATE_LIMITED": 5, } @@ -1865,125 +1865,125 @@ func init() { func init() { proto.RegisterFile("mimir.proto", fileDescriptor_86d4d7485f544059) } var fileDescriptor_86d4d7485f544059 = []byte{ - // 1885 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcd, 0x6f, 0x23, 0x59, - 0x11, 0x77, 0xdb, 0xed, 0xaf, 0x8a, 0xe3, 0xf4, 0xbc, 0x1d, 0x82, 0x37, 0xda, 0x75, 0x32, 0x8d, - 0x58, 0xc2, 0x08, 0x32, 0x68, 0x17, 0x66, 0xb5, 0xab, 0x41, 0x4b, 0xdb, 0xee, 0x99, 0x38, 0xeb, - 0x8f, 0xec, 0xeb, 0xf6, 0x2c, 0xc3, 0xa5, 0xd5, 0x71, 0x5e, 0x92, 0xd6, 0x76, 0xbb, 0x4d, 0x77, - 0x7b, 0x98, 0x70, 0xe2, 0x02, 0x42, 0x9c, 0x10, 0x12, 0x17, 0xc4, 0x8d, 0x03, 0xfc, 0x05, 0xfc, - 0x0d, 0x23, 0x21, 0xa4, 0x39, 0xae, 0x38, 0x8c, 0x98, 0xcc, 0x65, 0x8f, 0x7b, 0xe0, 0xc4, 0x09, - 0xbd, 0x7a, 0xfd, 0xe1, 0x76, 0x12, 0x18, 0x60, 0x6e, 0x5d, 0x55, 0xbf, 0xaa, 0xae, 0x57, 0xef, - 0x57, 0xe5, 0x6a, 0xc3, 0x9a, 0xe7, 0x78, 0x4e, 0xb0, 0x37, 0x0f, 0xfc, 0xc8, 0x27, 0xb5, 0xa9, - 0x1f, 0x44, 0xec, 0xc9, 0xfc, 0x68, 0xeb, 0xdb, 0xa7, 0x4e, 0x74, 0xb6, 0x38, 0xda, 0x9b, 0xfa, - 0xde, 0x9d, 0x53, 0xff, 0xd4, 0xbf, 0x83, 0x80, 0xa3, 0xc5, 0x09, 0x4a, 0x28, 0xe0, 0x93, 0x70, - 0x54, 0xff, 0x5c, 0x84, 0xc6, 0xa7, 0x81, 0x13, 0x31, 0xca, 0x7e, 0xbc, 0x60, 0x61, 0x44, 0x0e, - 0x01, 0x22, 0xc7, 0x63, 0x21, 0x0b, 0x1c, 0x16, 0xb6, 0xa4, 0x9d, 0xd2, 0xee, 0xda, 0xbb, 0x37, - 0xf7, 0x92, 0xf0, 0x7b, 0xa6, 0xe3, 0x31, 0x03, 0x6d, 0x9d, 0xad, 0xa7, 0xcf, 0xb7, 0x0b, 0x7f, - 0x7b, 0xbe, 0x4d, 0x0e, 0x03, 0x66, 0xbb, 0xae, 0x3f, 0x35, 0x53, 0x3f, 0xba, 0x14, 0x83, 0x7c, - 0x00, 0x15, 0xc3, 0x5f, 0x04, 0x53, 0xd6, 0x2a, 0xee, 0x48, 0xbb, 0xcd, 0x77, 0x6f, 0x65, 0xd1, - 0x96, 0xdf, 0xbc, 0x27, 0x40, 0xfa, 0x6c, 0xe1, 0xd1, 0xd8, 0x81, 0x7c, 0x08, 0x35, 0x8f, 0x45, - 0xf6, 0xb1, 0x1d, 0xd9, 0xad, 0x12, 0xa6, 0xd2, 0xca, 0x9c, 0x87, 0x2c, 0x0a, 0x9c, 0xe9, 0x30, - 0xb6, 0x77, 0xe4, 0xa7, 0xcf, 0xb7, 0x25, 0x9a, 0xe2, 0xc9, 0x3d, 0xd8, 0x0a, 0x3f, 0x73, 0xe6, - 0x96, 0x6b, 0x1f, 0x31, 0xd7, 0x9a, 0xd9, 0x1e, 0xb3, 0x1e, 0xdb, 0xae, 0x73, 0x6c, 0x47, 0x8e, - 0x3f, 0x6b, 0x7d, 0x51, 0xdd, 0x91, 0x76, 0x6b, 0xf4, 0xab, 0x1c, 0x32, 0xe0, 0x88, 0x91, 0xed, - 0xb1, 0x87, 0xa9, 0x5d, 0xdd, 0x06, 0xc8, 0xf2, 0x21, 0x55, 0x28, 0x69, 0x87, 0x7d, 0xa5, 0x40, - 0x6a, 0x20, 0xd3, 0xc9, 0x40, 0x57, 0x24, 0x75, 0x03, 0xd6, 0xe3, 0xec, 0xc3, 0xb9, 0x3f, 0x0b, - 0x99, 0xfa, 0x11, 0xdc, 0x40, 0x85, 0x1e, 0x04, 0x7e, 0xd0, 0x63, 0x91, 0xed, 0xb8, 0x21, 0xb9, - 0x0d, 0xe5, 0xae, 0xbd, 0x08, 0x59, 0x4b, 0xc2, 0xa3, 0x2f, 0x15, 0x12, 0x61, 0x68, 0xa3, 0x02, - 0xa2, 0xfe, 0x43, 0x02, 0xc8, 0xca, 0x4b, 0x34, 0xa8, 0x60, 0xea, 0xc9, 0x25, 0xbc, 0x91, 0xf9, - 0x62, 0xc2, 0x87, 0xb6, 0x13, 0x74, 0x6e, 0xc6, 0x77, 0xd0, 0x40, 0x95, 0x76, 0x6c, 0xcf, 0x23, - 0x16, 0xd0, 0xd8, 0x91, 0x7c, 0x07, 0xaa, 0xa1, 0xed, 0xcd, 0x5d, 0x16, 0xb6, 0x8a, 0x18, 0x43, - 0xc9, 0x62, 0x18, 0x68, 0xc0, 0xaa, 0x15, 0x68, 0x02, 0x23, 0x77, 0xa1, 0xce, 0x9e, 0x30, 0x6f, - 0xee, 0xda, 0x41, 0x18, 0x57, 0x9c, 0x2c, 0xe5, 0x1c, 0x9b, 0x62, 0xaf, 0x0c, 0x4a, 0x3e, 0x00, - 0x38, 0x73, 0xc2, 0xc8, 0x3f, 0x0d, 0x6c, 0x2f, 0x6c, 0xc9, 0xab, 0x09, 0xef, 0x27, 0xb6, 0xd8, - 0x73, 0x09, 0xac, 0x7e, 0x0f, 0xea, 0xe9, 0x79, 0x08, 0x01, 0x99, 0xdf, 0x14, 0x96, 0xab, 0x41, - 0xf1, 0x99, 0xdc, 0x84, 0xf2, 0x63, 0xdb, 0x5d, 0x08, 0xfa, 0x34, 0xa8, 0x10, 0x54, 0x0d, 0x2a, - 0xe2, 0x08, 0xe4, 0x16, 0x34, 0x90, 0x6d, 0x91, 0xed, 0xcd, 0x2d, 0x2f, 0x44, 0x58, 0x89, 0xae, - 0xa5, 0xba, 0x61, 0x98, 0x85, 0xe0, 0x71, 0xa5, 0x24, 0xc4, 0xef, 0x8a, 0xd0, 0xcc, 0x93, 0x88, - 0xbc, 0x0f, 0x72, 0x74, 0x3e, 0x4f, 0xae, 0xeb, 0x6b, 0xd7, 0x91, 0x2d, 0x16, 0xcd, 0xf3, 0x39, - 0xa3, 0xe8, 0x40, 0xbe, 0x05, 0xc4, 0x43, 0x9d, 0x75, 0x62, 0x7b, 0x8e, 0x7b, 0x8e, 0x84, 0xc3, - 0x54, 0xea, 0x54, 0x11, 0x96, 0xfb, 0x68, 0xe0, 0x3c, 0xe3, 0xc7, 0x3c, 0x63, 0xee, 0xbc, 0x25, - 0xa3, 0x1d, 0x9f, 0xb9, 0x6e, 0x31, 0x73, 0xa2, 0x56, 0x59, 0xe8, 0xf8, 0xb3, 0x7a, 0x0e, 0x90, - 0xbd, 0x89, 0xac, 0x41, 0x75, 0x32, 0xfa, 0x78, 0x34, 0xfe, 0x74, 0xa4, 0x14, 0xb8, 0xd0, 0x1d, - 0x4f, 0x46, 0xa6, 0x4e, 0x15, 0x89, 0xd4, 0xa1, 0xfc, 0x40, 0x9b, 0x3c, 0xd0, 0x95, 0x22, 0x59, - 0x87, 0xfa, 0x7e, 0xdf, 0x30, 0xc7, 0x0f, 0xa8, 0x36, 0x54, 0x4a, 0x84, 0x40, 0x13, 0x2d, 0x99, - 0x4e, 0xe6, 0xae, 0xc6, 0x64, 0x38, 0xd4, 0xe8, 0x23, 0xa5, 0xcc, 0x19, 0xdd, 0x1f, 0xdd, 0x1f, - 0x2b, 0x15, 0xd2, 0x80, 0x9a, 0x61, 0x6a, 0xa6, 0x6e, 0xe8, 0xa6, 0x52, 0x55, 0x3f, 0x86, 0x8a, - 0x78, 0xf5, 0x6b, 0x20, 0xa2, 0xfa, 0x0b, 0x09, 0x6a, 0x09, 0x79, 0x5e, 0x07, 0xb1, 0x73, 0x94, - 0x48, 0xee, 0xf3, 0x12, 0x11, 0x4a, 0x97, 0x88, 0xa0, 0xfe, 0xa5, 0x0c, 0xf5, 0x94, 0x8c, 0xe4, - 0x6d, 0xa8, 0x4f, 0xfd, 0xc5, 0x2c, 0xb2, 0x9c, 0x59, 0x84, 0x57, 0x2e, 0xef, 0x17, 0x68, 0x0d, - 0x55, 0xfd, 0x59, 0x44, 0x6e, 0xc1, 0x9a, 0x30, 0x9f, 0xb8, 0xbe, 0x1d, 0x89, 0x77, 0xed, 0x17, - 0x28, 0xa0, 0xf2, 0x3e, 0xd7, 0x11, 0x05, 0x4a, 0xe1, 0xc2, 0xc3, 0x37, 0x49, 0x94, 0x3f, 0x92, - 0x4d, 0xa8, 0x84, 0xd3, 0x33, 0xe6, 0xd9, 0x78, 0xb9, 0x37, 0x68, 0x2c, 0x91, 0xaf, 0x43, 0xf3, - 0xa7, 0x2c, 0xf0, 0xad, 0xe8, 0x2c, 0x60, 0xe1, 0x99, 0xef, 0x1e, 0xe3, 0x45, 0x4b, 0x74, 0x9d, - 0x6b, 0xcd, 0x44, 0x49, 0xde, 0x89, 0x61, 0x59, 0x5e, 0x15, 0xcc, 0x4b, 0xa2, 0x0d, 0xae, 0xef, - 0x26, 0xb9, 0xdd, 0x06, 0x65, 0x09, 0x27, 0x12, 0xac, 0x62, 0x82, 0x12, 0x6d, 0xa6, 0x48, 0x91, - 0xa4, 0x06, 0xcd, 0x19, 0x3b, 0xb5, 0x23, 0xe7, 0x31, 0xb3, 0xc2, 0xb9, 0x3d, 0x0b, 0x5b, 0xb5, - 0xd5, 0xb1, 0xde, 0x59, 0x4c, 0x3f, 0x63, 0x91, 0x31, 0xb7, 0x67, 0x71, 0x87, 0xae, 0x27, 0x1e, - 0x5c, 0x17, 0x92, 0x6f, 0xc0, 0x46, 0x1a, 0xe2, 0x98, 0xb9, 0x91, 0x1d, 0xb6, 0xea, 0x3b, 0xa5, - 0x5d, 0x42, 0xd3, 0xc8, 0x3d, 0xd4, 0xe6, 0x80, 0x98, 0x5b, 0xd8, 0x82, 0x9d, 0xd2, 0xae, 0x94, - 0x01, 0x31, 0x31, 0x3e, 0xde, 0x9a, 0x73, 0x3f, 0x74, 0x96, 0x92, 0x5a, 0xfb, 0xcf, 0x49, 0x25, - 0x1e, 0x69, 0x52, 0x69, 0x88, 0x38, 0xa9, 0x86, 0x48, 0x2a, 0x51, 0x67, 0x49, 0xa5, 0xc0, 0x38, - 0xa9, 0x75, 0x91, 0x54, 0xa2, 0x8e, 0x93, 0xba, 0x07, 0x10, 0xb0, 0x90, 0x45, 0xd6, 0x19, 0xaf, - 0x7c, 0x13, 0x87, 0xc0, 0xdb, 0x57, 0x8c, 0xb1, 0x3d, 0xca, 0x51, 0xfb, 0xce, 0x2c, 0xa2, 0xf5, - 0x20, 0x79, 0x24, 0x6f, 0x41, 0x3d, 0xe5, 0x5a, 0x6b, 0x03, 0xc9, 0x97, 0x29, 0xd4, 0x0f, 0xa1, - 0x9e, 0x7a, 0xe5, 0x5b, 0xb9, 0x0a, 0xa5, 0x47, 0xba, 0xa1, 0x48, 0xa4, 0x02, 0xc5, 0xd1, 0x58, - 0x29, 0x66, 0xed, 0x5c, 0xda, 0x92, 0x7f, 0xf9, 0x87, 0xb6, 0xd4, 0xa9, 0x42, 0x19, 0xf3, 0xee, - 0x34, 0x00, 0xb2, 0x6b, 0x57, 0xff, 0x2a, 0x43, 0x13, 0xaf, 0x38, 0xa3, 0x74, 0x08, 0x04, 0x6d, - 0x2c, 0xb0, 0x56, 0x4e, 0xb2, 0xde, 0xd1, 0xff, 0xf9, 0x7c, 0x5b, 0x5b, 0x5a, 0x0f, 0xe6, 0x81, - 0xef, 0xb1, 0xe8, 0x8c, 0x2d, 0xc2, 0xe5, 0x47, 0xcf, 0x3f, 0x66, 0xee, 0x9d, 0x74, 0x40, 0xef, - 0x75, 0x45, 0xb8, 0xec, 0xc4, 0xca, 0x74, 0x45, 0xf3, 0xff, 0x72, 0xfe, 0xed, 0xe5, 0x43, 0x09, - 0x16, 0xd3, 0x7a, 0xca, 0x61, 0xde, 0xec, 0xc2, 0x12, 0x37, 0x3b, 0x0a, 0x57, 0x74, 0xde, 0x6b, - 0x60, 0xd4, 0x6b, 0xe8, 0x94, 0x6f, 0x82, 0x92, 0x66, 0x71, 0x84, 0xd8, 0x84, 0x6c, 0x29, 0x07, - 0x45, 0x08, 0x84, 0xa6, 0x6f, 0x4b, 0xa0, 0xa2, 0x59, 0xd2, 0x1e, 0x8a, 0xa1, 0x07, 0x72, 0x4d, - 0x52, 0x8a, 0x07, 0x72, 0xad, 0xa2, 0x54, 0x0f, 0xe4, 0x5a, 0x5d, 0x81, 0x03, 0xb9, 0xd6, 0x50, - 0xd6, 0x0f, 0xe4, 0xda, 0x86, 0xa2, 0xd0, 0x6c, 0x8a, 0xd1, 0x95, 0xe9, 0x41, 0x57, 0xdb, 0x96, - 0xae, 0xb6, 0xcc, 0x32, 0x45, 0xef, 0x01, 0x64, 0xc7, 0xe3, 0xb7, 0xea, 0x9f, 0x9c, 0x84, 0x4c, - 0x8c, 0xc6, 0x1b, 0x34, 0x96, 0xb8, 0xde, 0x65, 0xb3, 0xd3, 0xe8, 0x0c, 0x2f, 0x64, 0x9d, 0xc6, - 0x92, 0xba, 0x00, 0x92, 0x27, 0x23, 0xfe, 0xa2, 0xbf, 0xc2, 0xaf, 0xf3, 0x3d, 0xa8, 0xa7, 0x74, - 0xc3, 0x77, 0xe5, 0xd6, 0xbc, 0x7c, 0xcc, 0x78, 0xcd, 0xcb, 0x1c, 0xd4, 0x19, 0x6c, 0x88, 0x45, - 0x20, 0x6b, 0x82, 0x94, 0x31, 0xd2, 0x15, 0x8c, 0x29, 0x66, 0x8c, 0x79, 0x0f, 0xaa, 0x49, 0xdd, - 0xc5, 0xae, 0xf3, 0xe6, 0x55, 0x2b, 0x0b, 0x22, 0x68, 0x82, 0x54, 0x43, 0xd8, 0x58, 0xb1, 0x91, - 0x36, 0xc0, 0x91, 0xbf, 0x98, 0x1d, 0xdb, 0xf1, 0xce, 0x2c, 0xed, 0x96, 0xe9, 0x92, 0x86, 0xe7, - 0xe3, 0xfa, 0x3f, 0x61, 0x41, 0xc2, 0x60, 0x14, 0xb8, 0x76, 0x31, 0x9f, 0xb3, 0x20, 0xe6, 0xb0, - 0x10, 0xb2, 0xdc, 0xe5, 0xa5, 0xdc, 0x55, 0x17, 0xde, 0x58, 0x39, 0x24, 0x16, 0x37, 0x37, 0x71, - 0x8a, 0x2b, 0x13, 0x87, 0xbc, 0x7f, 0xb9, 0xae, 0x6f, 0xae, 0x2e, 0x80, 0x69, 0xbc, 0xe5, 0x92, - 0xfe, 0x51, 0x86, 0xf5, 0x4f, 0x16, 0x2c, 0x38, 0x4f, 0x96, 0x5b, 0x72, 0x17, 0x2a, 0x61, 0x64, - 0x47, 0x8b, 0x30, 0xde, 0x8c, 0xda, 0x59, 0x9c, 0x1c, 0x70, 0xcf, 0x40, 0x14, 0x8d, 0xd1, 0xe4, - 0x07, 0x00, 0x8c, 0x2f, 0xba, 0x16, 0x6e, 0x55, 0x97, 0xf6, 0xff, 0xbc, 0x2f, 0xae, 0xc4, 0xb8, - 0x53, 0xd5, 0x59, 0xf2, 0xc8, 0xeb, 0x81, 0x02, 0x56, 0xa9, 0x4e, 0x85, 0x40, 0xf6, 0x78, 0x3e, - 0x81, 0x33, 0x3b, 0xc5, 0x32, 0xe5, 0x1a, 0xd4, 0x40, 0x7d, 0xcf, 0x8e, 0xec, 0xfd, 0x02, 0x8d, - 0x51, 0x1c, 0xff, 0x98, 0x4d, 0x23, 0x3f, 0xc0, 0x09, 0x94, 0xc3, 0x3f, 0x44, 0x7d, 0x82, 0x17, - 0x28, 0x8c, 0x3f, 0xb5, 0x5d, 0x3b, 0xc0, 0x9f, 0xdf, 0x7c, 0x7c, 0xd4, 0xa7, 0xf1, 0x51, 0xe2, - 0x78, 0xcf, 0x8e, 0x02, 0xe7, 0x09, 0x8e, 0xaf, 0x1c, 0x7e, 0x88, 0xfa, 0x04, 0x2f, 0x50, 0xea, - 0x3b, 0x50, 0x11, 0x95, 0xe2, 0xb3, 0x5e, 0xa7, 0x74, 0x4c, 0xc5, 0x4a, 0x67, 0x4c, 0xba, 0x5d, - 0xdd, 0x30, 0x14, 0x49, 0x0c, 0x7e, 0xf5, 0xb7, 0x12, 0xd4, 0xd3, 0xb2, 0xf0, 0x5d, 0x6d, 0x34, - 0x1e, 0xe9, 0x02, 0x6a, 0xf6, 0x87, 0xfa, 0x78, 0x62, 0x2a, 0x12, 0x5f, 0xdc, 0xba, 0xda, 0xa8, - 0xab, 0x0f, 0xf4, 0x9e, 0x58, 0x00, 0xf5, 0x1f, 0xea, 0xdd, 0x89, 0xd9, 0x1f, 0x8f, 0x94, 0x12, - 0x37, 0x76, 0xb4, 0x9e, 0xd5, 0xd3, 0x4c, 0x4d, 0x91, 0xb9, 0xd4, 0xe7, 0x3b, 0xe3, 0x48, 0x1b, - 0x28, 0x65, 0xb2, 0x01, 0x6b, 0x93, 0x91, 0xf6, 0x50, 0xeb, 0x0f, 0xb4, 0xce, 0x40, 0x57, 0x2a, - 0xdc, 0x77, 0x34, 0x36, 0xad, 0xfb, 0xe3, 0xc9, 0xa8, 0xa7, 0x54, 0xf9, 0xf2, 0xc8, 0x45, 0xad, - 0xdb, 0xd5, 0x0f, 0x4d, 0x84, 0xd4, 0xe2, 0x1f, 0xa4, 0x0a, 0xc8, 0x7c, 0x0f, 0x56, 0x75, 0x80, - 0xac, 0xde, 0xf9, 0x35, 0xbb, 0x7e, 0xdd, 0x5a, 0x76, 0x79, 0x02, 0xa8, 0x3f, 0x97, 0x00, 0xb2, - 0x7b, 0x20, 0x77, 0xb3, 0xef, 0x16, 0xb1, 0x22, 0x6e, 0xae, 0x5e, 0xd7, 0xd5, 0x5f, 0x2f, 0x1f, - 0xe5, 0xbe, 0x42, 0x8a, 0xab, 0x2d, 0x2d, 0x5c, 0xff, 0xdd, 0xb7, 0x88, 0x05, 0x8d, 0xe5, 0xf8, - 0x7c, 0xd4, 0x89, 0xdd, 0x1d, 0xf3, 0xa8, 0xd3, 0x58, 0xfa, 0xdf, 0xf7, 0xcf, 0x5f, 0x49, 0xb0, - 0xb1, 0x92, 0xc6, 0xb5, 0x2f, 0xc9, 0x8d, 0xc5, 0xe2, 0x2b, 0x8c, 0xc5, 0xc2, 0x52, 0x0f, 0xbf, - 0x4a, 0x32, 0xfc, 0xf2, 0x52, 0x32, 0x5f, 0xfd, 0x8d, 0xf4, 0x2a, 0x97, 0xd7, 0x01, 0xc8, 0x38, - 0x4e, 0xbe, 0x0b, 0x95, 0xdc, 0x7f, 0x07, 0x9b, 0xab, 0x9d, 0x10, 0xff, 0x7b, 0x20, 0x12, 0x8e, - 0xb1, 0xea, 0xef, 0x25, 0x68, 0x2c, 0x9b, 0xaf, 0x2d, 0xca, 0x7f, 0xff, 0x49, 0xdb, 0xc9, 0x91, - 0x42, 0xcc, 0xf9, 0xb7, 0xae, 0xab, 0x23, 0x7e, 0x7b, 0x5c, 0xe2, 0xc5, 0xed, 0xdf, 0x48, 0x00, - 0xd9, 0x07, 0x3b, 0xef, 0xbe, 0xfe, 0xe8, 0xa1, 0x36, 0xe8, 0xf7, 0x94, 0x02, 0xd9, 0x82, 0x4d, - 0xaa, 0x1f, 0x0e, 0xfa, 0x5d, 0xcd, 0xb0, 0x7a, 0xfd, 0x9e, 0xc5, 0x3b, 0x66, 0xa8, 0x99, 0xdd, - 0x7d, 0x45, 0x22, 0x5f, 0x81, 0x1b, 0xe6, 0x78, 0x6c, 0x0d, 0xb5, 0xd1, 0x23, 0xab, 0x3b, 0x98, - 0x18, 0xa6, 0x4e, 0x0d, 0xa5, 0x48, 0x9a, 0x00, 0xe8, 0xad, 0xc5, 0x3d, 0xba, 0x05, 0x9b, 0xfd, - 0xd1, 0x03, 0xdd, 0xe0, 0xa2, 0x45, 0x35, 0x53, 0xb7, 0x06, 0xfd, 0x61, 0xdf, 0xd4, 0x7b, 0x8a, - 0x4c, 0x5a, 0x70, 0x93, 0xea, 0x9f, 0x4c, 0x74, 0xc3, 0xcc, 0x5b, 0xca, 0x9d, 0xef, 0x3f, 0x7b, - 0xd1, 0x2e, 0x7c, 0xfe, 0xa2, 0x5d, 0xf8, 0xf2, 0x45, 0x5b, 0xfa, 0xd9, 0x45, 0x5b, 0xfa, 0xd3, - 0x45, 0x5b, 0x7a, 0x7a, 0xd1, 0x96, 0x9e, 0x5d, 0xb4, 0xa5, 0xbf, 0x5f, 0xb4, 0xa5, 0x2f, 0x2e, - 0xda, 0x85, 0x2f, 0x2f, 0xda, 0xd2, 0xaf, 0x5f, 0xb6, 0x0b, 0xcf, 0x5e, 0xb6, 0x0b, 0x9f, 0xbf, - 0x6c, 0x17, 0x7e, 0x54, 0xc5, 0xbf, 0x8d, 0xe6, 0x47, 0x47, 0x15, 0xfc, 0x03, 0xe8, 0xbd, 0x7f, - 0x05, 0x00, 0x00, 0xff, 0xff, 0xab, 0x91, 0xcb, 0x7a, 0x48, 0x12, 0x00, 0x00, + // 1881 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcd, 0x73, 0xdc, 0x48, + 0x15, 0x1f, 0xcd, 0x68, 0x3e, 0xf4, 0x3c, 0xb6, 0x95, 0xde, 0x10, 0x66, 0x53, 0xbb, 0x63, 0x47, + 0x14, 0x8b, 0x49, 0x81, 0x43, 0xed, 0x42, 0xb6, 0x76, 0x2b, 0xd4, 0xa2, 0x99, 0x51, 0xe2, 0xf1, + 0xce, 0x87, 0xb7, 0xa5, 0xc9, 0x12, 0x2e, 0x2a, 0x79, 0xdc, 0xb6, 0x55, 0x2b, 0x8d, 0x06, 0x49, + 0x13, 0x62, 0x4e, 0x5c, 0xa0, 0x28, 0x4e, 0x7b, 0xe1, 0x42, 0x71, 0xe3, 0x00, 0x7f, 0x01, 0x7f, + 0x43, 0xaa, 0x28, 0xaa, 0x72, 0xdc, 0xe2, 0x90, 0x22, 0xce, 0x65, 0x8f, 0x7b, 0xe0, 0xc4, 0x89, + 0xea, 0xd7, 0xfa, 0x18, 0x8d, 0x6d, 0x08, 0x90, 0x9b, 0xfa, 0xbd, 0x5f, 0x3f, 0xfd, 0xfa, 0xf5, + 0xef, 0xbd, 0x79, 0x1a, 0x58, 0xf3, 0x5d, 0xdf, 0x0d, 0x77, 0xe7, 0x61, 0x10, 0x07, 0xa4, 0x31, + 0x0d, 0xc2, 0x98, 0x3d, 0x99, 0x1f, 0xde, 0xfc, 0xee, 0x89, 0x1b, 0x9f, 0x2e, 0x0e, 0x77, 0xa7, + 0x81, 0x7f, 0xe7, 0x24, 0x38, 0x09, 0xee, 0x20, 0xe0, 0x70, 0x71, 0x8c, 0x2b, 0x5c, 0xe0, 0x93, + 0xd8, 0xa8, 0xfd, 0xb9, 0x0c, 0xcd, 0x4f, 0x43, 0x37, 0x66, 0x94, 0xfd, 0x74, 0xc1, 0xa2, 0x98, + 0x1c, 0x00, 0xc4, 0xae, 0xcf, 0x22, 0x16, 0xba, 0x2c, 0x6a, 0x49, 0xdb, 0x95, 0x9d, 0xb5, 0x77, + 0xaf, 0xef, 0xa6, 0xe1, 0x77, 0x2d, 0xd7, 0x67, 0x26, 0xfa, 0x3a, 0x37, 0x9f, 0x3e, 0xdf, 0x2a, + 0xfd, 0xed, 0xf9, 0x16, 0x39, 0x08, 0x99, 0xe3, 0x79, 0xc1, 0xd4, 0xca, 0xf6, 0xd1, 0xa5, 0x18, + 0xe4, 0x03, 0xa8, 0x99, 0xc1, 0x22, 0x9c, 0xb2, 0x56, 0x79, 0x5b, 0xda, 0xd9, 0x78, 0xf7, 0x56, + 0x1e, 0x6d, 0xf9, 0xcd, 0xbb, 0x02, 0x64, 0xcc, 0x16, 0x3e, 0x4d, 0x36, 0x90, 0x0f, 0xa1, 0xe1, + 0xb3, 0xd8, 0x39, 0x72, 0x62, 0xa7, 0x55, 0x41, 0x2a, 0xad, 0x7c, 0xf3, 0x90, 0xc5, 0xa1, 0x3b, + 0x1d, 0x26, 0xfe, 0x8e, 0xfc, 0xf4, 0xf9, 0x96, 0x44, 0x33, 0x3c, 0xb9, 0x07, 0x37, 0xa3, 0xcf, + 0xdc, 0xb9, 0xed, 0x39, 0x87, 0xcc, 0xb3, 0x67, 0x8e, 0xcf, 0xec, 0xc7, 0x8e, 0xe7, 0x1e, 0x39, + 0xb1, 0x1b, 0xcc, 0x5a, 0x5f, 0xd6, 0xb7, 0xa5, 0x9d, 0x06, 0xfd, 0x3a, 0x87, 0x0c, 0x38, 0x62, + 0xe4, 0xf8, 0xec, 0x61, 0xe6, 0xd7, 0xb6, 0x00, 0x72, 0x3e, 0xa4, 0x0e, 0x15, 0xfd, 0xa0, 0xaf, + 0x96, 0x48, 0x03, 0x64, 0x3a, 0x19, 0x18, 0xaa, 0xa4, 0x6d, 0xc2, 0x7a, 0xc2, 0x3e, 0x9a, 0x07, + 0xb3, 0x88, 0x69, 0x1f, 0xc1, 0x35, 0x34, 0x18, 0x61, 0x18, 0x84, 0x3d, 0x16, 0x3b, 0xae, 0x17, + 0x91, 0xdb, 0x50, 0xed, 0x3a, 0x8b, 0x88, 0xb5, 0x24, 0x3c, 0xfa, 0x52, 0x22, 0x11, 0x86, 0x3e, + 0x2a, 0x20, 0xda, 0x3f, 0x24, 0x80, 0x3c, 0xbd, 0x44, 0x87, 0x1a, 0x52, 0x4f, 0x2f, 0xe1, 0x8d, + 0x7c, 0x2f, 0x12, 0x3e, 0x70, 0xdc, 0xb0, 0x73, 0x3d, 0xb9, 0x83, 0x26, 0x9a, 0xf4, 0x23, 0x67, + 0x1e, 0xb3, 0x90, 0x26, 0x1b, 0xc9, 0xf7, 0xa0, 0x1e, 0x39, 0xfe, 0xdc, 0x63, 0x51, 0xab, 0x8c, + 0x31, 0xd4, 0x3c, 0x86, 0x89, 0x0e, 0xcc, 0x5a, 0x89, 0xa6, 0x30, 0x72, 0x17, 0x14, 0xf6, 0x84, + 0xf9, 0x73, 0xcf, 0x09, 0xa3, 0x24, 0xe3, 0x64, 0x89, 0x73, 0xe2, 0x4a, 0x76, 0xe5, 0x50, 0xf2, + 0x01, 0xc0, 0xa9, 0x1b, 0xc5, 0xc1, 0x49, 0xe8, 0xf8, 0x51, 0x4b, 0x5e, 0x25, 0xbc, 0x97, 0xfa, + 0x92, 0x9d, 0x4b, 0x60, 0xed, 0x07, 0xa0, 0x64, 0xe7, 0x21, 0x04, 0x64, 0x7e, 0x53, 0x98, 0xae, + 0x26, 0xc5, 0x67, 0x72, 0x1d, 0xaa, 0x8f, 0x1d, 0x6f, 0x21, 0xe4, 0xd3, 0xa4, 0x62, 0xa1, 0xe9, + 0x50, 0x13, 0x47, 0x20, 0xb7, 0xa0, 0x89, 0x6a, 0x8b, 0x1d, 0x7f, 0x6e, 0xfb, 0x11, 0xc2, 0x2a, + 0x74, 0x2d, 0xb3, 0x0d, 0xa3, 0x3c, 0x04, 0x8f, 0x2b, 0xa5, 0x21, 0x7e, 0x57, 0x86, 0x8d, 0xa2, + 0x88, 0xc8, 0xfb, 0x20, 0xc7, 0x67, 0xf3, 0xf4, 0xba, 0xbe, 0x71, 0x95, 0xd8, 0x92, 0xa5, 0x75, + 0x36, 0x67, 0x14, 0x37, 0x90, 0xef, 0x00, 0xf1, 0xd1, 0x66, 0x1f, 0x3b, 0xbe, 0xeb, 0x9d, 0xa1, + 0xe0, 0x90, 0x8a, 0x42, 0x55, 0xe1, 0xb9, 0x8f, 0x0e, 0xae, 0x33, 0x7e, 0xcc, 0x53, 0xe6, 0xcd, + 0x5b, 0x32, 0xfa, 0xf1, 0x99, 0xdb, 0x16, 0x33, 0x37, 0x6e, 0x55, 0x85, 0x8d, 0x3f, 0x6b, 0x67, + 0x00, 0xf9, 0x9b, 0xc8, 0x1a, 0xd4, 0x27, 0xa3, 0x8f, 0x47, 0xe3, 0x4f, 0x47, 0x6a, 0x89, 0x2f, + 0xba, 0xe3, 0xc9, 0xc8, 0x32, 0xa8, 0x2a, 0x11, 0x05, 0xaa, 0x0f, 0xf4, 0xc9, 0x03, 0x43, 0x2d, + 0x93, 0x75, 0x50, 0xf6, 0xfa, 0xa6, 0x35, 0x7e, 0x40, 0xf5, 0xa1, 0x5a, 0x21, 0x04, 0x36, 0xd0, + 0x93, 0xdb, 0x64, 0xbe, 0xd5, 0x9c, 0x0c, 0x87, 0x3a, 0x7d, 0xa4, 0x56, 0xb9, 0xa2, 0xfb, 0xa3, + 0xfb, 0x63, 0xb5, 0x46, 0x9a, 0xd0, 0x30, 0x2d, 0xdd, 0x32, 0x4c, 0xc3, 0x52, 0xeb, 0xda, 0xc7, + 0x50, 0x13, 0xaf, 0x7e, 0x0d, 0x42, 0xd4, 0x7e, 0x25, 0x41, 0x23, 0x15, 0xcf, 0xeb, 0x10, 0x76, + 0x41, 0x12, 0xe9, 0x7d, 0x5e, 0x10, 0x42, 0xe5, 0x82, 0x10, 0xb4, 0xbf, 0x54, 0x41, 0xc9, 0xc4, + 0x48, 0xde, 0x06, 0x65, 0x1a, 0x2c, 0x66, 0xb1, 0xed, 0xce, 0x62, 0xbc, 0x72, 0x79, 0xaf, 0x44, + 0x1b, 0x68, 0xea, 0xcf, 0x62, 0x72, 0x0b, 0xd6, 0x84, 0xfb, 0xd8, 0x0b, 0x9c, 0x58, 0xbc, 0x6b, + 0xaf, 0x44, 0x01, 0x8d, 0xf7, 0xb9, 0x8d, 0xa8, 0x50, 0x89, 0x16, 0x3e, 0xbe, 0x49, 0xa2, 0xfc, + 0x91, 0xdc, 0x80, 0x5a, 0x34, 0x3d, 0x65, 0xbe, 0x83, 0x97, 0x7b, 0x8d, 0x26, 0x2b, 0xf2, 0x4d, + 0xd8, 0xf8, 0x39, 0x0b, 0x03, 0x3b, 0x3e, 0x0d, 0x59, 0x74, 0x1a, 0x78, 0x47, 0x78, 0xd1, 0x12, + 0x5d, 0xe7, 0x56, 0x2b, 0x35, 0x92, 0x77, 0x12, 0x58, 0xce, 0xab, 0x86, 0xbc, 0x24, 0xda, 0xe4, + 0xf6, 0x6e, 0xca, 0xed, 0x36, 0xa8, 0x4b, 0x38, 0x41, 0xb0, 0x8e, 0x04, 0x25, 0xba, 0x91, 0x21, + 0x05, 0x49, 0x1d, 0x36, 0x66, 0xec, 0xc4, 0x89, 0xdd, 0xc7, 0xcc, 0x8e, 0xe6, 0xce, 0x2c, 0x6a, + 0x35, 0x56, 0xdb, 0x7a, 0x67, 0x31, 0xfd, 0x8c, 0xc5, 0xe6, 0xdc, 0x99, 0x25, 0x15, 0xba, 0x9e, + 0xee, 0xe0, 0xb6, 0x88, 0x7c, 0x0b, 0x36, 0xb3, 0x10, 0x47, 0xcc, 0x8b, 0x9d, 0xa8, 0xa5, 0x6c, + 0x57, 0x76, 0x08, 0xcd, 0x22, 0xf7, 0xd0, 0x5a, 0x00, 0x22, 0xb7, 0xa8, 0x05, 0xdb, 0x95, 0x1d, + 0x29, 0x07, 0x22, 0x31, 0xde, 0xde, 0x36, 0xe6, 0x41, 0xe4, 0x2e, 0x91, 0x5a, 0xfb, 0xcf, 0xa4, + 0xd2, 0x1d, 0x19, 0xa9, 0x2c, 0x44, 0x42, 0xaa, 0x29, 0x48, 0xa5, 0xe6, 0x9c, 0x54, 0x06, 0x4c, + 0x48, 0xad, 0x0b, 0x52, 0xa9, 0x39, 0x21, 0x75, 0x0f, 0x20, 0x64, 0x11, 0x8b, 0xed, 0x53, 0x9e, + 0xf9, 0x0d, 0x6c, 0x02, 0x6f, 0x5f, 0xd2, 0xc6, 0x76, 0x29, 0x47, 0xed, 0xb9, 0xb3, 0x98, 0x2a, + 0x61, 0xfa, 0x48, 0xde, 0x02, 0x25, 0xd3, 0x5a, 0x6b, 0x13, 0xc5, 0x97, 0x1b, 0xb4, 0x0f, 0x41, + 0xc9, 0x76, 0x15, 0x4b, 0xb9, 0x0e, 0x95, 0x47, 0x86, 0xa9, 0x4a, 0xa4, 0x06, 0xe5, 0xd1, 0x58, + 0x2d, 0xe7, 0xe5, 0x5c, 0xb9, 0x29, 0xff, 0xfa, 0x0f, 0x6d, 0xa9, 0x53, 0x87, 0x2a, 0xf2, 0xee, + 0x34, 0x01, 0xf2, 0x6b, 0xd7, 0xfe, 0x2a, 0xc3, 0x06, 0x5e, 0x71, 0x2e, 0xe9, 0x08, 0x08, 0xfa, + 0x58, 0x68, 0xaf, 0x9c, 0x64, 0xbd, 0x63, 0xfc, 0xf3, 0xf9, 0x96, 0xbe, 0x34, 0x1e, 0xcc, 0xc3, + 0xc0, 0x67, 0xf1, 0x29, 0x5b, 0x44, 0xcb, 0x8f, 0x7e, 0x70, 0xc4, 0xbc, 0x3b, 0x59, 0x83, 0xde, + 0xed, 0x8a, 0x70, 0xf9, 0x89, 0xd5, 0xe9, 0x8a, 0xe5, 0xff, 0xd5, 0xfc, 0xdb, 0xcb, 0x87, 0x12, + 0x2a, 0xa6, 0x4a, 0xa6, 0x61, 0x5e, 0xec, 0xc2, 0x93, 0x14, 0x3b, 0x2e, 0x2e, 0xa9, 0xbc, 0xd7, + 0xa0, 0xa8, 0xd7, 0x50, 0x29, 0xdf, 0x06, 0x35, 0x63, 0x71, 0x88, 0xd8, 0x54, 0x6c, 0x99, 0x06, + 0x45, 0x08, 0x84, 0x66, 0x6f, 0x4b, 0xa1, 0xa2, 0x58, 0xb2, 0x1a, 0x4a, 0xa0, 0xfb, 0x72, 0x43, + 0x52, 0xcb, 0xfb, 0x72, 0xa3, 0xa6, 0xd6, 0xf7, 0xe5, 0x86, 0xa2, 0xc2, 0xbe, 0xdc, 0x68, 0xaa, + 0xeb, 0xfb, 0x72, 0x63, 0x53, 0x55, 0x69, 0xde, 0xc5, 0xe8, 0x4a, 0xf7, 0xa0, 0xab, 0x65, 0x4b, + 0x57, 0x4b, 0x66, 0x59, 0xa2, 0xf7, 0x00, 0xf2, 0xe3, 0xf1, 0x5b, 0x0d, 0x8e, 0x8f, 0x23, 0x26, + 0x5a, 0xe3, 0x35, 0x9a, 0xac, 0xb8, 0xdd, 0x63, 0xb3, 0x93, 0xf8, 0x14, 0x2f, 0x64, 0x9d, 0x26, + 0x2b, 0x6d, 0x01, 0xa4, 0x28, 0x46, 0xfc, 0x45, 0x7f, 0x85, 0x5f, 0xe7, 0x7b, 0xa0, 0x64, 0x72, + 0xc3, 0x77, 0x15, 0xc6, 0xbc, 0x62, 0xcc, 0x64, 0xcc, 0xcb, 0x37, 0x68, 0x33, 0xd8, 0x14, 0x83, + 0x40, 0x5e, 0x04, 0x99, 0x62, 0xa4, 0x4b, 0x14, 0x53, 0xce, 0x15, 0xf3, 0x1e, 0xd4, 0xd3, 0xbc, + 0x8b, 0x59, 0xe7, 0xcd, 0xcb, 0x46, 0x16, 0x44, 0xd0, 0x14, 0xa9, 0x45, 0xb0, 0xb9, 0xe2, 0x23, + 0x6d, 0x80, 0xc3, 0x60, 0x31, 0x3b, 0x72, 0x92, 0x99, 0x59, 0xda, 0xa9, 0xd2, 0x25, 0x0b, 0xe7, + 0xe3, 0x05, 0x3f, 0x63, 0x61, 0xaa, 0x60, 0x5c, 0x70, 0xeb, 0x62, 0x3e, 0x67, 0x61, 0xa2, 0x61, + 0xb1, 0xc8, 0xb9, 0xcb, 0x4b, 0xdc, 0x35, 0x0f, 0xde, 0x58, 0x39, 0x24, 0x26, 0xb7, 0xd0, 0x71, + 0xca, 0x2b, 0x1d, 0x87, 0xbc, 0x7f, 0x31, 0xaf, 0x6f, 0xae, 0x0e, 0x80, 0x59, 0xbc, 0xe5, 0x94, + 0xfe, 0x51, 0x86, 0xf5, 0x4f, 0x16, 0x2c, 0x3c, 0x4b, 0x87, 0x5b, 0x72, 0x17, 0x6a, 0x51, 0xec, + 0xc4, 0x8b, 0x28, 0x99, 0x8c, 0xda, 0x79, 0x9c, 0x02, 0x70, 0xd7, 0x44, 0x14, 0x4d, 0xd0, 0xe4, + 0x47, 0x00, 0x8c, 0x0f, 0xba, 0x36, 0x4e, 0x55, 0x17, 0xe6, 0xff, 0xe2, 0x5e, 0x1c, 0x89, 0x71, + 0xa6, 0x52, 0x58, 0xfa, 0xc8, 0xf3, 0x81, 0x0b, 0xcc, 0x92, 0x42, 0xc5, 0x82, 0xec, 0x72, 0x3e, + 0xa1, 0x3b, 0x3b, 0xc1, 0x34, 0x15, 0x0a, 0xd4, 0x44, 0x7b, 0xcf, 0x89, 0x9d, 0xbd, 0x12, 0x4d, + 0x50, 0x1c, 0xff, 0x98, 0x4d, 0xe3, 0x20, 0xc4, 0x0e, 0x54, 0xc0, 0x3f, 0x44, 0x7b, 0x8a, 0x17, + 0x28, 0x8c, 0x3f, 0x75, 0x3c, 0x27, 0xc4, 0x9f, 0xdf, 0x62, 0x7c, 0xb4, 0x67, 0xf1, 0x71, 0xc5, + 0xf1, 0xbe, 0x13, 0x87, 0xee, 0x13, 0x6c, 0x5f, 0x05, 0xfc, 0x10, 0xed, 0x29, 0x5e, 0xa0, 0xb4, + 0x77, 0xa0, 0x26, 0x32, 0xc5, 0x7b, 0xbd, 0x41, 0xe9, 0x98, 0x8a, 0x91, 0xce, 0x9c, 0x74, 0xbb, + 0x86, 0x69, 0xaa, 0x92, 0x68, 0xfc, 0xda, 0x6f, 0x25, 0x50, 0xb2, 0xb4, 0xf0, 0x59, 0x6d, 0x34, + 0x1e, 0x19, 0x02, 0x6a, 0xf5, 0x87, 0xc6, 0x78, 0x62, 0xa9, 0x12, 0x1f, 0xdc, 0xba, 0xfa, 0xa8, + 0x6b, 0x0c, 0x8c, 0x9e, 0x18, 0x00, 0x8d, 0x1f, 0x1b, 0xdd, 0x89, 0xd5, 0x1f, 0x8f, 0xd4, 0x0a, + 0x77, 0x76, 0xf4, 0x9e, 0xdd, 0xd3, 0x2d, 0x5d, 0x95, 0xf9, 0xaa, 0xcf, 0x67, 0xc6, 0x91, 0x3e, + 0x50, 0xab, 0x64, 0x13, 0xd6, 0x26, 0x23, 0xfd, 0xa1, 0xde, 0x1f, 0xe8, 0x9d, 0x81, 0xa1, 0xd6, + 0xf8, 0xde, 0xd1, 0xd8, 0xb2, 0xef, 0x8f, 0x27, 0xa3, 0x9e, 0x5a, 0xe7, 0xc3, 0x23, 0x5f, 0xea, + 0xdd, 0xae, 0x71, 0x60, 0x21, 0xa4, 0x91, 0xfc, 0x20, 0xd5, 0x40, 0xe6, 0x73, 0xb0, 0x66, 0x00, + 0xe4, 0xf9, 0x2e, 0x8e, 0xd9, 0xca, 0x55, 0x63, 0xd9, 0xc5, 0x0e, 0xa0, 0xfd, 0x52, 0x02, 0xc8, + 0xef, 0x81, 0xdc, 0xcd, 0xbf, 0x5b, 0xc4, 0x88, 0x78, 0x63, 0xf5, 0xba, 0x2e, 0xff, 0x7a, 0xf9, + 0xa8, 0xf0, 0x15, 0x52, 0x5e, 0x2d, 0x69, 0xb1, 0xf5, 0xdf, 0x7d, 0x8b, 0xd8, 0xd0, 0x5c, 0x8e, + 0xcf, 0x5b, 0x9d, 0x98, 0xdd, 0x91, 0x87, 0x42, 0x93, 0xd5, 0xff, 0x3e, 0x7f, 0xfe, 0x46, 0x82, + 0xcd, 0x15, 0x1a, 0x57, 0xbe, 0xa4, 0xd0, 0x16, 0xcb, 0xaf, 0xd0, 0x16, 0x4b, 0x4b, 0x35, 0xfc, + 0x2a, 0x64, 0xf8, 0xe5, 0x65, 0x62, 0xbe, 0xfc, 0x1b, 0xe9, 0x55, 0x2e, 0xaf, 0x03, 0x90, 0x6b, + 0x9c, 0x7c, 0x1f, 0x6a, 0x85, 0xff, 0x0e, 0x6e, 0xac, 0x56, 0x42, 0xf2, 0xef, 0x81, 0x20, 0x9c, + 0x60, 0xb5, 0xdf, 0x4b, 0xd0, 0x5c, 0x76, 0x5f, 0x99, 0x94, 0xff, 0xfe, 0x93, 0xb6, 0x53, 0x10, + 0x85, 0xe8, 0xf3, 0x6f, 0x5d, 0x95, 0x47, 0xfc, 0xf6, 0xb8, 0xa0, 0x8b, 0xdb, 0x9f, 0x4b, 0x00, + 0xf9, 0x07, 0x3b, 0xaf, 0xbe, 0xfe, 0xe8, 0xa1, 0x3e, 0xe8, 0xf7, 0xd4, 0x12, 0xb9, 0x09, 0x37, + 0xa8, 0x71, 0x30, 0xe8, 0x77, 0x75, 0xd3, 0xee, 0xf5, 0x7b, 0x36, 0xaf, 0x98, 0xa1, 0x6e, 0x75, + 0xf7, 0x54, 0x89, 0x7c, 0x0d, 0xae, 0x59, 0xe3, 0xb1, 0x3d, 0xd4, 0x47, 0x8f, 0xec, 0xee, 0x60, + 0x62, 0x5a, 0x06, 0x35, 0xd5, 0x72, 0xa1, 0x26, 0x2b, 0x3c, 0x40, 0x7f, 0xf4, 0xc0, 0x30, 0x79, + 0xc1, 0xda, 0x54, 0xb7, 0x0c, 0x7b, 0xd0, 0x1f, 0xf6, 0x2d, 0xa3, 0xa7, 0xca, 0xa4, 0x05, 0xd7, + 0xa9, 0xf1, 0xc9, 0xc4, 0x30, 0xad, 0xa2, 0xa7, 0xda, 0xf9, 0xe1, 0xb3, 0x17, 0xed, 0xd2, 0x17, + 0x2f, 0xda, 0xa5, 0xaf, 0x5e, 0xb4, 0xa5, 0x5f, 0x9c, 0xb7, 0xa5, 0x3f, 0x9d, 0xb7, 0xa5, 0xa7, + 0xe7, 0x6d, 0xe9, 0xd9, 0x79, 0x5b, 0xfa, 0xfb, 0x79, 0x5b, 0xfa, 0xf2, 0xbc, 0x5d, 0xfa, 0xea, + 0xbc, 0x2d, 0x7d, 0xfe, 0xb2, 0x5d, 0x7a, 0xf6, 0xb2, 0x5d, 0xfa, 0xe2, 0x65, 0xbb, 0xf4, 0x93, + 0x3a, 0xfe, 0x69, 0x34, 0x3f, 0x3c, 0xac, 0xe1, 0xdf, 0x3f, 0xef, 0xfd, 0x2b, 0x00, 0x00, 0xff, + 0xff, 0x07, 0x8f, 0x20, 0x53, 0x46, 0x12, 0x00, 0x00, } func (x ErrorCause) String() string { diff --git a/pkg/mimirpb/mimir.proto b/pkg/mimirpb/mimir.proto index 467a06d292b..c839f7fc9f2 100644 --- a/pkg/mimirpb/mimir.proto +++ b/pkg/mimirpb/mimir.proto @@ -38,7 +38,7 @@ enum ErrorCause { INVALID = 0; REPLICAS_DID_NOT_MATCH = 1; TOO_MANY_CLUSTERS = 2; - VALIDATION = 3; + BAD_DATA = 3; INGESTION_RATE_LIMITED = 4; REQUEST_RATE_LIMITED = 5; } From fbaf64c6cce0a96a96bd3b2d1dc58ec616d7f4bd Mon Sep 17 00:00:00 2001 From: Yuri Nikolic Date: Tue, 17 Oct 2023 11:02:29 +0200 Subject: [PATCH 6/6] Fixing review findings Signed-off-by: Yuri Nikolic --- pkg/distributor/errors.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/distributor/errors.go b/pkg/distributor/errors.go index dd2c0db225b..49bd193f8a2 100644 --- a/pkg/distributor/errors.go +++ b/pkg/distributor/errors.go @@ -65,6 +65,7 @@ func (e replicasDidNotMatchError) errorCause() mimirpb.ErrorCause { return mimirpb.REPLICAS_DID_NOT_MATCH } +// Ensure that replicasDidNotMatchError implements distributorError. var _ distributorError = replicasDidNotMatchError{} // tooManyClustersError is an error stating that there are too many HA clusters. @@ -87,6 +88,7 @@ func (e tooManyClustersError) errorCause() mimirpb.ErrorCause { return mimirpb.TOO_MANY_CLUSTERS } +// Ensure that tooManyClustersError implements distributorError. var _ distributorError = tooManyClustersError{} // validationError is an error, used to represent all validation errors from the validation package. @@ -103,6 +105,7 @@ func (e validationError) errorCause() mimirpb.ErrorCause { return mimirpb.BAD_DATA } +// Ensure that validationError implements distributorError. var _ distributorError = validationError{} // ingestionRateLimitedError is an error used to represent the ingestion rate limited error. @@ -127,6 +130,7 @@ func (e ingestionRateLimitedError) errorCause() mimirpb.ErrorCause { return mimirpb.INGESTION_RATE_LIMITED } +// Ensure that ingestionRateLimitedError implements distributorError. var _ distributorError = ingestionRateLimitedError{} // requestRateLimitedError is an error used to represent the request rate limited error. @@ -151,6 +155,7 @@ func (e requestRateLimitedError) errorCause() mimirpb.ErrorCause { return mimirpb.REQUEST_RATE_LIMITED } +// Ensure that requestRateLimitedError implements distributorError. var _ distributorError = requestRateLimitedError{} // toGRPCError converts the given error into an appropriate gRPC error.