From 859864650ab4c95050a9991def3a68068f29458f Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 23 Sep 2020 23:45:50 +0200 Subject: [PATCH 01/20] tracing: make `span` a union type Clarify that a `span` is anywhere between zero to three subspans: - net/trace, - external opentracing Tracer, and - crdb-internal trace span, where zero subspans corresponds to what is today noopSpan (not implemented yet, but TODO added). Lots of threads to pull, but this is a good checkpoint since everything compiles and the diff is mechanical and small enough to actually review. Release note: None --- pkg/sql/txn_state.go | 2 + pkg/util/tracing/shadow.go | 4 +- pkg/util/tracing/tracer.go | 44 ++++++++++++--------- pkg/util/tracing/tracer_span.go | 70 ++++++++++++++++++++++----------- pkg/util/tracing/tracer_test.go | 2 +- 5 files changed, 77 insertions(+), 45 deletions(-) diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index da97cc0ad6f1..479dfe856d93 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -176,6 +176,8 @@ func (ts *txnState) resetForNewSQLTxn( ) } else { // Create a root span for this SQL txn. + // TODO(tbg): this is the only use of RecordableSpan. Can we instead interchange + // the span when we decide that it needs to be traced? sp = tranCtx.tracer.(*tracing.Tracer).StartRootSpan( opName, logtags.FromContext(connCtx), tracing.RecordableSpan) } diff --git a/pkg/util/tracing/shadow.go b/pkg/util/tracing/shadow.go index 1f04437d84c5..f05bb175cfd5 100644 --- a/pkg/util/tracing/shadow.go +++ b/pkg/util/tracing/shadow.go @@ -99,8 +99,8 @@ func linkShadowSpan( ReferencedContext: parentShadowCtx, }) } - s.shadowTr = shadowTr - s.shadowSpan = shadowTr.StartSpan(s.operation, opts...) + s.otSpan.shadowTr = shadowTr + s.otSpan.shadowSpan = shadowTr.StartSpan(s.operation, opts...) } func createLightStepTracer(token string) (shadowTracerManager, opentracing.Tracer) { diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index edbd11c9ddaa..80d7f93e700a 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -269,10 +269,12 @@ func (t *Tracer) StartSpan( } s := &span{ - tracer: t, - operation: operationName, - startTime: sso.StartTime, - logTags: logTags, + crdbSpan: crdbSpan{ + tracer: t, + operation: operationName, + startTime: sso.StartTime, + logTags: logTags, + }, } if s.startTime.IsZero() { s.startTime = time.Now() @@ -364,14 +366,16 @@ func (t *Tracer) StartRootSpan( } s := &span{ - spanMeta: spanMeta{ - TraceID: uint64(rand.Int63()), - SpanID: uint64(rand.Int63()), + crdbSpan: crdbSpan{ + spanMeta: spanMeta{ + TraceID: uint64(rand.Int63()), + SpanID: uint64(rand.Int63()), + }, + tracer: t, + operation: opName, + startTime: time.Now(), + logTags: logTags, }, - tracer: t, - operation: opName, - startTime: time.Now(), - logTags: logTags, } s.mu.duration = -1 @@ -426,11 +430,13 @@ func StartChildSpan( pSpan := parentSpan.(*span) s := &span{ - tracer: tr, - operation: opName, - startTime: time.Now(), - parentSpanID: pSpan.SpanID, - logTags: logTags, + crdbSpan: crdbSpan{ + tracer: tr, + operation: opName, + startTime: time.Now(), + parentSpanID: pSpan.SpanID, + logTags: logTags, + }, } // Copy baggage from parent. @@ -445,8 +451,8 @@ func StartChildSpan( s.TraceID = pSpan.TraceID s.SpanID = uint64(rand.Int63()) - if pSpan.shadowTr != nil { - linkShadowSpan(s, pSpan.shadowTr, pSpan.shadowSpan.Context(), opentracing.ChildOfRef) + if pSpan.otSpan.shadowTr != nil { + linkShadowSpan(s, pSpan.otSpan.shadowTr, pSpan.otSpan.shadowSpan.Context(), opentracing.ChildOfRef) } recordingType := pSpan.mu.recording.recordingType @@ -463,7 +469,7 @@ func StartChildSpan( } } - if pSpan.netTr != nil || pSpan.shadowTr != nil { + if pSpan.netTr != nil || pSpan.otSpan.shadowTr != nil { // Copy baggage items to tags so they show up in the shadow tracer UI or x/net/trace. for k, v := range s.mu.Baggage { s.SetTag(k, v) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 6daa9baecbd2..7bec3d0283b0 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -102,19 +102,13 @@ const ( SingleNodeRecording ) -type span struct { +type crdbSpan struct { spanMeta parentSpanID uint64 tracer *Tracer - // x/net/trace.Trace instance; nil if not tracing to x/net/trace. - netTr trace.Trace - // Shadow tracer and span; nil if not using a shadow tracer. - shadowTr *shadowTracer - shadowSpan opentracing.Span - operation string startTime time.Time @@ -161,11 +155,41 @@ type span struct { } } +func (s *crdbSpan) isRecording() bool { + return atomic.LoadInt32(&s.recording) != 0 +} + +type otSpan struct { + // TODO(tbg): see if we can lose the shadowTr here and rely on shadowSpan.Tracer(). + // Probably not - but worth checking. + // TODO(tbg): consider renaming 'shadow' -> 'ot' or 'external'. + shadowTr *shadowTracer + shadowSpan opentracing.Span +} + +type span struct { + crdbSpan // can be zero + // x/net/trace.Trace instance; nil if not tracing to x/net/trace. + netTr trace.Trace + // Shadow tracer and span; zero if not using a shadow tracer. + otSpan otSpan +} + +// TODO(tbg): remove this. We don't need *span to be an opentracing.Span. var _ opentracing.Span = &span{} -func (s *span) isRecording() bool { - return atomic.LoadInt32(&s.recording) != 0 +func (s *span) isBlackHole() bool { + return !s.crdbSpan.isRecording() && s.netTr == nil && s.otSpan == (otSpan{}) +} + +/* +TODO(tbg): replace noopSpan with this concept: +func (s *span) isNoop() bool { + // Special case: trace id zero implies that everything else, + // with the exception of the tracer, is also zero. + return s.isBlackHole() && s.crdbSpan.TraceID == 0 } +*/ // IsRecording returns true if the span is recording its events. func IsRecording(s opentracing.Span) bool { @@ -662,7 +686,7 @@ func IsBlackHoleSpan(s opentracing.Span) bool { return true } sp := s.(*span) - return !sp.isRecording() && sp.netTr == nil && sp.shadowTr == nil + return sp.isBlackHole() } // IsNoopContext returns true if the span context is from a "no-op" span. If @@ -698,8 +722,8 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { s.mu.Lock() s.mu.duration = finishTime.Sub(s.startTime) s.mu.Unlock() - if s.shadowTr != nil { - s.shadowSpan.Finish() + if s.otSpan.shadowTr != nil { + s.otSpan.shadowSpan.Finish() } if s.netTr != nil { s.netTr.Finish() @@ -723,9 +747,9 @@ func (s *span) Context() opentracing.SpanContext { span: s, Baggage: baggageCopy, } - if s.shadowTr != nil { - sc.shadowTr = s.shadowTr - sc.shadowCtx = s.shadowSpan.Context() + if s.otSpan.shadowTr != nil { + sc.shadowTr = s.otSpan.shadowTr + sc.shadowCtx = s.otSpan.shadowSpan.Context() } if s.isRecording() { @@ -736,8 +760,8 @@ func (s *span) Context() opentracing.SpanContext { // SetOperationName is part of the opentracing.Span interface. func (s *span) SetOperationName(operationName string) opentracing.Span { - if s.shadowTr != nil { - s.shadowSpan.SetOperationName(operationName) + if s.otSpan.shadowTr != nil { + s.otSpan.shadowSpan.SetOperationName(operationName) } s.operation = operationName return s @@ -749,8 +773,8 @@ func (s *span) SetTag(key string, value interface{}) opentracing.Span { } func (s *span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { - if s.shadowTr != nil { - s.shadowSpan.SetTag(key, value) + if s.otSpan.shadowTr != nil { + s.otSpan.shadowSpan.SetTag(key, value) } if s.netTr != nil { s.netTr.LazyPrintf("%s:%v", key, value) @@ -771,8 +795,8 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci // LogFields is part of the opentracing.Span interface. func (s *span) LogFields(fields ...otlog.Field) { - if s.shadowTr != nil { - s.shadowSpan.LogFields(fields...) + if s.otSpan.shadowTr != nil { + s.otSpan.shadowSpan.LogFields(fields...) } if s.netTr != nil { // TODO(radu): when LightStep supports arbitrary fields, we should make @@ -831,8 +855,8 @@ func (s *span) setBaggageItemLocked(restrictedKey, value string) opentracing.Spa } s.mu.Baggage[restrictedKey] = value - if s.shadowTr != nil { - s.shadowSpan.SetBaggageItem(restrictedKey, value) + if s.otSpan.shadowTr != nil { + s.otSpan.shadowSpan.SetBaggageItem(restrictedKey, value) } // Also set a tag so it shows up in the Lightstep UI or x/net/trace. s.setTagInner(restrictedKey, value, true /* locked */) diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index de68b195f653..c3a4f2bcdb07 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -295,7 +295,7 @@ func TestLightstepContext(t *testing.T) { } s2 := tr.StartSpan("child", opentracing.FollowsFrom(wireContext)) - s2Ctx := s2.(*span).shadowSpan.Context() + s2Ctx := s2.(*span).otSpan.shadowSpan.Context() // Verify that the baggage is correct in both the tracer context and in the // lightstep context. From 3e7d2581d84bf6e959de948ab6bd4da7277bd459 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 8 Oct 2020 13:41:59 +0200 Subject: [PATCH 02/20] tracing: remove noopSpan Remove the noopSpan and noopSpanContext types. Instead, we use a `*span` (and `*spanContext`) with a zero trace ID (and in fact, all zero except for the tracer). This allows a sleuth of further cleanups which are deferred to follow-up commits. Release note: None --- pkg/util/tracing/tracer.go | 56 ++++++++++++++++--------------- pkg/util/tracing/tracer_span.go | 59 +++++---------------------------- pkg/util/tracing/tracer_test.go | 14 ++++---- 3 files changed, 45 insertions(+), 84 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 80d7f93e700a..70130ee63104 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -91,7 +91,7 @@ var zipkinCollector = settings.RegisterPublicStringSetting( type Tracer struct { // Preallocated noopSpan, used to avoid creating spans when we are not using // x/net/trace or lightstep and we are not recording. - noopSpan noopSpan + noopSpan *span // If forceRealSpans is set, this Tracer will always create real spans (never // noopSpans), regardless of the recording or lightstep configuration. Used @@ -113,7 +113,7 @@ var _ opentracing.Tracer = &Tracer{} // backends. func NewTracer() *Tracer { t := &Tracer{} - t.noopSpan.tracer = t + t.noopSpan = &span{crdbSpan: crdbSpan{tracer: t}} return t } @@ -206,7 +206,7 @@ func (t *Tracer) StartSpan( if len(opts) == 1 { if o, ok := opts[0].(opentracing.SpanReference); ok { if IsNoopContext(o.ReferencedContext) { - return &t.noopSpan + return t.noopSpan } } } @@ -214,7 +214,7 @@ func (t *Tracer) StartSpan( shadowTr := t.getShadowTracer() if len(opts) == 0 && !t.useNetTrace() && shadowTr == nil && !t.forceRealSpans { - return &t.noopSpan + return t.noopSpan } var sso opentracing.StartSpanOptions @@ -265,7 +265,7 @@ func (t *Tracer) StartSpan( // part of a recording or snowball trace, avoid overhead and return a noop // span. if !recordable && recordingType == NoRecording && shadowTr == nil && !t.useNetTrace() && !t.forceRealSpans { - return &t.noopSpan + return t.noopSpan } s := &span{ @@ -362,7 +362,7 @@ func (t *Tracer) StartRootSpan( ) opentracing.Span { // In the usual case, we return noopSpan. if !t.AlwaysTrace() && recordable == NonRecordableSpan { - return &t.noopSpan + return t.noopSpan } s := &span{ @@ -424,7 +424,7 @@ func StartChildSpan( tr := parentSpan.Tracer().(*Tracer) // If tracing is disabled, avoid overhead and return a noop span. if IsBlackHoleSpan(parentSpan) { - return &tr.noopSpan + return tr.noopSpan } pSpan := parentSpan.(*span) @@ -549,18 +549,20 @@ func (fn textMapReaderFn) ForeachKey(handler func(key, val string) error) error return fn(handler) } +var noopSpanContext = &spanContext{} + // Extract is part of the opentracing.Tracer interface. // It always returns a valid context, even in error cases (this is assumed by the // grpc-opentracing interceptor). func (t *Tracer) Extract(format interface{}, carrier interface{}) (opentracing.SpanContext, error) { // We only support the HTTPHeaders/TextMap format. if format != opentracing.HTTPHeaders && format != opentracing.TextMap { - return noopSpanContext{}, opentracing.ErrUnsupportedFormat + return noopSpanContext, opentracing.ErrUnsupportedFormat } mapReader, ok := carrier.(opentracing.TextMapReader) if !ok { - return noopSpanContext{}, opentracing.ErrInvalidCarrier + return noopSpanContext, opentracing.ErrInvalidCarrier } var sc spanContext @@ -600,10 +602,10 @@ func (t *Tracer) Extract(format interface{}, carrier interface{}) (opentracing.S return nil }) if err != nil { - return noopSpanContext{}, err + return noopSpanContext, err } if sc.TraceID == 0 && sc.SpanID == 0 { - return noopSpanContext{}, nil + return noopSpanContext, nil } if sc.Baggage[Snowball] != "" { @@ -619,7 +621,7 @@ func (t *Tracer) Extract(format interface{}, carrier interface{}) (opentracing.S // Extract the shadow context using the un-encapsulated textmap. sc.shadowCtx, err = shadowTr.Extract(format, shadowCarrier) if err != nil { - return noopSpanContext{}, err + return noopSpanContext, err } } } @@ -644,17 +646,17 @@ func FinishSpan(span opentracing.Span) { // // See also ChildSpan() for a "parent-child relationship". func ForkCtxSpan(ctx context.Context, opName string) (context.Context, opentracing.Span) { - if span := opentracing.SpanFromContext(ctx); span != nil { - if _, noop := span.(*noopSpan); noop { + if sp := opentracing.SpanFromContext(ctx); sp != nil { + if sp.(*span).isNoop() { // Optimization: avoid ContextWithSpan call if tracing is disabled. - return ctx, span + return ctx, sp } - tr := span.Tracer() - if IsBlackHoleSpan(span) { - ns := &tr.(*Tracer).noopSpan + tr := sp.Tracer() + if IsBlackHoleSpan(sp) { + ns := tr.(*Tracer).noopSpan return opentracing.ContextWithSpan(ctx, ns), ns } - newSpan := tr.StartSpan(opName, opentracing.FollowsFrom(span.Context()), LogTagsFromCtx(ctx)) + newSpan := tr.StartSpan(opName, opentracing.FollowsFrom(sp.Context()), LogTagsFromCtx(ctx)) return opentracing.ContextWithSpan(ctx, newSpan), newSpan } return ctx, nil @@ -681,20 +683,20 @@ func ChildSpanSeparateRecording( func childSpan( ctx context.Context, opName string, separateRecording bool, ) (context.Context, opentracing.Span) { - span := opentracing.SpanFromContext(ctx) - if span == nil { + sp := opentracing.SpanFromContext(ctx) + if sp == nil { return ctx, nil } - if _, noop := span.(*noopSpan); noop { + if sp.(*span).isNoop() { // Optimization: avoid ContextWithSpan call if tracing is disabled. - return ctx, span + return ctx, sp } - tr := span.Tracer() - if IsBlackHoleSpan(span) { - ns := &tr.(*Tracer).noopSpan + tr := sp.Tracer() + if IsBlackHoleSpan(sp) { + ns := tr.(*Tracer).noopSpan return opentracing.ContextWithSpan(ctx, ns), ns } - newSpan := StartChildSpan(opName, span, logtags.FromContext(ctx), separateRecording) + newSpan := StartChildSpan(opName, sp, logtags.FromContext(ctx), separateRecording) return opentracing.ContextWithSpan(ctx, newSpan), newSpan } diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 7bec3d0283b0..e2bb6a5e82f6 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -89,6 +89,10 @@ func (sc *spanContext) ForeachBaggageItem(handler func(k, v string) bool) { } } +func (sc *spanContext) isNoop() bool { + return sc.spanMeta == spanMeta{} +} + // RecordingType is the type of recording that a span might be performing. type RecordingType int @@ -182,20 +186,14 @@ func (s *span) isBlackHole() bool { return !s.crdbSpan.isRecording() && s.netTr == nil && s.otSpan == (otSpan{}) } -/* -TODO(tbg): replace noopSpan with this concept: func (s *span) isNoop() bool { // Special case: trace id zero implies that everything else, // with the exception of the tracer, is also zero. return s.isBlackHole() && s.crdbSpan.TraceID == 0 } -*/ // IsRecording returns true if the span is recording its events. func IsRecording(s opentracing.Span) bool { - if _, noop := s.(*noopSpan); noop { - return false - } return s.(*span).isRecording() } @@ -238,13 +236,14 @@ func StartRecording(os opentracing.Span, recType RecordingType) { if recType == NoRecording { panic("StartRecording called with NoRecording") } - if _, noop := os.(*noopSpan); noop { + sp := os.(*span) + if sp.isNoop() { panic("StartRecording called on NoopSpan; use the Recordable option for StartSpan") } // If we're already recording (perhaps because the parent was recording when // this span was created), there's nothing to do. - if sp := os.(*span); !sp.isRecording() { + if !sp.isRecording() { sp.enableRecording(nil /* parent */, recType, false /* separateRecording */) } } @@ -292,9 +291,6 @@ type Recording []RecordedSpan // enabled. This can be called while spans that are part of the recording are // still open; it can run concurrently with operations on those spans. func GetRecording(os opentracing.Span) Recording { - if _, noop := os.(*noopSpan); noop { - return nil - } s := os.(*span) if !s.isRecording() { return nil @@ -680,11 +676,6 @@ func ImportRemoteSpans(os opentracing.Span, remoteSpans []RecordedSpan) error { // can use this method to figure out if they can short-circuit some // tracing-related work that would be discarded anyway. func IsBlackHoleSpan(s opentracing.Span) bool { - // There are two types of black holes: instances of noopSpan and, when tracing - // is disabled, real spans that are not recording. - if _, noop := s.(*noopSpan); noop { - return true - } sp := s.(*span) return sp.isBlackHole() } @@ -692,8 +683,8 @@ func IsBlackHoleSpan(s opentracing.Span) bool { // IsNoopContext returns true if the span context is from a "no-op" span. If // this is true, any span derived from this context will be a "black hole span". func IsNoopContext(spanCtx opentracing.SpanContext) bool { - _, noop := spanCtx.(noopSpanContext) - return noop + // TODO(tbg): remove this. + return spanCtx.(*spanContext).isNoop() } // SetSpanStats sets the stats on a span. stats.Stats() will also be added to @@ -965,35 +956,3 @@ func (s *span) addChild(child *span) { s.mu.recording.children = append(s.mu.recording.children, child) s.mu.Unlock() } - -type noopSpanContext struct{} - -var _ opentracing.SpanContext = noopSpanContext{} - -func (n noopSpanContext) ForeachBaggageItem(handler func(k, v string) bool) {} - -type noopSpan struct { - tracer *Tracer -} - -var _ opentracing.Span = &noopSpan{} - -func (n *noopSpan) Context() opentracing.SpanContext { return noopSpanContext{} } -func (n *noopSpan) BaggageItem(key string) string { return "" } -func (n *noopSpan) SetTag(key string, value interface{}) opentracing.Span { return n } -func (n *noopSpan) Finish() {} -func (n *noopSpan) FinishWithOptions(opts opentracing.FinishOptions) {} -func (n *noopSpan) SetOperationName(operationName string) opentracing.Span { return n } -func (n *noopSpan) Tracer() opentracing.Tracer { return n.tracer } -func (n *noopSpan) LogFields(fields ...otlog.Field) {} -func (n *noopSpan) LogKV(keyVals ...interface{}) {} -func (n *noopSpan) LogEvent(event string) {} -func (n *noopSpan) LogEventWithPayload(event string, payload interface{}) {} -func (n *noopSpan) Log(data opentracing.LogData) {} - -func (n *noopSpan) SetBaggageItem(key, val string) opentracing.Span { - if key == Snowball { - panic("attempting to set Snowball on a noop span; use the Recordable option to StartSpan") - } - return n -} diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index c3a4f2bcdb07..8fde946ec28c 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -22,20 +22,20 @@ func TestTracerRecording(t *testing.T) { tr := NewTracer() noop1 := tr.StartSpan("noop") - if _, noop := noop1.(*noopSpan); !noop { + if !noop1.(*span).isNoop() { t.Error("expected noop span") } noop1.LogKV("hello", "void") noop2 := tr.StartSpan("noop2", opentracing.ChildOf(noop1.Context())) - if _, noop := noop2.(*noopSpan); !noop { + if !noop2.(*span).isNoop() { t.Error("expected noop child span") } noop2.Finish() noop1.Finish() s1 := tr.StartSpan("a", Recordable) - if _, noop := s1.(*noopSpan); noop { + if s1.(*span).isNoop() { t.Error("Recordable (but not recording) span should not be noop") } if !IsBlackHoleSpan(s1) { @@ -44,7 +44,7 @@ func TestTracerRecording(t *testing.T) { // Unless recording is actually started, child spans are still noop. noop3 := tr.StartSpan("noop3", opentracing.ChildOf(s1.Context())) - if _, noop := noop3.(*noopSpan); !noop { + if !noop3.(*span).isNoop() { t.Error("expected noop child span") } noop3.Finish() @@ -180,7 +180,7 @@ func TestTracerInjectExtract(t *testing.T) { // Verify that noop spans become noop spans on the remote side. noop1 := tr.StartSpan("noop") - if _, noop := noop1.(*noopSpan); !noop { + if !noop1.(*span).isNoop() { t.Fatalf("expected noop span: %+v", noop1) } carrier := make(opentracing.HTTPHeadersCarrier) @@ -195,11 +195,11 @@ func TestTracerInjectExtract(t *testing.T) { if err != nil { t.Fatal(err) } - if _, noopCtx := wireContext.(noopSpanContext); !noopCtx { + if !wireContext.(*spanContext).isNoop() { t.Errorf("expected noop context: %v", wireContext) } noop2 := tr2.StartSpan("remote op", opentracing.FollowsFrom(wireContext)) - if _, noop := noop2.(*noopSpan); !noop { + if !noop2.(*span).isNoop() { t.Fatalf("expected noop span: %+v", noop2) } noop1.Finish() From 75dc94ac60aa163768cbbd44634a5bed9c1df588 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 8 Oct 2020 14:05:10 +0200 Subject: [PATCH 03/20] tracing: move tracer from crdbSpan to span That's where it belongs. Release note: None --- pkg/util/tracing/tracer.go | 8 ++++---- pkg/util/tracing/tracer_span.go | 35 ++++++++++++++++++++++++++++++--- 2 files changed, 36 insertions(+), 7 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 70130ee63104..f0f9d4572681 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -113,7 +113,7 @@ var _ opentracing.Tracer = &Tracer{} // backends. func NewTracer() *Tracer { t := &Tracer{} - t.noopSpan = &span{crdbSpan: crdbSpan{tracer: t}} + t.noopSpan = &span{tracer: t} return t } @@ -269,8 +269,8 @@ func (t *Tracer) StartSpan( } s := &span{ + tracer: t, crdbSpan: crdbSpan{ - tracer: t, operation: operationName, startTime: sso.StartTime, logTags: logTags, @@ -366,12 +366,12 @@ func (t *Tracer) StartRootSpan( } s := &span{ + tracer: t, crdbSpan: crdbSpan{ spanMeta: spanMeta{ TraceID: uint64(rand.Int63()), SpanID: uint64(rand.Int63()), }, - tracer: t, operation: opName, startTime: time.Now(), logTags: logTags, @@ -430,8 +430,8 @@ func StartChildSpan( pSpan := parentSpan.(*span) s := &span{ + tracer: tr, crdbSpan: crdbSpan{ - tracer: tr, operation: opName, startTime: time.Now(), parentSpanID: pSpan.SpanID, diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index e2bb6a5e82f6..f396d2c5b3c6 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -111,8 +111,6 @@ type crdbSpan struct { parentSpanID uint64 - tracer *Tracer - operation string startTime time.Time @@ -171,8 +169,39 @@ type otSpan struct { shadowSpan opentracing.Span } +// span is the tracing span that we use in CockroachDB. Depending on the tracing configuration, +// it can hold anywhere between zero and three destinations for trace information. +// +// The net/trace and opentracing spans are straightforward. If they are +// set, we forward information to them; and depending on whether they are +// set, spans descending from a parent will have these created as well. +// +// The CockroachDB-internal span (crdbSpan) is more complex as it has multiple features: +// +// 1. recording: crdbSpan supports "recordings", meaning that it provides a way to extract +// the data logged into a trace span. +// 2. optimizations for the non-tracing case. If tracing is off and the span is not required +// to support recording (NoRecording), we still want to be able to have a cheap span +// to give to the caller. This is a) because it frees the caller from +// distinguishing the tracing and non-tracing cases, and b) because the span +// has the dual purpose of propagating the *Tracer around, which is needed +// in case at some point down the line there is a need to create an actual +// span (for example, because a "recordable" child span is requested). +// +// In these cases, we return a singleton span that is empty save for the tracer. +// 3. snowball recording. As a special case of 1), we support a recording mode +// (SnowballRecording) which propagates to child spans across RPC boundaries. +// 4. parent span recording. To make matters even more complex, there is a single-node +// recording option (SingleNodeRecording) in which the parent span keeps track of +// its local children and returns their recording in its own. +// +// TODO(tbg): investigate whether the tracer in 2) is really needed. +// TODO(tbg): simplify the functionality of crdbSpan, which seems overly complex. type span struct { - crdbSpan // can be zero + tracer *Tracer // never nil + + // Internal trace span. Can be zero. + crdbSpan // x/net/trace.Trace instance; nil if not tracing to x/net/trace. netTr trace.Trace // Shadow tracer and span; zero if not using a shadow tracer. From b9ae26d8a24770ab38752177db64696a4c414db3 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 8 Oct 2020 14:09:37 +0200 Subject: [PATCH 04/20] tracing: improve comment on (*span).isNoop() Release note: None --- pkg/util/tracing/tracer_span.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index f396d2c5b3c6..791280c5516e 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -216,8 +216,9 @@ func (s *span) isBlackHole() bool { } func (s *span) isNoop() bool { - // Special case: trace id zero implies that everything else, - // with the exception of the tracer, is also zero. + // NB: this is the same as `s` being zero with the exception + // of the `tracer` field. However, `span` is not comparable, + // so this can't be expressed easily. return s.isBlackHole() && s.crdbSpan.TraceID == 0 } From e676d034f90d004cffc1635428ad32090aaf0110 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 10:33:56 +0200 Subject: [PATCH 05/20] tracing: harmonize subspan field names Release note: None --- pkg/util/tracing/shadow.go | 4 ++-- pkg/util/tracing/tracer.go | 6 +++--- pkg/util/tracing/tracer_span.go | 30 +++++++++++++++--------------- pkg/util/tracing/tracer_test.go | 2 +- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/pkg/util/tracing/shadow.go b/pkg/util/tracing/shadow.go index f05bb175cfd5..d9471daa0e7c 100644 --- a/pkg/util/tracing/shadow.go +++ b/pkg/util/tracing/shadow.go @@ -99,8 +99,8 @@ func linkShadowSpan( ReferencedContext: parentShadowCtx, }) } - s.otSpan.shadowTr = shadowTr - s.otSpan.shadowSpan = shadowTr.StartSpan(s.operation, opts...) + s.ot.shadowTr = shadowTr + s.ot.shadowSpan = shadowTr.StartSpan(s.operation, opts...) } func createLightStepTracer(token string) (shadowTracerManager, opentracing.Tracer) { diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index f0f9d4572681..2861945af6e6 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -451,8 +451,8 @@ func StartChildSpan( s.TraceID = pSpan.TraceID s.SpanID = uint64(rand.Int63()) - if pSpan.otSpan.shadowTr != nil { - linkShadowSpan(s, pSpan.otSpan.shadowTr, pSpan.otSpan.shadowSpan.Context(), opentracing.ChildOfRef) + if pSpan.ot.shadowTr != nil { + linkShadowSpan(s, pSpan.ot.shadowTr, pSpan.ot.shadowSpan.Context(), opentracing.ChildOfRef) } recordingType := pSpan.mu.recording.recordingType @@ -469,7 +469,7 @@ func StartChildSpan( } } - if pSpan.netTr != nil || pSpan.otSpan.shadowTr != nil { + if pSpan.netTr != nil || pSpan.ot.shadowTr != nil { // Copy baggage items to tags so they show up in the shadow tracer UI or x/net/trace. for k, v := range s.mu.Baggage { s.SetTag(k, v) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 791280c5516e..4e2beb03eac8 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -205,14 +205,14 @@ type span struct { // x/net/trace.Trace instance; nil if not tracing to x/net/trace. netTr trace.Trace // Shadow tracer and span; zero if not using a shadow tracer. - otSpan otSpan + ot otSpan } // TODO(tbg): remove this. We don't need *span to be an opentracing.Span. var _ opentracing.Span = &span{} func (s *span) isBlackHole() bool { - return !s.crdbSpan.isRecording() && s.netTr == nil && s.otSpan == (otSpan{}) + return !s.crdbSpan.isRecording() && s.netTr == nil && s.ot == (otSpan{}) } func (s *span) isNoop() bool { @@ -743,8 +743,8 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { s.mu.Lock() s.mu.duration = finishTime.Sub(s.startTime) s.mu.Unlock() - if s.otSpan.shadowTr != nil { - s.otSpan.shadowSpan.Finish() + if s.ot.shadowTr != nil { + s.ot.shadowSpan.Finish() } if s.netTr != nil { s.netTr.Finish() @@ -768,9 +768,9 @@ func (s *span) Context() opentracing.SpanContext { span: s, Baggage: baggageCopy, } - if s.otSpan.shadowTr != nil { - sc.shadowTr = s.otSpan.shadowTr - sc.shadowCtx = s.otSpan.shadowSpan.Context() + if s.ot.shadowTr != nil { + sc.shadowTr = s.ot.shadowTr + sc.shadowCtx = s.ot.shadowSpan.Context() } if s.isRecording() { @@ -781,8 +781,8 @@ func (s *span) Context() opentracing.SpanContext { // SetOperationName is part of the opentracing.Span interface. func (s *span) SetOperationName(operationName string) opentracing.Span { - if s.otSpan.shadowTr != nil { - s.otSpan.shadowSpan.SetOperationName(operationName) + if s.ot.shadowTr != nil { + s.ot.shadowSpan.SetOperationName(operationName) } s.operation = operationName return s @@ -794,8 +794,8 @@ func (s *span) SetTag(key string, value interface{}) opentracing.Span { } func (s *span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { - if s.otSpan.shadowTr != nil { - s.otSpan.shadowSpan.SetTag(key, value) + if s.ot.shadowTr != nil { + s.ot.shadowSpan.SetTag(key, value) } if s.netTr != nil { s.netTr.LazyPrintf("%s:%v", key, value) @@ -816,8 +816,8 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci // LogFields is part of the opentracing.Span interface. func (s *span) LogFields(fields ...otlog.Field) { - if s.otSpan.shadowTr != nil { - s.otSpan.shadowSpan.LogFields(fields...) + if s.ot.shadowTr != nil { + s.ot.shadowSpan.LogFields(fields...) } if s.netTr != nil { // TODO(radu): when LightStep supports arbitrary fields, we should make @@ -876,8 +876,8 @@ func (s *span) setBaggageItemLocked(restrictedKey, value string) opentracing.Spa } s.mu.Baggage[restrictedKey] = value - if s.otSpan.shadowTr != nil { - s.otSpan.shadowSpan.SetBaggageItem(restrictedKey, value) + if s.ot.shadowTr != nil { + s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) } // Also set a tag so it shows up in the Lightstep UI or x/net/trace. s.setTagInner(restrictedKey, value, true /* locked */) diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 8fde946ec28c..3119a5141bfd 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -295,7 +295,7 @@ func TestLightstepContext(t *testing.T) { } s2 := tr.StartSpan("child", opentracing.FollowsFrom(wireContext)) - s2Ctx := s2.(*span).otSpan.shadowSpan.Context() + s2Ctx := s2.(*span).ot.shadowSpan.Context() // Verify that the baggage is correct in both the tracer context and in the // lightstep context. From 9c02d1215f685e333b8d9d13ef984730fdbf72a5 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 10:39:27 +0200 Subject: [PATCH 06/20] tracing: un-embed crdbSpan Lots of top-level logic reaches into the `crdbSpan`, which is not ideal. We want top-level tracing methods to delegate to a method on each of the subspans of our main `span` type. This commit doesn't do anything about that, but it gets us one step closer by mechanically unembedding the crdbSpan field. The diff highlights a lot of code that needs to be moved into methods that sit on `*crdbSpan`. Release note: None --- pkg/util/tracing/shadow.go | 12 +- pkg/util/tracing/tracer.go | 52 ++++----- pkg/util/tracing/tracer_span.go | 158 +++++++++++++-------------- pkg/util/tracing/tracer_span_test.go | 2 +- 4 files changed, 112 insertions(+), 112 deletions(-) diff --git a/pkg/util/tracing/shadow.go b/pkg/util/tracing/shadow.go index d9471daa0e7c..6fad173d867e 100644 --- a/pkg/util/tracing/shadow.go +++ b/pkg/util/tracing/shadow.go @@ -86,12 +86,12 @@ func linkShadowSpan( // Create the shadow lightstep span. var opts []opentracing.StartSpanOption // Replicate the options, using the lightstep context in the reference. - opts = append(opts, opentracing.StartTime(s.startTime)) - if s.logTags != nil { - opts = append(opts, LogTags(s.logTags)) + opts = append(opts, opentracing.StartTime(s.crdb.startTime)) + if s.crdb.logTags != nil { + opts = append(opts, LogTags(s.crdb.logTags)) } - if s.mu.tags != nil { - opts = append(opts, s.mu.tags) + if s.crdb.mu.tags != nil { + opts = append(opts, s.crdb.mu.tags) } if parentShadowCtx != nil { opts = append(opts, opentracing.SpanReference{ @@ -100,7 +100,7 @@ func linkShadowSpan( }) } s.ot.shadowTr = shadowTr - s.ot.shadowSpan = shadowTr.StartSpan(s.operation, opts...) + s.ot.shadowSpan = shadowTr.StartSpan(s.crdb.operation, opts...) } func createLightStepTracer(token string) (shadowTracerManager, opentracing.Tracer) { diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 2861945af6e6..6f44fbe034cb 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -270,24 +270,24 @@ func (t *Tracer) StartSpan( s := &span{ tracer: t, - crdbSpan: crdbSpan{ + crdb: crdbSpan{ operation: operationName, startTime: sso.StartTime, logTags: logTags, }, } - if s.startTime.IsZero() { - s.startTime = time.Now() + if s.crdb.startTime.IsZero() { + s.crdb.startTime = time.Now() } - s.mu.duration = -1 + s.crdb.mu.duration = -1 if !hasParent { // No parent Span; allocate new trace id. - s.TraceID = uint64(rand.Int63()) + s.crdb.TraceID = uint64(rand.Int63()) } else { - s.TraceID = parentCtx.TraceID + s.crdb.TraceID = parentCtx.TraceID } - s.SpanID = uint64(rand.Int63()) + s.crdb.SpanID = uint64(rand.Int63()) if t.useNetTrace() { s.netTr = trace.New("tracing", operationName) @@ -295,12 +295,12 @@ func (t *Tracer) StartSpan( } if hasParent { - s.parentSpanID = parentCtx.SpanID + s.crdb.parentSpanID = parentCtx.SpanID // Copy baggage from parent. if l := len(parentCtx.Baggage); l > 0 { - s.mu.Baggage = make(map[string]string, l) + s.crdb.mu.Baggage = make(map[string]string, l) for k, v := range parentCtx.Baggage { - s.mu.Baggage[k] = v + s.crdb.mu.Baggage[k] = v } } } @@ -311,7 +311,7 @@ func (t *Tracer) StartSpan( // Copy baggage items to tags so they show up in the shadow tracer UI, // x/net/trace, or recordings. - for k, v := range s.mu.Baggage { + for k, v := range s.crdb.mu.Baggage { s.SetTag(k, v) } @@ -367,7 +367,7 @@ func (t *Tracer) StartRootSpan( s := &span{ tracer: t, - crdbSpan: crdbSpan{ + crdb: crdbSpan{ spanMeta: spanMeta{ TraceID: uint64(rand.Int63()), SpanID: uint64(rand.Int63()), @@ -377,7 +377,7 @@ func (t *Tracer) StartRootSpan( logTags: logTags, }, } - s.mu.duration = -1 + s.crdb.mu.duration = -1 shadowTracer := t.getShadowTracer() if shadowTracer != nil { @@ -431,36 +431,36 @@ func StartChildSpan( s := &span{ tracer: tr, - crdbSpan: crdbSpan{ + crdb: crdbSpan{ operation: opName, startTime: time.Now(), - parentSpanID: pSpan.SpanID, + parentSpanID: pSpan.crdb.SpanID, logTags: logTags, }, } // Copy baggage from parent. - pSpan.mu.Lock() - if l := len(pSpan.mu.Baggage); l > 0 { - s.mu.Baggage = make(map[string]string, l) - for k, v := range pSpan.mu.Baggage { - s.mu.Baggage[k] = v + pSpan.crdb.mu.Lock() + if l := len(pSpan.crdb.mu.Baggage); l > 0 { + s.crdb.mu.Baggage = make(map[string]string, l) + for k, v := range pSpan.crdb.mu.Baggage { + s.crdb.mu.Baggage[k] = v } } - s.TraceID = pSpan.TraceID - s.SpanID = uint64(rand.Int63()) + s.crdb.TraceID = pSpan.crdb.TraceID + s.crdb.SpanID = uint64(rand.Int63()) if pSpan.ot.shadowTr != nil { linkShadowSpan(s, pSpan.ot.shadowTr, pSpan.ot.shadowSpan.Context(), opentracing.ChildOfRef) } - recordingType := pSpan.mu.recording.recordingType + recordingType := pSpan.crdb.mu.recording.recordingType if pSpan.netTr != nil { s.netTr = trace.New("tracing", opName) s.netTr.SetMaxEvents(maxLogsPerSpan) - if startTags := s.logTags; startTags != nil { + if startTags := s.crdb.logTags; startTags != nil { tags := startTags.Get() for i := range tags { tag := &tags[i] @@ -471,12 +471,12 @@ func StartChildSpan( if pSpan.netTr != nil || pSpan.ot.shadowTr != nil { // Copy baggage items to tags so they show up in the shadow tracer UI or x/net/trace. - for k, v := range s.mu.Baggage { + for k, v := range s.crdb.mu.Baggage { s.SetTag(k, v) } } - pSpan.mu.Unlock() + pSpan.crdb.mu.Unlock() // Start recording if necessary. if recordingType != NoRecording { diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 4e2beb03eac8..e5128891926a 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -201,7 +201,7 @@ type span struct { tracer *Tracer // never nil // Internal trace span. Can be zero. - crdbSpan + crdb crdbSpan // x/net/trace.Trace instance; nil if not tracing to x/net/trace. netTr trace.Trace // Shadow tracer and span; zero if not using a shadow tracer. @@ -212,19 +212,19 @@ type span struct { var _ opentracing.Span = &span{} func (s *span) isBlackHole() bool { - return !s.crdbSpan.isRecording() && s.netTr == nil && s.ot == (otSpan{}) + return !s.crdb.isRecording() && s.netTr == nil && s.ot == (otSpan{}) } func (s *span) isNoop() bool { // NB: this is the same as `s` being zero with the exception // of the `tracer` field. However, `span` is not comparable, // so this can't be expressed easily. - return s.isBlackHole() && s.crdbSpan.TraceID == 0 + return s.isBlackHole() && s.crdb.TraceID == 0 } // IsRecording returns true if the span is recording its events. func IsRecording(s opentracing.Span) bool { - return s.(*span).isRecording() + return s.(*span).crdb.isRecording() } // enableRecording start recording on the span. From now on, log events and child spans @@ -235,10 +235,10 @@ func IsRecording(s opentracing.Span) bool { // If separate recording is specified, the child is not registered with the // parent. Thus, the parent's recording will not include this child. func (s *span) enableRecording(parent *span, recType RecordingType, separateRecording bool) { - s.mu.Lock() - defer s.mu.Unlock() - atomic.StoreInt32(&s.recording, 1) - s.mu.recording.recordingType = recType + s.crdb.mu.Lock() + defer s.crdb.mu.Unlock() + atomic.StoreInt32(&s.crdb.recording, 1) + s.crdb.mu.recording.recordingType = recType if parent != nil && !separateRecording { parent.addChild(s) } @@ -248,9 +248,9 @@ func (s *span) enableRecording(parent *span, recType RecordingType, separateReco // Clear any previously recorded info. This is needed by SQL SessionTracing, // who likes to start and stop recording repeatedly on the same span, and // collect the (separate) recordings every time. - s.mu.recording.recordedLogs = nil - s.mu.recording.children = nil - s.mu.recording.remoteSpans = nil + s.crdb.mu.recording.recordedLogs = nil + s.crdb.mu.recording.children = nil + s.crdb.mu.recording.remoteSpans = nil } // StartRecording enables recording on the span. Events from this point forward @@ -273,7 +273,7 @@ func StartRecording(os opentracing.Span, recType RecordingType) { // If we're already recording (perhaps because the parent was recording when // this span was created), there's nothing to do. - if !sp.isRecording() { + if !sp.crdb.isRecording() { sp.enableRecording(nil /* parent */, recType, false /* separateRecording */) } } @@ -290,17 +290,17 @@ func StopRecording(os opentracing.Span) { } func (s *span) disableRecording() { - s.mu.Lock() - atomic.StoreInt32(&s.recording, 0) + s.crdb.mu.Lock() + atomic.StoreInt32(&s.crdb.recording, 0) // We test the duration as a way to check if the span has been finished. If it // has, we don't want to do the call below as it might crash (at least if // there's a netTr). - if (s.mu.duration == -1) && (s.mu.recording.recordingType == SnowballRecording) { + if (s.crdb.mu.duration == -1) && (s.crdb.mu.recording.recordingType == SnowballRecording) { // Clear the Snowball baggage item, assuming that it was set by // enableRecording(). s.setBaggageItemLocked(Snowball, "") } - s.mu.Unlock() + s.crdb.mu.Unlock() } // IsRecordable returns true if {Start,Stop}Recording() can be called on this @@ -322,18 +322,18 @@ type Recording []RecordedSpan // still open; it can run concurrently with operations on those spans. func GetRecording(os opentracing.Span) Recording { s := os.(*span) - if !s.isRecording() { + if !s.crdb.isRecording() { return nil } - s.mu.Lock() + s.crdb.mu.Lock() // The capacity here is approximate since we don't know how many grandchildren // there are. - result := make(Recording, 0, 1+len(s.mu.recording.children)+len(s.mu.recording.remoteSpans)) + result := make(Recording, 0, 1+len(s.crdb.mu.recording.children)+len(s.crdb.mu.recording.remoteSpans)) // Shallow-copy the children so we can process them without the lock. - children := s.mu.recording.children + children := s.crdb.mu.recording.children result = append(result, s.getRecordingLocked()) - result = append(result, s.mu.recording.remoteSpans...) - s.mu.Unlock() + result = append(result, s.crdb.mu.recording.remoteSpans...) + s.crdb.mu.Unlock() for _, child := range children { result = append(result, GetRecording(child)...) @@ -686,18 +686,18 @@ type TraceCollection struct { // recorded traces from other nodes. func ImportRemoteSpans(os opentracing.Span, remoteSpans []RecordedSpan) error { s := os.(*span) - if !s.isRecording() { + if !s.crdb.isRecording() { return errors.New("adding Raw Spans to a span that isn't recording") } // Change the root of the remote recording to be a child of this span. This is // usually already the case, except with DistSQL traces where remote // processors run in spans that FollowFrom an RPC span that we don't collect. - remoteSpans[0].ParentSpanID = s.SpanID + remoteSpans[0].ParentSpanID = s.crdb.SpanID - s.mu.Lock() - s.mu.recording.remoteSpans = append(s.mu.recording.remoteSpans, remoteSpans...) - s.mu.Unlock() + s.crdb.mu.Lock() + s.crdb.mu.recording.remoteSpans = append(s.crdb.mu.recording.remoteSpans, remoteSpans...) + s.crdb.mu.Unlock() return nil } @@ -721,12 +721,12 @@ func IsNoopContext(spanCtx opentracing.SpanContext) bool { // the span tags. func SetSpanStats(os opentracing.Span, stats SpanStats) { s := os.(*span) - s.mu.Lock() - s.mu.stats = stats + s.crdb.mu.Lock() + s.crdb.mu.stats = stats for name, value := range stats.Stats() { s.setTagInner(StatTagPrefix+name, value, true /* locked */) } - s.mu.Unlock() + s.crdb.mu.Unlock() } // Finish is part of the opentracing.Span interface. @@ -740,9 +740,9 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { if finishTime.IsZero() { finishTime = time.Now() } - s.mu.Lock() - s.mu.duration = finishTime.Sub(s.startTime) - s.mu.Unlock() + s.crdb.mu.Lock() + s.crdb.mu.duration = finishTime.Sub(s.crdb.startTime) + s.crdb.mu.Unlock() if s.ot.shadowTr != nil { s.ot.shadowSpan.Finish() } @@ -757,14 +757,14 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { // that's not currently recording? That might save work and allocations when // creating child spans. func (s *span) Context() opentracing.SpanContext { - s.mu.Lock() - defer s.mu.Unlock() - baggageCopy := make(map[string]string, len(s.mu.Baggage)) - for k, v := range s.mu.Baggage { + s.crdb.mu.Lock() + defer s.crdb.mu.Unlock() + baggageCopy := make(map[string]string, len(s.crdb.mu.Baggage)) + for k, v := range s.crdb.mu.Baggage { baggageCopy[k] = v } sc := &spanContext{ - spanMeta: s.spanMeta, + spanMeta: s.crdb.spanMeta, span: s, Baggage: baggageCopy, } @@ -773,8 +773,8 @@ func (s *span) Context() opentracing.SpanContext { sc.shadowCtx = s.ot.shadowSpan.Context() } - if s.isRecording() { - sc.recordingType = s.mu.recording.recordingType + if s.crdb.isRecording() { + sc.recordingType = s.crdb.mu.recording.recordingType } return sc } @@ -784,7 +784,7 @@ func (s *span) SetOperationName(operationName string) opentracing.Span { if s.ot.shadowTr != nil { s.ot.shadowSpan.SetOperationName(operationName) } - s.operation = operationName + s.crdb.operation = operationName return s } @@ -802,14 +802,14 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci } // The internal tags will be used if we start a recording on this span. if !locked { - s.mu.Lock() + s.crdb.mu.Lock() } - if s.mu.tags == nil { - s.mu.tags = make(opentracing.Tags) + if s.crdb.mu.tags == nil { + s.crdb.mu.tags = make(opentracing.Tags) } - s.mu.tags[key] = value + s.crdb.mu.tags[key] = value if !locked { - s.mu.Unlock() + s.crdb.mu.Unlock() } return s } @@ -837,15 +837,15 @@ func (s *span) LogFields(fields ...otlog.Field) { s.netTr.LazyPrintf("%s", buf.String()) } } - if s.isRecording() { - s.mu.Lock() - if len(s.mu.recording.recordedLogs) < maxLogsPerSpan { - s.mu.recording.recordedLogs = append(s.mu.recording.recordedLogs, opentracing.LogRecord{ + if s.crdb.isRecording() { + s.crdb.mu.Lock() + if len(s.crdb.mu.recording.recordedLogs) < maxLogsPerSpan { + s.crdb.mu.recording.recordedLogs = append(s.crdb.mu.recording.recordedLogs, opentracing.LogRecord{ Timestamp: time.Now(), Fields: fields, }) } - s.mu.Unlock() + s.crdb.mu.Unlock() } } @@ -861,20 +861,20 @@ func (s *span) LogKV(alternatingKeyValues ...interface{}) { // SetBaggageItem is part of the opentracing.Span interface. func (s *span) SetBaggageItem(restrictedKey, value string) opentracing.Span { - s.mu.Lock() - defer s.mu.Unlock() + s.crdb.mu.Lock() + defer s.crdb.mu.Unlock() return s.setBaggageItemLocked(restrictedKey, value) } func (s *span) setBaggageItemLocked(restrictedKey, value string) opentracing.Span { - if oldVal, ok := s.mu.Baggage[restrictedKey]; ok && oldVal == value { + if oldVal, ok := s.crdb.mu.Baggage[restrictedKey]; ok && oldVal == value { // No-op. return s } - if s.mu.Baggage == nil { - s.mu.Baggage = make(map[string]string) + if s.crdb.mu.Baggage == nil { + s.crdb.mu.Baggage = make(map[string]string) } - s.mu.Baggage[restrictedKey] = value + s.crdb.mu.Baggage[restrictedKey] = value if s.ot.shadowTr != nil { s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) @@ -886,9 +886,9 @@ func (s *span) setBaggageItemLocked(restrictedKey, value string) opentracing.Spa // BaggageItem is part of the opentracing.Span interface. func (s *span) BaggageItem(restrictedKey string) string { - s.mu.Lock() - defer s.mu.Unlock() - return s.mu.Baggage[restrictedKey] + s.crdb.mu.Lock() + defer s.crdb.mu.Unlock() + return s.crdb.mu.Baggage[restrictedKey] } // Tracer is part of the opentracing.Span interface. @@ -915,12 +915,12 @@ func (s *span) Log(data opentracing.LogData) { // children. func (s *span) getRecordingLocked() RecordedSpan { rs := RecordedSpan{ - TraceID: s.TraceID, - SpanID: s.SpanID, - ParentSpanID: s.parentSpanID, - Operation: s.operation, - StartTime: s.startTime, - Duration: s.mu.duration, + TraceID: s.crdb.TraceID, + SpanID: s.crdb.SpanID, + ParentSpanID: s.crdb.parentSpanID, + Operation: s.crdb.operation, + StartTime: s.crdb.startTime, + Duration: s.crdb.mu.duration, } addTag := func(k, v string) { @@ -939,35 +939,35 @@ func (s *span) getRecordingLocked() RecordedSpan { addTag("unfinished", "") } - if s.mu.stats != nil { - stats, err := types.MarshalAny(s.mu.stats) + if s.crdb.mu.stats != nil { + stats, err := types.MarshalAny(s.crdb.mu.stats) if err != nil { panic(err) } rs.Stats = stats } - if len(s.mu.Baggage) > 0 { + if len(s.crdb.mu.Baggage) > 0 { rs.Baggage = make(map[string]string) - for k, v := range s.mu.Baggage { + for k, v := range s.crdb.mu.Baggage { rs.Baggage[k] = v } } - if s.logTags != nil { - tags := s.logTags.Get() + if s.crdb.logTags != nil { + tags := s.crdb.logTags.Get() for i := range tags { tag := &tags[i] addTag(tagName(tag.Key()), tag.ValueStr()) } } - if len(s.mu.tags) > 0 { - for k, v := range s.mu.tags { + if len(s.crdb.mu.tags) > 0 { + for k, v := range s.crdb.mu.tags { // We encode the tag values as strings. addTag(k, fmt.Sprint(v)) } } - rs.Logs = make([]LogRecord, len(s.mu.recording.recordedLogs)) - for i, r := range s.mu.recording.recordedLogs { + rs.Logs = make([]LogRecord, len(s.crdb.mu.recording.recordedLogs)) + for i, r := range s.crdb.mu.recording.recordedLogs { rs.Logs[i].Time = r.Timestamp rs.Logs[i].Fields = make([]LogRecord_Field, len(r.Fields)) for j, f := range r.Fields { @@ -982,7 +982,7 @@ func (s *span) getRecordingLocked() RecordedSpan { } func (s *span) addChild(child *span) { - s.mu.Lock() - s.mu.recording.children = append(s.mu.recording.children, child) - s.mu.Unlock() + s.crdb.mu.Lock() + s.crdb.mu.recording.children = append(s.crdb.mu.recording.children, child) + s.crdb.mu.Unlock() } diff --git a/pkg/util/tracing/tracer_span_test.go b/pkg/util/tracing/tracer_span_test.go index 5c6ea3798f67..4e7b85beb954 100644 --- a/pkg/util/tracing/tracer_span_test.go +++ b/pkg/util/tracing/tracer_span_test.go @@ -31,7 +31,7 @@ func TestRecordingString(t *testing.T) { StartRecording(root, SnowballRecording) root.LogFields(otlog.String(LogMessageField, "root 1")) // Hackily fix the timing on the first log message, so that we can check it later. - rootSp.mu.recording.recordedLogs[0].Timestamp = rootSp.startTime.Add(time.Millisecond) + rootSp.crdb.mu.recording.recordedLogs[0].Timestamp = rootSp.crdb.startTime.Add(time.Millisecond) // Sleep a bit so that everything that comes afterwards has higher timestamps // than the one we just assigned. Otherwise the sorting will be screwed up. time.Sleep(10 * time.Millisecond) From 5f47e41fa2a3adb940ca549028ccf4a2d0c298be Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 11:07:44 +0200 Subject: [PATCH 07/20] tracingpb: establish package Release note: None --- pkg/bench/ddl_analysis/ddl_analysis_bench.go | 3 +- pkg/roachpb/api.pb.go | 1281 +++++++++-------- pkg/roachpb/api.proto | 4 +- pkg/server/status.go | 4 +- pkg/sql/exec_util.go | 15 +- pkg/sql/execinfra/base.go | 3 +- pkg/sql/execinfrapb/data.go | 4 +- pkg/sql/execinfrapb/data.pb.go | 327 ++--- pkg/sql/execinfrapb/data.proto | 4 +- pkg/sql/execinfrapb/flow_diagram.go | 8 +- pkg/sql/explain_bundle.go | 5 +- pkg/sql/protoreflect/utils_test.go | 6 +- pkg/util/log/trace.go | 3 +- pkg/util/tracing/test_utils.go | 8 +- pkg/util/tracing/tracer.go | 3 +- pkg/util/tracing/tracer_span.go | 35 +- pkg/util/tracing/tracer_span_test.go | 15 +- .../tracing/{ => tracingpb}/recorded_span.go | 2 +- .../{ => tracingpb}/recorded_span.pb.go | 111 +- .../{ => tracingpb}/recorded_span.proto | 4 +- 20 files changed, 930 insertions(+), 915 deletions(-) rename pkg/util/tracing/{ => tracingpb}/recorded_span.go (98%) rename pkg/util/tracing/{ => tracingpb}/recorded_span.pb.go (90%) rename pkg/util/tracing/{ => tracingpb}/recorded_span.proto (97%) diff --git a/pkg/bench/ddl_analysis/ddl_analysis_bench.go b/pkg/bench/ddl_analysis/ddl_analysis_bench.go index 9cc5a257d224..924c62ce2398 100644 --- a/pkg/bench/ddl_analysis/ddl_analysis_bench.go +++ b/pkg/bench/ddl_analysis/ddl_analysis_bench.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/opentracing/opentracing-go" ) @@ -113,7 +114,7 @@ func countKvBatchRequestsInRecording(r tracing.Recording) int { return countKvBatchRequestsInSpan(r, root) } -func countKvBatchRequestsInSpan(r tracing.Recording, sp tracing.RecordedSpan) int { +func countKvBatchRequestsInSpan(r tracing.Recording, sp tracingpb.RecordedSpan) int { count := 0 // Count the number of OpTxnCoordSender operations while traversing the // tree of spans. diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 3967a822d0ce..b12b64dbd8a9 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -9,7 +9,7 @@ import math "math" import lock "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" -import tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" +import tracingpb "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{0} + return fileDescriptor_api_81c6bdcd9303d014, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -100,7 +100,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{1} + return fileDescriptor_api_81c6bdcd9303d014, []int{1} } type ChecksumMode int32 @@ -147,7 +147,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{2} + return fileDescriptor_api_81c6bdcd9303d014, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -178,7 +178,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{3} + return fileDescriptor_api_81c6bdcd9303d014, []int{3} } type ExternalStorageProvider int32 @@ -219,7 +219,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{4} + return fileDescriptor_api_81c6bdcd9303d014, []int{4} } type MVCCFilter int32 @@ -242,7 +242,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{5} + return fileDescriptor_api_81c6bdcd9303d014, []int{5} } type ResponseHeader_ResumeReason int32 @@ -268,7 +268,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{1, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -310,7 +310,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{25, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -331,7 +331,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{0} + return fileDescriptor_api_81c6bdcd9303d014, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,7 +402,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{1} + return fileDescriptor_api_81c6bdcd9303d014, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -436,7 +436,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{2} + return fileDescriptor_api_81c6bdcd9303d014, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -479,7 +479,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{3} + return fileDescriptor_api_81c6bdcd9303d014, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +522,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{4} + return fileDescriptor_api_81c6bdcd9303d014, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -556,7 +556,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{5} + return fileDescriptor_api_81c6bdcd9303d014, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -638,7 +638,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{6} + return fileDescriptor_api_81c6bdcd9303d014, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -673,7 +673,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{7} + return fileDescriptor_api_81c6bdcd9303d014, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -719,7 +719,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{8} + return fileDescriptor_api_81c6bdcd9303d014, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -753,7 +753,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{9} + return fileDescriptor_api_81c6bdcd9303d014, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -793,7 +793,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{10} + return fileDescriptor_api_81c6bdcd9303d014, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -830,7 +830,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{11} + return fileDescriptor_api_81c6bdcd9303d014, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -864,7 +864,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{12} + return fileDescriptor_api_81c6bdcd9303d014, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +898,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{13} + return fileDescriptor_api_81c6bdcd9303d014, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -950,7 +950,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{14} + return fileDescriptor_api_81c6bdcd9303d014, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +987,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{15} + return fileDescriptor_api_81c6bdcd9303d014, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1042,7 +1042,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{16} + return fileDescriptor_api_81c6bdcd9303d014, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1076,7 +1076,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{17} + return fileDescriptor_api_81c6bdcd9303d014, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1117,7 +1117,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{18} + return fileDescriptor_api_81c6bdcd9303d014, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1151,7 +1151,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{19} + return fileDescriptor_api_81c6bdcd9303d014, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1202,7 +1202,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{20} + return fileDescriptor_api_81c6bdcd9303d014, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1254,7 +1254,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{21} + return fileDescriptor_api_81c6bdcd9303d014, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1305,7 +1305,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{22} + return fileDescriptor_api_81c6bdcd9303d014, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1357,7 +1357,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{23} + return fileDescriptor_api_81c6bdcd9303d014, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1410,7 +1410,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{24} + return fileDescriptor_api_81c6bdcd9303d014, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1447,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{25} + return fileDescriptor_api_81c6bdcd9303d014, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1491,7 +1491,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{25, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1539,7 +1539,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{26} + return fileDescriptor_api_81c6bdcd9303d014, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1575,7 +1575,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{27} + return fileDescriptor_api_81c6bdcd9303d014, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1686,7 +1686,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{28} + return fileDescriptor_api_81c6bdcd9303d014, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1732,7 +1732,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{29} + return fileDescriptor_api_81c6bdcd9303d014, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1793,7 +1793,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{30} + return fileDescriptor_api_81c6bdcd9303d014, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1828,7 +1828,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{31} + return fileDescriptor_api_81c6bdcd9303d014, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1867,7 +1867,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{32} + return fileDescriptor_api_81c6bdcd9303d014, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1902,7 +1902,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{33} + return fileDescriptor_api_81c6bdcd9303d014, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1945,7 +1945,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{34} + return fileDescriptor_api_81c6bdcd9303d014, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1980,7 +1980,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{35} + return fileDescriptor_api_81c6bdcd9303d014, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2018,7 +2018,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{36} + return fileDescriptor_api_81c6bdcd9303d014, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2051,7 +2051,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{37} + return fileDescriptor_api_81c6bdcd9303d014, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2086,7 +2086,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{38} + return fileDescriptor_api_81c6bdcd9303d014, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2144,7 +2144,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{39} + return fileDescriptor_api_81c6bdcd9303d014, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2179,7 +2179,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{40} + return fileDescriptor_api_81c6bdcd9303d014, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2216,7 +2216,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{41} + return fileDescriptor_api_81c6bdcd9303d014, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2249,7 +2249,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{42} + return fileDescriptor_api_81c6bdcd9303d014, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2288,7 +2288,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{43} + return fileDescriptor_api_81c6bdcd9303d014, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2325,7 +2325,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{44} + return fileDescriptor_api_81c6bdcd9303d014, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2363,7 +2363,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{45} + return fileDescriptor_api_81c6bdcd9303d014, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2397,7 +2397,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{45, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2431,7 +2431,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{46} + return fileDescriptor_api_81c6bdcd9303d014, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2500,7 +2500,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{47} + return fileDescriptor_api_81c6bdcd9303d014, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2543,7 +2543,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{48} + return fileDescriptor_api_81c6bdcd9303d014, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2590,7 +2590,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{49} + return fileDescriptor_api_81c6bdcd9303d014, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2626,7 +2626,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{50} + return fileDescriptor_api_81c6bdcd9303d014, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2670,7 +2670,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{51} + return fileDescriptor_api_81c6bdcd9303d014, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2714,7 +2714,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{52} + return fileDescriptor_api_81c6bdcd9303d014, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2774,7 +2774,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{53} + return fileDescriptor_api_81c6bdcd9303d014, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2810,7 +2810,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{54} + return fileDescriptor_api_81c6bdcd9303d014, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2857,7 +2857,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{55} + return fileDescriptor_api_81c6bdcd9303d014, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2892,7 +2892,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{56} + return fileDescriptor_api_81c6bdcd9303d014, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2943,7 +2943,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{57} + return fileDescriptor_api_81c6bdcd9303d014, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2978,7 +2978,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{58} + return fileDescriptor_api_81c6bdcd9303d014, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3015,7 +3015,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{59} + return fileDescriptor_api_81c6bdcd9303d014, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3049,7 +3049,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{60} + return fileDescriptor_api_81c6bdcd9303d014, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3094,7 +3094,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{61} + return fileDescriptor_api_81c6bdcd9303d014, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3128,7 +3128,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{62} + return fileDescriptor_api_81c6bdcd9303d014, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3172,7 +3172,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{63} + return fileDescriptor_api_81c6bdcd9303d014, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3221,7 +3221,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{64} + return fileDescriptor_api_81c6bdcd9303d014, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3258,7 +3258,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{65} + return fileDescriptor_api_81c6bdcd9303d014, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3295,7 +3295,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{66} + return fileDescriptor_api_81c6bdcd9303d014, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3330,7 +3330,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{67} + return fileDescriptor_api_81c6bdcd9303d014, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3385,7 +3385,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{68} + return fileDescriptor_api_81c6bdcd9303d014, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3422,7 +3422,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{69} + return fileDescriptor_api_81c6bdcd9303d014, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3462,7 +3462,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70} + return fileDescriptor_api_81c6bdcd9303d014, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3496,7 +3496,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3529,7 +3529,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 1} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3569,7 +3569,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 2} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3608,7 +3608,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 3} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3644,7 +3644,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 4} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3683,7 +3683,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 5} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3723,7 +3723,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{70, 6} + return fileDescriptor_api_81c6bdcd9303d014, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3763,7 +3763,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{71} + return fileDescriptor_api_81c6bdcd9303d014, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3797,7 +3797,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{72} + return fileDescriptor_api_81c6bdcd9303d014, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3833,7 +3833,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{73} + return fileDescriptor_api_81c6bdcd9303d014, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3903,7 +3903,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{74} + return fileDescriptor_api_81c6bdcd9303d014, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3955,7 +3955,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{75} + return fileDescriptor_api_81c6bdcd9303d014, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3991,7 +3991,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{76} + return fileDescriptor_api_81c6bdcd9303d014, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4031,7 +4031,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{76, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4082,7 +4082,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{77} + return fileDescriptor_api_81c6bdcd9303d014, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4117,7 +4117,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{77, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4153,7 +4153,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{77, 1} + return fileDescriptor_api_81c6bdcd9303d014, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4188,7 +4188,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{78} + return fileDescriptor_api_81c6bdcd9303d014, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4226,7 +4226,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{79} + return fileDescriptor_api_81c6bdcd9303d014, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4263,7 +4263,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{80} + return fileDescriptor_api_81c6bdcd9303d014, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4296,7 +4296,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{80, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4341,7 +4341,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{81} + return fileDescriptor_api_81c6bdcd9303d014, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4379,7 +4379,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{82} + return fileDescriptor_api_81c6bdcd9303d014, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4432,7 +4432,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{83} + return fileDescriptor_api_81c6bdcd9303d014, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4466,7 +4466,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{84} + return fileDescriptor_api_81c6bdcd9303d014, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4510,7 +4510,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{85} + return fileDescriptor_api_81c6bdcd9303d014, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4544,7 +4544,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{86} + return fileDescriptor_api_81c6bdcd9303d014, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4583,7 +4583,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{87} + return fileDescriptor_api_81c6bdcd9303d014, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4617,7 +4617,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{88} + return fileDescriptor_api_81c6bdcd9303d014, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4666,7 +4666,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{89} + return fileDescriptor_api_81c6bdcd9303d014, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4715,7 +4715,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{90} + return fileDescriptor_api_81c6bdcd9303d014, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4750,7 +4750,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{91} + return fileDescriptor_api_81c6bdcd9303d014, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4792,7 +4792,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{92} + return fileDescriptor_api_81c6bdcd9303d014, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4875,7 +4875,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{93} + return fileDescriptor_api_81c6bdcd9303d014, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6325,7 +6325,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{94} + return fileDescriptor_api_81c6bdcd9303d014, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7860,7 +7860,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{95} + return fileDescriptor_api_81c6bdcd9303d014, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7898,7 +7898,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{96} + return fileDescriptor_api_81c6bdcd9303d014, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7934,7 +7934,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{97} + return fileDescriptor_api_81c6bdcd9303d014, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7971,7 +7971,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{98} + return fileDescriptor_api_81c6bdcd9303d014, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8022,7 +8022,7 @@ type BatchResponse_Header struct { Now hlc.Timestamp `protobuf:"bytes,5,opt,name=now,proto3" json:"now"` // collected_spans stores trace spans recorded during the execution of this // request. - CollectedSpans []tracing.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans,proto3" json:"collected_spans"` + CollectedSpans []tracingpb.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans,proto3" json:"collected_spans"` // Range or list of ranges used to execute the request. The server only // populates this if return_range_info is set on the request, or if the // server detects the client's client_range_info to be stale. @@ -8045,7 +8045,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{98, 0} + return fileDescriptor_api_81c6bdcd9303d014, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8083,7 +8083,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{99} + return fileDescriptor_api_81c6bdcd9303d014, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8121,7 +8121,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{100} + return fileDescriptor_api_81c6bdcd9303d014, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8160,7 +8160,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{101} + return fileDescriptor_api_81c6bdcd9303d014, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8201,7 +8201,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{102} + return fileDescriptor_api_81c6bdcd9303d014, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8242,7 +8242,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{103} + return fileDescriptor_api_81c6bdcd9303d014, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8279,7 +8279,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{104} + return fileDescriptor_api_81c6bdcd9303d014, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8316,7 +8316,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{105} + return fileDescriptor_api_81c6bdcd9303d014, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8357,7 +8357,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{106} + return fileDescriptor_api_81c6bdcd9303d014, []int{106} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8397,7 +8397,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{107} + return fileDescriptor_api_81c6bdcd9303d014, []int{107} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8433,7 +8433,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{108} + return fileDescriptor_api_81c6bdcd9303d014, []int{108} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8472,7 +8472,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_ef1498115920ae53, []int{109} + return fileDescriptor_api_81c6bdcd9303d014, []int{109} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -38241,7 +38241,7 @@ func (m *BatchResponse_Header) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CollectedSpans = append(m.CollectedSpans, tracing.RecordedSpan{}) + m.CollectedSpans = append(m.CollectedSpans, tracingpb.RecordedSpan{}) if err := m.CollectedSpans[len(m.CollectedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39782,509 +39782,510 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_ef1498115920ae53) } - -var fileDescriptor_api_ef1498115920ae53 = []byte{ - // 8008 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6b, 0x6c, 0x23, 0x49, - 0x92, 0x9e, 0x8a, 0xa4, 0x24, 0x32, 0x48, 0x51, 0xa5, 0x94, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, - 0xd9, 0xef, 0xde, 0x19, 0x69, 0xba, 0x7b, 0xc7, 0x33, 0x37, 0x3d, 0x37, 0x7b, 0x12, 0xc5, 0x6e, - 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0x7b, 0x67, 0x6e, 0xd7, 0xb5, 0xa5, 0xaa, 0x14, 0x55, 0x2b, - 0xb2, 0x8a, 0x5d, 0x55, 0xd4, 0xa3, 0x01, 0x03, 0xe7, 0x07, 0x70, 0xc6, 0xd9, 0x58, 0xf8, 0x87, - 0x6d, 0x18, 0x3e, 0x9f, 0x77, 0x80, 0x33, 0x7c, 0x06, 0x0e, 0x6b, 0xf8, 0xaf, 0x8d, 0xf3, 0xe3, - 0xc7, 0x19, 0x58, 0x1f, 0xce, 0xc0, 0xda, 0x80, 0x7d, 0x0b, 0x03, 0x16, 0x7c, 0x5a, 0xc0, 0x30, - 0xfc, 0xc3, 0x80, 0xef, 0x87, 0x0d, 0x0c, 0x60, 0xc3, 0xc8, 0x47, 0xbd, 0xc8, 0xe2, 0x43, 0x9a, - 0x1a, 0xdf, 0x00, 0xf7, 0x87, 0x60, 0x45, 0x66, 0x44, 0x65, 0x46, 0x66, 0x46, 0xc6, 0x97, 0x19, - 0x99, 0x05, 0x33, 0x96, 0xa9, 0xa8, 0xfb, 0xed, 0xdd, 0x65, 0xa5, 0xad, 0x2f, 0xb5, 0x2d, 0xd3, - 0x31, 0xd1, 0x8c, 0x6a, 0xaa, 0x07, 0x94, 0xbc, 0xc4, 0x13, 0xe7, 0xef, 0x1f, 0x1c, 0x2e, 0x1f, - 0x1c, 0xda, 0xd8, 0x3a, 0xc4, 0xd6, 0xb2, 0x6a, 0x1a, 0x6a, 0xc7, 0xb2, 0xb0, 0xa1, 0x9e, 0x2c, - 0x37, 0x4d, 0xf5, 0x80, 0xfe, 0xe8, 0x46, 0x83, 0xb1, 0xcf, 0x23, 0x57, 0xa2, 0xa6, 0x38, 0x0a, - 0xa7, 0xcd, 0xb9, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x65, 0x97, 0xda, 0xc2, 0x8e, 0x12, - 0xc8, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc0, 0xcb, 0xd8, 0x68, 0xe8, 0x06, 0x26, 0x19, 0x0e, - 0x55, 0x95, 0x27, 0xbe, 0x1d, 0x99, 0xf8, 0x88, 0xa7, 0x16, 0x3a, 0x8e, 0xde, 0x5c, 0xde, 0x6f, - 0xaa, 0xcb, 0x8e, 0xde, 0xc2, 0xb6, 0xa3, 0xb4, 0xda, 0x3c, 0x65, 0x91, 0xa6, 0x38, 0x96, 0xa2, - 0xea, 0x46, 0x63, 0xd9, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xb6, 0x62, 0xb8, 0x85, 0x6c, - 0x98, 0x0d, 0x93, 0xfe, 0x5d, 0x26, 0xff, 0x18, 0xb5, 0xf8, 0x7b, 0x02, 0x4c, 0x49, 0xf8, 0x75, - 0x07, 0xdb, 0x4e, 0x05, 0x2b, 0x1a, 0xb6, 0xd0, 0x55, 0x48, 0x1e, 0xe0, 0x93, 0x42, 0x72, 0x51, - 0xb8, 0x9b, 0x5b, 0x9d, 0xfc, 0xf2, 0x74, 0x21, 0xb9, 0x81, 0x4f, 0x24, 0x42, 0x43, 0x8b, 0x30, - 0x89, 0x0d, 0x4d, 0x26, 0xc9, 0xa9, 0x70, 0xf2, 0x04, 0x36, 0xb4, 0x0d, 0x7c, 0x82, 0xbe, 0x07, - 0x69, 0x9b, 0x48, 0x33, 0x54, 0x5c, 0x18, 0x5f, 0x14, 0xee, 0x8e, 0xaf, 0xfe, 0xca, 0x97, 0xa7, - 0x0b, 0x9f, 0x34, 0x74, 0x67, 0xbf, 0xb3, 0xbb, 0xa4, 0x9a, 0xad, 0x65, 0x4f, 0xfb, 0xda, 0xae, - 0xff, 0x7f, 0xb9, 0x7d, 0xd0, 0x58, 0xee, 0xae, 0xf9, 0x52, 0xfd, 0xd8, 0xa8, 0xe1, 0xd7, 0x92, - 0x27, 0xf1, 0xe3, 0xd4, 0x7f, 0xfb, 0x62, 0x41, 0x58, 0x4f, 0xa5, 0x05, 0x31, 0xb1, 0x9e, 0x4a, - 0x27, 0xc4, 0x64, 0xf1, 0xb7, 0x93, 0x90, 0x97, 0xb0, 0xdd, 0x36, 0x0d, 0x1b, 0xf3, 0xf2, 0xbf, - 0x0f, 0x49, 0xe7, 0xd8, 0xa0, 0xe5, 0xcf, 0x3e, 0xbc, 0xb6, 0xd4, 0xd3, 0xda, 0x4b, 0x75, 0x4b, - 0x31, 0x6c, 0x45, 0x75, 0x74, 0xd3, 0x90, 0x48, 0x56, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0x77, 0x5a, - 0x98, 0xaa, 0x8b, 0x56, 0x2d, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xad, 0xad, 0x18, 0x12, 0xb0, 0xbc, - 0xe4, 0x3f, 0xba, 0x0a, 0x69, 0xa3, 0xd3, 0x22, 0x0a, 0xb1, 0x69, 0x75, 0x93, 0xd2, 0xa4, 0xd1, - 0x69, 0x6d, 0xe0, 0x13, 0x1b, 0x7d, 0x17, 0x2e, 0x6b, 0xb8, 0x6d, 0x61, 0x55, 0x71, 0xb0, 0x26, - 0x5b, 0x8a, 0xd1, 0xc0, 0xb2, 0x6e, 0xec, 0x99, 0x76, 0x61, 0x62, 0x31, 0x79, 0x37, 0xfb, 0xf0, - 0xed, 0x08, 0xf9, 0x12, 0xc9, 0x55, 0x35, 0xf6, 0xcc, 0xd5, 0xd4, 0x4f, 0x4f, 0x17, 0xc6, 0xa4, - 0x39, 0x5f, 0x82, 0x97, 0x64, 0xa3, 0x1a, 0x4c, 0xf1, 0xe2, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x30, - 0xb9, 0x28, 0xdc, 0xcd, 0x3f, 0x5c, 0x8a, 0x12, 0x18, 0x52, 0x0d, 0x79, 0xec, 0xb4, 0xb0, 0x44, - 0xb9, 0xa4, 0x9c, 0x15, 0x78, 0x42, 0x6f, 0x41, 0x86, 0xd4, 0x64, 0xf7, 0xc4, 0xc1, 0x76, 0x21, - 0x4d, 0xab, 0x42, 0xaa, 0xb6, 0x4a, 0x9e, 0x8b, 0x9f, 0x42, 0x2e, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, - 0x72, 0x6d, 0x67, 0xb3, 0x2c, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x25, 0x8e, 0xa1, 0x39, - 0x10, 0x39, 0x6d, 0xa3, 0xfc, 0x99, 0xfc, 0xac, 0xba, 0x59, 0xad, 0x8b, 0xc2, 0x7c, 0xea, 0xaf, - 0xfe, 0xf6, 0xb5, 0xb1, 0xe2, 0x4b, 0x80, 0xa7, 0xd8, 0xe1, 0xdd, 0x0c, 0xad, 0xc2, 0xc4, 0x3e, - 0x2d, 0x4f, 0x41, 0xa0, 0x9a, 0x5e, 0x8c, 0x2c, 0x78, 0xa0, 0x4b, 0xae, 0xa6, 0x89, 0x36, 0x7e, - 0x76, 0xba, 0x20, 0x48, 0x9c, 0x93, 0xf5, 0x84, 0xe2, 0xbf, 0x14, 0x20, 0x4b, 0x05, 0xb3, 0x5a, - 0xa2, 0x52, 0x97, 0xe4, 0xeb, 0x43, 0x55, 0xd2, 0x2b, 0x1a, 0x2d, 0xc1, 0xf8, 0xa1, 0xd2, 0xec, - 0xe0, 0x42, 0x82, 0xca, 0x28, 0x44, 0xc8, 0x78, 0x49, 0xd2, 0x25, 0x96, 0x0d, 0x3d, 0x86, 0x9c, - 0x6e, 0x38, 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x1c, 0xc2, 0x96, 0x65, 0xb9, 0xe9, 0x43, 0xf1, 0x9f, - 0x09, 0x00, 0xdb, 0x9d, 0x38, 0x55, 0x83, 0xbe, 0x3d, 0x62, 0xf9, 0x79, 0x1f, 0xe3, 0xb5, 0xb8, - 0x0c, 0x13, 0xba, 0xd1, 0xd4, 0x0d, 0x56, 0xfe, 0xb4, 0xc4, 0x9f, 0xd0, 0x1c, 0x8c, 0xef, 0x36, - 0x75, 0x43, 0xa3, 0xa3, 0x22, 0x2d, 0xb1, 0x07, 0xae, 0x7e, 0x09, 0xb2, 0xb4, 0xec, 0x31, 0x6a, - 0xbf, 0xf8, 0x87, 0x09, 0xb8, 0x54, 0x32, 0x0d, 0x4d, 0x27, 0xc3, 0x53, 0x69, 0x7e, 0x23, 0x74, - 0xb3, 0x0e, 0x81, 0x81, 0x28, 0xe3, 0xe3, 0xf6, 0x88, 0x2d, 0x8d, 0x7c, 0xae, 0xf2, 0x71, 0x9b, - 0xd2, 0xa2, 0xf5, 0x89, 0xbe, 0x0d, 0x57, 0x94, 0x66, 0xd3, 0x3c, 0x92, 0xf5, 0x3d, 0x59, 0x33, - 0xb1, 0x2d, 0x1b, 0xa6, 0x23, 0xe3, 0x63, 0xdd, 0x76, 0xa8, 0x59, 0x49, 0x4b, 0xb3, 0x34, 0xb9, - 0xba, 0xb7, 0x66, 0x62, 0x7b, 0xcb, 0x74, 0xca, 0x24, 0x89, 0x8c, 0x59, 0x52, 0x18, 0x36, 0x66, - 0x27, 0x88, 0x41, 0x96, 0xd2, 0xf8, 0xb8, 0x4d, 0xc7, 0x2c, 0x6f, 0xa2, 0xef, 0xc3, 0xe5, 0x6e, - 0x6d, 0xc6, 0xd9, 0x5a, 0xff, 0x5e, 0x80, 0x7c, 0xd5, 0xd0, 0x9d, 0x6f, 0x44, 0x33, 0x79, 0xaa, - 0x4d, 0x06, 0x55, 0x7b, 0x1f, 0xc4, 0x3d, 0x45, 0x6f, 0x3e, 0x37, 0xea, 0x66, 0x6b, 0xd7, 0x76, - 0x4c, 0x03, 0xdb, 0x5c, 0xf7, 0x3d, 0x74, 0xae, 0xb3, 0x97, 0x30, 0xed, 0xd5, 0x29, 0x4e, 0x65, - 0xbd, 0x01, 0xb1, 0x6a, 0xa8, 0x16, 0x6e, 0x61, 0x23, 0x56, 0x6d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, - 0x4b, 0x35, 0x96, 0x94, 0x7c, 0x02, 0xaf, 0x53, 0x07, 0x66, 0x02, 0xef, 0x8e, 0xd3, 0x5c, 0x92, - 0x89, 0x03, 0x1f, 0xc9, 0x7e, 0x7b, 0x91, 0x89, 0x03, 0x1f, 0x31, 0xf3, 0xf6, 0x19, 0x4c, 0xad, - 0xe1, 0x26, 0x76, 0x70, 0xfc, 0xb6, 0x7f, 0x07, 0xf2, 0xae, 0xe8, 0x38, 0x1b, 0xe9, 0xb7, 0x04, - 0x40, 0x5c, 0x2e, 0x99, 0x71, 0xe3, 0x6c, 0xa7, 0x05, 0xe2, 0x66, 0x38, 0x1d, 0xcb, 0x60, 0xfe, - 0x02, 0xeb, 0xa5, 0xc0, 0x48, 0xd4, 0x65, 0xf0, 0x6d, 0x70, 0x2a, 0x68, 0x83, 0x3d, 0xb7, 0x87, - 0x38, 0x3c, 0x47, 0x30, 0x1b, 0x2a, 0x5e, 0xbc, 0x4d, 0x99, 0xa2, 0x25, 0x4b, 0x2c, 0x26, 0x83, - 0xbe, 0x1d, 0x25, 0x16, 0xff, 0xbe, 0x00, 0x33, 0xa5, 0x26, 0x56, 0xac, 0xd8, 0xf5, 0xf2, 0x1d, - 0x48, 0x6b, 0x58, 0xd1, 0x68, 0xc5, 0xd9, 0x80, 0x7f, 0x27, 0x20, 0x85, 0xf8, 0xb5, 0x4b, 0xfb, - 0x4d, 0x75, 0xa9, 0xee, 0x7a, 0xbc, 0x7c, 0xd4, 0x7b, 0x4c, 0xbc, 0x43, 0x7c, 0x06, 0x28, 0x58, - 0xbe, 0x38, 0x3b, 0xc5, 0x3f, 0x10, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, 0x95, 0x5f, 0x83, - 0xac, 0xa3, 0x58, 0x0d, 0xec, 0xc8, 0xc4, 0xa3, 0x3f, 0x4f, 0xfd, 0x81, 0xf1, 0x11, 0x32, 0xd7, - 0xc0, 0xe7, 0x30, 0x1b, 0x2a, 0x65, 0x9c, 0x2a, 0xf8, 0x5f, 0x02, 0x64, 0x6b, 0xaa, 0x62, 0xc4, - 0x59, 0xf7, 0x4f, 0x21, 0x6b, 0xab, 0x8a, 0x21, 0xef, 0x99, 0x56, 0x4b, 0x71, 0x68, 0xa7, 0xcf, - 0x87, 0xea, 0xee, 0xf9, 0xdd, 0xaa, 0x62, 0x3c, 0xa1, 0x99, 0x24, 0xb0, 0xbd, 0xff, 0xe8, 0x05, - 0x64, 0x0f, 0xf0, 0x89, 0xcc, 0xf1, 0x19, 0x9d, 0x29, 0xf3, 0x0f, 0xdf, 0x0f, 0xf0, 0x1f, 0x1c, - 0x2e, 0xb9, 0xb0, 0x6e, 0x29, 0x00, 0xeb, 0x96, 0x08, 0xc7, 0x52, 0xcd, 0xb1, 0xb0, 0xd1, 0x70, - 0xf6, 0x25, 0x38, 0xc0, 0x27, 0xcf, 0x98, 0x8c, 0xe0, 0x50, 0x5b, 0x4f, 0xa5, 0x93, 0x62, 0xaa, - 0xf8, 0xbf, 0x05, 0xc8, 0xb1, 0x8a, 0xc7, 0x39, 0xd4, 0x3e, 0x80, 0x94, 0x65, 0x1e, 0xb1, 0xa1, - 0x96, 0x7d, 0xf8, 0x56, 0x84, 0x88, 0x0d, 0x7c, 0x12, 0x9c, 0xe3, 0x68, 0x76, 0xb4, 0x0a, 0xdc, - 0x7b, 0x94, 0x29, 0x77, 0x72, 0x54, 0x6e, 0x60, 0x5c, 0x12, 0x91, 0x71, 0x07, 0xa6, 0x77, 0x15, - 0x47, 0xdd, 0x97, 0x2d, 0x5e, 0x48, 0x32, 0x1f, 0x26, 0xef, 0xe6, 0xa4, 0x3c, 0x25, 0xbb, 0x45, - 0xb7, 0x8b, 0xff, 0xc7, 0xed, 0xf5, 0x36, 0xfe, 0x33, 0xd9, 0xf2, 0xff, 0x57, 0xe0, 0xe3, 0xc9, - 0xad, 0xff, 0x9f, 0xb5, 0x0e, 0xf0, 0xe3, 0x04, 0x5c, 0x29, 0xed, 0x63, 0xf5, 0xa0, 0x64, 0x1a, - 0xb6, 0x6e, 0x3b, 0x44, 0x83, 0x71, 0xf6, 0x82, 0xb7, 0x20, 0x73, 0xa4, 0x3b, 0xfb, 0xb2, 0xa6, - 0xef, 0xed, 0x51, 0xcb, 0x97, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, 0xed, 0xa1, 0x47, 0x90, 0x6a, - 0x99, 0x1a, 0x73, 0xb2, 0xf3, 0x0f, 0x17, 0x22, 0xc4, 0xd3, 0xa2, 0xd9, 0x9d, 0xd6, 0xa6, 0xa9, - 0x61, 0x89, 0x66, 0x46, 0xd7, 0x00, 0x54, 0x42, 0x6d, 0x9b, 0xba, 0xe1, 0xf0, 0x59, 0x34, 0x40, - 0x41, 0x15, 0xc8, 0x38, 0xd8, 0x6a, 0xe9, 0x86, 0xe2, 0xe0, 0xc2, 0x38, 0x55, 0xde, 0xcd, 0xc8, - 0x82, 0xb7, 0x9b, 0xba, 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xb6, 0x63, 0x5a, 0x5c, 0x8b, 0x3e, - 0x33, 0xb7, 0xb8, 0x3f, 0x4a, 0x41, 0xa1, 0x57, 0x43, 0x71, 0xf6, 0x93, 0x6d, 0x98, 0x20, 0x38, - 0xbd, 0xe9, 0xf0, 0x9e, 0xf2, 0xb0, 0x9f, 0x22, 0x22, 0x4a, 0x40, 0xf1, 0x7e, 0xd3, 0xe1, 0x85, - 0xe7, 0x72, 0xe6, 0x7f, 0x4f, 0x80, 0x09, 0x96, 0x80, 0x1e, 0x40, 0x9a, 0x2f, 0x4c, 0x68, 0xb4, - 0x8c, 0xc9, 0xd5, 0xcb, 0x67, 0xa7, 0x0b, 0x93, 0x6c, 0xad, 0x61, 0xed, 0x4b, 0xff, 0xaf, 0x34, - 0x49, 0xf3, 0x55, 0x35, 0xd2, 0x66, 0xb6, 0xa3, 0x58, 0x0e, 0x5d, 0x04, 0x4a, 0x30, 0xcc, 0x41, - 0x09, 0x1b, 0xf8, 0x04, 0xad, 0xc3, 0x84, 0xed, 0x28, 0x4e, 0xc7, 0xe6, 0xad, 0x76, 0xae, 0xc2, - 0xd6, 0x28, 0xa7, 0xc4, 0x25, 0x10, 0x67, 0x48, 0xc3, 0x8e, 0xa2, 0x37, 0x69, 0x33, 0x66, 0x24, - 0xfe, 0x54, 0xfc, 0x4d, 0x01, 0x26, 0x58, 0x56, 0x74, 0x05, 0x66, 0xa5, 0x95, 0xad, 0xa7, 0x65, - 0xb9, 0xba, 0xb5, 0x56, 0xae, 0x97, 0xa5, 0xcd, 0xea, 0xd6, 0x4a, 0xbd, 0x2c, 0x8e, 0xa1, 0xcb, - 0x80, 0xdc, 0x84, 0xd2, 0xf3, 0xad, 0x5a, 0xb5, 0x56, 0x2f, 0x6f, 0xd5, 0x45, 0x81, 0xae, 0x51, - 0x50, 0x7a, 0x80, 0x9a, 0x40, 0x37, 0x61, 0xb1, 0x9b, 0x2a, 0xd7, 0xea, 0x2b, 0xf5, 0x9a, 0x5c, - 0xae, 0xd5, 0xab, 0x9b, 0x2b, 0xf5, 0xf2, 0x9a, 0x98, 0x1c, 0x90, 0x8b, 0xbc, 0x44, 0x92, 0xca, - 0xa5, 0xba, 0x98, 0x2a, 0xbe, 0x81, 0x4b, 0x12, 0x56, 0xcd, 0x56, 0xbb, 0xe3, 0x60, 0x52, 0x4a, - 0x3b, 0xce, 0xf1, 0x72, 0x05, 0x26, 0x35, 0xeb, 0x44, 0xb6, 0x3a, 0x06, 0x1f, 0x2d, 0x13, 0x9a, - 0x75, 0x22, 0x75, 0x0c, 0xde, 0x19, 0xff, 0x89, 0x00, 0x97, 0xbb, 0x5f, 0x1e, 0x67, 0x57, 0x7c, - 0x01, 0x59, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xe9, 0x28, 0xdc, 0x55, 0xb9, 0x1f, 0x90, 0xc4, - 0x17, 0xf0, 0x96, 0xbc, 0x05, 0xbc, 0xcd, 0x97, 0xa5, 0x12, 0x2d, 0xc8, 0x1a, 0xe1, 0x70, 0x4d, - 0x11, 0x15, 0x42, 0x29, 0xc5, 0x3f, 0x49, 0xc1, 0x54, 0xd9, 0xd0, 0xea, 0xc7, 0xb1, 0xce, 0x2e, - 0x97, 0x61, 0x42, 0x35, 0x5b, 0x2d, 0xdd, 0x71, 0xd5, 0xc4, 0x9e, 0xd0, 0x2f, 0x05, 0x1c, 0xcd, - 0xe4, 0x08, 0x8e, 0x96, 0xef, 0x62, 0xa2, 0x1f, 0xc0, 0x15, 0x62, 0x41, 0x2d, 0x43, 0x69, 0xca, - 0x4c, 0x9a, 0xec, 0x58, 0x7a, 0xa3, 0x81, 0x2d, 0xbe, 0x5c, 0x78, 0x37, 0xa2, 0x9c, 0x55, 0xce, - 0x51, 0xa2, 0x0c, 0x75, 0x96, 0x5f, 0xba, 0xa4, 0x47, 0x91, 0xd1, 0x27, 0x00, 0x64, 0x72, 0xa2, - 0x4b, 0x90, 0x36, 0xb7, 0x4d, 0xfd, 0xd6, 0x20, 0x5d, 0x73, 0x44, 0x18, 0xc8, 0xb3, 0x8d, 0x96, - 0x09, 0xb6, 0x78, 0xdd, 0xd1, 0x2d, 0x2c, 0x3f, 0x68, 0xab, 0x74, 0x31, 0x20, 0xbd, 0x9a, 0x3f, - 0x3b, 0x5d, 0x00, 0x89, 0x91, 0x1f, 0x6c, 0x97, 0x08, 0xd6, 0x60, 0xff, 0xdb, 0x2a, 0x7a, 0x05, - 0xf7, 0x02, 0x6b, 0x1a, 0x64, 0x2e, 0xe6, 0xd5, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xec, 0x63, 0x4b, - 0xf6, 0x96, 0x98, 0xe9, 0x7a, 0x60, 0x5a, 0xba, 0xe9, 0x33, 0x94, 0x14, 0x83, 0x95, 0x7e, 0xc5, - 0xa9, 0xd0, 0xcc, 0x9e, 0xce, 0x88, 0xf2, 0xdb, 0xa6, 0x6e, 0x9b, 0x46, 0x21, 0xc3, 0x94, 0xcf, - 0x9e, 0xd0, 0x3d, 0x10, 0x9d, 0x63, 0x43, 0xde, 0xc7, 0x8a, 0xe5, 0xec, 0x62, 0xc5, 0x21, 0xf3, - 0x36, 0xd0, 0x1c, 0xd3, 0xce, 0xb1, 0x51, 0x09, 0x90, 0xd1, 0x0b, 0x10, 0x75, 0x43, 0xde, 0x6b, - 0xea, 0x8d, 0x7d, 0x47, 0x3e, 0xb2, 0x74, 0x07, 0xdb, 0x85, 0x19, 0xaa, 0x90, 0xa8, 0x7e, 0x5b, - 0xe3, 0xab, 0xc3, 0xda, 0x2b, 0x92, 0x93, 0xab, 0x26, 0xaf, 0x1b, 0x4f, 0x28, 0x3f, 0x25, 0xda, - 0xde, 0xec, 0x3e, 0x29, 0xa6, 0x8b, 0xff, 0x59, 0x80, 0xbc, 0xdb, 0xe9, 0xe2, 0x1c, 0x1f, 0x77, - 0x41, 0x34, 0x0d, 0x2c, 0xb7, 0xf7, 0x15, 0x1b, 0x73, 0x6d, 0xf2, 0x29, 0x28, 0x6f, 0x1a, 0x78, - 0x9b, 0x90, 0x99, 0xd2, 0xd0, 0x36, 0xcc, 0xd8, 0x8e, 0xd2, 0xd0, 0x8d, 0x46, 0x40, 0xc9, 0xe3, - 0xa3, 0xbb, 0xfe, 0x22, 0xe7, 0xf6, 0xe8, 0x21, 0xbf, 0xe5, 0x8f, 0x04, 0x98, 0x59, 0xd1, 0x5a, - 0xba, 0x51, 0x6b, 0x37, 0xf5, 0x58, 0x57, 0x1a, 0x6e, 0x42, 0xc6, 0x26, 0x32, 0x7d, 0xe3, 0xef, - 0xa3, 0xc4, 0x34, 0x4d, 0x21, 0xb3, 0xc0, 0x33, 0x98, 0xc6, 0xc7, 0x6d, 0xdd, 0x52, 0x1c, 0xdd, - 0x34, 0x18, 0xac, 0x49, 0x8d, 0x5e, 0xb7, 0xbc, 0xcf, 0xeb, 0x43, 0x1b, 0x5e, 0xb3, 0xcf, 0x00, - 0x05, 0x2b, 0x16, 0x27, 0xbe, 0x91, 0x61, 0x96, 0x8a, 0xde, 0x31, 0xec, 0x98, 0xb5, 0xc6, 0xad, - 0xf3, 0xaf, 0xc2, 0x5c, 0xf8, 0x05, 0x71, 0x96, 0xfe, 0xfb, 0xbc, 0xc5, 0x37, 0xb1, 0xd5, 0xf8, - 0x1a, 0xd6, 0x5a, 0x5c, 0xbd, 0x73, 0xf1, 0x71, 0x96, 0xfc, 0x37, 0x04, 0xb8, 0x4a, 0x65, 0xd3, - 0x5d, 0x99, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, 0x2b, 0xc2, 0xbe, 0x01, 0x13, 0x0c, 0x29, 0xd3, - 0x1e, 0x3b, 0xbe, 0x9a, 0x25, 0x7e, 0x4d, 0xcd, 0x31, 0x2d, 0xe2, 0xd7, 0xf0, 0x24, 0x5e, 0x4f, - 0x05, 0xe6, 0xa3, 0xca, 0x12, 0xf3, 0x52, 0xc2, 0x0c, 0x77, 0x2f, 0x49, 0x17, 0x2f, 0xed, 0x13, - 0xbf, 0x0a, 0x95, 0x21, 0xab, 0xd2, 0x7f, 0xb2, 0x73, 0xd2, 0xc6, 0x54, 0x7e, 0x7e, 0x90, 0x67, - 0xca, 0xd8, 0xea, 0x27, 0x6d, 0x4c, 0xdc, 0x5b, 0xf7, 0x3f, 0x51, 0x57, 0xa0, 0xaa, 0x03, 0x7d, - 0x5b, 0x3a, 0xbe, 0x68, 0x5e, 0xd7, 0x3d, 0x0c, 0x69, 0xe2, 0x9f, 0x26, 0xb9, 0x2a, 0xd8, 0x9b, - 0x38, 0x53, 0xac, 0xde, 0xcc, 0xe7, 0xa1, 0x0d, 0xb2, 0x60, 0xf5, 0x13, 0xe7, 0xa8, 0x7e, 0x60, - 0x65, 0xde, 0xa7, 0xa2, 0xcf, 0x20, 0xb0, 0xf6, 0x2e, 0xb3, 0x9a, 0xb9, 0x68, 0xe9, 0x3c, 0x4a, - 0x99, 0xf1, 0xa5, 0x30, 0xba, 0x8d, 0x4a, 0x90, 0xc6, 0xc7, 0x6d, 0x59, 0xc3, 0xb6, 0xca, 0xcd, - 0x5a, 0xb1, 0xdf, 0x4e, 0x5e, 0x0f, 0x7e, 0x98, 0xc4, 0xc7, 0x6d, 0x42, 0x44, 0x3b, 0x64, 0x86, - 0x73, 0xdd, 0x09, 0x5a, 0x6c, 0x7b, 0x38, 0x1c, 0xf1, 0xfb, 0x0b, 0x17, 0x37, 0xed, 0x79, 0x12, - 0x4c, 0x04, 0x6f, 0xbb, 0x2f, 0x04, 0x78, 0x2b, 0xb2, 0xed, 0xe2, 0x9c, 0xec, 0x3e, 0x81, 0x14, - 0x55, 0x41, 0xe2, 0x9c, 0x2a, 0xa0, 0x5c, 0xc5, 0xdf, 0x75, 0x47, 0xbd, 0x84, 0x9b, 0x26, 0x51, - 0xef, 0xd7, 0xb0, 0xae, 0x36, 0xe9, 0x36, 0x7b, 0xe2, 0xdc, 0xcd, 0xee, 0xb2, 0x76, 0x99, 0x85, - 0xae, 0xc2, 0xc6, 0x69, 0x16, 0xfe, 0xb6, 0x00, 0xb3, 0x9e, 0x0f, 0x14, 0xb3, 0x3b, 0xfc, 0x01, - 0x24, 0x0d, 0xf3, 0xe8, 0x3c, 0x4b, 0x8b, 0x24, 0xbf, 0x3f, 0x6d, 0x85, 0xcb, 0x15, 0x67, 0xad, - 0xff, 0x4d, 0x02, 0x32, 0x4f, 0x4b, 0x71, 0xd6, 0xf5, 0x13, 0xbe, 0x84, 0xcd, 0x86, 0x7a, 0x54, - 0xb7, 0xf4, 0xde, 0xb7, 0xf4, 0xb4, 0xb4, 0x81, 0x4f, 0xdc, 0x6e, 0x49, 0xb8, 0xd0, 0x0a, 0x64, - 0x9c, 0x7d, 0x0b, 0xdb, 0xfb, 0x66, 0x53, 0x3b, 0x8f, 0xcf, 0xe2, 0x73, 0xcd, 0x1f, 0xc0, 0x38, - 0x95, 0xeb, 0x86, 0x51, 0x08, 0x11, 0x61, 0x14, 0xe4, 0x35, 0x9e, 0xdb, 0x97, 0x38, 0xcf, 0x6b, - 0x5c, 0x02, 0x6b, 0x1c, 0xcf, 0x37, 0x1a, 0x17, 0x27, 0x8a, 0x2f, 0x00, 0x48, 0xd5, 0xe2, 0x6c, - 0x9e, 0xbf, 0x9e, 0x84, 0xfc, 0x76, 0xc7, 0xde, 0x8f, 0xb9, 0x3f, 0x96, 0x00, 0xda, 0x1d, 0x9b, - 0x22, 0x8c, 0x63, 0x83, 0xd7, 0x7f, 0x48, 0x9c, 0x86, 0xab, 0x00, 0xc6, 0x57, 0x3f, 0x36, 0x50, - 0x85, 0x0b, 0xc1, 0xb2, 0x1f, 0xec, 0x71, 0x63, 0x10, 0x16, 0xad, 0x1f, 0x1b, 0x9b, 0xd8, 0x03, - 0xa1, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x49, 0xf2, 0x20, 0x3b, 0xe6, 0x79, 0x9a, 0x7c, 0x82, - 0xf0, 0xd4, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x4c, 0x5c, 0x13, 0x74, 0xe2, 0x8a, 0xaa, 0x0b, - 0x57, 0x23, 0x9d, 0xb2, 0xd2, 0x94, 0x95, 0x4c, 0x53, 0x73, 0x30, 0xbe, 0x67, 0x5a, 0x2a, 0xa6, - 0x11, 0x1c, 0x69, 0x89, 0x3d, 0x04, 0x5b, 0x75, 0x3d, 0x95, 0x4e, 0x8b, 0x99, 0xf5, 0x54, 0x3a, - 0x23, 0x42, 0xf1, 0x37, 0x05, 0x98, 0xf6, 0x9a, 0x23, 0x4e, 0x5b, 0x5e, 0x0a, 0xe9, 0xf2, 0xfc, - 0x0d, 0x42, 0xd4, 0x58, 0xfc, 0xb7, 0xd4, 0xb1, 0x51, 0xcd, 0x43, 0xda, 0x3e, 0x71, 0xf6, 0x97, - 0xc7, 0x2c, 0xa0, 0x27, 0x71, 0xde, 0x36, 0xa6, 0xb1, 0x3d, 0x0f, 0x60, 0x4e, 0x6f, 0x11, 0x2b, - 0xaf, 0x3b, 0xcd, 0x13, 0x8e, 0xca, 0x1c, 0xec, 0xee, 0x11, 0xcf, 0xfa, 0x69, 0x25, 0x37, 0x89, - 0x1b, 0x3e, 0xb6, 0xe7, 0xe3, 0xd7, 0x27, 0x4e, 0x85, 0x57, 0x61, 0xca, 0x62, 0xa2, 0x89, 0x77, - 0x72, 0x4e, 0x9d, 0xe7, 0x3c, 0x56, 0xa2, 0xf6, 0xdf, 0x49, 0xc0, 0xf4, 0x8b, 0x0e, 0xb6, 0x4e, - 0xbe, 0x49, 0x4a, 0xbf, 0x0d, 0xd3, 0x47, 0x8a, 0xee, 0xc8, 0x7b, 0xa6, 0x25, 0x77, 0xda, 0x9a, - 0xe2, 0xb8, 0x51, 0x25, 0x53, 0x84, 0xfc, 0xc4, 0xb4, 0x76, 0x28, 0x11, 0x61, 0x40, 0x07, 0x86, - 0x79, 0x64, 0xc8, 0x84, 0x4c, 0xd1, 0xf0, 0xb1, 0xc1, 0x17, 0xa3, 0x57, 0x3f, 0xfc, 0x4f, 0xa7, - 0x0b, 0x8f, 0x46, 0x8a, 0x1b, 0xa3, 0x91, 0x6f, 0x9d, 0x8e, 0xae, 0x2d, 0xed, 0xec, 0x54, 0xd7, - 0x24, 0x91, 0x8a, 0x7c, 0xc5, 0x24, 0xd6, 0x8f, 0x0d, 0x77, 0x16, 0xff, 0x87, 0x09, 0x10, 0x7d, - 0x4d, 0xc5, 0xd9, 0x9c, 0x65, 0xc8, 0xbe, 0xee, 0x60, 0x4b, 0xbf, 0x40, 0x63, 0x02, 0x67, 0x24, - 0x86, 0xe8, 0x73, 0xc8, 0x85, 0xf4, 0x90, 0xfc, 0x6a, 0x7a, 0xc8, 0x1e, 0xf9, 0x2a, 0x40, 0xf7, - 0x61, 0xc6, 0x39, 0x36, 0x64, 0x16, 0x17, 0xc8, 0x22, 0x4b, 0xdc, 0x30, 0x88, 0x69, 0x87, 0xe8, - 0x83, 0xd0, 0x69, 0x54, 0x89, 0x5d, 0xfc, 0xd7, 0x02, 0x20, 0xaa, 0xa8, 0x2a, 0xdb, 0x33, 0xf8, - 0xa6, 0xf4, 0xaa, 0xbb, 0x20, 0xd2, 0xf8, 0x4a, 0x59, 0xdf, 0x93, 0x5b, 0xba, 0x6d, 0xeb, 0x46, - 0x83, 0x77, 0xab, 0x3c, 0xa5, 0x57, 0xf7, 0x36, 0x19, 0x95, 0x37, 0xf8, 0x5f, 0x80, 0xd9, 0x50, - 0x35, 0xe2, 0x6c, 0xf2, 0xeb, 0x90, 0xdb, 0x33, 0x3b, 0x86, 0x26, 0xb3, 0x7d, 0x15, 0xbe, 0xd0, - 0x98, 0xa5, 0x34, 0xf6, 0xbe, 0xe2, 0xff, 0x4c, 0xc0, 0x9c, 0x84, 0x6d, 0xb3, 0x79, 0x88, 0xe3, - 0x57, 0x64, 0x05, 0xf8, 0x8e, 0x8e, 0x7c, 0x21, 0x7d, 0x66, 0x18, 0x33, 0x9b, 0xfe, 0xc2, 0x6b, - 0xf6, 0x37, 0x07, 0xf7, 0xdb, 0xde, 0x55, 0x7a, 0xbe, 0xda, 0x97, 0x0a, 0xad, 0xf6, 0x99, 0x30, - 0xad, 0x37, 0x0c, 0x93, 0xd8, 0x37, 0x1b, 0xbf, 0x36, 0x3a, 0x2d, 0x17, 0xdf, 0x2c, 0x0d, 0x2a, - 0x64, 0x95, 0xb1, 0xd4, 0xf0, 0xeb, 0xad, 0x4e, 0x8b, 0x7a, 0xd9, 0xab, 0x97, 0x49, 0x79, 0xcf, - 0x4e, 0x17, 0xf2, 0xa1, 0x34, 0x5b, 0xca, 0xeb, 0xde, 0x33, 0x91, 0xce, 0x9b, 0xfc, 0x7b, 0x70, - 0xa9, 0x4b, 0xe5, 0x71, 0xfa, 0x43, 0xff, 0x2a, 0x09, 0x57, 0xc3, 0xe2, 0xe3, 0x46, 0x2d, 0xdf, - 0xf4, 0x66, 0xad, 0xc0, 0x54, 0x4b, 0x37, 0x2e, 0xb6, 0x68, 0x99, 0x6b, 0xe9, 0x86, 0xbf, 0x4c, - 0x1c, 0xd1, 0x41, 0x26, 0xfe, 0x3f, 0x74, 0x10, 0x05, 0xe6, 0xa3, 0x5a, 0x30, 0xce, 0x5e, 0xf2, - 0x23, 0x01, 0x72, 0x71, 0xaf, 0xc3, 0x5d, 0x2c, 0x22, 0x8e, 0xd7, 0xb9, 0x0e, 0x53, 0x5f, 0xc3, - 0xc2, 0xdd, 0xef, 0x08, 0x80, 0xea, 0x56, 0xc7, 0x20, 0x80, 0xf8, 0x99, 0xd9, 0x88, 0xb3, 0xb2, - 0x73, 0x30, 0xae, 0x1b, 0x1a, 0x3e, 0xa6, 0x95, 0x4d, 0x49, 0xec, 0x21, 0xb4, 0x59, 0x99, 0x1c, - 0x69, 0xb3, 0xd2, 0x0f, 0x8b, 0x09, 0x15, 0x34, 0x4e, 0x2d, 0xfc, 0xe3, 0x04, 0xcc, 0xf2, 0xea, - 0xc4, 0xbe, 0x70, 0xf9, 0x6d, 0x18, 0x6f, 0x12, 0x99, 0x03, 0xda, 0x9c, 0xbe, 0xd3, 0x6d, 0x73, - 0x9a, 0x19, 0xfd, 0x32, 0x40, 0xdb, 0xc2, 0x87, 0x32, 0x63, 0x4d, 0x8e, 0xc4, 0x9a, 0x21, 0x1c, - 0x94, 0x80, 0xbe, 0x0b, 0xd3, 0x64, 0x84, 0xb7, 0x2d, 0xb3, 0x6d, 0xda, 0xc4, 0xa1, 0xb1, 0x47, - 0x43, 0x45, 0x33, 0x67, 0xa7, 0x0b, 0x53, 0x9b, 0xba, 0xb1, 0xcd, 0x19, 0xeb, 0x35, 0x89, 0x98, - 0x0a, 0xef, 0xd1, 0x1d, 0x80, 0xff, 0x41, 0x80, 0xb9, 0xaf, 0x6d, 0xa9, 0xf7, 0x4f, 0x43, 0x63, - 0xde, 0xcc, 0x23, 0xd2, 0xc7, 0xaa, 0xb1, 0x67, 0xc6, 0xbf, 0x00, 0xff, 0x23, 0x01, 0x66, 0x02, - 0xe2, 0xe3, 0xf4, 0x64, 0x2e, 0xa4, 0xb3, 0xe2, 0xaf, 0x12, 0xdf, 0x26, 0xd8, 0xed, 0xe3, 0x1c, - 0x54, 0xff, 0x3c, 0x01, 0x97, 0x4b, 0x6c, 0x1b, 0xdb, 0x8d, 0xf1, 0x88, 0xb3, 0x97, 0x14, 0x60, - 0xf2, 0x10, 0x5b, 0xb6, 0x6e, 0xb2, 0x19, 0x76, 0x4a, 0x72, 0x1f, 0xd1, 0x3c, 0xa4, 0x6d, 0x43, - 0x69, 0xdb, 0xfb, 0xa6, 0xbb, 0x73, 0xe7, 0x3d, 0x7b, 0xf1, 0x28, 0xe3, 0x17, 0x8f, 0x47, 0x99, - 0x18, 0x1c, 0x8f, 0x32, 0xf9, 0x95, 0xe3, 0x51, 0xf8, 0x36, 0xd9, 0x1f, 0x08, 0x70, 0xa5, 0x47, - 0x7f, 0x71, 0xf6, 0x99, 0x1f, 0x42, 0x56, 0xe5, 0x82, 0x89, 0x35, 0x66, 0x3b, 0x81, 0x55, 0x92, - 0xed, 0x82, 0x60, 0xe5, 0xec, 0x74, 0x01, 0xdc, 0xa2, 0x56, 0xd7, 0xb8, 0x8a, 0xc8, 0x7f, 0xad, - 0xf8, 0x1f, 0x73, 0x30, 0x5d, 0x3e, 0x66, 0xeb, 0xdc, 0x35, 0xe6, 0x0f, 0xa0, 0x27, 0x90, 0x6e, - 0x5b, 0xe6, 0xa1, 0xee, 0x56, 0x23, 0x1f, 0x0a, 0x43, 0x70, 0xab, 0xd1, 0xc5, 0xb5, 0xcd, 0x39, - 0x24, 0x8f, 0x17, 0xd5, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xe6, 0x13, 0xbd, 0xe9, 0xf6, 0xff, 0xf7, - 0x87, 0x0b, 0x5a, 0xf2, 0x78, 0xb6, 0x15, 0x67, 0xdf, 0x6d, 0x0a, 0x8f, 0x88, 0xaa, 0x90, 0xae, - 0x38, 0x4e, 0x9b, 0x24, 0x72, 0x6b, 0x72, 0x67, 0x04, 0xa1, 0x84, 0xc5, 0x8d, 0x6c, 0x75, 0xd9, - 0x51, 0x1d, 0x66, 0x9e, 0x9a, 0x66, 0xa3, 0x89, 0x4b, 0x4d, 0xb3, 0xa3, 0x95, 0x4c, 0x63, 0x4f, - 0x6f, 0x70, 0x7b, 0x7c, 0x7b, 0x04, 0x99, 0x4f, 0x4b, 0x35, 0xa9, 0x57, 0x00, 0x5a, 0x81, 0x74, - 0xed, 0x11, 0x17, 0xc6, 0x1c, 0xb8, 0x5b, 0x23, 0x08, 0xab, 0x3d, 0x92, 0x3c, 0x36, 0xb4, 0x0e, - 0xd9, 0x95, 0x37, 0x1d, 0x0b, 0x73, 0x29, 0x13, 0x7d, 0x63, 0x20, 0xba, 0xa5, 0x50, 0x2e, 0x29, - 0xc8, 0x8c, 0x6a, 0x90, 0x7f, 0x65, 0x5a, 0x07, 0x4d, 0x53, 0x71, 0x6b, 0x38, 0x49, 0xc5, 0x7d, - 0x6b, 0x04, 0x71, 0x2e, 0xa3, 0xd4, 0x25, 0x02, 0x7d, 0x0f, 0xa6, 0x49, 0x63, 0xd4, 0x95, 0xdd, - 0xa6, 0x5b, 0xc8, 0x34, 0x95, 0xfa, 0xee, 0x08, 0x52, 0x3d, 0x4e, 0x77, 0xa3, 0xa5, 0x4b, 0xd4, - 0xfc, 0x77, 0x61, 0x2a, 0xd4, 0x09, 0x10, 0x82, 0x54, 0x9b, 0xb4, 0xb7, 0x40, 0x63, 0x95, 0xe8, - 0x7f, 0xf4, 0x1e, 0x4c, 0x1a, 0xa6, 0x86, 0xdd, 0x11, 0x32, 0xb5, 0x3a, 0x77, 0x76, 0xba, 0x30, - 0xb1, 0x65, 0x6a, 0xcc, 0x5d, 0xe1, 0xff, 0xa4, 0x09, 0x92, 0xc9, 0x75, 0x56, 0xe6, 0x6f, 0x43, - 0x8a, 0xb4, 0x3e, 0x31, 0x52, 0xbb, 0x8a, 0x8d, 0x77, 0x2c, 0x9d, 0xcb, 0x74, 0x1f, 0x79, 0xbe, - 0x9f, 0x0b, 0x90, 0xa8, 0x3d, 0x22, 0x8e, 0xfa, 0x6e, 0x47, 0x3d, 0xc0, 0x0e, 0xcf, 0xc5, 0x9f, - 0xa8, 0x03, 0x6f, 0xe1, 0x3d, 0x9d, 0xf9, 0x50, 0x19, 0x89, 0x3f, 0xa1, 0x77, 0x00, 0x14, 0x55, - 0xc5, 0xb6, 0x2d, 0xbb, 0x67, 0xfc, 0x32, 0x52, 0x86, 0x51, 0x36, 0xf0, 0x09, 0x61, 0xb3, 0xb1, - 0x6a, 0x61, 0xc7, 0x0d, 0xba, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xad, 0xb6, 0xec, 0x98, 0x07, 0xd8, - 0xa0, 0x7d, 0x26, 0x43, 0x8c, 0x4f, 0xab, 0x5d, 0x27, 0x04, 0x62, 0x37, 0xb1, 0xa1, 0xf9, 0x46, - 0x2e, 0x23, 0x79, 0xcf, 0x44, 0xa4, 0x85, 0x1b, 0x3a, 0x3f, 0xa6, 0x96, 0x91, 0xf8, 0x13, 0xd1, - 0x98, 0xd2, 0x71, 0xf6, 0x69, 0xab, 0x64, 0x24, 0xfa, 0x9f, 0x57, 0xed, 0xef, 0x0a, 0x90, 0x7c, - 0x5a, 0xaa, 0x9d, 0xbb, 0x6e, 0xae, 0xc4, 0xa4, 0x2f, 0x91, 0xc6, 0x3a, 0xea, 0xcd, 0xa6, 0x6e, - 0x34, 0x88, 0x4b, 0xf3, 0x43, 0xac, 0xba, 0x35, 0xcb, 0x73, 0xf2, 0x36, 0xa3, 0xa2, 0x45, 0xc8, - 0xaa, 0x16, 0xd6, 0xb0, 0xe1, 0xe8, 0x4a, 0xd3, 0xe6, 0x55, 0x0c, 0x92, 0x78, 0xe1, 0x7e, 0x5d, - 0x80, 0x71, 0xda, 0x79, 0xd1, 0xdb, 0x90, 0x51, 0x4d, 0xc3, 0x51, 0x74, 0x83, 0x5b, 0xa1, 0x8c, - 0xe4, 0x13, 0xfa, 0x16, 0xf2, 0x3a, 0xe4, 0x14, 0x55, 0x35, 0x3b, 0x86, 0x23, 0x1b, 0x4a, 0x0b, - 0xf3, 0xc2, 0x66, 0x39, 0x6d, 0x4b, 0x69, 0x61, 0xb4, 0x00, 0xee, 0xa3, 0x77, 0xd2, 0x32, 0x23, - 0x01, 0x27, 0x6d, 0xe0, 0x13, 0x5e, 0x92, 0x3f, 0x10, 0x20, 0xed, 0x76, 0x7a, 0x52, 0x98, 0x06, - 0x36, 0xb0, 0xa5, 0x38, 0xa6, 0x57, 0x18, 0x8f, 0xd0, 0x3d, 0xe3, 0x65, 0xfc, 0x19, 0x6f, 0x0e, - 0xc6, 0x1d, 0xd2, 0xaf, 0x79, 0x39, 0xd8, 0x03, 0x5d, 0x97, 0x6e, 0x2a, 0x0d, 0xb6, 0x14, 0x97, - 0x91, 0xd8, 0x03, 0xa9, 0x12, 0x8f, 0xd7, 0x65, 0xda, 0xe1, 0x4f, 0xa4, 0xbc, 0x2c, 0x9e, 0x74, - 0x17, 0x37, 0x74, 0x83, 0x76, 0x80, 0xa4, 0x04, 0x94, 0xb4, 0x4a, 0x28, 0xe8, 0x2d, 0xc8, 0xb0, - 0x0c, 0xd8, 0xd0, 0x68, 0x2f, 0x48, 0x4a, 0x69, 0x4a, 0x28, 0xbb, 0x47, 0xc9, 0xe6, 0x0f, 0x20, - 0xe3, 0x8d, 0x31, 0xd2, 0x90, 0x1d, 0xdb, 0x53, 0x2a, 0xfd, 0x8f, 0xde, 0x87, 0xb9, 0xd7, 0x1d, - 0xa5, 0xa9, 0xef, 0xd1, 0x55, 0x36, 0x92, 0x8d, 0xe9, 0x8f, 0xd5, 0x07, 0x79, 0x69, 0x54, 0x02, - 0x55, 0xa3, 0x3b, 0x24, 0x93, 0xfe, 0x90, 0x0c, 0x6e, 0x9b, 0x14, 0x7f, 0x22, 0xc0, 0x0c, 0x0b, - 0x19, 0x62, 0x51, 0xaf, 0xf1, 0x39, 0x18, 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x9d, 0x26, 0x4d, - 0x0c, 0x3c, 0x4d, 0xea, 0x9d, 0x65, 0x50, 0x1c, 0x85, 0x9e, 0x28, 0x45, 0x90, 0x22, 0xff, 0xd9, - 0xf1, 0x5b, 0x89, 0xfe, 0xf7, 0x83, 0x30, 0x82, 0xc5, 0x8d, 0xd3, 0xe1, 0x5a, 0x86, 0x4b, 0x44, - 0xfb, 0x65, 0x43, 0xb5, 0x4e, 0xda, 0x8e, 0x6e, 0x1a, 0xcf, 0xe9, 0xaf, 0x8d, 0xc4, 0xc0, 0x26, - 0x16, 0xdd, 0xbb, 0xe2, 0x65, 0xf9, 0xfd, 0x09, 0x98, 0x2a, 0x1f, 0xb7, 0x4d, 0x2b, 0xd6, 0x45, - 0xad, 0x55, 0x98, 0xe4, 0x88, 0x7f, 0xc0, 0xb6, 0x72, 0x97, 0xad, 0x76, 0x77, 0x6c, 0x39, 0x23, - 0x5a, 0x05, 0x60, 0xf1, 0xa9, 0x34, 0xee, 0x28, 0x79, 0x8e, 0xcd, 0x35, 0xca, 0x46, 0xa8, 0x68, - 0x0b, 0xb2, 0xad, 0x43, 0x55, 0x95, 0xf7, 0xf4, 0xa6, 0xc3, 0x03, 0xfc, 0xa2, 0xa3, 0xd3, 0x37, - 0x5f, 0x96, 0x4a, 0x4f, 0x68, 0x26, 0x16, 0x6b, 0xe7, 0x3f, 0x4b, 0x40, 0x24, 0xb0, 0xff, 0xe8, - 0x5d, 0xe0, 0xa7, 0x7c, 0x64, 0xdb, 0x3d, 0xd0, 0xb7, 0x3a, 0x75, 0x76, 0xba, 0x90, 0x91, 0x28, - 0xb5, 0x56, 0xab, 0x4b, 0x19, 0x96, 0xa1, 0x66, 0x3b, 0xe8, 0x06, 0x4c, 0x99, 0x2d, 0xdd, 0x91, - 0x5d, 0x1f, 0x88, 0xbb, 0x8d, 0x39, 0x42, 0x74, 0x7d, 0x24, 0x54, 0x87, 0x3b, 0xd8, 0xa0, 0xa3, - 0x80, 0xd4, 0x53, 0xde, 0x65, 0x6b, 0x91, 0x0e, 0x1b, 0xef, 0xb2, 0xd9, 0x76, 0xf4, 0x96, 0xfe, - 0x86, 0x6e, 0x6c, 0xf3, 0xbd, 0xa5, 0x1b, 0x2c, 0x3b, 0xa9, 0xdf, 0x2a, 0x5d, 0xa4, 0xe4, 0x79, - 0x9f, 0x07, 0xb2, 0xa2, 0x5f, 0x17, 0xe0, 0x32, 0x57, 0xa4, 0xbc, 0x4b, 0xc3, 0xeb, 0x95, 0xa6, - 0xee, 0x9c, 0xc8, 0x07, 0x87, 0x85, 0x34, 0x75, 0x4e, 0x7f, 0x29, 0xb2, 0x41, 0x02, 0xfd, 0x60, - 0xc9, 0x6d, 0x96, 0x93, 0x67, 0x9c, 0x79, 0xe3, 0xb0, 0x6c, 0x38, 0xd6, 0xc9, 0xea, 0x95, 0xb3, - 0xd3, 0x85, 0xd9, 0xde, 0xd4, 0x97, 0xd2, 0xac, 0xdd, 0xcb, 0x82, 0x2a, 0x00, 0xd8, 0xeb, 0x8d, - 0x34, 0x92, 0x30, 0xda, 0xbd, 0x88, 0xec, 0xb6, 0x52, 0x80, 0x17, 0xdd, 0x05, 0x91, 0x1f, 0xb0, - 0xd9, 0xd3, 0x9b, 0x58, 0xb6, 0xf5, 0x37, 0x98, 0xc6, 0x1d, 0x26, 0xa5, 0x3c, 0xa3, 0x13, 0x11, - 0x35, 0xfd, 0x0d, 0x9e, 0xff, 0x21, 0x14, 0xfa, 0x95, 0x3e, 0x38, 0x10, 0x32, 0x6c, 0x13, 0xf7, - 0xa3, 0xf0, 0x8a, 0xcc, 0x08, 0x5d, 0xd5, 0x5d, 0x95, 0x49, 0x7c, 0xe4, 0x9a, 0xa0, 0xdf, 0x4d, - 0xc0, 0xd4, 0x6a, 0xa7, 0x79, 0xf0, 0xbc, 0x5d, 0xeb, 0xb4, 0x5a, 0x8a, 0x75, 0x42, 0x4c, 0x25, - 0x33, 0x1d, 0xa4, 0x98, 0x02, 0x33, 0x95, 0xd4, 0x36, 0xe8, 0x6f, 0x30, 0x99, 0xcc, 0x82, 0x47, - 0xca, 0xd9, 0xf1, 0x01, 0x5a, 0x93, 0xc0, 0x39, 0x71, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x21, 0x90, - 0x91, 0x2e, 0x9f, 0xc8, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0xe5, 0xc0, 0xa4, 0x14, 0x08, 0xbd, 0xa9, - 0x92, 0xe4, 0x32, 0x4b, 0x45, 0x75, 0xc8, 0x91, 0x8c, 0x27, 0x32, 0x9d, 0x6c, 0xdc, 0x45, 0xdb, - 0x07, 0x11, 0x95, 0x0b, 0x95, 0x7b, 0x89, 0x6a, 0xa9, 0x44, 0x79, 0xe8, 0x5f, 0x29, 0x8b, 0x7d, - 0xca, 0xfc, 0xa7, 0x20, 0x76, 0x67, 0x08, 0x6a, 0x34, 0xc5, 0x34, 0x3a, 0x17, 0xd4, 0x68, 0x32, - 0xa0, 0xad, 0xf5, 0x54, 0x3a, 0x25, 0x8e, 0x17, 0xff, 0x38, 0x09, 0x79, 0xb7, 0xb3, 0xc5, 0x89, - 0x66, 0x56, 0x61, 0x9c, 0x74, 0x0d, 0x37, 0x50, 0xe4, 0xf6, 0x80, 0x3e, 0xce, 0x43, 0xd5, 0x49, - 0x97, 0x71, 0xf1, 0x30, 0x65, 0x8d, 0xc3, 0xec, 0xcc, 0xff, 0xc5, 0x04, 0xa4, 0x28, 0x80, 0x78, - 0x00, 0x29, 0x3a, 0x75, 0x08, 0xa3, 0x4c, 0x1d, 0x34, 0xab, 0x37, 0xd9, 0x25, 0x02, 0xfe, 0x27, - 0x71, 0xe6, 0xf6, 0x95, 0x0f, 0x1e, 0x3c, 0xa4, 0x26, 0x27, 0x27, 0xf1, 0x27, 0xb4, 0x4a, 0x23, - 0x98, 0x4c, 0xcb, 0xc1, 0x1a, 0x77, 0xdc, 0x17, 0x87, 0xb5, 0xaf, 0x3b, 0x4d, 0xb9, 0x7c, 0xe8, - 0x2a, 0x24, 0x89, 0x2d, 0x9b, 0x64, 0xd1, 0x0d, 0x67, 0xa7, 0x0b, 0x49, 0x62, 0xc5, 0x08, 0x0d, - 0x2d, 0x43, 0x36, 0x6c, 0x38, 0x84, 0xbb, 0x19, 0x66, 0x1e, 0x03, 0x83, 0x1e, 0x9a, 0xde, 0x00, - 0x63, 0xa0, 0x95, 0xb7, 0xf1, 0xaf, 0x8d, 0xc3, 0x54, 0xb5, 0x15, 0xf7, 0xc4, 0xb2, 0x12, 0x6e, - 0xe1, 0x28, 0xb4, 0x13, 0x7a, 0x69, 0x44, 0x03, 0x87, 0xe6, 0xf4, 0xe4, 0xf9, 0xe6, 0xf4, 0x2a, - 0x71, 0x81, 0xf9, 0x1d, 0x11, 0xc9, 0x3e, 0xc0, 0x26, 0xfc, 0x7e, 0xea, 0xc5, 0x48, 0x84, 0xc7, - 0x3f, 0xbc, 0x41, 0x23, 0x54, 0x3e, 0xa5, 0x9e, 0x36, 0xeb, 0x65, 0x13, 0xa3, 0xf7, 0xb2, 0x49, - 0x6c, 0x68, 0x74, 0x6a, 0x0b, 0xdb, 0xd5, 0xc9, 0x8b, 0xdb, 0xd5, 0xf9, 0x37, 0xbc, 0xb3, 0x7e, - 0x0c, 0x49, 0x4d, 0x77, 0x1b, 0x67, 0xf4, 0x09, 0x9b, 0x30, 0x0d, 0xe9, 0xb5, 0xa9, 0x60, 0xaf, - 0x0d, 0x2e, 0x70, 0xcc, 0x3f, 0x07, 0xf0, 0x35, 0x84, 0x16, 0x61, 0xc2, 0x6c, 0x6a, 0xee, 0x19, - 0x96, 0xa9, 0xd5, 0xcc, 0xd9, 0xe9, 0xc2, 0xf8, 0xf3, 0xa6, 0x56, 0x5d, 0x93, 0xc6, 0xcd, 0xa6, - 0x56, 0xd5, 0xe8, 0x35, 0x1d, 0xf8, 0x48, 0xf6, 0x02, 0xd6, 0x72, 0xd2, 0xa4, 0x81, 0x8f, 0xd6, - 0xb0, 0xad, 0x76, 0x05, 0xd2, 0x90, 0x2e, 0xf8, 0x63, 0x01, 0xf2, 0x6e, 0x6b, 0xc4, 0x6b, 0x66, - 0xd2, 0x7a, 0x8b, 0x0f, 0xbb, 0xe4, 0xf9, 0x86, 0x9d, 0xcb, 0xc7, 0xcf, 0x00, 0xff, 0x86, 0xc0, - 0x83, 0x95, 0x6b, 0xaa, 0xe2, 0x10, 0x67, 0x23, 0xc6, 0xa1, 0x72, 0x0f, 0x44, 0x4b, 0x31, 0x34, - 0xb3, 0xa5, 0xbf, 0xc1, 0x6c, 0x45, 0xd4, 0xe6, 0x9b, 0x9b, 0xd3, 0x1e, 0x9d, 0x2e, 0xf9, 0xb9, - 0x0b, 0xba, 0xbf, 0x9f, 0xe0, 0x81, 0xcd, 0x5e, 0x61, 0xe2, 0x54, 0xda, 0x0f, 0x60, 0xa6, 0xfb, - 0x16, 0x15, 0x77, 0x14, 0xbf, 0x17, 0x21, 0x2f, 0xaa, 0x20, 0x2c, 0x10, 0xd1, 0x8d, 0x9c, 0xef, - 0xba, 0x51, 0xc5, 0x46, 0x25, 0xc8, 0x06, 0x2f, 0x67, 0x49, 0x8e, 0x7c, 0x39, 0x0b, 0x58, 0xde, - 0x95, 0x2c, 0xf3, 0xbf, 0x02, 0xe3, 0x34, 0xf9, 0x02, 0xa6, 0x9b, 0xb7, 0xe9, 0x7f, 0x4d, 0xc0, - 0x4d, 0x5a, 0xfa, 0x97, 0xd8, 0xd2, 0xf7, 0x4e, 0xb6, 0x2d, 0xd3, 0xc1, 0xaa, 0x83, 0x35, 0xff, - 0x30, 0x4a, 0xac, 0xf6, 0x30, 0xd3, 0x76, 0x5f, 0x70, 0xae, 0x00, 0x34, 0x8f, 0x0b, 0x6d, 0xc0, - 0x34, 0x0f, 0x26, 0x50, 0x9a, 0xfa, 0x21, 0x96, 0x15, 0xe7, 0x3c, 0xb3, 0xde, 0x14, 0xe3, 0x5d, - 0x21, 0xac, 0x2b, 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, 0x1a, 0xbf, 0x59, 0xe8, 0xe9, 0x57, 0x5b, - 0x4d, 0x4c, 0xb3, 0x88, 0x86, 0xea, 0x9a, 0x94, 0x66, 0x92, 0xbd, 0xdd, 0xa0, 0x3f, 0x12, 0xe0, - 0xd6, 0x10, 0x45, 0xc7, 0xd9, 0x81, 0xe7, 0x21, 0x7d, 0x48, 0x5e, 0xa4, 0x73, 0x4d, 0xa7, 0x25, - 0xef, 0x19, 0x6d, 0xc2, 0xd4, 0x9e, 0xa2, 0x37, 0xfd, 0x8e, 0xdd, 0x3f, 0x6a, 0x31, 0x3a, 0x98, - 0x36, 0xc7, 0xd8, 0x59, 0x4f, 0x2e, 0xfe, 0x38, 0x01, 0x33, 0x2b, 0x9a, 0x56, 0xab, 0x71, 0xdb, - 0x18, 0x5f, 0x7f, 0x71, 0x41, 0x69, 0xc2, 0x07, 0xa5, 0xe8, 0x3d, 0x40, 0x9a, 0x6e, 0xb3, 0x5b, - 0x4b, 0xec, 0x7d, 0x45, 0x33, 0x8f, 0xfc, 0x78, 0x8c, 0x19, 0x37, 0xa5, 0xe6, 0x26, 0xa0, 0x1a, - 0x50, 0x44, 0x24, 0xdb, 0x8e, 0xe2, 0x6d, 0x29, 0xdd, 0x1a, 0xe9, 0xec, 0x18, 0x83, 0x4a, 0xde, - 0xa3, 0x94, 0x21, 0x72, 0xe8, 0x5f, 0xe2, 0xdb, 0xeb, 0xa4, 0xea, 0x8e, 0xac, 0xd8, 0xee, 0x41, - 0x21, 0x76, 0x5f, 0x4a, 0x9e, 0xd1, 0x57, 0xec, 0xe0, 0xf9, 0x1f, 0x76, 0x8e, 0xc1, 0x57, 0x50, - 0x9c, 0x10, 0xfa, 0x1f, 0x09, 0x90, 0x97, 0xf0, 0x9e, 0x85, 0xed, 0x58, 0x97, 0x12, 0x9e, 0x40, - 0xce, 0x62, 0x52, 0xe5, 0x3d, 0xcb, 0x6c, 0x9d, 0x67, 0x8c, 0x65, 0x39, 0xe3, 0x13, 0xcb, 0x6c, - 0x85, 0xae, 0x90, 0x78, 0x09, 0xd3, 0x5e, 0x49, 0xe3, 0x54, 0xc1, 0x4f, 0xe8, 0x79, 0x69, 0x26, - 0x38, 0xee, 0xc0, 0x88, 0xaf, 0x43, 0x0f, 0x74, 0x0f, 0x2b, 0x58, 0xdc, 0x38, 0x95, 0xf1, 0xdf, - 0x05, 0xc8, 0xd7, 0x3a, 0xbb, 0xec, 0xd2, 0xac, 0xf8, 0xf4, 0x50, 0x86, 0x4c, 0x13, 0xef, 0x39, - 0xf2, 0x85, 0x62, 0xef, 0xd3, 0x84, 0x95, 0x9e, 0x3f, 0x78, 0x0a, 0x60, 0xd1, 0xd3, 0x75, 0x54, - 0x4e, 0xf2, 0x9c, 0x72, 0x32, 0x94, 0xd7, 0x77, 0x9f, 0x8a, 0x3f, 0x49, 0xc0, 0xb4, 0x57, 0xd9, - 0x38, 0xad, 0xe7, 0xab, 0x90, 0xd5, 0x48, 0x9e, 0xc7, 0x6a, 0xcc, 0xf0, 0xb8, 0x90, 0x68, 0xcb, - 0xb1, 0x04, 0xb3, 0xd4, 0xb9, 0x91, 0x95, 0x76, 0xbb, 0xa9, 0xbb, 0x20, 0x99, 0xda, 0xa5, 0x94, - 0x34, 0x43, 0x93, 0x56, 0x58, 0x0a, 0x85, 0xc7, 0xa4, 0xff, 0xed, 0x59, 0x18, 0xbf, 0xc1, 0x32, - 0xc5, 0x6b, 0xe7, 0x89, 0x7b, 0xc9, 0x32, 0xc6, 0x1a, 0xe1, 0xe3, 0x3d, 0xef, 0xfb, 0x30, 0x43, - 0x35, 0x1b, 0xf7, 0x09, 0x61, 0xde, 0x1c, 0xbf, 0x95, 0x00, 0x14, 0x94, 0xff, 0xf5, 0xb5, 0x48, - 0x22, 0xbe, 0x16, 0x79, 0x17, 0x10, 0x8b, 0x85, 0xb4, 0xe5, 0x36, 0xb6, 0x64, 0x1b, 0xab, 0x26, - 0xbf, 0xca, 0x49, 0x90, 0x44, 0x9e, 0xb2, 0x8d, 0xad, 0x1a, 0xa5, 0xa3, 0xc7, 0x00, 0xbe, 0xd7, - 0xc6, 0xa7, 0x93, 0x81, 0x4e, 0x9b, 0x94, 0xf1, 0xdc, 0xb5, 0xe2, 0x8f, 0xe6, 0x21, 0xc7, 0x35, - 0xb9, 0x63, 0xe8, 0xa6, 0x81, 0x1e, 0x40, 0xb2, 0xc1, 0xb7, 0x19, 0xb2, 0x91, 0x0b, 0x7d, 0xfe, - 0xed, 0x75, 0x95, 0x31, 0x89, 0xe4, 0x25, 0x2c, 0xed, 0x8e, 0x13, 0xe1, 0x3c, 0xf9, 0x11, 0xdf, - 0x41, 0x96, 0x76, 0xc7, 0x41, 0x35, 0x98, 0x56, 0xfd, 0xbb, 0xb8, 0x64, 0xc2, 0x9e, 0xec, 0x0b, - 0xc0, 0x22, 0xef, 0x40, 0xab, 0x8c, 0x49, 0x79, 0x35, 0x94, 0x80, 0x4a, 0xc1, 0xcb, 0x9f, 0x52, - 0x3d, 0x01, 0x63, 0xfe, 0x51, 0xe4, 0xf0, 0xc5, 0x53, 0x95, 0xb1, 0xc0, 0x1d, 0x51, 0xe8, 0x63, - 0x98, 0xd0, 0xe8, 0xa5, 0x42, 0xbc, 0x5f, 0x47, 0x75, 0xbd, 0xd0, 0x3d, 0x4e, 0x95, 0x31, 0x89, - 0x73, 0xa0, 0x75, 0xc8, 0xb1, 0x7f, 0xcc, 0x89, 0xe1, 0xa8, 0xf4, 0x56, 0x7f, 0x09, 0x81, 0xa9, - 0xa1, 0x32, 0x26, 0x65, 0x35, 0x9f, 0x8a, 0xbe, 0x0d, 0x29, 0x5b, 0x55, 0x5c, 0x5c, 0x7a, 0xad, - 0xcf, 0x7d, 0x20, 0x3e, 0x33, 0xcd, 0x8d, 0x1e, 0xb3, 0x5b, 0x29, 0x9d, 0x63, 0x77, 0xa1, 0x30, - 0xaa, 0xf8, 0xa1, 0x53, 0xe6, 0xa4, 0xf8, 0x98, 0x12, 0xd0, 0x53, 0xc8, 0x2a, 0xc4, 0x1b, 0x94, - 0xe9, 0xa9, 0x4c, 0xba, 0x32, 0x18, 0xbd, 0x07, 0xdf, 0x73, 0xa2, 0xb6, 0x42, 0x8f, 0xb2, 0xbb, - 0x44, 0x5f, 0x50, 0x0b, 0x5b, 0x0d, 0x5c, 0xc8, 0x0e, 0x16, 0x14, 0x0c, 0x10, 0xf3, 0x04, 0x51, - 0x22, 0xf1, 0x0a, 0xbd, 0x43, 0xd2, 0xb4, 0x52, 0xb9, 0xbe, 0xfb, 0xbd, 0x11, 0x27, 0x86, 0x2a, - 0x63, 0x52, 0x6e, 0x3f, 0x40, 0x46, 0x4b, 0x90, 0x68, 0xa8, 0x85, 0xa9, 0xbe, 0x23, 0xc4, 0x3b, - 0x0f, 0x53, 0x19, 0x93, 0x12, 0x0d, 0x15, 0x7d, 0x0a, 0x69, 0x76, 0xa0, 0xe1, 0xd8, 0x28, 0xe4, - 0xfb, 0xda, 0x89, 0xf0, 0xb1, 0x90, 0xca, 0x98, 0x44, 0xcf, 0x50, 0x90, 0xf7, 0x6d, 0x43, 0xde, - 0x62, 0x11, 0x76, 0x6e, 0x6c, 0xac, 0xd8, 0x77, 0x0f, 0x3c, 0x2a, 0x3c, 0xb6, 0x42, 0xd1, 0x41, - 0x80, 0x8e, 0x7e, 0x00, 0x73, 0x61, 0x89, 0xbc, 0xa7, 0xcd, 0xf4, 0xdd, 0xcf, 0xed, 0x1b, 0xa4, - 0x59, 0x19, 0x93, 0x90, 0xd5, 0x93, 0x88, 0x3e, 0x84, 0x71, 0xd6, 0x6a, 0x88, 0x8a, 0x8c, 0x0a, - 0xee, 0xe8, 0x6a, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, 0x87, 0x96, 0xc9, 0x4d, 0xb3, 0x51, 0x98, - 0xed, 0xdb, 0xf9, 0x7b, 0x43, 0xe5, 0x48, 0xe7, 0x77, 0x7c, 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, - 0x48, 0xa4, 0xb9, 0xbe, 0xed, 0x1e, 0x11, 0x71, 0x56, 0xa1, 0x07, 0x03, 0x7c, 0x32, 0x29, 0x9a, - 0xc5, 0x2e, 0xaf, 0x91, 0xe9, 0x98, 0xba, 0xd4, 0xb7, 0x68, 0xbd, 0x77, 0xfc, 0x54, 0xa8, 0xd7, - 0xe4, 0x51, 0xd1, 0x4b, 0x10, 0xf9, 0xb5, 0x12, 0xfe, 0xae, 0xc4, 0x65, 0x2a, 0xef, 0x5e, 0xa4, - 0xe9, 0x8a, 0x0a, 0xdd, 0xa9, 0x8c, 0x49, 0xd3, 0x6a, 0x38, 0x05, 0x7d, 0x06, 0x33, 0x54, 0x9e, - 0xac, 0xfa, 0xf7, 0x81, 0x14, 0x0a, 0x3d, 0xf7, 0x4a, 0xf4, 0xbf, 0x3a, 0xc4, 0x95, 0x2c, 0xaa, - 0x5d, 0x49, 0xa4, 0x1b, 0xeb, 0x86, 0xee, 0x50, 0x2b, 0x3b, 0xdf, 0xb7, 0x1b, 0x87, 0xef, 0x2e, - 0x24, 0xdd, 0x58, 0x67, 0x14, 0xd2, 0x8d, 0x1d, 0x1e, 0xa6, 0xc6, 0x9b, 0xe3, 0xed, 0xbe, 0xdd, - 0x38, 0x2a, 0x9e, 0x8d, 0x74, 0x63, 0x27, 0x48, 0x27, 0xdd, 0x98, 0x19, 0x88, 0x2e, 0xb9, 0xef, - 0xf4, 0xed, 0xc6, 0x7d, 0xcf, 0x45, 0x93, 0x6e, 0xac, 0xf4, 0x24, 0xa2, 0x35, 0x00, 0xe6, 0xd4, - 0xd0, 0x49, 0xf1, 0x5a, 0xdf, 0xc9, 0xa0, 0x3b, 0x50, 0x8d, 0x4c, 0x06, 0x4d, 0x97, 0x46, 0x0c, - 0x19, 0x85, 0x52, 0x32, 0xdd, 0xa2, 0x2d, 0x2c, 0xf4, 0x35, 0x64, 0x3d, 0x9b, 0xa7, 0xc4, 0x90, - 0x1d, 0x79, 0x44, 0x32, 0xab, 0xb0, 0xf5, 0xe2, 0xc2, 0x62, 0x7f, 0xb3, 0x1c, 0xdc, 0x3c, 0xa2, - 0x66, 0x99, 0x12, 0xd0, 0x0a, 0x64, 0xc8, 0x9c, 0x7f, 0x42, 0xcd, 0xd0, 0xf5, 0xbe, 0xfe, 0x69, - 0xd7, 0xc9, 0x97, 0xca, 0x98, 0x94, 0x7e, 0xcd, 0x49, 0xe4, 0xf5, 0x6c, 0xdd, 0xac, 0x50, 0xec, - 0xfb, 0xfa, 0xd0, 0xaa, 0x2b, 0x79, 0x3d, 0xe3, 0x40, 0x2a, 0x5c, 0x62, 0x6d, 0xc5, 0x8f, 0x25, - 0x5b, 0xfc, 0x0c, 0x6d, 0xe1, 0x06, 0x15, 0xd5, 0x77, 0xe9, 0x29, 0xf2, 0xb4, 0x74, 0x65, 0x4c, - 0x9a, 0x55, 0x7a, 0x53, 0xc9, 0x80, 0xe7, 0x53, 0x0f, 0x5b, 0xb0, 0x2a, 0xdc, 0xec, 0x3b, 0xe0, - 0x23, 0x56, 0xfb, 0xc8, 0x80, 0x57, 0x02, 0x64, 0x36, 0x01, 0x69, 0xb2, 0x6d, 0xb3, 0x0d, 0xfd, - 0x5b, 0x03, 0x26, 0xa0, 0xae, 0x35, 0x02, 0x36, 0x01, 0x69, 0x35, 0xc6, 0x49, 0x04, 0xa9, 0x4d, - 0xac, 0x58, 0xdc, 0xcc, 0xde, 0xee, 0x2b, 0xa8, 0xe7, 0x3a, 0x40, 0x22, 0x48, 0xf5, 0x88, 0xc4, - 0xe1, 0xb1, 0xdc, 0xfb, 0x68, 0xb8, 0xc3, 0x78, 0xa7, 0xaf, 0xc3, 0x13, 0x79, 0x6d, 0x0e, 0x71, - 0x78, 0xac, 0x50, 0x02, 0xfa, 0x65, 0x98, 0xe4, 0x80, 0xae, 0x70, 0x77, 0x80, 0x1b, 0x1b, 0x44, - 0xe2, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0x90, 0x64, 0xd5, 0xbb, 0x37, 0xc0, 0xca, 0xf6, - 0x60, 0x59, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0x7e, 0x5f, 0x2b, - 0xdb, 0x7b, 0xa2, 0x86, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x20, 0xaa, 0xf0, 0xad, - 0xbe, 0x35, 0x0b, 0x63, 0x4a, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0xb9, 0xc4, 0x4c, 0xd3, 0xef, - 0xf6, 0xbf, 0x01, 0xa0, 0x1b, 0x7a, 0x54, 0xdc, 0xc5, 0x4c, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, - 0xbc, 0x33, 0xd7, 0xd4, 0x7b, 0x83, 0x0d, 0x55, 0xd4, 0x51, 0x6e, 0xcf, 0x50, 0x85, 0x12, 0x69, - 0x51, 0xd9, 0x21, 0x36, 0x3a, 0xbe, 0x97, 0x06, 0x5c, 0x56, 0xd0, 0x75, 0xa0, 0x90, 0x16, 0xd5, - 0x23, 0xfa, 0x43, 0xa8, 0xc3, 0x6e, 0xd5, 0x28, 0x2c, 0x0f, 0x1e, 0x42, 0xe1, 0xdb, 0x3d, 0xbc, - 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x7a, 0x18, 0xef, 0x0f, 0x9e, 0x33, 0xbb, 0x5d, 0x0b, 0x36, - 0x67, 0x72, 0x9f, 0xe2, 0x2f, 0x09, 0xb0, 0xc8, 0xca, 0x46, 0xd7, 0xfb, 0x4e, 0x64, 0x6f, 0xed, - 0x34, 0x70, 0x7c, 0xe2, 0x01, 0x7d, 0xc1, 0x87, 0xfd, 0x8a, 0x3b, 0x64, 0x2d, 0xb8, 0x32, 0x26, - 0xbd, 0xa3, 0x0c, 0xca, 0xb7, 0x3a, 0xc9, 0xb7, 0x54, 0xbd, 0x73, 0xa4, 0xd3, 0xa2, 0xb8, 0x9e, - 0x4a, 0x5f, 0x11, 0x0b, 0xeb, 0xa9, 0xf4, 0x55, 0x71, 0x7e, 0x3d, 0x95, 0x7e, 0x4b, 0x7c, 0xbb, - 0xf8, 0x3f, 0xae, 0xc2, 0x94, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0xfa, 0x21, - 0x22, 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0xfa, 0x41, 0x22, 0x9f, 0x87, 0x60, - 0xa2, 0x7a, 0x3f, 0x4c, 0x74, 0x6f, 0x04, 0x4c, 0xe4, 0x89, 0xea, 0x06, 0x45, 0x6b, 0xbd, 0xa0, - 0xe8, 0xe6, 0x60, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x5d, 0xa8, 0xe8, 0xfa, 0x00, 0x54, - 0xe4, 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x1e, 0x06, 0x8b, 0x3c, 0x39, 0x21, 0x5c, - 0xf4, 0x41, 0x08, 0x17, 0x2d, 0xf4, 0xc5, 0x45, 0x1e, 0x37, 0x03, 0x46, 0x9f, 0x74, 0x03, 0xa3, - 0xeb, 0x03, 0x80, 0x91, 0x5f, 0x03, 0x8e, 0x8c, 0x2a, 0x51, 0xc8, 0xe8, 0xd6, 0x10, 0x64, 0xe4, - 0x49, 0x09, 0x42, 0xa3, 0x4a, 0x14, 0x34, 0xba, 0x35, 0x04, 0x1a, 0x75, 0x49, 0x62, 0xd8, 0x68, - 0x2b, 0x1a, 0x1b, 0xdd, 0x19, 0x8a, 0x8d, 0x3c, 0x69, 0x61, 0x70, 0xb4, 0x1c, 0x00, 0x47, 0xef, - 0xf4, 0x01, 0x47, 0x1e, 0x2b, 0x41, 0x47, 0xdf, 0xe9, 0x41, 0x47, 0xc5, 0x41, 0xe8, 0xc8, 0xe3, - 0xf5, 0xe0, 0xd1, 0x8b, 0x3e, 0xf0, 0xe8, 0xee, 0x70, 0x78, 0xe4, 0x09, 0xeb, 0xc2, 0x47, 0xca, - 0x40, 0x7c, 0xf4, 0xde, 0x88, 0xf8, 0xc8, 0x93, 0x1e, 0x05, 0x90, 0x3e, 0x0a, 0x03, 0xa4, 0xc5, - 0xfe, 0x00, 0xc9, 0x13, 0xc3, 0x11, 0xd2, 0x46, 0x24, 0x42, 0xba, 0x3d, 0x0c, 0x21, 0xf9, 0xe3, - 0x20, 0x08, 0x91, 0xb6, 0xa2, 0x21, 0xd2, 0x9d, 0xa1, 0x10, 0xc9, 0x6f, 0xfe, 0x10, 0x46, 0xda, - 0x88, 0xc4, 0x48, 0xb7, 0x87, 0x61, 0x24, 0xbf, 0x70, 0x41, 0x90, 0xf4, 0xaa, 0x2f, 0x48, 0xba, - 0x3f, 0x0a, 0x48, 0xf2, 0x84, 0xf6, 0xa0, 0xa4, 0xcf, 0xfb, 0xa3, 0xa4, 0x6f, 0x9d, 0xe3, 0x82, - 0xc5, 0x48, 0x98, 0xf4, 0x9d, 0x1e, 0x98, 0x54, 0x1c, 0x04, 0x93, 0xfc, 0xfe, 0xec, 0xe2, 0x24, - 0x65, 0x20, 0xaa, 0x79, 0x6f, 0x44, 0x54, 0xe3, 0x77, 0xbe, 0x08, 0x58, 0x53, 0x8e, 0x80, 0x35, - 0x37, 0x07, 0xc3, 0x1a, 0xdf, 0x9c, 0xfb, 0xb8, 0xa6, 0x12, 0x85, 0x6b, 0x6e, 0x0d, 0xc1, 0x35, - 0xbe, 0x15, 0x0a, 0x00, 0x9b, 0xc7, 0x5d, 0xc0, 0xe6, 0xfa, 0xd0, 0x88, 0xa1, 0x00, 0xb2, 0x59, - 0xed, 0x45, 0x36, 0x37, 0x06, 0x22, 0x1b, 0x4f, 0x82, 0x0f, 0x6d, 0x1e, 0x77, 0x41, 0x9b, 0xeb, - 0x03, 0xa0, 0x8d, 0x5f, 0x00, 0x8e, 0x6d, 0xb4, 0xc1, 0xd8, 0x66, 0x69, 0x54, 0x6c, 0xe3, 0x09, - 0x8e, 0x04, 0x37, 0x5b, 0xd1, 0xe0, 0xe6, 0xce, 0x88, 0x9b, 0xf6, 0x3d, 0xe8, 0xa6, 0x12, 0x85, - 0x6e, 0x6e, 0x0d, 0x41, 0x37, 0xc1, 0x39, 0xc4, 0x83, 0x37, 0x95, 0x28, 0x78, 0x73, 0x6b, 0x08, - 0xbc, 0xf1, 0x25, 0x05, 0xf0, 0x4d, 0xbd, 0x1f, 0xbe, 0xb9, 0x37, 0x02, 0xbe, 0xf1, 0x9d, 0x97, - 0x2e, 0x80, 0xf3, 0x69, 0x37, 0xc0, 0x29, 0x0e, 0x02, 0x38, 0xfe, 0x88, 0x74, 0x11, 0xce, 0x56, - 0x34, 0xc2, 0xb9, 0x33, 0x14, 0xe1, 0x04, 0x8d, 0x64, 0x00, 0xe2, 0x6c, 0x44, 0x42, 0x9c, 0xdb, - 0xc3, 0x20, 0x8e, 0x6f, 0x24, 0x83, 0x18, 0xe7, 0xd3, 0x6e, 0x8c, 0x53, 0x1c, 0x84, 0x71, 0xfc, - 0xca, 0xb9, 0x20, 0xa7, 0x12, 0x05, 0x72, 0x6e, 0x0d, 0x01, 0x39, 0x7e, 0xe3, 0x05, 0x50, 0x8e, - 0x32, 0x10, 0xe5, 0xbc, 0x37, 0x22, 0xca, 0xe9, 0x32, 0x5c, 0x61, 0x98, 0x53, 0x89, 0x82, 0x39, - 0xb7, 0x86, 0xc0, 0x9c, 0x40, 0x61, 0x7d, 0x9c, 0xb3, 0x15, 0x8d, 0x73, 0xee, 0x0c, 0xc5, 0x39, - 0x5d, 0xa3, 0xc9, 0x05, 0x3a, 0x1b, 0x91, 0x40, 0xe7, 0xf6, 0x30, 0xa0, 0xd3, 0x35, 0xf1, 0x71, - 0xe7, 0xe0, 0x2f, 0x8f, 0x8e, 0x74, 0x3e, 0x3a, 0x3f, 0xd2, 0xf1, 0xde, 0x19, 0x0b, 0xd4, 0x59, - 0x4f, 0xa5, 0xdf, 0x16, 0xdf, 0x29, 0xfe, 0xb5, 0x49, 0x98, 0xa8, 0x78, 0xb1, 0x30, 0x7e, 0x29, - 0x85, 0x8b, 0x5c, 0xc6, 0x84, 0xd6, 0xc8, 0x88, 0xa5, 0x76, 0x6f, 0xf8, 0xbd, 0x7b, 0xbd, 0x77, - 0xc2, 0x71, 0xd6, 0x0b, 0x9c, 0x6f, 0x46, 0x1f, 0xc0, 0x54, 0xc7, 0xc6, 0x96, 0xdc, 0xb6, 0x74, - 0xd3, 0xd2, 0x1d, 0x76, 0x56, 0x44, 0x58, 0x15, 0xbf, 0x3c, 0x5d, 0xc8, 0xed, 0xd8, 0xd8, 0xda, - 0xe6, 0x74, 0x29, 0xd7, 0x09, 0x3c, 0xb9, 0x5f, 0xc8, 0x1a, 0x1f, 0xfd, 0x0b, 0x59, 0x2f, 0x40, - 0xb4, 0xb0, 0xa2, 0x85, 0x3c, 0x10, 0x76, 0xd9, 0x51, 0x74, 0x9f, 0xa1, 0xc7, 0xb0, 0xdc, 0x9c, - 0xf4, 0xd2, 0xa3, 0x69, 0x2b, 0x4c, 0x44, 0x0f, 0xe0, 0x52, 0x4b, 0x39, 0xa6, 0xf1, 0x94, 0xb2, - 0xeb, 0xd4, 0xd1, 0x18, 0x49, 0xf6, 0xf1, 0x29, 0xd4, 0x52, 0x8e, 0xe9, 0xe7, 0xb6, 0x58, 0x12, - 0xfd, 0x3e, 0xc6, 0x2d, 0xc8, 0x6b, 0xba, 0xed, 0xe8, 0x86, 0xea, 0xf0, 0x6b, 0x72, 0xd9, 0x15, - 0xb3, 0x53, 0x2e, 0x95, 0xdd, 0x85, 0x7b, 0x1f, 0x66, 0x78, 0xb8, 0x7d, 0x60, 0x8b, 0x90, 0x5f, - 0x35, 0xcb, 0x12, 0xbc, 0x5d, 0x41, 0x54, 0x82, 0xe9, 0x86, 0xe2, 0xe0, 0x23, 0xe5, 0x44, 0x76, - 0xcf, 0x6a, 0x65, 0xe9, 0x2d, 0x91, 0x6f, 0x9d, 0x9d, 0x2e, 0x4c, 0x3d, 0x65, 0x49, 0x3d, 0x47, - 0xb6, 0xa6, 0x1a, 0x81, 0x04, 0x0d, 0xdd, 0x81, 0x69, 0xc5, 0x3e, 0x31, 0x54, 0xaa, 0x1e, 0x6c, - 0xd8, 0x1d, 0x9b, 0x42, 0x8a, 0xb4, 0x94, 0xa7, 0xe4, 0x92, 0x4b, 0x45, 0xd7, 0x21, 0xc7, 0x63, - 0xd1, 0xd9, 0x37, 0x7b, 0xa6, 0x69, 0x55, 0xf9, 0x07, 0x20, 0xe8, 0x67, 0x7b, 0xd0, 0x63, 0x98, - 0xe7, 0x17, 0xe3, 0x1f, 0x29, 0x96, 0x26, 0x53, 0xad, 0xfb, 0xfd, 0x53, 0xa4, 0x62, 0xaf, 0xb0, - 0x8b, 0xf0, 0x49, 0x06, 0xa2, 0x6a, 0xff, 0x52, 0x85, 0x2d, 0x98, 0x51, 0x9b, 0xba, 0x87, 0x00, - 0x58, 0xcd, 0x67, 0xfa, 0xda, 0xd9, 0x12, 0xcd, 0xeb, 0x6f, 0x91, 0x4e, 0xab, 0x61, 0x02, 0xaa, - 0x01, 0xbd, 0x44, 0x46, 0x6e, 0x9b, 0x4d, 0x5d, 0x3d, 0xa1, 0xce, 0x7f, 0xf8, 0x52, 0xef, 0x81, - 0xd7, 0xec, 0xbf, 0x52, 0x74, 0x67, 0x9b, 0x72, 0x4a, 0x70, 0xe4, 0xfd, 0x67, 0x97, 0xf0, 0xae, - 0xa7, 0xd2, 0x39, 0x71, 0x6a, 0x3d, 0x95, 0xce, 0x8b, 0xd3, 0xc5, 0xbf, 0x25, 0xc0, 0x74, 0x57, - 0x59, 0x50, 0x05, 0x2e, 0x69, 0xde, 0x50, 0x91, 0xf9, 0x51, 0x26, 0xdd, 0x34, 0xf8, 0x3d, 0xe5, - 0xb3, 0x5f, 0x9e, 0x2e, 0x4c, 0xd3, 0xdc, 0x4f, 0xbd, 0x24, 0x69, 0xce, 0xe7, 0xf0, 0xa9, 0xe8, - 0x23, 0xc8, 0x33, 0xf7, 0xd1, 0xfb, 0x30, 0x1d, 0x8d, 0x2f, 0x5f, 0x9d, 0xf9, 0xf2, 0x74, 0x61, - 0x8a, 0xfa, 0x8c, 0xee, 0x0d, 0xc2, 0xd2, 0x54, 0x33, 0xf8, 0x58, 0xfc, 0x9b, 0x02, 0xe4, 0x42, - 0x87, 0x83, 0x1e, 0x77, 0xed, 0xa0, 0x5f, 0x8d, 0xc6, 0x9d, 0xfd, 0x82, 0xee, 0xd2, 0xbc, 0x9f, - 0xbb, 0x11, 0x8c, 0x0b, 0xfd, 0x71, 0x0b, 0x5d, 0x85, 0x71, 0xc3, 0x36, 0x5c, 0xb6, 0x8f, 0x53, - 0x7f, 0xe7, 0x8b, 0x85, 0xb1, 0xe2, 0x4f, 0x52, 0x30, 0x15, 0x3e, 0x04, 0x54, 0xed, 0x2a, 0x57, - 0xd4, 0xbc, 0x10, 0xe2, 0x58, 0x1a, 0x70, 0x6f, 0x62, 0xc6, 0xff, 0x2c, 0x00, 0x2b, 0xe6, 0xe2, - 0x80, 0x38, 0x81, 0x60, 0x39, 0x7d, 0xc6, 0xf9, 0xbf, 0x92, 0xf4, 0xec, 0xeb, 0x12, 0x8c, 0xd3, - 0xdb, 0x79, 0x78, 0xd1, 0xa2, 0xce, 0x97, 0x97, 0x49, 0xba, 0xc4, 0xb2, 0x11, 0x7b, 0x5c, 0xbf, - 0xd0, 0xe5, 0x78, 0xfe, 0x30, 0x38, 0xff, 0x17, 0x00, 0xf9, 0x15, 0x89, 0xe3, 0xe7, 0xbb, 0x22, - 0x91, 0xed, 0xe8, 0x37, 0x9b, 0x6c, 0xae, 0x63, 0x16, 0x69, 0xa2, 0xe7, 0x10, 0x37, 0x15, 0xc1, - 0x3f, 0xcc, 0xb8, 0x24, 0xf1, 0x0f, 0x33, 0x06, 0xa2, 0x40, 0xf3, 0x9e, 0x08, 0x66, 0xbe, 0xba, - 0x02, 0x52, 0x27, 0x2f, 0x12, 0x90, 0xca, 0x42, 0x99, 0x79, 0x7f, 0xf9, 0x77, 0x02, 0x0f, 0x07, - 0x79, 0x66, 0x9a, 0x07, 0x1d, 0x2f, 0x90, 0x74, 0x3e, 0x78, 0x41, 0x61, 0xfa, 0xcb, 0xd3, 0x85, - 0x94, 0xe4, 0xdd, 0x50, 0x18, 0x65, 0xef, 0x13, 0x5f, 0xcd, 0xde, 0x5f, 0x87, 0x5c, 0xdb, 0xc2, - 0x7b, 0xd8, 0x51, 0xf7, 0x65, 0xa3, 0xd3, 0xe2, 0xe7, 0x50, 0xb2, 0x2e, 0x6d, 0xab, 0xd3, 0x42, - 0xf7, 0x40, 0xf4, 0xb2, 0x70, 0x64, 0xed, 0xde, 0x51, 0xe5, 0xd2, 0x39, 0x0e, 0x2f, 0xfe, 0x89, - 0x00, 0xb3, 0xa1, 0x3a, 0xf1, 0x91, 0xb0, 0x0e, 0x59, 0xdf, 0x08, 0xd8, 0x05, 0xe1, 0x9c, 0x01, - 0x95, 0x41, 0x66, 0x24, 0xc3, 0x65, 0xf7, 0xb5, 0xf4, 0xd2, 0x7b, 0x5f, 0x6c, 0xe2, 0x9c, 0x62, - 0x2f, 0xf9, 0x72, 0xd6, 0x02, 0x2f, 0xf0, 0x86, 0x46, 0x72, 0xa4, 0xa1, 0x51, 0xfc, 0xb1, 0x00, - 0x22, 0x7d, 0xc1, 0x13, 0x8c, 0xb5, 0x58, 0x6c, 0x92, 0x1b, 0xae, 0x9c, 0x18, 0xfd, 0xa4, 0x49, - 0xe8, 0xa3, 0x1d, 0xc9, 0xf0, 0x47, 0x3b, 0x8a, 0x5f, 0x08, 0x90, 0xf7, 0x4a, 0xc8, 0x3e, 0x78, - 0x37, 0xe0, 0x1e, 0xcc, 0x8b, 0x7d, 0xe6, 0xcd, 0xbd, 0xae, 0x63, 0xa4, 0x6f, 0xf0, 0x05, 0xaf, - 0xeb, 0x60, 0x9f, 0x24, 0xfb, 0x7b, 0x6e, 0xcf, 0x21, 0x45, 0x2c, 0xf9, 0x57, 0x31, 0x5c, 0xe0, - 0xd0, 0x8d, 0x44, 0xbf, 0x1b, 0x6a, 0x36, 0x0f, 0xd9, 0x3d, 0x29, 0x23, 0x19, 0x2b, 0xc4, 0x83, - 0xa0, 0x80, 0xaf, 0xc1, 0x69, 0xf5, 0x1a, 0xfd, 0xa2, 0x28, 0xfb, 0x6f, 0x17, 0x9f, 0x04, 0x14, - 0x48, 0x1b, 0x9f, 0x68, 0x69, 0x24, 0x03, 0xea, 0x6a, 0x89, 0xf5, 0x95, 0x3f, 0x0c, 0xb6, 0x44, - 0xf9, 0x90, 0x60, 0xaf, 0x47, 0x90, 0x3c, 0x54, 0x9a, 0x83, 0x82, 0xbf, 0x42, 0x2d, 0x27, 0x91, - 0xdc, 0xe8, 0x49, 0xe8, 0x06, 0x8b, 0x44, 0x7f, 0x9c, 0xd0, 0xab, 0xd2, 0xd0, 0x4d, 0x17, 0x1f, - 0x86, 0xfb, 0xfa, 0xc0, 0xd7, 0x07, 0x3b, 0xfd, 0xc7, 0xa9, 0x9f, 0x7e, 0xb1, 0x20, 0x14, 0x3f, - 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xdb, 0x04, 0x1b, 0xd2, 0x01, 0x44, 0x6c, 0xb7, 0x67, 0xc9, - 0xd2, 0x6d, 0xba, 0x4a, 0x60, 0xb0, 0x11, 0x9f, 0x91, 0xbc, 0xe7, 0xe2, 0xbf, 0x10, 0xe0, 0x4a, - 0x2f, 0x27, 0x53, 0x48, 0xd4, 0xa1, 0xbe, 0x49, 0xd5, 0xf4, 0x6f, 0x74, 0x1b, 0xde, 0xb1, 0xdc, - 0xec, 0xc4, 0x07, 0xe4, 0xef, 0x94, 0x5b, 0x0a, 0x1d, 0xe9, 0xfc, 0xe0, 0x71, 0x9e, 0x93, 0x37, - 0x19, 0xd5, 0x1f, 0xf4, 0xa9, 0xd1, 0x06, 0x7d, 0x1d, 0xa6, 0xd7, 0x4d, 0xdd, 0x20, 0xae, 0xa6, - 0x5b, 0xdf, 0x15, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, 0xee, 0xa9, 0x6e, 0xd6, 0xa6, 0xf3, - 0x51, 0x85, 0x65, 0x39, 0xa4, 0x29, 0xc6, 0xc1, 0x1f, 0x8b, 0x3f, 0x13, 0x40, 0xf4, 0xc5, 0x72, - 0xe3, 0xf9, 0x2e, 0x80, 0xda, 0xec, 0xd8, 0x0e, 0xb6, 0xdc, 0xc3, 0x34, 0x39, 0x16, 0x7c, 0x5d, - 0x62, 0xd4, 0xea, 0x9a, 0x94, 0xe1, 0x19, 0xaa, 0x1a, 0xba, 0x11, 0xbe, 0xde, 0x60, 0x7c, 0x15, - 0xce, 0x7a, 0x2e, 0x35, 0x40, 0xb7, 0x21, 0x6d, 0x3b, 0xa6, 0xe5, 0x81, 0x9a, 0xf1, 0xd5, 0xec, - 0x59, 0xe0, 0xfa, 0x75, 0x7a, 0x6c, 0x97, 0xe4, 0x5b, 0x81, 0x3c, 0x99, 0x8f, 0x0f, 0xb1, 0x57, - 0xa5, 0xd4, 0xf0, 0x2a, 0x31, 0x0e, 0xfe, 0x78, 0xbf, 0x06, 0xb3, 0x11, 0x13, 0x11, 0xca, 0x03, - 0x04, 0xbe, 0xf6, 0xc2, 0xbf, 0x53, 0xbb, 0xb2, 0x26, 0xef, 0x6c, 0x95, 0x9e, 0x6f, 0x6e, 0x56, - 0xeb, 0xf5, 0xf2, 0x9a, 0x28, 0x20, 0x11, 0x72, 0xa1, 0x6f, 0xc5, 0x24, 0xd8, 0x97, 0x6b, 0xef, - 0xff, 0x39, 0x00, 0xff, 0x13, 0x54, 0x44, 0xd6, 0x46, 0xf9, 0x33, 0xf9, 0xe5, 0xca, 0xb3, 0x9d, - 0x72, 0x4d, 0x1c, 0x43, 0x08, 0xf2, 0xab, 0x2b, 0xf5, 0x52, 0x45, 0x96, 0xca, 0xb5, 0xed, 0xe7, - 0x5b, 0xb5, 0xb2, 0xfb, 0xc5, 0xdb, 0xfb, 0x6b, 0x90, 0x0b, 0x5e, 0x07, 0x83, 0x66, 0x61, 0xba, - 0x54, 0x29, 0x97, 0x36, 0xe4, 0x97, 0xd5, 0x15, 0xf9, 0xc5, 0x4e, 0x79, 0xa7, 0x2c, 0x8e, 0xd1, - 0xa2, 0x51, 0xe2, 0x93, 0x9d, 0x67, 0xcf, 0x44, 0x01, 0x4d, 0x43, 0x96, 0x3d, 0xd3, 0xef, 0xca, - 0x88, 0x89, 0xfb, 0x9b, 0x90, 0x0d, 0x5c, 0x1c, 0x4b, 0x5e, 0xb7, 0xbd, 0x53, 0xab, 0xc8, 0xf5, - 0xea, 0x66, 0xb9, 0x56, 0x5f, 0xd9, 0xdc, 0x66, 0x32, 0x28, 0x6d, 0x65, 0xf5, 0xb9, 0x54, 0x17, - 0x05, 0xef, 0xb9, 0xfe, 0x7c, 0xa7, 0x54, 0x71, 0xab, 0x51, 0x4c, 0xa5, 0x93, 0x62, 0xf2, 0xfe, - 0xaf, 0x09, 0x70, 0xa5, 0xcf, 0xa5, 0x28, 0x28, 0x0b, 0x93, 0x3b, 0x06, 0xbd, 0x39, 0x53, 0x1c, - 0x43, 0x53, 0x81, 0x7b, 0x51, 0x44, 0x01, 0xa5, 0xd9, 0x9d, 0x14, 0x62, 0x02, 0x4d, 0x40, 0xa2, - 0xf6, 0x48, 0x4c, 0x92, 0x92, 0x06, 0xae, 0x15, 0x11, 0x53, 0x28, 0xc3, 0x6f, 0x45, 0x10, 0xc7, - 0x51, 0xce, 0xbf, 0x96, 0x40, 0x9c, 0x20, 0xa2, 0xbc, 0x83, 0xfd, 0xe2, 0xe4, 0xfd, 0xeb, 0x10, - 0x38, 0x24, 0x8d, 0x00, 0x26, 0x9e, 0x29, 0x0e, 0xb6, 0x1d, 0x71, 0x0c, 0x4d, 0x42, 0x72, 0xa5, - 0xd9, 0x14, 0x85, 0x87, 0x3f, 0x4f, 0x42, 0xda, 0xfd, 0x72, 0x0a, 0x7a, 0x06, 0xe3, 0x6c, 0xcd, - 0x75, 0xa1, 0xbf, 0x4b, 0x4b, 0x07, 0xc5, 0xfc, 0xe2, 0x30, 0x9f, 0xb7, 0x38, 0x86, 0xfe, 0x3c, - 0x64, 0x03, 0x4e, 0x03, 0xea, 0xbb, 0x6e, 0x14, 0x72, 0x94, 0xe6, 0x6f, 0x0f, 0xcb, 0xe6, 0xc9, - 0x7f, 0x05, 0x19, 0xcf, 0x88, 0xa1, 0x1b, 0x83, 0x4c, 0x9c, 0x2b, 0x7b, 0xb0, 0x1d, 0x24, 0x56, - 0xaa, 0x38, 0xf6, 0xbe, 0x80, 0x2c, 0x40, 0xbd, 0x46, 0x0c, 0x45, 0x6d, 0xc5, 0xf7, 0xb5, 0x92, - 0xf3, 0xf7, 0x47, 0xca, 0xed, 0xbf, 0xf3, 0x05, 0xa4, 0x88, 0x85, 0x40, 0x51, 0x6e, 0x4e, 0x97, - 0x45, 0x9a, 0xbf, 0x31, 0x30, 0x8f, 0xab, 0x9f, 0xd5, 0x7b, 0x3f, 0xfd, 0xe3, 0x6b, 0x63, 0x3f, - 0x3d, 0xbb, 0x26, 0xfc, 0xec, 0xec, 0x9a, 0xf0, 0xf3, 0xb3, 0x6b, 0xc2, 0x7f, 0x39, 0xbb, 0x26, - 0xfc, 0x8d, 0x5f, 0x5c, 0x1b, 0xfb, 0xd9, 0x2f, 0xae, 0x8d, 0xfd, 0xfc, 0x17, 0xd7, 0xc6, 0x3e, - 0x9f, 0xe4, 0xdc, 0xbb, 0x13, 0xf4, 0xf3, 0xe4, 0x8f, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, - 0x89, 0x1a, 0x92, 0x30, 0xc1, 0x7d, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_81c6bdcd9303d014) } + +var fileDescriptor_api_81c6bdcd9303d014 = []byte{ + // 8018 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6f, 0x6c, 0x1b, 0x49, + 0x96, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0x91, 0xa2, 0xa8, 0x92, 0x6c, 0xd3, 0x9a, 0x19, 0xcb, 0xa6, + 0xff, 0x7b, 0x67, 0xa4, 0xb1, 0xbd, 0x93, 0x99, 0x1b, 0xcf, 0xcd, 0x9e, 0x44, 0xd1, 0x26, 0x25, + 0x4b, 0x96, 0x9b, 0x94, 0xbd, 0x33, 0xb7, 0x9b, 0xde, 0x56, 0x77, 0x89, 0xea, 0x15, 0xd9, 0x4d, + 0x77, 0x37, 0xf5, 0xc7, 0x40, 0x80, 0x4b, 0xf2, 0xe1, 0x82, 0x4b, 0x30, 0xc8, 0x87, 0x24, 0x08, + 0x72, 0xb9, 0xec, 0x00, 0x17, 0xe4, 0x02, 0x1c, 0x36, 0xc8, 0xb7, 0x20, 0xc1, 0xe5, 0xcf, 0x87, + 0x0b, 0xb0, 0x39, 0x5c, 0x80, 0x4d, 0x80, 0xe4, 0x16, 0x01, 0x22, 0xe4, 0xb4, 0x40, 0x10, 0xe4, + 0x43, 0x80, 0xdc, 0x87, 0x04, 0x18, 0x20, 0x41, 0x50, 0xff, 0xfa, 0x0f, 0xd9, 0xa4, 0x28, 0x6f, + 0x4f, 0x6e, 0x80, 0xfb, 0x22, 0xb1, 0x5f, 0xd5, 0x7b, 0x5d, 0xf5, 0xaa, 0xea, 0xd5, 0xfb, 0x55, + 0xbd, 0xaa, 0x86, 0x19, 0xdb, 0x52, 0xb5, 0xbd, 0xce, 0xce, 0x92, 0xda, 0x31, 0x16, 0x3b, 0xb6, + 0xe5, 0x5a, 0x68, 0x46, 0xb3, 0xb4, 0x7d, 0x4a, 0x5e, 0xe4, 0x89, 0xf3, 0xf7, 0xf6, 0x0f, 0x96, + 0xf6, 0x0f, 0x1c, 0x6c, 0x1f, 0x60, 0x7b, 0x49, 0xb3, 0x4c, 0xad, 0x6b, 0xdb, 0xd8, 0xd4, 0x8e, + 0x97, 0x5a, 0x96, 0xb6, 0x4f, 0xff, 0x18, 0x66, 0x93, 0xb1, 0xcf, 0x23, 0x21, 0x51, 0x57, 0x5d, + 0x95, 0xd3, 0xe6, 0x04, 0x0d, 0xdb, 0xb6, 0x65, 0x3b, 0x9c, 0x7a, 0x51, 0x50, 0xdb, 0xd8, 0x55, + 0x03, 0xb9, 0xdf, 0x72, 0x5c, 0xcb, 0x56, 0x9b, 0x78, 0x09, 0x9b, 0x4d, 0xc3, 0xc4, 0x24, 0xc3, + 0x81, 0xa6, 0xf1, 0xc4, 0xb7, 0x23, 0x13, 0x1f, 0xf2, 0xd4, 0x62, 0xd7, 0x35, 0x5a, 0x4b, 0x7b, + 0x2d, 0x6d, 0xc9, 0x35, 0xda, 0xd8, 0x71, 0xd5, 0x76, 0x87, 0xa7, 0xdc, 0xa3, 0x29, 0xae, 0xad, + 0x6a, 0x86, 0xd9, 0x14, 0xff, 0x3b, 0x3b, 0x4b, 0x36, 0xd6, 0x2c, 0x5b, 0xc7, 0xba, 0xe2, 0x74, + 0x54, 0x53, 0x14, 0xb7, 0x69, 0x35, 0x2d, 0xfa, 0x73, 0x89, 0xfc, 0x62, 0xd4, 0xd2, 0xef, 0x49, + 0x30, 0x25, 0xe3, 0x57, 0x5d, 0xec, 0xb8, 0x55, 0xac, 0xea, 0xd8, 0x46, 0x97, 0x21, 0xb9, 0x8f, + 0x8f, 0x8b, 0xc9, 0xab, 0xd2, 0x9d, 0xdc, 0xca, 0xe4, 0x57, 0x27, 0x0b, 0xc9, 0x75, 0x7c, 0x2c, + 0x13, 0x1a, 0xba, 0x0a, 0x93, 0xd8, 0xd4, 0x15, 0x92, 0x9c, 0x0a, 0x27, 0x4f, 0x60, 0x53, 0x5f, + 0xc7, 0xc7, 0xe8, 0x7b, 0x90, 0x76, 0x88, 0x34, 0x53, 0xc3, 0xc5, 0xf1, 0xab, 0xd2, 0x9d, 0xf1, + 0x95, 0x5f, 0xf9, 0xea, 0x64, 0xe1, 0x93, 0xa6, 0xe1, 0xee, 0x75, 0x77, 0x16, 0x35, 0xab, 0xbd, + 0xe4, 0xb5, 0x83, 0xbe, 0xe3, 0xff, 0x5e, 0xea, 0xec, 0x37, 0x97, 0x7a, 0x75, 0xb0, 0xd8, 0x38, + 0x32, 0xeb, 0xf8, 0x95, 0xec, 0x49, 0xfc, 0x38, 0xf5, 0xdf, 0xbe, 0x5c, 0x90, 0xd6, 0x52, 0x69, + 0xa9, 0x90, 0x58, 0x4b, 0xa5, 0x13, 0x85, 0x64, 0xe9, 0xb7, 0x93, 0x90, 0x97, 0xb1, 0xd3, 0xb1, + 0x4c, 0x07, 0xf3, 0xf2, 0xbf, 0x0f, 0x49, 0xf7, 0xc8, 0xa4, 0xe5, 0xcf, 0x3e, 0xb8, 0xb2, 0xd8, + 0xd7, 0xee, 0x8b, 0x0d, 0x5b, 0x35, 0x1d, 0x55, 0x73, 0x0d, 0xcb, 0x94, 0x49, 0x56, 0xf4, 0x11, + 0x64, 0x6d, 0xec, 0x74, 0xdb, 0x98, 0xaa, 0x8b, 0x56, 0x2d, 0xfb, 0xe0, 0x52, 0x04, 0x67, 0xbd, + 0xa3, 0x9a, 0x32, 0xb0, 0xbc, 0xe4, 0x37, 0xba, 0x0c, 0x69, 0xb3, 0xdb, 0x26, 0x0a, 0x71, 0x68, + 0x75, 0x93, 0xf2, 0xa4, 0xd9, 0x6d, 0xaf, 0xe3, 0x63, 0x07, 0x7d, 0x17, 0x2e, 0xea, 0xb8, 0x63, + 0x63, 0x4d, 0x75, 0xb1, 0xae, 0xd8, 0xaa, 0xd9, 0xc4, 0x8a, 0x61, 0xee, 0x5a, 0x4e, 0x71, 0xe2, + 0x6a, 0xf2, 0x4e, 0xf6, 0xc1, 0xdb, 0x11, 0xf2, 0x65, 0x92, 0xab, 0x66, 0xee, 0x5a, 0x2b, 0xa9, + 0x9f, 0x9c, 0x2c, 0x8c, 0xc9, 0x73, 0xbe, 0x04, 0x2f, 0xc9, 0x41, 0x75, 0x98, 0xe2, 0xc5, 0xb5, + 0xb1, 0xea, 0x58, 0x66, 0x71, 0xf2, 0xaa, 0x74, 0x27, 0xff, 0x60, 0x31, 0x4a, 0x60, 0x48, 0x35, + 0xe4, 0xb1, 0xdb, 0xc6, 0x32, 0xe5, 0x92, 0x73, 0x76, 0xe0, 0x09, 0xbd, 0x05, 0x19, 0x52, 0x93, + 0x9d, 0x63, 0x17, 0x3b, 0xc5, 0x34, 0xad, 0x0a, 0xa9, 0xda, 0x0a, 0x79, 0x2e, 0x7d, 0x0a, 0xb9, + 0x20, 0x2b, 0x42, 0x90, 0x97, 0x2b, 0xf5, 0xed, 0x8d, 0x8a, 0xb2, 0xbd, 0xb9, 0xbe, 0xf9, 0xec, + 0xe5, 0x66, 0x61, 0x0c, 0xcd, 0x41, 0x81, 0xd3, 0xd6, 0x2b, 0x9f, 0x29, 0x4f, 0x6b, 0x1b, 0xb5, + 0x46, 0x41, 0x9a, 0x4f, 0xfd, 0x95, 0xdf, 0xbe, 0x32, 0x56, 0x7a, 0x01, 0xf0, 0x04, 0xbb, 0xbc, + 0x9b, 0xa1, 0x15, 0x98, 0xd8, 0xa3, 0xe5, 0x29, 0x4a, 0x54, 0xd3, 0x57, 0x23, 0x0b, 0x1e, 0xe8, + 0x92, 0x2b, 0x69, 0xa2, 0x8d, 0x9f, 0x9e, 0x2c, 0x48, 0x32, 0xe7, 0x64, 0x3d, 0xa1, 0xf4, 0x2f, + 0x25, 0xc8, 0x52, 0xc1, 0xac, 0x96, 0xa8, 0xdc, 0x23, 0xf9, 0xda, 0x99, 0x2a, 0xe9, 0x17, 0x8d, + 0x16, 0x61, 0xfc, 0x40, 0x6d, 0x75, 0x71, 0x31, 0x41, 0x65, 0x14, 0x23, 0x64, 0xbc, 0x20, 0xe9, + 0x32, 0xcb, 0x86, 0x1e, 0x41, 0xce, 0x30, 0x5d, 0x6c, 0xba, 0x0a, 0x63, 0x4b, 0x9e, 0xc1, 0x96, + 0x65, 0xb9, 0xe9, 0x43, 0xe9, 0x9f, 0x49, 0x00, 0x5b, 0xdd, 0x38, 0x55, 0x83, 0xbe, 0x3d, 0x62, + 0xf9, 0x79, 0x1f, 0xe3, 0xb5, 0xb8, 0x08, 0x13, 0x86, 0xd9, 0x32, 0x4c, 0x56, 0xfe, 0xb4, 0xcc, + 0x9f, 0xd0, 0x1c, 0x8c, 0xef, 0xb4, 0x0c, 0x53, 0xa7, 0xa3, 0x22, 0x2d, 0xb3, 0x07, 0xae, 0x7e, + 0x19, 0xb2, 0xb4, 0xec, 0x31, 0x6a, 0xbf, 0xf4, 0x87, 0x09, 0xb8, 0x50, 0xb6, 0x4c, 0xdd, 0x20, + 0xc3, 0x53, 0x6d, 0x7d, 0x23, 0x74, 0xb3, 0x06, 0x81, 0x81, 0xa8, 0xe0, 0xa3, 0xce, 0x88, 0x2d, + 0x8d, 0x7c, 0xae, 0xca, 0x51, 0x87, 0xd2, 0xa2, 0xf5, 0x89, 0xbe, 0x0d, 0x97, 0xd4, 0x56, 0xcb, + 0x3a, 0x54, 0x8c, 0x5d, 0x45, 0xb7, 0xb0, 0xa3, 0x98, 0x96, 0xab, 0xe0, 0x23, 0xc3, 0x71, 0xa9, + 0x59, 0x49, 0xcb, 0xb3, 0x34, 0xb9, 0xb6, 0xbb, 0x6a, 0x61, 0x67, 0xd3, 0x72, 0x2b, 0x24, 0x89, + 0x8c, 0x59, 0x52, 0x18, 0x36, 0x66, 0x27, 0x88, 0x41, 0x96, 0xd3, 0xf8, 0xa8, 0x43, 0xc7, 0x2c, + 0x6f, 0xa2, 0xef, 0xc3, 0xc5, 0x5e, 0x6d, 0xc6, 0xd9, 0x5a, 0xff, 0x5e, 0x82, 0x7c, 0xcd, 0x34, + 0xdc, 0x6f, 0x44, 0x33, 0x79, 0xaa, 0x4d, 0x06, 0x55, 0x7b, 0x0f, 0x0a, 0xbb, 0xaa, 0xd1, 0x7a, + 0x66, 0x36, 0xac, 0xf6, 0x8e, 0xe3, 0x5a, 0x26, 0x76, 0xb8, 0xee, 0xfb, 0xe8, 0x5c, 0x67, 0x2f, + 0x60, 0xda, 0xab, 0x53, 0x9c, 0xca, 0x7a, 0x0d, 0x85, 0x9a, 0xa9, 0xd9, 0xb8, 0x8d, 0xcd, 0x58, + 0xb5, 0xf5, 0x36, 0x64, 0x0c, 0x21, 0x97, 0x6a, 0x2c, 0x29, 0xfb, 0x04, 0x5e, 0xa7, 0x2e, 0xcc, + 0x04, 0xde, 0x1d, 0xa7, 0xb9, 0x24, 0x13, 0x07, 0x3e, 0x54, 0xfc, 0xf6, 0x22, 0x13, 0x07, 0x3e, + 0x64, 0xe6, 0xed, 0x33, 0x98, 0x5a, 0xc5, 0x2d, 0xec, 0xe2, 0xf8, 0x6d, 0xff, 0x36, 0xe4, 0x85, + 0xe8, 0x38, 0x1b, 0xe9, 0xb7, 0x24, 0x40, 0x5c, 0x2e, 0x99, 0x71, 0xe3, 0x6c, 0xa7, 0x05, 0xe2, + 0x66, 0xb8, 0x5d, 0xdb, 0x64, 0xfe, 0x02, 0xeb, 0xa5, 0xc0, 0x48, 0xd4, 0x65, 0xf0, 0x6d, 0x70, + 0x2a, 0x68, 0x83, 0x3d, 0xb7, 0x87, 0x38, 0x3c, 0x87, 0x30, 0x1b, 0x2a, 0x5e, 0xbc, 0x4d, 0x99, + 0xa2, 0x25, 0x4b, 0x5c, 0x4d, 0x06, 0x7d, 0x3b, 0x4a, 0x2c, 0xfd, 0x3d, 0x09, 0x66, 0xca, 0x2d, + 0xac, 0xda, 0xb1, 0xeb, 0xe5, 0x3b, 0x90, 0xd6, 0xb1, 0xaa, 0xd3, 0x8a, 0xb3, 0x01, 0xff, 0x4e, + 0x40, 0x0a, 0xf1, 0x70, 0x17, 0xf7, 0x5a, 0xda, 0x62, 0x43, 0xf8, 0xbe, 0x7c, 0xd4, 0x7b, 0x4c, + 0xbc, 0x43, 0x7c, 0x06, 0x28, 0x58, 0xbe, 0x38, 0x3b, 0xc5, 0xdf, 0x97, 0x00, 0xc9, 0xf8, 0x00, + 0xdb, 0x6e, 0xec, 0x95, 0x5f, 0x85, 0xac, 0xab, 0xda, 0x4d, 0xec, 0x2a, 0xc4, 0xb7, 0x3f, 0x4f, + 0xfd, 0x81, 0xf1, 0x11, 0x32, 0xd7, 0xc0, 0xe7, 0x30, 0x1b, 0x2a, 0x65, 0x9c, 0x2a, 0xf8, 0x5f, + 0x12, 0x64, 0xeb, 0x9a, 0x6a, 0xc6, 0x59, 0xf7, 0x4f, 0x21, 0xeb, 0x68, 0xaa, 0xa9, 0xec, 0x5a, + 0x76, 0x5b, 0x75, 0x69, 0xa7, 0xcf, 0x87, 0xea, 0xee, 0xf9, 0xdd, 0x9a, 0x6a, 0x3e, 0xa6, 0x99, + 0x64, 0x70, 0xbc, 0xdf, 0xe8, 0x39, 0x64, 0xf7, 0xf1, 0xb1, 0xc2, 0x91, 0x1a, 0x9d, 0x29, 0xf3, + 0x0f, 0xde, 0x0f, 0xf0, 0xef, 0x1f, 0x2c, 0x0a, 0x80, 0xb7, 0x18, 0x00, 0x78, 0x8b, 0x84, 0x63, + 0xb1, 0xee, 0xda, 0xd8, 0x6c, 0xba, 0x7b, 0x32, 0xec, 0xe3, 0xe3, 0xa7, 0x4c, 0x46, 0x70, 0xa8, + 0xad, 0xa5, 0xd2, 0xc9, 0x42, 0xaa, 0xf4, 0xbf, 0x25, 0xc8, 0xb1, 0x8a, 0xc7, 0x39, 0xd4, 0x3e, + 0x80, 0x94, 0x6d, 0x1d, 0xb2, 0xa1, 0x96, 0x7d, 0xf0, 0x56, 0x84, 0x88, 0x75, 0x7c, 0x1c, 0x9c, + 0xe3, 0x68, 0x76, 0xb4, 0x02, 0xdc, 0x7b, 0x54, 0x28, 0x77, 0x72, 0x54, 0x6e, 0x60, 0x5c, 0x32, + 0x91, 0x71, 0x1b, 0xa6, 0x77, 0x54, 0x57, 0xdb, 0x53, 0x6c, 0x5e, 0x48, 0x32, 0x1f, 0x26, 0xef, + 0xe4, 0xe4, 0x3c, 0x25, 0x8b, 0xa2, 0x3b, 0xa5, 0xff, 0x23, 0x7a, 0xbd, 0x83, 0xff, 0x4c, 0xb6, + 0xfc, 0xff, 0x95, 0xf8, 0x78, 0x12, 0xf5, 0xff, 0xb3, 0xd6, 0x01, 0x7e, 0x94, 0x80, 0x4b, 0xe5, + 0x3d, 0xac, 0xed, 0x97, 0x2d, 0xd3, 0x31, 0x1c, 0x97, 0x68, 0x30, 0xce, 0x5e, 0xf0, 0x16, 0x64, + 0x0e, 0x0d, 0x77, 0x4f, 0xd1, 0x8d, 0xdd, 0x5d, 0x6a, 0xf9, 0xd2, 0x72, 0x9a, 0x10, 0x56, 0x8d, + 0xdd, 0x5d, 0xf4, 0x10, 0x52, 0x6d, 0x4b, 0x67, 0x4e, 0x76, 0xfe, 0xc1, 0x42, 0x84, 0x78, 0x5a, + 0x34, 0xa7, 0xdb, 0xde, 0xb0, 0x74, 0x2c, 0xd3, 0xcc, 0xe8, 0x0a, 0x80, 0x46, 0xa8, 0x1d, 0xcb, + 0x30, 0x5d, 0x3e, 0x8b, 0x06, 0x28, 0xa8, 0x0a, 0x19, 0x17, 0xdb, 0x6d, 0xc3, 0x54, 0x5d, 0x5c, + 0x1c, 0xa7, 0xca, 0xbb, 0x11, 0x59, 0xf0, 0x4e, 0xcb, 0xd0, 0xd4, 0x55, 0xec, 0x68, 0xb6, 0xd1, + 0x71, 0x2d, 0x9b, 0x6b, 0xd1, 0x67, 0xe6, 0x16, 0xf7, 0x8b, 0x14, 0x14, 0xfb, 0x35, 0x14, 0x67, + 0x3f, 0xd9, 0x82, 0x09, 0x82, 0xd3, 0x5b, 0x2e, 0xef, 0x29, 0x0f, 0x06, 0x29, 0x22, 0xa2, 0x04, + 0x14, 0xef, 0xb7, 0x5c, 0x5e, 0x78, 0x2e, 0x67, 0xfe, 0xf7, 0x24, 0x98, 0x60, 0x09, 0xe8, 0x3e, + 0xa4, 0xf9, 0xc2, 0x84, 0x4e, 0xcb, 0x98, 0x5c, 0xb9, 0x78, 0x7a, 0xb2, 0x30, 0xc9, 0xd6, 0x1a, + 0x56, 0xbf, 0xf2, 0x7f, 0xca, 0x93, 0x34, 0x5f, 0x4d, 0x27, 0x6d, 0xe6, 0xb8, 0xaa, 0xed, 0xd2, + 0x45, 0xa0, 0x04, 0xc3, 0x1c, 0x94, 0xb0, 0x8e, 0x8f, 0xd1, 0x1a, 0x4c, 0x38, 0xae, 0xea, 0x76, + 0x1d, 0xde, 0x6a, 0xe7, 0x2a, 0x6c, 0x9d, 0x72, 0xca, 0x5c, 0x02, 0x71, 0x86, 0x74, 0xec, 0xaa, + 0x46, 0x8b, 0x36, 0x63, 0x46, 0xe6, 0x4f, 0xa5, 0xdf, 0x94, 0x60, 0x82, 0x65, 0x45, 0x97, 0x60, + 0x56, 0x5e, 0xde, 0x7c, 0x52, 0x51, 0x6a, 0x9b, 0xab, 0x95, 0x46, 0x45, 0xde, 0xa8, 0x6d, 0x2e, + 0x37, 0x2a, 0x85, 0x31, 0x74, 0x11, 0x90, 0x48, 0x28, 0x3f, 0xdb, 0xac, 0xd7, 0xea, 0x8d, 0xca, + 0x66, 0xa3, 0x20, 0xd1, 0x35, 0x0a, 0x4a, 0x0f, 0x50, 0x13, 0xe8, 0x06, 0x5c, 0xed, 0xa5, 0x2a, + 0xf5, 0xc6, 0x72, 0xa3, 0xae, 0x54, 0xea, 0x8d, 0xda, 0xc6, 0x72, 0xa3, 0xb2, 0x5a, 0x48, 0x0e, + 0xc9, 0x45, 0x5e, 0x22, 0xcb, 0x95, 0x72, 0xa3, 0x90, 0x2a, 0xbd, 0x86, 0x0b, 0x32, 0xd6, 0xac, + 0x76, 0xa7, 0xeb, 0x62, 0x52, 0x4a, 0x27, 0xce, 0xf1, 0x72, 0x09, 0x26, 0x75, 0xfb, 0x58, 0xb1, + 0xbb, 0x26, 0x1f, 0x2d, 0x13, 0xba, 0x7d, 0x2c, 0x77, 0x4d, 0xde, 0x19, 0xff, 0xb1, 0x04, 0x17, + 0x7b, 0x5f, 0x1e, 0x67, 0x57, 0x7c, 0x0e, 0x59, 0x55, 0xd7, 0xb1, 0xae, 0xe8, 0xb8, 0xe5, 0xaa, + 0xdc, 0x55, 0xb9, 0x17, 0x90, 0xc4, 0x17, 0xf0, 0x16, 0xbd, 0x05, 0xbc, 0x8d, 0x17, 0xe5, 0x32, + 0x2d, 0xc8, 0x2a, 0xe1, 0x10, 0xa6, 0x88, 0x0a, 0xa1, 0x94, 0xd2, 0x9f, 0xa4, 0x60, 0xaa, 0x62, + 0xea, 0x8d, 0xa3, 0x58, 0x67, 0x97, 0x8b, 0x30, 0xa1, 0x59, 0xed, 0xb6, 0xe1, 0x0a, 0x35, 0xb1, + 0x27, 0xf4, 0x4b, 0x01, 0x47, 0x33, 0x39, 0x82, 0xa3, 0xe5, 0xbb, 0x98, 0xe8, 0x07, 0x70, 0x89, + 0x58, 0x50, 0xdb, 0x54, 0x5b, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xd1, 0x6c, 0x62, 0x9b, 0x2f, 0x17, + 0xde, 0x89, 0x28, 0x67, 0x8d, 0x73, 0x94, 0x29, 0x43, 0x83, 0xe5, 0x97, 0x2f, 0x18, 0x51, 0x64, + 0xf4, 0x09, 0x00, 0x99, 0x9c, 0xe8, 0x12, 0xa4, 0xc3, 0x6d, 0xd3, 0xa0, 0x35, 0x48, 0x61, 0x8e, + 0x08, 0x03, 0x79, 0x76, 0xd0, 0x12, 0xc1, 0x16, 0xaf, 0xba, 0x86, 0x8d, 0x95, 0xfb, 0x1d, 0x8d, + 0x2e, 0x06, 0xa4, 0x57, 0xf2, 0xa7, 0x27, 0x0b, 0x20, 0x33, 0xf2, 0xfd, 0xad, 0x32, 0xc1, 0x1a, + 0xec, 0x77, 0x47, 0x43, 0x2f, 0xe1, 0x6e, 0x60, 0x4d, 0x83, 0xcc, 0xc5, 0xbc, 0x5a, 0xaa, 0xab, + 0xec, 0x19, 0xcd, 0x3d, 0x6c, 0x2b, 0xde, 0x62, 0x33, 0x5d, 0x0f, 0x4c, 0xcb, 0x37, 0x7c, 0x86, + 0xb2, 0x6a, 0xb2, 0xd2, 0x2f, 0xbb, 0x55, 0x9a, 0xd9, 0xd3, 0x19, 0x51, 0x7e, 0xc7, 0x32, 0x1c, + 0xcb, 0x2c, 0x66, 0x98, 0xf2, 0xd9, 0x13, 0xba, 0x0b, 0x05, 0xf7, 0xc8, 0x54, 0xf6, 0xb0, 0x6a, + 0xbb, 0x3b, 0x58, 0x75, 0xc9, 0xbc, 0x0d, 0x34, 0xc7, 0xb4, 0x7b, 0x64, 0x56, 0x03, 0x64, 0xf4, + 0x1c, 0x0a, 0x86, 0xa9, 0xec, 0xb6, 0x8c, 0xe6, 0x9e, 0xab, 0x1c, 0xda, 0x86, 0x8b, 0x9d, 0xe2, + 0x0c, 0x55, 0x48, 0x54, 0xbf, 0xad, 0xf3, 0xd5, 0x61, 0xfd, 0x25, 0xc9, 0xc9, 0x55, 0x93, 0x37, + 0xcc, 0xc7, 0x94, 0x9f, 0x12, 0x1d, 0x6f, 0x76, 0x9f, 0x2c, 0xa4, 0x4b, 0xff, 0x59, 0x82, 0xbc, + 0xe8, 0x74, 0x71, 0x8e, 0x8f, 0x3b, 0x50, 0xb0, 0x4c, 0xac, 0x74, 0xf6, 0x54, 0x07, 0x73, 0x6d, + 0xf2, 0x29, 0x28, 0x6f, 0x99, 0x78, 0x8b, 0x90, 0x99, 0xd2, 0xd0, 0x16, 0xcc, 0x38, 0xae, 0xda, + 0x34, 0xcc, 0x66, 0x40, 0xc9, 0xe3, 0xa3, 0xbb, 0xfe, 0x05, 0xce, 0xed, 0xd1, 0x43, 0x7e, 0xcb, + 0x1f, 0x49, 0x30, 0xb3, 0xac, 0xb7, 0x0d, 0xb3, 0xde, 0x69, 0x19, 0xb1, 0xae, 0x34, 0xdc, 0x80, + 0x8c, 0x43, 0x64, 0xfa, 0xc6, 0xdf, 0x47, 0x89, 0x69, 0x9a, 0x42, 0x66, 0x81, 0xa7, 0x30, 0x8d, + 0x8f, 0x3a, 0x86, 0xad, 0xba, 0x86, 0x65, 0x32, 0x58, 0x93, 0x1a, 0xbd, 0x6e, 0x79, 0x9f, 0xd7, + 0x87, 0x36, 0xbc, 0x66, 0x9f, 0x01, 0x0a, 0x56, 0x2c, 0x4e, 0x7c, 0xa3, 0xc0, 0x2c, 0x15, 0xbd, + 0x6d, 0x3a, 0x31, 0x6b, 0x8d, 0x5b, 0xe7, 0x5f, 0x85, 0xb9, 0xf0, 0x0b, 0xe2, 0x2c, 0xfd, 0xf7, + 0x79, 0x8b, 0x6f, 0x60, 0xbb, 0xf9, 0x35, 0xac, 0xb5, 0x08, 0xbd, 0x73, 0xf1, 0x71, 0x96, 0xfc, + 0x37, 0x24, 0xb8, 0x4c, 0x65, 0xd3, 0x5d, 0x99, 0x5d, 0x6c, 0x3f, 0xc5, 0xaa, 0x13, 0x2b, 0xc2, + 0xbe, 0x0e, 0x13, 0x0c, 0x29, 0xd3, 0x1e, 0x3b, 0xbe, 0x92, 0x25, 0x7e, 0x4d, 0xdd, 0xb5, 0x6c, + 0xe2, 0xd7, 0xf0, 0x24, 0x5e, 0x4f, 0x15, 0xe6, 0xa3, 0xca, 0x12, 0xf3, 0x52, 0xc2, 0x0c, 0x77, + 0x2f, 0x49, 0x17, 0x2f, 0xef, 0x11, 0xbf, 0x0a, 0x55, 0x20, 0xab, 0xd1, 0x5f, 0x8a, 0x7b, 0xdc, + 0xc1, 0x54, 0x7e, 0x7e, 0x98, 0x67, 0xca, 0xd8, 0x1a, 0xc7, 0x1d, 0x4c, 0xdc, 0x5b, 0xf1, 0x9b, + 0xa8, 0x2b, 0x50, 0xd5, 0xa1, 0xbe, 0x2d, 0x1d, 0x5f, 0x34, 0xaf, 0x70, 0x0f, 0x43, 0x9a, 0xf8, + 0xa7, 0x49, 0xae, 0x0a, 0xf6, 0x26, 0xce, 0x14, 0xab, 0x37, 0xf3, 0x79, 0x68, 0x83, 0x2c, 0x58, + 0xfd, 0xc4, 0x39, 0xaa, 0x1f, 0x58, 0x99, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xac, 0xbd, 0x2b, 0xac, + 0x66, 0x02, 0x2d, 0x9d, 0x47, 0x29, 0x33, 0xbe, 0x14, 0x46, 0x77, 0x50, 0x19, 0xd2, 0xf8, 0xa8, + 0xa3, 0xe8, 0xd8, 0xd1, 0xb8, 0x59, 0x2b, 0x0d, 0xda, 0xc9, 0xeb, 0xc3, 0x0f, 0x93, 0xf8, 0xa8, + 0x43, 0x88, 0x68, 0x9b, 0xcc, 0x70, 0xc2, 0x9d, 0xa0, 0xc5, 0x76, 0xce, 0x86, 0x23, 0x7e, 0x7f, + 0xe1, 0xe2, 0xa6, 0x3d, 0x4f, 0x82, 0x89, 0xe0, 0x6d, 0xf7, 0xa5, 0x04, 0x6f, 0x45, 0xb6, 0x5d, + 0x9c, 0x93, 0xdd, 0x27, 0x90, 0xa2, 0x2a, 0x48, 0x9c, 0x53, 0x05, 0x94, 0xab, 0xf4, 0xbb, 0x62, + 0xd4, 0xcb, 0xb8, 0x65, 0x11, 0xf5, 0x7e, 0x0d, 0xeb, 0x6a, 0x93, 0xa2, 0xd9, 0x13, 0xe7, 0x6e, + 0x76, 0xc1, 0xda, 0x63, 0x16, 0x7a, 0x0a, 0x1b, 0xa7, 0x59, 0xf8, 0x5b, 0x12, 0xcc, 0x7a, 0x3e, + 0x50, 0xcc, 0xee, 0xf0, 0x07, 0x90, 0x34, 0xad, 0xc3, 0xf3, 0x2c, 0x2d, 0x92, 0xfc, 0xfe, 0xb4, + 0x15, 0x2e, 0x57, 0x9c, 0xb5, 0xfe, 0x37, 0x09, 0xc8, 0x3c, 0x29, 0xc7, 0x59, 0xd7, 0x4f, 0xf8, + 0x12, 0x36, 0x1b, 0xea, 0x51, 0xdd, 0xd2, 0x7b, 0xdf, 0xe2, 0x93, 0xf2, 0x3a, 0x3e, 0x16, 0xdd, + 0x92, 0x70, 0xa1, 0x65, 0xc8, 0xb8, 0x7b, 0x36, 0x76, 0xf6, 0xac, 0x96, 0x7e, 0x1e, 0x9f, 0xc5, + 0xe7, 0x9a, 0xdf, 0x87, 0x71, 0x2a, 0x57, 0x84, 0x51, 0x48, 0x11, 0x61, 0x14, 0xe4, 0x35, 0x9e, + 0xdb, 0x97, 0x38, 0xcf, 0x6b, 0x04, 0x81, 0x35, 0x8e, 0xe7, 0x1b, 0x8d, 0x17, 0x26, 0x4a, 0xcf, + 0x01, 0x48, 0xd5, 0xe2, 0x6c, 0x9e, 0xbf, 0x96, 0x84, 0xfc, 0x56, 0xd7, 0xd9, 0x8b, 0xb9, 0x3f, + 0x96, 0x01, 0x3a, 0x5d, 0x87, 0x22, 0x8c, 0x23, 0x93, 0xd7, 0xff, 0x8c, 0x38, 0x0d, 0xa1, 0x00, + 0xc6, 0xd7, 0x38, 0x32, 0x51, 0x95, 0x0b, 0xc1, 0x8a, 0x1f, 0xec, 0x71, 0x7d, 0x18, 0x16, 0x6d, + 0x1c, 0x99, 0x1b, 0xd8, 0x03, 0xa1, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x49, 0xf2, 0xa0, 0xb8, + 0xd6, 0x79, 0x9a, 0x7c, 0x82, 0xf0, 0x34, 0x2c, 0xf4, 0x08, 0x32, 0x8c, 0x9b, 0x4c, 0x5c, 0x13, + 0x74, 0xe2, 0x8a, 0xaa, 0x0b, 0x57, 0x23, 0x9d, 0xb2, 0xd2, 0x94, 0x95, 0x4c, 0x53, 0x73, 0x30, + 0xbe, 0x6b, 0xd9, 0x1a, 0xa6, 0x11, 0x1c, 0x69, 0x99, 0x3d, 0x04, 0x5b, 0x75, 0x2d, 0x95, 0x4e, + 0x17, 0x32, 0x6b, 0xa9, 0x74, 0xa6, 0x00, 0xa5, 0xdf, 0x94, 0x60, 0xda, 0x6b, 0x8e, 0x38, 0x6d, + 0x79, 0x39, 0xa4, 0xcb, 0xf3, 0x37, 0x08, 0x51, 0x63, 0xe9, 0xdf, 0x52, 0xc7, 0x46, 0xb3, 0x0e, + 0x68, 0xfb, 0xc4, 0xd9, 0x5f, 0x1e, 0xb1, 0x80, 0x9e, 0xc4, 0x79, 0xdb, 0x98, 0xc6, 0xf6, 0xdc, + 0x87, 0x39, 0xa3, 0x4d, 0xac, 0xbc, 0xe1, 0xb6, 0x8e, 0x39, 0x2a, 0x73, 0xb1, 0xd8, 0x23, 0x9e, + 0xf5, 0xd3, 0xca, 0x22, 0x89, 0x1b, 0x3e, 0xb6, 0xe7, 0xe3, 0xd7, 0x27, 0x4e, 0x85, 0xd7, 0x60, + 0xca, 0x66, 0xa2, 0x89, 0x77, 0x72, 0x4e, 0x9d, 0xe7, 0x3c, 0x56, 0xa2, 0xf6, 0xdf, 0x49, 0xc0, + 0xf4, 0xf3, 0x2e, 0xb6, 0x8f, 0xbf, 0x49, 0x4a, 0xbf, 0x05, 0xd3, 0x87, 0xaa, 0xe1, 0x2a, 0xbb, + 0x96, 0xad, 0x74, 0x3b, 0xba, 0xea, 0x8a, 0xa8, 0x92, 0x29, 0x42, 0x7e, 0x6c, 0xd9, 0xdb, 0x94, + 0x88, 0x30, 0xa0, 0x7d, 0xd3, 0x3a, 0x34, 0x15, 0x42, 0xa6, 0x68, 0xf8, 0xc8, 0xe4, 0x8b, 0xd1, + 0x2b, 0x1f, 0xfe, 0xa7, 0x93, 0x85, 0x87, 0x23, 0xc5, 0x8d, 0xd1, 0x18, 0xb8, 0x6e, 0xd7, 0xd0, + 0x17, 0xb7, 0xb7, 0x6b, 0xab, 0x72, 0x81, 0x8a, 0x7c, 0xc9, 0x24, 0x36, 0x8e, 0x4c, 0x31, 0x8b, + 0xff, 0x83, 0x04, 0x14, 0x7c, 0x4d, 0xc5, 0xd9, 0x9c, 0x15, 0xc8, 0xbe, 0xea, 0x62, 0xdb, 0x78, + 0x83, 0xc6, 0x04, 0xce, 0x48, 0x0c, 0xd1, 0xe7, 0x90, 0x0b, 0xe9, 0x21, 0xf9, 0x8b, 0xe9, 0x21, + 0x7b, 0xe8, 0xab, 0x00, 0xdd, 0x83, 0x19, 0xf7, 0xc8, 0x54, 0x58, 0x5c, 0x20, 0x8b, 0x2c, 0x11, + 0x61, 0x10, 0xd3, 0x2e, 0xd1, 0x07, 0xa1, 0xd3, 0xa8, 0x12, 0xa7, 0xf4, 0xaf, 0x25, 0x40, 0x54, + 0x51, 0x35, 0xb6, 0x67, 0xf0, 0x4d, 0xe9, 0x55, 0x77, 0xa0, 0x40, 0x23, 0x2d, 0x15, 0x63, 0x57, + 0x69, 0x1b, 0x8e, 0x63, 0x98, 0x4d, 0xde, 0xad, 0xf2, 0x94, 0x5e, 0xdb, 0xdd, 0x60, 0x54, 0xde, + 0xe0, 0x7f, 0x01, 0x66, 0x43, 0xd5, 0x88, 0xb3, 0xc9, 0xaf, 0x41, 0x6e, 0xd7, 0xea, 0x9a, 0xba, + 0xc2, 0xf6, 0x55, 0xf8, 0x42, 0x63, 0x96, 0xd2, 0xd8, 0xfb, 0x4a, 0xff, 0x33, 0x01, 0x73, 0x32, + 0x76, 0xac, 0xd6, 0x01, 0x8e, 0x5f, 0x91, 0x55, 0xe0, 0x3b, 0x3a, 0xca, 0x1b, 0xe9, 0x33, 0xc3, + 0x98, 0xd9, 0xf4, 0x17, 0x5e, 0xb3, 0xbf, 0x31, 0xbc, 0xdf, 0xf6, 0xaf, 0xd2, 0xf3, 0xd5, 0xbe, + 0x54, 0x68, 0xb5, 0xcf, 0x82, 0x69, 0xa3, 0x69, 0x5a, 0xc4, 0xbe, 0x39, 0xf8, 0x95, 0xd9, 0x6d, + 0x0b, 0x7c, 0xb3, 0x38, 0xac, 0x90, 0x35, 0xc6, 0x52, 0xc7, 0xaf, 0x36, 0xbb, 0x6d, 0xea, 0x65, + 0xaf, 0x5c, 0x24, 0xe5, 0x3d, 0x3d, 0x59, 0xc8, 0x87, 0xd2, 0x1c, 0x39, 0x6f, 0x78, 0xcf, 0x44, + 0x3a, 0x6f, 0xf2, 0xef, 0xc1, 0x85, 0x1e, 0x95, 0xc7, 0xe9, 0x0f, 0xfd, 0xab, 0x24, 0x5c, 0x0e, + 0x8b, 0x8f, 0x1b, 0xb5, 0x7c, 0xd3, 0x9b, 0xb5, 0x0a, 0x53, 0x6d, 0xc3, 0x7c, 0xb3, 0x45, 0xcb, + 0x5c, 0xdb, 0x30, 0xfd, 0x65, 0xe2, 0x88, 0x0e, 0x32, 0xf1, 0xff, 0xa1, 0x83, 0xa8, 0x30, 0x1f, + 0xd5, 0x82, 0x71, 0xf6, 0x92, 0x2f, 0x24, 0xc8, 0xc5, 0xbd, 0x0e, 0xf7, 0x66, 0x11, 0x71, 0xbc, + 0xce, 0x0d, 0x98, 0xfa, 0x1a, 0x16, 0xee, 0x7e, 0x47, 0x02, 0xd4, 0xb0, 0xbb, 0x26, 0x01, 0xc4, + 0x4f, 0xad, 0x66, 0x9c, 0x95, 0x9d, 0x83, 0x71, 0xc3, 0xd4, 0xf1, 0x11, 0xad, 0x6c, 0x4a, 0x66, + 0x0f, 0xa1, 0xcd, 0xca, 0xe4, 0x48, 0x9b, 0x95, 0x7e, 0x58, 0x4c, 0xa8, 0xa0, 0x71, 0x6a, 0xe1, + 0x1f, 0x25, 0x60, 0x96, 0x57, 0x27, 0xf6, 0x85, 0xcb, 0x6f, 0xc3, 0x78, 0x8b, 0xc8, 0x1c, 0xd2, + 0xe6, 0xf4, 0x9d, 0xa2, 0xcd, 0x69, 0x66, 0xf4, 0xcb, 0x00, 0x1d, 0x1b, 0x1f, 0x28, 0x8c, 0x35, + 0x39, 0x12, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, 0x2e, 0x4c, 0x93, 0x11, 0xde, 0xb1, 0xad, 0x8e, + 0xe5, 0x10, 0x87, 0xc6, 0x19, 0x0d, 0x15, 0xcd, 0x9c, 0x9e, 0x2c, 0x4c, 0x6d, 0x18, 0xe6, 0x16, + 0x67, 0x6c, 0xd4, 0x65, 0x62, 0x2a, 0xbc, 0x47, 0x31, 0x00, 0xff, 0x83, 0x04, 0x73, 0x5f, 0xdb, + 0x52, 0xef, 0x9f, 0x86, 0xc6, 0xbc, 0x99, 0xa7, 0x40, 0x1f, 0x6b, 0xe6, 0xae, 0x15, 0xff, 0x02, + 0xfc, 0x17, 0x12, 0xcc, 0x04, 0xc4, 0xc7, 0xe9, 0xc9, 0xbc, 0x91, 0xce, 0x4a, 0xbf, 0x4a, 0x7c, + 0x9b, 0x60, 0xb7, 0x8f, 0x73, 0x50, 0xfd, 0xf3, 0x04, 0x5c, 0x2c, 0xb3, 0x6d, 0x6c, 0x11, 0xe3, + 0x11, 0x67, 0x2f, 0x29, 0xc2, 0xe4, 0x01, 0xb6, 0x1d, 0xc3, 0x62, 0x33, 0xec, 0x94, 0x2c, 0x1e, + 0xd1, 0x3c, 0xa4, 0x1d, 0x53, 0xed, 0x38, 0x7b, 0x96, 0xd8, 0xb9, 0xf3, 0x9e, 0xbd, 0x78, 0x94, + 0xf1, 0x37, 0x8f, 0x47, 0x99, 0x18, 0x1e, 0x8f, 0x32, 0xf9, 0x0b, 0xc7, 0xa3, 0xf0, 0x6d, 0xb2, + 0x3f, 0x90, 0xe0, 0x52, 0x9f, 0xfe, 0xe2, 0xec, 0x33, 0x3f, 0x84, 0xac, 0xc6, 0x05, 0x13, 0x6b, + 0xcc, 0x76, 0x02, 0x6b, 0x24, 0xdb, 0x1b, 0x82, 0x95, 0xd3, 0x93, 0x05, 0x10, 0x45, 0xad, 0xad, + 0x72, 0x15, 0x91, 0xdf, 0x7a, 0xe9, 0x3f, 0xe6, 0x60, 0xba, 0x72, 0xc4, 0xd6, 0xb9, 0xeb, 0xcc, + 0x1f, 0x40, 0x8f, 0x21, 0xdd, 0xb1, 0xad, 0x03, 0x43, 0x54, 0x23, 0x1f, 0x0a, 0x43, 0x10, 0xd5, + 0xe8, 0xe1, 0xda, 0xe2, 0x1c, 0xb2, 0xc7, 0x8b, 0x1a, 0x90, 0x79, 0x6a, 0x69, 0x6a, 0xeb, 0xb1, + 0xd1, 0x12, 0xfd, 0xff, 0xfd, 0xb3, 0x05, 0x2d, 0x7a, 0x3c, 0x5b, 0xaa, 0xbb, 0x27, 0x9a, 0xc2, + 0x23, 0xa2, 0x1a, 0xa4, 0xab, 0xae, 0xdb, 0x21, 0x89, 0xdc, 0x9a, 0xdc, 0x1e, 0x41, 0x28, 0x61, + 0x11, 0x91, 0xad, 0x82, 0x1d, 0x35, 0x60, 0xe6, 0x89, 0x65, 0x35, 0x5b, 0xb8, 0xdc, 0xb2, 0xba, + 0x7a, 0xd9, 0x32, 0x77, 0x8d, 0x26, 0xb7, 0xc7, 0xb7, 0x46, 0x90, 0xf9, 0xa4, 0x5c, 0x97, 0xfb, + 0x05, 0xa0, 0x65, 0x48, 0xd7, 0x1f, 0x72, 0x61, 0xcc, 0x81, 0xbb, 0x39, 0x82, 0xb0, 0xfa, 0x43, + 0xd9, 0x63, 0x43, 0x6b, 0x90, 0x5d, 0x7e, 0xdd, 0xb5, 0x31, 0x97, 0x32, 0x31, 0x30, 0x06, 0xa2, + 0x57, 0x0a, 0xe5, 0x92, 0x83, 0xcc, 0xa8, 0x0e, 0xf9, 0x97, 0x96, 0xbd, 0xdf, 0xb2, 0x54, 0x51, + 0xc3, 0x49, 0x2a, 0xee, 0x5b, 0x23, 0x88, 0x13, 0x8c, 0x72, 0x8f, 0x08, 0xf4, 0x3d, 0x98, 0x26, + 0x8d, 0xd1, 0x50, 0x77, 0x5a, 0xa2, 0x90, 0x69, 0x2a, 0xf5, 0xdd, 0x11, 0xa4, 0x7a, 0x9c, 0x62, + 0xa3, 0xa5, 0x47, 0xd4, 0xfc, 0x77, 0x61, 0x2a, 0xd4, 0x09, 0x10, 0x82, 0x54, 0x87, 0xb4, 0xb7, + 0x44, 0x63, 0x95, 0xe8, 0x6f, 0xf4, 0x1e, 0x4c, 0x9a, 0x96, 0x8e, 0xc5, 0x08, 0x99, 0x5a, 0x99, + 0x3b, 0x3d, 0x59, 0x98, 0xd8, 0xb4, 0x74, 0xe6, 0xae, 0xf0, 0x5f, 0xf2, 0x04, 0xc9, 0x24, 0x9c, + 0x95, 0xf9, 0x5b, 0x90, 0x22, 0xad, 0x4f, 0x8c, 0xd4, 0x8e, 0xea, 0xe0, 0x6d, 0xdb, 0xe0, 0x32, + 0xc5, 0x23, 0xcf, 0xf7, 0x33, 0x09, 0x12, 0xf5, 0x87, 0xc4, 0x51, 0xdf, 0xe9, 0x6a, 0xfb, 0xd8, + 0xe5, 0xb9, 0xf8, 0x13, 0x75, 0xe0, 0x6d, 0xbc, 0x6b, 0x30, 0x1f, 0x2a, 0x23, 0xf3, 0x27, 0xf4, + 0x0e, 0x80, 0xaa, 0x69, 0xd8, 0x71, 0x14, 0x71, 0xc6, 0x2f, 0x23, 0x67, 0x18, 0x65, 0x1d, 0x1f, + 0x13, 0x36, 0x07, 0x6b, 0x36, 0x76, 0x45, 0xd0, 0x15, 0x7b, 0x22, 0x6c, 0x2e, 0x6e, 0x77, 0x14, + 0xd7, 0xda, 0xc7, 0x26, 0xed, 0x33, 0x19, 0x62, 0x7c, 0xda, 0x9d, 0x06, 0x21, 0x10, 0xbb, 0x89, + 0x4d, 0xdd, 0x37, 0x72, 0x19, 0xd9, 0x7b, 0x26, 0x22, 0x6d, 0xdc, 0x34, 0xf8, 0x31, 0xb5, 0x8c, + 0xcc, 0x9f, 0x88, 0xc6, 0xd4, 0xae, 0xbb, 0x47, 0x5b, 0x25, 0x23, 0xd3, 0xdf, 0xbc, 0x6a, 0x7f, + 0x47, 0x82, 0xe4, 0x93, 0x72, 0xfd, 0xdc, 0x75, 0x13, 0x12, 0x93, 0xbe, 0x44, 0x1a, 0xeb, 0x68, + 0xb4, 0x5a, 0x86, 0xd9, 0x24, 0x2e, 0xcd, 0x0f, 0xb1, 0x26, 0x6a, 0x96, 0xe7, 0xe4, 0x2d, 0x46, + 0x45, 0x57, 0x21, 0xab, 0xd9, 0x58, 0xc7, 0xa6, 0x6b, 0xa8, 0x2d, 0x87, 0x57, 0x31, 0x48, 0xe2, + 0x85, 0xfb, 0x75, 0x09, 0xc6, 0x69, 0xe7, 0x45, 0x6f, 0x43, 0x46, 0xb3, 0x4c, 0x57, 0x35, 0x4c, + 0x6e, 0x85, 0x32, 0xb2, 0x4f, 0x18, 0x58, 0xc8, 0x6b, 0x90, 0x53, 0x35, 0xcd, 0xea, 0x9a, 0xae, + 0x62, 0xaa, 0x6d, 0xcc, 0x0b, 0x9b, 0xe5, 0xb4, 0x4d, 0xb5, 0x8d, 0xd1, 0x02, 0x88, 0x47, 0xef, + 0xa4, 0x65, 0x46, 0x06, 0x4e, 0x5a, 0xc7, 0xc7, 0xbc, 0x24, 0x7f, 0x20, 0x41, 0x5a, 0x74, 0x7a, + 0x52, 0x98, 0x26, 0x36, 0xb1, 0xad, 0xba, 0x96, 0x57, 0x18, 0x8f, 0xd0, 0x3b, 0xe3, 0x65, 0xfc, + 0x19, 0x6f, 0x0e, 0xc6, 0x5d, 0xd2, 0xaf, 0x79, 0x39, 0xd8, 0x03, 0x5d, 0x97, 0x6e, 0xa9, 0x4d, + 0xb6, 0x14, 0x97, 0x91, 0xd9, 0x03, 0xa9, 0x12, 0x8f, 0xd7, 0x65, 0xda, 0xe1, 0x4f, 0xa4, 0xbc, + 0x2c, 0x9e, 0x74, 0x07, 0x37, 0x0d, 0x93, 0x76, 0x80, 0xa4, 0x0c, 0x94, 0xb4, 0x42, 0x28, 0xe8, + 0x2d, 0xc8, 0xb0, 0x0c, 0xd8, 0xd4, 0x69, 0x2f, 0x48, 0xca, 0x69, 0x4a, 0xa8, 0x88, 0xa3, 0x64, + 0xf3, 0xfb, 0x90, 0xf1, 0xc6, 0x18, 0x69, 0xc8, 0xae, 0xe3, 0x29, 0x95, 0xfe, 0x46, 0xef, 0xc3, + 0xdc, 0xab, 0xae, 0xda, 0x32, 0x76, 0xe9, 0x2a, 0x1b, 0xc9, 0xc6, 0xf4, 0xc7, 0xea, 0x83, 0xbc, + 0x34, 0x2a, 0x81, 0xaa, 0x51, 0x0c, 0xc9, 0xa4, 0x3f, 0x24, 0x83, 0xdb, 0x26, 0xa5, 0x1f, 0x4b, + 0x30, 0xc3, 0x42, 0x86, 0x58, 0xd4, 0x6b, 0x7c, 0x0e, 0xc6, 0xc7, 0x90, 0xd1, 0x55, 0x57, 0x65, + 0xa7, 0x49, 0x13, 0x43, 0x4f, 0x93, 0x7a, 0x67, 0x19, 0x54, 0x57, 0xa5, 0x27, 0x4a, 0x11, 0xa4, + 0xc8, 0x6f, 0x76, 0xfc, 0x56, 0xa6, 0xbf, 0xfd, 0x20, 0x8c, 0x60, 0x71, 0xe3, 0x74, 0xb8, 0x96, + 0xe0, 0x02, 0xd1, 0x7e, 0xc5, 0xd4, 0xec, 0xe3, 0x8e, 0x6b, 0x58, 0xe6, 0x33, 0xfa, 0xd7, 0x41, + 0x85, 0xc0, 0x26, 0x16, 0xdd, 0xbb, 0xe2, 0x65, 0xf9, 0xfd, 0x09, 0x98, 0xaa, 0x1c, 0x75, 0x2c, + 0x3b, 0xd6, 0x45, 0xad, 0x15, 0x98, 0xe4, 0x88, 0x7f, 0xc8, 0xb6, 0x72, 0x8f, 0xad, 0x16, 0x3b, + 0xb6, 0x9c, 0x11, 0xad, 0x00, 0xb0, 0xf8, 0x54, 0x1a, 0x77, 0x94, 0x3c, 0xc7, 0xe6, 0x1a, 0x65, + 0x23, 0x54, 0xb4, 0x09, 0xd9, 0xf6, 0x81, 0xa6, 0x29, 0xbb, 0x46, 0xcb, 0xe5, 0x01, 0x7e, 0xd1, + 0xd1, 0xe9, 0x1b, 0x2f, 0xca, 0xe5, 0xc7, 0x34, 0x13, 0x8b, 0xb5, 0xf3, 0x9f, 0x65, 0x20, 0x12, + 0xd8, 0x6f, 0xf4, 0x2e, 0xf0, 0x53, 0x3e, 0x8a, 0x23, 0x0e, 0xf4, 0xad, 0x4c, 0x9d, 0x9e, 0x2c, + 0x64, 0x64, 0x4a, 0xad, 0xd7, 0x1b, 0x72, 0x86, 0x65, 0xa8, 0x3b, 0x2e, 0xba, 0x0e, 0x53, 0x56, + 0xdb, 0x70, 0x15, 0xe1, 0x03, 0x71, 0xb7, 0x31, 0x47, 0x88, 0xc2, 0x47, 0x42, 0x0d, 0xb8, 0x8d, + 0x4d, 0x3a, 0x0a, 0x48, 0x3d, 0x95, 0x1d, 0xb6, 0x16, 0xe9, 0xb2, 0xf1, 0xae, 0x58, 0x1d, 0xd7, + 0x68, 0x1b, 0xaf, 0xe9, 0xc6, 0x36, 0xdf, 0x5b, 0xba, 0xce, 0xb2, 0x93, 0xfa, 0xad, 0xd0, 0x45, + 0x4a, 0x9e, 0xf7, 0x59, 0x20, 0x2b, 0xfa, 0x75, 0x09, 0x2e, 0x72, 0x45, 0x2a, 0x3b, 0x34, 0xbc, + 0x5e, 0x6d, 0x19, 0xee, 0xb1, 0xb2, 0x7f, 0x50, 0x4c, 0x53, 0xe7, 0xf4, 0x97, 0x22, 0x1b, 0x24, + 0xd0, 0x0f, 0x16, 0x45, 0xb3, 0x1c, 0x3f, 0xe5, 0xcc, 0xeb, 0x07, 0x15, 0xd3, 0xb5, 0x8f, 0x57, + 0x2e, 0x9d, 0x9e, 0x2c, 0xcc, 0xf6, 0xa7, 0xbe, 0x90, 0x67, 0x9d, 0x7e, 0x16, 0x54, 0x05, 0xc0, + 0x5e, 0x6f, 0xa4, 0x91, 0x84, 0xd1, 0xee, 0x45, 0x64, 0xb7, 0x95, 0x03, 0xbc, 0xe8, 0x0e, 0x14, + 0xf8, 0x01, 0x9b, 0x5d, 0xa3, 0x85, 0x15, 0xc7, 0x78, 0x8d, 0x69, 0xdc, 0x61, 0x52, 0xce, 0x33, + 0x3a, 0x11, 0x51, 0x37, 0x5e, 0xe3, 0xf9, 0x1f, 0x42, 0x71, 0x50, 0xe9, 0x83, 0x03, 0x21, 0xc3, + 0x36, 0x71, 0x3f, 0x0a, 0xaf, 0xc8, 0x8c, 0xd0, 0x55, 0xc5, 0xaa, 0x4c, 0xe2, 0x23, 0x61, 0x82, + 0x7e, 0x37, 0x01, 0x53, 0x2b, 0xdd, 0xd6, 0xfe, 0xb3, 0x4e, 0xbd, 0xdb, 0x6e, 0xab, 0xf6, 0x31, + 0x31, 0x95, 0xcc, 0x74, 0x90, 0x62, 0x4a, 0xcc, 0x54, 0x52, 0xdb, 0x60, 0xbc, 0xc6, 0x64, 0x32, + 0x0b, 0x1e, 0x29, 0x67, 0xc7, 0x07, 0x68, 0x4d, 0x02, 0xe7, 0xc4, 0xad, 0x43, 0x07, 0x7d, 0x04, + 0xc5, 0x40, 0x46, 0xba, 0x7c, 0xa2, 0x60, 0xd3, 0xb5, 0x0d, 0xcc, 0x96, 0x03, 0x93, 0x72, 0x20, + 0xf4, 0xa6, 0x46, 0x92, 0x2b, 0x2c, 0x15, 0x35, 0x20, 0x47, 0x32, 0x1e, 0x2b, 0x74, 0xb2, 0x11, + 0x8b, 0xb6, 0xf7, 0x23, 0x2a, 0x17, 0x2a, 0xf7, 0x22, 0xd5, 0x52, 0x99, 0xf2, 0xd0, 0x9f, 0x72, + 0x16, 0xfb, 0x94, 0xf9, 0x4f, 0xa1, 0xd0, 0x9b, 0x21, 0xa8, 0xd1, 0x14, 0xd3, 0xe8, 0x5c, 0x50, + 0xa3, 0xc9, 0x80, 0xb6, 0xd6, 0x52, 0xe9, 0x54, 0x61, 0xbc, 0xf4, 0xc7, 0x49, 0xc8, 0x8b, 0xce, + 0x16, 0x27, 0x9a, 0x59, 0x81, 0x71, 0xd2, 0x35, 0x44, 0xa0, 0xc8, 0xad, 0x21, 0x7d, 0x9c, 0x87, + 0xaa, 0x93, 0x2e, 0x23, 0xf0, 0x30, 0x65, 0x8d, 0xc3, 0xec, 0xcc, 0xff, 0xc5, 0x04, 0xa4, 0x28, + 0x80, 0xb8, 0x0f, 0x29, 0x3a, 0x75, 0x48, 0xa3, 0x4c, 0x1d, 0x34, 0xab, 0x37, 0xd9, 0x25, 0x02, + 0xfe, 0x27, 0x71, 0xe6, 0xf6, 0xd4, 0x0f, 0xee, 0x3f, 0xa0, 0x26, 0x27, 0x27, 0xf3, 0x27, 0xb4, + 0x42, 0x23, 0x98, 0x2c, 0xdb, 0xc5, 0x3a, 0x77, 0xdc, 0xaf, 0x9e, 0xd5, 0xbe, 0x62, 0x9a, 0x12, + 0x7c, 0xe8, 0x32, 0x24, 0x89, 0x2d, 0x9b, 0x64, 0xd1, 0x0d, 0xa7, 0x27, 0x0b, 0x49, 0x62, 0xc5, + 0x08, 0x0d, 0x2d, 0x41, 0x36, 0x6c, 0x38, 0xa4, 0x3b, 0x19, 0x66, 0x1e, 0x03, 0x83, 0x1e, 0x5a, + 0xde, 0x00, 0x63, 0xa0, 0x95, 0xb7, 0xf1, 0xaf, 0x8d, 0xc3, 0x54, 0xad, 0x1d, 0xf7, 0xc4, 0xb2, + 0x1c, 0x6e, 0xe1, 0x28, 0xb4, 0x13, 0x7a, 0x69, 0x44, 0x03, 0x87, 0xe6, 0xf4, 0xe4, 0xf9, 0xe6, + 0xf4, 0x1a, 0x71, 0x81, 0xf9, 0x1d, 0x11, 0xc9, 0x01, 0xc0, 0x26, 0xfc, 0x7e, 0xea, 0xc5, 0xc8, + 0x84, 0xc7, 0x3f, 0xbc, 0x41, 0x23, 0x54, 0x3e, 0xa5, 0x9e, 0x36, 0xeb, 0x65, 0x13, 0xa3, 0xf7, + 0xb2, 0x49, 0x6c, 0xea, 0x74, 0x6a, 0x0b, 0xdb, 0xd5, 0xc9, 0x37, 0xb7, 0xab, 0xf3, 0xaf, 0x79, + 0x67, 0xfd, 0x18, 0x92, 0xba, 0x21, 0x1a, 0x67, 0xf4, 0x09, 0x9b, 0x30, 0x9d, 0xd1, 0x6b, 0x53, + 0xc1, 0x5e, 0x1b, 0x5c, 0xe0, 0x98, 0x7f, 0x06, 0xe0, 0x6b, 0x08, 0x5d, 0x85, 0x09, 0xab, 0xa5, + 0x8b, 0x33, 0x2c, 0x53, 0x2b, 0x99, 0xd3, 0x93, 0x85, 0xf1, 0x67, 0x2d, 0xbd, 0xb6, 0x2a, 0x8f, + 0x5b, 0x2d, 0xbd, 0xa6, 0xd3, 0x6b, 0x3a, 0xf0, 0xa1, 0xe2, 0x05, 0xac, 0xe5, 0xe4, 0x49, 0x13, + 0x1f, 0xae, 0x62, 0x47, 0xeb, 0x09, 0xa4, 0x21, 0x5d, 0xf0, 0x47, 0x12, 0xe4, 0x45, 0x6b, 0xc4, + 0x6b, 0x66, 0xd2, 0x46, 0x9b, 0x0f, 0xbb, 0xe4, 0xf9, 0x86, 0x9d, 0xe0, 0xe3, 0x67, 0x80, 0x7f, + 0x43, 0xe2, 0xc1, 0xca, 0x75, 0x4d, 0x75, 0x89, 0xb3, 0x11, 0xe3, 0x50, 0xb9, 0x0b, 0x05, 0x5b, + 0x35, 0x75, 0xab, 0x6d, 0xbc, 0xc6, 0x6c, 0x45, 0xd4, 0xe1, 0x9b, 0x9b, 0xd3, 0x1e, 0x9d, 0x2e, + 0xf9, 0x89, 0x05, 0xdd, 0xdf, 0x4f, 0xf0, 0xc0, 0x66, 0xaf, 0x30, 0x71, 0x2a, 0xed, 0x07, 0x30, + 0xd3, 0x7b, 0x8b, 0x8a, 0x18, 0xc5, 0xef, 0x45, 0xc8, 0x8b, 0x2a, 0x08, 0x0b, 0x44, 0x14, 0x91, + 0xf3, 0x3d, 0x37, 0xaa, 0x38, 0xa8, 0x0c, 0xd9, 0xe0, 0xe5, 0x2c, 0xc9, 0x91, 0x2f, 0x67, 0x01, + 0xdb, 0xbb, 0x92, 0x65, 0xfe, 0x57, 0x60, 0x9c, 0x26, 0xbf, 0x81, 0xe9, 0xe6, 0x6d, 0xfa, 0x5f, + 0x13, 0x70, 0x83, 0x96, 0xfe, 0x05, 0xb6, 0x8d, 0xdd, 0xe3, 0x2d, 0xdb, 0x72, 0xb1, 0xe6, 0x62, + 0xdd, 0x3f, 0x8c, 0x12, 0xab, 0x3d, 0xcc, 0x74, 0xc4, 0x0b, 0xce, 0x15, 0x80, 0xe6, 0x71, 0xa1, + 0x75, 0x98, 0xe6, 0xc1, 0x04, 0x6a, 0xcb, 0x38, 0xc0, 0x8a, 0xea, 0x9e, 0x67, 0xd6, 0x9b, 0x62, + 0xbc, 0xcb, 0x84, 0x75, 0xd9, 0x45, 0x3a, 0x64, 0xb8, 0x30, 0x43, 0xe7, 0x37, 0x0b, 0x3d, 0xf9, + 0xc5, 0x56, 0x13, 0xd3, 0x2c, 0xa2, 0xa1, 0xb6, 0x2a, 0xa7, 0x99, 0x64, 0x6f, 0x37, 0xe8, 0x8f, + 0x24, 0xb8, 0x79, 0x86, 0xa2, 0xe3, 0xec, 0xc0, 0xf3, 0x90, 0x3e, 0x20, 0x2f, 0x32, 0xb8, 0xa6, + 0xd3, 0xb2, 0xf7, 0x8c, 0x36, 0x60, 0x6a, 0x57, 0x35, 0x5a, 0x7e, 0xc7, 0x1e, 0x1c, 0xb5, 0x18, + 0x1d, 0x4c, 0x9b, 0x63, 0xec, 0xac, 0x27, 0x97, 0x7e, 0x94, 0x80, 0x99, 0x65, 0x5d, 0xaf, 0xd7, + 0xb9, 0x6d, 0x8c, 0xaf, 0xbf, 0x08, 0x50, 0x9a, 0xf0, 0x41, 0x29, 0x7a, 0x0f, 0x90, 0x6e, 0x38, + 0xec, 0xd6, 0x12, 0x67, 0x4f, 0xd5, 0xad, 0x43, 0x3f, 0x1e, 0x63, 0x46, 0xa4, 0xd4, 0x45, 0x02, + 0xaa, 0x03, 0x45, 0x44, 0x8a, 0xe3, 0xaa, 0xde, 0x96, 0xd2, 0xcd, 0x91, 0xce, 0x8e, 0x31, 0xa8, + 0xe4, 0x3d, 0xca, 0x19, 0x22, 0x87, 0xfe, 0x24, 0xbe, 0xbd, 0x41, 0xaa, 0xee, 0x2a, 0xaa, 0x23, + 0x0e, 0x0a, 0xb1, 0xfb, 0x52, 0xf2, 0x8c, 0xbe, 0xec, 0x04, 0xcf, 0xff, 0xb0, 0x73, 0x0c, 0xbe, + 0x82, 0xe2, 0x84, 0xd0, 0xff, 0x50, 0x82, 0xbc, 0x8c, 0x77, 0x6d, 0xec, 0xc4, 0xba, 0x94, 0xf0, + 0x18, 0x72, 0x36, 0x93, 0xaa, 0xec, 0xda, 0x56, 0xfb, 0x3c, 0x63, 0x2c, 0xcb, 0x19, 0x1f, 0xdb, + 0x56, 0x3b, 0x74, 0x85, 0xc4, 0x0b, 0x98, 0xf6, 0x4a, 0x1a, 0xa7, 0x0a, 0x7e, 0x4c, 0xcf, 0x4b, + 0x33, 0xc1, 0x71, 0x07, 0x46, 0x7c, 0x1d, 0x7a, 0xa0, 0x7b, 0x58, 0xc1, 0xe2, 0xc6, 0xa9, 0x8c, + 0xff, 0x2e, 0x41, 0xbe, 0xde, 0xdd, 0x61, 0x97, 0x66, 0xc5, 0xa7, 0x87, 0x0a, 0x64, 0x5a, 0x78, + 0xd7, 0x55, 0xde, 0x28, 0xf6, 0x3e, 0x4d, 0x58, 0xe9, 0xf9, 0x83, 0x27, 0x00, 0x36, 0x3d, 0x5d, + 0x47, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x43, 0x79, 0x7d, 0xf7, 0xa9, 0xf4, 0xe3, 0x04, 0x4c, 0x7b, + 0x95, 0x8d, 0xd3, 0x7a, 0xbe, 0x0c, 0x59, 0x8d, 0xe4, 0x79, 0xac, 0xc6, 0x0c, 0x8f, 0x0b, 0x89, + 0xb6, 0x1c, 0x8b, 0x30, 0x4b, 0x9d, 0x1b, 0x45, 0xed, 0x74, 0x5a, 0x86, 0x00, 0xc9, 0xd4, 0x2e, + 0xa5, 0xe4, 0x19, 0x9a, 0xb4, 0xcc, 0x52, 0x28, 0x3c, 0x26, 0xfd, 0x6f, 0xd7, 0xc6, 0xf8, 0x35, + 0x56, 0x28, 0x5e, 0x3b, 0x4f, 0xdc, 0x4b, 0x96, 0x31, 0xd6, 0x09, 0x1f, 0xef, 0x79, 0xdf, 0x87, + 0x19, 0xaa, 0xd9, 0xb8, 0x4f, 0x08, 0xf3, 0xe6, 0xf8, 0xad, 0x04, 0xa0, 0xa0, 0xfc, 0xaf, 0xaf, + 0x45, 0x12, 0xf1, 0xb5, 0xc8, 0xbb, 0x80, 0x58, 0x2c, 0xa4, 0xa3, 0x74, 0xb0, 0xad, 0x38, 0x58, + 0xb3, 0xf8, 0x55, 0x4e, 0x92, 0x5c, 0xe0, 0x29, 0x5b, 0xd8, 0xae, 0x53, 0x3a, 0x7a, 0x04, 0xe0, + 0x7b, 0x6d, 0x7c, 0x3a, 0x19, 0xea, 0xb4, 0xc9, 0x19, 0xcf, 0x5d, 0x2b, 0x7d, 0x31, 0x0f, 0x39, + 0xae, 0xc9, 0x6d, 0xd3, 0xb0, 0x4c, 0x74, 0x1f, 0x92, 0x4d, 0xbe, 0xcd, 0x90, 0x8d, 0x5c, 0xe8, + 0xf3, 0x6f, 0xaf, 0xab, 0x8e, 0xc9, 0x24, 0x2f, 0x61, 0xe9, 0x74, 0xdd, 0x08, 0xe7, 0xc9, 0x8f, + 0xf8, 0x0e, 0xb2, 0x74, 0xba, 0x2e, 0xaa, 0xc3, 0xb4, 0xe6, 0xdf, 0xc5, 0xa5, 0x10, 0xf6, 0xe4, + 0x40, 0x00, 0x16, 0x79, 0x07, 0x5a, 0x75, 0x4c, 0xce, 0x6b, 0xa1, 0x04, 0x54, 0x0e, 0x5e, 0xfe, + 0x94, 0xea, 0x0b, 0x18, 0xf3, 0x8f, 0x22, 0x87, 0x2f, 0x9e, 0xaa, 0x8e, 0x05, 0xee, 0x88, 0x42, + 0x1f, 0xc3, 0x84, 0x4e, 0x2f, 0x15, 0xe2, 0xfd, 0x3a, 0xaa, 0xeb, 0x85, 0xee, 0x71, 0xaa, 0x8e, + 0xc9, 0x9c, 0x03, 0xad, 0x41, 0x8e, 0xfd, 0x62, 0x4e, 0x0c, 0x47, 0xa5, 0x37, 0x07, 0x4b, 0x08, + 0x4c, 0x0d, 0xd5, 0x31, 0x39, 0xab, 0xfb, 0x54, 0xf4, 0x6d, 0x48, 0x39, 0x9a, 0x2a, 0x70, 0xe9, + 0x95, 0x01, 0xf7, 0x81, 0xf8, 0xcc, 0x34, 0x37, 0x7a, 0xc4, 0x6e, 0xa5, 0x74, 0x8f, 0xc4, 0x42, + 0x61, 0x54, 0xf1, 0x43, 0xa7, 0xcc, 0x49, 0xf1, 0x31, 0x25, 0xa0, 0x27, 0x90, 0x55, 0x89, 0x37, + 0xa8, 0xd0, 0x53, 0x99, 0x74, 0x65, 0x30, 0x7a, 0x0f, 0xbe, 0xef, 0x44, 0x6d, 0x95, 0x1e, 0x65, + 0x17, 0x44, 0x5f, 0x50, 0x1b, 0xdb, 0x4d, 0x5c, 0xcc, 0x0e, 0x17, 0x14, 0x0c, 0x10, 0xf3, 0x04, + 0x51, 0x22, 0xf1, 0x0a, 0xbd, 0x43, 0xd2, 0xb4, 0x52, 0xb9, 0x81, 0xfb, 0xbd, 0x11, 0x27, 0x86, + 0xaa, 0x63, 0x72, 0x6e, 0x2f, 0x40, 0x46, 0x8b, 0x90, 0x68, 0x6a, 0xc5, 0xa9, 0x81, 0x23, 0xc4, + 0x3b, 0x0f, 0x53, 0x1d, 0x93, 0x13, 0x4d, 0x0d, 0x7d, 0x0a, 0x69, 0x76, 0xa0, 0xe1, 0xc8, 0x2c, + 0xe6, 0x07, 0xda, 0x89, 0xf0, 0xb1, 0x90, 0xea, 0x98, 0x4c, 0xcf, 0x50, 0x90, 0xf7, 0x6d, 0x41, + 0xde, 0x66, 0x11, 0x76, 0x22, 0x36, 0xb6, 0x30, 0x70, 0x0f, 0x3c, 0x2a, 0x3c, 0xb6, 0x4a, 0xd1, + 0x41, 0x80, 0x8e, 0x7e, 0x00, 0x73, 0x61, 0x89, 0xbc, 0xa7, 0xcd, 0x0c, 0xdc, 0xcf, 0x1d, 0x18, + 0xa4, 0x59, 0x1d, 0x93, 0x91, 0xdd, 0x97, 0x88, 0x3e, 0x84, 0x71, 0xd6, 0x6a, 0x88, 0x8a, 0x8c, + 0x0a, 0xee, 0xe8, 0x69, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x97, 0x87, 0x96, 0x29, 0x2d, 0xab, 0x59, + 0x9c, 0x1d, 0xd8, 0xf9, 0xfb, 0x43, 0xe5, 0x48, 0xe7, 0x77, 0x7d, 0x2a, 0x69, 0x77, 0x9b, 0xa5, + 0xf0, 0x48, 0xa4, 0xb9, 0x81, 0xed, 0x1e, 0x11, 0x71, 0x56, 0xa5, 0x07, 0x03, 0x7c, 0x32, 0x29, + 0x9a, 0xcd, 0x2e, 0xaf, 0x51, 0xe8, 0x98, 0xba, 0x30, 0xb0, 0x68, 0xfd, 0x77, 0xfc, 0x54, 0xa9, + 0xd7, 0xe4, 0x51, 0xd1, 0x0b, 0x28, 0xf0, 0x6b, 0x25, 0xfc, 0x5d, 0x89, 0x8b, 0x54, 0xde, 0xdd, + 0x48, 0xd3, 0x15, 0x15, 0xba, 0x53, 0x1d, 0x93, 0xa7, 0xb5, 0x70, 0x0a, 0xfa, 0x0c, 0x66, 0xa8, + 0x3c, 0x45, 0xf3, 0xef, 0x03, 0x29, 0x16, 0xfb, 0xee, 0x95, 0x18, 0x7c, 0x75, 0x88, 0x90, 0x5c, + 0xd0, 0x7a, 0x92, 0x48, 0x37, 0x36, 0x4c, 0xc3, 0xa5, 0x56, 0x76, 0x7e, 0x60, 0x37, 0x0e, 0xdf, + 0x5d, 0x48, 0xba, 0xb1, 0xc1, 0x28, 0xa4, 0x1b, 0xbb, 0x3c, 0x4c, 0x8d, 0x37, 0xc7, 0xdb, 0x03, + 0xbb, 0x71, 0x54, 0x3c, 0x1b, 0xe9, 0xc6, 0x6e, 0x90, 0x4e, 0xba, 0x31, 0x33, 0x10, 0x3d, 0x72, + 0xdf, 0x19, 0xd8, 0x8d, 0x07, 0x9e, 0x8b, 0x26, 0xdd, 0x58, 0xed, 0x4b, 0x44, 0xab, 0x00, 0xcc, + 0xa9, 0xa1, 0x93, 0xe2, 0x95, 0x81, 0x93, 0x41, 0x6f, 0xa0, 0x1a, 0x99, 0x0c, 0x5a, 0x82, 0x46, + 0x0c, 0x19, 0x85, 0x52, 0x0a, 0xdd, 0xa2, 0x2d, 0x2e, 0x0c, 0x34, 0x64, 0x7d, 0x9b, 0xa7, 0xc4, + 0x90, 0x1d, 0x7a, 0x44, 0x32, 0xab, 0xb0, 0xf5, 0xe2, 0xe2, 0xd5, 0xc1, 0x66, 0x39, 0xb8, 0x79, + 0x44, 0xcd, 0x32, 0x25, 0xa0, 0x65, 0xc8, 0x90, 0x39, 0xff, 0x98, 0x9a, 0xa1, 0x6b, 0x03, 0xfd, + 0xd3, 0x9e, 0x93, 0x2f, 0xd5, 0x31, 0x39, 0xfd, 0x8a, 0x93, 0xc8, 0xeb, 0xd9, 0xba, 0x59, 0xb1, + 0x34, 0xf0, 0xf5, 0xa1, 0x55, 0x57, 0xf2, 0x7a, 0xc6, 0x81, 0x34, 0xb8, 0xc0, 0xda, 0x8a, 0x1f, + 0x4b, 0xb6, 0xf9, 0x19, 0xda, 0xe2, 0x75, 0x2a, 0x6a, 0xe0, 0xd2, 0x53, 0xe4, 0x69, 0xe9, 0xea, + 0x98, 0x3c, 0xab, 0xf6, 0xa7, 0x92, 0x01, 0xcf, 0xa7, 0x1e, 0xb6, 0x60, 0x55, 0xbc, 0x31, 0x70, + 0xc0, 0x47, 0xac, 0xf6, 0x91, 0x01, 0xaf, 0x06, 0xc8, 0x6c, 0x02, 0xd2, 0x15, 0xc7, 0x61, 0x1b, + 0xfa, 0x37, 0x87, 0x4c, 0x40, 0x3d, 0x6b, 0x04, 0x6c, 0x02, 0xd2, 0xeb, 0x8c, 0x93, 0x08, 0xd2, + 0x5a, 0x58, 0xb5, 0xb9, 0x99, 0xbd, 0x35, 0x50, 0x50, 0xdf, 0x75, 0x80, 0x44, 0x90, 0xe6, 0x11, + 0x89, 0xc3, 0x63, 0x8b, 0xfb, 0x68, 0xb8, 0xc3, 0x78, 0x7b, 0xa0, 0xc3, 0x13, 0x79, 0x6d, 0x0e, + 0x71, 0x78, 0xec, 0x50, 0x02, 0xfa, 0x65, 0x98, 0xe4, 0x80, 0xae, 0x78, 0x67, 0x88, 0x1b, 0x1b, + 0x44, 0xe2, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0x90, 0x64, 0xd5, 0xbb, 0x3b, 0xc4, 0xca, + 0xf6, 0x61, 0x59, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0xde, 0x40, + 0x2b, 0xdb, 0x7f, 0xa2, 0x86, 0x58, 0xd9, 0x57, 0x3e, 0x95, 0xd4, 0xcc, 0x61, 0x20, 0xaa, 0xf8, + 0xad, 0x81, 0x35, 0x0b, 0x63, 0x4a, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0xb9, 0xc4, 0x4c, 0xd3, + 0xef, 0x0e, 0xbe, 0x01, 0xa0, 0x17, 0x7a, 0x54, 0xc5, 0x62, 0x26, 0xd3, 0xb0, 0x67, 0xa8, 0x6c, + 0x7e, 0xde, 0x99, 0x6b, 0xea, 0xbd, 0xe1, 0x86, 0x2a, 0xea, 0x28, 0xb7, 0x67, 0xa8, 0x42, 0x89, + 0xb4, 0xa8, 0xec, 0x10, 0x1b, 0x1d, 0xdf, 0x8b, 0x43, 0x2e, 0x2b, 0xe8, 0x39, 0x50, 0x48, 0x8b, + 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x65, 0xb7, 0x6a, 0x14, 0x97, 0x86, 0x0f, 0xa1, 0xf0, 0xed, 0x1e, + 0xde, 0x10, 0xe2, 0x64, 0x6f, 0xce, 0x14, 0x1e, 0xc6, 0xfb, 0xc3, 0xe7, 0xcc, 0x5e, 0xd7, 0x82, + 0xcd, 0x99, 0xdc, 0xa7, 0xf8, 0x4b, 0x12, 0x5c, 0x65, 0x65, 0xa3, 0xeb, 0x7d, 0xc7, 0x8a, 0xb7, + 0x76, 0x1a, 0x38, 0x3e, 0x71, 0x9f, 0xbe, 0xe0, 0xc3, 0x41, 0xc5, 0x3d, 0x63, 0x2d, 0xb8, 0x3a, + 0x26, 0xbf, 0xa3, 0x0e, 0xcb, 0xb7, 0x32, 0xc9, 0xb7, 0x54, 0xbd, 0x73, 0xa4, 0xd3, 0x85, 0xc2, + 0x5a, 0x2a, 0x7d, 0xa9, 0x50, 0x5c, 0x4b, 0xa5, 0x2f, 0x17, 0xe6, 0xd7, 0x52, 0xe9, 0xb7, 0x0a, + 0x6f, 0x97, 0xfe, 0xc7, 0x65, 0x98, 0x12, 0xc8, 0x8f, 0x21, 0xa2, 0x07, 0x41, 0x44, 0x74, 0x65, + 0x10, 0x22, 0xe2, 0x58, 0x91, 0x43, 0xa2, 0x07, 0x41, 0x48, 0x74, 0x65, 0x10, 0x24, 0xf2, 0x79, + 0x08, 0x26, 0x6a, 0x0c, 0xc2, 0x44, 0x77, 0x47, 0xc0, 0x44, 0x9e, 0xa8, 0x5e, 0x50, 0xb4, 0xda, + 0x0f, 0x8a, 0x6e, 0x0c, 0x07, 0x45, 0x9e, 0xa8, 0x00, 0x2a, 0x7a, 0xd4, 0x83, 0x8a, 0xae, 0x0d, + 0x41, 0x45, 0x1e, 0xbf, 0x80, 0x45, 0xeb, 0x91, 0xb0, 0xe8, 0xd6, 0x59, 0xb0, 0xc8, 0x93, 0x13, + 0xc2, 0x45, 0x1f, 0x84, 0x70, 0xd1, 0xc2, 0x40, 0x5c, 0xe4, 0x71, 0x33, 0x60, 0xf4, 0x49, 0x2f, + 0x30, 0xba, 0x36, 0x04, 0x18, 0xf9, 0x35, 0xe0, 0xc8, 0xa8, 0x1a, 0x85, 0x8c, 0x6e, 0x9e, 0x81, + 0x8c, 0x3c, 0x29, 0x41, 0x68, 0x54, 0x8d, 0x82, 0x46, 0x37, 0xcf, 0x80, 0x46, 0x3d, 0x92, 0x18, + 0x36, 0xda, 0x8c, 0xc6, 0x46, 0xb7, 0xcf, 0xc4, 0x46, 0x9e, 0xb4, 0x30, 0x38, 0x5a, 0x0a, 0x80, + 0xa3, 0x77, 0x06, 0x80, 0x23, 0x8f, 0x95, 0xa0, 0xa3, 0xef, 0xf4, 0xa1, 0xa3, 0xd2, 0x30, 0x74, + 0xe4, 0xf1, 0x7a, 0xf0, 0xe8, 0xf9, 0x00, 0x78, 0x74, 0xe7, 0x6c, 0x78, 0xe4, 0x09, 0xeb, 0xc1, + 0x47, 0xea, 0x50, 0x7c, 0xf4, 0xde, 0x88, 0xf8, 0xc8, 0x93, 0x1e, 0x05, 0x90, 0x3e, 0x0a, 0x03, + 0xa4, 0xab, 0x83, 0x01, 0x92, 0x27, 0x86, 0x23, 0xa4, 0xf5, 0x48, 0x84, 0x74, 0xeb, 0x2c, 0x84, + 0xe4, 0x8f, 0x83, 0x20, 0x44, 0xda, 0x8c, 0x86, 0x48, 0xb7, 0xcf, 0x84, 0x48, 0x7e, 0xf3, 0x87, + 0x30, 0xd2, 0x7a, 0x24, 0x46, 0xba, 0x75, 0x16, 0x46, 0xf2, 0x0b, 0x17, 0x04, 0x49, 0x2f, 0x07, + 0x82, 0xa4, 0x7b, 0xa3, 0x80, 0x24, 0x4f, 0x68, 0x1f, 0x4a, 0xfa, 0x7c, 0x30, 0x4a, 0xfa, 0xd6, + 0x39, 0x2e, 0x58, 0x8c, 0x84, 0x49, 0xdf, 0xe9, 0x83, 0x49, 0xa5, 0x61, 0x30, 0xc9, 0xef, 0xcf, + 0x02, 0x27, 0xa9, 0x43, 0x51, 0xcd, 0x7b, 0x23, 0xa2, 0x1a, 0xbf, 0xf3, 0x45, 0xc0, 0x9a, 0x4a, + 0x04, 0xac, 0xb9, 0x31, 0x1c, 0xd6, 0xf8, 0xe6, 0xdc, 0xc7, 0x35, 0xd5, 0x28, 0x5c, 0x73, 0xf3, + 0x0c, 0x5c, 0xe3, 0x5b, 0xa1, 0x00, 0xb0, 0x79, 0xd4, 0x03, 0x6c, 0xae, 0x9d, 0x19, 0x31, 0x14, + 0x40, 0x36, 0x2b, 0xfd, 0xc8, 0xe6, 0xfa, 0x50, 0x64, 0xe3, 0x49, 0xf0, 0xa1, 0xcd, 0xa3, 0x1e, + 0x68, 0x73, 0x6d, 0x08, 0xb4, 0xf1, 0x0b, 0xc0, 0xb1, 0x8d, 0x3e, 0x1c, 0xdb, 0x2c, 0x8e, 0x8a, + 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x33, 0x1a, 0xdc, 0xdc, 0x1e, 0x71, 0xd3, 0xbe, 0x0f, 0xdd, + 0x54, 0xa3, 0xd0, 0xcd, 0xcd, 0x33, 0xd0, 0x4d, 0x70, 0x0e, 0xf1, 0xe0, 0x4d, 0x35, 0x0a, 0xde, + 0xdc, 0x3c, 0x03, 0xde, 0xf8, 0x92, 0x02, 0xf8, 0xa6, 0x31, 0x08, 0xdf, 0xdc, 0x1d, 0x01, 0xdf, + 0xf8, 0xce, 0x4b, 0x0f, 0xc0, 0xf9, 0xb4, 0x17, 0xe0, 0x94, 0x86, 0x01, 0x1c, 0x7f, 0x44, 0x0a, + 0x84, 0xb3, 0x19, 0x8d, 0x70, 0x6e, 0x9f, 0x89, 0x70, 0x82, 0x46, 0x32, 0x00, 0x71, 0xd6, 0x23, + 0x21, 0xce, 0xad, 0xb3, 0x20, 0x8e, 0x6f, 0x24, 0x83, 0x18, 0xe7, 0xd3, 0x5e, 0x8c, 0x53, 0x1a, + 0x86, 0x71, 0xfc, 0xca, 0x09, 0x90, 0x53, 0x8d, 0x02, 0x39, 0x37, 0xcf, 0x00, 0x39, 0x7e, 0xe3, + 0x05, 0x50, 0x8e, 0x3a, 0x14, 0xe5, 0xbc, 0x37, 0x22, 0xca, 0xe9, 0x31, 0x5c, 0x61, 0x98, 0x53, + 0x8d, 0x82, 0x39, 0x37, 0xcf, 0x80, 0x39, 0x81, 0xc2, 0xfa, 0x38, 0x67, 0x33, 0x1a, 0xe7, 0xdc, + 0x3e, 0x13, 0xe7, 0xf4, 0x8c, 0x26, 0x01, 0x74, 0xd6, 0x23, 0x81, 0xce, 0xad, 0xb3, 0x80, 0x4e, + 0xcf, 0xc4, 0xc7, 0x9d, 0x83, 0xbf, 0x3c, 0x3a, 0xd2, 0xf9, 0xe8, 0xfc, 0x48, 0xc7, 0x7b, 0x67, + 0x2c, 0x50, 0x67, 0x2d, 0x95, 0x7e, 0xbb, 0xf0, 0x4e, 0xe9, 0xaf, 0x4e, 0xc2, 0x44, 0xd5, 0x8b, + 0x85, 0xf1, 0x4b, 0x29, 0xbd, 0xc9, 0x65, 0x4c, 0x68, 0x95, 0x8c, 0x58, 0x6a, 0xf7, 0xce, 0xbe, + 0x77, 0xaf, 0xff, 0x4e, 0x38, 0xce, 0xfa, 0x06, 0xe7, 0x9b, 0xd1, 0x07, 0x30, 0xd5, 0x75, 0xb0, + 0xad, 0x74, 0x6c, 0xc3, 0xb2, 0x0d, 0x97, 0x9d, 0x15, 0x91, 0x56, 0x0a, 0x5f, 0x9d, 0x2c, 0xe4, + 0xb6, 0x1d, 0x6c, 0x6f, 0x71, 0xba, 0x9c, 0xeb, 0x06, 0x9e, 0xc4, 0x17, 0xb2, 0xc6, 0x47, 0xff, + 0x42, 0xd6, 0x73, 0x28, 0xd8, 0x58, 0xd5, 0x43, 0x1e, 0x08, 0xbb, 0xec, 0x28, 0xba, 0xcf, 0xd0, + 0x63, 0x58, 0x22, 0x27, 0xbd, 0xf4, 0x68, 0xda, 0x0e, 0x13, 0xd1, 0x7d, 0xb8, 0xd0, 0x56, 0x8f, + 0x68, 0x3c, 0xa5, 0x22, 0x9c, 0x3a, 0x1a, 0x23, 0xc9, 0x3e, 0x3e, 0x85, 0xda, 0xea, 0x11, 0xfd, + 0xdc, 0x16, 0x4b, 0xa2, 0xdf, 0xc7, 0xb8, 0x09, 0x79, 0xdd, 0x70, 0x5c, 0xc3, 0xd4, 0x5c, 0x7e, + 0x4d, 0x2e, 0xbb, 0x62, 0x76, 0x4a, 0x50, 0xd9, 0x5d, 0xb8, 0xf7, 0x60, 0x86, 0x87, 0xdb, 0x07, + 0xb6, 0x08, 0xf9, 0x55, 0xb3, 0x2c, 0xc1, 0xdb, 0x15, 0x44, 0x65, 0x98, 0x6e, 0xaa, 0x2e, 0x3e, + 0x54, 0x8f, 0x15, 0x71, 0x56, 0x2b, 0x4b, 0x6f, 0x89, 0x7c, 0xeb, 0xf4, 0x64, 0x61, 0xea, 0x09, + 0x4b, 0xea, 0x3b, 0xb2, 0x35, 0xd5, 0x0c, 0x24, 0xe8, 0xe8, 0x36, 0x4c, 0xab, 0xce, 0xb1, 0xa9, + 0x51, 0xf5, 0x60, 0xd3, 0xe9, 0x3a, 0x14, 0x52, 0xa4, 0xe5, 0x3c, 0x25, 0x97, 0x05, 0x15, 0x5d, + 0x83, 0x1c, 0x8f, 0x45, 0x67, 0xdf, 0xec, 0x99, 0xa6, 0x55, 0xe5, 0x1f, 0x80, 0xa0, 0x9f, 0xed, + 0x41, 0x8f, 0x60, 0x9e, 0x5f, 0x8c, 0x7f, 0xa8, 0xda, 0xba, 0x42, 0xb5, 0xee, 0xf7, 0xcf, 0x02, + 0x15, 0x7b, 0x89, 0x5d, 0x84, 0x4f, 0x32, 0x10, 0x55, 0xfb, 0x97, 0x2a, 0x6c, 0xc2, 0x8c, 0xd6, + 0x32, 0x3c, 0x04, 0xc0, 0x6a, 0x3e, 0x33, 0xd0, 0xce, 0x96, 0x69, 0x5e, 0x7f, 0x8b, 0x74, 0x5a, + 0x0b, 0x13, 0x50, 0x1d, 0xe8, 0x25, 0x32, 0x4a, 0xc7, 0x6a, 0x19, 0xda, 0x31, 0x75, 0xfe, 0xc3, + 0x97, 0x7a, 0x0f, 0xbd, 0x66, 0xff, 0xa5, 0x6a, 0xb8, 0x5b, 0x94, 0x53, 0x86, 0x43, 0xef, 0x37, + 0xbb, 0x84, 0x77, 0x2d, 0x95, 0xce, 0x15, 0xa6, 0xd6, 0x52, 0xe9, 0x7c, 0x61, 0xba, 0xf4, 0x37, + 0x25, 0x98, 0xee, 0x29, 0x0b, 0xaa, 0xc2, 0x05, 0xdd, 0x1b, 0x2a, 0x0a, 0x3f, 0xca, 0x64, 0x58, + 0x26, 0xbf, 0xa7, 0x7c, 0xf6, 0xab, 0x93, 0x85, 0x69, 0x9a, 0xfb, 0x89, 0x97, 0x24, 0xcf, 0xf9, + 0x1c, 0x3e, 0x15, 0x7d, 0x04, 0x79, 0xe6, 0x3e, 0x7a, 0x1f, 0xa6, 0xa3, 0xf1, 0xe5, 0x2b, 0x33, + 0x5f, 0x9d, 0x2c, 0x4c, 0x51, 0x9f, 0x51, 0xdc, 0x20, 0x2c, 0x4f, 0xb5, 0x82, 0x8f, 0xa5, 0xbf, + 0x21, 0x41, 0x2e, 0x74, 0x38, 0xe8, 0x51, 0xcf, 0x0e, 0xfa, 0xe5, 0x68, 0xdc, 0x39, 0x28, 0xe8, + 0x2e, 0xcd, 0xfb, 0xb9, 0x88, 0x60, 0x5c, 0x18, 0x8c, 0x5b, 0xe8, 0x2a, 0x8c, 0x08, 0xdb, 0x10, + 0x6c, 0x1f, 0xa7, 0xfe, 0xf6, 0x97, 0x0b, 0x63, 0xa5, 0x7f, 0x92, 0x82, 0xa9, 0xf0, 0x21, 0xa0, + 0x5a, 0x4f, 0xb9, 0xa2, 0xe6, 0x85, 0x10, 0xc7, 0xe2, 0x90, 0x7b, 0x13, 0x33, 0xfe, 0x67, 0x01, + 0x58, 0x31, 0xaf, 0x0e, 0x89, 0x13, 0x08, 0x96, 0xd3, 0x67, 0x9c, 0xff, 0x22, 0xe9, 0xd9, 0xd7, + 0x45, 0x18, 0xa7, 0xb7, 0xf3, 0xf0, 0xa2, 0x45, 0x9d, 0x2f, 0xaf, 0x90, 0x74, 0x99, 0x65, 0x23, + 0xf6, 0xb8, 0xf1, 0x46, 0x97, 0xe3, 0xf9, 0xc3, 0xe0, 0xfc, 0x5f, 0x00, 0xe4, 0x57, 0x24, 0x8e, + 0x9f, 0xef, 0x8a, 0x44, 0xf4, 0x7d, 0x98, 0xd6, 0xac, 0x56, 0x8b, 0xcd, 0x75, 0xcc, 0x22, 0xf5, + 0x5f, 0x62, 0x42, 0x45, 0xf0, 0x4f, 0x33, 0x2e, 0x7a, 0x9f, 0x68, 0x5c, 0x94, 0xf9, 0x27, 0x1a, + 0x03, 0xf1, 0xa0, 0x79, 0x4f, 0x18, 0x33, 0x64, 0x3d, 0xa1, 0xa9, 0x93, 0x6f, 0x12, 0x9a, 0xca, + 0x82, 0x9a, 0x79, 0xcf, 0xf9, 0x77, 0x12, 0x0f, 0x0c, 0x79, 0x6a, 0x59, 0xfb, 0x5d, 0x2f, 0xa4, + 0x74, 0x3e, 0x78, 0x55, 0x61, 0xfa, 0xab, 0x93, 0x85, 0x94, 0xec, 0xdd, 0x55, 0x18, 0x65, 0xf9, + 0x13, 0xbf, 0x98, 0xe5, 0xbf, 0x06, 0xb9, 0x8e, 0x8d, 0x77, 0xb1, 0xab, 0xed, 0x29, 0x66, 0xb7, + 0xcd, 0x4f, 0xa4, 0x64, 0x05, 0x6d, 0xb3, 0xdb, 0x46, 0x77, 0xa1, 0xe0, 0x65, 0xe1, 0x18, 0x5b, + 0xdc, 0x56, 0x25, 0xe8, 0x1c, 0x91, 0x97, 0xfe, 0x44, 0x82, 0xd9, 0x50, 0x9d, 0xf8, 0x98, 0x58, + 0x83, 0xac, 0x6f, 0x0e, 0x9c, 0xa2, 0x74, 0xce, 0xd0, 0xca, 0x20, 0x33, 0x52, 0xe0, 0xa2, 0x78, + 0x2d, 0xbd, 0xfe, 0xde, 0x17, 0x9b, 0x38, 0xa7, 0xd8, 0x0b, 0xbe, 0x9c, 0xd5, 0xc0, 0x0b, 0xbc, + 0x41, 0x92, 0x1c, 0x69, 0x90, 0x94, 0x7e, 0x24, 0x41, 0x81, 0xbe, 0xe0, 0x31, 0xc6, 0x7a, 0x2c, + 0xd6, 0x49, 0x04, 0x2e, 0x27, 0x46, 0x3f, 0x73, 0x12, 0xfa, 0x7c, 0x47, 0x32, 0xfc, 0xf9, 0x8e, + 0xd2, 0x97, 0x12, 0xe4, 0xbd, 0x12, 0xb2, 0x4f, 0xdf, 0x0d, 0xb9, 0x11, 0xf3, 0xcd, 0x3e, 0xf8, + 0x26, 0x2e, 0xee, 0x18, 0xe9, 0x6b, 0x7c, 0xc1, 0x8b, 0x3b, 0xd8, 0xc7, 0xc9, 0xfe, 0xae, 0xe8, + 0x39, 0xa4, 0x88, 0x65, 0xff, 0x52, 0x86, 0x37, 0x38, 0x7e, 0x23, 0xd3, 0x2f, 0x88, 0x5a, 0xad, + 0x03, 0x76, 0x63, 0xca, 0x48, 0x66, 0x0b, 0xf1, 0x70, 0x28, 0xe0, 0xab, 0x71, 0x7a, 0xa3, 0x4e, + 0xbf, 0x2d, 0xca, 0x7e, 0x3b, 0xa5, 0xc7, 0x01, 0x05, 0xd2, 0xc6, 0x27, 0x5a, 0x1a, 0xc9, 0x94, + 0x0a, 0x2d, 0xb1, 0xbe, 0xf2, 0x87, 0xc1, 0x96, 0xa8, 0x1c, 0x10, 0x14, 0xf6, 0x10, 0x92, 0x07, + 0x6a, 0x6b, 0x58, 0x18, 0x58, 0xa8, 0xe5, 0x64, 0x92, 0x1b, 0x3d, 0x0e, 0xdd, 0x65, 0x91, 0x18, + 0x8c, 0x18, 0xfa, 0x55, 0x1a, 0xba, 0xf3, 0xe2, 0xc3, 0x70, 0x5f, 0x1f, 0xfa, 0xfa, 0x60, 0xa7, + 0xff, 0x38, 0xf5, 0x93, 0x2f, 0x17, 0xa4, 0xd2, 0x87, 0x70, 0xf9, 0x89, 0xe5, 0x38, 0x46, 0x87, + 0xa0, 0x44, 0x3a, 0x80, 0x88, 0x15, 0xf7, 0x2c, 0x59, 0xba, 0x43, 0xd7, 0x0b, 0x4c, 0x36, 0xe2, + 0x33, 0xb2, 0xf7, 0x5c, 0xfa, 0x17, 0x12, 0x5c, 0xea, 0xe7, 0x64, 0x0a, 0x89, 0x3a, 0xde, 0x37, + 0xa9, 0x59, 0xfe, 0xdd, 0x6e, 0x67, 0x77, 0x2c, 0x91, 0x9d, 0x78, 0x83, 0xfc, 0x9d, 0x4a, 0x5b, + 0xa5, 0x23, 0x9d, 0x1f, 0x41, 0xce, 0x73, 0xf2, 0x06, 0xa3, 0xfa, 0x83, 0x3e, 0x35, 0xda, 0xa0, + 0x6f, 0xc0, 0xf4, 0x9a, 0x65, 0x98, 0xc4, 0xe9, 0x14, 0xf5, 0x5d, 0x86, 0xfc, 0x8e, 0x61, 0xaa, + 0xf6, 0xb1, 0x22, 0xce, 0x77, 0xb3, 0x36, 0x9d, 0x8f, 0x2a, 0x2c, 0xcb, 0x21, 0x4f, 0x31, 0x0e, + 0xfe, 0x58, 0xfa, 0xa9, 0x04, 0x05, 0x5f, 0x2c, 0x37, 0x9e, 0xef, 0x02, 0x68, 0xad, 0xae, 0xe3, + 0x62, 0x5b, 0x1c, 0xab, 0xc9, 0xb1, 0x30, 0xec, 0x32, 0xa3, 0xd6, 0x56, 0xe5, 0x0c, 0xcf, 0x50, + 0xd3, 0xd1, 0xf5, 0xf0, 0x45, 0x07, 0xe3, 0x2b, 0x70, 0xda, 0x77, 0xbd, 0x01, 0xba, 0x05, 0x69, + 0xc7, 0xb5, 0x6c, 0x0f, 0xde, 0x8c, 0xaf, 0x64, 0x4f, 0x03, 0x17, 0xb1, 0xd3, 0x03, 0xbc, 0x24, + 0xdf, 0x32, 0xe4, 0xc9, 0xcc, 0x7c, 0x80, 0xbd, 0x2a, 0xa5, 0xce, 0xae, 0x12, 0xe3, 0xe0, 0x8f, + 0xf7, 0xea, 0x30, 0x1b, 0x31, 0x11, 0xa1, 0x3c, 0x40, 0xe0, 0xbb, 0x2f, 0xfc, 0x8b, 0xb5, 0xcb, + 0xab, 0xca, 0xf6, 0x66, 0xf9, 0xd9, 0xc6, 0x46, 0xad, 0xd1, 0xa8, 0xac, 0x16, 0x24, 0x54, 0x80, + 0x5c, 0xe8, 0xab, 0x31, 0x09, 0xf6, 0x0d, 0xdb, 0x7b, 0x7f, 0x0e, 0xc0, 0xff, 0x18, 0x15, 0x91, + 0xb5, 0x5e, 0xf9, 0x4c, 0x79, 0xb1, 0xfc, 0x74, 0xbb, 0x52, 0x2f, 0x8c, 0x21, 0x04, 0xf9, 0x95, + 0xe5, 0x46, 0xb9, 0xaa, 0xc8, 0x95, 0xfa, 0xd6, 0xb3, 0xcd, 0x7a, 0x45, 0x7c, 0xfb, 0xf6, 0xde, + 0x2a, 0xe4, 0x82, 0x17, 0xc3, 0xa0, 0x59, 0x98, 0x2e, 0x57, 0x2b, 0xe5, 0x75, 0xe5, 0x45, 0x6d, + 0x59, 0x79, 0xbe, 0x5d, 0xd9, 0xae, 0x14, 0xc6, 0x68, 0xd1, 0x28, 0xf1, 0xf1, 0xf6, 0xd3, 0xa7, + 0x05, 0x09, 0x4d, 0x43, 0x96, 0x3d, 0xd3, 0x2f, 0xcc, 0x14, 0x12, 0xf7, 0x36, 0x20, 0x1b, 0xb8, + 0x42, 0x96, 0xbc, 0x6e, 0x6b, 0xbb, 0x5e, 0x55, 0x1a, 0xb5, 0x8d, 0x4a, 0xbd, 0xb1, 0xbc, 0xb1, + 0xc5, 0x64, 0x50, 0xda, 0xf2, 0xca, 0x33, 0xb9, 0x51, 0x90, 0xbc, 0xe7, 0xc6, 0xb3, 0xed, 0x72, + 0x55, 0x54, 0xa3, 0x94, 0x4a, 0x27, 0x0b, 0xc9, 0x7b, 0xbf, 0x26, 0xc1, 0xa5, 0x01, 0xd7, 0xa3, + 0xa0, 0x2c, 0x4c, 0x6e, 0x9b, 0xf4, 0x0e, 0xcd, 0xc2, 0x18, 0x9a, 0x0a, 0xdc, 0x90, 0x52, 0x90, + 0x50, 0x9a, 0xdd, 0x4e, 0x51, 0x48, 0xa0, 0x09, 0x48, 0xd4, 0x1f, 0x16, 0x92, 0xa4, 0xa4, 0x81, + 0x0b, 0x46, 0x0a, 0x29, 0x94, 0xe1, 0xf7, 0x23, 0x14, 0xc6, 0x51, 0xce, 0xbf, 0xa0, 0xa0, 0x30, + 0x41, 0x44, 0x79, 0x47, 0xfc, 0x0b, 0x93, 0xf7, 0xae, 0x41, 0xe0, 0xb8, 0x34, 0x02, 0x98, 0x78, + 0xaa, 0xba, 0xd8, 0x71, 0x0b, 0x63, 0x68, 0x12, 0x92, 0xcb, 0xad, 0x56, 0x41, 0x7a, 0xf0, 0xb3, + 0x24, 0xa4, 0xc5, 0x37, 0x54, 0xd0, 0x53, 0x18, 0x67, 0xab, 0xaf, 0x0b, 0x83, 0x9d, 0x5b, 0x3a, + 0x28, 0xe6, 0xaf, 0x9e, 0xe5, 0xfd, 0x96, 0xc6, 0xd0, 0x9f, 0x87, 0x6c, 0xc0, 0x69, 0x40, 0x03, + 0x57, 0x90, 0x42, 0x8e, 0xd2, 0xfc, 0xad, 0xb3, 0xb2, 0x79, 0xf2, 0x5f, 0x42, 0xc6, 0x33, 0x62, + 0xe8, 0xfa, 0x30, 0x13, 0x27, 0x64, 0x0f, 0xb7, 0x83, 0xc4, 0x4a, 0x95, 0xc6, 0xde, 0x97, 0x90, + 0x0d, 0xa8, 0xdf, 0x88, 0xa1, 0xa8, 0x4d, 0xf9, 0x81, 0x56, 0x72, 0xfe, 0xde, 0x48, 0xb9, 0xfd, + 0x77, 0x3e, 0x87, 0x14, 0xb1, 0x10, 0x28, 0xca, 0xcd, 0xe9, 0xb1, 0x48, 0xf3, 0xd7, 0x87, 0xe6, + 0x11, 0xfa, 0x59, 0xb9, 0xfb, 0x93, 0x3f, 0xbe, 0x32, 0xf6, 0x93, 0xd3, 0x2b, 0xd2, 0x4f, 0x4f, + 0xaf, 0x48, 0x3f, 0x3b, 0xbd, 0x22, 0xfd, 0x97, 0xd3, 0x2b, 0xd2, 0x5f, 0xff, 0xf9, 0x95, 0xb1, + 0x9f, 0xfe, 0xfc, 0xca, 0xd8, 0xcf, 0x7e, 0x7e, 0x65, 0xec, 0xf3, 0x49, 0xce, 0xbd, 0x33, 0x41, + 0x3f, 0x54, 0xfe, 0xf0, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xf3, 0x3b, 0xf2, 0x8a, 0xd5, 0x7d, + 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 1ea90c229c7b..74608a38cdf9 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -19,7 +19,7 @@ import "roachpb/metadata.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/mvcc3.proto"; import "util/hlc/timestamp.proto"; -import "util/tracing/recorded_span.proto"; +import "util/tracing/tracingpb/recorded_span.proto"; import "gogoproto/gogo.proto"; // ReadConsistencyType specifies what type of consistency is observed @@ -2100,7 +2100,7 @@ message BatchResponse { util.hlc.Timestamp now = 5 [(gogoproto.nullable) = false]; // collected_spans stores trace spans recorded during the execution of this // request. - repeated util.tracing.RecordedSpan collected_spans = 6 [(gogoproto.nullable) = false]; + repeated util.tracing.tracingpb.RecordedSpan collected_spans = 6 [(gogoproto.nullable) = false]; // Range or list of ranges used to execute the request. The server only // populates this if return_range_info is set on the request, or if the // server detects the client's client_range_info to be stale. diff --git a/pkg/server/status.go b/pkg/server/status.go index bd0cd481447a..9ff67f5949de 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -60,7 +60,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" gwruntime "github.com/grpc-ecosystem/grpc-gateway/runtime" "go.etcd.io/etcd/raft" @@ -562,7 +562,7 @@ func (s *statusServer) Allocator( return output, nil } -func recordedSpansToTraceEvents(spans []tracing.RecordedSpan) []*serverpb.TraceEvent { +func recordedSpansToTraceEvents(spans []tracingpb.RecordedSpan) []*serverpb.TraceEvent { var output []*serverpb.TraceEvent var buf bytes.Buffer for _, sp := range spans { diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index eb79462ca999..c37d0c2ae782 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -70,6 +70,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/opentracing/opentracing-go" @@ -1433,8 +1434,8 @@ func (st *SessionTracing) getSessionTrace() ([]traceRow, error) { } // getRecording returns the recorded spans of the current trace. -func (st *SessionTracing) getRecording() []tracing.RecordedSpan { - var spans []tracing.RecordedSpan +func (st *SessionTracing) getRecording() []tracingpb.RecordedSpan { + var spans []tracingpb.RecordedSpan if st.firstTxnSpan != nil { spans = append(spans, tracing.GetRecording(st.firstTxnSpan)...) } @@ -1553,7 +1554,7 @@ func (st *SessionTracing) StopTracing() error { st.showResults = false st.recordingType = tracing.NoRecording - var spans []tracing.RecordedSpan + var spans []tracingpb.RecordedSpan if st.firstTxnSpan != nil { spans = append(spans, tracing.GetRecording(st.firstTxnSpan)...) @@ -1726,7 +1727,7 @@ var logMessageRE = regexp.MustCompile( // // Note that what's described above is not the order in which SHOW TRACE FOR SESSION // displays the information: SHOW TRACE will sort by the age column. -func generateSessionTraceVTable(spans []tracing.RecordedSpan) ([]traceRow, error) { +func generateSessionTraceVTable(spans []tracingpb.RecordedSpan) ([]traceRow, error) { // Get all the log messages, in the right order. var allLogs []logRecordRow @@ -1831,7 +1832,7 @@ func generateSessionTraceVTable(spans []tracing.RecordedSpan) ([]traceRow, error // getOrderedChildSpans returns all the spans in allSpans that are children of // spanID. It assumes the input is ordered by start time, in which case the // output is also ordered. -func getOrderedChildSpans(spanID uint64, allSpans []tracing.RecordedSpan) []spanWithIndex { +func getOrderedChildSpans(spanID uint64, allSpans []tracingpb.RecordedSpan) []spanWithIndex { children := make([]spanWithIndex, 0) for i := range allSpans { if allSpans[i].ParentSpanID == spanID { @@ -1853,7 +1854,7 @@ func getOrderedChildSpans(spanID uint64, allSpans []tracing.RecordedSpan) []span // seenSpans is modified to record all the spans that are part of the subtrace // rooted at span. func getMessagesForSubtrace( - span spanWithIndex, allSpans []tracing.RecordedSpan, seenSpans map[uint64]struct{}, + span spanWithIndex, allSpans []tracingpb.RecordedSpan, seenSpans map[uint64]struct{}, ) ([]logRecordRow, error) { if _, ok := seenSpans[span.SpanID]; ok { return nil, errors.Errorf("duplicate span %d", span.SpanID) @@ -1940,7 +1941,7 @@ type logRecordRow struct { } type spanWithIndex struct { - *tracing.RecordedSpan + *tracingpb.RecordedSpan index int } diff --git a/pkg/sql/execinfra/base.go b/pkg/sql/execinfra/base.go index e460a63be492..e2489e83464e 100644 --- a/pkg/sql/execinfra/base.go +++ b/pkg/sql/execinfra/base.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/opentracing/opentracing-go" @@ -234,7 +235,7 @@ func DrainAndForwardMetadata(ctx context.Context, src RowSource, dst RowReceiver } // GetTraceData returns the trace data. -func GetTraceData(ctx context.Context) []tracing.RecordedSpan { +func GetTraceData(ctx context.Context) []tracingpb.RecordedSpan { if sp := opentracing.SpanFromContext(ctx); sp != nil { return tracing.GetRecording(sp) } diff --git a/pkg/sql/execinfrapb/data.go b/pkg/sql/execinfrapb/data.go index 1493ac5312bf..e7310ac6280e 100644 --- a/pkg/sql/execinfrapb/data.go +++ b/pkg/sql/execinfrapb/data.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) @@ -179,7 +179,7 @@ type ProducerMetadata struct { // TODO(vivek): change to type Error Err error // TraceData is sent if snowball tracing is enabled. - TraceData []tracing.RecordedSpan + TraceData []tracingpb.RecordedSpan // LeafTxnFinalState contains the final state of the LeafTxn to be // sent from leaf flows to the RootTxn held by the flow's ultimate // receiver. diff --git a/pkg/sql/execinfrapb/data.pb.go b/pkg/sql/execinfrapb/data.pb.go index 8d94b9b2366c..ae9495b421f9 100644 --- a/pkg/sql/execinfrapb/data.pb.go +++ b/pkg/sql/execinfrapb/data.pb.go @@ -17,7 +17,7 @@ import types1 "github.com/gogo/protobuf/types" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import types "github.com/cockroachdb/cockroach/pkg/sql/types" -import tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" +import tracingpb "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -71,7 +71,7 @@ func (x *Ordering_Column_Direction) UnmarshalJSON(data []byte) error { return nil } func (Ordering_Column_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{2, 0, 0} + return fileDescriptor_data_a11511dae3716e94, []int{2, 0, 0} } type StreamEndpointSpec_Type int32 @@ -116,7 +116,7 @@ func (x *StreamEndpointSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (StreamEndpointSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{3, 0} + return fileDescriptor_data_a11511dae3716e94, []int{3, 0} } type InputSyncSpec_Type int32 @@ -156,7 +156,7 @@ func (x *InputSyncSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{4, 0} + return fileDescriptor_data_a11511dae3716e94, []int{4, 0} } type OutputRouterSpec_Type int32 @@ -204,7 +204,7 @@ func (x *OutputRouterSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{5, 0} + return fileDescriptor_data_a11511dae3716e94, []int{5, 0} } // Error is a generic representation including a string message. @@ -217,7 +217,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{0} + return fileDescriptor_data_a11511dae3716e94, []int{0} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +245,7 @@ var xxx_messageInfo_Error proto.InternalMessageInfo func (m *Expression) Reset() { *m = Expression{} } func (*Expression) ProtoMessage() {} func (*Expression) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{1} + return fileDescriptor_data_a11511dae3716e94, []int{1} } func (m *Expression) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -280,7 +280,7 @@ func (m *Ordering) Reset() { *m = Ordering{} } func (m *Ordering) String() string { return proto.CompactTextString(m) } func (*Ordering) ProtoMessage() {} func (*Ordering) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{2} + return fileDescriptor_data_a11511dae3716e94, []int{2} } func (m *Ordering) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -314,7 +314,7 @@ func (m *Ordering_Column) Reset() { *m = Ordering_Column{} } func (m *Ordering_Column) String() string { return proto.CompactTextString(m) } func (*Ordering_Column) ProtoMessage() {} func (*Ordering_Column) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{2, 0} + return fileDescriptor_data_a11511dae3716e94, []int{2, 0} } func (m *Ordering_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -361,7 +361,7 @@ func (m *StreamEndpointSpec) Reset() { *m = StreamEndpointSpec{} } func (m *StreamEndpointSpec) String() string { return proto.CompactTextString(m) } func (*StreamEndpointSpec) ProtoMessage() {} func (*StreamEndpointSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{3} + return fileDescriptor_data_a11511dae3716e94, []int{3} } func (m *StreamEndpointSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +400,7 @@ func (m *InputSyncSpec) Reset() { *m = InputSyncSpec{} } func (m *InputSyncSpec) String() string { return proto.CompactTextString(m) } func (*InputSyncSpec) ProtoMessage() {} func (*InputSyncSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{4} + return fileDescriptor_data_a11511dae3716e94, []int{4} } func (m *InputSyncSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -444,7 +444,7 @@ func (m *OutputRouterSpec) Reset() { *m = OutputRouterSpec{} } func (m *OutputRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec) ProtoMessage() {} func (*OutputRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{5} + return fileDescriptor_data_a11511dae3716e94, []int{5} } func (m *OutputRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -487,7 +487,7 @@ func (m *OutputRouterSpec_RangeRouterSpec) Reset() { *m = OutputRouterSp func (m *OutputRouterSpec_RangeRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{5, 0} + return fileDescriptor_data_a11511dae3716e94, []int{5, 0} } func (m *OutputRouterSpec_RangeRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -529,7 +529,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) String() string { } func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{5, 0, 0} + return fileDescriptor_data_a11511dae3716e94, []int{5, 0, 0} } func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -566,7 +566,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_Span) Reset() { *m = OutputRou func (m *OutputRouterSpec_RangeRouterSpec_Span) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec_Span) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{5, 0, 1} + return fileDescriptor_data_a11511dae3716e94, []int{5, 0, 1} } func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -600,7 +600,7 @@ func (m *DatumInfo) Reset() { *m = DatumInfo{} } func (m *DatumInfo) String() string { return proto.CompactTextString(m) } func (*DatumInfo) ProtoMessage() {} func (*DatumInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{6} + return fileDescriptor_data_a11511dae3716e94, []int{6} } func (m *DatumInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -635,7 +635,7 @@ func (m *ProducerHeader) Reset() { *m = ProducerHeader{} } func (m *ProducerHeader) String() string { return proto.CompactTextString(m) } func (*ProducerHeader) ProtoMessage() {} func (*ProducerHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{7} + return fileDescriptor_data_a11511dae3716e94, []int{7} } func (m *ProducerHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -678,7 +678,7 @@ func (m *ProducerData) Reset() { *m = ProducerData{} } func (m *ProducerData) String() string { return proto.CompactTextString(m) } func (*ProducerData) ProtoMessage() {} func (*ProducerData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{8} + return fileDescriptor_data_a11511dae3716e94, []int{8} } func (m *ProducerData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,7 @@ func (m *ProducerMessage) Reset() { *m = ProducerMessage{} } func (m *ProducerMessage) String() string { return proto.CompactTextString(m) } func (*ProducerMessage) ProtoMessage() {} func (*ProducerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{9} + return fileDescriptor_data_a11511dae3716e94, []int{9} } func (m *ProducerMessage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +766,7 @@ func (m *RemoteProducerMetadata) Reset() { *m = RemoteProducerMetadata{} func (m *RemoteProducerMetadata) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata) ProtoMessage() {} func (*RemoteProducerMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10} + return fileDescriptor_data_a11511dae3716e94, []int{10} } func (m *RemoteProducerMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1090,7 +1090,7 @@ func (m *RemoteProducerMetadata_RangeInfos) Reset() { *m = RemoteProduce func (m *RemoteProducerMetadata_RangeInfos) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RangeInfos) ProtoMessage() {} func (*RemoteProducerMetadata_RangeInfos) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 0} + return fileDescriptor_data_a11511dae3716e94, []int{10, 0} } func (m *RemoteProducerMetadata_RangeInfos) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1116,14 +1116,14 @@ func (m *RemoteProducerMetadata_RangeInfos) XXX_DiscardUnknown() { var xxx_messageInfo_RemoteProducerMetadata_RangeInfos proto.InternalMessageInfo type RemoteProducerMetadata_TraceData struct { - CollectedSpans []tracing.RecordedSpan `protobuf:"bytes,1,rep,name=collected_spans,json=collectedSpans" json:"collected_spans"` + CollectedSpans []tracingpb.RecordedSpan `protobuf:"bytes,1,rep,name=collected_spans,json=collectedSpans" json:"collected_spans"` } func (m *RemoteProducerMetadata_TraceData) Reset() { *m = RemoteProducerMetadata_TraceData{} } func (m *RemoteProducerMetadata_TraceData) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_TraceData) ProtoMessage() {} func (*RemoteProducerMetadata_TraceData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 1} + return fileDescriptor_data_a11511dae3716e94, []int{10, 1} } func (m *RemoteProducerMetadata_TraceData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1164,7 @@ func (m *RemoteProducerMetadata_RowNum) Reset() { *m = RemoteProducerMet func (m *RemoteProducerMetadata_RowNum) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RowNum) ProtoMessage() {} func (*RemoteProducerMetadata_RowNum) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 2} + return fileDescriptor_data_a11511dae3716e94, []int{10, 2} } func (m *RemoteProducerMetadata_RowNum) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1204,7 +1204,7 @@ func (m *RemoteProducerMetadata_SamplerProgress) Reset() { func (m *RemoteProducerMetadata_SamplerProgress) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_SamplerProgress) ProtoMessage() {} func (*RemoteProducerMetadata_SamplerProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 3} + return fileDescriptor_data_a11511dae3716e94, []int{10, 3} } func (m *RemoteProducerMetadata_SamplerProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1245,7 +1245,7 @@ func (m *RemoteProducerMetadata_BulkProcessorProgress) String() string { } func (*RemoteProducerMetadata_BulkProcessorProgress) ProtoMessage() {} func (*RemoteProducerMetadata_BulkProcessorProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 4} + return fileDescriptor_data_a11511dae3716e94, []int{10, 4} } func (m *RemoteProducerMetadata_BulkProcessorProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1282,7 +1282,7 @@ func (m *RemoteProducerMetadata_Metrics) Reset() { *m = RemoteProducerMe func (m *RemoteProducerMetadata_Metrics) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_Metrics) ProtoMessage() {} func (*RemoteProducerMetadata_Metrics) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{10, 5} + return fileDescriptor_data_a11511dae3716e94, []int{10, 5} } func (m *RemoteProducerMetadata_Metrics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1322,7 +1322,7 @@ func (m *DistSQLVersionGossipInfo) Reset() { *m = DistSQLVersionGossipIn func (m *DistSQLVersionGossipInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLVersionGossipInfo) ProtoMessage() {} func (*DistSQLVersionGossipInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{11} + return fileDescriptor_data_a11511dae3716e94, []int{11} } func (m *DistSQLVersionGossipInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,7 @@ func (m *DistSQLDrainingInfo) Reset() { *m = DistSQLDrainingInfo{} } func (m *DistSQLDrainingInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLDrainingInfo) ProtoMessage() {} func (*DistSQLDrainingInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_5e18e048f2322ce2, []int{12} + return fileDescriptor_data_a11511dae3716e94, []int{12} } func (m *DistSQLDrainingInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4978,7 +4978,7 @@ func (m *RemoteProducerMetadata_TraceData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CollectedSpans = append(m.CollectedSpans, tracing.RecordedSpan{}) + m.CollectedSpans = append(m.CollectedSpans, tracingpb.RecordedSpan{}) if err := m.CollectedSpans[len(m.CollectedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5858,137 +5858,138 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_5e18e048f2322ce2) } - -var fileDescriptor_data_5e18e048f2322ce2 = []byte{ - // 2056 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcb, 0x73, 0x5b, 0x49, - 0xd5, 0xd7, 0x95, 0x65, 0x3d, 0x8e, 0x5f, 0x72, 0x7f, 0x49, 0x46, 0x9f, 0x08, 0xb6, 0x47, 0xa1, - 0xa6, 0x02, 0xcc, 0x5c, 0x25, 0xce, 0x22, 0x83, 0x81, 0x4a, 0x2c, 0x4b, 0x8e, 0x94, 0x71, 0x6c, - 0xcf, 0x95, 0xc3, 0x54, 0x66, 0xa0, 0x2e, 0x57, 0xf7, 0xb6, 0xe4, 0x4b, 0xee, 0x2b, 0xdd, 0x7d, - 0x63, 0x7b, 0x43, 0x51, 0x05, 0x4b, 0x16, 0xac, 0x28, 0x96, 0xc3, 0x86, 0x0d, 0x0b, 0x76, 0xfc, - 0x09, 0x90, 0x15, 0x35, 0xcb, 0x14, 0x8b, 0x14, 0x38, 0x1b, 0x96, 0x6c, 0x99, 0x15, 0xd5, 0x2f, - 0xbd, 0x62, 0x27, 0x78, 0x32, 0x1b, 0xbb, 0xfb, 0x9c, 0xf3, 0xfb, 0xf5, 0x39, 0xa7, 0x4f, 0x9f, - 0x6e, 0x5d, 0xa8, 0xd2, 0x27, 0x41, 0x1d, 0x1f, 0x63, 0xd7, 0x8f, 0xfa, 0xc4, 0x49, 0x7a, 0x75, - 0xcf, 0x61, 0x8e, 0x99, 0x90, 0x98, 0xc5, 0xa8, 0xe2, 0xc6, 0xee, 0x63, 0x12, 0x3b, 0xee, 0xa1, - 0x49, 0x9f, 0x04, 0xa6, 0xe7, 0x53, 0x46, 0x9f, 0x04, 0x24, 0x8d, 0xaa, 0x57, 0x7e, 0x16, 0xf7, - 0x68, 0x9d, 0xff, 0x49, 0x7a, 0xe2, 0x9f, 0x44, 0x54, 0x97, 0x85, 0x75, 0xd2, 0xab, 0x3b, 0x89, - 0xaf, 0x44, 0x48, 0x8b, 0x46, 0xc4, 0xd5, 0x4b, 0x5a, 0x86, 0x09, 0x89, 0x89, 0x06, 0x5f, 0x96, - 0xb3, 0x69, 0xf1, 0x2a, 0xf7, 0x30, 0x19, 0x1c, 0xf9, 0x04, 0xd7, 0x93, 0x81, 0x50, 0x4d, 0x1a, - 0x5c, 0xe3, 0x06, 0xae, 0xc3, 0x9c, 0x20, 0x1e, 0xd4, 0x3d, 0x4c, 0xdd, 0xa4, 0x57, 0xa7, 0x8c, - 0xa4, 0x2e, 0x4b, 0x09, 0xf6, 0x94, 0xd1, 0x7b, 0x67, 0x18, 0xe1, 0xc8, 0x8d, 0x3d, 0xec, 0xd9, - 0x9e, 0xc3, 0xd2, 0x50, 0x3b, 0xc1, 0xed, 0xd8, 0x49, 0x82, 0xa9, 0xfc, 0xab, 0xc4, 0x6b, 0x29, - 0xf3, 0x83, 0x3a, 0x23, 0x8e, 0xeb, 0x47, 0x83, 0x3a, 0xc1, 0x6e, 0x4c, 0x38, 0x92, 0x26, 0x4e, - 0xa4, 0x63, 0x1a, 0xc4, 0x83, 0x58, 0x0c, 0xeb, 0x7c, 0xa4, 0xa4, 0x15, 0x81, 0x3b, 0x0c, 0xdc, - 0x3a, 0xf3, 0x43, 0x4c, 0x99, 0x13, 0x26, 0x4a, 0xf3, 0xff, 0x83, 0x38, 0x1e, 0x04, 0xb8, 0x2e, - 0x66, 0xbd, 0xb4, 0x5f, 0x77, 0xa2, 0x13, 0xa9, 0xaa, 0x7d, 0x0a, 0xb3, 0x2d, 0x1e, 0x20, 0xba, - 0x03, 0xd0, 0x4f, 0x83, 0xc0, 0x16, 0xe1, 0x56, 0x66, 0xd6, 0x8c, 0xeb, 0x73, 0xeb, 0x6b, 0xe6, - 0x68, 0x57, 0x74, 0xc2, 0xcc, 0x96, 0x8c, 0x44, 0xa0, 0xac, 0x12, 0xc7, 0x88, 0xe1, 0x46, 0xee, - 0x77, 0x9f, 0xaf, 0x66, 0xee, 0xe7, 0x8a, 0x46, 0x39, 0x7b, 0x3f, 0x57, 0xcc, 0x96, 0x67, 0x6a, - 0xfb, 0x00, 0xad, 0xe3, 0x84, 0x60, 0x4a, 0xfd, 0x38, 0x42, 0x2b, 0x50, 0x78, 0x8a, 0x09, 0x1f, - 0x56, 0x8c, 0x35, 0xe3, 0x7a, 0xa9, 0x91, 0x7b, 0xf6, 0x62, 0x35, 0x63, 0x69, 0x21, 0xaa, 0x40, - 0x0e, 0x1f, 0x27, 0xa4, 0x92, 0x1d, 0x53, 0x0a, 0xc9, 0x46, 0x91, 0x33, 0xff, 0xfb, 0xf7, 0xab, - 0x99, 0xda, 0xaf, 0xb2, 0x50, 0xdc, 0x23, 0x1e, 0x26, 0x7e, 0x34, 0x40, 0x1d, 0x28, 0xb8, 0x71, - 0x90, 0x86, 0x11, 0xad, 0x18, 0x6b, 0x33, 0xd7, 0xe7, 0xd6, 0xbf, 0x6d, 0x9e, 0x57, 0x44, 0xa6, - 0x06, 0x99, 0x5b, 0x02, 0xa1, 0xd7, 0x56, 0xf8, 0xea, 0x1f, 0x0c, 0xc8, 0x4b, 0x0d, 0xfa, 0xa6, - 0x60, 0xb5, 0x7d, 0xef, 0x58, 0xb8, 0xb9, 0xa0, 0x4c, 0xf3, 0x6e, 0x1c, 0x74, 0xbc, 0x63, 0xf4, - 0x09, 0x94, 0x3c, 0x9f, 0x60, 0x97, 0xf1, 0x38, 0xb8, 0xab, 0x8b, 0xeb, 0xb7, 0xfe, 0xe7, 0x65, - 0xcd, 0xa6, 0x86, 0x2a, 0xd6, 0x11, 0x57, 0x6d, 0x05, 0x4a, 0x43, 0x2d, 0x2a, 0xc0, 0xcc, 0x66, - 0x77, 0xab, 0x9c, 0x41, 0x45, 0xc8, 0x35, 0x5b, 0xdd, 0xad, 0xb2, 0xb1, 0x91, 0xfb, 0xd7, 0xe7, - 0xab, 0xea, 0x6f, 0xed, 0xaf, 0x59, 0x40, 0x5d, 0x46, 0xb0, 0x13, 0xb6, 0x22, 0x2f, 0x89, 0xfd, - 0x88, 0x75, 0x13, 0xec, 0xa2, 0x8f, 0x20, 0xc7, 0xeb, 0x48, 0xf8, 0xbd, 0xb8, 0x7e, 0xf3, 0x7c, - 0xb7, 0x5e, 0xc5, 0x9a, 0x07, 0x27, 0x09, 0xd6, 0x49, 0xe7, 0x24, 0xe8, 0x7b, 0x50, 0xa2, 0xc2, - 0xcc, 0xf6, 0x3d, 0x11, 0xe8, 0x6c, 0xe3, 0x2a, 0x57, 0x9f, 0xbe, 0x58, 0x2d, 0x4a, 0x7c, 0xa7, - 0xf9, 0xe5, 0xd8, 0xd8, 0x2a, 0x4a, 0xf3, 0x8e, 0x87, 0x9e, 0xc0, 0x22, 0x73, 0xc8, 0x00, 0x33, - 0x3b, 0x8a, 0x3d, 0xcc, 0xf1, 0x39, 0x81, 0xff, 0x48, 0xe1, 0xe7, 0x0f, 0x84, 0x76, 0x37, 0xf6, - 0xb0, 0xe0, 0xb8, 0x35, 0xf0, 0xd9, 0x61, 0xda, 0x33, 0xdd, 0x38, 0xac, 0x0f, 0x7d, 0xf6, 0x7a, - 0xa3, 0x71, 0x3d, 0x79, 0x3c, 0xa8, 0xab, 0x73, 0x6c, 0x4a, 0x98, 0x35, 0xcf, 0x46, 0x24, 0x5e, - 0xed, 0x06, 0xe4, 0x78, 0x04, 0xa8, 0x04, 0xb3, 0x3b, 0x7b, 0x5b, 0x9b, 0x3b, 0xe5, 0x0c, 0x02, - 0xc8, 0x5b, 0xad, 0x07, 0x7b, 0x07, 0xad, 0xb2, 0x81, 0x96, 0x61, 0xa1, 0xfb, 0x68, 0x77, 0xcb, - 0xb6, 0x5a, 0xdd, 0xfd, 0xbd, 0xdd, 0x6e, 0x4b, 0x94, 0xe8, 0x4c, 0x39, 0x57, 0xfb, 0x5b, 0x16, - 0x16, 0x3a, 0x51, 0x92, 0xb2, 0xee, 0x49, 0xe4, 0x8a, 0x24, 0x6e, 0x4f, 0x24, 0xf1, 0xfd, 0xf3, - 0x93, 0x38, 0x01, 0x7b, 0x35, 0x7f, 0x4d, 0x28, 0xc6, 0x6a, 0xf7, 0x45, 0xfa, 0xe6, 0xd6, 0x6b, - 0x6f, 0xae, 0x13, 0xc5, 0x30, 0x44, 0xa2, 0x1d, 0x28, 0xc8, 0xb4, 0xd2, 0xca, 0x8c, 0xa8, 0xf1, - 0xf7, 0x2f, 0xb2, 0xab, 0xba, 0xcc, 0x15, 0x05, 0xfa, 0x21, 0xcc, 0xcb, 0x8a, 0xb7, 0x45, 0xbf, - 0xa9, 0xe4, 0x04, 0x65, 0x75, 0x8a, 0x92, 0xe2, 0xd0, 0x94, 0xfd, 0xe8, 0xc0, 0x9a, 0x93, 0xf6, - 0x3c, 0x3a, 0x5a, 0xab, 0xa9, 0x24, 0x2f, 0x40, 0xe9, 0xe1, 0xee, 0x9e, 0xd5, 0x6c, 0x59, 0xad, - 0x66, 0x39, 0x83, 0xe6, 0xa0, 0xa0, 0x27, 0x46, 0xed, 0x3f, 0x79, 0x28, 0xef, 0xa5, 0x2c, 0x49, - 0x99, 0x15, 0xa7, 0x0c, 0x13, 0x91, 0xd3, 0xce, 0x44, 0x4e, 0xeb, 0xaf, 0xc9, 0xc3, 0x14, 0xf2, - 0xd5, 0xb4, 0x8e, 0x25, 0x24, 0xfb, 0xf6, 0x09, 0x79, 0x17, 0xe6, 0x0f, 0x1d, 0x7a, 0x68, 0xeb, - 0x3e, 0xc2, 0x73, 0xbc, 0x60, 0xcd, 0x71, 0x99, 0x3c, 0xb1, 0x14, 0x05, 0xb0, 0x4c, 0x9c, 0x68, - 0x80, 0x6d, 0x22, 0xbc, 0xb2, 0x69, 0x82, 0x5d, 0x51, 0xcf, 0x73, 0xeb, 0x1b, 0x17, 0x08, 0xc4, - 0xe2, 0x1c, 0xa3, 0xb9, 0x72, 0x64, 0x89, 0x4c, 0x8a, 0xd1, 0x4d, 0x58, 0xf6, 0x7c, 0xea, 0xf4, - 0x02, 0x6c, 0xf7, 0xd2, 0x7e, 0x5f, 0x96, 0xcf, 0xec, 0x9a, 0x71, 0xbd, 0xa8, 0x10, 0x65, 0xa5, - 0x6e, 0x68, 0x6d, 0xf5, 0x2f, 0x33, 0xb0, 0x34, 0xc5, 0x8e, 0x3e, 0x83, 0x59, 0x7e, 0x5d, 0xe8, - 0xc6, 0x78, 0xe7, 0xab, 0x3b, 0x6a, 0x76, 0x13, 0x47, 0x77, 0x2b, 0xc9, 0xc9, 0x93, 0xe6, 0xe1, - 0xbe, 0x93, 0x06, 0xcc, 0xf6, 0x30, 0x65, 0xb2, 0x39, 0x58, 0x73, 0x4a, 0xd6, 0xc4, 0x94, 0xa1, - 0x10, 0x4a, 0xe2, 0xc2, 0xf3, 0xa3, 0x81, 0x2e, 0xdc, 0xce, 0x5b, 0xf8, 0x20, 0xf7, 0xa2, 0xa5, - 0x18, 0x75, 0xef, 0x1c, 0xae, 0x50, 0x7d, 0x0a, 0x8b, 0x93, 0x26, 0xe8, 0x2a, 0xe4, 0xe5, 0x9e, - 0xbe, 0xd2, 0xc4, 0x79, 0x8f, 0xdf, 0x86, 0xa2, 0x06, 0xab, 0x1e, 0xfe, 0xad, 0xe9, 0x33, 0xf0, - 0x24, 0xe8, 0x39, 0x14, 0x9b, 0x4d, 0x7e, 0x5d, 0x4f, 0x2d, 0x3c, 0xc4, 0x56, 0x77, 0x20, 0xc7, - 0xd3, 0x83, 0x2e, 0xc1, 0x2c, 0x65, 0x0e, 0x61, 0x62, 0xb1, 0x79, 0x4b, 0x4e, 0x50, 0x19, 0x66, - 0x70, 0x24, 0x7b, 0xe7, 0xbc, 0xc5, 0x87, 0xdc, 0x2b, 0x59, 0x79, 0xe2, 0x7e, 0x9d, 0xd5, 0x5e, - 0x49, 0x59, 0xed, 0x8e, 0x3a, 0x5e, 0x65, 0x98, 0xdf, 0xdf, 0xec, 0x76, 0xed, 0x83, 0xb6, 0xb5, - 0xf7, 0xf0, 0x5e, 0x5b, 0xb6, 0xb2, 0x07, 0x1d, 0xcb, 0xda, 0xb3, 0xca, 0x06, 0x3f, 0x6d, 0x8d, - 0x47, 0x76, 0x7b, 0xb3, 0xdb, 0x2e, 0x67, 0xd1, 0x3c, 0x14, 0x1b, 0x8f, 0x6c, 0x6b, 0x73, 0xf7, - 0x5e, 0xab, 0x3c, 0x53, 0xfb, 0xa5, 0x01, 0x25, 0xe1, 0x70, 0x27, 0xea, 0xc7, 0x13, 0x41, 0x1a, - 0x5f, 0x3d, 0x48, 0x64, 0xaa, 0xc3, 0x2b, 0x9b, 0xd8, 0xeb, 0x9a, 0x85, 0xb0, 0xab, 0xfd, 0x1c, - 0x16, 0xf7, 0x49, 0xec, 0xa5, 0x2e, 0x26, 0x6d, 0xec, 0x78, 0x98, 0xa0, 0x9b, 0x50, 0xe8, 0x07, - 0xf1, 0x11, 0xbf, 0x08, 0x44, 0x82, 0x1a, 0x15, 0xbe, 0xc4, 0xdf, 0x5f, 0xac, 0xe6, 0xb7, 0x83, - 0xf8, 0xa8, 0xd3, 0x3c, 0x1d, 0x8e, 0xac, 0x3c, 0x37, 0xec, 0x78, 0x6f, 0x71, 0xfb, 0xd4, 0xfe, - 0x68, 0xc0, 0xbc, 0x76, 0xa0, 0xe9, 0x30, 0x07, 0x7d, 0x03, 0x4a, 0xc4, 0x39, 0xb2, 0x7b, 0x27, - 0x0c, 0x53, 0xb5, 0x43, 0x45, 0xe2, 0x1c, 0x35, 0xf8, 0x1c, 0x59, 0x50, 0x0c, 0x31, 0x73, 0xf8, - 0x83, 0x51, 0x35, 0x94, 0x1b, 0xe7, 0x17, 0xaa, 0x85, 0xc3, 0x98, 0x61, 0x4d, 0xfe, 0x40, 0xe1, - 0x74, 0xc6, 0x34, 0x0f, 0xfa, 0x0e, 0x2c, 0x46, 0x69, 0x68, 0xe3, 0x30, 0x61, 0x27, 0x36, 0x89, - 0x8f, 0xe8, 0xc4, 0x76, 0xcf, 0x47, 0x69, 0xd8, 0xe2, 0x2a, 0x2b, 0x3e, 0xa2, 0xb5, 0xe7, 0x06, - 0x2c, 0x8d, 0x08, 0x29, 0x75, 0x06, 0x18, 0xdd, 0x85, 0xfc, 0xa1, 0xc8, 0x9c, 0xf0, 0x76, 0x6e, - 0xfd, 0xfa, 0xf9, 0x1e, 0x4d, 0x66, 0xda, 0x52, 0x38, 0xb4, 0x09, 0x79, 0x76, 0x92, 0xc8, 0xf2, - 0xe6, 0x31, 0x5d, 0x3b, 0x9f, 0x61, 0x58, 0x30, 0xba, 0x1a, 0x25, 0x10, 0xdd, 0x85, 0x9c, 0x48, - 0x8a, 0x7c, 0x09, 0xbe, 0xf7, 0x66, 0x17, 0x9a, 0xa3, 0x54, 0x08, 0x64, 0xed, 0xcf, 0x8b, 0x70, - 0xe5, 0xec, 0x8c, 0xa1, 0x1f, 0x03, 0xc8, 0xa6, 0xea, 0x47, 0xfd, 0x58, 0x45, 0xf9, 0xfd, 0x8b, - 0xe6, 0x5d, 0xb6, 0x09, 0xee, 0x3a, 0x6d, 0x67, 0xac, 0x12, 0xd1, 0x33, 0x74, 0x1b, 0x66, 0xe5, - 0x2b, 0x56, 0x96, 0xec, 0xea, 0xf9, 0xc4, 0xe2, 0xe5, 0xda, 0xce, 0x58, 0xd2, 0x1e, 0x7d, 0x06, - 0xc0, 0x9f, 0xdd, 0xd8, 0x1e, 0x8b, 0x7c, 0xe3, 0xc2, 0x6e, 0x1d, 0x70, 0x0a, 0x9e, 0x0d, 0xee, - 0x15, 0xd3, 0x13, 0xf4, 0x09, 0x5c, 0x0a, 0xb0, 0xd3, 0xb7, 0xd9, 0x71, 0x64, 0xf7, 0xfd, 0xc8, - 0x09, 0x6c, 0xca, 0x1c, 0x86, 0xd5, 0x5d, 0x32, 0x7e, 0x36, 0xf5, 0x4b, 0x67, 0x07, 0x3b, 0xfd, - 0x83, 0xe3, 0x68, 0x9b, 0x1b, 0x77, 0xb9, 0x6d, 0x3b, 0x63, 0x2d, 0x07, 0xd3, 0x42, 0x64, 0x41, - 0x81, 0xc4, 0x47, 0x76, 0x94, 0x86, 0xe2, 0xa6, 0x98, 0x5b, 0xbf, 0x7d, 0xf1, 0x4c, 0xc6, 0x47, - 0xbb, 0x69, 0xd8, 0xce, 0x58, 0x79, 0x22, 0x46, 0x28, 0x84, 0x32, 0x75, 0xc2, 0x24, 0xc0, 0xc4, - 0x4e, 0x48, 0x3c, 0xe0, 0x6f, 0xf8, 0x4a, 0x41, 0x90, 0xdf, 0xbd, 0x30, 0x79, 0x57, 0x12, 0xed, - 0x2b, 0x9e, 0x76, 0xc6, 0x5a, 0xa2, 0x93, 0x22, 0x74, 0x00, 0x85, 0x10, 0x33, 0xe2, 0xbb, 0xb4, - 0x52, 0x14, 0xab, 0x7c, 0x78, 0xe1, 0x55, 0x1e, 0x48, 0x7c, 0x3b, 0x63, 0x69, 0x2a, 0xf4, 0x0b, - 0x03, 0xde, 0xe9, 0xa5, 0xc1, 0x63, 0x1e, 0x82, 0x8b, 0x29, 0x8d, 0xc7, 0x82, 0x29, 0x89, 0x65, - 0xb6, 0x2f, 0xbc, 0x4c, 0x23, 0x0d, 0x1e, 0xef, 0x6b, 0xba, 0xb1, 0x90, 0x2e, 0xf7, 0xce, 0x52, - 0x54, 0xf7, 0x00, 0x46, 0x55, 0x8a, 0x36, 0xa7, 0xca, 0x9e, 0x1f, 0xcd, 0xab, 0x67, 0x6c, 0xfc, - 0x10, 0xa2, 0xaf, 0xba, 0x61, 0x6d, 0x57, 0x7f, 0x0a, 0xa5, 0x61, 0x7d, 0xa1, 0x2e, 0x2c, 0xb9, - 0x71, 0x10, 0x60, 0x97, 0xa9, 0xdf, 0x87, 0xfa, 0xc2, 0x1f, 0xaf, 0x26, 0xfe, 0xab, 0xd0, 0x54, - 0xbf, 0x26, 0x4d, 0x4b, 0xfd, 0x9a, 0x1c, 0xbb, 0xd5, 0x17, 0x87, 0x14, 0x5c, 0x48, 0xab, 0x47, - 0x90, 0x97, 0xe5, 0x80, 0x3e, 0x80, 0x12, 0xc5, 0x91, 0x87, 0x89, 0xee, 0xdc, 0xa5, 0x46, 0x79, - 0xd8, 0x84, 0x85, 0x42, 0x34, 0x5e, 0x39, 0xf2, 0xf8, 0x2f, 0x27, 0x5d, 0x87, 0xd9, 0xf1, 0xeb, - 0x4d, 0x95, 0xd4, 0x2a, 0x14, 0x03, 0x87, 0x32, 0x3b, 0xa4, 0x03, 0x71, 0xb4, 0xf4, 0x8b, 0xa6, - 0xc0, 0xa5, 0x0f, 0xe8, 0xa0, 0x4a, 0x61, 0x69, 0xaa, 0x54, 0xd0, 0x77, 0x61, 0x91, 0xf7, 0x4f, - 0xbd, 0x81, 0x58, 0xba, 0x91, 0x53, 0xc8, 0x05, 0xae, 0xdb, 0xd7, 0x2a, 0x74, 0x0b, 0xd0, 0xa1, - 0x4f, 0x59, 0x3c, 0x20, 0x4e, 0x68, 0xab, 0x67, 0x92, 0xbc, 0x3c, 0xf4, 0x52, 0xcb, 0x43, 0x7d, - 0x53, 0xa9, 0xab, 0x7f, 0xca, 0xc1, 0xe5, 0x33, 0xf7, 0x14, 0x6d, 0xf3, 0xe4, 0x72, 0x77, 0xa6, - 0x93, 0xfb, 0xce, 0x19, 0x3b, 0x36, 0x99, 0x4f, 0x85, 0x12, 0xf9, 0x44, 0xbf, 0x36, 0x00, 0x8d, - 0x88, 0xfa, 0xc4, 0xd1, 0xbf, 0x1d, 0x39, 0xd7, 0x4f, 0xbe, 0x9e, 0x02, 0x34, 0xb7, 0xf4, 0x02, - 0xdb, 0x8a, 0xbf, 0x15, 0x31, 0x72, 0x62, 0x2d, 0xbb, 0xd3, 0x72, 0xc4, 0x00, 0x08, 0xa6, 0x69, - 0x88, 0xed, 0x24, 0xd6, 0x6f, 0xb3, 0x87, 0x5f, 0x93, 0x17, 0x96, 0x20, 0xde, 0x8f, 0xa9, 0x5c, - 0xbd, 0x44, 0xf4, 0x1c, 0xb5, 0xa0, 0xac, 0x8f, 0x9e, 0xed, 0x61, 0xe6, 0xf8, 0x01, 0x55, 0x8d, - 0xef, 0x92, 0x29, 0x3f, 0x4e, 0x98, 0xfa, 0xe3, 0x84, 0xb9, 0x19, 0x9d, 0xe8, 0xe7, 0xb1, 0xc6, - 0x34, 0x25, 0xa4, 0xda, 0x84, 0x2b, 0x67, 0x47, 0xca, 0x1f, 0x5b, 0x8f, 0xf1, 0x89, 0x28, 0x8f, - 0x59, 0x8b, 0x0f, 0xf9, 0xa3, 0xec, 0xa9, 0x13, 0xa4, 0xf2, 0xe1, 0x92, 0xb5, 0xe4, 0x64, 0x23, - 0xfb, 0xa1, 0x51, 0xfd, 0x01, 0x2c, 0x4e, 0x7a, 0xfa, 0x26, 0xf4, 0xcc, 0x38, 0xfa, 0x63, 0x28, - 0xa8, 0x5e, 0x83, 0xae, 0x01, 0x88, 0x57, 0x85, 0x4d, 0xb0, 0x23, 0x4b, 0x73, 0x46, 0x9f, 0x58, - 0x21, 0xb7, 0xb0, 0xe3, 0xa1, 0x77, 0xa1, 0x24, 0x6a, 0x58, 0xd8, 0x64, 0xc7, 0x6c, 0x8a, 0x5c, - 0xcc, 0x4d, 0x1a, 0x05, 0xb5, 0xd8, 0xfd, 0x5c, 0x31, 0x5f, 0x2e, 0xd4, 0x7e, 0x6b, 0x40, 0xa5, - 0xe9, 0x53, 0xd6, 0xfd, 0x78, 0xe7, 0x47, 0xf2, 0xe3, 0xc8, 0xbd, 0x98, 0x52, 0x3f, 0x11, 0x97, - 0xdb, 0x8d, 0xc9, 0xcf, 0x28, 0x0b, 0x8d, 0x2b, 0x9c, 0xec, 0xcb, 0x17, 0xab, 0x8b, 0x93, 0x90, - 0xd1, 0x87, 0x95, 0x36, 0x5c, 0x0a, 0xfd, 0xc8, 0x76, 0x5c, 0x17, 0x27, 0xbc, 0x04, 0x35, 0x3c, - 0xfb, 0x5a, 0x38, 0x0a, 0xfd, 0x68, 0x53, 0x41, 0x94, 0xac, 0x76, 0x1b, 0xfe, 0x4f, 0x59, 0x35, - 0x89, 0xe3, 0x47, 0x7e, 0x34, 0x10, 0x2e, 0xad, 0x41, 0xd1, 0x53, 0x73, 0xe1, 0x93, 0x3e, 0x6e, - 0x43, 0x69, 0xe3, 0x83, 0x67, 0xff, 0x5c, 0xc9, 0x3c, 0x3b, 0x5d, 0x31, 0xbe, 0x38, 0x5d, 0x31, - 0x9e, 0x9f, 0xae, 0x18, 0xff, 0x38, 0x5d, 0x31, 0x7e, 0xf3, 0x72, 0x25, 0xf3, 0xc5, 0xcb, 0x95, - 0xcc, 0xf3, 0x97, 0x2b, 0x99, 0x4f, 0xe7, 0xc6, 0xbe, 0x0b, 0xfe, 0x37, 0x00, 0x00, 0xff, 0xff, - 0xa9, 0xdd, 0x16, 0xec, 0x29, 0x14, 0x00, 0x00, +func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_a11511dae3716e94) } + +var fileDescriptor_data_a11511dae3716e94 = []byte{ + // 2065 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcb, 0x73, 0x1b, 0xc7, + 0xd1, 0xc7, 0x82, 0x20, 0x1e, 0xcd, 0x17, 0x38, 0x9f, 0x2c, 0xe3, 0x43, 0x14, 0x92, 0x86, 0x52, + 0x2e, 0xc5, 0xb1, 0x17, 0x12, 0x75, 0x90, 0xc3, 0x24, 0x25, 0x11, 0x04, 0x28, 0x40, 0xa6, 0x48, + 0x7a, 0x41, 0xc5, 0x25, 0x3b, 0xae, 0xad, 0xc5, 0xee, 0x00, 0x5c, 0x6b, 0x5f, 0x9a, 0x99, 0x15, + 0xc9, 0x4b, 0x2a, 0x55, 0xc9, 0x31, 0x87, 0x9c, 0x52, 0x39, 0x3a, 0x97, 0x5c, 0x72, 0xc8, 0xbf, + 0x90, 0x53, 0xa2, 0x53, 0xca, 0x47, 0x55, 0x0e, 0xaa, 0x84, 0xba, 0xe4, 0x98, 0x6b, 0x7c, 0x4a, + 0xcd, 0x0b, 0x2f, 0x91, 0x52, 0x68, 0xf9, 0x02, 0xcc, 0xf4, 0xf4, 0xef, 0x37, 0xdd, 0x3d, 0x3d, + 0x3d, 0x33, 0x0b, 0x55, 0xfa, 0x38, 0xa8, 0xe3, 0x63, 0xec, 0xfa, 0x51, 0x9f, 0x38, 0x49, 0xaf, + 0xee, 0x39, 0xcc, 0x31, 0x13, 0x12, 0xb3, 0x18, 0x55, 0xdc, 0xd8, 0x7d, 0x44, 0x62, 0xc7, 0x3d, + 0x34, 0xe9, 0xe3, 0xc0, 0xf4, 0x7c, 0xca, 0xe8, 0xe3, 0x80, 0xa4, 0x51, 0xf5, 0xf2, 0x17, 0x71, + 0x8f, 0xd6, 0xf9, 0x4f, 0xd2, 0x13, 0x7f, 0x12, 0x51, 0x5d, 0x16, 0xda, 0x49, 0xaf, 0xee, 0x24, + 0xbe, 0x12, 0x21, 0x2d, 0x1a, 0x11, 0x57, 0x2f, 0x69, 0x19, 0x26, 0x24, 0x26, 0x1a, 0xfc, 0x96, + 0xec, 0x4d, 0x8b, 0x57, 0xb9, 0x85, 0xc9, 0xe0, 0xc8, 0x27, 0xb8, 0x9e, 0x0c, 0xc4, 0xd0, 0xa4, + 0xc2, 0x55, 0xae, 0xe0, 0x3a, 0xcc, 0x09, 0xe2, 0x41, 0xdd, 0xc3, 0xd4, 0x4d, 0x7a, 0x75, 0xca, + 0x48, 0xea, 0xb2, 0x94, 0x60, 0x4f, 0x29, 0xbd, 0x7b, 0x86, 0x12, 0x8e, 0xdc, 0xd8, 0xc3, 0x9e, + 0xed, 0x39, 0x2c, 0x0d, 0xb5, 0x11, 0x5c, 0x8f, 0x9d, 0x24, 0x98, 0xca, 0x5f, 0x25, 0x7e, 0x2f, + 0x65, 0x7e, 0x50, 0x67, 0xc4, 0x71, 0xfd, 0x68, 0xa0, 0xff, 0x93, 0x5e, 0x9d, 0x60, 0x37, 0x26, + 0x9c, 0x83, 0x26, 0x4e, 0xa4, 0xbd, 0x1b, 0xc4, 0x83, 0x58, 0x34, 0xeb, 0xbc, 0xa5, 0xa4, 0x15, + 0xc1, 0x70, 0x18, 0xb8, 0x75, 0xe6, 0x87, 0x98, 0x32, 0x27, 0x4c, 0xd4, 0xc8, 0xff, 0x0f, 0xe2, + 0x78, 0x10, 0xe0, 0xba, 0xe8, 0xf5, 0xd2, 0x7e, 0xdd, 0x89, 0x4e, 0xe4, 0x50, 0xed, 0x53, 0x98, + 0x6d, 0x71, 0x57, 0xd1, 0x6d, 0x80, 0x7e, 0x1a, 0x04, 0xb6, 0x70, 0xbc, 0x32, 0xb3, 0x66, 0x5c, + 0x9b, 0x5b, 0x5f, 0x33, 0x47, 0xeb, 0xa3, 0x43, 0x67, 0xb6, 0xa4, 0x4f, 0x02, 0x65, 0x95, 0x38, + 0x46, 0x34, 0x37, 0x72, 0xbf, 0xfb, 0x72, 0x35, 0x73, 0x2f, 0x57, 0x34, 0xca, 0xd9, 0x7b, 0xb9, + 0x62, 0xb6, 0x3c, 0x53, 0xdb, 0x07, 0x68, 0x1d, 0x27, 0x04, 0x53, 0xea, 0xc7, 0x11, 0x5a, 0x81, + 0xc2, 0x13, 0x4c, 0x78, 0xb3, 0x62, 0xac, 0x19, 0xd7, 0x4a, 0x8d, 0xdc, 0xd3, 0xe7, 0xab, 0x19, + 0x4b, 0x0b, 0x51, 0x05, 0x72, 0xf8, 0x38, 0x21, 0x95, 0xec, 0xd8, 0xa0, 0x90, 0x6c, 0x14, 0x39, + 0xf3, 0xbf, 0x7f, 0xbf, 0x9a, 0xa9, 0xfd, 0x2a, 0x0b, 0xc5, 0x3d, 0xe2, 0x61, 0xe2, 0x47, 0x03, + 0xd4, 0x81, 0x82, 0x1b, 0x07, 0x69, 0x18, 0xd1, 0x8a, 0xb1, 0x36, 0x73, 0x6d, 0x6e, 0xfd, 0xfb, + 0xe6, 0x79, 0xe9, 0x64, 0x6a, 0x90, 0xb9, 0x25, 0x10, 0x7a, 0x6e, 0x85, 0xaf, 0xfe, 0xc1, 0x80, + 0xbc, 0x1c, 0x41, 0xdf, 0x15, 0xac, 0xb6, 0xef, 0x1d, 0x0b, 0x33, 0x17, 0x94, 0x6a, 0xde, 0x8d, + 0x83, 0x8e, 0x77, 0x8c, 0x3e, 0x81, 0x92, 0xe7, 0x13, 0xec, 0x32, 0xee, 0x07, 0x37, 0x75, 0x71, + 0xfd, 0xe6, 0xff, 0x3c, 0xad, 0xd9, 0xd4, 0x50, 0xc5, 0x3a, 0xe2, 0xaa, 0xad, 0x40, 0x69, 0x38, + 0x8a, 0x0a, 0x30, 0xb3, 0xd9, 0xdd, 0x2a, 0x67, 0x50, 0x11, 0x72, 0xcd, 0x56, 0x77, 0xab, 0x6c, + 0x6c, 0xe4, 0xfe, 0xf5, 0xe5, 0xaa, 0xfa, 0xad, 0xfd, 0x35, 0x0b, 0xa8, 0xcb, 0x08, 0x76, 0xc2, + 0x56, 0xe4, 0x25, 0xb1, 0x1f, 0xb1, 0x6e, 0x82, 0x5d, 0xf4, 0x11, 0xe4, 0x78, 0x46, 0x09, 0xbb, + 0x17, 0xd7, 0x6f, 0x9c, 0x6f, 0xd6, 0xcb, 0x58, 0xf3, 0xe0, 0x24, 0xc1, 0x3a, 0xe8, 0x9c, 0x04, + 0xfd, 0x10, 0x4a, 0x54, 0xa8, 0xd9, 0xbe, 0x27, 0x1c, 0x9d, 0x6d, 0x5c, 0xe1, 0xc3, 0xa7, 0xcf, + 0x57, 0x8b, 0x12, 0xdf, 0x69, 0x7e, 0x3d, 0xd6, 0xb6, 0x8a, 0x52, 0xbd, 0xe3, 0xa1, 0xc7, 0xb0, + 0xc8, 0x1c, 0x32, 0xc0, 0xcc, 0x8e, 0x62, 0x0f, 0x73, 0x7c, 0x4e, 0xe0, 0x3f, 0x52, 0xf8, 0xf9, + 0x03, 0x31, 0xba, 0x1b, 0x7b, 0x58, 0x70, 0xdc, 0x1c, 0xf8, 0xec, 0x30, 0xed, 0x99, 0x6e, 0x1c, + 0xd6, 0x87, 0x36, 0x7b, 0xbd, 0x51, 0xbb, 0x9e, 0x3c, 0x1a, 0xd4, 0xd5, 0x8e, 0x36, 0x25, 0xcc, + 0x9a, 0x67, 0x23, 0x12, 0xaf, 0x76, 0x1d, 0x72, 0xdc, 0x03, 0x54, 0x82, 0xd9, 0x9d, 0xbd, 0xad, + 0xcd, 0x9d, 0x72, 0x06, 0x01, 0xe4, 0xad, 0xd6, 0xfd, 0xbd, 0x83, 0x56, 0xd9, 0x40, 0xcb, 0xb0, + 0xd0, 0x7d, 0xb8, 0xbb, 0x65, 0x5b, 0xad, 0xee, 0xfe, 0xde, 0x6e, 0xb7, 0x25, 0x52, 0x74, 0xa6, + 0x9c, 0xab, 0xfd, 0x2d, 0x0b, 0x0b, 0x9d, 0x28, 0x49, 0x59, 0xf7, 0x24, 0x72, 0x45, 0x10, 0xb7, + 0x27, 0x82, 0xf8, 0xfe, 0xf9, 0x41, 0x9c, 0x80, 0xbd, 0x1c, 0xbf, 0x26, 0x14, 0x63, 0xb5, 0xfa, + 0x22, 0x7c, 0x73, 0xeb, 0xb5, 0xd7, 0xe7, 0x89, 0x62, 0x18, 0x22, 0xd1, 0x0e, 0x14, 0x64, 0x58, + 0x69, 0x65, 0x46, 0xe4, 0xf8, 0xfb, 0x17, 0x59, 0x55, 0x9d, 0xe6, 0x8a, 0x02, 0xfd, 0x04, 0xe6, + 0x65, 0xc6, 0xdb, 0xa2, 0xf2, 0x54, 0x72, 0x82, 0xb2, 0x3a, 0x45, 0x49, 0x71, 0x68, 0xca, 0xca, + 0x74, 0x60, 0xcd, 0x49, 0x7d, 0xee, 0x1d, 0xad, 0xd5, 0x54, 0x90, 0x17, 0xa0, 0xf4, 0x60, 0x77, + 0xcf, 0x6a, 0xb6, 0xac, 0x56, 0xb3, 0x9c, 0x41, 0x73, 0x50, 0xd0, 0x1d, 0xa3, 0xf6, 0x9f, 0x3c, + 0x94, 0xf7, 0x52, 0x96, 0xa4, 0xcc, 0x8a, 0x53, 0x86, 0x89, 0x88, 0x69, 0x67, 0x22, 0xa6, 0xf5, + 0x57, 0xc4, 0x61, 0x0a, 0xf9, 0x72, 0x58, 0xc7, 0x02, 0x92, 0x7d, 0xf3, 0x80, 0xbc, 0x03, 0xf3, + 0x87, 0x0e, 0x3d, 0xb4, 0x75, 0x1d, 0xe1, 0x31, 0x5e, 0xb0, 0xe6, 0xb8, 0x4c, 0xee, 0x58, 0x8a, + 0x02, 0x58, 0x26, 0x4e, 0x34, 0xc0, 0x36, 0x11, 0x56, 0xd9, 0x34, 0xc1, 0xae, 0xc8, 0xe7, 0xb9, + 0xf5, 0x8d, 0x0b, 0x38, 0x62, 0x71, 0x8e, 0x51, 0x5f, 0x19, 0xb2, 0x44, 0x26, 0xc5, 0xe8, 0x06, + 0x2c, 0x7b, 0x3e, 0x75, 0x7a, 0x01, 0xb6, 0x7b, 0x69, 0xbf, 0x2f, 0xd3, 0x67, 0x76, 0xcd, 0xb8, + 0x56, 0x54, 0x88, 0xb2, 0x1a, 0x6e, 0xe8, 0xd1, 0xea, 0x5f, 0x66, 0x60, 0x69, 0x8a, 0x1d, 0x7d, + 0x06, 0xb3, 0xfc, 0xb8, 0xd0, 0x85, 0xf1, 0xf6, 0x37, 0x37, 0xd4, 0xec, 0x26, 0x8e, 0xae, 0x56, + 0x92, 0x93, 0x07, 0xcd, 0xc3, 0x7d, 0x27, 0x0d, 0x98, 0xed, 0x61, 0xca, 0x64, 0x71, 0xb0, 0xe6, + 0x94, 0xac, 0x89, 0x29, 0x43, 0x21, 0x94, 0xc4, 0xd1, 0xe7, 0x47, 0x03, 0x9d, 0xb8, 0x9d, 0x37, + 0xb0, 0x41, 0xae, 0x45, 0x4b, 0x31, 0xea, 0xda, 0x39, 0x9c, 0xa1, 0xfa, 0x04, 0x16, 0x27, 0x55, + 0xd0, 0x15, 0xc8, 0xcb, 0x35, 0x7d, 0xa9, 0x88, 0xf3, 0x1a, 0xbf, 0x0d, 0x45, 0x0d, 0x56, 0x35, + 0xfc, 0x7b, 0xd3, 0x7b, 0xe0, 0x71, 0xd0, 0x73, 0x28, 0x36, 0x9b, 0xfc, 0xe0, 0x9e, 0x9a, 0x78, + 0x88, 0xad, 0xee, 0x40, 0x8e, 0x87, 0x07, 0x5d, 0x82, 0x59, 0xca, 0x1c, 0xc2, 0xc4, 0x64, 0xf3, + 0x96, 0xec, 0xa0, 0x32, 0xcc, 0xe0, 0x48, 0xd6, 0xce, 0x79, 0x8b, 0x37, 0xb9, 0x55, 0x32, 0xf3, + 0xc4, 0xf9, 0x3a, 0xab, 0xad, 0x92, 0xb2, 0xda, 0x6d, 0xb5, 0xbd, 0xca, 0x30, 0xbf, 0xbf, 0xd9, + 0xed, 0xda, 0x07, 0x6d, 0x6b, 0xef, 0xc1, 0xdd, 0xb6, 0x2c, 0x65, 0xf7, 0x3b, 0x96, 0xb5, 0x67, + 0x95, 0x0d, 0xbe, 0xdb, 0x1a, 0x0f, 0xed, 0xf6, 0x66, 0xb7, 0x5d, 0xce, 0xa2, 0x79, 0x28, 0x36, + 0x1e, 0xda, 0xd6, 0xe6, 0xee, 0xdd, 0x56, 0x79, 0xa6, 0xf6, 0x4b, 0x03, 0x4a, 0xc2, 0xe0, 0x4e, + 0xd4, 0x8f, 0x27, 0x9c, 0x34, 0xbe, 0xb9, 0x93, 0xc8, 0x54, 0x9b, 0x57, 0x16, 0xb1, 0x57, 0x15, + 0x0b, 0xa1, 0x57, 0xfb, 0x39, 0x2c, 0xee, 0x93, 0xd8, 0x4b, 0x5d, 0x4c, 0xda, 0xd8, 0xf1, 0x30, + 0x41, 0x37, 0xa0, 0xd0, 0x0f, 0xe2, 0x23, 0x7e, 0x10, 0x88, 0x00, 0x35, 0x2a, 0x7c, 0x8a, 0xbf, + 0x3f, 0x5f, 0xcd, 0x6f, 0x07, 0xf1, 0x51, 0xa7, 0x79, 0x3a, 0x6c, 0x59, 0x79, 0xae, 0xd8, 0xf1, + 0xde, 0xe0, 0xf4, 0xa9, 0xfd, 0xd1, 0x80, 0x79, 0x6d, 0x40, 0xd3, 0x61, 0x0e, 0xfa, 0x0e, 0x94, + 0x88, 0x73, 0x64, 0xf7, 0x4e, 0x18, 0xa6, 0x6a, 0x85, 0x8a, 0xc4, 0x39, 0x6a, 0xf0, 0x3e, 0xb2, + 0xa0, 0x18, 0x62, 0xe6, 0xf0, 0xab, 0xa3, 0x2a, 0x28, 0xd7, 0xcf, 0x4f, 0x54, 0x0b, 0x87, 0x31, + 0xc3, 0x9a, 0xfc, 0xbe, 0xc2, 0xe9, 0x88, 0x69, 0x1e, 0xf4, 0x1e, 0x2c, 0x46, 0x69, 0x68, 0xe3, + 0x30, 0x61, 0x27, 0x36, 0x89, 0x8f, 0xe8, 0xc4, 0x72, 0xcf, 0x47, 0x69, 0xd8, 0xe2, 0x43, 0x56, + 0x7c, 0x44, 0x6b, 0xcf, 0x0c, 0x58, 0x1a, 0x11, 0x52, 0xea, 0x0c, 0x30, 0xba, 0x03, 0xf9, 0x43, + 0x11, 0x39, 0x61, 0xed, 0xdc, 0xfa, 0xb5, 0xf3, 0x2d, 0x9a, 0x8c, 0xb4, 0xa5, 0x70, 0x68, 0x13, + 0xf2, 0xec, 0x24, 0x91, 0xe9, 0xcd, 0x7d, 0xba, 0x7a, 0x3e, 0xc3, 0x30, 0x61, 0x74, 0x36, 0x4a, + 0x20, 0xba, 0x03, 0x39, 0x11, 0x14, 0x79, 0x13, 0x7c, 0xf7, 0xf5, 0x26, 0x34, 0x47, 0xa1, 0x10, + 0xc8, 0xda, 0x9f, 0x17, 0xe1, 0xf2, 0xd9, 0x11, 0x43, 0x3f, 0x03, 0x90, 0x45, 0xd5, 0x8f, 0xfa, + 0xb1, 0xf2, 0xf2, 0x47, 0x17, 0x8d, 0xbb, 0x2c, 0x13, 0xdc, 0x74, 0xda, 0xce, 0x58, 0x25, 0xa2, + 0x7b, 0xe8, 0x16, 0xcc, 0xca, 0x5b, 0xac, 0x4c, 0xd9, 0xd5, 0xf3, 0x89, 0xc5, 0xcd, 0xb5, 0x9d, + 0xb1, 0xa4, 0x3e, 0xfa, 0x0c, 0x80, 0x5f, 0xbc, 0xb1, 0x3d, 0xe6, 0xf9, 0xc6, 0x85, 0xcd, 0x3a, + 0xe0, 0x14, 0x3c, 0x1a, 0xdc, 0x2a, 0xa6, 0x3b, 0xe8, 0x13, 0xb8, 0x14, 0x60, 0xa7, 0x6f, 0xb3, + 0xe3, 0xc8, 0xee, 0xfb, 0x91, 0x13, 0xd8, 0x94, 0x39, 0x0c, 0xab, 0xb3, 0x64, 0x7c, 0x6f, 0xea, + 0x9b, 0xce, 0x0e, 0x76, 0xfa, 0x07, 0xc7, 0xd1, 0x36, 0x57, 0xee, 0x72, 0xdd, 0x76, 0xc6, 0x5a, + 0x0e, 0xa6, 0x85, 0xc8, 0x82, 0x02, 0x89, 0x8f, 0xec, 0x28, 0x0d, 0xc5, 0x49, 0x31, 0xb7, 0x7e, + 0xeb, 0xe2, 0x91, 0x8c, 0x8f, 0x76, 0xd3, 0xb0, 0x9d, 0xb1, 0xf2, 0x44, 0xb4, 0x50, 0x08, 0x65, + 0xea, 0x84, 0x49, 0x80, 0x89, 0x9d, 0x90, 0x78, 0xc0, 0xef, 0xf0, 0x95, 0x82, 0x20, 0xbf, 0x73, + 0x61, 0xf2, 0xae, 0x24, 0xda, 0x57, 0x3c, 0xed, 0x8c, 0xb5, 0x44, 0x27, 0x45, 0xe8, 0x00, 0x0a, + 0x21, 0x66, 0xc4, 0x77, 0x69, 0xa5, 0x28, 0x66, 0xf9, 0xf0, 0xc2, 0xb3, 0xdc, 0x97, 0xf8, 0x76, + 0xc6, 0xd2, 0x54, 0xe8, 0x17, 0x06, 0xbc, 0xdd, 0x4b, 0x83, 0x47, 0xdc, 0x05, 0x17, 0x53, 0x1a, + 0x8f, 0x39, 0x53, 0x12, 0xd3, 0x6c, 0x5f, 0x78, 0x9a, 0x46, 0x1a, 0x3c, 0xda, 0xd7, 0x74, 0x63, + 0x2e, 0xbd, 0xd5, 0x3b, 0x6b, 0xa0, 0xba, 0x07, 0x30, 0xca, 0x52, 0xb4, 0x39, 0x95, 0xf6, 0x7c, + 0x6b, 0x5e, 0x39, 0x63, 0xe1, 0x87, 0x10, 0x7d, 0xd4, 0x0d, 0x73, 0xbb, 0xfa, 0x05, 0x94, 0x86, + 0xf9, 0x85, 0x3e, 0x87, 0x25, 0x37, 0x0e, 0x02, 0xec, 0x32, 0xf5, 0x3e, 0xd4, 0x07, 0xbe, 0x39, + 0x46, 0xca, 0x5f, 0x85, 0xa6, 0x7a, 0x4f, 0x9a, 0xc3, 0x77, 0xa5, 0x69, 0xa9, 0x77, 0xe5, 0xd8, + 0xf9, 0xbe, 0x38, 0x24, 0xe3, 0x42, 0x5a, 0x3d, 0x82, 0xbc, 0x4c, 0x0c, 0xf4, 0x01, 0x94, 0x28, + 0x8e, 0x3c, 0x4c, 0x74, 0x0d, 0x2f, 0x35, 0xca, 0xc3, 0x72, 0x2c, 0x06, 0x44, 0x09, 0x96, 0x2d, + 0x8f, 0xbf, 0xa1, 0x74, 0x46, 0x66, 0xc7, 0x0f, 0x3a, 0x95, 0x5c, 0xab, 0x50, 0x0c, 0x1c, 0xca, + 0xec, 0x90, 0x0e, 0xc4, 0x26, 0xd3, 0x77, 0x9b, 0x02, 0x97, 0xde, 0xa7, 0x83, 0x2a, 0x85, 0xa5, + 0xa9, 0xa4, 0x41, 0x3f, 0x80, 0x45, 0x5e, 0x49, 0xf5, 0x52, 0x62, 0x69, 0x46, 0x4e, 0x21, 0x17, + 0xf8, 0xd8, 0xbe, 0x1e, 0x42, 0x37, 0x01, 0x1d, 0xfa, 0x94, 0xc5, 0x03, 0xe2, 0x84, 0xb6, 0xba, + 0x30, 0xc9, 0x63, 0x44, 0x4f, 0xb5, 0x3c, 0x1c, 0x6f, 0xaa, 0xe1, 0xea, 0x9f, 0x72, 0xf0, 0xd6, + 0x99, 0xab, 0x8b, 0xb6, 0x79, 0x98, 0xb9, 0x39, 0xd3, 0x61, 0x7e, 0xfb, 0x8c, 0xb5, 0x9b, 0x8c, + 0xa7, 0x42, 0x89, 0x78, 0xa2, 0x5f, 0x1b, 0x80, 0x46, 0x44, 0x7d, 0xe2, 0xe8, 0x57, 0x24, 0xe7, + 0xfa, 0xfc, 0xdb, 0x49, 0x45, 0x73, 0x4b, 0x4f, 0xb0, 0xad, 0xf8, 0x5b, 0x11, 0x23, 0x27, 0xd6, + 0xb2, 0x3b, 0x2d, 0x47, 0x0c, 0x80, 0x60, 0x9a, 0x86, 0xd8, 0x4e, 0x62, 0x7d, 0x4b, 0x7b, 0xf0, + 0x2d, 0x59, 0x61, 0x09, 0xe2, 0xfd, 0x98, 0xca, 0xd9, 0x4b, 0x44, 0xf7, 0x51, 0x0b, 0xca, 0x7a, + 0x13, 0xda, 0x1e, 0x66, 0x8e, 0x1f, 0x50, 0x55, 0x02, 0x2f, 0x99, 0xf2, 0x33, 0x85, 0xa9, 0x3f, + 0x53, 0x98, 0x9b, 0xd1, 0x89, 0xbe, 0x28, 0x6b, 0x4c, 0x53, 0x42, 0xaa, 0x4d, 0xb8, 0x7c, 0xb6, + 0xa7, 0xfc, 0xda, 0xf5, 0x08, 0x9f, 0x88, 0xf4, 0x98, 0xb5, 0x78, 0x93, 0x5f, 0xcf, 0x9e, 0x38, + 0x41, 0x2a, 0xaf, 0x30, 0x59, 0x4b, 0x76, 0x36, 0xb2, 0x1f, 0x1a, 0xd5, 0x1f, 0xc3, 0xe2, 0xa4, + 0xa5, 0xaf, 0x43, 0xcf, 0x8c, 0xa3, 0x3f, 0x86, 0x82, 0xaa, 0x3a, 0xe8, 0x2a, 0x80, 0xb8, 0x5f, + 0xd8, 0x04, 0x3b, 0x32, 0x35, 0x67, 0xf4, 0xde, 0x15, 0x72, 0x0b, 0x3b, 0x1e, 0x7a, 0x07, 0x4a, + 0x22, 0x87, 0x85, 0x4e, 0x76, 0x4c, 0xa7, 0xc8, 0xc5, 0x5c, 0xa5, 0x51, 0x50, 0x93, 0xdd, 0xcb, + 0x15, 0xf3, 0xe5, 0x42, 0xed, 0xb7, 0x06, 0x54, 0x9a, 0x3e, 0x65, 0xdd, 0x8f, 0x77, 0x7e, 0x2a, + 0x3f, 0x93, 0xdc, 0x8d, 0x29, 0xf5, 0x13, 0x71, 0xcc, 0x5d, 0x9f, 0xfc, 0xa0, 0xb2, 0xd0, 0xb8, + 0xcc, 0xc9, 0xbe, 0x7e, 0xbe, 0xba, 0x38, 0x09, 0x19, 0x7d, 0x62, 0x69, 0xc3, 0xa5, 0xd0, 0x8f, + 0x6c, 0xc7, 0x75, 0x71, 0xc2, 0x53, 0x50, 0xc3, 0xb3, 0xaf, 0x84, 0xa3, 0xd0, 0x8f, 0x36, 0x15, + 0x44, 0xc9, 0x6a, 0xb7, 0xe0, 0xff, 0x94, 0x56, 0x93, 0x38, 0x7e, 0xe4, 0x47, 0x03, 0x61, 0xd2, + 0x1a, 0x14, 0x3d, 0xd5, 0x17, 0x36, 0xe9, 0xed, 0x36, 0x94, 0x36, 0x3e, 0x78, 0xfa, 0xcf, 0x95, + 0xcc, 0xd3, 0xd3, 0x15, 0xe3, 0xab, 0xd3, 0x15, 0xe3, 0xd9, 0xe9, 0x8a, 0xf1, 0x8f, 0xd3, 0x15, + 0xe3, 0x37, 0x2f, 0x56, 0x32, 0x5f, 0xbd, 0x58, 0xc9, 0x3c, 0x7b, 0xb1, 0x92, 0xf9, 0x74, 0x6e, + 0xec, 0x5b, 0xe1, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x7a, 0xbb, 0x53, 0xa9, 0x3d, 0x14, 0x00, + 0x00, } diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index e4b9eecc39e8..fd6c38338c73 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -28,7 +28,7 @@ import "sql/pgwire/pgerror/errors.proto"; import "sql/catalog/descpb/structured.proto"; import "sql/catalog/descpb/encoded_datum.proto"; import "sql/types/types.proto"; -import "util/tracing/recorded_span.proto"; +import "util/tracing/tracingpb/recorded_span.proto"; import "gogoproto/gogo.proto"; import "util/hlc/timestamp.proto"; import "google/protobuf/any.proto"; @@ -247,7 +247,7 @@ message RemoteProducerMetadata { repeated roachpb.RangeInfo range_info = 1 [(gogoproto.nullable) = false]; } message TraceData { - repeated util.tracing.RecordedSpan collected_spans = 1 [(gogoproto.nullable) = false]; + repeated util.tracing.tracingpb.RecordedSpan collected_spans = 1 [(gogoproto.nullable) = false]; } // RowNum is used to count the rows sent from a processor. It is used in tests // to check that metadata is propagated correctly. diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index d7181fbc5a1d..a88fe61f2d62 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/dustin/go-humanize" "github.com/gogo/protobuf/types" @@ -531,7 +531,7 @@ type FlowDiagram interface { ToURL() (string, url.URL, error) // AddSpans adds stats extracted from the input spans to the diagram. - AddSpans([]tracing.RecordedSpan) + AddSpans([]tracingpb.RecordedSpan) } type diagramData struct { @@ -555,7 +555,7 @@ func (d diagramData) ToURL() (string, url.URL, error) { } // AddSpans implements the FlowDiagram interface. -func (d *diagramData) AddSpans(spans []tracing.RecordedSpan) { +func (d *diagramData) AddSpans(spans []tracingpb.RecordedSpan) { processorStats, streamStats := extractStatsFromSpans(d.flowID, spans) for i := range d.Processors { if statDetails, ok := processorStats[int(d.Processors[i].processorID)]; ok { @@ -762,7 +762,7 @@ func encodeJSONToURL(json bytes.Buffer) (string, url.URL, error) { // and returns a map from that processor id to a slice of stat descriptions // that can be added to a plan. func extractStatsFromSpans( - flowID FlowID, spans []tracing.RecordedSpan, + flowID FlowID, spans []tracingpb.RecordedSpan, ) (processorStats, streamStats map[int][]string) { processorStats = make(map[int][]string) streamStats = make(map[int][]string) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index a03beb6a98f1..490c88efd5e8 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/jsonpb" ) @@ -126,8 +127,8 @@ func traceToJSON(trace tracing.Recording) (tree.Datum, string, error) { return d, str, nil } -func normalizeSpan(s tracing.RecordedSpan, trace tracing.Recording) tracing.NormalizedSpan { - var n tracing.NormalizedSpan +func normalizeSpan(s tracingpb.RecordedSpan, trace tracing.Recording) tracingpb.NormalizedSpan { + var n tracingpb.NormalizedSpan n.Operation = s.Operation n.StartTime = s.StartTime n.Duration = s.Duration diff --git a/pkg/sql/protoreflect/utils_test.go b/pkg/sql/protoreflect/utils_test.go index 11bff1025d26..be81f946c605 100644 --- a/pkg/sql/protoreflect/utils_test.go +++ b/pkg/sql/protoreflect/utils_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/gogo/protobuf/jsonpb" pbtypes "github.com/gogo/protobuf/types" "github.com/stretchr/testify/require" @@ -76,8 +76,8 @@ func TestMessageToJSONBRoundTrip(t *testing.T) { }, { // Message with embedded google.protobuf.Any message; // nested inside other message; with maps - pbname: "cockroach.util.tracing.RecordedSpan", - message: &tracing.RecordedSpan{ + pbname: "cockroach.util.tracing.tracingpb.RecordedSpan", + message: &tracingpb.RecordedSpan{ TraceID: 123, Tags: map[string]string{"one": "1", "two": "2", "three": "3"}, Stats: makeAny(t, &descpb.ColumnDescriptor{Name: "bogus stats"}), diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 88915e8e5da1..1567a6e246b0 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" opentracing "github.com/opentracing/opentracing-go" @@ -146,7 +147,7 @@ func eventInternal(sp opentracing.Span, el *ctxEventLog, isErr bool, entry Entry if sp != nil { // TODO(radu): pass tags directly to sp.LogKV when LightStep supports // that. - sp.LogFields(otlog.String(tracing.LogMessageField, msg)) + sp.LogFields(otlog.String(tracingpb.LogMessageField, msg)) // if isErr { // // TODO(radu): figure out a way to signal that this is an error. We // // could use a different "error" key (provided it shows up in diff --git a/pkg/util/tracing/test_utils.go b/pkg/util/tracing/test_utils.go index cf28bbdcd179..6f5e2874b35f 100644 --- a/pkg/util/tracing/test_utils.go +++ b/pkg/util/tracing/test_utils.go @@ -10,7 +10,11 @@ package tracing -import "strings" +import ( + "strings" + + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" +) // FindMsgInRecording returns the index of the first span containing msg in its // logs, or -1 if no span is found. @@ -24,7 +28,7 @@ func FindMsgInRecording(recording Recording, msg string) int { } // LogsContainMsg returns true if a span's logs contain the given message. -func LogsContainMsg(sp RecordedSpan, msg string) bool { +func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { for _, l := range sp.Logs { // NOTE: With out logs, each LogRecord has a single field ("event") and // value. diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 6f44fbe034cb..b4941d023ada 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" opentracing "github.com/opentracing/opentracing-go" @@ -774,7 +775,7 @@ func StartSnowballTrace( // Note: this test function is in this file because it needs to be used by // both tests in the tracing package and tests outside of it, and the function // itself depends on tracing. -func TestingCheckRecordedSpans(recSpans []RecordedSpan, expected string) error { +func TestingCheckRecordedSpans(recSpans []tracingpb.RecordedSpan, expected string) error { expected = strings.TrimSpace(expected) var rows []string row := func(format string, args ...interface{}) { diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index e5128891926a..056c6851ee93 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" proto "github.com/gogo/protobuf/proto" @@ -139,7 +140,7 @@ type crdbSpan struct { // started recording. children []*span // remoteSpan contains the list of remote child spans manually imported. - remoteSpans []RecordedSpan + remoteSpans []tracingpb.RecordedSpan } // tags are only set when recording. These are tags that have been added to @@ -315,7 +316,7 @@ func IsRecordable(os opentracing.Span) bool { // Recording represents a group of RecordedSpans, as returned by GetRecording. // Spans are sorted by StartTime. -type Recording []RecordedSpan +type Recording []tracingpb.RecordedSpan // GetRecording retrieves the current recording, if the span has recording // enabled. This can be called while spans that are part of the recording are @@ -423,13 +424,13 @@ func (r Recording) String() string { } // OrphanSpans returns the spans with parents missing from the recording. -func (r Recording) OrphanSpans() []RecordedSpan { +func (r Recording) OrphanSpans() []tracingpb.RecordedSpan { spanIDs := make(map[uint64]struct{}) for _, sp := range r { spanIDs[sp.SpanID] = struct{}{} } - var orphans []RecordedSpan + var orphans []tracingpb.RecordedSpan for i, sp := range r { if i == 0 { // The first span can be a root span. Note that any other root span will @@ -460,20 +461,20 @@ func (r Recording) FindLogMessage(pattern string) (string, bool) { // FindSpan returns the span with the given operation. The bool retval is false // if the span is not found. -func (r Recording) FindSpan(operation string) (RecordedSpan, bool) { +func (r Recording) FindSpan(operation string) (tracingpb.RecordedSpan, bool) { for _, sp := range r { if sp.Operation == operation { return sp, true } } - return RecordedSpan{}, false + return tracingpb.RecordedSpan{}, false } // visitSpan returns the log messages for sp, and all of sp's children. // // All messages from a span are kept together. Sibling spans are ordered within // the parent in their start order. -func (r Recording) visitSpan(sp RecordedSpan, depth int) []traceLogData { +func (r Recording) visitSpan(sp tracingpb.RecordedSpan, depth int) []traceLogData { ownLogs := make([]traceLogData, 0, len(sp.Logs)+1) conv := func(l opentracing.LogRecord, ref time.Time) traceLogData { @@ -594,7 +595,7 @@ func (r Recording) ToJaegerJSON(stmt string) (string, error) { // getProcessID figures out what "process" a span belongs to. It looks for an // "node: " tag. The processes map is populated with an entry for every // node present in the trace. - getProcessID := func(sp RecordedSpan) jaegerjson.ProcessID { + getProcessID := func(sp tracingpb.RecordedSpan) jaegerjson.ProcessID { node := "unknown node" for k, v := range sp.Tags { if k == "node" { @@ -684,7 +685,7 @@ type TraceCollection struct { // ImportRemoteSpans adds RecordedSpan data to the recording of the given span; // these spans will be part of the result of GetRecording. Used to import // recorded traces from other nodes. -func ImportRemoteSpans(os opentracing.Span, remoteSpans []RecordedSpan) error { +func ImportRemoteSpans(os opentracing.Span, remoteSpans []tracingpb.RecordedSpan) error { s := os.(*span) if !s.crdb.isRecording() { return errors.New("adding Raw Spans to a span that isn't recording") @@ -823,7 +824,7 @@ func (s *span) LogFields(fields ...otlog.Field) { // TODO(radu): when LightStep supports arbitrary fields, we should make // the formatting of the message consistent with that. Until then we treat // legacy events that just have an "event" key specially. - if len(fields) == 1 && fields[0].Key() == LogMessageField { + if len(fields) == 1 && fields[0].Key() == tracingpb.LogMessageField { s.netTr.LazyPrintf("%s", fields[0].Value()) } else { var buf bytes.Buffer @@ -898,12 +899,12 @@ func (s *span) Tracer() opentracing.Tracer { // LogEvent is part of the opentracing.Span interface. Deprecated. func (s *span) LogEvent(event string) { - s.LogFields(otlog.String(LogMessageField, event)) + s.LogFields(otlog.String(tracingpb.LogMessageField, event)) } // LogEventWithPayload is part of the opentracing.Span interface. Deprecated. func (s *span) LogEventWithPayload(event string, payload interface{}) { - s.LogFields(otlog.String(LogMessageField, event), otlog.Object("payload", payload)) + s.LogFields(otlog.String(tracingpb.LogMessageField, event), otlog.Object("payload", payload)) } // Log is part of the opentracing.Span interface. Deprecated. @@ -913,8 +914,8 @@ func (s *span) Log(data opentracing.LogData) { // getRecordingLocked returns the span's recording. This does not include // children. -func (s *span) getRecordingLocked() RecordedSpan { - rs := RecordedSpan{ +func (s *span) getRecordingLocked() tracingpb.RecordedSpan { + rs := tracingpb.RecordedSpan{ TraceID: s.crdb.TraceID, SpanID: s.crdb.SpanID, ParentSpanID: s.crdb.parentSpanID, @@ -966,12 +967,12 @@ func (s *span) getRecordingLocked() RecordedSpan { addTag(k, fmt.Sprint(v)) } } - rs.Logs = make([]LogRecord, len(s.crdb.mu.recording.recordedLogs)) + rs.Logs = make([]tracingpb.LogRecord, len(s.crdb.mu.recording.recordedLogs)) for i, r := range s.crdb.mu.recording.recordedLogs { rs.Logs[i].Time = r.Timestamp - rs.Logs[i].Fields = make([]LogRecord_Field, len(r.Fields)) + rs.Logs[i].Fields = make([]tracingpb.LogRecord_Field, len(r.Fields)) for j, f := range r.Fields { - rs.Logs[i].Fields[j] = LogRecord_Field{ + rs.Logs[i].Fields[j] = tracingpb.LogRecord_Field{ Key: f.Key(), Value: fmt.Sprint(f.Value()), } diff --git a/pkg/util/tracing/tracer_span_test.go b/pkg/util/tracing/tracer_span_test.go index 4e7b85beb954..ca00d3c4ed2d 100644 --- a/pkg/util/tracing/tracer_span_test.go +++ b/pkg/util/tracing/tracer_span_test.go @@ -16,6 +16,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" @@ -29,7 +30,7 @@ func TestRecordingString(t *testing.T) { root := tr.StartSpan("root", Recordable) rootSp := root.(*span) StartRecording(root, SnowballRecording) - root.LogFields(otlog.String(LogMessageField, "root 1")) + root.LogFields(otlog.String(tracingpb.LogMessageField, "root 1")) // Hackily fix the timing on the first log message, so that we can check it later. rootSp.crdb.mu.recording.recordedLogs[0].Timestamp = rootSp.crdb.startTime.Add(time.Millisecond) // Sleep a bit so that everything that comes afterwards has higher timestamps @@ -41,8 +42,8 @@ func TestRecordingString(t *testing.T) { require.NoError(t, err) wireContext, err := tr2.Extract(opentracing.HTTPHeaders, carrier) remoteChild := tr2.StartSpan("remote child", opentracing.FollowsFrom(wireContext)) - root.LogFields(otlog.String(LogMessageField, "root 2")) - remoteChild.LogFields(otlog.String(LogMessageField, "remote child 1")) + root.LogFields(otlog.String(tracingpb.LogMessageField, "root 2")) + remoteChild.LogFields(otlog.String(tracingpb.LogMessageField, "remote child 1")) require.NoError(t, err) remoteChild.Finish() remoteRec := GetRecording(remoteChild) @@ -50,14 +51,14 @@ func TestRecordingString(t *testing.T) { require.NoError(t, err) root.Finish() - root.LogFields(otlog.String(LogMessageField, "root 3")) + root.LogFields(otlog.String(tracingpb.LogMessageField, "root 3")) ch2 := StartChildSpan("local child", root, nil /* logTags */, false /* separateRecording */) - root.LogFields(otlog.String(LogMessageField, "root 4")) - ch2.LogFields(otlog.String(LogMessageField, "local child 1")) + root.LogFields(otlog.String(tracingpb.LogMessageField, "root 4")) + ch2.LogFields(otlog.String(tracingpb.LogMessageField, "local child 1")) ch2.Finish() - root.LogFields(otlog.String(LogMessageField, "root 5")) + root.LogFields(otlog.String(tracingpb.LogMessageField, "root 5")) root.Finish() rec := GetRecording(root) diff --git a/pkg/util/tracing/recorded_span.go b/pkg/util/tracing/tracingpb/recorded_span.go similarity index 98% rename from pkg/util/tracing/recorded_span.go rename to pkg/util/tracing/tracingpb/recorded_span.go index 4d6c99646356..1996f898bec7 100644 --- a/pkg/util/tracing/recorded_span.go +++ b/pkg/util/tracing/tracingpb/recorded_span.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tracing +package tracingpb import ( "fmt" diff --git a/pkg/util/tracing/recorded_span.pb.go b/pkg/util/tracing/tracingpb/recorded_span.pb.go similarity index 90% rename from pkg/util/tracing/recorded_span.pb.go rename to pkg/util/tracing/tracingpb/recorded_span.pb.go index 87d083b71458..01525c2d0bfb 100644 --- a/pkg/util/tracing/recorded_span.pb.go +++ b/pkg/util/tracing/tracingpb/recorded_span.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: util/tracing/recorded_span.proto +// source: util/tracing/tracingpb/recorded_span.proto -package tracing +package tracingpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" @@ -39,7 +39,7 @@ func (m *LogRecord) Reset() { *m = LogRecord{} } func (m *LogRecord) String() string { return proto.CompactTextString(m) } func (*LogRecord) ProtoMessage() {} func (*LogRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_ec83bd4619f89f60, []int{0} + return fileDescriptor_recorded_span_cdc6b364ab4d9de8, []int{0} } func (m *LogRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -73,7 +73,7 @@ func (m *LogRecord_Field) Reset() { *m = LogRecord_Field{} } func (m *LogRecord_Field) String() string { return proto.CompactTextString(m) } func (*LogRecord_Field) ProtoMessage() {} func (*LogRecord_Field) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_ec83bd4619f89f60, []int{0, 0} + return fileDescriptor_recorded_span_cdc6b364ab4d9de8, []int{0, 0} } func (m *LogRecord_Field) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -132,7 +132,7 @@ type RecordedSpan struct { func (m *RecordedSpan) Reset() { *m = RecordedSpan{} } func (*RecordedSpan) ProtoMessage() {} func (*RecordedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_ec83bd4619f89f60, []int{1} + return fileDescriptor_recorded_span_cdc6b364ab4d9de8, []int{1} } func (m *RecordedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -175,7 +175,7 @@ func (m *NormalizedSpan) Reset() { *m = NormalizedSpan{} } func (m *NormalizedSpan) String() string { return proto.CompactTextString(m) } func (*NormalizedSpan) ProtoMessage() {} func (*NormalizedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_ec83bd4619f89f60, []int{2} + return fileDescriptor_recorded_span_cdc6b364ab4d9de8, []int{2} } func (m *NormalizedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -201,13 +201,13 @@ func (m *NormalizedSpan) XXX_DiscardUnknown() { var xxx_messageInfo_NormalizedSpan proto.InternalMessageInfo func init() { - proto.RegisterType((*LogRecord)(nil), "cockroach.util.tracing.LogRecord") - proto.RegisterType((*LogRecord_Field)(nil), "cockroach.util.tracing.LogRecord.Field") - proto.RegisterType((*RecordedSpan)(nil), "cockroach.util.tracing.RecordedSpan") - proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.RecordedSpan.BaggageEntry") - proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.RecordedSpan.TagsEntry") - proto.RegisterType((*NormalizedSpan)(nil), "cockroach.util.tracing.NormalizedSpan") - proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.NormalizedSpan.TagsEntry") + proto.RegisterType((*LogRecord)(nil), "cockroach.util.tracing.tracingpb.LogRecord") + proto.RegisterType((*LogRecord_Field)(nil), "cockroach.util.tracing.tracingpb.LogRecord.Field") + proto.RegisterType((*RecordedSpan)(nil), "cockroach.util.tracing.tracingpb.RecordedSpan") + proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.tracingpb.RecordedSpan.BaggageEntry") + proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.tracingpb.RecordedSpan.TagsEntry") + proto.RegisterType((*NormalizedSpan)(nil), "cockroach.util.tracing.tracingpb.NormalizedSpan") + proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.tracingpb.NormalizedSpan.TagsEntry") } func (m *LogRecord) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -1769,48 +1769,49 @@ var ( ) func init() { - proto.RegisterFile("util/tracing/recorded_span.proto", fileDescriptor_recorded_span_ec83bd4619f89f60) + proto.RegisterFile("util/tracing/tracingpb/recorded_span.proto", fileDescriptor_recorded_span_cdc6b364ab4d9de8) } -var fileDescriptor_recorded_span_ec83bd4619f89f60 = []byte{ - // 623 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x53, 0xcd, 0x6e, 0xd3, 0x4c, - 0x14, 0x8d, 0x13, 0xe7, 0xef, 0x36, 0xaa, 0xaa, 0x51, 0xf5, 0xc9, 0xb5, 0x3e, 0xd9, 0xa5, 0x48, - 0xa5, 0xb0, 0xb0, 0xa1, 0x48, 0x50, 0x95, 0x05, 0xc2, 0xa4, 0x88, 0x08, 0x84, 0x90, 0xe9, 0x8a, - 0x4d, 0x34, 0xb5, 0xa7, 0x53, 0xab, 0xae, 0xc7, 0xb2, 0x27, 0x48, 0xe1, 0x29, 0xba, 0xec, 0x92, - 0x07, 0xe0, 0x09, 0x90, 0xd8, 0x77, 0xd9, 0x65, 0x57, 0x01, 0xdc, 0x17, 0x41, 0x33, 0x9e, 0xa4, - 0x7f, 0x54, 0x4d, 0xdb, 0x9d, 0x67, 0xee, 0x39, 0xe7, 0xde, 0x39, 0xe7, 0x1a, 0x16, 0x07, 0x3c, - 0x8a, 0x5d, 0x9e, 0xe1, 0x20, 0x4a, 0xa8, 0x9b, 0x91, 0x80, 0x65, 0x21, 0x09, 0xfb, 0x79, 0x8a, - 0x13, 0x27, 0xcd, 0x18, 0x67, 0xe8, 0xbf, 0x80, 0x05, 0xbb, 0x19, 0xc3, 0xc1, 0x8e, 0x23, 0xb0, - 0x8e, 0xc2, 0x9a, 0xf3, 0x94, 0x51, 0x26, 0x21, 0xae, 0xf8, 0x2a, 0xd1, 0xe6, 0x02, 0x65, 0x8c, - 0xc6, 0xc4, 0x95, 0xa7, 0xad, 0xc1, 0xb6, 0x8b, 0x93, 0xa1, 0x2a, 0xd9, 0x17, 0x4b, 0x3c, 0xda, - 0x23, 0x39, 0xc7, 0x7b, 0xa9, 0x02, 0x58, 0x17, 0x01, 0xe1, 0x20, 0xc3, 0x3c, 0x62, 0x6a, 0x92, - 0xa5, 0x9f, 0x1a, 0xb4, 0xdf, 0x33, 0xea, 0xcb, 0x21, 0xd1, 0x1a, 0xe8, 0x42, 0xc0, 0xd0, 0x16, - 0xb5, 0x95, 0x99, 0x55, 0xd3, 0x29, 0xc9, 0xce, 0x98, 0xec, 0x6c, 0x8e, 0xd5, 0xbd, 0xd6, 0xe1, - 0xc8, 0xae, 0xec, 0xff, 0xb2, 0x35, 0x5f, 0x32, 0xd0, 0x06, 0x34, 0xb6, 0x23, 0x12, 0x87, 0xb9, - 0x51, 0x5d, 0xac, 0xad, 0xcc, 0xac, 0x3e, 0x70, 0xfe, 0xfd, 0x44, 0x67, 0xd2, 0xcc, 0x79, 0x23, - 0xf0, 0x9e, 0x2e, 0x84, 0x7c, 0x45, 0x36, 0x5d, 0xa8, 0xcb, 0x6b, 0x34, 0x07, 0xb5, 0x5d, 0x32, - 0x94, 0x83, 0xb4, 0x7d, 0xf1, 0x89, 0xe6, 0xa1, 0xfe, 0x05, 0xc7, 0x03, 0x62, 0x54, 0xe5, 0x5d, - 0x79, 0x58, 0xfa, 0x5e, 0x87, 0x8e, 0xaf, 0x1c, 0xfe, 0x94, 0xe2, 0x04, 0x2d, 0x43, 0x4b, 0xb4, - 0x22, 0xfd, 0x28, 0x94, 0x6c, 0xdd, 0x9b, 0x29, 0x46, 0x76, 0x73, 0x53, 0xdc, 0xf5, 0xba, 0x7e, - 0x53, 0x16, 0x7b, 0x21, 0xba, 0x0f, 0x4d, 0x11, 0x88, 0x80, 0x55, 0x25, 0x0c, 0x8a, 0x91, 0xdd, - 0x10, 0x12, 0xbd, 0xae, 0xdf, 0x10, 0xa5, 0x5e, 0x88, 0x9e, 0xc1, 0x6c, 0x8a, 0x33, 0x92, 0xf0, - 0xfe, 0x18, 0x5b, 0x93, 0xd8, 0xb9, 0x62, 0x64, 0x77, 0x3e, 0xca, 0x8a, 0x62, 0x74, 0xd2, 0xd3, - 0x53, 0x88, 0xfe, 0x87, 0x36, 0x4b, 0x49, 0x69, 0xb4, 0xa1, 0xcb, 0x79, 0x4f, 0x2f, 0xd0, 0x3b, - 0x68, 0x6e, 0x61, 0x4a, 0x31, 0x25, 0x46, 0x5d, 0x9a, 0xf5, 0xe4, 0x2a, 0xb3, 0xce, 0xbe, 0xcc, - 0xf1, 0x4a, 0xce, 0x46, 0xc2, 0xb3, 0xa1, 0x3f, 0x56, 0x40, 0x1e, 0xe8, 0x1c, 0xd3, 0xdc, 0x68, - 0x48, 0x25, 0x67, 0x2a, 0xa5, 0x4d, 0x4c, 0xf3, 0x52, 0x46, 0x72, 0xd1, 0x6b, 0x80, 0x9c, 0xe3, - 0x8c, 0xf7, 0x65, 0xf8, 0xcd, 0x1b, 0x84, 0xdf, 0x96, 0x3c, 0x51, 0x41, 0x2f, 0xa1, 0x35, 0xde, - 0x2d, 0xa3, 0x25, 0x25, 0x16, 0x2e, 0x49, 0x74, 0x15, 0xa0, 0x54, 0x38, 0x10, 0x0a, 0x13, 0x12, - 0x7a, 0x01, 0x7a, 0xcc, 0x68, 0x6e, 0xb4, 0xe5, 0x4b, 0xee, 0x5d, 0xbb, 0x40, 0x6a, 0x75, 0x24, - 0x09, 0x3d, 0x82, 0x7a, 0xce, 0x31, 0xcf, 0x0d, 0x90, 0xad, 0xe7, 0x2f, 0xb5, 0x7e, 0x95, 0x0c, - 0xfd, 0x12, 0x62, 0xae, 0x43, 0xe7, 0xac, 0x97, 0xd3, 0xee, 0xda, 0x7a, 0x75, 0x4d, 0x33, 0x9f, - 0x43, 0x7b, 0xe2, 0xde, 0x4d, 0x88, 0xeb, 0xfa, 0xc1, 0x37, 0xbb, 0xb2, 0xf4, 0xa3, 0x06, 0xb3, - 0x1f, 0x58, 0xb6, 0x87, 0xe3, 0xe8, 0xab, 0x5a, 0xd8, 0x73, 0xbb, 0xa2, 0x5d, 0xdc, 0x95, 0xae, - 0x8a, 0xb7, 0xfc, 0xab, 0x1e, 0x5f, 0x65, 0xca, 0x79, 0xcd, 0x6b, 0x02, 0xae, 0xdd, 0x3d, 0x60, - 0xfd, 0x2e, 0x01, 0xd7, 0x6f, 0x13, 0xf0, 0x5b, 0x68, 0x05, 0x3b, 0x51, 0x1c, 0x66, 0x24, 0x51, - 0xbb, 0xbe, 0x3c, 0x9d, 0x19, 0x4a, 0x65, 0xc2, 0xbe, 0x75, 0x84, 0xde, 0xc3, 0xc3, 0x3f, 0x56, - 0xe5, 0xb0, 0xb0, 0xb4, 0xa3, 0xc2, 0xd2, 0x8e, 0x0b, 0x4b, 0xfb, 0x5d, 0x58, 0xda, 0xfe, 0x89, - 0x55, 0x39, 0x3a, 0xb1, 0x2a, 0xc7, 0x27, 0x56, 0xe5, 0x73, 0x53, 0x8d, 0xb0, 0xd5, 0x90, 0x8e, - 0x3c, 0xfd, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x83, 0x1a, 0x5c, 0x3e, 0x0b, 0x06, 0x00, 0x00, +var fileDescriptor_recorded_span_cdc6b364ab4d9de8 = []byte{ + // 626 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x53, 0xcd, 0x6a, 0xdb, 0x4c, + 0x14, 0xb5, 0x6c, 0xf9, 0x47, 0x37, 0x26, 0x84, 0x21, 0x0b, 0xc5, 0x7c, 0x48, 0x21, 0x1f, 0x94, + 0x90, 0x80, 0xdc, 0xa6, 0xd0, 0x06, 0x77, 0x51, 0xea, 0x26, 0x05, 0x43, 0x48, 0x8b, 0x9a, 0x6e, + 0xba, 0x31, 0x63, 0x69, 0x32, 0x11, 0x51, 0x34, 0x42, 0x1a, 0x17, 0xdc, 0xa7, 0xc8, 0x32, 0xcb, + 0xbe, 0x4d, 0x43, 0x57, 0x59, 0x66, 0xe5, 0xb6, 0xca, 0x8b, 0x94, 0x19, 0x8d, 0x94, 0xbf, 0x85, + 0xeb, 0x7a, 0x25, 0xcd, 0xbd, 0xe7, 0x1c, 0xdd, 0x39, 0xf7, 0x08, 0xb6, 0xc6, 0x3c, 0x08, 0xbb, + 0x3c, 0xc1, 0x5e, 0x10, 0xd1, 0xe2, 0x19, 0x8f, 0xba, 0x09, 0xf1, 0x58, 0xe2, 0x13, 0x7f, 0x98, + 0xc6, 0x38, 0x72, 0xe2, 0x84, 0x71, 0x86, 0xd6, 0x3d, 0xe6, 0x9d, 0x26, 0x0c, 0x7b, 0x27, 0x8e, + 0x60, 0x39, 0x0a, 0xed, 0x94, 0xac, 0xce, 0x2a, 0x65, 0x94, 0x49, 0x70, 0x57, 0xbc, 0xe5, 0xbc, + 0xce, 0x1a, 0x65, 0x8c, 0x86, 0xa4, 0x2b, 0x4f, 0xa3, 0xf1, 0x71, 0x17, 0x47, 0x13, 0xd5, 0xb2, + 0x1f, 0xb6, 0x78, 0x70, 0x46, 0x52, 0x8e, 0xcf, 0x62, 0x05, 0xb0, 0x1e, 0x02, 0xfc, 0x71, 0x82, + 0x79, 0xc0, 0xd4, 0x4c, 0x1b, 0xdf, 0x35, 0x30, 0x0e, 0x18, 0x75, 0xe5, 0xb8, 0x68, 0x17, 0x74, + 0x21, 0x60, 0x6a, 0xeb, 0xda, 0xe6, 0xd2, 0x4e, 0xc7, 0xc9, 0xc9, 0x4e, 0x41, 0x76, 0x8e, 0x0a, + 0xf5, 0x7e, 0xeb, 0x72, 0x6a, 0x57, 0xce, 0x7f, 0xda, 0x9a, 0x2b, 0x19, 0xe8, 0x3d, 0x34, 0x8e, + 0x03, 0x12, 0xfa, 0xa9, 0x59, 0x5d, 0xaf, 0x6d, 0x2e, 0xed, 0x3c, 0x73, 0x66, 0x5d, 0xd6, 0x29, + 0x3f, 0xeb, 0xbc, 0x13, 0xcc, 0xbe, 0x2e, 0x24, 0x5d, 0x25, 0xd3, 0xe9, 0x42, 0x5d, 0x96, 0xd1, + 0x0a, 0xd4, 0x4e, 0xc9, 0x44, 0x8e, 0x64, 0xb8, 0xe2, 0x15, 0xad, 0x42, 0xfd, 0x0b, 0x0e, 0xc7, + 0xc4, 0xac, 0xca, 0x5a, 0x7e, 0xd8, 0xf8, 0x51, 0x87, 0xb6, 0xab, 0x5c, 0xff, 0x18, 0xe3, 0x08, + 0x3d, 0x81, 0x96, 0xf8, 0x18, 0x19, 0x06, 0xbe, 0x64, 0xeb, 0xfd, 0xa5, 0x6c, 0x6a, 0x37, 0x8f, + 0x44, 0x6d, 0xb0, 0xe7, 0x36, 0x65, 0x73, 0xe0, 0xa3, 0xff, 0xa1, 0x29, 0x96, 0x24, 0x60, 0x55, + 0x09, 0x83, 0x6c, 0x6a, 0x37, 0x84, 0xc4, 0x60, 0xcf, 0x6d, 0x88, 0xd6, 0xc0, 0x47, 0x2f, 0x60, + 0x39, 0xc6, 0x09, 0x89, 0xf8, 0xb0, 0xc0, 0xd6, 0x24, 0x76, 0x25, 0x9b, 0xda, 0xed, 0x0f, 0xb2, + 0xa3, 0x18, 0xed, 0xf8, 0xf6, 0xe4, 0xa3, 0xff, 0xc0, 0x60, 0x31, 0xc9, 0x2d, 0x37, 0x75, 0x39, + 0xef, 0x6d, 0x01, 0x7d, 0x82, 0xe6, 0x08, 0x53, 0x8a, 0x29, 0x31, 0xeb, 0xd2, 0xb6, 0x57, 0xb3, + 0x6d, 0xbb, 0x7b, 0x47, 0xa7, 0x9f, 0xb3, 0xf7, 0x23, 0x9e, 0x4c, 0xdc, 0x42, 0x0b, 0x1d, 0x80, + 0xce, 0x31, 0x4d, 0xcd, 0x86, 0xd4, 0xdc, 0x9d, 0x53, 0xf3, 0x08, 0xd3, 0x34, 0x17, 0x94, 0x2a, + 0xe8, 0x2d, 0x40, 0xca, 0x71, 0xc2, 0x87, 0x32, 0x1a, 0xcd, 0x39, 0xa2, 0x61, 0x48, 0x9e, 0xe8, + 0xa0, 0xd7, 0xd0, 0x2a, 0x92, 0x67, 0xb6, 0xa4, 0xc4, 0xda, 0x23, 0x89, 0x3d, 0x05, 0xc8, 0x15, + 0x2e, 0x84, 0x42, 0x49, 0x42, 0xfb, 0xa0, 0x87, 0x8c, 0xa6, 0xa6, 0x21, 0xef, 0xb4, 0x3d, 0x47, + 0xbc, 0x54, 0xb0, 0x24, 0x1d, 0x6d, 0x41, 0x3d, 0xe5, 0x98, 0xa7, 0x26, 0xc8, 0x21, 0x56, 0x1f, + 0x0d, 0xf1, 0x26, 0x9a, 0xb8, 0x39, 0xa4, 0xd3, 0x83, 0xf6, 0x5d, 0x7f, 0xff, 0x36, 0x89, 0xbd, + 0xea, 0xae, 0xd6, 0x79, 0x09, 0x46, 0xe9, 0xe3, 0x3c, 0xc4, 0x9e, 0x7e, 0xf1, 0xcd, 0xae, 0x6c, + 0x5c, 0xd7, 0x60, 0xf9, 0x90, 0x25, 0x67, 0x38, 0x0c, 0xbe, 0xaa, 0x38, 0xdf, 0x4b, 0x92, 0xf6, + 0x30, 0x49, 0x87, 0x6a, 0xe5, 0xf9, 0xdf, 0xd7, 0x9b, 0x6d, 0xcf, 0x7d, 0xf5, 0x19, 0x4b, 0xaf, + 0x2d, 0xbe, 0x74, 0x7d, 0x91, 0xa5, 0xd7, 0x17, 0x5b, 0xba, 0x0b, 0x2d, 0xef, 0x24, 0x08, 0xfd, + 0x84, 0x44, 0xea, 0x9f, 0x78, 0x3a, 0xaf, 0x41, 0x4a, 0xaf, 0xd4, 0xf9, 0xe7, 0x05, 0xf7, 0xb7, + 0x2f, 0x7f, 0x5b, 0x95, 0xcb, 0xcc, 0xd2, 0xae, 0x32, 0x4b, 0xbb, 0xce, 0x2c, 0xed, 0x57, 0x66, + 0x69, 0xe7, 0x37, 0x56, 0xe5, 0xea, 0xc6, 0xaa, 0x5c, 0xdf, 0x58, 0x95, 0xcf, 0x46, 0x39, 0xc4, + 0xa8, 0x21, 0x7d, 0x7a, 0xfe, 0x27, 0x00, 0x00, 0xff, 0xff, 0x96, 0x4b, 0x70, 0x33, 0x67, 0x06, + 0x00, 0x00, } diff --git a/pkg/util/tracing/recorded_span.proto b/pkg/util/tracing/tracingpb/recorded_span.proto similarity index 97% rename from pkg/util/tracing/recorded_span.proto rename to pkg/util/tracing/tracingpb/recorded_span.proto index c3d698cb522a..a9d82bc74ede 100644 --- a/pkg/util/tracing/recorded_span.proto +++ b/pkg/util/tracing/tracingpb/recorded_span.proto @@ -9,8 +9,8 @@ // licenses/APL.txt. syntax = "proto3"; -package cockroach.util.tracing; -option go_package = "tracing"; +package cockroach.util.tracing.tracingpb; +option go_package = "tracingpb"; import "gogoproto/gogo.proto"; import "google/protobuf/any.proto"; From b251640d336d94027fec409f88a1e03b26d99926 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 11:49:07 +0200 Subject: [PATCH 08/20] tracing: move parts of recording, baggage, tag functionality to crdbSpan This starts towards a picture in which operations on a `*span` essentially turn into an opaque invocation of something on a `crdbSpan` plus possibly additional logic for the net/trace and external span. The hope is that by pulling further on this thread `crdbSpan` can move into its own package and thus have a clean interaction with the tracer and top-level methods. Release note: None --- pkg/util/tracing/tracer.go | 8 +- pkg/util/tracing/tracer_span.go | 151 ++++++++++++++++++-------------- 2 files changed, 90 insertions(+), 69 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index b4941d023ada..a7562440f8ec 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -328,7 +328,11 @@ func (t *Tracer) StartSpan( // recordings and so it's done after all of s' fields not protected by a lock // are set. if recordingType != NoRecording { - s.enableRecording(parentCtx.span, recordingType, false /* separateRecording */) + var p *crdbSpan + if parentCtx.span != nil { + p = &parentCtx.span.crdb + } + s.crdb.enableRecording(p, recordingType, false /* separateRecording */) } return s } @@ -481,7 +485,7 @@ func StartChildSpan( // Start recording if necessary. if recordingType != NoRecording { - s.enableRecording(pSpan, recordingType, separateRecording) + s.crdb.enableRecording(&pSpan.crdb, recordingType, separateRecording) } return s diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 056c6851ee93..e50dc4a76fa6 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -138,7 +138,7 @@ type crdbSpan struct { recordedLogs []opentracing.LogRecord // children contains the list of child spans started after this span // started recording. - children []*span + children []*crdbSpan // remoteSpan contains the list of remote child spans manually imported. remoteSpans []tracingpb.RecordedSpan } @@ -159,7 +159,7 @@ type crdbSpan struct { } func (s *crdbSpan) isRecording() bool { - return atomic.LoadInt32(&s.recording) != 0 + return s != nil && atomic.LoadInt32(&s.recording) != 0 } type otSpan struct { @@ -235,11 +235,13 @@ func IsRecording(s opentracing.Span) bool { // parent. // If separate recording is specified, the child is not registered with the // parent. Thus, the parent's recording will not include this child. -func (s *span) enableRecording(parent *span, recType RecordingType, separateRecording bool) { - s.crdb.mu.Lock() - defer s.crdb.mu.Unlock() - atomic.StoreInt32(&s.crdb.recording, 1) - s.crdb.mu.recording.recordingType = recType +func (s *crdbSpan) enableRecording( + parent *crdbSpan, recType RecordingType, separateRecording bool, +) { + s.mu.Lock() + defer s.mu.Unlock() + atomic.StoreInt32(&s.recording, 1) + s.mu.recording.recordingType = recType if parent != nil && !separateRecording { parent.addChild(s) } @@ -249,9 +251,9 @@ func (s *span) enableRecording(parent *span, recType RecordingType, separateReco // Clear any previously recorded info. This is needed by SQL SessionTracing, // who likes to start and stop recording repeatedly on the same span, and // collect the (separate) recordings every time. - s.crdb.mu.recording.recordedLogs = nil - s.crdb.mu.recording.children = nil - s.crdb.mu.recording.remoteSpans = nil + s.mu.recording.recordedLogs = nil + s.mu.recording.children = nil + s.mu.recording.remoteSpans = nil } // StartRecording enables recording on the span. Events from this point forward @@ -275,7 +277,7 @@ func StartRecording(os opentracing.Span, recType RecordingType) { // If we're already recording (perhaps because the parent was recording when // this span was created), there's nothing to do. if !sp.crdb.isRecording() { - sp.enableRecording(nil /* parent */, recType, false /* separateRecording */) + sp.crdb.enableRecording(nil /* parent */, recType, false /* separateRecording */) } } @@ -291,17 +293,21 @@ func StopRecording(os opentracing.Span) { } func (s *span) disableRecording() { - s.crdb.mu.Lock() - atomic.StoreInt32(&s.crdb.recording, 0) + s.crdb.disableRecording() +} + +func (s *crdbSpan) disableRecording() { + s.mu.Lock() + defer s.mu.Unlock() + atomic.StoreInt32(&s.recording, 0) // We test the duration as a way to check if the span has been finished. If it // has, we don't want to do the call below as it might crash (at least if // there's a netTr). - if (s.crdb.mu.duration == -1) && (s.crdb.mu.recording.recordingType == SnowballRecording) { + if (s.mu.duration == -1) && (s.mu.recording.recordingType == SnowballRecording) { // Clear the Snowball baggage item, assuming that it was set by // enableRecording(). s.setBaggageItemLocked(Snowball, "") } - s.crdb.mu.Unlock() } // IsRecordable returns true if {Start,Stop}Recording() can be called on this @@ -322,22 +328,25 @@ type Recording []tracingpb.RecordedSpan // enabled. This can be called while spans that are part of the recording are // still open; it can run concurrently with operations on those spans. func GetRecording(os opentracing.Span) Recording { - s := os.(*span) - if !s.crdb.isRecording() { + return os.(*span).crdb.getRecording() +} + +func (s *crdbSpan) getRecording() Recording { + if !s.isRecording() { return nil } - s.crdb.mu.Lock() + s.mu.Lock() // The capacity here is approximate since we don't know how many grandchildren // there are. - result := make(Recording, 0, 1+len(s.crdb.mu.recording.children)+len(s.crdb.mu.recording.remoteSpans)) + result := make(Recording, 0, 1+len(s.mu.recording.children)+len(s.mu.recording.remoteSpans)) // Shallow-copy the children so we can process them without the lock. - children := s.crdb.mu.recording.children + children := s.mu.recording.children result = append(result, s.getRecordingLocked()) - result = append(result, s.crdb.mu.recording.remoteSpans...) - s.crdb.mu.Unlock() + result = append(result, s.mu.recording.remoteSpans...) + s.mu.Unlock() for _, child := range children { - result = append(result, GetRecording(child)...) + result = append(result, child.getRecording()...) } // Sort the spans by StartTime, except the first span (the root of this @@ -794,6 +803,13 @@ func (s *span) SetTag(key string, value interface{}) opentracing.Span { return s.setTagInner(key, value, false /* locked */) } +func (s *crdbSpan) setTagLocked(key string, value interface{}) { + if s.mu.tags == nil { + s.mu.tags = make(opentracing.Tags) + } + s.mu.tags[key] = value +} + func (s *span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { if s.ot.shadowTr != nil { s.ot.shadowSpan.SetTag(key, value) @@ -804,14 +820,9 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci // The internal tags will be used if we start a recording on this span. if !locked { s.crdb.mu.Lock() + defer s.crdb.mu.Unlock() } - if s.crdb.mu.tags == nil { - s.crdb.mu.tags = make(opentracing.Tags) - } - s.crdb.mu.tags[key] = value - if !locked { - s.crdb.mu.Unlock() - } + s.crdb.setTagLocked(key, value) return s } @@ -862,27 +873,33 @@ func (s *span) LogKV(alternatingKeyValues ...interface{}) { // SetBaggageItem is part of the opentracing.Span interface. func (s *span) SetBaggageItem(restrictedKey, value string) opentracing.Span { - s.crdb.mu.Lock() - defer s.crdb.mu.Unlock() - return s.setBaggageItemLocked(restrictedKey, value) + s.crdb.SetBaggageItemAndTag(restrictedKey, value) + if s.ot.shadowTr != nil { + s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) + s.ot.shadowSpan.SetTag(restrictedKey, value) + } + // NB: nothing to do for net/trace. + + return s +} + +func (s *crdbSpan) SetBaggageItemAndTag(restrictedKey, value string) { + s.mu.Lock() + defer s.mu.Unlock() + s.setBaggageItemLocked(restrictedKey, value) + s.setTagLocked(restrictedKey, value) } -func (s *span) setBaggageItemLocked(restrictedKey, value string) opentracing.Span { - if oldVal, ok := s.crdb.mu.Baggage[restrictedKey]; ok && oldVal == value { +func (s *crdbSpan) setBaggageItemLocked(restrictedKey, value string) { + if oldVal, ok := s.mu.Baggage[restrictedKey]; ok && oldVal == value { // No-op. - return s + return } - if s.crdb.mu.Baggage == nil { - s.crdb.mu.Baggage = make(map[string]string) + if s.mu.Baggage == nil { + s.mu.Baggage = make(map[string]string) } - s.crdb.mu.Baggage[restrictedKey] = value - - if s.ot.shadowTr != nil { - s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) - } - // Also set a tag so it shows up in the Lightstep UI or x/net/trace. - s.setTagInner(restrictedKey, value, true /* locked */) - return s + s.mu.Baggage[restrictedKey] = value + s.setTagLocked(restrictedKey, value) } // BaggageItem is part of the opentracing.Span interface. @@ -914,14 +931,14 @@ func (s *span) Log(data opentracing.LogData) { // getRecordingLocked returns the span's recording. This does not include // children. -func (s *span) getRecordingLocked() tracingpb.RecordedSpan { +func (s *crdbSpan) getRecordingLocked() tracingpb.RecordedSpan { rs := tracingpb.RecordedSpan{ - TraceID: s.crdb.TraceID, - SpanID: s.crdb.SpanID, - ParentSpanID: s.crdb.parentSpanID, - Operation: s.crdb.operation, - StartTime: s.crdb.startTime, - Duration: s.crdb.mu.duration, + TraceID: s.TraceID, + SpanID: s.SpanID, + ParentSpanID: s.parentSpanID, + Operation: s.operation, + StartTime: s.startTime, + Duration: s.mu.duration, } addTag := func(k, v string) { @@ -940,35 +957,35 @@ func (s *span) getRecordingLocked() tracingpb.RecordedSpan { addTag("unfinished", "") } - if s.crdb.mu.stats != nil { - stats, err := types.MarshalAny(s.crdb.mu.stats) + if s.mu.stats != nil { + stats, err := types.MarshalAny(s.mu.stats) if err != nil { panic(err) } rs.Stats = stats } - if len(s.crdb.mu.Baggage) > 0 { + if len(s.mu.Baggage) > 0 { rs.Baggage = make(map[string]string) - for k, v := range s.crdb.mu.Baggage { + for k, v := range s.mu.Baggage { rs.Baggage[k] = v } } - if s.crdb.logTags != nil { - tags := s.crdb.logTags.Get() + if s.logTags != nil { + tags := s.logTags.Get() for i := range tags { tag := &tags[i] addTag(tagName(tag.Key()), tag.ValueStr()) } } - if len(s.crdb.mu.tags) > 0 { - for k, v := range s.crdb.mu.tags { + if len(s.mu.tags) > 0 { + for k, v := range s.mu.tags { // We encode the tag values as strings. addTag(k, fmt.Sprint(v)) } } - rs.Logs = make([]tracingpb.LogRecord, len(s.crdb.mu.recording.recordedLogs)) - for i, r := range s.crdb.mu.recording.recordedLogs { + rs.Logs = make([]tracingpb.LogRecord, len(s.mu.recording.recordedLogs)) + for i, r := range s.mu.recording.recordedLogs { rs.Logs[i].Time = r.Timestamp rs.Logs[i].Fields = make([]tracingpb.LogRecord_Field, len(r.Fields)) for j, f := range r.Fields { @@ -982,8 +999,8 @@ func (s *span) getRecordingLocked() tracingpb.RecordedSpan { return rs } -func (s *span) addChild(child *span) { - s.crdb.mu.Lock() - s.crdb.mu.recording.children = append(s.crdb.mu.recording.children, child) - s.crdb.mu.Unlock() +func (s *crdbSpan) addChild(child *crdbSpan) { + s.mu.Lock() + s.mu.recording.children = append(s.mu.recording.children, child) + s.mu.Unlock() } From cf0a026b3b9a9297c5b10fc93b22b22bb373ce91 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 11:53:45 +0200 Subject: [PATCH 09/20] tracing: delegate to crdbSpan in ImportRemoteSpans Release note: None --- pkg/util/tracing/tracer_span.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index e50dc4a76fa6..143635a5fe32 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -695,19 +695,21 @@ type TraceCollection struct { // these spans will be part of the result of GetRecording. Used to import // recorded traces from other nodes. func ImportRemoteSpans(os opentracing.Span, remoteSpans []tracingpb.RecordedSpan) error { - s := os.(*span) - if !s.crdb.isRecording() { - return errors.New("adding Raw Spans to a span that isn't recording") - } + return os.(*span).crdb.ImportRemoteSpans(remoteSpans) +} +func (s *crdbSpan) ImportRemoteSpans(remoteSpans []tracingpb.RecordedSpan) error { + if !s.isRecording() { + return errors.AssertionFailedf("adding Raw Spans to a span that isn't recording") + } // Change the root of the remote recording to be a child of this span. This is // usually already the case, except with DistSQL traces where remote // processors run in spans that FollowFrom an RPC span that we don't collect. - remoteSpans[0].ParentSpanID = s.crdb.SpanID + remoteSpans[0].ParentSpanID = s.SpanID - s.crdb.mu.Lock() - s.crdb.mu.recording.remoteSpans = append(s.crdb.mu.recording.remoteSpans, remoteSpans...) - s.crdb.mu.Unlock() + s.mu.Lock() + s.mu.recording.remoteSpans = append(s.mu.recording.remoteSpans, remoteSpans...) + s.mu.Unlock() return nil } From 4f4c5130d8b9d6a2f3bf2238faa27be263865839 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 9 Oct 2020 13:30:23 +0200 Subject: [PATCH 10/20] tracing: delegate to crdbSpan in LogFields Release note: None --- pkg/util/tracing/tracer_span.go | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 143635a5fe32..0a76dc02ccfd 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -851,15 +851,20 @@ func (s *span) LogFields(fields ...otlog.Field) { s.netTr.LazyPrintf("%s", buf.String()) } } - if s.crdb.isRecording() { - s.crdb.mu.Lock() - if len(s.crdb.mu.recording.recordedLogs) < maxLogsPerSpan { - s.crdb.mu.recording.recordedLogs = append(s.crdb.mu.recording.recordedLogs, opentracing.LogRecord{ - Timestamp: time.Now(), - Fields: fields, - }) - } - s.crdb.mu.Unlock() + s.crdb.LogFields(fields...) +} + +func (s *crdbSpan) LogFields(fields ...otlog.Field) { + if !s.isRecording() { + return + } + s.mu.Lock() + defer s.mu.Unlock() + if len(s.mu.recording.recordedLogs) < maxLogsPerSpan { + s.mu.recording.recordedLogs = append(s.mu.recording.recordedLogs, opentracing.LogRecord{ + Timestamp: time.Now(), + Fields: fields, + }) } } From 4e5b594f79fcaa239398cdf553de749dc0b60fee Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 9 Oct 2020 13:35:45 +0200 Subject: [PATCH 11/20] tracing: delegate to crdbSpan in BaggageItem Release note: None --- pkg/util/tracing/tracer_span.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 0a76dc02ccfd..eeb7edc5bedd 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -911,9 +911,19 @@ func (s *crdbSpan) setBaggageItemLocked(restrictedKey, value string) { // BaggageItem is part of the opentracing.Span interface. func (s *span) BaggageItem(restrictedKey string) string { - s.crdb.mu.Lock() - defer s.crdb.mu.Unlock() - return s.crdb.mu.Baggage[restrictedKey] + if s := s.crdb.BaggageItem(restrictedKey); s != "" { + return s + } + if s.ot.shadowSpan == nil { + return "" + } + return s.ot.shadowSpan.BaggageItem(restrictedKey) +} + +func (s *crdbSpan) BaggageItem(restrictedKey string) string { + s.mu.Lock() + defer s.mu.Unlock() + return s.mu.Baggage[restrictedKey] } // Tracer is part of the opentracing.Span interface. From d5c6cf7655f70affa69b6d68b2396b373b901343 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 15:58:06 +0200 Subject: [PATCH 12/20] tracing: improve comments on recording modes I had to understand the semantics by reading the code, and now that I think I understand them, want to save the next person from having to do the same. Release note: None --- pkg/util/tracing/tracer_span.go | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index eeb7edc5bedd..a009414ff8b9 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -98,12 +98,20 @@ func (sc *spanContext) isNoop() bool { type RecordingType int const ( - // NoRecording means that the span isn't recording. + // NoRecording means that the span isn't recording. Child spans created from + // it similarly won't be recording by default. NoRecording RecordingType = iota - // SnowballRecording means that remote child spans (generally opened through - // RPCs) are also recorded. + // SnowballRecording means that the span is recording and that derived + // spans will be as well, in the same mode (this includes remote spans, + // i.e. this mode crosses RPC boundaries). Derived spans will maintain + // their own recording, and this recording will be included in that of + // any local parent spans. SnowballRecording - // SingleNodeRecording means that only spans on the current node are recorded. + // SingleNodeRecording means that the span is recording and that locally + // derived spans will as well (i.e. a remote span typically won't be + // recording by default, in contrast to SnowballRecording). Similar to + // SnowballRecording, children have their own recording which is also + // included in that of their parents. SingleNodeRecording ) @@ -265,6 +273,9 @@ func (s *crdbSpan) enableRecording( // // If recording was already started on this span (either directly or because a // parent span is recording), the old recording is lost. +// +// Children spans created from the span while it is *not* recording will not +// necessarily be recordable. func StartRecording(os opentracing.Span, recType RecordingType) { if recType == NoRecording { panic("StartRecording called with NoRecording") From d433733da64326358730ace7fe59636b685b437b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 9 Oct 2020 17:19:22 +0200 Subject: [PATCH 13/20] tracing: create *span in only one method We were previously maintaining very similar code in three places: 1. `(*Tracer).StartSpan` 2. `StartRootSpan` 3. `StartChildSpan` This commit creates a method `(*Tracer).startSpanGeneric` and makes the above a thin wrapper around it. This allows us to simplify just one version of the code. Merging the three also showed minor differences: tags didn't seem to propagate the same in all cases, and a bug (that I just introduced) about what constitutes a `noopSpanContext` was highlighted and fixed. I still have plenty of questions about this code and far from good confidence that I didn't introduce any bugs. I did pay some attention to performance regressions, but no benchmarks are introduced at this point. My interest is to simplify the package first, then make it cheap (again). We can live with a temporary performance regression, as long as it is limited to the case in which tracing is actually enabled. Release note: None --- pkg/sql/distsql/server.go | 4 +- pkg/util/tracing/tracer.go | 249 +++++++++------------------ pkg/util/tracing/tracer_span.go | 41 ++++- pkg/util/tracing/tracer_span_test.go | 5 +- pkg/util/tracing/tracer_test.go | 8 +- 5 files changed, 123 insertions(+), 184 deletions(-) diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 2735ee8a792a..1d13788062bc 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -205,7 +205,9 @@ func (ds *ServerImpl) setupFlow( // TODO(andrei): localState.IsLocal is not quite the right thing to use. // If that field is unset, we might still want to create a child span if // this flow is run synchronously. - sp = tracing.StartChildSpan(opName, parentSpan, logtags.FromContext(ctx), false /* separateRecording */) + sp = ds.Tracer.(*tracing.Tracer).StartChildSpan( + opName, parentSpan.(*tracing.Span).SpanContext(), logtags.FromContext(ctx), + false /* separateRecording */) } else { // We use FollowsFrom because the flow's span outlives the SetupFlow request. // TODO(andrei): We should use something more efficient than StartSpan; we diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index a7562440f8ec..bfb06e1fb370 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -212,14 +212,12 @@ func (t *Tracer) StartSpan( } } - shadowTr := t.getShadowTracer() - - if len(opts) == 0 && !t.useNetTrace() && shadowTr == nil && !t.forceRealSpans { + if len(opts) == 0 && !t.AlwaysTrace() { return t.noopSpan } var sso opentracing.StartSpanOptions - var recordable bool + var recordable RecordableOpt var logTags *logtags.Buffer for _, o := range opts { switch to := o.(type) { @@ -234,10 +232,8 @@ func (t *Tracer) StartSpan( } } - var hasParent bool var parentType opentracing.SpanReferenceType - var parentCtx *spanContext - var recordingType RecordingType + var parentCtx spanContext for _, r := range sso.References { if r.Type != opentracing.ChildOfRef && r.Type != opentracing.FollowsFromRef { continue @@ -248,93 +244,15 @@ func (t *Tracer) StartSpan( if IsNoopContext(r.ReferencedContext) { continue } - hasParent = true parentType = r.Type // Note that the logic around here doesn't support spans with multiple // references. Luckily, we don't have such cases. - parentCtx = r.ReferencedContext.(*spanContext) - recordingType = parentCtx.recordingType + parentCtx = *r.ReferencedContext.(*spanContext) break } - if hasParent { - // We use the parent's shadow tracer, to avoid inconsistency inside a - // trace when the shadow tracer changes. - shadowTr = parentCtx.shadowTr - } - - // If tracing is disabled, the Recordable option wasn't passed, and we're not - // part of a recording or snowball trace, avoid overhead and return a noop - // span. - if !recordable && recordingType == NoRecording && shadowTr == nil && !t.useNetTrace() && !t.forceRealSpans { - return t.noopSpan - } - - s := &span{ - tracer: t, - crdb: crdbSpan{ - operation: operationName, - startTime: sso.StartTime, - logTags: logTags, - }, - } - if s.crdb.startTime.IsZero() { - s.crdb.startTime = time.Now() - } - s.crdb.mu.duration = -1 - - if !hasParent { - // No parent Span; allocate new trace id. - s.crdb.TraceID = uint64(rand.Int63()) - } else { - s.crdb.TraceID = parentCtx.TraceID - } - s.crdb.SpanID = uint64(rand.Int63()) - - if t.useNetTrace() { - s.netTr = trace.New("tracing", operationName) - s.netTr.SetMaxEvents(maxLogsPerSpan) - } - if hasParent { - s.crdb.parentSpanID = parentCtx.SpanID - // Copy baggage from parent. - if l := len(parentCtx.Baggage); l > 0 { - s.crdb.mu.Baggage = make(map[string]string, l) - for k, v := range parentCtx.Baggage { - s.crdb.mu.Baggage[k] = v - } - } - } - - for k, v := range sso.Tags { - s.SetTag(k, v) - } - - // Copy baggage items to tags so they show up in the shadow tracer UI, - // x/net/trace, or recordings. - for k, v := range s.crdb.mu.Baggage { - s.SetTag(k, v) - } - - if shadowTr != nil { - var parentShadowCtx opentracing.SpanContext - if hasParent { - parentShadowCtx = parentCtx.shadowCtx - } - linkShadowSpan(s, shadowTr, parentShadowCtx, parentType) - } - - // Start recording if necessary. This publishes the span to be retrieved by - // recordings and so it's done after all of s' fields not protected by a lock - // are set. - if recordingType != NoRecording { - var p *crdbSpan - if parentCtx.span != nil { - p = &parentCtx.span.crdb - } - s.crdb.enableRecording(p, recordingType, false /* separateRecording */) - } - return s + return t.startSpanGeneric( + operationName, parentCtx, parentType, recordable, logTags, false /* separateRecording */) } // RecordableOpt specifies whether a root span should be recordable. @@ -365,47 +283,7 @@ func (t *Tracer) AlwaysTrace() bool { func (t *Tracer) StartRootSpan( opName string, logTags *logtags.Buffer, recordable RecordableOpt, ) opentracing.Span { - // In the usual case, we return noopSpan. - if !t.AlwaysTrace() && recordable == NonRecordableSpan { - return t.noopSpan - } - - s := &span{ - tracer: t, - crdb: crdbSpan{ - spanMeta: spanMeta{ - TraceID: uint64(rand.Int63()), - SpanID: uint64(rand.Int63()), - }, - operation: opName, - startTime: time.Now(), - logTags: logTags, - }, - } - s.crdb.mu.duration = -1 - - shadowTracer := t.getShadowTracer() - if shadowTracer != nil { - linkShadowSpan( - s, shadowTracer, nil, /* parentShadowCtx */ - opentracing.SpanReferenceType(0) /* parentType - ignored*/) - } - - if t.useNetTrace() { - var tags []logtags.Tag - if logTags != nil { - tags = logTags.Get() - } - - s.netTr = trace.New("tracing", opName) - s.netTr.SetMaxEvents(maxLogsPerSpan) - for i := range tags { - tag := &tags[i] - s.netTr.LazyPrintf("%s:%v", tag.Key(), tag.Value()) - } - } - - return s + return t.StartChildSpan(opName, SpanContext{}, logTags, false /* separateRecording */) } // StartChildSpan creates a child span of the given parent span. This is @@ -423,69 +301,101 @@ func (t *Tracer) StartRootSpan( // recording; for example DistSQL processors each report their own recording, // and we don't want the parent's recording to include a child's because then we // might double-report that child. -func StartChildSpan( - opName string, parentSpan opentracing.Span, logTags *logtags.Buffer, separateRecording bool, +// +// TODO(tbg): I don't think we need separateRecording because children can consume +// and (atomically) clear their recording to avoid it getting consumed twice. +func (t *Tracer) StartChildSpan( + opName string, parentContext SpanContext, logTags *logtags.Buffer, separateRecording bool, +) opentracing.Span { + return t.startSpanGeneric( + opName, + parentContext, + opentracing.ChildOfRef, + false, /* recordable */ + logTags, + separateRecording, + ) +} + +// startSpanGeneric is the internal workhorse for creating spans. It serves two purposes: +// +// 1. creating root spans. In this case, parentContext and parentType are zero. A noop span +// is returned when nothing forces an actual span to be created, i.e. there is no shadow +// tracer and internal tracing active, plus no recordability is requested. +// 2. creating derived spans. In this case, parentContext and parentType are nonzero. If the +// parent is not recording and 'recordable' is zero, and nothing else forces a real span, +// a noopSpan results. +func (t *Tracer) startSpanGeneric( + opName string, + parentContext SpanContext, + parentType opentracing.SpanReferenceType, + recordable RecordableOpt, + logTags *logtags.Buffer, + separateRecording bool, ) opentracing.Span { - tr := parentSpan.Tracer().(*Tracer) // If tracing is disabled, avoid overhead and return a noop span. - if IsBlackHoleSpan(parentSpan) { - return tr.noopSpan + if !t.AlwaysTrace() && + parentContext.TraceID == 0 && + parentContext.recordingType == NoRecording && + recordable == NonRecordableSpan { + return t.noopSpan } - pSpan := parentSpan.(*span) - s := &span{ - tracer: tr, + tracer: t, crdb: crdbSpan{ operation: opName, startTime: time.Now(), - parentSpanID: pSpan.crdb.SpanID, + parentSpanID: parentContext.SpanID, logTags: logTags, }, } + s.crdb.mu.duration = -1 // unfinished // Copy baggage from parent. - pSpan.crdb.mu.Lock() - if l := len(pSpan.crdb.mu.Baggage); l > 0 { - s.crdb.mu.Baggage = make(map[string]string, l) - for k, v := range pSpan.crdb.mu.Baggage { - s.crdb.mu.Baggage[k] = v - } + // + // NB: this could be optimized. + for k, v := range parentContext.Baggage { + s.SetBaggageItem(k, v) } - s.crdb.TraceID = pSpan.crdb.TraceID + traceID := parentContext.TraceID + if traceID == 0 { + traceID = uint64(rand.Int63()) + } + s.crdb.TraceID = traceID s.crdb.SpanID = uint64(rand.Int63()) - if pSpan.ot.shadowTr != nil { - linkShadowSpan(s, pSpan.ot.shadowTr, pSpan.ot.shadowSpan.Context(), opentracing.ChildOfRef) + // We use the shadowTr from the parent context over that of our + // tracer because the tracer's might have changed and be incompatible. + shadowTr := parentContext.shadowTr + if shadowTr == nil { + shadowTr = t.getShadowTracer() } - recordingType := pSpan.crdb.mu.recording.recordingType + if shadowTr != nil { + linkShadowSpan(s, shadowTr, parentContext.shadowCtx, parentType) + } - if pSpan.netTr != nil { + if t.useNetTrace() { s.netTr = trace.New("tracing", opName) s.netTr.SetMaxEvents(maxLogsPerSpan) - if startTags := s.crdb.logTags; startTags != nil { - tags := startTags.Get() - for i := range tags { + if logTags != nil { + tags := logTags.Get() + for i := range logTags.Get() { tag := &tags[i] s.netTr.LazyPrintf("%s:%v", tagName(tag.Key()), tag.Value()) } } } - if pSpan.netTr != nil || pSpan.ot.shadowTr != nil { - // Copy baggage items to tags so they show up in the shadow tracer UI or x/net/trace. - for k, v := range s.crdb.mu.Baggage { - s.SetTag(k, v) - } - } - - pSpan.crdb.mu.Unlock() - // Start recording if necessary. - if recordingType != NoRecording { - s.crdb.enableRecording(&pSpan.crdb, recordingType, separateRecording) + if parentContext.recordingType != NoRecording { + var p *crdbSpan + if parentContext.span != nil { + p = &parentContext.span.crdb + } + s.crdb.enableRecording(p, parentContext.recordingType, separateRecording) } return s @@ -688,20 +598,21 @@ func ChildSpanSeparateRecording( func childSpan( ctx context.Context, opName string, separateRecording bool, ) (context.Context, opentracing.Span) { - sp := opentracing.SpanFromContext(ctx) - if sp == nil { + otSpan := opentracing.SpanFromContext(ctx) + if otSpan == nil { return ctx, nil } - if sp.(*span).isNoop() { + sp := otSpan.(*span) + if sp.isNoop() { // Optimization: avoid ContextWithSpan call if tracing is disabled. return ctx, sp } - tr := sp.Tracer() + tr := sp.Tracer().(*Tracer) if IsBlackHoleSpan(sp) { - ns := tr.(*Tracer).noopSpan + ns := tr.noopSpan return opentracing.ContextWithSpan(ctx, ns), ns } - newSpan := StartChildSpan(opName, sp, logtags.FromContext(ctx), separateRecording) + newSpan := tr.StartChildSpan(opName, sp.SpanContext(), logtags.FromContext(ctx), separateRecording) return opentracing.ContextWithSpan(ctx, newSpan), newSpan } diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index a009414ff8b9..9356190b2ba3 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -90,10 +90,6 @@ func (sc *spanContext) ForeachBaggageItem(handler func(k, v string) bool) { } } -func (sc *spanContext) isNoop() bool { - return sc.spanMeta == spanMeta{} -} - // RecordingType is the type of recording that a span might be performing. type RecordingType int @@ -736,8 +732,12 @@ func IsBlackHoleSpan(s opentracing.Span) bool { // IsNoopContext returns true if the span context is from a "no-op" span. If // this is true, any span derived from this context will be a "black hole span". func IsNoopContext(spanCtx opentracing.SpanContext) bool { - // TODO(tbg): remove this. - return spanCtx.(*spanContext).isNoop() + sc := spanCtx.(*spanContext) + return sc.isNoop() +} + +func (sc *spanContext) isNoop() bool { + return sc.recordingType == NoRecording && sc.shadowTr == nil } // SetSpanStats sets the stats on a span. stats.Stats() will also be added to @@ -782,11 +782,36 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { func (s *span) Context() opentracing.SpanContext { s.crdb.mu.Lock() defer s.crdb.mu.Unlock() - baggageCopy := make(map[string]string, len(s.crdb.mu.Baggage)) + sc := s.SpanContext() + return &sc +} + +// TODO(tbg): these are temporary to get things to compile without larger +// mechanical refactors. + +// SpanContext is information about a span, used to derive spans +// from a parent in a way that's uniform between local and remote +// parents. For local parents, this generally references their Span +// to unlock features such as sharing recordings with the parent. For +// remote parents, it only contains the TraceID and related metadata. +type SpanContext = spanContext + +// Span is the tracing span used throughout CockroachDB. +type Span = span + +// spanContext returns a spanContext. Note that this returns a value, +// not a pointer, which the caller can use to avoid heap allocations. +func (s *span) SpanContext() SpanContext { + n := len(s.crdb.mu.Baggage) + // In the common case, we have no baggage, so avoid making an empty map. + var baggageCopy map[string]string + if n > 0 { + baggageCopy = make(map[string]string, n) + } for k, v := range s.crdb.mu.Baggage { baggageCopy[k] = v } - sc := &spanContext{ + sc := spanContext{ spanMeta: s.crdb.spanMeta, span: s, Baggage: baggageCopy, diff --git a/pkg/util/tracing/tracer_span_test.go b/pkg/util/tracing/tracer_span_test.go index ca00d3c4ed2d..36183bd1ecf1 100644 --- a/pkg/util/tracing/tracer_span_test.go +++ b/pkg/util/tracing/tracer_span_test.go @@ -53,7 +53,7 @@ func TestRecordingString(t *testing.T) { root.LogFields(otlog.String(tracingpb.LogMessageField, "root 3")) - ch2 := StartChildSpan("local child", root, nil /* logTags */, false /* separateRecording */) + ch2 := tr.StartChildSpan("local child", root.(*Span).SpanContext(), nil /* logTags */, false /* separateRecording */) root.LogFields(otlog.String(tracingpb.LogMessageField, "root 4")) ch2.LogFields(otlog.String(tracingpb.LogMessageField, "local child 1")) ch2.Finish() @@ -76,6 +76,7 @@ span remote child: tags: sb=1 event: remote child 1 span local child: + tags: sb=1 event: local child 1 `) require.NoError(t, err) @@ -86,7 +87,7 @@ event:root 1 event:remote child 1 event:root 2 event:root 3 - === operation:local child + === operation:local child sb:1 event:local child 1 event:root 4 event:root 5 diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 3119a5141bfd..dfb94f34162d 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -131,7 +131,7 @@ func TestStartChildSpan(t *testing.T) { tr := NewTracer() sp1 := tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) - sp2 := StartChildSpan("child", sp1, nil /* logTags */, false /*separateRecording*/) + sp2 := tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, false /*separateRecording*/) sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` @@ -143,7 +143,7 @@ func TestStartChildSpan(t *testing.T) { sp1 = tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) - sp2 = StartChildSpan("child", sp1, nil /* logTags */, true /*separateRecording*/) + sp2 = tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, true /*separateRecording*/) sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` @@ -159,8 +159,8 @@ func TestStartChildSpan(t *testing.T) { sp1 = tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) - sp2 = StartChildSpan( - "child", sp1, logtags.SingleTagBuffer("key", "val"), false, /*separateRecording*/ + sp2 = tr.StartChildSpan( + "child", sp1.(*Span).SpanContext(), logtags.SingleTagBuffer("key", "val"), false, /*separateRecording*/ ) sp2.Finish() sp1.Finish() From d1646dd54461b776992b7d5c5b21755821327416 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 12:23:06 +0200 Subject: [PATCH 14/20] tracing: assert that we never have >1 span reference Release note: None --- pkg/util/tracing/tracer.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index bfb06e1fb370..fb10cacc8de1 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -234,6 +234,12 @@ func (t *Tracer) StartSpan( var parentType opentracing.SpanReferenceType var parentCtx spanContext + + // Note that the logic around here doesn't support spans with multiple + // references. Luckily, we don't have such cases. + if len(sso.References) > 1 { + panic("multiple references are unsupported") + } for _, r := range sso.References { if r.Type != opentracing.ChildOfRef && r.Type != opentracing.FollowsFromRef { continue @@ -245,8 +251,6 @@ func (t *Tracer) StartSpan( continue } parentType = r.Type - // Note that the logic around here doesn't support spans with multiple - // references. Luckily, we don't have such cases. parentCtx = *r.ReferencedContext.(*spanContext) break } From f55f719a9723da1affce6f97369e496c35d6213f Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 12:23:23 +0200 Subject: [PATCH 15/20] tracing: explain blackhole spans Release note: None --- pkg/util/tracing/tracer_span.go | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 9356190b2ba3..ab2f85a73da2 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -721,9 +721,13 @@ func (s *crdbSpan) ImportRemoteSpans(remoteSpans []tracingpb.RecordedSpan) error } // IsBlackHoleSpan returns true if events for this span are just dropped. This -// is the case when tracing is disabled and we're not recording. Tracing clients -// can use this method to figure out if they can short-circuit some +// is the case when the span is not recording and no external tracer is configured. +// Tracing clients can use this method to figure out if they can short-circuit some // tracing-related work that would be discarded anyway. +// +// The child of a blackhole span is a non-recordable blackhole span[*]. These incur +// only minimal overhead. It is therefore not worth it to call this method to avoid +// starting spans. func IsBlackHoleSpan(s opentracing.Span) bool { sp := s.(*span) return sp.isBlackHole() @@ -731,6 +735,9 @@ func IsBlackHoleSpan(s opentracing.Span) bool { // IsNoopContext returns true if the span context is from a "no-op" span. If // this is true, any span derived from this context will be a "black hole span". +// +// You should never need to care about this method. It is exported for technical +// rasons. func IsNoopContext(spanCtx opentracing.SpanContext) bool { sc := spanCtx.(*spanContext) return sc.isNoop() From cf48395669b9836fc0205a2216fdacf8f9a36b6f Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 12:30:03 +0200 Subject: [PATCH 16/20] tracing: propagate shadow tracers to children only when identical Release note: None --- pkg/util/tracing/tracer.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index fb10cacc8de1..873da6e1c997 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -372,12 +372,8 @@ func (t *Tracer) startSpanGeneric( // We use the shadowTr from the parent context over that of our // tracer because the tracer's might have changed and be incompatible. - shadowTr := parentContext.shadowTr - if shadowTr == nil { - shadowTr = t.getShadowTracer() - } - - if shadowTr != nil { + shadowTr := t.getShadowTracer() + if shadowTr != nil && (parentContext.shadowTr == nil || shadowTr == parentContext.shadowTr) { linkShadowSpan(s, shadowTr, parentContext.shadowCtx, parentType) } From 374e74543a6b4a495e417341bbd1a06125e73792 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 12:47:46 +0200 Subject: [PATCH 17/20] tracing: remove a few references to shadowTracer Release note: None --- pkg/util/tracing/tracer_span.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index ab2f85a73da2..3a2666a39b8a 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -773,7 +773,7 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { s.crdb.mu.Lock() s.crdb.mu.duration = finishTime.Sub(s.crdb.startTime) s.crdb.mu.Unlock() - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { s.ot.shadowSpan.Finish() } if s.netTr != nil { @@ -823,7 +823,7 @@ func (s *span) SpanContext() SpanContext { span: s, Baggage: baggageCopy, } - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { sc.shadowTr = s.ot.shadowTr sc.shadowCtx = s.ot.shadowSpan.Context() } @@ -836,7 +836,7 @@ func (s *span) SpanContext() SpanContext { // SetOperationName is part of the opentracing.Span interface. func (s *span) SetOperationName(operationName string) opentracing.Span { - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetOperationName(operationName) } s.crdb.operation = operationName @@ -856,7 +856,7 @@ func (s *crdbSpan) setTagLocked(key string, value interface{}) { } func (s *span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetTag(key, value) } if s.netTr != nil { @@ -873,7 +873,7 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci // LogFields is part of the opentracing.Span interface. func (s *span) LogFields(fields ...otlog.Field) { - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { s.ot.shadowSpan.LogFields(fields...) } if s.netTr != nil { @@ -924,7 +924,7 @@ func (s *span) LogKV(alternatingKeyValues ...interface{}) { // SetBaggageItem is part of the opentracing.Span interface. func (s *span) SetBaggageItem(restrictedKey, value string) opentracing.Span { s.crdb.SetBaggageItemAndTag(restrictedKey, value) - if s.ot.shadowTr != nil { + if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) s.ot.shadowSpan.SetTag(restrictedKey, value) } From 1921231d3cdc1ccfa46e567ff7acd92308594c30 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 15:24:53 +0200 Subject: [PATCH 18/20] tracing: properly export Span and SpanContext They will be used instead of `opentracing.Span{,Context}` in future commits. Release note: None --- pkg/util/log/ambient_context_test.go | 6 +- pkg/util/log/trace_test.go | 6 +- pkg/util/tracing/shadow.go | 14 +- pkg/util/tracing/tags.go | 12 +- pkg/util/tracing/tags_test.go | 6 +- pkg/util/tracing/test_utils.go | 6 +- pkg/util/tracing/tracer.go | 92 +++++----- pkg/util/tracing/tracer_span.go | 250 +++++++++++++-------------- pkg/util/tracing/tracer_span_test.go | 18 +- pkg/util/tracing/tracer_test.go | 80 ++++----- 10 files changed, 241 insertions(+), 249 deletions(-) diff --git a/pkg/util/log/ambient_context_test.go b/pkg/util/log/ambient_context_test.go index 698dab41f4b6..f7b4ae1124a6 100644 --- a/pkg/util/log/ambient_context_test.go +++ b/pkg/util/log/ambient_context_test.go @@ -61,10 +61,10 @@ func TestAnnotateCtxSpan(t *testing.T) { sp1.Finish() if err := tracing.TestingCheckRecordedSpans(tracing.GetRecording(sp1), ` - span root: + Span root: event: a event: c - span child: + Span child: tags: ambient= event: [ambient] b `); err != nil { @@ -79,7 +79,7 @@ func TestAnnotateCtxSpan(t *testing.T) { Event(ctx, "a") sp.Finish() if err := tracing.TestingCheckRecordedSpans(tracing.GetRecording(sp), ` - span s: + Span s: tags: ambient= event: [ambient] a `); err != nil { diff --git a/pkg/util/log/trace_test.go b/pkg/util/log/trace_test.go index df42dbdfef05..d274d2f767c1 100644 --- a/pkg/util/log/trace_test.go +++ b/pkg/util/log/trace_test.go @@ -81,7 +81,7 @@ func TestTrace(t *testing.T) { sp.Finish() if err := tracing.TestingCheckRecordedSpans(tracing.GetRecording(sp), ` - span s: + Span s: event: test1 event: test2 event: testerr @@ -108,7 +108,7 @@ func TestTraceWithTags(t *testing.T) { sp.Finish() if err := tracing.TestingCheckRecordedSpans(tracing.GetRecording(sp), ` - span s: + Span s: event: [tag=1] test1 event: [tag=1] test2 event: [tag=1] testerr @@ -199,7 +199,7 @@ func TestEventLogAndTrace(t *testing.T) { el.Finish() if err := tracing.TestingCheckRecordedSpans(tracing.GetRecording(sp), ` - span s: + Span s: event: test3 event: test4 event: test5err diff --git a/pkg/util/tracing/shadow.go b/pkg/util/tracing/shadow.go index 6fad173d867e..937bf5ed4c3c 100644 --- a/pkg/util/tracing/shadow.go +++ b/pkg/util/tracing/shadow.go @@ -10,8 +10,8 @@ // // A "shadow" tracer can be any opentracing.Tracer implementation that is used // in addition to the normal functionality of our tracer. It works by attaching -// a shadow span to every span, and attaching a shadow context to every span -// context. When injecting a span context, we encapsulate the shadow context +// a shadow Span to every Span, and attaching a shadow context to every Span +// context. When injecting a Span context, we encapsulate the shadow context // inside ours. package tracing @@ -69,21 +69,21 @@ func (st *shadowTracer) Close() { st.manager.Close(st) } -// linkShadowSpan creates and links a Shadow span to the passed-in span (i.e. +// linkShadowSpan creates and links a Shadow Span to the passed-in Span (i.e. // fills in s.shadowTr and s.shadowSpan). This should only be called when // shadow tracing is enabled. // -// The Shadow span will have a parent if parentShadowCtx is not nil. +// The Shadow Span will have a parent if parentShadowCtx is not nil. // parentType is ignored if parentShadowCtx is nil. // -// The tags (including logTags) from s are copied to the Shadow span. +// The tags (including logTags) from s are copied to the Shadow Span. func linkShadowSpan( - s *span, + s *Span, shadowTr *shadowTracer, parentShadowCtx opentracing.SpanContext, parentType opentracing.SpanReferenceType, ) { - // Create the shadow lightstep span. + // Create the shadow lightstep Span. var opts []opentracing.StartSpanOption // Replicate the options, using the lightstep context in the reference. opts = append(opts, opentracing.StartTime(s.crdb.startTime)) diff --git a/pkg/util/tracing/tags.go b/pkg/util/tracing/tags.go index cf4913aa616b..04106893a7f4 100644 --- a/pkg/util/tracing/tags.go +++ b/pkg/util/tracing/tags.go @@ -17,14 +17,14 @@ import ( opentracing "github.com/opentracing/opentracing-go" ) -// LogTagsOption is a StartSpanOption that uses log tags to populate the span tags. +// LogTagsOption is a StartSpanOption that uses log tags to populate the Span tags. type logTagsOption logtags.Buffer var _ opentracing.StartSpanOption = &logTagsOption{} // Apply is part of the opentracing.StartSpanOption interface. // -// The tags in the buffer go through the log tag -> span tag remapping (see +// The tags in the buffer go through the log tag -> Span tag remapping (see // tagName()). // // Note that our tracer does not call Apply() for this options. Instead, it @@ -46,14 +46,14 @@ func (lt *logTagsOption) Apply(o *opentracing.StartSpanOptions) { } } -// LogTags returns a StartSpanOption that sets the span tags to the given log -// tags. When applied, the returned option will apply any logtag name->span tag +// LogTags returns a StartSpanOption that sets the Span tags to the given log +// tags. When applied, the returned option will apply any logtag name->Span tag // name remapping that has been registered via RegisterTagRemapping. func LogTags(tags *logtags.Buffer) opentracing.StartSpanOption { return (*logTagsOption)(tags) } -// LogTagsFromCtx returns a StartSpanOption that sets the span tags to the log +// LogTagsFromCtx returns a StartSpanOption that sets the Span tags to the log // tags in the context. func LogTagsFromCtx(ctx context.Context) opentracing.StartSpanOption { return (*logTagsOption)(logtags.FromContext(ctx)) @@ -62,7 +62,7 @@ func LogTagsFromCtx(ctx context.Context) opentracing.StartSpanOption { // tagRemap is a map that records desired conversions var tagRemap = make(map[string]string) -// RegisterTagRemapping sets the span tag name that corresponds to the given log +// RegisterTagRemapping sets the Span tag name that corresponds to the given log // tag name. Should be called as part of an init() function. func RegisterTagRemapping(logTag, spanTag string) { tagRemap[logTag] = spanTag diff --git a/pkg/util/tracing/tags_test.go b/pkg/util/tracing/tags_test.go index 38c7b3ff542c..7f90825ce0a2 100644 --- a/pkg/util/tracing/tags_test.go +++ b/pkg/util/tracing/tags_test.go @@ -28,7 +28,7 @@ func TestLogTags(t *testing.T) { StartRecording(sp1, SingleNodeRecording) sp1.Finish() require.NoError(t, TestingCheckRecordedSpans(GetRecording(sp1), ` - span foo: + Span foo: tags: tag1=val1 tag2=val2 `)) require.NoError(t, shadowTracer.expectSingleSpanWithTags("tag1", "tag2")) @@ -41,7 +41,7 @@ func TestLogTags(t *testing.T) { StartRecording(sp2, SingleNodeRecording) sp2.Finish() require.NoError(t, TestingCheckRecordedSpans(GetRecording(sp2), ` - span bar: + Span bar: tags: one=val1 two=val2 `)) require.NoError(t, shadowTracer.expectSingleSpanWithTags("one", "two")) @@ -51,7 +51,7 @@ func TestLogTags(t *testing.T) { StartRecording(sp3, SingleNodeRecording) sp3.Finish() require.NoError(t, TestingCheckRecordedSpans(GetRecording(sp3), ` - span baz: + Span baz: tags: one=val1 two=val2 `)) require.NoError(t, shadowTracer.expectSingleSpanWithTags("one", "two")) diff --git a/pkg/util/tracing/test_utils.go b/pkg/util/tracing/test_utils.go index 6f5e2874b35f..313b9da5391d 100644 --- a/pkg/util/tracing/test_utils.go +++ b/pkg/util/tracing/test_utils.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" ) -// FindMsgInRecording returns the index of the first span containing msg in its -// logs, or -1 if no span is found. +// FindMsgInRecording returns the index of the first Span containing msg in its +// logs, or -1 if no Span is found. func FindMsgInRecording(recording Recording, msg string) int { for i, sp := range recording { if LogsContainMsg(sp, msg) { @@ -27,7 +27,7 @@ func FindMsgInRecording(recording Recording, msg string) int { return -1 } -// LogsContainMsg returns true if a span's logs contain the given message. +// LogsContainMsg returns true if a Span's logs contain the given message. func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { for _, l := range sp.Logs { // NOTE: With out logs, each LogRecord has a single field ("event") and diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 873da6e1c997..9249b27311ee 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -80,7 +80,7 @@ var zipkinCollector = settings.RegisterPublicStringSetting( // events can be retrieved at any time. // // - lightstep traces. This is implemented by maintaining a "shadow" lightstep -// span inside each of our spans. +// Span inside each of our spans. // // Even when tracing is disabled, we still use this Tracer (with x/net/trace and // lightstep disabled) because of its recording capability (snowball @@ -92,7 +92,7 @@ var zipkinCollector = settings.RegisterPublicStringSetting( type Tracer struct { // Preallocated noopSpan, used to avoid creating spans when we are not using // x/net/trace or lightstep and we are not recording. - noopSpan *span + noopSpan *Span // If forceRealSpans is set, this Tracer will always create real spans (never // noopSpans), regardless of the recording or lightstep configuration. Used @@ -114,7 +114,7 @@ var _ opentracing.Tracer = &Tracer{} // backends. func NewTracer() *Tracer { t := &Tracer{} - t.noopSpan = &span{tracer: t} + t.noopSpan = &Span{tracer: t} return t } @@ -183,11 +183,11 @@ type recordableOption struct{} // Apply is part of the opentracing.StartSpanOption interface. func (recordableOption) Apply(*opentracing.StartSpanOptions) {} -// Recordable is a StartSpanOption that forces creation of a real span. +// Recordable is a StartSpanOption that forces creation of a real Span. // // When tracing is disabled all spans are noopSpans; these spans aren't // capable of recording, so this option should be passed to StartSpan if the -// caller wants to be able to call StartRecording on the resulting span. +// caller wants to be able to call StartRecording on the resulting Span. var Recordable opentracing.StartSpanOption = recordableOption{} // StartSpan is part of the opentracing.Tracer interface. @@ -203,7 +203,7 @@ func (t *Tracer) StartSpan( ) opentracing.Span { // Fast paths to avoid the allocation of StartSpanOptions below when tracing // is disabled: if we have no options or a single SpanReference (the common - // case) with a noop context, return a noop span now. + // case) with a noop context, return a noop Span now. if len(opts) == 1 { if o, ok := opts[0].(opentracing.SpanReference); ok { if IsNoopContext(o.ReferencedContext) { @@ -233,7 +233,7 @@ func (t *Tracer) StartSpan( } var parentType opentracing.SpanReferenceType - var parentCtx spanContext + var parentCtx SpanContext // Note that the logic around here doesn't support spans with multiple // references. Luckily, we don't have such cases. @@ -251,7 +251,7 @@ func (t *Tracer) StartSpan( continue } parentType = r.Type - parentCtx = *r.ReferencedContext.(*spanContext) + parentCtx = *r.ReferencedContext.(*SpanContext) break } @@ -259,15 +259,15 @@ func (t *Tracer) StartSpan( operationName, parentCtx, parentType, recordable, logTags, false /* separateRecording */) } -// RecordableOpt specifies whether a root span should be recordable. +// RecordableOpt specifies whether a root Span should be recordable. type RecordableOpt bool const ( - // RecordableSpan means that the root span will be recordable. This means that - // a real span will be created (and so it carries a cost). + // RecordableSpan means that the root Span will be recordable. This means that + // a real Span will be created (and so it carries a cost). RecordableSpan RecordableOpt = true - // NonRecordableSpan means that the root span will not be recordable. This - // means that the static noop span might be returned. + // NonRecordableSpan means that the root Span will not be recordable. This + // means that the static noop Span might be returned. NonRecordableSpan RecordableOpt = false ) @@ -278,7 +278,7 @@ func (t *Tracer) AlwaysTrace() bool { return t.useNetTrace() || shadowTracer != nil || t.forceRealSpans } -// StartRootSpan creates a root span. This is functionally equivalent to: +// StartRootSpan creates a root Span. This is functionally equivalent to: // parentSpan.Tracer().(*Tracer).StartSpan(opName, LogTags(...), [Recordable]) // Compared to that, it's more efficient, particularly in terms of memory // allocations because the opentracing.StartSpanOption interface is not used. @@ -290,16 +290,16 @@ func (t *Tracer) StartRootSpan( return t.StartChildSpan(opName, SpanContext{}, logTags, false /* separateRecording */) } -// StartChildSpan creates a child span of the given parent span. This is +// StartChildSpan creates a child Span of the given parent Span. This is // functionally equivalent to: // parentSpan.Tracer().(*Tracer).StartSpan(opName, opentracing.ChildOf(parentSpan.Context())) // Compared to that, it's more efficient, particularly in terms of memory // allocations; among others, it saves the call to parentSpan.Context. // // This only works for creating children of local parents (i.e. the caller needs -// to have a reference to the parent span). +// to have a reference to the parent Span). // -// If separateRecording is true and the parent span is recording, the child's +// If separateRecording is true and the parent Span is recording, the child's // recording will not be part of the parent's recording. This is useful when the // child's recording will be reported to a collector separate from the parent's // recording; for example DistSQL processors each report their own recording, @@ -323,11 +323,11 @@ func (t *Tracer) StartChildSpan( // startSpanGeneric is the internal workhorse for creating spans. It serves two purposes: // -// 1. creating root spans. In this case, parentContext and parentType are zero. A noop span -// is returned when nothing forces an actual span to be created, i.e. there is no shadow +// 1. creating root spans. In this case, parentContext and parentType are zero. A noop Span +// is returned when nothing forces an actual Span to be created, i.e. there is no shadow // tracer and internal tracing active, plus no recordability is requested. // 2. creating derived spans. In this case, parentContext and parentType are nonzero. If the -// parent is not recording and 'recordable' is zero, and nothing else forces a real span, +// parent is not recording and 'recordable' is zero, and nothing else forces a real Span, // a noopSpan results. func (t *Tracer) startSpanGeneric( opName string, @@ -337,7 +337,7 @@ func (t *Tracer) startSpanGeneric( logTags *logtags.Buffer, separateRecording bool, ) opentracing.Span { - // If tracing is disabled, avoid overhead and return a noop span. + // If tracing is disabled, avoid overhead and return a noop Span. if !t.AlwaysTrace() && parentContext.TraceID == 0 && parentContext.recordingType == NoRecording && @@ -345,7 +345,7 @@ func (t *Tracer) startSpanGeneric( return t.noopSpan } - s := &span{ + s := &Span{ tracer: t, crdb: crdbSpan{ operation: opName, @@ -430,7 +430,7 @@ func (t *Tracer) Inject( return opentracing.ErrInvalidCarrier } - sc, ok := osc.(*spanContext) + sc, ok := osc.(*SpanContext) if !ok { return opentracing.ErrInvalidSpanContext } @@ -464,7 +464,7 @@ func (fn textMapReaderFn) ForeachKey(handler func(key, val string) error) error return fn(handler) } -var noopSpanContext = &spanContext{} +var noopSpanContext = &SpanContext{} // Extract is part of the opentracing.Tracer interface. // It always returns a valid context, even in error cases (this is assumed by the @@ -480,7 +480,7 @@ func (t *Tracer) Extract(format interface{}, carrier interface{}) (opentracing.S return noopSpanContext, opentracing.ErrInvalidCarrier } - var sc spanContext + var sc SpanContext var shadowType string var shadowCarrier opentracing.TextMapCarrier @@ -544,8 +544,8 @@ func (t *Tracer) Extract(format interface{}, carrier interface{}) (opentracing.S return &sc, nil } -// FinishSpan closes the given span (if not nil). It is a convenience wrapper -// for span.Finish() which tolerates nil spans. +// FinishSpan closes the given Span (if not nil). It is a convenience wrapper +// for Span.Finish() which tolerates nil spans. func FinishSpan(span opentracing.Span) { if span != nil { span.Finish() @@ -556,13 +556,13 @@ func FinishSpan(span opentracing.Span) { // that "follows from" the original Span. This allows the resulting context to be // used in an async task that might outlive the original operation. // -// Returns the new context and the new span (if any). The span should be +// Returns the new context and the new Span (if any). The Span should be // closed via FinishSpan. // // See also ChildSpan() for a "parent-child relationship". func ForkCtxSpan(ctx context.Context, opName string) (context.Context, opentracing.Span) { if sp := opentracing.SpanFromContext(ctx); sp != nil { - if sp.(*span).isNoop() { + if sp.(*Span).isNoop() { // Optimization: avoid ContextWithSpan call if tracing is disabled. return ctx, sp } @@ -577,17 +577,17 @@ func ForkCtxSpan(ctx context.Context, opName string) (context.Context, opentraci return ctx, nil } -// ChildSpan opens a span as a child of the current span in the context (if +// ChildSpan opens a Span as a child of the current Span in the context (if // there is one). -// The span's tags are inherited from the ctx's log tags automatically. +// The Span's tags are inherited from the ctx's log tags automatically. // -// Returns the new context and the new span (if any). The span should be +// Returns the new context and the new Span (if any). The Span should be // closed via FinishSpan. func ChildSpan(ctx context.Context, opName string) (context.Context, opentracing.Span) { return childSpan(ctx, opName, false /* separateRecording */) } -// ChildSpanSeparateRecording is like ChildSpan but the new span has separate +// ChildSpanSeparateRecording is like ChildSpan but the new Span has separate // recording (see StartChildSpan). func ChildSpanSeparateRecording( ctx context.Context, opName string, @@ -602,7 +602,7 @@ func childSpan( if otSpan == nil { return ctx, nil } - sp := otSpan.(*span) + sp := otSpan.(*Span) if sp.isNoop() { // Optimization: avoid ContextWithSpan call if tracing is disabled. return ctx, sp @@ -621,8 +621,8 @@ func childSpan( // Span. The returned closure must be called after the request has been fully // processed. // -// Note that, if there's already a span in the context, this method does nothing -// even if the current context's log tags are different from that span's tags. +// Note that, if there's already a Span in the context, this method does nothing +// even if the current context's log tags are different from that Span's tags. func EnsureContext( ctx context.Context, tracer opentracing.Tracer, opName string, ) (context.Context, func()) { @@ -634,10 +634,10 @@ func EnsureContext( } // EnsureChildSpan is the same as EnsureContext, except it creates a child -// span for the input context if the input context already has an active +// Span for the input context if the input context already has an active // trace. // -// The caller is responsible for closing the span (via Span.Finish). +// The caller is responsible for closing the Span (via Span.Finish). func EnsureChildSpan( ctx context.Context, tracer opentracing.Tracer, name string, ) (context.Context, opentracing.Span) { @@ -649,8 +649,8 @@ func EnsureChildSpan( } // StartSnowballTrace takes in a context and returns a derived one with a -// "snowball span" in it. The caller takes ownership of this span from the -// returned context and is in charge of Finish()ing it. The span has recording +// "snowball Span" in it. The caller takes ownership of this Span from the +// returned context and is in charge of Finish()ing it. The Span has recording // enabled. // // TODO(andrei): remove this method once EXPLAIN(TRACE) is gone. @@ -670,15 +670,15 @@ func StartSnowballTrace( } // TestingCheckRecordedSpans checks whether a recording looks like an expected -// one represented by a string with one line per expected span and one line per +// one represented by a string with one line per expected Span and one line per // expected event (i.e. log message). // // Use with something like: -// if err := TestingCheckRecordedSpans(tracing.GetRecording(span), ` -// span root: +// if err := TestingCheckRecordedSpans(tracing.GetRecording(Span), ` +// Span root: // event: a // event: c -// span child: +// Span child: // event: [ambient] b // `); err != nil { // t.Fatal(err) @@ -698,7 +698,7 @@ func TestingCheckRecordedSpans(recSpans []tracingpb.RecordedSpan, expected strin } for _, rs := range recSpans { - row("span %s:", rs.Operation) + row("Span %s:", rs.Operation) if len(rs.Tags) > 0 { var tags []string for k, v := range rs.Tags { @@ -750,7 +750,7 @@ func matchesWithoutFileLine(msg string, expected string) bool { return len(groups) == 3 && fmt.Sprintf("event: %s", groups[2]) == expected } -// ContextWithRecordingSpan returns a context with an embedded trace span which +// ContextWithRecordingSpan returns a context with an embedded trace Span which // returns its contents when getRecording is called and must be stopped by // calling the cancel method when done with the context (getRecording() needs to // be called before cancel()). diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index 3a2666a39b8a..f95abae96b31 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -34,16 +34,21 @@ import ( "golang.org/x/net/trace" ) -// spanMeta stores span information that is common to span and spanContext. +// spanMeta stores Span information that is common to Span and SpanContext. type spanMeta struct { - // A probabilistically unique identifier for a [multi-span] trace. + // A probabilistically unique identifier for a [multi-Span] trace. TraceID uint64 - // A probabilistically unique identifier for a span. + // A probabilistically unique identifier for a Span. SpanID uint64 } -type spanContext struct { +// SpanContext is information about a Span, used to derive spans +// from a parent in a way that's uniform between local and remote +// parents. For local parents, this generally references their Span +// to unlock features such as sharing recordings with the parent. For +// remote parents, it only contains the TraceID and related metadata. +type SpanContext struct { spanMeta // Underlying shadow tracer info and context (optional). @@ -57,32 +62,32 @@ type spanContext struct { // Children of remote spans act as roots when it comes to recordings - someone // is responsible for calling GetRecording() on them and marshaling the // recording back to the parent (generally an RPC handler does this). - span *span + span *Span - // The span's associated baggage. + // The Span's associated baggage. Baggage map[string]string } const ( // TagPrefix is prefixed to all tags that should be output in SHOW TRACE. TagPrefix = "cockroach." - // StatTagPrefix is prefixed to all stats output in span tags. + // StatTagPrefix is prefixed to all stats output in Span tags. StatTagPrefix = TagPrefix + "stat." ) -// SpanStats are stats that can be added to a span. +// SpanStats are stats that can be added to a Span. type SpanStats interface { proto.Message // Stats returns the stats that the object represents as a map from stat name - // to value to be added to span tags. The keys will be prefixed with + // to value to be added to Span tags. The keys will be prefixed with // StatTagPrefix. Stats() map[string]string } -var _ opentracing.SpanContext = &spanContext{} +var _ opentracing.SpanContext = &SpanContext{} // ForeachBaggageItem is part of the opentracing.SpanContext interface. -func (sc *spanContext) ForeachBaggageItem(handler func(k, v string) bool) { +func (sc *SpanContext) ForeachBaggageItem(handler func(k, v string) bool) { for k, v := range sc.Baggage { if !handler(k, v) { break @@ -90,21 +95,21 @@ func (sc *spanContext) ForeachBaggageItem(handler func(k, v string) bool) { } } -// RecordingType is the type of recording that a span might be performing. +// RecordingType is the type of recording that a Span might be performing. type RecordingType int const ( - // NoRecording means that the span isn't recording. Child spans created from + // NoRecording means that the Span isn't recording. Child spans created from // it similarly won't be recording by default. NoRecording RecordingType = iota - // SnowballRecording means that the span is recording and that derived + // SnowballRecording means that the Span is recording and that derived // spans will be as well, in the same mode (this includes remote spans, // i.e. this mode crosses RPC boundaries). Derived spans will maintain // their own recording, and this recording will be included in that of // any local parent spans. SnowballRecording - // SingleNodeRecording means that the span is recording and that locally - // derived spans will as well (i.e. a remote span typically won't be + // SingleNodeRecording means that the Span is recording and that locally + // derived spans will as well (i.e. a remote Span typically won't be // recording by default, in contrast to SnowballRecording). Similar to // SnowballRecording, children have their own recording which is also // included in that of their parents. @@ -119,11 +124,11 @@ type crdbSpan struct { operation string startTime time.Time - // logTags are set to the log tags that were available when this span was + // logTags are set to the log tags that were available when this Span was // created, so that there's no need to eagerly copy all of those log tags into - // this span's tags. If the span's tags are actually requested, these logTags + // this Span's tags. If the Span's tags are actually requested, these logTags // will be copied out at that point. - // Note that these tags have not gone through the log tag -> span tag + // Note that these tags have not gone through the log tag -> Span tag // remapping procedure; tagName() needs to be called before exposing each // tag's key to a user. logTags *logtags.Buffer @@ -140,7 +145,7 @@ type crdbSpan struct { recording struct { recordingType RecordingType recordedLogs []opentracing.LogRecord - // children contains the list of child spans started after this span + // children contains the list of child spans started after this Span // started recording. children []*crdbSpan // remoteSpan contains the list of remote child spans manually imported. @@ -148,16 +153,16 @@ type crdbSpan struct { } // tags are only set when recording. These are tags that have been added to - // this span, and will be appended to the tags in logTags when someone + // this Span, and will be appended to the tags in logTags when someone // needs to actually observe the total set of tags that is a part of this - // span. + // Span. // TODO(radu): perhaps we want a recording to capture all the tags (even // those that were set before recording started)? tags opentracing.Tags stats SpanStats - // The span's associated baggage. + // The Span's associated baggage. Baggage map[string]string } } @@ -174,68 +179,68 @@ type otSpan struct { shadowSpan opentracing.Span } -// span is the tracing span that we use in CockroachDB. Depending on the tracing configuration, +// Span is the tracing Span that we use in CockroachDB. Depending on the tracing configuration, // it can hold anywhere between zero and three destinations for trace information. // // The net/trace and opentracing spans are straightforward. If they are // set, we forward information to them; and depending on whether they are // set, spans descending from a parent will have these created as well. // -// The CockroachDB-internal span (crdbSpan) is more complex as it has multiple features: +// The CockroachDB-internal Span (crdbSpan) is more complex as it has multiple features: // // 1. recording: crdbSpan supports "recordings", meaning that it provides a way to extract -// the data logged into a trace span. -// 2. optimizations for the non-tracing case. If tracing is off and the span is not required -// to support recording (NoRecording), we still want to be able to have a cheap span +// the data logged into a trace Span. +// 2. optimizations for the non-tracing case. If tracing is off and the Span is not required +// to support recording (NoRecording), we still want to be able to have a cheap Span // to give to the caller. This is a) because it frees the caller from -// distinguishing the tracing and non-tracing cases, and b) because the span +// distinguishing the tracing and non-tracing cases, and b) because the Span // has the dual purpose of propagating the *Tracer around, which is needed // in case at some point down the line there is a need to create an actual -// span (for example, because a "recordable" child span is requested). +// Span (for example, because a "recordable" child Span is requested). // -// In these cases, we return a singleton span that is empty save for the tracer. +// In these cases, we return a singleton Span that is empty save for the tracer. // 3. snowball recording. As a special case of 1), we support a recording mode // (SnowballRecording) which propagates to child spans across RPC boundaries. -// 4. parent span recording. To make matters even more complex, there is a single-node -// recording option (SingleNodeRecording) in which the parent span keeps track of +// 4. parent Span recording. To make matters even more complex, there is a single-node +// recording option (SingleNodeRecording) in which the parent Span keeps track of // its local children and returns their recording in its own. // // TODO(tbg): investigate whether the tracer in 2) is really needed. // TODO(tbg): simplify the functionality of crdbSpan, which seems overly complex. -type span struct { +type Span struct { tracer *Tracer // never nil - // Internal trace span. Can be zero. + // Internal trace Span. Can be zero. crdb crdbSpan // x/net/trace.Trace instance; nil if not tracing to x/net/trace. netTr trace.Trace - // Shadow tracer and span; zero if not using a shadow tracer. + // Shadow tracer and Span; zero if not using a shadow tracer. ot otSpan } -// TODO(tbg): remove this. We don't need *span to be an opentracing.Span. -var _ opentracing.Span = &span{} +// TODO(tbg): remove this. We don't need *Span to be an opentracing.Span. +var _ opentracing.Span = &Span{} -func (s *span) isBlackHole() bool { +func (s *Span) isBlackHole() bool { return !s.crdb.isRecording() && s.netTr == nil && s.ot == (otSpan{}) } -func (s *span) isNoop() bool { +func (s *Span) isNoop() bool { // NB: this is the same as `s` being zero with the exception - // of the `tracer` field. However, `span` is not comparable, + // of the `tracer` field. However, `Span` is not comparable, // so this can't be expressed easily. return s.isBlackHole() && s.crdb.TraceID == 0 } -// IsRecording returns true if the span is recording its events. +// IsRecording returns true if the Span is recording its events. func IsRecording(s opentracing.Span) bool { - return s.(*span).crdb.isRecording() + return s.(*Span).crdb.isRecording() } -// enableRecording start recording on the span. From now on, log events and child spans +// enableRecording start recording on the Span. From now on, log events and child spans // will be stored. // -// If parent != nil, the span will be registered as a child of the respective +// If parent != nil, the Span will be registered as a child of the respective // parent. // If separate recording is specified, the child is not registered with the // parent. Thus, the parent's recording will not include this child. @@ -253,53 +258,53 @@ func (s *crdbSpan) enableRecording( s.setBaggageItemLocked(Snowball, "1") } // Clear any previously recorded info. This is needed by SQL SessionTracing, - // who likes to start and stop recording repeatedly on the same span, and + // who likes to start and stop recording repeatedly on the same Span, and // collect the (separate) recordings every time. s.mu.recording.recordedLogs = nil s.mu.recording.children = nil s.mu.recording.remoteSpans = nil } -// StartRecording enables recording on the span. Events from this point forward +// StartRecording enables recording on the Span. Events from this point forward // are recorded; also, all direct and indirect child spans started from now on // will be part of the same recording. // -// Recording is not supported by noop spans; to ensure a real span is always +// Recording is not supported by noop spans; to ensure a real Span is always // created, use the Recordable option to StartSpan. // -// If recording was already started on this span (either directly or because a -// parent span is recording), the old recording is lost. +// If recording was already started on this Span (either directly or because a +// parent Span is recording), the old recording is lost. // -// Children spans created from the span while it is *not* recording will not +// Children spans created from the Span while it is *not* recording will not // necessarily be recordable. func StartRecording(os opentracing.Span, recType RecordingType) { if recType == NoRecording { panic("StartRecording called with NoRecording") } - sp := os.(*span) + sp := os.(*Span) if sp.isNoop() { panic("StartRecording called on NoopSpan; use the Recordable option for StartSpan") } // If we're already recording (perhaps because the parent was recording when - // this span was created), there's nothing to do. + // this Span was created), there's nothing to do. if !sp.crdb.isRecording() { sp.crdb.enableRecording(nil /* parent */, recType, false /* separateRecording */) } } -// StopRecording disables recording on this span. Child spans that were created +// StopRecording disables recording on this Span. Child spans that were created // since recording was started will continue to record until they finish. // // Calling this after StartRecording is not required; the recording will go away // when all the spans finish. // -// StopRecording() can be called on a Finish()ed span. +// StopRecording() can be called on a Finish()ed Span. func StopRecording(os opentracing.Span) { - os.(*span).disableRecording() + os.(*Span).disableRecording() } -func (s *span) disableRecording() { +func (s *Span) disableRecording() { s.crdb.disableRecording() } @@ -307,7 +312,7 @@ func (s *crdbSpan) disableRecording() { s.mu.Lock() defer s.mu.Unlock() atomic.StoreInt32(&s.recording, 0) - // We test the duration as a way to check if the span has been finished. If it + // We test the duration as a way to check if the Span has been finished. If it // has, we don't want to do the call below as it might crash (at least if // there's a netTr). if (s.mu.duration == -1) && (s.mu.recording.recordingType == SnowballRecording) { @@ -318,12 +323,12 @@ func (s *crdbSpan) disableRecording() { } // IsRecordable returns true if {Start,Stop}Recording() can be called on this -// span. +// Span. // -// In other words, this tests if the span is our custom type, and not a noopSpan +// In other words, this tests if the Span is our custom type, and not a noopSpan // or anything else. func IsRecordable(os opentracing.Span) bool { - _, isCockroachSpan := os.(*span) + _, isCockroachSpan := os.(*Span) return isCockroachSpan } @@ -331,11 +336,11 @@ func IsRecordable(os opentracing.Span) bool { // Spans are sorted by StartTime. type Recording []tracingpb.RecordedSpan -// GetRecording retrieves the current recording, if the span has recording +// GetRecording retrieves the current recording, if the Span has recording // enabled. This can be called while spans that are part of the recording are // still open; it can run concurrently with operations on those spans. func GetRecording(os opentracing.Span) Recording { - return os.(*span).crdb.getRecording() + return os.(*Span).crdb.getRecording() } func (s *crdbSpan) getRecording() Recording { @@ -356,7 +361,7 @@ func (s *crdbSpan) getRecording() Recording { result = append(result, child.getRecording()...) } - // Sort the spans by StartTime, except the first span (the root of this + // Sort the spans by StartTime, except the first Span (the root of this // recording) which stays in place. toSort := result[1:] sort.Slice(toSort, func(i, j int) bool { @@ -370,12 +375,12 @@ type traceLogData struct { depth int // timeSincePrev represents the duration since the previous log line (previous in the // set of log lines that this is part of). This is always computed relative to a log line - // from the same span, except for start of span in which case the duration is computed relative + // from the same Span, except for start of Span in which case the duration is computed relative // to the last log in the parent occurring before this start. For example: - // start span A - // log 1 // duration relative to "start span A" - // start span B // duration relative to "log 1" - // log 2 // duration relative to "start span B" + // start Span A + // log 1 // duration relative to "start Span A" + // start Span B // duration relative to "log 1" + // log 2 // duration relative to "start Span B" // log 3 // duration relative to "log 1" timeSincePrev time.Duration } @@ -390,9 +395,9 @@ type traceLogData struct { // // Each log line show the time since the beginning of the trace // and since the previous log line. Span starts are shown with special "=== -// " lines. For a span start, the time since the relative log line -// can be negative when the span start follows a message from the parent that -// was generated after the child span started (or even after the child +// " lines. For a Span start, the time since the relative log line +// can be negative when the Span start follows a message from the parent that +// was generated after the child Span started (or even after the child // finished). // // TODO(andrei): this should be unified with @@ -449,7 +454,7 @@ func (r Recording) OrphanSpans() []tracingpb.RecordedSpan { var orphans []tracingpb.RecordedSpan for i, sp := range r { if i == 0 { - // The first span can be a root span. Note that any other root span will + // The first Span can be a root Span. Note that any other root Span will // be considered an orphan. continue } @@ -475,8 +480,8 @@ func (r Recording) FindLogMessage(pattern string) (string, bool) { return "", false } -// FindSpan returns the span with the given operation. The bool retval is false -// if the span is not found. +// FindSpan returns the Span with the given operation. The bool retval is false +// if the Span is not found. func (r Recording) FindSpan(operation string) (tracingpb.RecordedSpan, bool) { for _, sp := range r { if sp.Operation == operation { @@ -488,7 +493,7 @@ func (r Recording) FindSpan(operation string) (tracingpb.RecordedSpan, bool) { // visitSpan returns the log messages for sp, and all of sp's children. // -// All messages from a span are kept together. Sibling spans are ordered within +// All messages from a Span are kept together. Sibling spans are ordered within // the parent in their start order. func (r Recording) visitSpan(sp tracingpb.RecordedSpan, depth int) []traceLogData { ownLogs := make([]traceLogData, 0, len(sp.Logs)+1) @@ -505,7 +510,7 @@ func (r Recording) visitSpan(sp tracingpb.RecordedSpan, depth int) []traceLogDat } } - // Add a log line representing the start of the span. + // Add a log line representing the start of the Span. lr := opentracing.LogRecord{ Timestamp: sp.StartTime, Fields: []otlog.Field{otlog.String("=== operation", sp.Operation)}, @@ -604,11 +609,11 @@ func (r Recording) ToJaegerJSON(stmt string) (string, error) { return jaegerjson.SpanID(strconv.FormatUint(spanID, 10)) } - // Each span in Jaeger belongs to a "process" that generated it. Spans + // Each Span in Jaeger belongs to a "process" that generated it. Spans // belonging to different colors are colored differently in Jaeger. We're // going to map our different nodes to different processes. processes := make(map[jaegerjson.ProcessID]jaegerjson.Process) - // getProcessID figures out what "process" a span belongs to. It looks for an + // getProcessID figures out what "process" a Span belongs to. It looks for an // "node: " tag. The processes map is populated with an entry for every // node present in the trace. getProcessID := func(sp tracingpb.RecordedSpan) jaegerjson.ProcessID { @@ -698,20 +703,20 @@ type TraceCollection struct { Data []jaegerjson.Trace `json:"data"` } -// ImportRemoteSpans adds RecordedSpan data to the recording of the given span; +// ImportRemoteSpans adds RecordedSpan data to the recording of the given Span; // these spans will be part of the result of GetRecording. Used to import // recorded traces from other nodes. func ImportRemoteSpans(os opentracing.Span, remoteSpans []tracingpb.RecordedSpan) error { - return os.(*span).crdb.ImportRemoteSpans(remoteSpans) + return os.(*Span).crdb.ImportRemoteSpans(remoteSpans) } func (s *crdbSpan) ImportRemoteSpans(remoteSpans []tracingpb.RecordedSpan) error { if !s.isRecording() { - return errors.AssertionFailedf("adding Raw Spans to a span that isn't recording") + return errors.AssertionFailedf("adding Raw Spans to a Span that isn't recording") } - // Change the root of the remote recording to be a child of this span. This is + // Change the root of the remote recording to be a child of this Span. This is // usually already the case, except with DistSQL traces where remote - // processors run in spans that FollowFrom an RPC span that we don't collect. + // processors run in spans that FollowFrom an RPC Span that we don't collect. remoteSpans[0].ParentSpanID = s.SpanID s.mu.Lock() @@ -720,37 +725,37 @@ func (s *crdbSpan) ImportRemoteSpans(remoteSpans []tracingpb.RecordedSpan) error return nil } -// IsBlackHoleSpan returns true if events for this span are just dropped. This -// is the case when the span is not recording and no external tracer is configured. +// IsBlackHoleSpan returns true if events for this Span are just dropped. This +// is the case when the Span is not recording and no external tracer is configured. // Tracing clients can use this method to figure out if they can short-circuit some // tracing-related work that would be discarded anyway. // -// The child of a blackhole span is a non-recordable blackhole span[*]. These incur +// The child of a blackhole Span is a non-recordable blackhole Span[*]. These incur // only minimal overhead. It is therefore not worth it to call this method to avoid // starting spans. func IsBlackHoleSpan(s opentracing.Span) bool { - sp := s.(*span) + sp := s.(*Span) return sp.isBlackHole() } -// IsNoopContext returns true if the span context is from a "no-op" span. If -// this is true, any span derived from this context will be a "black hole span". +// IsNoopContext returns true if the Span context is from a "no-op" Span. If +// this is true, any Span derived from this context will be a "black hole Span". // // You should never need to care about this method. It is exported for technical // rasons. func IsNoopContext(spanCtx opentracing.SpanContext) bool { - sc := spanCtx.(*spanContext) + sc := spanCtx.(*SpanContext) return sc.isNoop() } -func (sc *spanContext) isNoop() bool { +func (sc *SpanContext) isNoop() bool { return sc.recordingType == NoRecording && sc.shadowTr == nil } -// SetSpanStats sets the stats on a span. stats.Stats() will also be added to -// the span tags. +// SetSpanStats sets the stats on a Span. stats.Stats() will also be added to +// the Span tags. func SetSpanStats(os opentracing.Span, stats SpanStats) { - s := os.(*span) + s := os.(*Span) s.crdb.mu.Lock() s.crdb.mu.stats = stats for name, value := range stats.Stats() { @@ -760,12 +765,12 @@ func SetSpanStats(os opentracing.Span, stats SpanStats) { } // Finish is part of the opentracing.Span interface. -func (s *span) Finish() { +func (s *Span) Finish() { s.FinishWithOptions(opentracing.FinishOptions{}) } // FinishWithOptions is part of the opentracing.Span interface. -func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { +func (s *Span) FinishWithOptions(opts opentracing.FinishOptions) { finishTime := opts.FinishTime if finishTime.IsZero() { finishTime = time.Now() @@ -783,32 +788,19 @@ func (s *span) FinishWithOptions(opts opentracing.FinishOptions) { // Context is part of the opentracing.Span interface. // -// TODO(andrei, radu): Should this return noopSpanContext for a Recordable span +// TODO(andrei, radu): Should this return noopSpanContext for a Recordable Span // that's not currently recording? That might save work and allocations when // creating child spans. -func (s *span) Context() opentracing.SpanContext { +func (s *Span) Context() opentracing.SpanContext { s.crdb.mu.Lock() defer s.crdb.mu.Unlock() sc := s.SpanContext() return &sc } -// TODO(tbg): these are temporary to get things to compile without larger -// mechanical refactors. - -// SpanContext is information about a span, used to derive spans -// from a parent in a way that's uniform between local and remote -// parents. For local parents, this generally references their Span -// to unlock features such as sharing recordings with the parent. For -// remote parents, it only contains the TraceID and related metadata. -type SpanContext = spanContext - -// Span is the tracing span used throughout CockroachDB. -type Span = span - -// spanContext returns a spanContext. Note that this returns a value, +// SpanContext returns a SpanContext. Note that this returns a value, // not a pointer, which the caller can use to avoid heap allocations. -func (s *span) SpanContext() SpanContext { +func (s *Span) SpanContext() SpanContext { n := len(s.crdb.mu.Baggage) // In the common case, we have no baggage, so avoid making an empty map. var baggageCopy map[string]string @@ -818,7 +810,7 @@ func (s *span) SpanContext() SpanContext { for k, v := range s.crdb.mu.Baggage { baggageCopy[k] = v } - sc := spanContext{ + sc := SpanContext{ spanMeta: s.crdb.spanMeta, span: s, Baggage: baggageCopy, @@ -835,7 +827,7 @@ func (s *span) SpanContext() SpanContext { } // SetOperationName is part of the opentracing.Span interface. -func (s *span) SetOperationName(operationName string) opentracing.Span { +func (s *Span) SetOperationName(operationName string) opentracing.Span { if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetOperationName(operationName) } @@ -844,7 +836,7 @@ func (s *span) SetOperationName(operationName string) opentracing.Span { } // SetTag is part of the opentracing.Span interface. -func (s *span) SetTag(key string, value interface{}) opentracing.Span { +func (s *Span) SetTag(key string, value interface{}) opentracing.Span { return s.setTagInner(key, value, false /* locked */) } @@ -855,14 +847,14 @@ func (s *crdbSpan) setTagLocked(key string, value interface{}) { s.mu.tags[key] = value } -func (s *span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { +func (s *Span) setTagInner(key string, value interface{}, locked bool) opentracing.Span { if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetTag(key, value) } if s.netTr != nil { s.netTr.LazyPrintf("%s:%v", key, value) } - // The internal tags will be used if we start a recording on this span. + // The internal tags will be used if we start a recording on this Span. if !locked { s.crdb.mu.Lock() defer s.crdb.mu.Unlock() @@ -872,7 +864,7 @@ func (s *span) setTagInner(key string, value interface{}, locked bool) opentraci } // LogFields is part of the opentracing.Span interface. -func (s *span) LogFields(fields ...otlog.Field) { +func (s *Span) LogFields(fields ...otlog.Field) { if s.ot.shadowSpan != nil { s.ot.shadowSpan.LogFields(fields...) } @@ -912,7 +904,7 @@ func (s *crdbSpan) LogFields(fields ...otlog.Field) { } // LogKV is part of the opentracing.Span interface. -func (s *span) LogKV(alternatingKeyValues ...interface{}) { +func (s *Span) LogKV(alternatingKeyValues ...interface{}) { fields, err := otlog.InterleavedKVToFields(alternatingKeyValues...) if err != nil { s.LogFields(otlog.Error(err), otlog.String("function", "LogKV")) @@ -922,7 +914,7 @@ func (s *span) LogKV(alternatingKeyValues ...interface{}) { } // SetBaggageItem is part of the opentracing.Span interface. -func (s *span) SetBaggageItem(restrictedKey, value string) opentracing.Span { +func (s *Span) SetBaggageItem(restrictedKey, value string) opentracing.Span { s.crdb.SetBaggageItemAndTag(restrictedKey, value) if s.ot.shadowSpan != nil { s.ot.shadowSpan.SetBaggageItem(restrictedKey, value) @@ -953,7 +945,7 @@ func (s *crdbSpan) setBaggageItemLocked(restrictedKey, value string) { } // BaggageItem is part of the opentracing.Span interface. -func (s *span) BaggageItem(restrictedKey string) string { +func (s *Span) BaggageItem(restrictedKey string) string { if s := s.crdb.BaggageItem(restrictedKey); s != "" { return s } @@ -970,26 +962,26 @@ func (s *crdbSpan) BaggageItem(restrictedKey string) string { } // Tracer is part of the opentracing.Span interface. -func (s *span) Tracer() opentracing.Tracer { +func (s *Span) Tracer() opentracing.Tracer { return s.tracer } // LogEvent is part of the opentracing.Span interface. Deprecated. -func (s *span) LogEvent(event string) { +func (s *Span) LogEvent(event string) { s.LogFields(otlog.String(tracingpb.LogMessageField, event)) } // LogEventWithPayload is part of the opentracing.Span interface. Deprecated. -func (s *span) LogEventWithPayload(event string, payload interface{}) { +func (s *Span) LogEventWithPayload(event string, payload interface{}) { s.LogFields(otlog.String(tracingpb.LogMessageField, event), otlog.Object("payload", payload)) } // Log is part of the opentracing.Span interface. Deprecated. -func (s *span) Log(data opentracing.LogData) { +func (s *Span) Log(data opentracing.LogData) { panic("unimplemented") } -// getRecordingLocked returns the span's recording. This does not include +// getRecordingLocked returns the Span's recording. This does not include // children. func (s *crdbSpan) getRecordingLocked() tracingpb.RecordedSpan { rs := tracingpb.RecordedSpan{ @@ -1010,7 +1002,7 @@ func (s *crdbSpan) getRecordingLocked() tracingpb.RecordedSpan { switch rs.Duration { case -1: - // -1 indicates an unfinished span. For a recording it's better to put some + // -1 indicates an unfinished Span. For a recording it's better to put some // duration in it, otherwise tools get confused. For example, we export // recordings to Jaeger, and spans with a zero duration don't look nice. rs.Duration = timeutil.Now().Sub(rs.StartTime) diff --git a/pkg/util/tracing/tracer_span_test.go b/pkg/util/tracing/tracer_span_test.go index 36183bd1ecf1..62f2b3d0a2d8 100644 --- a/pkg/util/tracing/tracer_span_test.go +++ b/pkg/util/tracing/tracer_span_test.go @@ -28,7 +28,7 @@ func TestRecordingString(t *testing.T) { tr2 := NewTracer() root := tr.StartSpan("root", Recordable) - rootSp := root.(*span) + rootSp := root.(*Span) StartRecording(root, SnowballRecording) root.LogFields(otlog.String(tracingpb.LogMessageField, "root 1")) // Hackily fix the timing on the first log message, so that we can check it later. @@ -65,17 +65,17 @@ func TestRecordingString(t *testing.T) { // Sanity check that the recording looks like we want. Note that this is not // its String() representation; this just list all the spans in order. err = TestingCheckRecordedSpans(rec, ` -span root: +Span root: tags: sb=1 event: root 1 event: root 2 event: root 3 event: root 4 event: root 5 -span remote child: +Span remote child: tags: sb=1 event: remote child 1 -span local child: +Span local child: tags: sb=1 event: local child 1 `) @@ -157,19 +157,19 @@ func TestRecordingInRecording(t *testing.T) { rootRec := GetRecording(root) require.NoError(t, TestingCheckRecordedSpans(rootRec, ` -span root: +Span root: tags: sb=1 -span child: +Span child: tags: sb=1 -span grandchild: +Span grandchild: tags: sb=1 `)) childRec := GetRecording(child) require.NoError(t, TestingCheckRecordedSpans(childRec, ` -span child: +Span child: tags: sb=1 -span grandchild: +Span grandchild: tags: sb=1 `)) diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index dfb94f34162d..d03243870d4a 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -22,30 +22,30 @@ func TestTracerRecording(t *testing.T) { tr := NewTracer() noop1 := tr.StartSpan("noop") - if !noop1.(*span).isNoop() { - t.Error("expected noop span") + if !noop1.(*Span).isNoop() { + t.Error("expected noop Span") } noop1.LogKV("hello", "void") noop2 := tr.StartSpan("noop2", opentracing.ChildOf(noop1.Context())) - if !noop2.(*span).isNoop() { - t.Error("expected noop child span") + if !noop2.(*Span).isNoop() { + t.Error("expected noop child Span") } noop2.Finish() noop1.Finish() s1 := tr.StartSpan("a", Recordable) - if s1.(*span).isNoop() { - t.Error("Recordable (but not recording) span should not be noop") + if s1.(*Span).isNoop() { + t.Error("Recordable (but not recording) Span should not be noop") } if !IsBlackHoleSpan(s1) { - t.Error("Recordable span should be black hole") + t.Error("Recordable Span should be black hole") } // Unless recording is actually started, child spans are still noop. noop3 := tr.StartSpan("noop3", opentracing.ChildOf(s1.Context())) - if !noop3.(*span).isNoop() { - t.Error("expected noop child span") + if !noop3.(*Span).isNoop() { + t.Error("expected noop child Span") } noop3.Finish() @@ -54,15 +54,15 @@ func TestTracerRecording(t *testing.T) { s1.LogKV("x", 2) s2 := tr.StartSpan("b", opentracing.ChildOf(s1.Context())) if IsBlackHoleSpan(s2) { - t.Error("recording span should not be black hole") + t.Error("recording Span should not be black hole") } s2.LogKV("x", 3) if err := TestingCheckRecordedSpans(GetRecording(s1), ` - span a: + Span a: tags: unfinished= x: 2 - span b: + Span b: tags: unfinished= x: 3 `); err != nil { @@ -70,7 +70,7 @@ func TestTracerRecording(t *testing.T) { } if err := TestingCheckRecordedSpans(GetRecording(s2), ` - span b: + Span b: tags: unfinished= x: 3 `); err != nil { @@ -84,12 +84,12 @@ func TestTracerRecording(t *testing.T) { s2.Finish() if err := TestingCheckRecordedSpans(GetRecording(s1), ` - span a: + Span a: tags: unfinished= x: 2 - span b: + Span b: x: 3 - span c: + Span c: tags: tag=val unfinished= x: 4 `); err != nil { @@ -97,12 +97,12 @@ func TestTracerRecording(t *testing.T) { } s3.Finish() if err := TestingCheckRecordedSpans(GetRecording(s1), ` - span a: + Span a: tags: unfinished= x: 2 - span b: + Span b: x: 3 - span c: + Span c: tags: tag=val x: 4 `); err != nil { @@ -114,10 +114,10 @@ func TestTracerRecording(t *testing.T) { t.Fatal(err) } - // The child span is still recording. + // The child Span is still recording. s3.LogKV("x", 5) if err := TestingCheckRecordedSpans(GetRecording(s3), ` - span c: + Span c: tags: tag=val x: 4 x: 5 @@ -135,8 +135,8 @@ func TestStartChildSpan(t *testing.T) { sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` - span parent: - span child: + Span parent: + Span child: `); err != nil { t.Fatal(err) } @@ -147,12 +147,12 @@ func TestStartChildSpan(t *testing.T) { sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` - span parent: + Span parent: `); err != nil { t.Fatal(err) } if err := TestingCheckRecordedSpans(GetRecording(sp2), ` - span child: + Span child: `); err != nil { t.Fatal(err) } @@ -165,8 +165,8 @@ func TestStartChildSpan(t *testing.T) { sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` - span parent: - span child: + Span parent: + Span child: tags: key=val `); err != nil { t.Fatal(err) @@ -180,27 +180,27 @@ func TestTracerInjectExtract(t *testing.T) { // Verify that noop spans become noop spans on the remote side. noop1 := tr.StartSpan("noop") - if !noop1.(*span).isNoop() { - t.Fatalf("expected noop span: %+v", noop1) + if !noop1.(*Span).isNoop() { + t.Fatalf("expected noop Span: %+v", noop1) } carrier := make(opentracing.HTTPHeadersCarrier) if err := tr.Inject(noop1.Context(), opentracing.HTTPHeaders, carrier); err != nil { t.Fatal(err) } if len(carrier) != 0 { - t.Errorf("noop span has carrier: %+v", carrier) + t.Errorf("noop Span has carrier: %+v", carrier) } wireContext, err := tr2.Extract(opentracing.HTTPHeaders, carrier) if err != nil { t.Fatal(err) } - if !wireContext.(*spanContext).isNoop() { + if !wireContext.(*SpanContext).isNoop() { t.Errorf("expected noop context: %v", wireContext) } noop2 := tr2.StartSpan("remote op", opentracing.FollowsFrom(wireContext)) - if !noop2.(*span).isNoop() { - t.Fatalf("expected noop span: %+v", noop2) + if !noop2.(*Span).isNoop() { + t.Fatalf("expected noop Span: %+v", noop2) } noop1.Finish() noop2.Finish() @@ -223,8 +223,8 @@ func TestTracerInjectExtract(t *testing.T) { s2 := tr2.StartSpan("remote op", opentracing.FollowsFrom(wireContext)) // Compare TraceIDs - trace1 := s1.Context().(*spanContext).TraceID - trace2 := s2.Context().(*spanContext).TraceID + trace1 := s1.Context().(*SpanContext).TraceID + trace2 := s2.Context().(*SpanContext).TraceID if trace1 != trace2 { t.Errorf("TraceID doesn't match: parent %d child %d", trace1, trace2) } @@ -234,7 +234,7 @@ func TestTracerInjectExtract(t *testing.T) { // Verify that recording was started automatically. rec := GetRecording(s2) if err := TestingCheckRecordedSpans(rec, ` - span remote op: + Span remote op: tags: sb=1 x: 1 `); err != nil { @@ -242,7 +242,7 @@ func TestTracerInjectExtract(t *testing.T) { } if err := TestingCheckRecordedSpans(GetRecording(s1), ` - span a: + Span a: tags: sb=1 unfinished= `); err != nil { t.Fatal(err) @@ -254,9 +254,9 @@ func TestTracerInjectExtract(t *testing.T) { s1.Finish() if err := TestingCheckRecordedSpans(GetRecording(s1), ` - span a: + Span a: tags: sb=1 - span remote op: + Span remote op: tags: sb=1 x: 1 `); err != nil { @@ -295,7 +295,7 @@ func TestLightstepContext(t *testing.T) { } s2 := tr.StartSpan("child", opentracing.FollowsFrom(wireContext)) - s2Ctx := s2.(*span).ot.shadowSpan.Context() + s2Ctx := s2.(*Span).ot.shadowSpan.Context() // Verify that the baggage is correct in both the tracer context and in the // lightstep context. From 9f5a99e94ca7eeb36fe2fc12727e8172f69288e5 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 15:50:55 +0200 Subject: [PATCH 19/20] tracing: don't ignore recordability in StartChildSpan Improve commentary about recordability while I'm there. Release note: None --- pkg/sql/distsql/server.go | 2 +- pkg/util/tracing/tracer.go | 25 +++++++++++++++++-------- pkg/util/tracing/tracer_span_test.go | 2 +- pkg/util/tracing/tracer_test.go | 6 +++--- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 1d13788062bc..43feb3b366bb 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -206,7 +206,7 @@ func (ds *ServerImpl) setupFlow( // If that field is unset, we might still want to create a child span if // this flow is run synchronously. sp = ds.Tracer.(*tracing.Tracer).StartChildSpan( - opName, parentSpan.(*tracing.Span).SpanContext(), logtags.FromContext(ctx), + opName, parentSpan.(*tracing.Span).SpanContext(), logtags.FromContext(ctx), tracing.NonRecordableSpan, false /* separateRecording */) } else { // We use FollowsFrom because the flow's span outlives the SetupFlow request. diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 9249b27311ee..44f37100f2e4 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -259,15 +259,20 @@ func (t *Tracer) StartSpan( operationName, parentCtx, parentType, recordable, logTags, false /* separateRecording */) } -// RecordableOpt specifies whether a root Span should be recordable. +// RecordableOpt specifies whether a Span should be recordable. +// +// By default, spans derived from a *recording* (on top of recordable) span +// are recordable (and recording). Otherwise, they are non-recordable. type RecordableOpt bool const ( - // RecordableSpan means that the root Span will be recordable. This means that + // RecordableSpan means that a Span will be recordable. This means that // a real Span will be created (and so it carries a cost). RecordableSpan RecordableOpt = true - // NonRecordableSpan means that the root Span will not be recordable. This - // means that the static noop Span might be returned. + // NonRecordableSpan means that a Span will not be recordable. Using this + // option when possible can improve performance. + // + // TODO(tbg): with always-on tracing, it won't matter. NonRecordableSpan RecordableOpt = false ) @@ -287,7 +292,7 @@ func (t *Tracer) AlwaysTrace() bool { func (t *Tracer) StartRootSpan( opName string, logTags *logtags.Buffer, recordable RecordableOpt, ) opentracing.Span { - return t.StartChildSpan(opName, SpanContext{}, logTags, false /* separateRecording */) + return t.StartChildSpan(opName, SpanContext{}, logTags, recordable, false /* separateRecording */) } // StartChildSpan creates a child Span of the given parent Span. This is @@ -309,13 +314,17 @@ func (t *Tracer) StartRootSpan( // TODO(tbg): I don't think we need separateRecording because children can consume // and (atomically) clear their recording to avoid it getting consumed twice. func (t *Tracer) StartChildSpan( - opName string, parentContext SpanContext, logTags *logtags.Buffer, separateRecording bool, + opName string, + parentContext SpanContext, + logTags *logtags.Buffer, + recordable RecordableOpt, + separateRecording bool, ) opentracing.Span { return t.startSpanGeneric( opName, parentContext, opentracing.ChildOfRef, - false, /* recordable */ + recordable, logTags, separateRecording, ) @@ -612,7 +621,7 @@ func childSpan( ns := tr.noopSpan return opentracing.ContextWithSpan(ctx, ns), ns } - newSpan := tr.StartChildSpan(opName, sp.SpanContext(), logtags.FromContext(ctx), separateRecording) + newSpan := tr.StartChildSpan(opName, sp.SpanContext(), logtags.FromContext(ctx), NonRecordableSpan, separateRecording) return opentracing.ContextWithSpan(ctx, newSpan), newSpan } diff --git a/pkg/util/tracing/tracer_span_test.go b/pkg/util/tracing/tracer_span_test.go index 62f2b3d0a2d8..958475962f03 100644 --- a/pkg/util/tracing/tracer_span_test.go +++ b/pkg/util/tracing/tracer_span_test.go @@ -53,7 +53,7 @@ func TestRecordingString(t *testing.T) { root.LogFields(otlog.String(tracingpb.LogMessageField, "root 3")) - ch2 := tr.StartChildSpan("local child", root.(*Span).SpanContext(), nil /* logTags */, false /* separateRecording */) + ch2 := tr.StartChildSpan("local child", root.(*Span).SpanContext(), nil /* logTags */, false /* recordable */, false /* separateRecording */) root.LogFields(otlog.String(tracingpb.LogMessageField, "root 4")) ch2.LogFields(otlog.String(tracingpb.LogMessageField, "local child 1")) ch2.Finish() diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index d03243870d4a..2d8f587cc00d 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -131,7 +131,7 @@ func TestStartChildSpan(t *testing.T) { tr := NewTracer() sp1 := tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) - sp2 := tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, false /*separateRecording*/) + sp2 := tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, false /* recordable */, false /*separateRecording*/) sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` @@ -143,7 +143,7 @@ func TestStartChildSpan(t *testing.T) { sp1 = tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) - sp2 = tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, true /*separateRecording*/) + sp2 = tr.StartChildSpan("child", sp1.(*Span).SpanContext(), nil /* logTags */, false /* recordable */, true /*separateRecording*/) sp2.Finish() sp1.Finish() if err := TestingCheckRecordedSpans(GetRecording(sp1), ` @@ -160,7 +160,7 @@ func TestStartChildSpan(t *testing.T) { sp1 = tr.StartSpan("parent", Recordable) StartRecording(sp1, SingleNodeRecording) sp2 = tr.StartChildSpan( - "child", sp1.(*Span).SpanContext(), logtags.SingleTagBuffer("key", "val"), false, /*separateRecording*/ + "child", sp1.(*Span).SpanContext(), logtags.SingleTagBuffer("key", "val"), false /* recordable */, false, /*separateRecording*/ ) sp2.Finish() sp1.Finish() From 2ace2ab92a05c8ffc0f1e0b522544b856e2a880a Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 16 Oct 2020 16:48:20 +0200 Subject: [PATCH 20/20] tracing: respect sso.Tags in StartSpan This got lost during the refactors and caused some test failures. Release note: None --- pkg/util/tracing/tracer.go | 5 ++++- pkg/util/tracing/tracer_span.go | 7 +++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 44f37100f2e4..5124874df9ef 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -256,7 +256,7 @@ func (t *Tracer) StartSpan( } return t.startSpanGeneric( - operationName, parentCtx, parentType, recordable, logTags, false /* separateRecording */) + operationName, parentCtx, parentType, recordable, logTags, false /* separateRecording */, sso.Tags) } // RecordableOpt specifies whether a Span should be recordable. @@ -327,6 +327,7 @@ func (t *Tracer) StartChildSpan( recordable, logTags, separateRecording, + nil, /* tags */ ) } @@ -345,6 +346,7 @@ func (t *Tracer) startSpanGeneric( recordable RecordableOpt, logTags *logtags.Buffer, separateRecording bool, + tags map[string]interface{}, ) opentracing.Span { // If tracing is disabled, avoid overhead and return a noop Span. if !t.AlwaysTrace() && @@ -364,6 +366,7 @@ func (t *Tracer) startSpanGeneric( }, } s.crdb.mu.duration = -1 // unfinished + s.crdb.mu.tags = tags // Copy baggage from parent. // diff --git a/pkg/util/tracing/tracer_span.go b/pkg/util/tracing/tracer_span.go index f95abae96b31..1adac43d6bea 100644 --- a/pkg/util/tracing/tracer_span.go +++ b/pkg/util/tracing/tracer_span.go @@ -305,6 +305,9 @@ func StopRecording(os opentracing.Span) { } func (s *Span) disableRecording() { + if s.isNoop() { + panic("can't disable recording a noop Span") + } s.crdb.disableRecording() } @@ -328,8 +331,8 @@ func (s *crdbSpan) disableRecording() { // In other words, this tests if the Span is our custom type, and not a noopSpan // or anything else. func IsRecordable(os opentracing.Span) bool { - _, isCockroachSpan := os.(*Span) - return isCockroachSpan + sp, isCockroachSpan := os.(*Span) + return isCockroachSpan && !sp.isNoop() } // Recording represents a group of RecordedSpans, as returned by GetRecording.