mirror of
https://github.com/grafana/tempo.git
synced 2025-03-14 03:06:42 +00:00
Allow skipping spans for metric generation (#4728)
* Allow skipping spans for metric generation * use request body instead of context where possible * decode to PushSpansRequest
This commit is contained in:
@ -33,6 +33,7 @@ import (
|
||||
"github.com/grafana/tempo/modules/distributor/forwarder"
|
||||
"github.com/grafana/tempo/modules/distributor/receiver"
|
||||
"github.com/grafana/tempo/modules/distributor/usage"
|
||||
"github.com/grafana/tempo/modules/generator"
|
||||
generator_client "github.com/grafana/tempo/modules/generator/client"
|
||||
ingester_client "github.com/grafana/tempo/modules/ingester/client"
|
||||
"github.com/grafana/tempo/modules/overrides"
|
||||
@ -559,7 +560,7 @@ func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string
|
||||
return nil
|
||||
}
|
||||
|
||||
func (d *Distributor) sendToGenerators(ctx context.Context, userID string, keys []uint32, traces []*rebatchedTrace) error {
|
||||
func (d *Distributor) sendToGenerators(ctx context.Context, userID string, keys []uint32, traces []*rebatchedTrace, noGenerateMetrics bool) error {
|
||||
// If an instance is unhealthy write to the next one (i.e. write extend is enabled)
|
||||
op := ring.Write
|
||||
|
||||
@ -571,7 +572,8 @@ func (d *Distributor) sendToGenerators(ctx context.Context, userID string, keys
|
||||
localCtx = user.InjectOrgID(localCtx, userID)
|
||||
|
||||
req := tempopb.PushSpansRequest{
|
||||
Batches: nil,
|
||||
Batches: nil,
|
||||
SkipMetricsGeneration: noGenerateMetrics,
|
||||
}
|
||||
for _, j := range indexes {
|
||||
req.Batches = append(req.Batches, traces[j].trace.ResourceSpans...)
|
||||
@ -627,8 +629,9 @@ func (d *Distributor) sendToKafka(ctx context.Context, userID string, keys []uin
|
||||
localCtx = user.InjectOrgID(localCtx, userID)
|
||||
|
||||
req := &tempopb.PushBytesRequest{
|
||||
Traces: make([]tempopb.PreallocBytes, len(indexes)),
|
||||
Ids: make([][]byte, len(indexes)),
|
||||
Traces: make([]tempopb.PreallocBytes, len(indexes)),
|
||||
Ids: make([][]byte, len(indexes)),
|
||||
SkipMetricsGeneration: generator.ExtractNoGenerateMetrics(ctx),
|
||||
}
|
||||
|
||||
for i, j := range indexes {
|
||||
|
@ -10,6 +10,7 @@ import (
|
||||
"fmt"
|
||||
"maps"
|
||||
"math/rand"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
@ -23,13 +24,18 @@ import (
|
||||
"github.com/grafana/dskit/ring"
|
||||
ring_client "github.com/grafana/dskit/ring/client"
|
||||
"github.com/grafana/dskit/user"
|
||||
"github.com/grafana/tempo/modules/generator"
|
||||
"github.com/grafana/tempo/pkg/ingest"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/twmb/franz-go/pkg/kfake"
|
||||
"github.com/twmb/franz-go/pkg/kgo"
|
||||
"go.opentelemetry.io/collector/pdata/ptrace"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/health/grpc_health_v1"
|
||||
"google.golang.org/grpc/metadata"
|
||||
|
||||
"github.com/grafana/tempo/modules/distributor/receiver"
|
||||
generator_client "github.com/grafana/tempo/modules/generator/client"
|
||||
@ -1641,6 +1647,102 @@ func TestIngesterPushBytes(t *testing.T) {
|
||||
assert.Equal(t, maxLiveDiscardedCount, 35)
|
||||
}
|
||||
|
||||
func TestPushTracesSkipMetricsGenerationIngestStorage(t *testing.T) {
|
||||
const topic = "test-topic"
|
||||
|
||||
kafka, err := kfake.NewCluster(kfake.NumBrokers(1), kfake.AllowAutoTopicCreation())
|
||||
require.NoError(t, err)
|
||||
t.Cleanup(kafka.Close)
|
||||
|
||||
limitCfg := overrides.Config{}
|
||||
limitCfg.RegisterFlagsAndApplyDefaults(&flag.FlagSet{})
|
||||
|
||||
distributorCfg, ingesterClientCfg, overridesSvc, _,
|
||||
ingesterRing, limits, middleware := setupDependencies(t, limitCfg)
|
||||
|
||||
distributorCfg.KafkaWritePathEnabled = true
|
||||
distributorCfg.KafkaConfig = ingest.KafkaConfig{}
|
||||
distributorCfg.KafkaConfig.RegisterFlags(&flag.FlagSet{})
|
||||
distributorCfg.KafkaConfig.Address = kafka.ListenAddrs()[0]
|
||||
distributorCfg.KafkaConfig.Topic = topic
|
||||
|
||||
d, err := New(
|
||||
distributorCfg,
|
||||
ingesterClientCfg,
|
||||
ingesterRing,
|
||||
generator_client.Config{},
|
||||
nil,
|
||||
singlePartitionRingReader{},
|
||||
overridesSvc,
|
||||
middleware,
|
||||
kitlog.NewLogfmtLogger(os.Stdout),
|
||||
limits,
|
||||
prometheus.NewRegistry(),
|
||||
)
|
||||
require.NoError(t, err)
|
||||
|
||||
traces := batchesToTraces(t, []*v1.ResourceSpans{test.MakeBatch(10, nil)})
|
||||
|
||||
reader, err := kgo.NewClient(kgo.SeedBrokers(kafka.ListenAddrs()...), kgo.ConsumeTopics(topic))
|
||||
require.NoError(t, err)
|
||||
|
||||
t.Run("with no-generate-metrics header", func(t *testing.T) {
|
||||
// Inject the header into the incoming context. In a real call this would be done
|
||||
// by the gRPC server logic if the client sends that header in the outgoing
|
||||
// context.
|
||||
ctx := metadata.NewIncomingContext(ctx, metadata.Pairs(generator.NoGenerateMetricsContextKey, ""))
|
||||
_, err = d.PushTraces(ctx, traces)
|
||||
require.NoError(t, err)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
var recordProcessed bool
|
||||
fetches := reader.PollFetches(ctx)
|
||||
fetches.EachRecord(func(record *kgo.Record) {
|
||||
recordProcessed = true
|
||||
req, err := ingest.NewDecoder().Decode(record.Value)
|
||||
require.NoError(t, err)
|
||||
require.True(t, req.SkipMetricsGeneration)
|
||||
|
||||
reqs, err := ingest.NewPushBytesDecoder().Decode(record.Value)
|
||||
require.NoError(t, err)
|
||||
for req, err := range reqs {
|
||||
require.NoError(t, err)
|
||||
require.True(t, req.SkipMetricsGeneration)
|
||||
}
|
||||
})
|
||||
// Expect that we've fetched at least one record.
|
||||
require.True(t, recordProcessed)
|
||||
})
|
||||
|
||||
t.Run("without no-generate-metrics header", func(t *testing.T) {
|
||||
_, err = d.PushTraces(ctx, traces)
|
||||
require.NoError(t, err)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
var recordProcessed bool
|
||||
fetches := reader.PollFetches(ctx)
|
||||
fetches.EachRecord(func(record *kgo.Record) {
|
||||
recordProcessed = true
|
||||
req, err := ingest.NewDecoder().Decode(record.Value)
|
||||
require.NoError(t, err)
|
||||
require.False(t, req.SkipMetricsGeneration)
|
||||
|
||||
reqs, err := ingest.NewPushBytesDecoder().Decode(record.Value)
|
||||
require.NoError(t, err)
|
||||
for req, err := range reqs {
|
||||
require.NoError(t, err)
|
||||
require.False(t, req.SkipMetricsGeneration)
|
||||
}
|
||||
})
|
||||
// Expect that we've fetched at least one record.
|
||||
require.True(t, recordProcessed)
|
||||
})
|
||||
}
|
||||
|
||||
type testLogSpan struct {
|
||||
Msg string `json:"msg"`
|
||||
Level string `json:"level"`
|
||||
@ -1726,6 +1828,16 @@ func prepare(t *testing.T, limits overrides.Config, logger kitlog.Logger) (*Dist
|
||||
logger = kitlog.NewNopLogger()
|
||||
}
|
||||
|
||||
distributorConfig, clientConfig, overrides, ingesters, ingestersRing, l, mw := setupDependencies(t, limits)
|
||||
d, err := New(distributorConfig, clientConfig, ingestersRing, generator_client.Config{}, nil, nil, overrides, mw, logger, l, prometheus.NewPedanticRegistry())
|
||||
require.NoError(t, err)
|
||||
|
||||
return d, ingesters
|
||||
}
|
||||
|
||||
func setupDependencies(t *testing.T, limits overrides.Config) (Config, ingester_client.Config, overrides.Service, map[string]*mockIngester, *mockRing, dslog.Level, receiver.Middleware) {
|
||||
t.Helper()
|
||||
|
||||
var (
|
||||
distributorConfig Config
|
||||
clientConfig ingester_client.Config
|
||||
@ -1765,10 +1877,8 @@ func prepare(t *testing.T, limits overrides.Config, logger kitlog.Logger) (*Dist
|
||||
l := dslog.Level{}
|
||||
_ = l.Set("error")
|
||||
mw := receiver.MultiTenancyMiddleware()
|
||||
d, err := New(distributorConfig, clientConfig, ingestersRing, generator_client.Config{}, nil, nil, overrides, mw, logger, l, prometheus.NewPedanticRegistry())
|
||||
require.NoError(t, err)
|
||||
|
||||
return d, ingesters
|
||||
return distributorConfig, clientConfig, overrides, ingesters, ingestersRing, l, mw
|
||||
}
|
||||
|
||||
type mockIngester struct {
|
||||
@ -1898,3 +2008,14 @@ func (r mockRing) InstancesWithTokensInZoneCount(_ string) int {
|
||||
func (r mockRing) ZonesCount() int {
|
||||
return 0
|
||||
}
|
||||
|
||||
type singlePartitionRingReader struct{}
|
||||
|
||||
func (m singlePartitionRingReader) PartitionRing() *ring.PartitionRing {
|
||||
desc := ring.PartitionRingDesc{
|
||||
Partitions: map[int32]ring.PartitionDesc{
|
||||
0: {Id: 0, Tokens: []uint32{0}, State: ring.PartitionActive},
|
||||
},
|
||||
}
|
||||
return ring.NewPartitionRing(desc)
|
||||
}
|
||||
|
@ -13,6 +13,7 @@ import (
|
||||
"go.uber.org/multierr"
|
||||
|
||||
"github.com/grafana/tempo/modules/distributor/queue"
|
||||
"github.com/grafana/tempo/modules/generator"
|
||||
"github.com/grafana/tempo/modules/overrides"
|
||||
)
|
||||
|
||||
@ -34,12 +35,13 @@ var (
|
||||
}, []string{"tenant"})
|
||||
)
|
||||
|
||||
type forwardFunc func(ctx context.Context, tenantID string, keys []uint32, traces []*rebatchedTrace) error
|
||||
type forwardFunc func(ctx context.Context, tenantID string, keys []uint32, traces []*rebatchedTrace, noGenerateMetrics bool) error
|
||||
|
||||
type request struct {
|
||||
tenantID string
|
||||
keys []uint32
|
||||
traces []*rebatchedTrace
|
||||
tenantID string
|
||||
keys []uint32
|
||||
traces []*rebatchedTrace
|
||||
noGenerateMetrics bool
|
||||
}
|
||||
|
||||
// generatorForwarder queues up traces to be sent to the metrics-generators
|
||||
@ -84,7 +86,7 @@ func (f *generatorForwarder) SendTraces(ctx context.Context, tenantID string, ke
|
||||
}
|
||||
|
||||
q := f.getOrCreateQueue(tenantID)
|
||||
err := q.Push(ctx, &request{tenantID: tenantID, keys: keys, traces: traces})
|
||||
err := q.Push(ctx, &request{tenantID: tenantID, keys: keys, traces: traces, noGenerateMetrics: generator.ExtractNoGenerateMetrics(ctx)})
|
||||
if err != nil {
|
||||
_ = level.Error(f.logger).Log("msg", "failed to push traces to queue", "tenant", tenantID, "err", err)
|
||||
metricForwarderPushesFailures.WithLabelValues(tenantID).Inc()
|
||||
@ -216,7 +218,7 @@ func (f *generatorForwarder) stop(_ error) error {
|
||||
}
|
||||
|
||||
func (f *generatorForwarder) processFunc(ctx context.Context, data *request) {
|
||||
if err := f.forwardFunc(ctx, data.tenantID, data.keys, data.traces); err != nil {
|
||||
if err := f.forwardFunc(ctx, data.tenantID, data.keys, data.traces, data.noGenerateMetrics); err != nil {
|
||||
_ = level.Warn(f.logger).Log("msg", "failed to forward request to metrics generator", "err", err)
|
||||
}
|
||||
}
|
||||
|
@ -11,7 +11,9 @@ import (
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"google.golang.org/grpc/metadata"
|
||||
|
||||
"github.com/grafana/tempo/modules/generator"
|
||||
"github.com/grafana/tempo/modules/overrides"
|
||||
v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1"
|
||||
"github.com/grafana/tempo/pkg/util"
|
||||
@ -34,13 +36,18 @@ func TestForwarder(t *testing.T) {
|
||||
o, err := overrides.NewOverrides(oCfg, nil, prometheus.DefaultRegisterer)
|
||||
require.NoError(t, err)
|
||||
|
||||
noGenerateMetricsRequestCount := 0
|
||||
|
||||
wg := sync.WaitGroup{}
|
||||
f := newGeneratorForwarder(
|
||||
log.NewNopLogger(),
|
||||
func(ctx context.Context, userID string, k []uint32, traces []*rebatchedTrace) error {
|
||||
func(_ context.Context, userID string, k []uint32, traces []*rebatchedTrace, noGenerateMetrics bool) error {
|
||||
assert.Equal(t, tenantID, userID)
|
||||
assert.Equal(t, keys, k)
|
||||
assert.Equal(t, rebatchedTraces, traces)
|
||||
if noGenerateMetrics {
|
||||
noGenerateMetricsRequestCount++
|
||||
}
|
||||
wg.Done()
|
||||
return nil
|
||||
},
|
||||
@ -52,12 +59,17 @@ func TestForwarder(t *testing.T) {
|
||||
}()
|
||||
|
||||
wg.Add(1)
|
||||
f.SendTraces(context.Background(), tenantID, keys, rebatchedTraces)
|
||||
// Mark this request as "to-be-ignored" for metrics generation.
|
||||
ctx = metadata.NewIncomingContext(ctx, metadata.Pairs(generator.NoGenerateMetricsContextKey, ""))
|
||||
f.SendTraces(ctx, tenantID, keys, rebatchedTraces)
|
||||
wg.Wait()
|
||||
|
||||
wg.Add(1)
|
||||
f.SendTraces(context.Background(), tenantID, keys, rebatchedTraces)
|
||||
wg.Wait()
|
||||
|
||||
// Expect to receive one request for which no metrics should be generated in metrics-generator.
|
||||
require.Equal(t, 1, noGenerateMetricsRequestCount)
|
||||
}
|
||||
|
||||
func TestForwarder_shutdown(t *testing.T) {
|
||||
@ -78,12 +90,13 @@ func TestForwarder_shutdown(t *testing.T) {
|
||||
signalCh := make(chan struct{})
|
||||
f := newGeneratorForwarder(
|
||||
log.NewNopLogger(),
|
||||
func(ctx context.Context, userID string, k []uint32, traces []*rebatchedTrace) error {
|
||||
func(_ context.Context, userID string, k []uint32, traces []*rebatchedTrace, noGenerateMetrics bool) error {
|
||||
<-signalCh
|
||||
|
||||
assert.Equal(t, tenantID, userID)
|
||||
assert.Equal(t, keys, k)
|
||||
assert.Equal(t, rebatchedTraces, traces)
|
||||
assert.False(t, noGenerateMetrics)
|
||||
return nil
|
||||
},
|
||||
o,
|
||||
|
@ -16,6 +16,7 @@ import (
|
||||
"go.opentelemetry.io/collector/component"
|
||||
"go.opentelemetry.io/collector/config/configgrpc"
|
||||
"go.opentelemetry.io/collector/config/confighttp"
|
||||
"go.opentelemetry.io/collector/config/configopaque"
|
||||
"go.opentelemetry.io/collector/config/configtls"
|
||||
"go.opentelemetry.io/collector/exporter"
|
||||
"go.opentelemetry.io/collector/exporter/otlpexporter"
|
||||
@ -30,12 +31,14 @@ import (
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/protobuf/types/known/durationpb"
|
||||
|
||||
"github.com/grafana/tempo/modules/generator"
|
||||
"github.com/grafana/tempo/pkg/tempopb"
|
||||
)
|
||||
|
||||
// These tests use the OpenTelemetry Collector Exporters to validate the different protocols
|
||||
func TestShim_integration(t *testing.T) {
|
||||
randomTraces := testdata.GenerateTraces(5)
|
||||
headers := map[string]configopaque.String{generator.NoGenerateMetricsContextKey: "true"}
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
@ -60,6 +63,7 @@ func TestShim_integration(t *testing.T) {
|
||||
TLSSetting: configtls.ClientConfig{
|
||||
Insecure: true,
|
||||
},
|
||||
Headers: headers,
|
||||
},
|
||||
},
|
||||
expectedTransport: "grpc",
|
||||
@ -77,6 +81,7 @@ func TestShim_integration(t *testing.T) {
|
||||
exporterCfg: &otlphttpexporter.Config{
|
||||
ClientConfig: confighttp.ClientConfig{
|
||||
Endpoint: "http://127.0.0.1:4318",
|
||||
Headers: headers,
|
||||
},
|
||||
Encoding: otlphttpexporter.EncodingJSON,
|
||||
},
|
||||
@ -95,6 +100,7 @@ func TestShim_integration(t *testing.T) {
|
||||
exporterCfg: &otlphttpexporter.Config{
|
||||
ClientConfig: confighttp.ClientConfig{
|
||||
Endpoint: "http://127.0.0.1:4318",
|
||||
Headers: headers,
|
||||
},
|
||||
Encoding: otlphttpexporter.EncodingProto,
|
||||
},
|
||||
@ -103,7 +109,7 @@ func TestShim_integration(t *testing.T) {
|
||||
}
|
||||
for _, testCase := range testCases {
|
||||
t.Run(testCase.name, func(t *testing.T) {
|
||||
pusher := &capturingPusher{}
|
||||
pusher := &capturingPusher{t: t}
|
||||
reg := prometheus.NewPedanticRegistry()
|
||||
|
||||
stopShim := runReceiverShim(t, testCase.receiverCfg, pusher, reg)
|
||||
@ -198,6 +204,7 @@ func (m *mockHost) GetExtensions() map[component.ID]component.Component {
|
||||
|
||||
type capturingPusher struct {
|
||||
traces []ptrace.Traces
|
||||
t *testing.T
|
||||
}
|
||||
|
||||
func (p *capturingPusher) GetAndClearTraces() []ptrace.Traces {
|
||||
@ -206,8 +213,12 @@ func (p *capturingPusher) GetAndClearTraces() []ptrace.Traces {
|
||||
return traces
|
||||
}
|
||||
|
||||
func (p *capturingPusher) PushTraces(_ context.Context, t ptrace.Traces) (*tempopb.PushResponse, error) {
|
||||
func (p *capturingPusher) PushTraces(ctx context.Context, t ptrace.Traces) (*tempopb.PushResponse, error) {
|
||||
p.traces = append(p.traces, t)
|
||||
|
||||
// Ensure that headers from the exporter config are propagated.
|
||||
assert.True(p.t, generator.ExtractNoGenerateMetrics(ctx))
|
||||
|
||||
return &tempopb.PushResponse{}, nil
|
||||
}
|
||||
|
||||
|
@ -19,9 +19,11 @@ import (
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/twmb/franz-go/pkg/kadm"
|
||||
"github.com/twmb/franz-go/pkg/kgo"
|
||||
"go.opentelemetry.io/collector/client"
|
||||
"go.opentelemetry.io/otel"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.uber.org/atomic"
|
||||
"google.golang.org/grpc/metadata"
|
||||
|
||||
"github.com/grafana/tempo/modules/generator/storage"
|
||||
objStorage "github.com/grafana/tempo/modules/storage"
|
||||
@ -38,6 +40,12 @@ const (
|
||||
// We use a safe default instead of exposing to config option to the user
|
||||
// in order to simplify the config.
|
||||
ringNumTokens = 256
|
||||
|
||||
// NoGenerateMetricsContextKey is used in request contexts/headers to signal to
|
||||
// the metrics generator that it should not generate metrics for the spans
|
||||
// contained in the requests. This is intended to be used by clients that send
|
||||
// requests for which span-derived metrics have already been generated elsewhere.
|
||||
NoGenerateMetricsContextKey = "no-generate-metrics"
|
||||
)
|
||||
|
||||
var tracer = otel.Tracer("modules/generator")
|
||||
@ -358,7 +366,7 @@ func (g *Generator) createInstance(id string) (*instance, error) {
|
||||
}
|
||||
}
|
||||
|
||||
inst, err := newInstance(g.cfg, id, g.overrides, wal, reg, g.logger, tracesWAL, tracesQueryWAL, g.store)
|
||||
inst, err := newInstance(g.cfg, id, g.overrides, wal, g.logger, tracesWAL, tracesQueryWAL, g.store)
|
||||
if err != nil {
|
||||
_ = wal.Close()
|
||||
return nil, err
|
||||
@ -448,3 +456,20 @@ func (g *Generator) QueryRange(ctx context.Context, req *tempopb.QueryRangeReque
|
||||
|
||||
return instance.QueryRange(ctx, req)
|
||||
}
|
||||
|
||||
// ExtractNoGenerateMetrics checks for presence of context keys that indicate no
|
||||
// span-derived metrics should be generated for the request. If any such context
|
||||
// key is present, this will return true, otherwise it will return false.
|
||||
func ExtractNoGenerateMetrics(ctx context.Context) bool {
|
||||
// check gRPC context
|
||||
if len(metadata.ValueFromIncomingContext(ctx, NoGenerateMetricsContextKey)) > 0 {
|
||||
return true
|
||||
}
|
||||
|
||||
// check http context
|
||||
if len(client.FromContext(ctx).Metadata.Get(NoGenerateMetricsContextKey)) > 0 {
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
@ -10,7 +10,6 @@ import (
|
||||
"github.com/go-kit/log/level"
|
||||
|
||||
"github.com/grafana/tempo/pkg/ingest"
|
||||
"github.com/grafana/tempo/pkg/tempopb"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
"github.com/twmb/franz-go/pkg/kgo"
|
||||
@ -145,9 +144,7 @@ func (g *Generator) readCh(ctx context.Context) {
|
||||
continue
|
||||
}
|
||||
|
||||
i.pushSpansFromQueue(ctx, r.Timestamp, &tempopb.PushSpansRequest{
|
||||
Batches: resourceSpans,
|
||||
})
|
||||
i.pushSpansFromQueue(ctx, r.Timestamp, resourceSpans)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -165,7 +165,7 @@ func BenchmarkPushSpans(b *testing.B) {
|
||||
wal, err := storage.New(walcfg, o, tenant, reg, log)
|
||||
require.NoError(b, err)
|
||||
|
||||
inst, err := newInstance(cfg, tenant, o, wal, reg, log, nil, nil, nil)
|
||||
inst, err := newInstance(cfg, tenant, o, wal, log, nil, nil, nil)
|
||||
require.NoError(b, err)
|
||||
defer inst.shutdown()
|
||||
|
||||
@ -243,7 +243,7 @@ func BenchmarkCollect(b *testing.B) {
|
||||
wal, err := storage.New(walcfg, o, tenant, reg, log)
|
||||
require.NoError(b, err)
|
||||
|
||||
inst, err := newInstance(cfg, tenant, o, wal, reg, log, nil, nil, nil)
|
||||
inst, err := newInstance(cfg, tenant, o, wal, log, nil, nil, nil)
|
||||
require.NoError(b, err)
|
||||
defer inst.shutdown()
|
||||
|
||||
|
@ -64,6 +64,12 @@ var (
|
||||
Name: "metrics_generator_spans_discarded_total",
|
||||
Help: "The total number of discarded spans received per tenant",
|
||||
}, []string{"tenant", "reason"})
|
||||
metricSkippedProcessorPushes = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "tempo",
|
||||
Name: "metrics_generator_metrics_generation_skipped_processor_pushes_total",
|
||||
Help: "The total number of processor pushes skipped because the request indicated that" +
|
||||
" metrics should not be generated.",
|
||||
}, []string{"tenant"})
|
||||
)
|
||||
|
||||
const (
|
||||
@ -95,11 +101,10 @@ type instance struct {
|
||||
|
||||
shutdownCh chan struct{}
|
||||
|
||||
reg prometheus.Registerer
|
||||
logger log.Logger
|
||||
}
|
||||
|
||||
func newInstance(cfg *Config, instanceID string, overrides metricsGeneratorOverrides, wal storage.Storage, reg prometheus.Registerer, logger log.Logger, traceWAL, rf1TraceWAL *wal.WAL, writer tempodb.Writer) (*instance, error) {
|
||||
func newInstance(cfg *Config, instanceID string, overrides metricsGeneratorOverrides, wal storage.Storage, logger log.Logger, traceWAL, rf1TraceWAL *wal.WAL, writer tempodb.Writer) (*instance, error) {
|
||||
logger = log.With(logger, "tenant", instanceID)
|
||||
|
||||
i := &instance{
|
||||
@ -117,7 +122,6 @@ func newInstance(cfg *Config, instanceID string, overrides metricsGeneratorOverr
|
||||
|
||||
shutdownCh: make(chan struct{}, 1),
|
||||
|
||||
reg: reg,
|
||||
logger: logger,
|
||||
}
|
||||
|
||||
@ -397,7 +401,16 @@ func (i *instance) pushSpans(ctx context.Context, req *tempopb.PushSpansRequest)
|
||||
defer i.processorsMtx.RUnlock()
|
||||
|
||||
for _, processor := range i.processors {
|
||||
processor.PushSpans(ctx, req)
|
||||
switch processor.Name() {
|
||||
case localblocks.Name:
|
||||
processor.PushSpans(ctx, req)
|
||||
case spanmetrics.Name, servicegraphs.Name:
|
||||
if req.SkipMetricsGeneration {
|
||||
metricSkippedProcessorPushes.WithLabelValues(i.instanceID).Inc()
|
||||
break
|
||||
}
|
||||
processor.PushSpans(ctx, req)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -407,11 +420,17 @@ func (i *instance) pushSpansFromQueue(ctx context.Context, ts time.Time, req *te
|
||||
defer i.processorsMtx.RUnlock()
|
||||
|
||||
for _, processor := range i.processors {
|
||||
// Same as normal push except we skip the local blocks processor
|
||||
if processor.Name() == localblocks.Name {
|
||||
continue
|
||||
switch processor.Name() {
|
||||
case localblocks.Name:
|
||||
// don't push to this processor as queue consumer, instead use queue based local
|
||||
// blocks if configured.
|
||||
case spanmetrics.Name, servicegraphs.Name:
|
||||
if req.SkipMetricsGeneration {
|
||||
metricSkippedProcessorPushes.WithLabelValues(i.instanceID).Inc()
|
||||
break
|
||||
}
|
||||
processor.PushSpans(ctx, req)
|
||||
}
|
||||
processor.PushSpans(ctx, req)
|
||||
}
|
||||
|
||||
// Now we push to the non-flushing local blocks if present
|
||||
|
@ -5,17 +5,20 @@ import (
|
||||
"flag"
|
||||
"os"
|
||||
"sort"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/go-kit/log"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/testutil"
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
prometheus_storage "github.com/prometheus/prometheus/storage"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/grafana/tempo/modules/generator/processor/servicegraphs"
|
||||
"github.com/grafana/tempo/modules/generator/processor/spanmetrics"
|
||||
@ -33,10 +36,10 @@ func Test_instance_concurrency(t *testing.T) {
|
||||
servicegraphs.Name: {},
|
||||
}
|
||||
|
||||
instance1, err := newInstance(&Config{}, "test", overrides, &noopStorage{}, prometheus.DefaultRegisterer, log.NewNopLogger(), nil, nil, nil)
|
||||
instance1, err := newInstance(&Config{}, "test", overrides, &noopStorage{}, log.NewNopLogger(), nil, nil, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
instance2, err := newInstance(&Config{}, "test", overrides, &noopStorage{}, prometheus.DefaultRegisterer, log.NewNopLogger(), nil, nil, nil)
|
||||
instance2, err := newInstance(&Config{}, "test", overrides, &noopStorage{}, log.NewNopLogger(), nil, nil, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
end := make(chan struct{})
|
||||
@ -81,13 +84,62 @@ func Test_instance_concurrency(t *testing.T) {
|
||||
close(end)
|
||||
}
|
||||
|
||||
func TestInstancePushSpansSkipProcessors(t *testing.T) {
|
||||
overrides := &mockOverrides{}
|
||||
overrides.processors = map[string]struct{}{
|
||||
spanmetrics.Name: {},
|
||||
servicegraphs.Name: {},
|
||||
}
|
||||
const tenantID = "skip-processors-test"
|
||||
|
||||
i, err := newInstance(&Config{}, tenantID, overrides, &noopStorage{}, log.NewNopLogger(), nil, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
req := test.MakeBatch(1, nil)
|
||||
|
||||
// Expose this series so it's present at the initial zero value even if not created/incremented by the test.
|
||||
_ = metricSkippedProcessorPushes.WithLabelValues(tenantID)
|
||||
|
||||
t.Run("use metrics-generating processors", func(t *testing.T) {
|
||||
i.pushSpans(context.Background(), &tempopb.PushSpansRequest{Batches: []*v1.ResourceSpans{req}})
|
||||
|
||||
expectMetrics := `
|
||||
# HELP tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total The total number of processor pushes skipped because the request indicated that metrics should not be generated.
|
||||
# TYPE tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total counter
|
||||
tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total{tenant="skip-processors-test"} 0
|
||||
`
|
||||
err := testutil.GatherAndCompare(
|
||||
prometheus.DefaultGatherer,
|
||||
strings.NewReader(expectMetrics),
|
||||
"tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total",
|
||||
)
|
||||
require.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("skip metrics-generating processors", func(t *testing.T) {
|
||||
i.pushSpans(context.Background(), &tempopb.PushSpansRequest{Batches: []*v1.ResourceSpans{req}, SkipMetricsGeneration: true})
|
||||
|
||||
expectMetrics := `
|
||||
# HELP tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total The total number of processor pushes skipped because the request indicated that metrics should not be generated.
|
||||
# TYPE tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total counter
|
||||
tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total{tenant="skip-processors-test"} 2
|
||||
`
|
||||
err := testutil.GatherAndCompare(
|
||||
prometheus.DefaultGatherer,
|
||||
strings.NewReader(expectMetrics),
|
||||
"tempo_metrics_generator_metrics_generation_skipped_processor_pushes_total",
|
||||
)
|
||||
require.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_instance_updateProcessors(t *testing.T) {
|
||||
cfg := Config{}
|
||||
cfg.RegisterFlagsAndApplyDefaults("", &flag.FlagSet{})
|
||||
logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stdout))
|
||||
overrides := mockOverrides{}
|
||||
|
||||
instance, err := newInstance(&cfg, "test", &overrides, &noopStorage{}, prometheus.DefaultRegisterer, logger, nil, nil, nil)
|
||||
instance, err := newInstance(&cfg, "test", &overrides, &noopStorage{}, logger, nil, nil, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// stop the update goroutine
|
||||
|
@ -11,7 +11,6 @@ import (
|
||||
"github.com/twmb/franz-go/pkg/kgo"
|
||||
|
||||
"github.com/grafana/tempo/pkg/tempopb"
|
||||
v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1"
|
||||
)
|
||||
|
||||
var encoderPool = sync.Pool{
|
||||
@ -152,7 +151,7 @@ func sovPush(x uint64) (n int) {
|
||||
// GeneratorCodec is the interface used to convert data from Kafka records to the
|
||||
// tempopb.PushSpansRequest expected by the generator processors.
|
||||
type GeneratorCodec interface {
|
||||
Decode([]byte) (iter.Seq2[[]*v1.ResourceSpans, error], error)
|
||||
Decode([]byte) (iter.Seq2[*tempopb.PushSpansRequest, error], error)
|
||||
}
|
||||
|
||||
// PushBytesDecoder unmarshals tempopb.PushBytesRequest.
|
||||
@ -165,7 +164,7 @@ func NewPushBytesDecoder() *PushBytesDecoder {
|
||||
}
|
||||
|
||||
// Decode implements GeneratorCodec.
|
||||
func (d *PushBytesDecoder) Decode(data []byte) (iter.Seq2[[]*v1.ResourceSpans, error], error) {
|
||||
func (d *PushBytesDecoder) Decode(data []byte) (iter.Seq2[*tempopb.PushSpansRequest, error], error) {
|
||||
d.dec.Reset()
|
||||
spanBytes, err := d.dec.Decode(data)
|
||||
if err != nil {
|
||||
@ -173,12 +172,15 @@ func (d *PushBytesDecoder) Decode(data []byte) (iter.Seq2[[]*v1.ResourceSpans, e
|
||||
}
|
||||
|
||||
trace := tempopb.Trace{}
|
||||
return func(yield func([]*v1.ResourceSpans, error) bool) {
|
||||
return func(yield func(*tempopb.PushSpansRequest, error) bool) {
|
||||
for _, tr := range spanBytes.Traces {
|
||||
trace.Reset()
|
||||
err = trace.Unmarshal(tr.Slice)
|
||||
|
||||
yield(trace.ResourceSpans, err)
|
||||
yield(&tempopb.PushSpansRequest{
|
||||
Batches: trace.ResourceSpans,
|
||||
SkipMetricsGeneration: spanBytes.SkipMetricsGeneration,
|
||||
}, err)
|
||||
|
||||
tempopb.ReuseByteSlices([][]byte{tr.Slice})
|
||||
}
|
||||
@ -195,14 +197,20 @@ func NewOTLPDecoder() *OTLPDecoder {
|
||||
}
|
||||
|
||||
// Decode implements GeneratorCodec.
|
||||
func (d *OTLPDecoder) Decode(data []byte) (iter.Seq2[[]*v1.ResourceSpans, error], error) {
|
||||
func (d *OTLPDecoder) Decode(data []byte) (iter.Seq2[*tempopb.PushSpansRequest, error], error) {
|
||||
d.trace.ResourceSpans = d.trace.ResourceSpans[:0]
|
||||
err := d.trace.Unmarshal(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return func(yield func([]*v1.ResourceSpans, error) bool) {
|
||||
yield(d.trace.ResourceSpans, nil)
|
||||
return func(yield func(*tempopb.PushSpansRequest, error) bool) {
|
||||
yield(&tempopb.PushSpansRequest{
|
||||
Batches: d.trace.ResourceSpans,
|
||||
// ptrace.Traces does not contain a flag that translates to this field, if we
|
||||
// ever want to skip spans in this record type we'll need to propagate this via
|
||||
// record metadata.
|
||||
SkipMetricsGeneration: false,
|
||||
}, nil)
|
||||
}, nil
|
||||
}
|
||||
|
@ -1987,6 +1987,9 @@ type PushBytesRequest struct {
|
||||
Traces []PreallocBytes `protobuf:"bytes,2,rep,name=traces,proto3,customtype=PreallocBytes" json:"traces"`
|
||||
// trace ids. length must match traces
|
||||
Ids [][]byte `protobuf:"bytes,3,rep,name=ids,proto3" json:"ids,omitempty"`
|
||||
// indicates whether metrics generation should be skipped
|
||||
// for traces contained in this request.
|
||||
SkipMetricsGeneration bool `protobuf:"varint,5,opt,name=skipMetricsGeneration,proto3" json:"skipMetricsGeneration,omitempty"`
|
||||
}
|
||||
|
||||
func (m *PushBytesRequest) Reset() { *m = PushBytesRequest{} }
|
||||
@ -2029,9 +2032,19 @@ func (m *PushBytesRequest) GetIds() [][]byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *PushBytesRequest) GetSkipMetricsGeneration() bool {
|
||||
if m != nil {
|
||||
return m.SkipMetricsGeneration
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type PushSpansRequest struct {
|
||||
// just send entire OTel spans for now
|
||||
Batches []*v11.ResourceSpans `protobuf:"bytes,1,rep,name=batches,proto3" json:"batches,omitempty"`
|
||||
// indicates whether metrics generation should be skipped
|
||||
// for traces contained in this request.
|
||||
SkipMetricsGeneration bool `protobuf:"varint,2,opt,name=skipMetricsGeneration,proto3" json:"skipMetricsGeneration,omitempty"`
|
||||
}
|
||||
|
||||
func (m *PushSpansRequest) Reset() { *m = PushSpansRequest{} }
|
||||
@ -2074,6 +2087,13 @@ func (m *PushSpansRequest) GetBatches() []*v11.ResourceSpans {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *PushSpansRequest) GetSkipMetricsGeneration() bool {
|
||||
if m != nil {
|
||||
return m.SkipMetricsGeneration
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type TraceBytes struct {
|
||||
// pre-marshalled Traces
|
||||
Traces [][]byte `protobuf:"bytes,1,rep,name=traces,proto3" json:"traces,omitempty"`
|
||||
@ -3595,191 +3615,192 @@ func init() {
|
||||
func init() { proto.RegisterFile("pkg/tempopb/tempo.proto", fileDescriptor_f22805646f4f62b6) }
|
||||
|
||||
var fileDescriptor_f22805646f4f62b6 = []byte{
|
||||
// 2939 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x3a, 0x4b, 0x6f, 0x23, 0xc7,
|
||||
0xd1, 0x1a, 0xf1, 0xa9, 0x22, 0x29, 0x51, 0x2d, 0x59, 0xe6, 0x72, 0xd7, 0xd2, 0x7a, 0xbc, 0x30,
|
||||
0xf4, 0xf9, 0x41, 0x69, 0xe9, 0x35, 0x3e, 0xaf, 0x9d, 0x38, 0x90, 0x56, 0xcc, 0x5a, 0xb6, 0x5e,
|
||||
0x6e, 0xd2, 0xb2, 0x11, 0x18, 0x10, 0x46, 0x64, 0x2f, 0x77, 0x20, 0x72, 0x86, 0x9e, 0x19, 0xca,
|
||||
0x52, 0x0e, 0x46, 0x12, 0x20, 0x87, 0x00, 0x39, 0xe4, 0x90, 0x1c, 0xf2, 0x0b, 0x82, 0xe4, 0x92,
|
||||
0x43, 0xf2, 0x0f, 0x82, 0x18, 0x0e, 0x82, 0x04, 0x06, 0x72, 0x31, 0x72, 0x30, 0x02, 0xfb, 0x90,
|
||||
0xfc, 0x8c, 0xa0, 0xab, 0xbb, 0xe7, 0x4d, 0x69, 0xd7, 0xbb, 0x46, 0x7c, 0xf0, 0x89, 0xdd, 0xd5,
|
||||
0xd5, 0xd5, 0x55, 0xd5, 0xf5, 0xec, 0x21, 0x3c, 0x39, 0x3a, 0xe9, 0xaf, 0x79, 0x6c, 0x38, 0xb2,
|
||||
0x47, 0xc7, 0xe2, 0xb7, 0x31, 0x72, 0x6c, 0xcf, 0x26, 0x05, 0x09, 0xac, 0x2f, 0x75, 0xed, 0xe1,
|
||||
0xd0, 0xb6, 0xd6, 0x4e, 0x6f, 0xae, 0x89, 0x91, 0x40, 0xa8, 0xbf, 0xd8, 0x37, 0xbd, 0xfb, 0xe3,
|
||||
0xe3, 0x46, 0xd7, 0x1e, 0xae, 0xf5, 0xed, 0xbe, 0xbd, 0x86, 0xe0, 0xe3, 0xf1, 0x3d, 0x9c, 0xe1,
|
||||
0x04, 0x47, 0x12, 0x7d, 0xd1, 0x73, 0x8c, 0x2e, 0xe3, 0x54, 0x70, 0x20, 0xa0, 0xfa, 0x1f, 0x35,
|
||||
0xa8, 0x76, 0xf8, 0x7c, 0xf3, 0x7c, 0x7b, 0x8b, 0xb2, 0x0f, 0xc6, 0xcc, 0xf5, 0x48, 0x0d, 0x0a,
|
||||
0x88, 0xb3, 0xbd, 0x55, 0xd3, 0xae, 0x6b, 0xab, 0x65, 0xaa, 0xa6, 0x64, 0x19, 0xe0, 0x78, 0x60,
|
||||
0x77, 0x4f, 0xda, 0x9e, 0xe1, 0x78, 0xb5, 0xe9, 0xeb, 0xda, 0xea, 0x0c, 0x0d, 0x41, 0x48, 0x1d,
|
||||
0x8a, 0x38, 0x6b, 0x59, 0xbd, 0x5a, 0x06, 0x57, 0xfd, 0x39, 0xb9, 0x06, 0x33, 0x1f, 0x8c, 0x99,
|
||||
0x73, 0xbe, 0x6b, 0xf7, 0x58, 0x2d, 0x87, 0x8b, 0x01, 0x80, 0xbc, 0x00, 0xf3, 0xc6, 0x60, 0x60,
|
||||
0x7f, 0x78, 0x60, 0x38, 0x9e, 0x69, 0x0c, 0x90, 0xa7, 0x5a, 0xfe, 0xba, 0xb6, 0x5a, 0xa4, 0xc9,
|
||||
0x05, 0xfd, 0x3f, 0x1a, 0xcc, 0x87, 0xd8, 0x76, 0x47, 0xb6, 0xe5, 0x32, 0x72, 0x03, 0x72, 0xc8,
|
||||
0x28, 0x72, 0x5d, 0x6a, 0xce, 0x36, 0xa4, 0x0a, 0x1b, 0x88, 0x4a, 0xc5, 0x22, 0x79, 0x09, 0x0a,
|
||||
0x43, 0xe6, 0x39, 0x66, 0xd7, 0x45, 0x01, 0x4a, 0xcd, 0x2b, 0x51, 0x3c, 0x4e, 0x72, 0x57, 0x20,
|
||||
0x50, 0x85, 0x49, 0x6e, 0x43, 0xde, 0xf5, 0x0c, 0x6f, 0xec, 0xa2, 0x58, 0xb3, 0xcd, 0xa7, 0x93,
|
||||
0x7b, 0x14, 0x1b, 0x8d, 0x36, 0x22, 0x52, 0xb9, 0x81, 0x6b, 0x73, 0xc8, 0x5c, 0xd7, 0xe8, 0xb3,
|
||||
0x5a, 0x16, 0xa5, 0x56, 0x53, 0xfd, 0x19, 0xc8, 0x0b, 0x5c, 0x52, 0x86, 0xe2, 0x9d, 0xfd, 0xdd,
|
||||
0x83, 0x9d, 0x56, 0xa7, 0x55, 0x9d, 0x22, 0x25, 0x28, 0x1c, 0x6c, 0xd0, 0xce, 0xf6, 0xc6, 0x4e,
|
||||
0x55, 0xd3, 0x5f, 0x0d, 0x5d, 0x90, 0x64, 0x8b, 0x3c, 0x0b, 0xb3, 0xa6, 0xe5, 0x8e, 0x58, 0xd7,
|
||||
0x63, 0xbd, 0xcd, 0x73, 0x8f, 0xb9, 0x28, 0x71, 0x96, 0xc6, 0xa0, 0xfa, 0xdf, 0xa7, 0xa1, 0xd2,
|
||||
0x66, 0x86, 0xd3, 0xbd, 0xaf, 0xae, 0xf6, 0x55, 0xc8, 0x76, 0x8c, 0x3e, 0xc7, 0xcf, 0xac, 0x96,
|
||||
0x9a, 0xd7, 0x7d, 0x29, 0x22, 0x58, 0x0d, 0x8e, 0xd2, 0xb2, 0x3c, 0xe7, 0x7c, 0x33, 0xfb, 0xc9,
|
||||
0xe7, 0x2b, 0x53, 0x14, 0xf7, 0x90, 0x1b, 0x50, 0xd9, 0x35, 0xad, 0xad, 0xb1, 0x63, 0x78, 0xa6,
|
||||
0x6d, 0xed, 0x0a, 0xf5, 0x55, 0x68, 0x14, 0x88, 0x58, 0xc6, 0x59, 0x08, 0x2b, 0x23, 0xb1, 0xc2,
|
||||
0x40, 0xb2, 0x08, 0xb9, 0x1d, 0x73, 0x68, 0x7a, 0xa8, 0x92, 0x0a, 0x15, 0x13, 0x0e, 0x75, 0xd1,
|
||||
0xb2, 0x72, 0x02, 0x8a, 0x13, 0x52, 0x85, 0x0c, 0xb3, 0x7a, 0x68, 0x0c, 0x15, 0xca, 0x87, 0x1c,
|
||||
0xef, 0x6d, 0x6e, 0x39, 0xb5, 0x22, 0x2a, 0x54, 0x4c, 0xc8, 0x2a, 0xcc, 0xb5, 0x47, 0x86, 0xe5,
|
||||
0x1e, 0x30, 0x87, 0xff, 0xb6, 0x99, 0x57, 0x9b, 0xc1, 0x3d, 0x71, 0x70, 0xfd, 0xff, 0x61, 0xc6,
|
||||
0x17, 0x91, 0x93, 0x3f, 0x61, 0xe7, 0xa8, 0xc1, 0x19, 0xca, 0x87, 0x9c, 0xfc, 0xa9, 0x31, 0x18,
|
||||
0x33, 0x69, 0xe0, 0x62, 0xf2, 0xea, 0xf4, 0x2b, 0x9a, 0xfe, 0x71, 0x06, 0x88, 0x50, 0xd5, 0x26,
|
||||
0x37, 0x6b, 0xa5, 0xd5, 0x5b, 0x30, 0xe3, 0x2a, 0x05, 0x4a, 0xe3, 0x5b, 0x4a, 0x57, 0x2d, 0x0d,
|
||||
0x10, 0xb9, 0x61, 0xa0, 0x73, 0x6c, 0x6f, 0xc9, 0x83, 0xd4, 0x94, 0xbb, 0x0a, 0x8a, 0x7e, 0xc0,
|
||||
0x8d, 0x46, 0xe8, 0x2f, 0x00, 0x70, 0x0d, 0x8f, 0x8c, 0x3e, 0x73, 0x3b, 0xb6, 0x20, 0x2d, 0x75,
|
||||
0x18, 0x05, 0x72, 0x57, 0x64, 0x56, 0xd7, 0xee, 0x99, 0x56, 0x5f, 0x7a, 0x9b, 0x3f, 0xe7, 0x14,
|
||||
0x4c, 0xab, 0xc7, 0xce, 0x38, 0xb9, 0xb6, 0xf9, 0x43, 0x26, 0x75, 0x1b, 0x05, 0x12, 0x1d, 0xca,
|
||||
0x9e, 0xed, 0x19, 0x03, 0xca, 0xba, 0xb6, 0xd3, 0x73, 0x6b, 0x05, 0x44, 0x8a, 0xc0, 0x38, 0x4e,
|
||||
0xcf, 0xf0, 0x8c, 0x96, 0x3a, 0x49, 0x5c, 0x48, 0x04, 0xc6, 0xe5, 0x3c, 0x65, 0x8e, 0x6b, 0xda,
|
||||
0x16, 0xde, 0xc7, 0x0c, 0x55, 0x53, 0x42, 0x20, 0xeb, 0xf2, 0xe3, 0x01, 0xad, 0x17, 0xc7, 0x3c,
|
||||
0xc4, 0xdc, 0xb3, 0x6d, 0x8f, 0x39, 0xc8, 0x58, 0x09, 0xcf, 0x0c, 0x41, 0xc8, 0x16, 0x54, 0x7b,
|
||||
0xac, 0x67, 0x76, 0x0d, 0x8f, 0xf5, 0xee, 0xd8, 0x83, 0xf1, 0xd0, 0x72, 0x6b, 0x65, 0xb4, 0xe6,
|
||||
0x9a, 0xaf, 0xf2, 0xad, 0x28, 0x02, 0x4d, 0xec, 0xd0, 0xff, 0xa4, 0xc1, 0x5c, 0x0c, 0x8b, 0xdc,
|
||||
0x82, 0x9c, 0xdb, 0xb5, 0x47, 0x4c, 0xba, 0xf8, 0xf2, 0x24, 0x72, 0x8d, 0x36, 0xc7, 0xa2, 0x02,
|
||||
0x99, 0xcb, 0x60, 0x19, 0x43, 0x65, 0x2b, 0x38, 0x26, 0x37, 0x21, 0xeb, 0x9d, 0x8f, 0x44, 0x1c,
|
||||
0x9a, 0x6d, 0x3e, 0x35, 0x91, 0x50, 0xe7, 0x7c, 0xc4, 0x28, 0xa2, 0xea, 0x2b, 0x90, 0x43, 0xb2,
|
||||
0xa4, 0x08, 0xd9, 0xf6, 0xc1, 0xc6, 0x5e, 0x75, 0x8a, 0x07, 0x05, 0xda, 0x6a, 0xef, 0xbf, 0x43,
|
||||
0xef, 0xb4, 0xaa, 0x9a, 0x4e, 0x20, 0xcb, 0xd1, 0x09, 0x40, 0xbe, 0xdd, 0xa1, 0xdb, 0x7b, 0x77,
|
||||
0xab, 0x53, 0xfa, 0x19, 0xcc, 0x2a, 0xeb, 0x92, 0x21, 0xf0, 0x16, 0xe4, 0x31, 0xca, 0x29, 0x0f,
|
||||
0xbf, 0x16, 0x8d, 0x53, 0x02, 0x7b, 0x97, 0x79, 0x06, 0xbf, 0x21, 0x2a, 0x71, 0xc9, 0x7a, 0x3c,
|
||||
0x24, 0xc6, 0xad, 0x37, 0x1e, 0x0f, 0xf5, 0x7f, 0x64, 0x60, 0x21, 0x85, 0x62, 0x3c, 0x75, 0xcc,
|
||||
0x04, 0xa9, 0x63, 0x15, 0xe6, 0x1c, 0xdb, 0xf6, 0xda, 0xcc, 0x39, 0x35, 0xbb, 0x6c, 0x2f, 0x50,
|
||||
0x59, 0x1c, 0xcc, 0xad, 0x93, 0x83, 0x90, 0x3c, 0xe2, 0x89, 0x4c, 0x12, 0x05, 0xf2, 0x84, 0x81,
|
||||
0x2e, 0xd1, 0x31, 0x87, 0xec, 0x1d, 0xcb, 0x3c, 0xdb, 0x33, 0x2c, 0x1b, 0x3d, 0x21, 0x4b, 0x93,
|
||||
0x0b, 0xdc, 0xaa, 0x7a, 0x41, 0x48, 0x12, 0xe1, 0x25, 0x04, 0x21, 0xcf, 0x41, 0xc1, 0x95, 0x31,
|
||||
0x23, 0x8f, 0x1a, 0xa8, 0x06, 0x1a, 0x10, 0x70, 0xaa, 0x10, 0xc8, 0x0b, 0x50, 0x94, 0x43, 0xee,
|
||||
0x13, 0x99, 0x54, 0x64, 0x1f, 0x83, 0x50, 0x28, 0xbb, 0x42, 0x38, 0x1e, 0xeb, 0xdd, 0x5a, 0x11,
|
||||
0x77, 0x34, 0x2e, 0xba, 0x97, 0x46, 0x3b, 0xb4, 0x01, 0x83, 0x14, 0x8d, 0xd0, 0xa8, 0x1f, 0xc2,
|
||||
0x7c, 0x02, 0x25, 0x25, 0x8e, 0x3d, 0x1f, 0x8e, 0x63, 0xa5, 0xe6, 0x13, 0xa1, 0x4b, 0x0d, 0x36,
|
||||
0x87, 0xc3, 0xdb, 0x0e, 0x94, 0xc3, 0x4b, 0x18, 0x87, 0x46, 0x86, 0x75, 0xc7, 0x1e, 0x5b, 0x1e,
|
||||
0x12, 0xe6, 0x71, 0x48, 0x01, 0xb8, 0x4e, 0x99, 0xe3, 0xd8, 0x8e, 0x58, 0x16, 0xc9, 0x20, 0x04,
|
||||
0xd1, 0x7f, 0xaa, 0x41, 0x41, 0xea, 0x83, 0x3c, 0x03, 0x39, 0xbe, 0x51, 0x99, 0x65, 0x25, 0xa2,
|
||||
0x30, 0x2a, 0xd6, 0x30, 0x53, 0x1a, 0x5e, 0xf7, 0x3e, 0xeb, 0x49, 0x6a, 0x6a, 0x4a, 0x5e, 0x03,
|
||||
0x30, 0x3c, 0xcf, 0x31, 0x8f, 0xc7, 0x3c, 0xd9, 0x65, 0x90, 0xc6, 0x55, 0x9f, 0x86, 0x2c, 0x8b,
|
||||
0x4e, 0x6f, 0x36, 0xde, 0x62, 0xe7, 0x87, 0x5c, 0x1a, 0x1a, 0x42, 0xe7, 0xbe, 0x9e, 0xe5, 0xc7,
|
||||
0x90, 0x25, 0xc8, 0xf3, 0x83, 0x7c, 0xdb, 0x94, 0xb3, 0x54, 0x17, 0x4e, 0x35, 0xaf, 0xcc, 0x24,
|
||||
0xf3, 0xba, 0x01, 0x15, 0x65, 0x4c, 0x7c, 0xee, 0x4a, 0x43, 0x8c, 0x02, 0x63, 0x52, 0xe4, 0x1e,
|
||||
0x4e, 0x8a, 0x5f, 0xfb, 0xb9, 0x5c, 0x55, 0x01, 0xab, 0x30, 0xe7, 0xe7, 0xfb, 0x8e, 0x72, 0x7a,
|
||||
0xcc, 0x77, 0x31, 0x70, 0x4a, 0xbd, 0x30, 0x9d, 0x56, 0x2f, 0x90, 0xeb, 0x50, 0xc2, 0xe8, 0x8e,
|
||||
0xc9, 0x4d, 0x65, 0xee, 0x30, 0x88, 0x0b, 0xda, 0xb5, 0x87, 0xa3, 0x01, 0xf3, 0x58, 0xef, 0x4d,
|
||||
0xfb, 0xd8, 0x55, 0xb9, 0x27, 0x02, 0xe4, 0x76, 0x83, 0x9b, 0x10, 0x43, 0x38, 0x5b, 0x00, 0xe0,
|
||||
0x7c, 0x07, 0x24, 0x05, 0x3b, 0x79, 0x64, 0x27, 0x0e, 0x8e, 0xf0, 0x8d, 0x39, 0x1c, 0x73, 0x50,
|
||||
0x98, 0x6f, 0x84, 0xea, 0x7f, 0xd6, 0xb8, 0x43, 0x70, 0xdd, 0xf0, 0xb4, 0xae, 0xb2, 0xf2, 0xa2,
|
||||
0x8a, 0xe7, 0xe2, 0xb6, 0x65, 0xbc, 0x5e, 0x84, 0x1c, 0x56, 0x9d, 0x2a, 0xb9, 0xe3, 0x24, 0xa8,
|
||||
0x3c, 0x32, 0x29, 0x95, 0x47, 0x36, 0xa8, 0x3c, 0x56, 0x61, 0x6e, 0x68, 0x9c, 0xf1, 0x53, 0x78,
|
||||
0x39, 0x81, 0xd4, 0x85, 0x7c, 0x71, 0x30, 0x69, 0xc2, 0xa2, 0xeb, 0x19, 0x03, 0x86, 0x37, 0xe9,
|
||||
0x76, 0xee, 0x3b, 0xcc, 0xbd, 0x6f, 0x0f, 0x54, 0x19, 0x93, 0xba, 0xa6, 0xff, 0x2e, 0x0b, 0x4b,
|
||||
0x81, 0x1c, 0x91, 0x12, 0xe3, 0x95, 0x64, 0x89, 0x51, 0x8f, 0x05, 0xe9, 0x90, 0xec, 0xdf, 0x96,
|
||||
0x19, 0xdf, 0x88, 0x32, 0x23, 0xcd, 0x5c, 0x2a, 0xe9, 0xe6, 0xb2, 0x0e, 0x0b, 0x81, 0x49, 0x04,
|
||||
0xd6, 0x32, 0x8b, 0xd8, 0x69, 0x4b, 0xfa, 0x67, 0x19, 0xb8, 0xea, 0x5f, 0xbc, 0xb0, 0xa4, 0x88,
|
||||
0xc5, 0x7c, 0x37, 0x69, 0x31, 0x2b, 0x49, 0x8b, 0x11, 0x1b, 0xbf, 0x35, 0x9b, 0x6f, 0x54, 0x75,
|
||||
0xda, 0x53, 0x5d, 0x86, 0x70, 0x69, 0x59, 0xdb, 0xd5, 0xa1, 0xe8, 0x19, 0x7d, 0x5e, 0xfc, 0x88,
|
||||
0x34, 0x3a, 0x43, 0xfd, 0x39, 0x69, 0xc6, 0x2b, 0xb8, 0xe0, 0x38, 0x55, 0x55, 0x24, 0x6a, 0xb8,
|
||||
0x8f, 0x60, 0x31, 0x38, 0xe5, 0xb0, 0xe9, 0x9f, 0xd3, 0x84, 0x3c, 0x86, 0x4a, 0x95, 0xac, 0xd3,
|
||||
0xe2, 0xcc, 0x61, 0x53, 0x14, 0xc1, 0x12, 0xf3, 0x2b, 0x9d, 0xff, 0x5a, 0x38, 0x68, 0x4b, 0x82,
|
||||
0x7e, 0x2e, 0xd6, 0x42, 0xb9, 0x98, 0x40, 0xd6, 0xe3, 0x4d, 0xeb, 0x34, 0x0a, 0x8d, 0x63, 0xfd,
|
||||
0x63, 0x2d, 0x14, 0x2a, 0x23, 0x46, 0x8c, 0x35, 0xa8, 0xd0, 0x8b, 0x5f, 0x83, 0x8a, 0xe9, 0x65,
|
||||
0xb1, 0x3f, 0x9b, 0x12, 0xfb, 0x73, 0x41, 0xec, 0xd7, 0xa1, 0x2c, 0xbc, 0x56, 0x1c, 0x27, 0xcd,
|
||||
0x32, 0x02, 0x9b, 0xe4, 0xc6, 0x85, 0xc9, 0x6e, 0x7c, 0x02, 0x4f, 0x26, 0xe4, 0x90, 0x17, 0xc1,
|
||||
0xd3, 0xa8, 0x7f, 0x9a, 0xb8, 0xf1, 0x00, 0xf0, 0x95, 0x54, 0x7e, 0x0b, 0x8a, 0xea, 0x18, 0xd4,
|
||||
0xea, 0xb9, 0x9f, 0x1d, 0x71, 0x9c, 0xde, 0xf9, 0xea, 0x3f, 0xd2, 0xe0, 0x4a, 0x8c, 0xc7, 0x90,
|
||||
0xb9, 0xac, 0xc5, 0xb9, 0x2c, 0x35, 0xe7, 0x83, 0xea, 0x56, 0xae, 0x3c, 0x2a, 0xe3, 0x7f, 0xd1,
|
||||
0x60, 0x2e, 0xb6, 0xf8, 0xa0, 0xaf, 0x20, 0xd1, 0x6a, 0x64, 0x3a, 0x5e, 0x8d, 0x24, 0x2a, 0x9a,
|
||||
0x4c, 0x5a, 0x45, 0x13, 0xab, 0x8c, 0xb2, 0xc9, 0xca, 0x28, 0xa5, 0xaa, 0xc9, 0xa5, 0x56, 0x35,
|
||||
0xfa, 0x1e, 0xe4, 0xb0, 0x2e, 0x23, 0x2d, 0xa8, 0x38, 0xcc, 0xb5, 0xc7, 0x4e, 0x97, 0xb5, 0x43,
|
||||
0xc5, 0x71, 0x10, 0xa5, 0xc5, 0x4b, 0xdd, 0xe9, 0xcd, 0x06, 0x0d, 0xa3, 0xd1, 0xe8, 0x2e, 0x7d,
|
||||
0x0f, 0xca, 0x07, 0x63, 0x37, 0xe8, 0x01, 0x5f, 0x87, 0x0a, 0x56, 0xe1, 0xee, 0xe6, 0x79, 0x47,
|
||||
0x3e, 0x87, 0x65, 0x56, 0x67, 0x43, 0x5a, 0xe6, 0xd8, 0x2d, 0x8e, 0x41, 0x99, 0xe1, 0xda, 0x16,
|
||||
0x8d, 0xa2, 0xeb, 0x6d, 0xa8, 0x72, 0x0c, 0x64, 0x56, 0xf9, 0xd4, 0x8b, 0x7e, 0x5f, 0xc9, 0x9d,
|
||||
0xb0, 0xbc, 0xf9, 0xc4, 0x27, 0x9f, 0xaf, 0x4c, 0xfd, 0xf3, 0xf3, 0x95, 0xca, 0x81, 0xc3, 0x8c,
|
||||
0xc1, 0xc0, 0xee, 0x0a, 0x6c, 0xd5, 0x50, 0x56, 0x21, 0x63, 0xf6, 0x44, 0xa1, 0x5e, 0xa6, 0x7c,
|
||||
0xa8, 0xef, 0x0a, 0xa2, 0x42, 0x00, 0x49, 0xf4, 0x36, 0x14, 0x8e, 0xb1, 0xc0, 0x7f, 0x60, 0xc9,
|
||||
0x15, 0xbe, 0x7e, 0x03, 0x40, 0xbe, 0x8a, 0xf1, 0x1b, 0x5e, 0x8a, 0x74, 0xbd, 0x65, 0xc5, 0x86,
|
||||
0xfe, 0x3a, 0xcc, 0xec, 0x98, 0xd6, 0x49, 0x7b, 0x60, 0x76, 0x79, 0x53, 0x9e, 0x1b, 0x98, 0xd6,
|
||||
0x89, 0x3a, 0xeb, 0x6a, 0xf2, 0x2c, 0x7e, 0x46, 0x83, 0x6f, 0xa0, 0x02, 0x53, 0xff, 0x89, 0x06,
|
||||
0x84, 0x03, 0x95, 0x39, 0x06, 0x85, 0xa5, 0x08, 0x23, 0x5a, 0x38, 0x8c, 0xd4, 0xa0, 0xd0, 0x77,
|
||||
0xec, 0xf1, 0x68, 0x53, 0x85, 0x17, 0x35, 0xe5, 0xf8, 0x03, 0x7c, 0xec, 0x12, 0xfd, 0x83, 0x98,
|
||||
0x3c, 0x68, 0xd8, 0xd1, 0x7f, 0xc6, 0xbd, 0x2f, 0x60, 0xa2, 0x3d, 0x1e, 0x0e, 0x0d, 0xe7, 0xfc,
|
||||
0x7f, 0xc3, 0xcb, 0x6f, 0x35, 0x58, 0x88, 0x28, 0x24, 0x88, 0x54, 0xcc, 0xf5, 0xcc, 0x21, 0x4f,
|
||||
0x62, 0xc8, 0x49, 0x91, 0x06, 0x80, 0x68, 0x1b, 0x29, 0x3a, 0x8f, 0x50, 0x1b, 0xf9, 0x2c, 0xcc,
|
||||
0xa2, 0xfd, 0xb5, 0x7d, 0x14, 0xc1, 0x5a, 0x0c, 0x4a, 0x1a, 0x41, 0xd8, 0xc8, 0xe2, 0x0d, 0x2e,
|
||||
0x46, 0x9a, 0xc8, 0x44, 0xc8, 0xf8, 0x0e, 0x94, 0xa9, 0xf1, 0xe1, 0x1b, 0xa6, 0xeb, 0xd9, 0x7d,
|
||||
0xc7, 0x18, 0x72, 0x23, 0x39, 0x1e, 0x77, 0x4f, 0x98, 0x27, 0xc3, 0x84, 0x9c, 0x71, 0xd9, 0xbb,
|
||||
0x21, 0xce, 0xc4, 0x44, 0x7f, 0x13, 0x8a, 0xaa, 0x0d, 0x4b, 0xe9, 0xac, 0x5f, 0x88, 0x76, 0xd6,
|
||||
0x4b, 0xd1, 0x6e, 0xfe, 0xed, 0x1d, 0xde, 0x3e, 0x9b, 0x5d, 0x15, 0x3f, 0x7f, 0xa9, 0x41, 0x29,
|
||||
0xc4, 0x22, 0xd9, 0x84, 0xf9, 0x81, 0xe1, 0x31, 0xab, 0x7b, 0x7e, 0x74, 0x5f, 0xb1, 0x27, 0xad,
|
||||
0x32, 0xe8, 0xd1, 0xc3, 0xbc, 0xd3, 0xaa, 0xc4, 0x0f, 0xa4, 0xf9, 0x3f, 0xc8, 0xbb, 0xcc, 0x31,
|
||||
0xa5, 0x43, 0x86, 0x43, 0xae, 0xdf, 0x3d, 0x4a, 0x04, 0x2e, 0xb8, 0x70, 0x70, 0xa9, 0x58, 0x39,
|
||||
0xd3, 0xff, 0x16, 0xb5, 0x6e, 0x69, 0x58, 0xc9, 0xa6, 0xff, 0x92, 0xdb, 0x9a, 0x4e, 0xbd, 0xad,
|
||||
0x80, 0xbf, 0xcc, 0x65, 0xfc, 0x55, 0x21, 0x33, 0xba, 0x7d, 0x5b, 0xb6, 0xcc, 0x7c, 0x28, 0x20,
|
||||
0x2f, 0xcb, 0xf8, 0xc9, 0x87, 0x02, 0xb2, 0x2e, 0xfb, 0x44, 0x3e, 0x44, 0xc8, 0xcb, 0xeb, 0xb2,
|
||||
0x21, 0xe4, 0x43, 0xfd, 0x5d, 0xa8, 0xa7, 0xf9, 0x89, 0x34, 0xd1, 0xdb, 0x30, 0xe3, 0x22, 0xc8,
|
||||
0x64, 0xc9, 0x10, 0x90, 0xb2, 0x2f, 0xc0, 0xd6, 0x7f, 0xa5, 0x41, 0x25, 0x72, 0xb1, 0x91, 0xdc,
|
||||
0x99, 0x93, 0xb9, 0xb3, 0x0c, 0x9a, 0x85, 0xca, 0xc8, 0x50, 0xcd, 0xe2, 0xb3, 0x7b, 0xa8, 0x6f,
|
||||
0x8d, 0x6a, 0xf7, 0xf8, 0xcc, 0x95, 0xaf, 0xff, 0x9a, 0xcb, 0x67, 0xc7, 0x28, 0x5c, 0x91, 0x6a,
|
||||
0xc7, 0x7c, 0xd6, 0x93, 0x82, 0x69, 0x3d, 0x7c, 0xa3, 0x10, 0x1f, 0x1a, 0x0a, 0x48, 0x5b, 0x7d,
|
||||
0x45, 0x20, 0x90, 0x3d, 0x31, 0xad, 0x1e, 0x96, 0xb0, 0x39, 0x8a, 0x63, 0x9d, 0x89, 0x07, 0x6f,
|
||||
0xc9, 0xf8, 0x96, 0xe1, 0x19, 0xbc, 0x3e, 0x75, 0x98, 0x3b, 0x1e, 0x78, 0x9d, 0x20, 0xb5, 0x87,
|
||||
0x20, 0xbc, 0xb6, 0x13, 0x33, 0x69, 0x36, 0xf5, 0x54, 0x1f, 0x42, 0x0c, 0x2a, 0x31, 0x79, 0x14,
|
||||
0x9c, 0x4f, 0xac, 0x72, 0x33, 0x19, 0x18, 0xc7, 0x6c, 0x10, 0xaa, 0xb3, 0x02, 0x00, 0xe7, 0x03,
|
||||
0x27, 0x87, 0xa1, 0x6a, 0x22, 0x04, 0x21, 0x6b, 0x30, 0xed, 0x29, 0xd3, 0x58, 0x99, 0xcc, 0xc3,
|
||||
0x81, 0x6d, 0x5a, 0x1e, 0x9d, 0xf6, 0x5c, 0xee, 0x43, 0x4b, 0xe9, 0xcb, 0x78, 0x19, 0xa6, 0x64,
|
||||
0xa2, 0x42, 0x71, 0xcc, 0xad, 0xe3, 0xd4, 0x18, 0xe0, 0xc1, 0x1a, 0xe5, 0x43, 0x9e, 0x9f, 0xd9,
|
||||
0x19, 0x1b, 0x8e, 0x06, 0x86, 0xd3, 0x91, 0x2f, 0x94, 0x19, 0xfc, 0xb8, 0x15, 0x07, 0x93, 0xe7,
|
||||
0xa0, 0xaa, 0x40, 0xea, 0x8b, 0x85, 0x34, 0xce, 0x04, 0x5c, 0x6f, 0xc3, 0x02, 0x7e, 0x7c, 0xd8,
|
||||
0xb6, 0x5c, 0xcf, 0xb0, 0xbc, 0x8b, 0xa3, 0xb2, 0x1f, 0x65, 0x65, 0xa4, 0x89, 0x44, 0x59, 0xe1,
|
||||
0x9b, 0x18, 0x65, 0xcf, 0x60, 0x31, 0x4a, 0x54, 0x9a, 0x70, 0xc3, 0xf7, 0x29, 0x61, 0xbf, 0x41,
|
||||
0xd8, 0x91, 0x98, 0x6d, 0x5c, 0xf5, 0x1d, 0xeb, 0xe1, 0x9f, 0x75, 0x7f, 0xac, 0x41, 0x25, 0x42,
|
||||
0x8b, 0xdc, 0x86, 0x3c, 0x5e, 0x5b, 0xd2, 0x67, 0x92, 0xef, 0x55, 0xf2, 0x6b, 0x91, 0xdc, 0x10,
|
||||
0x2d, 0x26, 0x35, 0x19, 0x0c, 0xc9, 0x0a, 0x94, 0x46, 0x8e, 0x3d, 0x3c, 0x92, 0x54, 0xc5, 0xdb,
|
||||
0x2e, 0x70, 0xd0, 0x0e, 0x42, 0xf4, 0xdf, 0x67, 0x60, 0x1e, 0xc5, 0xa7, 0x86, 0xd5, 0x67, 0x8f,
|
||||
0x45, 0xa3, 0xd8, 0xca, 0x79, 0x6c, 0x24, 0xaf, 0x11, 0xc7, 0xd1, 0xef, 0x91, 0x85, 0xf8, 0xf7,
|
||||
0xc8, 0x50, 0xfb, 0x5b, 0xbc, 0xa0, 0xfd, 0x9d, 0xb9, 0xb4, 0xfd, 0x85, 0xb4, 0xf6, 0x37, 0xd4,
|
||||
0x74, 0x96, 0xa2, 0x4d, 0x67, 0xb8, 0x31, 0x2e, 0xc7, 0x1a, 0x63, 0xd5, 0x90, 0x56, 0x26, 0x36,
|
||||
0xa4, 0xb3, 0x0f, 0xd4, 0x90, 0xce, 0x3d, 0xf4, 0x3b, 0x06, 0xcf, 0xef, 0xd2, 0xf4, 0xdd, 0x5a,
|
||||
0x55, 0xc8, 0xec, 0x03, 0x74, 0x17, 0x48, 0xf8, 0xc2, 0xa4, 0xb5, 0x3e, 0x1f, 0xb3, 0xd6, 0x85,
|
||||
0x20, 0x49, 0x9a, 0x43, 0xf6, 0xc8, 0xa6, 0xfa, 0x11, 0x14, 0x5b, 0x92, 0x83, 0xc7, 0x6f, 0xa4,
|
||||
0x4f, 0x43, 0x99, 0x87, 0x11, 0xd7, 0x33, 0x86, 0xa3, 0xa3, 0xa1, 0xb0, 0xd2, 0x0c, 0x2d, 0xf9,
|
||||
0xb0, 0x5d, 0x57, 0xdf, 0x80, 0x7c, 0xdb, 0xe0, 0x2d, 0x42, 0x02, 0x79, 0x3a, 0x81, 0x1c, 0x9c,
|
||||
0xa2, 0x85, 0x4e, 0xd1, 0x3f, 0xd5, 0x00, 0x02, 0x5d, 0x3c, 0x8a, 0x14, 0x6b, 0x50, 0x70, 0x91,
|
||||
0x19, 0x55, 0x0e, 0xcc, 0x05, 0xea, 0x43, 0xb8, 0xc4, 0x57, 0x58, 0x97, 0x7a, 0x21, 0x79, 0x39,
|
||||
0x7c, 0xe3, 0xd9, 0x58, 0x0a, 0x57, 0x8a, 0x97, 0x54, 0x03, 0xcc, 0xe7, 0xde, 0x87, 0xb9, 0x58,
|
||||
0x77, 0x41, 0xca, 0x50, 0xdc, 0xdb, 0x3f, 0x6a, 0x51, 0xba, 0x4f, 0xab, 0x53, 0x64, 0x01, 0xe6,
|
||||
0x76, 0x37, 0xde, 0x3b, 0xda, 0xd9, 0x3e, 0x6c, 0x1d, 0x75, 0xe8, 0xc6, 0x9d, 0x56, 0xbb, 0xaa,
|
||||
0x71, 0x20, 0x8e, 0x8f, 0x3a, 0xfb, 0xfb, 0x47, 0x3b, 0x1b, 0xf4, 0x6e, 0xab, 0x3a, 0x4d, 0xe6,
|
||||
0xa1, 0xf2, 0xce, 0xde, 0x5b, 0x7b, 0xfb, 0xef, 0xee, 0xc9, 0xcd, 0x99, 0xe6, 0xcf, 0x35, 0xc8,
|
||||
0x73, 0xf2, 0xcc, 0x21, 0xdf, 0x83, 0x19, 0xbf, 0x49, 0x21, 0x57, 0x22, 0xad, 0x4d, 0xb8, 0x71,
|
||||
0xa9, 0x3f, 0x11, 0x59, 0x52, 0xc6, 0xa9, 0x4f, 0x91, 0x0d, 0x28, 0xf9, 0xc8, 0x87, 0xcd, 0xaf,
|
||||
0x42, 0xa2, 0xf9, 0x6f, 0x0d, 0xaa, 0xd2, 0x2e, 0xef, 0x32, 0x8b, 0x39, 0x86, 0x67, 0xfb, 0x8c,
|
||||
0x61, 0xbf, 0x12, 0xa3, 0x1a, 0x6e, 0x7e, 0x26, 0x33, 0xb6, 0x0d, 0x70, 0x97, 0x79, 0xaa, 0x56,
|
||||
0xbc, 0x9a, 0x9e, 0x1c, 0x05, 0x8d, 0x6b, 0x13, 0x32, 0xa7, 0x22, 0x75, 0x17, 0x20, 0x70, 0x4c,
|
||||
0x12, 0xe4, 0xfa, 0x44, 0x78, 0xad, 0x5f, 0x4d, 0x5d, 0xf3, 0x25, 0xfd, 0x4d, 0x16, 0x0a, 0x7c,
|
||||
0xc1, 0x64, 0x0e, 0x79, 0x03, 0x2a, 0xdf, 0x37, 0xad, 0x9e, 0xff, 0xa7, 0x04, 0x72, 0x25, 0xed,
|
||||
0xbf, 0x10, 0x82, 0x6c, 0x7d, 0xf2, 0xdf, 0x24, 0xf0, 0x0a, 0xca, 0xea, 0xf3, 0x65, 0x97, 0x59,
|
||||
0x1e, 0x99, 0xf0, 0xcd, 0xbc, 0xfe, 0x64, 0x02, 0xee, 0x93, 0x68, 0x41, 0x29, 0xf4, 0x3d, 0x3e,
|
||||
0xac, 0xad, 0xc4, 0x57, 0xfa, 0x8b, 0xc8, 0xdc, 0x05, 0x08, 0x9e, 0xa2, 0xc8, 0x05, 0x0f, 0xeb,
|
||||
0xf5, 0xab, 0xa9, 0x6b, 0x3e, 0xa1, 0xb7, 0x94, 0x48, 0xe2, 0x4d, 0xeb, 0x42, 0x52, 0x4f, 0xa5,
|
||||
0xbe, 0xab, 0x85, 0x88, 0x1d, 0xc2, 0x5c, 0xec, 0xd9, 0x85, 0x5c, 0xf6, 0x82, 0x5b, 0xbf, 0x3e,
|
||||
0x19, 0xc1, 0xa7, 0xfb, 0x83, 0xd0, 0xc3, 0x9b, 0x7a, 0xce, 0xb9, 0x9c, 0xb2, 0x3e, 0x09, 0x21,
|
||||
0xcc, 0x73, 0xf3, 0xaf, 0x59, 0xa8, 0xb6, 0x3d, 0x87, 0x19, 0x43, 0xd3, 0xea, 0x2b, 0x93, 0x79,
|
||||
0x0d, 0xf2, 0x32, 0xf1, 0x3d, 0xec, 0x15, 0xaf, 0x6b, 0xdc, 0x1f, 0x1e, 0xcb, 0xdd, 0xac, 0x6b,
|
||||
0x64, 0xf7, 0x31, 0xde, 0xce, 0xba, 0x46, 0xde, 0xfb, 0x7a, 0xee, 0x67, 0x5d, 0x23, 0xef, 0x7f,
|
||||
0x7d, 0x37, 0xb4, 0xae, 0x91, 0x03, 0x98, 0x97, 0xb1, 0xe2, 0xb1, 0x44, 0x87, 0x75, 0x8d, 0x1c,
|
||||
0xc2, 0x42, 0x98, 0xa2, 0x2c, 0x21, 0xc9, 0xb5, 0xe8, 0xbe, 0x68, 0x91, 0x1c, 0xd2, 0x70, 0x5a,
|
||||
0xb5, 0xcb, 0xe9, 0x36, 0xff, 0xa0, 0x41, 0x41, 0x45, 0xc2, 0xa3, 0xd4, 0x6e, 0x55, 0xbf, 0xa8,
|
||||
0x87, 0x93, 0x07, 0x3d, 0x73, 0x21, 0xce, 0x63, 0x8f, 0x96, 0x9b, 0xb5, 0x4f, 0xbe, 0x58, 0xd6,
|
||||
0x3e, 0xfd, 0x62, 0x59, 0xfb, 0xd7, 0x17, 0xcb, 0xda, 0x2f, 0xbe, 0x5c, 0x9e, 0xfa, 0xf4, 0xcb,
|
||||
0xe5, 0xa9, 0xcf, 0xbe, 0x5c, 0x9e, 0x3a, 0xce, 0xe3, 0xbf, 0xee, 0x5e, 0xfa, 0x6f, 0x00, 0x00,
|
||||
0x00, 0xff, 0xff, 0x0c, 0x92, 0xeb, 0xb7, 0xf6, 0x27, 0x00, 0x00,
|
||||
// 2959 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x1a, 0xcb, 0x6e, 0x23, 0xc7,
|
||||
0x51, 0x23, 0xbe, 0x8b, 0xa4, 0x44, 0xb5, 0xb4, 0x32, 0x97, 0xbb, 0x96, 0xd6, 0xe3, 0x85, 0xa1,
|
||||
0xf8, 0x41, 0x69, 0xe9, 0x35, 0xe2, 0xb5, 0x13, 0x07, 0xd2, 0x8a, 0x59, 0xcb, 0xd6, 0xcb, 0x4d,
|
||||
0x5a, 0x36, 0x02, 0x03, 0xc2, 0x88, 0xec, 0xe5, 0x0e, 0x44, 0xce, 0xd0, 0x33, 0x43, 0x59, 0xca,
|
||||
0xc1, 0xc8, 0x03, 0x39, 0x04, 0xc8, 0x21, 0x87, 0xe4, 0x90, 0x2f, 0x08, 0x92, 0x4b, 0x0e, 0xc9,
|
||||
0x1f, 0x04, 0x31, 0x1c, 0x04, 0x09, 0x0c, 0xe4, 0x62, 0xe4, 0x60, 0x04, 0xf6, 0x21, 0xf9, 0x8c,
|
||||
0xa0, 0xab, 0xbb, 0xe7, 0xc5, 0xa1, 0xb4, 0xeb, 0x95, 0x11, 0x1f, 0x7c, 0x62, 0x57, 0x75, 0x75,
|
||||
0x75, 0x75, 0x75, 0x3d, 0x7b, 0x08, 0x4f, 0x0c, 0x8f, 0x7b, 0xab, 0x1e, 0x1b, 0x0c, 0xed, 0xe1,
|
||||
0x91, 0xf8, 0xad, 0x0f, 0x1d, 0xdb, 0xb3, 0x49, 0x4e, 0x22, 0x6b, 0x8b, 0x1d, 0x7b, 0x30, 0xb0,
|
||||
0xad, 0xd5, 0x93, 0x5b, 0xab, 0x62, 0x24, 0x08, 0x6a, 0x2f, 0xf4, 0x4c, 0xef, 0xc1, 0xe8, 0xa8,
|
||||
0xde, 0xb1, 0x07, 0xab, 0x3d, 0xbb, 0x67, 0xaf, 0x22, 0xfa, 0x68, 0x74, 0x1f, 0x21, 0x04, 0x70,
|
||||
0x24, 0xc9, 0x17, 0x3c, 0xc7, 0xe8, 0x30, 0xce, 0x05, 0x07, 0x02, 0xab, 0xff, 0x49, 0x83, 0x4a,
|
||||
0x9b, 0xc3, 0x1b, 0x67, 0x5b, 0x9b, 0x94, 0xbd, 0x3f, 0x62, 0xae, 0x47, 0xaa, 0x90, 0x43, 0x9a,
|
||||
0xad, 0xcd, 0xaa, 0x76, 0x43, 0x5b, 0x29, 0x51, 0x05, 0x92, 0x25, 0x80, 0xa3, 0xbe, 0xdd, 0x39,
|
||||
0x6e, 0x79, 0x86, 0xe3, 0x55, 0xa7, 0x6f, 0x68, 0x2b, 0x05, 0x1a, 0xc2, 0x90, 0x1a, 0xe4, 0x11,
|
||||
0x6a, 0x5a, 0xdd, 0x6a, 0x0a, 0x67, 0x7d, 0x98, 0x5c, 0x87, 0xc2, 0xfb, 0x23, 0xe6, 0x9c, 0xed,
|
||||
0xd8, 0x5d, 0x56, 0xcd, 0xe0, 0x64, 0x80, 0x20, 0xcf, 0xc3, 0x9c, 0xd1, 0xef, 0xdb, 0x1f, 0xec,
|
||||
0x1b, 0x8e, 0x67, 0x1a, 0x7d, 0x94, 0xa9, 0x9a, 0xbd, 0xa1, 0xad, 0xe4, 0xe9, 0xf8, 0x84, 0xfe,
|
||||
0x5f, 0x0d, 0xe6, 0x42, 0x62, 0xbb, 0x43, 0xdb, 0x72, 0x19, 0xb9, 0x09, 0x19, 0x14, 0x14, 0xa5,
|
||||
0x2e, 0x36, 0x66, 0xea, 0x52, 0x85, 0x75, 0x24, 0xa5, 0x62, 0x92, 0xbc, 0x08, 0xb9, 0x01, 0xf3,
|
||||
0x1c, 0xb3, 0xe3, 0xe2, 0x01, 0x8a, 0x8d, 0xab, 0x51, 0x3a, 0xce, 0x72, 0x47, 0x10, 0x50, 0x45,
|
||||
0x49, 0xee, 0x40, 0xd6, 0xf5, 0x0c, 0x6f, 0xe4, 0xe2, 0xb1, 0x66, 0x1a, 0x4f, 0x8d, 0xaf, 0x51,
|
||||
0x62, 0xd4, 0x5b, 0x48, 0x48, 0xe5, 0x02, 0xae, 0xcd, 0x01, 0x73, 0x5d, 0xa3, 0xc7, 0xaa, 0x69,
|
||||
0x3c, 0xb5, 0x02, 0xf5, 0xa7, 0x21, 0x2b, 0x68, 0x49, 0x09, 0xf2, 0x77, 0xf7, 0x76, 0xf6, 0xb7,
|
||||
0x9b, 0xed, 0x66, 0x65, 0x8a, 0x14, 0x21, 0xb7, 0xbf, 0x4e, 0xdb, 0x5b, 0xeb, 0xdb, 0x15, 0x4d,
|
||||
0x7f, 0x25, 0x74, 0x41, 0x52, 0x2c, 0xf2, 0x0c, 0xcc, 0x98, 0x96, 0x3b, 0x64, 0x1d, 0x8f, 0x75,
|
||||
0x37, 0xce, 0x3c, 0xe6, 0xe2, 0x89, 0xd3, 0x34, 0x86, 0xd5, 0xff, 0x31, 0x0d, 0xe5, 0x16, 0x33,
|
||||
0x9c, 0xce, 0x03, 0x75, 0xb5, 0xaf, 0x40, 0xba, 0x6d, 0xf4, 0x38, 0x7d, 0x6a, 0xa5, 0xd8, 0xb8,
|
||||
0xe1, 0x9f, 0x22, 0x42, 0x55, 0xe7, 0x24, 0x4d, 0xcb, 0x73, 0xce, 0x36, 0xd2, 0x1f, 0x7f, 0xb6,
|
||||
0x3c, 0x45, 0x71, 0x0d, 0xb9, 0x09, 0xe5, 0x1d, 0xd3, 0xda, 0x1c, 0x39, 0x86, 0x67, 0xda, 0xd6,
|
||||
0x8e, 0x50, 0x5f, 0x99, 0x46, 0x91, 0x48, 0x65, 0x9c, 0x86, 0xa8, 0x52, 0x92, 0x2a, 0x8c, 0x24,
|
||||
0x0b, 0x90, 0xd9, 0x36, 0x07, 0xa6, 0x87, 0x2a, 0x29, 0x53, 0x01, 0x70, 0xac, 0x8b, 0x96, 0x95,
|
||||
0x11, 0x58, 0x04, 0x48, 0x05, 0x52, 0xcc, 0xea, 0xa2, 0x31, 0x94, 0x29, 0x1f, 0x72, 0xba, 0xb7,
|
||||
0xb8, 0xe5, 0x54, 0xf3, 0xa8, 0x50, 0x01, 0x90, 0x15, 0x98, 0x6d, 0x0d, 0x0d, 0xcb, 0xdd, 0x67,
|
||||
0x0e, 0xff, 0x6d, 0x31, 0xaf, 0x5a, 0xc0, 0x35, 0x71, 0x74, 0xed, 0xdb, 0x50, 0xf0, 0x8f, 0xc8,
|
||||
0xd9, 0x1f, 0xb3, 0x33, 0xd4, 0x60, 0x81, 0xf2, 0x21, 0x67, 0x7f, 0x62, 0xf4, 0x47, 0x4c, 0x1a,
|
||||
0xb8, 0x00, 0x5e, 0x99, 0x7e, 0x59, 0xd3, 0x3f, 0x4a, 0x01, 0x11, 0xaa, 0xda, 0xe0, 0x66, 0xad,
|
||||
0xb4, 0x7a, 0x1b, 0x0a, 0xae, 0x52, 0xa0, 0x34, 0xbe, 0xc5, 0x64, 0xd5, 0xd2, 0x80, 0x90, 0x1b,
|
||||
0x06, 0x3a, 0xc7, 0xd6, 0xa6, 0xdc, 0x48, 0x81, 0xdc, 0x55, 0xf0, 0xe8, 0xfb, 0xdc, 0x68, 0x84,
|
||||
0xfe, 0x02, 0x04, 0xd7, 0xf0, 0xd0, 0xe8, 0x31, 0xb7, 0x6d, 0x0b, 0xd6, 0x52, 0x87, 0x51, 0x24,
|
||||
0x77, 0x45, 0x66, 0x75, 0xec, 0xae, 0x69, 0xf5, 0xa4, 0xb7, 0xf9, 0x30, 0xe7, 0x60, 0x5a, 0x5d,
|
||||
0x76, 0xca, 0xd9, 0xb5, 0xcc, 0x1f, 0x32, 0xa9, 0xdb, 0x28, 0x92, 0xe8, 0x50, 0xf2, 0x6c, 0xcf,
|
||||
0xe8, 0x53, 0xd6, 0xb1, 0x9d, 0xae, 0x5b, 0xcd, 0x21, 0x51, 0x04, 0xc7, 0x69, 0xba, 0x86, 0x67,
|
||||
0x34, 0xd5, 0x4e, 0xe2, 0x42, 0x22, 0x38, 0x7e, 0xce, 0x13, 0xe6, 0xb8, 0xa6, 0x6d, 0xe1, 0x7d,
|
||||
0x14, 0xa8, 0x02, 0x09, 0x81, 0xb4, 0xcb, 0xb7, 0x07, 0xb4, 0x5e, 0x1c, 0xf3, 0x10, 0x73, 0xdf,
|
||||
0xb6, 0x3d, 0xe6, 0xa0, 0x60, 0x45, 0xdc, 0x33, 0x84, 0x21, 0x9b, 0x50, 0xe9, 0xb2, 0xae, 0xd9,
|
||||
0x31, 0x3c, 0xd6, 0xbd, 0x6b, 0xf7, 0x47, 0x03, 0xcb, 0xad, 0x96, 0xd0, 0x9a, 0xab, 0xbe, 0xca,
|
||||
0x37, 0xa3, 0x04, 0x74, 0x6c, 0x85, 0xfe, 0x67, 0x0d, 0x66, 0x63, 0x54, 0xe4, 0x36, 0x64, 0xdc,
|
||||
0x8e, 0x3d, 0x64, 0xd2, 0xc5, 0x97, 0x26, 0xb1, 0xab, 0xb7, 0x38, 0x15, 0x15, 0xc4, 0xfc, 0x0c,
|
||||
0x96, 0x31, 0x50, 0xb6, 0x82, 0x63, 0x72, 0x0b, 0xd2, 0xde, 0xd9, 0x50, 0xc4, 0xa1, 0x99, 0xc6,
|
||||
0x93, 0x13, 0x19, 0xb5, 0xcf, 0x86, 0x8c, 0x22, 0xa9, 0xbe, 0x0c, 0x19, 0x64, 0x4b, 0xf2, 0x90,
|
||||
0x6e, 0xed, 0xaf, 0xef, 0x56, 0xa6, 0x78, 0x50, 0xa0, 0xcd, 0xd6, 0xde, 0xdb, 0xf4, 0x6e, 0xb3,
|
||||
0xa2, 0xe9, 0x04, 0xd2, 0x9c, 0x9c, 0x00, 0x64, 0x5b, 0x6d, 0xba, 0xb5, 0x7b, 0xaf, 0x32, 0xa5,
|
||||
0x9f, 0xc2, 0x8c, 0xb2, 0x2e, 0x19, 0x02, 0x6f, 0x43, 0x16, 0xa3, 0x9c, 0xf2, 0xf0, 0xeb, 0xd1,
|
||||
0x38, 0x25, 0xa8, 0x77, 0x98, 0x67, 0xf0, 0x1b, 0xa2, 0x92, 0x96, 0xac, 0xc5, 0x43, 0x62, 0xdc,
|
||||
0x7a, 0xe3, 0xf1, 0x50, 0xff, 0x67, 0x0a, 0xe6, 0x13, 0x38, 0xc6, 0x53, 0x47, 0x21, 0x48, 0x1d,
|
||||
0x2b, 0x30, 0xeb, 0xd8, 0xb6, 0xd7, 0x62, 0xce, 0x89, 0xd9, 0x61, 0xbb, 0x81, 0xca, 0xe2, 0x68,
|
||||
0x6e, 0x9d, 0x1c, 0x85, 0xec, 0x91, 0x4e, 0x64, 0x92, 0x28, 0x92, 0x27, 0x0c, 0x74, 0x89, 0xb6,
|
||||
0x39, 0x60, 0x6f, 0x5b, 0xe6, 0xe9, 0xae, 0x61, 0xd9, 0xe8, 0x09, 0x69, 0x3a, 0x3e, 0xc1, 0xad,
|
||||
0xaa, 0x1b, 0x84, 0x24, 0x11, 0x5e, 0x42, 0x18, 0xf2, 0x2c, 0xe4, 0x5c, 0x19, 0x33, 0xb2, 0xa8,
|
||||
0x81, 0x4a, 0xa0, 0x01, 0x81, 0xa7, 0x8a, 0x80, 0x3c, 0x0f, 0x79, 0x39, 0xe4, 0x3e, 0x91, 0x4a,
|
||||
0x24, 0xf6, 0x29, 0x08, 0x85, 0x92, 0x2b, 0x0e, 0xc7, 0x63, 0xbd, 0x5b, 0xcd, 0xe3, 0x8a, 0xfa,
|
||||
0x79, 0xf7, 0x52, 0x6f, 0x85, 0x16, 0x60, 0x90, 0xa2, 0x11, 0x1e, 0xb5, 0x03, 0x98, 0x1b, 0x23,
|
||||
0x49, 0x88, 0x63, 0xcf, 0x85, 0xe3, 0x58, 0xb1, 0x71, 0x25, 0x74, 0xa9, 0xc1, 0xe2, 0x70, 0x78,
|
||||
0xdb, 0x86, 0x52, 0x78, 0x0a, 0xe3, 0xd0, 0xd0, 0xb0, 0xee, 0xda, 0x23, 0xcb, 0x43, 0xc6, 0x3c,
|
||||
0x0e, 0x29, 0x04, 0xd7, 0x29, 0x73, 0x1c, 0xdb, 0x11, 0xd3, 0x22, 0x19, 0x84, 0x30, 0xfa, 0xcf,
|
||||
0x34, 0xc8, 0x49, 0x7d, 0x90, 0xa7, 0x21, 0xc3, 0x17, 0x2a, 0xb3, 0x2c, 0x47, 0x14, 0x46, 0xc5,
|
||||
0x1c, 0x66, 0x4a, 0xc3, 0xeb, 0x3c, 0x60, 0x5d, 0xc9, 0x4d, 0x81, 0xe4, 0x55, 0x00, 0xc3, 0xf3,
|
||||
0x1c, 0xf3, 0x68, 0xc4, 0x93, 0x5d, 0x0a, 0x79, 0x5c, 0xf3, 0x79, 0xc8, 0xb2, 0xe8, 0xe4, 0x56,
|
||||
0xfd, 0x4d, 0x76, 0x76, 0xc0, 0x4f, 0x43, 0x43, 0xe4, 0xdc, 0xd7, 0xd3, 0x7c, 0x1b, 0xb2, 0x08,
|
||||
0x59, 0xbe, 0x91, 0x6f, 0x9b, 0x12, 0x4a, 0x74, 0xe1, 0x44, 0xf3, 0x4a, 0x4d, 0x32, 0xaf, 0x9b,
|
||||
0x50, 0x56, 0xc6, 0xc4, 0x61, 0x57, 0x1a, 0x62, 0x14, 0x19, 0x3b, 0x45, 0xe6, 0xd1, 0x4e, 0xf1,
|
||||
0x1b, 0x3f, 0x97, 0xab, 0x2a, 0x60, 0x05, 0x66, 0xfd, 0x7c, 0xdf, 0x56, 0x4e, 0x8f, 0xf9, 0x2e,
|
||||
0x86, 0x4e, 0xa8, 0x17, 0xa6, 0x93, 0xea, 0x05, 0x72, 0x03, 0x8a, 0x18, 0xdd, 0x31, 0xb9, 0xa9,
|
||||
0xcc, 0x1d, 0x46, 0xf1, 0x83, 0x76, 0xec, 0xc1, 0xb0, 0xcf, 0x3c, 0xd6, 0x7d, 0xc3, 0x3e, 0x72,
|
||||
0x55, 0xee, 0x89, 0x20, 0xb9, 0xdd, 0xe0, 0x22, 0xa4, 0x10, 0xce, 0x16, 0x20, 0xb8, 0xdc, 0x01,
|
||||
0x4b, 0x21, 0x4e, 0x16, 0xc5, 0x89, 0xa3, 0x23, 0x72, 0x63, 0x0e, 0xc7, 0x1c, 0x14, 0x96, 0x1b,
|
||||
0xb1, 0xfa, 0x5f, 0x34, 0xee, 0x10, 0x5c, 0x37, 0x3c, 0xad, 0xab, 0xac, 0xbc, 0xa0, 0xe2, 0xb9,
|
||||
0xb8, 0x6d, 0x19, 0xaf, 0x17, 0x20, 0x83, 0x55, 0xa7, 0x4a, 0xee, 0x08, 0x04, 0x95, 0x47, 0x2a,
|
||||
0xa1, 0xf2, 0x48, 0x07, 0x95, 0xc7, 0x0a, 0xcc, 0x0e, 0x8c, 0x53, 0xbe, 0x0b, 0x2f, 0x27, 0x90,
|
||||
0xbb, 0x38, 0x5f, 0x1c, 0x4d, 0x1a, 0xb0, 0xe0, 0x7a, 0x46, 0x9f, 0xe1, 0x4d, 0xba, 0xed, 0x07,
|
||||
0x0e, 0x73, 0x1f, 0xd8, 0x7d, 0x55, 0xc6, 0x24, 0xce, 0xe9, 0xbf, 0x4f, 0xc3, 0x62, 0x70, 0x8e,
|
||||
0x48, 0x89, 0xf1, 0xf2, 0x78, 0x89, 0x51, 0x8b, 0x05, 0xe9, 0xd0, 0xd9, 0xbf, 0x29, 0x33, 0xbe,
|
||||
0x16, 0x65, 0x46, 0x92, 0xb9, 0x94, 0x93, 0xcd, 0x65, 0x0d, 0xe6, 0x03, 0x93, 0x08, 0xac, 0x65,
|
||||
0x06, 0xa9, 0x93, 0xa6, 0xf4, 0x4f, 0x53, 0x70, 0xcd, 0xbf, 0x78, 0x61, 0x49, 0x11, 0x8b, 0xf9,
|
||||
0xee, 0xb8, 0xc5, 0x2c, 0x8f, 0x5b, 0x8c, 0x58, 0xf8, 0x8d, 0xd9, 0x7c, 0xad, 0xaa, 0xd3, 0xae,
|
||||
0xea, 0x32, 0x84, 0x4b, 0xcb, 0xda, 0xae, 0x06, 0x79, 0xcf, 0xe8, 0xf1, 0xe2, 0x47, 0xa4, 0xd1,
|
||||
0x02, 0xf5, 0x61, 0xd2, 0x88, 0x57, 0x70, 0xc1, 0x76, 0xaa, 0xaa, 0x18, 0xab, 0xe1, 0x3e, 0x84,
|
||||
0x85, 0x60, 0x97, 0x83, 0x86, 0xbf, 0x4f, 0x03, 0xb2, 0x18, 0x2a, 0x55, 0xb2, 0x4e, 0x8a, 0x33,
|
||||
0x07, 0x0d, 0x51, 0x04, 0x4b, 0xca, 0x2f, 0xb5, 0xff, 0xab, 0xe1, 0xa0, 0x2d, 0x19, 0xfa, 0xb9,
|
||||
0x58, 0x0b, 0xe5, 0x62, 0x02, 0x69, 0x8f, 0x37, 0xad, 0xd3, 0x78, 0x68, 0x1c, 0xeb, 0x1f, 0x69,
|
||||
0xa1, 0x50, 0x19, 0x31, 0x62, 0xac, 0x41, 0x85, 0x5e, 0xfc, 0x1a, 0x54, 0x80, 0x17, 0xc5, 0xfe,
|
||||
0x74, 0x42, 0xec, 0xcf, 0x04, 0xb1, 0x5f, 0x87, 0x92, 0xf0, 0x5a, 0xb1, 0x9d, 0x34, 0xcb, 0x08,
|
||||
0x6e, 0x92, 0x1b, 0xe7, 0x26, 0xbb, 0xf1, 0x31, 0x3c, 0x31, 0x76, 0x0e, 0x79, 0x11, 0x3c, 0x8d,
|
||||
0xfa, 0xbb, 0x89, 0x1b, 0x0f, 0x10, 0x5f, 0x4a, 0xe5, 0xb7, 0x21, 0xaf, 0xb6, 0x41, 0xad, 0x9e,
|
||||
0xf9, 0xd9, 0x11, 0xc7, 0xc9, 0x9d, 0xaf, 0xfe, 0x23, 0x0d, 0xae, 0xc6, 0x64, 0x0c, 0x99, 0xcb,
|
||||
0x6a, 0x5c, 0xca, 0x62, 0x63, 0x2e, 0xa8, 0x6e, 0xe5, 0xcc, 0xe3, 0x0a, 0xfe, 0x57, 0x0d, 0x66,
|
||||
0x63, 0x93, 0x0f, 0xfb, 0x0a, 0x12, 0xad, 0x46, 0xa6, 0xe3, 0xd5, 0xc8, 0x58, 0x45, 0x93, 0x4a,
|
||||
0xaa, 0x68, 0x62, 0x95, 0x51, 0x7a, 0xbc, 0x32, 0x4a, 0xa8, 0x6a, 0x32, 0x89, 0x55, 0x8d, 0xbe,
|
||||
0x0b, 0x19, 0xac, 0xcb, 0x48, 0x13, 0xca, 0x0e, 0x73, 0xed, 0x91, 0xd3, 0x61, 0xad, 0x50, 0x71,
|
||||
0x1c, 0x44, 0x69, 0xf1, 0x52, 0x77, 0x72, 0xab, 0x4e, 0xc3, 0x64, 0x34, 0xba, 0x4a, 0xdf, 0x85,
|
||||
0xd2, 0xfe, 0xc8, 0x0d, 0x7a, 0xc0, 0xd7, 0xa0, 0x8c, 0x55, 0xb8, 0xbb, 0x71, 0xd6, 0x96, 0xcf,
|
||||
0x61, 0xa9, 0x95, 0x99, 0x90, 0x96, 0x39, 0x75, 0x93, 0x53, 0x50, 0x66, 0xb8, 0xb6, 0x45, 0xa3,
|
||||
0xe4, 0xfa, 0xcf, 0x35, 0xa8, 0x70, 0x12, 0x94, 0x56, 0x39, 0xd5, 0x0b, 0x7e, 0x63, 0xc9, 0xbd,
|
||||
0xb0, 0xb4, 0x71, 0xe5, 0xe3, 0xcf, 0x96, 0xa7, 0xfe, 0xf5, 0xd9, 0x72, 0x79, 0xdf, 0x61, 0x46,
|
||||
0xbf, 0x6f, 0x77, 0x04, 0xb5, 0xea, 0x28, 0x2b, 0x90, 0x32, 0xbb, 0xa2, 0x52, 0x2f, 0x51, 0x3e,
|
||||
0x24, 0xb7, 0xe1, 0x8a, 0x7b, 0x6c, 0x0e, 0xe5, 0xe5, 0xdd, 0x63, 0x16, 0x13, 0xa5, 0x31, 0x6a,
|
||||
0x29, 0x4f, 0x93, 0x27, 0xf5, 0x9f, 0x4a, 0x59, 0xc4, 0xc1, 0xa5, 0x2c, 0x77, 0x20, 0x77, 0x84,
|
||||
0x8d, 0xc1, 0x43, 0x6b, 0x4c, 0xd1, 0x4f, 0x96, 0x62, 0xfa, 0x3c, 0x29, 0x6e, 0x02, 0xc8, 0x37,
|
||||
0x38, 0x6e, 0x4f, 0x8b, 0x91, 0x1e, 0xbb, 0xa4, 0xce, 0xac, 0xbf, 0x06, 0x85, 0x6d, 0xd3, 0x3a,
|
||||
0x6e, 0xf5, 0xcd, 0x0e, 0x23, 0xb7, 0x20, 0xd3, 0x37, 0xad, 0x63, 0x25, 0xe1, 0xb5, 0x71, 0x09,
|
||||
0xb9, 0x64, 0x75, 0xbe, 0x80, 0x0a, 0x4a, 0xfd, 0x27, 0x1a, 0x10, 0x8e, 0x54, 0xc6, 0x1f, 0x94,
|
||||
0xb1, 0x22, 0x68, 0x69, 0xe1, 0xa0, 0x55, 0x85, 0x5c, 0xcf, 0xb1, 0x47, 0xc3, 0x0d, 0x15, 0xcc,
|
||||
0x14, 0xc8, 0xe9, 0xfb, 0xf8, 0xb4, 0x26, 0xba, 0x15, 0x01, 0x3c, 0x6c, 0x90, 0xe3, 0x97, 0x7f,
|
||||
0x35, 0x24, 0x44, 0x6b, 0x34, 0x18, 0x18, 0xce, 0xd9, 0xff, 0x47, 0x96, 0xdf, 0x69, 0x30, 0x1f,
|
||||
0x51, 0x48, 0x10, 0x17, 0x99, 0xeb, 0x99, 0x03, 0x9e, 0x32, 0x51, 0x92, 0x3c, 0x0d, 0x10, 0xd1,
|
||||
0xa6, 0x55, 0xf4, 0x39, 0xa1, 0xa6, 0xf5, 0x19, 0x98, 0x41, 0x6b, 0x6f, 0xf9, 0x24, 0x42, 0xb4,
|
||||
0x18, 0x96, 0xd4, 0x83, 0x20, 0x95, 0xc6, 0x1b, 0x5c, 0x88, 0xb4, 0xac, 0x63, 0x01, 0xea, 0x3b,
|
||||
0x50, 0xa2, 0xc6, 0x07, 0xaf, 0x9b, 0xae, 0x67, 0xf7, 0x1c, 0x63, 0xc0, 0x8d, 0xe4, 0x68, 0xd4,
|
||||
0x39, 0x66, 0x9e, 0x0c, 0x4a, 0x12, 0xe2, 0x67, 0xef, 0x84, 0x24, 0x13, 0x80, 0xfe, 0x06, 0xe4,
|
||||
0x55, 0xd3, 0x97, 0xd0, 0xc7, 0x3f, 0x1f, 0xed, 0xe3, 0x17, 0xa3, 0x6f, 0x07, 0x6f, 0x6d, 0xf3,
|
||||
0x66, 0xdd, 0xec, 0xa8, 0x68, 0xfd, 0x2b, 0x0d, 0x8a, 0x21, 0x11, 0xc9, 0x06, 0xcc, 0xf5, 0x0d,
|
||||
0x8f, 0x59, 0x9d, 0xb3, 0xc3, 0x07, 0x4a, 0x3c, 0x69, 0x95, 0xc1, 0x8b, 0x40, 0x58, 0x76, 0x5a,
|
||||
0x91, 0xf4, 0xc1, 0x69, 0xbe, 0x05, 0x59, 0x97, 0x39, 0xa6, 0xf4, 0xfe, 0x70, 0x80, 0xf7, 0x7b,
|
||||
0x55, 0x49, 0xc0, 0x0f, 0x2e, 0xc2, 0x89, 0x54, 0xac, 0x84, 0xf4, 0xbf, 0x47, 0xad, 0x5b, 0x1a,
|
||||
0xd6, 0xf8, 0x13, 0xc3, 0x05, 0xb7, 0x35, 0x9d, 0x78, 0x5b, 0x81, 0x7c, 0xa9, 0x8b, 0xe4, 0xab,
|
||||
0x40, 0x6a, 0x78, 0xe7, 0x8e, 0x6c, 0xd0, 0xf9, 0x50, 0x60, 0x5e, 0x92, 0xd1, 0x9a, 0x0f, 0x05,
|
||||
0x66, 0x4d, 0x76, 0xa5, 0x7c, 0x88, 0x98, 0x97, 0xd6, 0x64, 0xfb, 0xc9, 0x87, 0xfa, 0x3b, 0x50,
|
||||
0x4b, 0xf2, 0x13, 0x69, 0xa2, 0x77, 0xa0, 0xe0, 0x22, 0xca, 0x64, 0xe3, 0x21, 0x20, 0x61, 0x5d,
|
||||
0x40, 0xad, 0xff, 0x5a, 0x83, 0x72, 0xe4, 0x62, 0x23, 0x99, 0x3a, 0x23, 0x33, 0x75, 0x09, 0x34,
|
||||
0x11, 0xb4, 0x52, 0x54, 0xb3, 0x38, 0x74, 0x1f, 0xf5, 0xad, 0x51, 0xed, 0x3e, 0x87, 0x5c, 0xf9,
|
||||
0xad, 0x41, 0x73, 0x39, 0x74, 0x24, 0x83, 0xac, 0x76, 0xc4, 0xa1, 0xae, 0x3c, 0x98, 0xd6, 0xc5,
|
||||
0x17, 0x11, 0xf1, 0x59, 0x23, 0x87, 0xbc, 0xd5, 0x37, 0x0b, 0x02, 0xe9, 0x63, 0xd3, 0xea, 0x62,
|
||||
0xc1, 0x9c, 0xa1, 0x38, 0xd6, 0x99, 0x78, 0x5e, 0x97, 0x82, 0x6f, 0x1a, 0x9e, 0xc1, 0xab, 0x61,
|
||||
0x87, 0xb9, 0xa3, 0xbe, 0xd7, 0x0e, 0x0a, 0x89, 0x10, 0x86, 0x57, 0x92, 0x02, 0x92, 0x66, 0x53,
|
||||
0x4b, 0xf4, 0x21, 0xa4, 0xa0, 0x92, 0x92, 0x47, 0xc1, 0xb9, 0xb1, 0x59, 0x6e, 0x26, 0x7d, 0xe3,
|
||||
0x88, 0xf5, 0x43, 0x55, 0x5d, 0x80, 0xe0, 0x72, 0x20, 0x70, 0x10, 0xaa, 0x5d, 0x42, 0x18, 0xb2,
|
||||
0x0a, 0xd3, 0x9e, 0x32, 0x8d, 0xe5, 0xc9, 0x32, 0xec, 0xdb, 0xa6, 0xe5, 0xd1, 0x69, 0xcf, 0xe5,
|
||||
0x3e, 0xb4, 0x98, 0x3c, 0x8d, 0x97, 0x61, 0x4a, 0x21, 0xca, 0x14, 0xc7, 0xdc, 0x3a, 0x4e, 0x8c,
|
||||
0x3e, 0x6e, 0xac, 0x51, 0x3e, 0xe4, 0xd5, 0x00, 0x3b, 0x65, 0x83, 0x61, 0xdf, 0x70, 0xda, 0xf2,
|
||||
0x3d, 0x34, 0x85, 0x9f, 0xd2, 0xe2, 0x68, 0xf2, 0x2c, 0x54, 0x14, 0x4a, 0x7d, 0x1f, 0x91, 0xc6,
|
||||
0x39, 0x86, 0xd7, 0x5b, 0x30, 0x8f, 0x9f, 0x3a, 0xb6, 0x2c, 0xd7, 0x33, 0x2c, 0xef, 0xfc, 0xa8,
|
||||
0xec, 0x47, 0x59, 0x19, 0x69, 0x22, 0x51, 0x56, 0xf8, 0x26, 0x46, 0xd9, 0x53, 0x58, 0x88, 0x32,
|
||||
0x95, 0x26, 0x5c, 0xf7, 0x7d, 0x4a, 0xd8, 0x6f, 0x10, 0x76, 0x24, 0x65, 0x0b, 0x67, 0x7d, 0xc7,
|
||||
0x7a, 0xf4, 0x47, 0xe4, 0x1f, 0x6b, 0x50, 0x8e, 0xf0, 0x22, 0x77, 0x20, 0x8b, 0xd7, 0x36, 0xee,
|
||||
0x33, 0xe3, 0xaf, 0x63, 0xf2, 0xdb, 0x94, 0x5c, 0x10, 0x2d, 0x5d, 0x35, 0x19, 0x0c, 0xc9, 0x32,
|
||||
0x14, 0x87, 0x8e, 0x3d, 0x38, 0x94, 0x5c, 0xc5, 0x4b, 0x32, 0x70, 0xd4, 0x36, 0x62, 0xf4, 0x3f,
|
||||
0xa4, 0x60, 0x0e, 0x8f, 0x4f, 0x0d, 0xab, 0xc7, 0x2e, 0x45, 0xa3, 0xd8, 0x38, 0x7a, 0x6c, 0x28,
|
||||
0xaf, 0x11, 0xc7, 0xd1, 0xaf, 0x9f, 0xb9, 0xf8, 0xd7, 0xcf, 0x50, 0xb3, 0x9d, 0x3f, 0xa7, 0xd9,
|
||||
0x2e, 0x5c, 0xd8, 0x6c, 0x43, 0x52, 0xb3, 0x1d, 0x6a, 0x71, 0x8b, 0xd1, 0x16, 0x37, 0xdc, 0x86,
|
||||
0x97, 0x62, 0x6d, 0xb8, 0x6a, 0x7f, 0xcb, 0x13, 0xdb, 0xdf, 0x99, 0x87, 0x6a, 0x7f, 0x67, 0x1f,
|
||||
0xf9, 0xd5, 0x84, 0xe7, 0x77, 0x69, 0xfa, 0x6e, 0xb5, 0x22, 0xce, 0xec, 0x23, 0x74, 0x17, 0x48,
|
||||
0xf8, 0xc2, 0xa4, 0xb5, 0x3e, 0x17, 0xb3, 0xd6, 0xf9, 0x20, 0x49, 0x9a, 0x03, 0xf6, 0xd8, 0xa6,
|
||||
0xfa, 0x21, 0xe4, 0x9b, 0x52, 0x82, 0xcb, 0x37, 0xd2, 0xa7, 0xa0, 0xc4, 0xc3, 0x88, 0xeb, 0x19,
|
||||
0x83, 0xe1, 0xe1, 0x40, 0x58, 0x69, 0x8a, 0x16, 0x7d, 0xdc, 0x8e, 0xab, 0xaf, 0x43, 0xb6, 0x65,
|
||||
0xf0, 0x86, 0x64, 0x8c, 0x78, 0x7a, 0x8c, 0x38, 0xd8, 0x45, 0x0b, 0xed, 0xa2, 0x7f, 0xa2, 0x01,
|
||||
0x04, 0xba, 0x78, 0x9c, 0x53, 0xac, 0x42, 0xce, 0x45, 0x61, 0x54, 0x39, 0x30, 0x1b, 0xa8, 0x0f,
|
||||
0xf1, 0x92, 0x5e, 0x51, 0x5d, 0xe8, 0x85, 0xe4, 0xa5, 0xf0, 0x8d, 0xa7, 0x63, 0x29, 0x5c, 0x29,
|
||||
0x5e, 0x72, 0x0d, 0x28, 0x9f, 0x7d, 0x0f, 0x66, 0x63, 0xbd, 0x0c, 0x29, 0x41, 0x7e, 0x77, 0xef,
|
||||
0xb0, 0x49, 0xe9, 0x1e, 0xad, 0x4c, 0x91, 0x79, 0x98, 0xdd, 0x59, 0x7f, 0xf7, 0x70, 0x7b, 0xeb,
|
||||
0xa0, 0x79, 0xd8, 0xa6, 0xeb, 0x77, 0x9b, 0xad, 0x8a, 0xc6, 0x91, 0x38, 0x3e, 0x6c, 0xef, 0xed,
|
||||
0x1d, 0x6e, 0xaf, 0xd3, 0x7b, 0xcd, 0xca, 0x34, 0x99, 0x83, 0xf2, 0xdb, 0xbb, 0x6f, 0xee, 0xee,
|
||||
0xbd, 0xb3, 0x2b, 0x17, 0xa7, 0x1a, 0xbf, 0xd0, 0x20, 0xcb, 0xd9, 0x33, 0x87, 0x7c, 0x0f, 0x0a,
|
||||
0x7e, 0x47, 0x44, 0xae, 0x46, 0x1a, 0xa9, 0x70, 0x97, 0x54, 0xbb, 0x12, 0x99, 0x52, 0xc6, 0xa9,
|
||||
0x4f, 0x91, 0x75, 0x28, 0xfa, 0xc4, 0x07, 0x8d, 0x2f, 0xc3, 0xa2, 0xf1, 0x1f, 0x0d, 0x2a, 0xd1,
|
||||
0xd6, 0xc4, 0xf6, 0x05, 0xc3, 0x2e, 0x27, 0xc6, 0x35, 0xdc, 0x32, 0x4d, 0x16, 0x6c, 0x0b, 0xe0,
|
||||
0x1e, 0xf3, 0x54, 0xad, 0x78, 0x2d, 0x39, 0x39, 0x0a, 0x1e, 0xd7, 0x27, 0x64, 0x4e, 0xc5, 0xea,
|
||||
0x1e, 0x40, 0xe0, 0x98, 0x24, 0xc8, 0xf5, 0x63, 0xe1, 0xb5, 0x76, 0x2d, 0x71, 0xce, 0x3f, 0xe9,
|
||||
0x6f, 0xd3, 0x90, 0xe3, 0x13, 0x26, 0x73, 0xc8, 0xeb, 0x50, 0xfe, 0xbe, 0x69, 0x75, 0xfd, 0xbf,
|
||||
0x40, 0x90, 0xab, 0x49, 0xff, 0xbc, 0x10, 0x6c, 0x6b, 0x93, 0xff, 0x94, 0x81, 0x57, 0x50, 0x52,
|
||||
0x1f, 0x4b, 0x3b, 0xcc, 0xf2, 0xc8, 0x84, 0x2f, 0xf4, 0xb5, 0x27, 0xc6, 0xf0, 0x3e, 0x8b, 0x26,
|
||||
0x14, 0x43, 0x5f, 0xff, 0xc3, 0xda, 0x1a, 0xfb, 0x4f, 0xc0, 0x79, 0x6c, 0xee, 0x01, 0x04, 0x0f,
|
||||
0x5f, 0xe4, 0x9c, 0x67, 0xfc, 0xda, 0xb5, 0xc4, 0x39, 0x9f, 0xd1, 0x9b, 0xea, 0x48, 0xe2, 0x05,
|
||||
0xed, 0x5c, 0x56, 0x4f, 0x26, 0xbe, 0xe2, 0x85, 0x98, 0x1d, 0xc0, 0x6c, 0xec, 0x91, 0x87, 0x5c,
|
||||
0xf4, 0x5e, 0x5c, 0xbb, 0x31, 0x99, 0xc0, 0xe7, 0xfb, 0x83, 0xd0, 0x33, 0x9f, 0x7a, 0x3c, 0xba,
|
||||
0x98, 0xb3, 0x3e, 0x89, 0x20, 0x2c, 0x73, 0xe3, 0x6f, 0x69, 0xa8, 0xb4, 0x3c, 0x87, 0x19, 0x03,
|
||||
0xd3, 0xea, 0x29, 0x93, 0x79, 0x15, 0xb2, 0x32, 0xf1, 0x3d, 0xea, 0x15, 0xaf, 0x69, 0xdc, 0x1f,
|
||||
0x2e, 0xe5, 0x6e, 0xd6, 0x34, 0xb2, 0x73, 0x89, 0xb7, 0xb3, 0xa6, 0x91, 0x77, 0xbf, 0x9a, 0xfb,
|
||||
0x59, 0xd3, 0xc8, 0x7b, 0x5f, 0xdd, 0x0d, 0xad, 0x69, 0x64, 0x1f, 0xe6, 0x64, 0xac, 0xb8, 0x94,
|
||||
0xe8, 0xb0, 0xa6, 0x91, 0x03, 0x98, 0x0f, 0x73, 0x94, 0x25, 0x24, 0xb9, 0x1e, 0x5d, 0x17, 0x2d,
|
||||
0x92, 0x43, 0x1a, 0x4e, 0xaa, 0x76, 0x39, 0xdf, 0xc6, 0x1f, 0x35, 0xc8, 0xa9, 0x48, 0x78, 0x98,
|
||||
0xd8, 0xad, 0xea, 0xe7, 0xf5, 0x70, 0x72, 0xa3, 0xa7, 0xcf, 0xa5, 0xb9, 0xf4, 0x68, 0xb9, 0x51,
|
||||
0xfd, 0xf8, 0xf3, 0x25, 0xed, 0x93, 0xcf, 0x97, 0xb4, 0x7f, 0x7f, 0xbe, 0xa4, 0xfd, 0xf2, 0x8b,
|
||||
0xa5, 0xa9, 0x4f, 0xbe, 0x58, 0x9a, 0xfa, 0xf4, 0x8b, 0xa5, 0xa9, 0xa3, 0x2c, 0xfe, 0xc7, 0xef,
|
||||
0xc5, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x5f, 0x4a, 0x7d, 0xf9, 0x64, 0x28, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
@ -6402,6 +6423,16 @@ func (m *PushBytesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.SkipMetricsGeneration {
|
||||
i--
|
||||
if m.SkipMetricsGeneration {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x28
|
||||
}
|
||||
if len(m.Ids) > 0 {
|
||||
for iNdEx := len(m.Ids) - 1; iNdEx >= 0; iNdEx-- {
|
||||
i -= len(m.Ids[iNdEx])
|
||||
@ -6448,6 +6479,16 @@ func (m *PushSpansRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.SkipMetricsGeneration {
|
||||
i--
|
||||
if m.SkipMetricsGeneration {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x10
|
||||
}
|
||||
if len(m.Batches) > 0 {
|
||||
for iNdEx := len(m.Batches) - 1; iNdEx >= 0; iNdEx-- {
|
||||
{
|
||||
@ -8306,6 +8347,9 @@ func (m *PushBytesRequest) Size() (n int) {
|
||||
n += 1 + l + sovTempo(uint64(l))
|
||||
}
|
||||
}
|
||||
if m.SkipMetricsGeneration {
|
||||
n += 2
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
@ -8321,6 +8365,9 @@ func (m *PushSpansRequest) Size() (n int) {
|
||||
n += 1 + l + sovTempo(uint64(l))
|
||||
}
|
||||
}
|
||||
if m.SkipMetricsGeneration {
|
||||
n += 2
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
@ -13477,6 +13524,26 @@ func (m *PushBytesRequest) Unmarshal(dAtA []byte) error {
|
||||
m.Ids = append(m.Ids, make([]byte, postIndex-iNdEx))
|
||||
copy(m.Ids[len(m.Ids)-1], dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 5:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field SkipMetricsGeneration", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTempo
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.SkipMetricsGeneration = bool(v != 0)
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipTempo(dAtA[iNdEx:])
|
||||
@ -13561,6 +13628,26 @@ func (m *PushSpansRequest) Unmarshal(dAtA []byte) error {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field SkipMetricsGeneration", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTempo
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.SkipMetricsGeneration = bool(v != 0)
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipTempo(dAtA[iNdEx:])
|
||||
|
@ -37,7 +37,7 @@ service StreamingQuerier {
|
||||
rpc SearchTagsV2(SearchTagsRequest) returns (stream SearchTagsV2Response) {}
|
||||
rpc SearchTagValues(SearchTagValuesRequest) returns (stream SearchTagValuesResponse) {}
|
||||
rpc SearchTagValuesV2(SearchTagValuesRequest) returns (stream SearchTagValuesV2Response) {}
|
||||
rpc MetricsQueryRange(QueryRangeRequest) returns (stream QueryRangeResponse) {}
|
||||
rpc MetricsQueryRange(QueryRangeRequest) returns (stream QueryRangeResponse) {}
|
||||
rpc MetricsQueryInstant(QueryInstantRequest) returns (stream QueryInstantResponse) {}
|
||||
}
|
||||
|
||||
@ -162,11 +162,11 @@ message SearchMetrics {
|
||||
|
||||
message SearchTagsRequest {
|
||||
string scope = 1;
|
||||
string query = 2;
|
||||
string query = 2;
|
||||
uint32 start = 3;
|
||||
uint32 end = 4;
|
||||
uint32 maxTagsPerScope = 5;
|
||||
uint32 staleValuesThreshold =6;
|
||||
uint32 staleValuesThreshold = 6;
|
||||
}
|
||||
|
||||
// SearchTagsBlockRequest takes SearchTagsRequest parameters as well as all information necessary
|
||||
@ -279,11 +279,19 @@ message PushBytesRequest {
|
||||
// trace ids. length must match traces
|
||||
repeated bytes ids = 3;
|
||||
// id 4 previously claimed by SearchData
|
||||
|
||||
// indicates whether metrics generation should be skipped
|
||||
// for traces contained in this request.
|
||||
bool skipMetricsGeneration = 5;
|
||||
}
|
||||
|
||||
message PushSpansRequest {
|
||||
// just send entire OTel spans for now
|
||||
repeated tempopb.trace.v1.ResourceSpans batches = 1;
|
||||
|
||||
// indicates whether metrics generation should be skipped
|
||||
// for traces contained in this request.
|
||||
bool skipMetricsGeneration = 2;
|
||||
}
|
||||
|
||||
message TraceBytes {
|
||||
@ -445,7 +453,7 @@ message Sample {
|
||||
message TimeSeries {
|
||||
// Series labels containing name and value. Data-type aware.
|
||||
repeated tempopb.common.v1.KeyValue labels = 1 [(gogoproto.nullable) = false];
|
||||
|
||||
|
||||
// Sorted by time, oldest sample first.
|
||||
repeated Sample samples = 2 [(gogoproto.nullable) = false];
|
||||
|
||||
|
Reference in New Issue
Block a user