diff --git a/pkg/kv/kvserver/concurrency/BUILD.bazel b/pkg/kv/kvserver/concurrency/BUILD.bazel index 24e6721ffb46..2315600f9854 100644 --- a/pkg/kv/kvserver/concurrency/BUILD.bazel +++ b/pkg/kv/kvserver/concurrency/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", + "//pkg/util/tracing", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 8888b91d31e4..14cd6e20f844 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -59,6 +59,7 @@ type Config struct { // Configs + Knobs. MaxLockTableSize int64 DisableTxnPushing bool + OnContentionEvent func(*roachpb.ContentionEvent) // may be nil; allowed to mutate the event TxnWaitKnobs txnwait.TestingKnobs } @@ -92,6 +93,7 @@ func NewManager(cfg Config) Manager { ir: cfg.IntentResolver, lt: lt, disableTxnPushing: cfg.DisableTxnPushing, + onContentionEvent: cfg.OnContentionEvent, }, // TODO(nvanbenschoten): move pkg/storage/txnwait to a new // pkg/storage/concurrency/txnwait package. diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index 71af16437848..affee03eb6e9 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -164,16 +164,17 @@ func TestConcurrencyManagerBasic(t *testing.T) { } latchSpans, lockSpans := c.collectSpans(t, txn, ts, reqs) - c.requestsByName[reqName] = concurrency.Request{ - Txn: txn, - Timestamp: ts, - // TODO(nvanbenschoten): test Priority - ReadConsistency: readConsistency, - WaitPolicy: waitPolicy, - Requests: reqUnions, - LatchSpans: latchSpans, - LockSpans: lockSpans, - } + c.requestsByName[reqName] = testReq{ + Request: concurrency.Request{ + Txn: txn, + Timestamp: ts, + // TODO(nvanbenschoten): test Priority + ReadConsistency: readConsistency, + WaitPolicy: waitPolicy, + Requests: reqUnions, + LatchSpans: latchSpans, + LockSpans: lockSpans, + }} return "" case "sequence": @@ -190,8 +191,8 @@ func TestConcurrencyManagerBasic(t *testing.T) { c.mu.Unlock() opName := fmt.Sprintf("sequence %s", reqName) - mon.runAsync(opName, func(ctx context.Context) { - guard, resp, err := m.SequenceReq(ctx, prev, req) + cancel := mon.runAsync(opName, func(ctx context.Context) { + guard, resp, err := m.SequenceReq(ctx, prev, req.Request) if err != nil { log.Eventf(ctx, "sequencing complete, returned error: %v", err) } else if resp != nil { @@ -205,6 +206,8 @@ func TestConcurrencyManagerBasic(t *testing.T) { log.Event(ctx, "sequencing complete, returned no guard") } }) + req.cancel = cancel + c.requestsByName[reqName] = req return c.waitAndCollect(t, mon) case "finish": @@ -477,6 +480,11 @@ func TestConcurrencyManagerBasic(t *testing.T) { }) } +type testReq struct { + cancel func() + concurrency.Request +} + // cluster encapsulates the state of a running cluster and a set of requests. // It serves as the test harness in TestConcurrencyManagerBasic - maintaining // transaction and request declarations, recording the state of in-flight @@ -491,7 +499,7 @@ type cluster struct { // Definitions. txnCounter uint32 txnsByName map[string]*roachpb.Transaction - requestsByName map[string]concurrency.Request + requestsByName map[string]testReq // Request state. Cleared on reset. mu syncutil.Mutex @@ -511,6 +519,7 @@ type txnRecord struct { type txnPush struct { ctx context.Context pusher, pushee uuid.UUID + count int } func newCluster() *cluster { @@ -520,7 +529,7 @@ func newCluster() *cluster { rangeDesc: &roachpb.RangeDescriptor{RangeID: 1}, txnsByName: make(map[string]*roachpb.Transaction), - requestsByName: make(map[string]concurrency.Request), + requestsByName: make(map[string]testReq), guardsByReqName: make(map[string]*concurrency.Guard), txnRecords: make(map[uuid.UUID]*txnRecord), txnPushes: make(map[uuid.UUID]*txnPush), @@ -533,6 +542,9 @@ func (c *cluster) makeConfig() concurrency.Config { RangeDesc: c.rangeDesc, Settings: c.st, IntentResolver: c, + OnContentionEvent: func(ev *roachpb.ContentionEvent) { + ev.Duration = 1234 * time.Millisecond // for determinism + }, TxnWaitMetrics: txnwait.NewMetrics(time.Minute), } } @@ -680,11 +692,18 @@ func (r *txnRecord) asTxn() (*roachpb.Transaction, chan struct{}) { func (c *cluster) registerPush(ctx context.Context, pusher, pushee uuid.UUID) (*txnPush, error) { c.mu.Lock() defer c.mu.Unlock() - if _, ok := c.txnPushes[pusher]; ok { - return nil, errors.Errorf("txn %v already pushing", pusher) + if p, ok := c.txnPushes[pusher]; ok { + if pushee != p.pushee { + return nil, errors.Errorf("pusher %s can't push two txns %s and %s at the same time", + pusher.Short(), pushee.Short(), p.pushee.Short(), + ) + } + p.count++ + return p, nil } p := &txnPush{ ctx: ctx, + count: 1, pusher: pusher, pushee: pushee, } @@ -695,7 +714,17 @@ func (c *cluster) registerPush(ctx context.Context, pusher, pushee uuid.UUID) (* func (c *cluster) unregisterPush(push *txnPush) { c.mu.Lock() defer c.mu.Unlock() - delete(c.txnPushes, push.pusher) + p, ok := c.txnPushes[push.pusher] + if !ok { + return + } + p.count-- + if p.count == 0 { + delete(c.txnPushes, push.pusher) + } + if p.count < 0 { + panic(fmt.Sprintf("negative count: %+v", p)) + } } // detectDeadlocks looks at all in-flight transaction pushes and determines @@ -792,7 +821,7 @@ func (c *cluster) resetNamespace() { defer c.mu.Unlock() c.txnCounter = 0 c.txnsByName = make(map[string]*roachpb.Transaction) - c.requestsByName = make(map[string]concurrency.Request) + c.requestsByName = make(map[string]testReq) c.txnRecords = make(map[uuid.UUID]*txnRecord) } @@ -871,7 +900,7 @@ func (m *monitor) runSync(opName string, fn func(context.Context)) { atomic.StoreInt32(&g.finished, 1) } -func (m *monitor) runAsync(opName string, fn func(context.Context)) { +func (m *monitor) runAsync(opName string, fn func(context.Context)) (cancel func()) { m.seq++ ctx, collect, cancel := tracing.ContextWithRecordingSpan(context.Background(), opName) g := &monitoredGoroutine{ @@ -887,6 +916,7 @@ func (m *monitor) runAsync(opName string, fn func(context.Context)) { fn(ctx) atomic.StoreInt32(&g.finished, 1) }() + return cancel } func (m *monitor) numMonitored() int { diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index ad22e3aba5c5..f64f834e07a6 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -80,7 +80,7 @@ type waitingState struct { // Represents who the request is waiting for. The conflicting // transaction may be a lock holder of a conflicting lock or a // conflicting request being sequenced through the same lockTable. - txn *enginepb.TxnMeta // always non-nil + txn *enginepb.TxnMeta // always non-nil in waitFor{,Distinguished,Self} key roachpb.Key // the key of the conflict held bool // is the conflict a held lock? @@ -968,7 +968,12 @@ func (l *lockState) informActiveWaiters() { g := qg.guard var state waitingState if g.isSameTxnAsReservation(waitForState) { - state = waitingState{kind: waitSelf} + state = waitingState{ + kind: waitSelf, + key: waitForState.key, + txn: waitForState.txn, + held: waitForState.held, // false + } } else { state = waitForState state.guardAccess = spanset.SpanReadWrite @@ -1339,7 +1344,12 @@ func (l *lockState) tryActiveWait( g.key = l.key g.mu.startWait = true if g.isSameTxnAsReservation(waitForState) { - g.mu.state = waitingState{kind: waitSelf} + g.mu.state = waitingState{ + kind: waitSelf, + key: waitForState.key, + txn: waitForState.txn, + held: waitForState.held, // false + } } else { state := waitForState state.guardAccess = sa diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index 826be2269263..b60b796775a0 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -11,6 +11,7 @@ package concurrency import ( + "bytes" "context" "math" "time" @@ -26,6 +27,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/uuid" "github.com/cockroachdb/errors" ) @@ -99,6 +101,9 @@ type lockTableWaiterImpl struct { // When set, WriteIntentError are propagated instead of pushing // conflicting transactions. disableTxnPushing bool + // When set, called just before each ContentionEvent is emitted. + // Is allowed to mutate the event. + onContentionEvent func(ev *roachpb.ContentionEvent) } // IntentResolver is an interface used by lockTableWaiterImpl to push @@ -131,11 +136,24 @@ func (w *lockTableWaiterImpl) WaitOn( var timer *timeutil.Timer var timerC <-chan time.Time var timerWaitingState waitingState + + h := contentionEventHelper{ + sp: tracing.SpanFromContext(ctx), + onEvent: w.onContentionEvent, + } + defer h.emit() + for { select { + // newStateC will be signaled for the transaction we are currently + // contending on. We will continue to receive updates about this + // transaction until it no longer contends with us, at which point + // either one of the other channels fires or we receive state + // about another contending transaction on newStateC. case <-newStateC: timerC = nil state := guard.CurState() + h.emitAndInit(state) switch state.kind { case waitFor, waitForDistinguished: if req.WaitPolicy == lock.WaitPolicy_Error { @@ -654,6 +672,75 @@ func (c *txnCache) insertFrontLocked(txn *roachpb.Transaction) { c.txns[0] = txn } +// contentionEventHelper tracks and emits ContentionEvents. +type contentionEventHelper struct { + sp *tracing.Span + onEvent func(event *roachpb.ContentionEvent) // may be nil + + // Internal. + ev *roachpb.ContentionEvent + tBegin time.Time +} + +// emit emits the open contention event, if any. +func (h *contentionEventHelper) emit() { + if h.ev == nil { + return + } + h.ev.Duration = timeutil.Since(h.tBegin) + if h.onEvent != nil { + // NB: this is intentionally above the call to LogStructured so that + // this interceptor gets to mutate the event (used for test determinism). + h.onEvent(h.ev) + } + h.sp.LogStructured(h.ev) + h.ev = nil +} + +// emitAndInit compares the waitingState's active txn (if any) against the current +// ContentionEvent (if any). If the they match, we are continuing to handle the +// same event and no action is taken. If they differ, the open event (if any) is +// finalized and added to the Span, and a new event initialized from the inputs. +func (h *contentionEventHelper) emitAndInit(s waitingState) { + if h.sp == nil { + // No span to attach payloads to - don't do any work. + // + // TODO(tbg): we could special case the noop span here too, but the plan is for + // nobody to use noop spans any more (trace.mode=background). + return + } + + // If true, we want to emit the current event and possibly start a new one. + // Otherwise, + switch s.kind { + case waitFor, waitForDistinguished, waitSelf: + // If we're tracking an event and see a different txn/key, the event is + // done and we initialize the new event tracking the new txn/key. + // + // NB: we're guaranteed to have `s.{txn,key}` populated here. + if h.ev != nil && + (!h.ev.TxnMeta.ID.Equal(s.txn.ID) || !bytes.Equal(h.ev.Key, s.key)) { + h.emit() // h.ev is now nil + } + + if h.ev == nil { + h.ev = &roachpb.ContentionEvent{ + Key: s.key, + TxnMeta: *s.txn, + } + h.tBegin = timeutil.Now() + } + case waitElsewhere, doneWaiting: + // If we have an event, emit it now and that's it - the case we're in + // does not give us a new transaction/key. + if h.ev != nil { + h.emit() + } + default: + panic("unhandled waitingState.kind") + } +} + func newWriteIntentErr(ws waitingState) *Error { return roachpb.NewError(&roachpb.WriteIntentError{ Intents: []roachpb.Intent{roachpb.MakeIntent(ws.txn, ws.key)}, diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 2ae9ee912137..57819aebbcb2 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/stretchr/testify/require" ) @@ -343,7 +344,11 @@ func testWaitNoopUntilDone(t *testing.T, k waitKind, makeReq func() Request) { w, _, g := setupLockTableWaiterTest() defer w.stopper.Stop(ctx) - g.state = waitingState{kind: k} + txn := makeTxnProto("noop-wait-txn") + g.state = waitingState{ + kind: k, + txn: &txn.TxnMeta, + } g.notify() defer notifyUntilDone(t, g)() @@ -645,3 +650,49 @@ func BenchmarkTxnCache(b *testing.B) { } } } + +func TestContentionEventHelper(t *testing.T) { + // This is mostly a regression test that ensures that we don't + // accidentally update tBegin when continuing to handle the same event. + // General coverage of the helper results from TestConcurrencyManagerBasic. + + tr := tracing.NewTracer() + sp := tr.StartSpan("foo", tracing.WithForceRealSpan()) + + var sl []*roachpb.ContentionEvent + h := contentionEventHelper{ + sp: sp, + onEvent: func(ev *roachpb.ContentionEvent) { + sl = append(sl, ev) + }, + } + txn := makeTxnProto("foo") + h.emitAndInit(waitingState{ + kind: waitForDistinguished, + key: roachpb.Key("a"), + txn: &txn.TxnMeta, + }) + require.Empty(t, sl) + require.NotZero(t, h.tBegin) + tBegin := h.tBegin + + // Another event for the same txn/key should not mutate tBegin + // or emit an event. + h.emitAndInit(waitingState{ + kind: waitFor, + key: roachpb.Key("a"), + txn: &txn.TxnMeta, + }) + require.Empty(t, sl) + require.Equal(t, tBegin, h.tBegin) + + h.emitAndInit(waitingState{ + kind: waitForDistinguished, + key: roachpb.Key("b"), + txn: &txn.TxnMeta, + }) + require.Len(t, sl, 1) + require.Equal(t, txn.TxnMeta, sl[0].TxnMeta) + require.Equal(t, roachpb.Key("a"), sl[0].Key) + require.NotZero(t, sl[0].Duration) +} diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic index 083adfa9d6c9..c3b75b23521f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic @@ -123,6 +123,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [2] sequence req3: resolving intent "k" for txn 00000002 with COMMITTED status +[2] sequence req3: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req3: acquiring latches [2] sequence req3: scanning lock table for conflicting locks [2] sequence req3: sequencing complete, returned guard @@ -215,9 +216,11 @@ on-txn-updated txn=txn2 status=pending ts=18,1 ---- [-] update txn: increasing timestamp of txn2 [2] sequence req5: resolving intent "k" for txn 00000002 with PENDING status +[2] sequence req5: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req5: acquiring latches [2] sequence req5: scanning lock table for conflicting locks [2] sequence req5: sequencing complete, returned guard +[3] sequence req6: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s [3] sequence req6: acquiring latches [3] sequence req6: scanning lock table for conflicting locks [3] sequence req6: sequencing complete, returned guard @@ -248,6 +251,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req7: resolving intent "k" for txn 00000002 with COMMITTED status +[4] sequence req7: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s [4] sequence req7: acquiring latches [4] sequence req7: scanning lock table for conflicting locks [4] sequence req7: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents index ed110c41bfd4..6f997847fa1c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -102,6 +102,7 @@ on-txn-updated txn=txn2 status=aborted ---- [-] update txn: aborting txn2 [3] sequence req1: resolving intent "a" for txn 00000002 with ABORTED status +[3] sequence req1: conflicted with 00000002-0000-0000-0000-000000000000 on "a" for 1.23s [3] sequence req1: resolving a batch of 9 intent(s) [3] sequence req1: resolving intent "b" for txn 00000002 with ABORTED status [3] sequence req1: resolving intent "c" for txn 00000002 with ABORTED status @@ -178,6 +179,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [3] sequence req1: resolving intent "a" for txn 00000002 with COMMITTED status +[3] sequence req1: conflicted with 00000002-0000-0000-0000-000000000000 on "a" for 1.23s [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: sequencing complete, returned guard @@ -362,6 +364,7 @@ on-txn-updated txn=txn2 status=aborted ---- [-] update txn: aborting txn2 [4] sequence req1: resolving intent "a" for txn 00000002 with ABORTED status +[4] sequence req1: conflicted with 00000002-0000-0000-0000-000000000000 on "a" for 1.23s [4] sequence req1: resolving a batch of 1 intent(s) [4] sequence req1: resolving intent "b" for txn 00000002 with ABORTED status [4] sequence req1: acquiring latches @@ -576,9 +579,11 @@ on-txn-updated txn=txn3 status=aborted ---- [-] update txn: aborting txn3 [3] sequence req1: resolving intent "c" for txn 00000003 with ABORTED status +[3] sequence req1: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [3] sequence req1: pushing txn 00000005 to abort [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction [6] sequence req2: resolving intent "a" for txn 00000003 with ABORTED status +[6] sequence req2: conflicted with 00000003-0000-0000-0000-000000000000 on "a" for 1.23s [6] sequence req2: pushing timestamp of txn 00000004 above 11.000000000,1 [6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -609,6 +614,7 @@ on-txn-updated txn=txn4 status=aborted ---- [-] update txn: aborting txn4 [6] sequence req2: resolving intent "b" for txn 00000004 with ABORTED status +[6] sequence req2: conflicted with 00000004-0000-0000-0000-000000000000 on "b" for 1.23s [6] sequence req2: resolving a batch of 1 intent(s) [6] sequence req2: resolving intent "d" for txn 00000003 with ABORTED status [6] sequence req2: acquiring latches @@ -639,6 +645,7 @@ on-txn-updated txn=txn5 status=aborted ---- [-] update txn: aborting txn5 [3] sequence req1: resolving intent "e" for txn 00000005 with ABORTED status +[3] sequence req1: conflicted with 00000005-0000-0000-0000-000000000000 on "e" for 1.23s [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks index 3bec6cb7831e..c1c674dc0dec 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks @@ -161,8 +161,10 @@ on-txn-updated txn=txn1 status=aborted ---- [-] update txn: aborting txn1 [4] sequence req1r: detected pusher aborted +[4] sequence req1r: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [4] sequence req1r: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): [6] sequence req3r: resolving intent "a" for txn 00000001 with ABORTED status +[6] sequence req3r: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [6] sequence req3r: acquiring latches [6] sequence req3r: scanning lock table for conflicting locks [6] sequence req3r: sequencing complete, returned guard @@ -176,6 +178,7 @@ on-txn-updated txn=txn3 status=committed ---- [-] update txn: committing txn3 [5] sequence req2r: resolving intent "c" for txn 00000003 with COMMITTED status +[5] sequence req2r: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [5] sequence req2r: acquiring latches [5] sequence req2r: scanning lock table for conflicting locks [5] sequence req2r: sequencing complete, returned guard @@ -376,12 +379,15 @@ on-txn-updated txn=txn1 status=aborted ---- [-] update txn: aborting txn1 [4] sequence req4w: resolving intent "a" for txn 00000001 with ABORTED status +[4] sequence req4w: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [4] sequence req4w: acquiring latches [4] sequence req4w: scanning lock table for conflicting locks [4] sequence req4w: sequencing complete, returned guard [5] sequence req1w2: detected pusher aborted +[5] sequence req1w2: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): [7] sequence req3w2: resolving intent "a" for txn 00000001 with ABORTED status +[7] sequence req3w2: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [7] sequence req3w2: pushing txn 00000004 to detect request deadlock [7] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -389,6 +395,7 @@ on-txn-updated txn=txn1 status=aborted finish req=req4w ---- [-] finish req4w: finishing request +[7] sequence req3w2: conflicted with 00000004-0000-0000-0000-000000000000 on "a" for 1.23s [7] sequence req3w2: acquiring latches [7] sequence req3w2: scanning lock table for conflicting locks [7] sequence req3w2: sequencing complete, returned guard @@ -402,6 +409,7 @@ on-txn-updated txn=txn3 status=committed ---- [-] update txn: committing txn3 [6] sequence req2w2: resolving intent "c" for txn 00000003 with COMMITTED status +[6] sequence req2w2: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [6] sequence req2w2: acquiring latches [6] sequence req2w2: scanning lock table for conflicting locks [6] sequence req2w2: sequencing complete, returned guard @@ -537,6 +545,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req4w: resolving intent "b" for txn 00000002 with COMMITTED status +[4] sequence req4w: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [4] sequence req4w: pushing txn 00000003 to abort [4] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -612,7 +621,9 @@ on-txn-updated txn=txn4 status=aborted ---- [-] update txn: aborting txn4 [4] sequence req4w: detected pusher aborted +[4] sequence req4w: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [4] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): +[5] sequence req1w2: conflicted with 00000004-0000-0000-0000-000000000000 on "b" for 1.23s [5] sequence req1w2: acquiring latches [5] sequence req1w2: scanning lock table for conflicting locks [5] sequence req1w2: sequencing complete, returned guard @@ -626,6 +637,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [6] sequence req3w2: resolving intent "a" for txn 00000001 with COMMITTED status +[6] sequence req3w2: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: sequencing complete, returned guard @@ -761,6 +773,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req4w: resolving intent "b" for txn 00000002 with COMMITTED status +[4] sequence req4w: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [4] sequence req4w: pushing txn 00000003 to abort [4] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -836,8 +849,10 @@ on-txn-updated txn=txn1 status=aborted ---- [-] update txn: aborting txn1 [5] sequence req1w2: detected pusher aborted +[5] sequence req1w2: conflicted with 00000004-0000-0000-0000-000000000000 on "b" for 1.23s [5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): [6] sequence req3w2: resolving intent "a" for txn 00000001 with ABORTED status +[6] sequence req3w2: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: sequencing complete, returned guard @@ -851,6 +866,7 @@ on-txn-updated txn=txn3 status=committed ---- [-] update txn: committing txn3 [4] sequence req4w: resolving intent "c" for txn 00000003 with COMMITTED status +[4] sequence req4w: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [4] sequence req4w: acquiring latches [4] sequence req4w: scanning lock table for conflicting locks [4] sequence req4w: sequencing complete, returned guard @@ -1006,6 +1022,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [5] sequence req4w: resolving intent "a" for txn 00000001 with COMMITTED status +[5] sequence req4w: conflicted with 00000001-0000-0000-0000-000000000000 on "a" for 1.23s [5] sequence req4w: pushing txn 00000002 to abort [5] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -1013,9 +1030,11 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req5w: resolving intent "b" for txn 00000002 with COMMITTED status +[4] sequence req5w: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [4] sequence req5w: pushing txn 00000003 to abort [4] sequence req5w: blocked on select in concurrency_test.(*cluster).PushTransaction [5] sequence req4w: resolving intent "b" for txn 00000002 with COMMITTED status +[5] sequence req4w: conflicted with 00000002-0000-0000-0000-000000000000 on "b" for 1.23s [5] sequence req4w: pushing txn 00000005 to detect request deadlock [5] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -1081,7 +1100,9 @@ on-txn-updated txn=txn4 status=aborted ---- [-] update txn: aborting txn4 [5] sequence req4w: detected pusher aborted +[5] sequence req4w: conflicted with 00000005-0000-0000-0000-000000000000 on "b" for 1.23s [5] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): +[6] sequence req3w2: conflicted with 00000004-0000-0000-0000-000000000000 on "a" for 1.23s [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: sequencing complete, returned guard @@ -1095,6 +1116,7 @@ on-txn-updated txn=txn3 status=committed ---- [-] update txn: committing txn3 [4] sequence req5w: resolving intent "c" for txn 00000003 with COMMITTED status +[4] sequence req5w: conflicted with 00000003-0000-0000-0000-000000000000 on "c" for 1.23s [4] sequence req5w: acquiring latches [4] sequence req5w: scanning lock table for conflicting locks [4] sequence req5w: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race index 79b552e44052..de4ececdc98f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race @@ -180,10 +180,12 @@ on-txn-updated txn=txn3 status=committed ---- [-] update txn: committing txn3 [5] sequence req4: resolving intent "k" for txn 00000003 with COMMITTED status +[5] sequence req4: conflicted with 00000003-0000-0000-0000-000000000000 on "k" for 1.23s [5] sequence req4: acquiring latches [5] sequence req4: scanning lock table for conflicting locks [5] sequence req4: sequencing complete, returned guard [7] sequence req2: resolving intent "k" for txn 00000003 with COMMITTED status +[7] sequence req2: conflicted with 00000003-0000-0000-0000-000000000000 on "k" for 1.23s [7] sequence req2: acquiring latches [7] sequence req2: scanning lock table for conflicting locks [7] sequence req2: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock index d2505161371d..f97c3962e024 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock @@ -45,6 +45,7 @@ on-txn-updated txn=txn1 status=aborted ---- [-] update txn: aborting txn1 [3] sequence req1: resolving intent "k" for txn 00000001 with ABORTED status +[3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener index 78d24ff6b059..b20129e8d9ee 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -112,6 +112,7 @@ local: num=0 on-lease-updated leaseholder=false lease-seq=2 ---- [-] transfer lease: released +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -260,6 +261,7 @@ sequence req=reqRes2 on-lock-updated req=reqRes2 txn=txn2 key=k status=committed ---- [-] update lock: committing txn 00000002 @ k +[11] sequence req3: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s [11] sequence req3: acquiring latches [11] sequence req3: blocked on select in spanlatch.(*Manager).waitForSignal @@ -374,6 +376,7 @@ local: num=0 on-split ---- [-] split range: complete +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -418,6 +421,7 @@ sequence req=reqRes1 on-lock-updated req=reqRes1 txn=txn1 key=k status=committed ---- [-] update lock: committing txn 00000001 @ k +[4] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [4] sequence req2: acquiring latches [4] sequence req2: blocked on select in spanlatch.(*Manager).waitForSignal @@ -544,6 +548,7 @@ local: num=0 on-merge ---- [-] merge range: complete +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -692,6 +697,7 @@ local: num=0 on-snapshot-applied ---- [-] snapshot replica: applied +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -736,6 +742,7 @@ sequence req=reqRes1 on-lock-updated req=reqRes1 txn=txn1 key=k status=committed ---- [-] update lock: committing txn 00000001 @ k +[4] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [4] sequence req2: acquiring latches [4] sequence req2: blocked on select in spanlatch.(*Manager).waitForSignal diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index 841ec0064efe..d282b510e1b5 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -47,6 +47,7 @@ on-txn-updated txn=txn1 status=pending ts=15,2 ---- [-] update txn: increasing timestamp of txn1 [3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status +[3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: sequencing complete, returned guard @@ -157,10 +158,12 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [3] sequence req1: resolving intent "k" for txn 00000001 with COMMITTED status +[3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: sequencing complete, returned guard [5] sequence req2-retry: resolving intent "k" for txn 00000001 with COMMITTED status +[5] sequence req2-retry: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [5] sequence req2-retry: acquiring latches [5] sequence req2-retry: scanning lock table for conflicting locks [5] sequence req2-retry: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update index 75879afad3cd..d542aa6f5b8f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -70,6 +70,7 @@ on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 [2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -188,6 +189,7 @@ on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 [2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -315,6 +317,7 @@ on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 [2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard @@ -386,6 +389,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [3] sequence req4: resolving intent "k" for txn 00000001 with COMMITTED status +[3] sequence req4: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [3] sequence req4: acquiring latches [3] sequence req4: scanning lock table for conflicting locks [3] sequence req4: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error index 4c88fe2ad979..4ce065b29d26 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -104,6 +104,7 @@ sequence req=reqNoWait1 [4] sequence reqNoWait1: waiting in lock wait-queues [4] sequence reqNoWait1: pushing txn 00000001 to check if abandoned [4] sequence reqNoWait1: pushee not abandoned +[4] sequence reqNoWait1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s [4] sequence reqNoWait1: sequencing complete, returned error: conflicting intents on "k" # ------------------------------------------------------------- @@ -115,6 +116,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [3] sequence req3: resolving intent "k2" for txn 00000001 with COMMITTED status +[3] sequence req3: conflicted with 00000001-0000-0000-0000-000000000000 on "k2" for 1.23s [3] sequence req3: pushing txn 00000002 to abort [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -157,6 +159,7 @@ sequence req=reqNoWait2 [6] sequence reqNoWait2: acquiring latches [6] sequence reqNoWait2: scanning lock table for conflicting locks [6] sequence reqNoWait2: waiting in lock wait-queues +[6] sequence reqNoWait2: conflicted with 00000003-0000-0000-0000-000000000000 on "k2" for 1.23s [6] sequence reqNoWait2: sequencing complete, returned error: conflicting intents on "k2" # ------------------------------------------------------------- @@ -188,6 +191,7 @@ sequence req=reqNoWait3 [9] sequence reqNoWait3: waiting in lock wait-queues [9] sequence reqNoWait3: pushing txn 00000002 to check if abandoned [9] sequence reqNoWait3: pushee not abandoned +[9] sequence reqNoWait3: conflicted with 00000002-0000-0000-0000-000000000000 on "k4" for 1.23s [9] sequence reqNoWait3: sequencing complete, returned error: conflicting intents on "k4" debug-lock-table @@ -213,6 +217,7 @@ on-txn-updated txn=txn2 status=aborted ---- [-] update txn: aborting txn2 [3] sequence req3: resolving intent "k3" for txn 00000002 with ABORTED status +[3] sequence req3: conflicted with 00000002-0000-0000-0000-000000000000 on "k3" for 1.23s [3] sequence req3: acquiring latches [3] sequence req3: scanning lock table for conflicting locks [3] sequence req3: sequencing complete, returned guard diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self new file mode 100644 index 000000000000..624ffbcbbe69 --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self @@ -0,0 +1,128 @@ +# Our main txn that will get into state waitSelf. +new-txn name=txn ts=10,1 epoch=0 +---- + +# Will hold a lock discovered by txn1. +new-txn name=txnOld ts=10,1 epoch=0 +---- + +# Will separate two requests made by the main txn. +new-txn name=txnMiddle ts=10,1 epoch=0 +---- + +new-request name=reqOld txn=txnOld ts=10,1 + put key=k value=v +---- + +new-request name=reqTxn1 txn=txn ts=10,1 + put key=k value=w +---- + +new-request name=reqTxnMiddle txn=txnMiddle ts=10,1 + put key=k value=w +---- + +new-request name=reqTxn2 txn=txn ts=20,1 + put key=k value=x +---- + +sequence req=reqOld +---- +[1] sequence reqOld: sequencing request +[1] sequence reqOld: acquiring latches +[1] sequence reqOld: scanning lock table for conflicting locks +[1] sequence reqOld: sequencing complete, returned guard + +on-lock-acquired req=reqOld key=k +---- +[-] acquire lock: txn 00000002 @ k + +finish req=reqOld +---- +[-] finish reqOld: finishing request + +sequence req=reqTxn1 +---- +[2] sequence reqTxn1: sequencing request +[2] sequence reqTxn1: acquiring latches +[2] sequence reqTxn1: scanning lock table for conflicting locks +[2] sequence reqTxn1: waiting in lock wait-queues +[2] sequence reqTxn1: pushing txn 00000002 to abort +[2] sequence reqTxn1: blocked on select in concurrency_test.(*cluster).PushTransaction + +sequence req=reqTxnMiddle +---- +[3] sequence reqTxnMiddle: sequencing request +[3] sequence reqTxnMiddle: acquiring latches +[3] sequence reqTxnMiddle: scanning lock table for conflicting locks +[3] sequence reqTxnMiddle: waiting in lock wait-queues +[3] sequence reqTxnMiddle: pushing txn 00000002 to abort +[3] sequence reqTxnMiddle: blocked on select in concurrency_test.(*cluster).PushTransaction + +sequence req=reqTxn2 +---- +[4] sequence reqTxn2: sequencing request +[4] sequence reqTxn2: acquiring latches +[4] sequence reqTxn2: scanning lock table for conflicting locks +[4] sequence reqTxn2: waiting in lock wait-queues +[4] sequence reqTxn2: pushing txn 00000002 to abort +[4] sequence reqTxn2: blocked on select in concurrency_test.(*cluster).PushTransaction + +on-txn-updated txn=txnOld status=committed +---- +[-] update txn: committing txnOld +[2] sequence reqTxn1: resolving intent "k" for txn 00000002 with COMMITTED status +[2] sequence reqTxn1: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s +[2] sequence reqTxn1: acquiring latches +[2] sequence reqTxn1: scanning lock table for conflicting locks +[2] sequence reqTxn1: sequencing complete, returned guard +[3] sequence reqTxnMiddle: resolving intent "k" for txn 00000002 with COMMITTED status +[3] sequence reqTxnMiddle: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s +[3] sequence reqTxnMiddle: pushing txn 00000001 to detect request deadlock +[3] sequence reqTxnMiddle: blocked on select in concurrency_test.(*cluster).PushTransaction +[4] sequence reqTxn2: resolving intent "k" for txn 00000002 with COMMITTED status +[4] sequence reqTxn2: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 1.23s +[4] sequence reqTxn2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn + +debug-lock-table +---- +global: num=1 + lock: "k" + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 + active: true req: 4, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 3 +local: num=0 + + +# This is the interesting step - we see reqTxn2 announce that it conflicted with +# itself (txn #1), which indicates transitioning out of the waitSelf state. +# +# TODO(nvanbenschoten): would be nice to make this more explicit, but I'm not sure +# how to change `(*lockTableImpl).Format` to make it happen in debug-lock-table above. +finish req=reqTxn1 +---- +[-] finish reqTxn1: finishing request +[3] sequence reqTxnMiddle: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s +[3] sequence reqTxnMiddle: acquiring latches +[3] sequence reqTxnMiddle: scanning lock table for conflicting locks +[3] sequence reqTxnMiddle: sequencing complete, returned guard +[4] sequence reqTxn2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s +[4] sequence reqTxn2: pushing txn 00000003 to detect request deadlock +[4] sequence reqTxn2: blocked on select in concurrency_test.(*cluster).PushTransaction + +finish req=reqTxnMiddle +---- +[-] finish reqTxnMiddle: finishing request +[4] sequence reqTxn2: conflicted with 00000003-0000-0000-0000-000000000000 on "k" for 1.23s +[4] sequence reqTxn2: acquiring latches +[4] sequence reqTxn2: scanning lock table for conflicting locks +[4] sequence reqTxn2: sequencing complete, returned guard + +finish req=reqTxn2 +---- +[-] finish reqTxn2: finishing request + +reset +---- diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 2ce4cc2e40c9..08cd781a8d1f 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" ) // UserPriority is a custom type for transaction's user priority. @@ -1519,3 +1520,13 @@ func (r *JoinNodeResponse) CreateStoreIdent() (StoreIdent, error) { } return sIdent, nil } + +// SafeFormat implements redact.SafeFormatter. +func (c *ContentionEvent) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("conflicted with %s on %s for %.2fs", c.TxnMeta.ID, c.Key, c.Duration.Seconds()) +} + +// String implements fmt.Stringer. +func (c *ContentionEvent) String() string { + return redact.StringWithoutMarkers(c) +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 287f37d3f1de..5dbc03ed243e 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -74,7 +74,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{0} + return fileDescriptor_api_e9d594aefb977c8a, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -102,7 +102,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{1} + return fileDescriptor_api_e9d594aefb977c8a, []int{1} } type ChecksumMode int32 @@ -149,7 +149,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{2} + return fileDescriptor_api_e9d594aefb977c8a, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -180,7 +180,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{3} + return fileDescriptor_api_e9d594aefb977c8a, []int{3} } type ExternalStorageProvider int32 @@ -221,7 +221,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{4} + return fileDescriptor_api_e9d594aefb977c8a, []int{4} } type MVCCFilter int32 @@ -244,7 +244,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{5} + return fileDescriptor_api_e9d594aefb977c8a, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{1, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{25, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -333,7 +333,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_5f7f23e8e4f12d4b, []int{0} + return fileDescriptor_api_e9d594aefb977c8a, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -405,7 +405,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_5f7f23e8e4f12d4b, []int{1} + return fileDescriptor_api_e9d594aefb977c8a, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +439,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_5f7f23e8e4f12d4b, []int{2} + return fileDescriptor_api_e9d594aefb977c8a, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -482,7 +482,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_5f7f23e8e4f12d4b, []int{3} + return fileDescriptor_api_e9d594aefb977c8a, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +525,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_5f7f23e8e4f12d4b, []int{4} + return fileDescriptor_api_e9d594aefb977c8a, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +559,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_5f7f23e8e4f12d4b, []int{5} + return fileDescriptor_api_e9d594aefb977c8a, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +648,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_5f7f23e8e4f12d4b, []int{6} + return fileDescriptor_api_e9d594aefb977c8a, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -683,7 +683,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_5f7f23e8e4f12d4b, []int{7} + return fileDescriptor_api_e9d594aefb977c8a, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +729,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_5f7f23e8e4f12d4b, []int{8} + return fileDescriptor_api_e9d594aefb977c8a, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +763,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_5f7f23e8e4f12d4b, []int{9} + return fileDescriptor_api_e9d594aefb977c8a, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -803,7 +803,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_5f7f23e8e4f12d4b, []int{10} + return fileDescriptor_api_e9d594aefb977c8a, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -840,7 +840,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_5f7f23e8e4f12d4b, []int{11} + return fileDescriptor_api_e9d594aefb977c8a, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -874,7 +874,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_5f7f23e8e4f12d4b, []int{12} + return fileDescriptor_api_e9d594aefb977c8a, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,7 +908,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_5f7f23e8e4f12d4b, []int{13} + return fileDescriptor_api_e9d594aefb977c8a, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -960,7 +960,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_5f7f23e8e4f12d4b, []int{14} + return fileDescriptor_api_e9d594aefb977c8a, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -997,7 +997,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_5f7f23e8e4f12d4b, []int{15} + return fileDescriptor_api_e9d594aefb977c8a, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1052,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_5f7f23e8e4f12d4b, []int{16} + return fileDescriptor_api_e9d594aefb977c8a, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1086,7 +1086,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_5f7f23e8e4f12d4b, []int{17} + return fileDescriptor_api_e9d594aefb977c8a, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1128,7 +1128,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_5f7f23e8e4f12d4b, []int{18} + return fileDescriptor_api_e9d594aefb977c8a, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1162,7 +1162,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_5f7f23e8e4f12d4b, []int{19} + return fileDescriptor_api_e9d594aefb977c8a, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1213,7 +1213,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_5f7f23e8e4f12d4b, []int{20} + return fileDescriptor_api_e9d594aefb977c8a, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1265,7 +1265,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_5f7f23e8e4f12d4b, []int{21} + return fileDescriptor_api_e9d594aefb977c8a, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1316,7 +1316,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_5f7f23e8e4f12d4b, []int{22} + return fileDescriptor_api_e9d594aefb977c8a, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1368,7 +1368,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_5f7f23e8e4f12d4b, []int{23} + return fileDescriptor_api_e9d594aefb977c8a, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1421,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_5f7f23e8e4f12d4b, []int{24} + return fileDescriptor_api_e9d594aefb977c8a, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1458,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_5f7f23e8e4f12d4b, []int{25} + return fileDescriptor_api_e9d594aefb977c8a, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1502,7 +1502,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_5f7f23e8e4f12d4b, []int{25, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1550,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_5f7f23e8e4f12d4b, []int{26} + return fileDescriptor_api_e9d594aefb977c8a, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1586,7 +1586,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_5f7f23e8e4f12d4b, []int{27} + return fileDescriptor_api_e9d594aefb977c8a, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1697,7 +1697,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_5f7f23e8e4f12d4b, []int{28} + return fileDescriptor_api_e9d594aefb977c8a, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1743,7 +1743,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_5f7f23e8e4f12d4b, []int{29} + return fileDescriptor_api_e9d594aefb977c8a, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1804,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_5f7f23e8e4f12d4b, []int{30} + return fileDescriptor_api_e9d594aefb977c8a, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1839,7 +1839,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_5f7f23e8e4f12d4b, []int{31} + return fileDescriptor_api_e9d594aefb977c8a, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1878,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_5f7f23e8e4f12d4b, []int{32} + return fileDescriptor_api_e9d594aefb977c8a, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1913,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_5f7f23e8e4f12d4b, []int{33} + return fileDescriptor_api_e9d594aefb977c8a, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1956,7 +1956,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_5f7f23e8e4f12d4b, []int{34} + return fileDescriptor_api_e9d594aefb977c8a, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1991,7 +1991,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_5f7f23e8e4f12d4b, []int{35} + return fileDescriptor_api_e9d594aefb977c8a, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2029,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_5f7f23e8e4f12d4b, []int{36} + return fileDescriptor_api_e9d594aefb977c8a, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,7 +2062,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_5f7f23e8e4f12d4b, []int{37} + return fileDescriptor_api_e9d594aefb977c8a, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2097,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_5f7f23e8e4f12d4b, []int{38} + return fileDescriptor_api_e9d594aefb977c8a, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2155,7 +2155,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_5f7f23e8e4f12d4b, []int{39} + return fileDescriptor_api_e9d594aefb977c8a, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2190,7 +2190,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_5f7f23e8e4f12d4b, []int{40} + return fileDescriptor_api_e9d594aefb977c8a, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2228,7 +2228,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_5f7f23e8e4f12d4b, []int{41} + return fileDescriptor_api_e9d594aefb977c8a, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2261,7 +2261,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_5f7f23e8e4f12d4b, []int{42} + return fileDescriptor_api_e9d594aefb977c8a, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2303,7 +2303,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_5f7f23e8e4f12d4b, []int{43} + return fileDescriptor_api_e9d594aefb977c8a, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,7 +2340,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_5f7f23e8e4f12d4b, []int{44} + return fileDescriptor_api_e9d594aefb977c8a, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2378,7 +2378,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_5f7f23e8e4f12d4b, []int{45} + return fileDescriptor_api_e9d594aefb977c8a, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2412,7 +2412,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_5f7f23e8e4f12d4b, []int{45, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2446,7 +2446,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_5f7f23e8e4f12d4b, []int{46} + return fileDescriptor_api_e9d594aefb977c8a, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2515,7 +2515,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_5f7f23e8e4f12d4b, []int{47} + return fileDescriptor_api_e9d594aefb977c8a, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2558,7 +2558,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_5f7f23e8e4f12d4b, []int{48} + return fileDescriptor_api_e9d594aefb977c8a, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2605,7 +2605,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_5f7f23e8e4f12d4b, []int{49} + return fileDescriptor_api_e9d594aefb977c8a, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2641,7 +2641,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_5f7f23e8e4f12d4b, []int{50} + return fileDescriptor_api_e9d594aefb977c8a, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2685,7 +2685,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_5f7f23e8e4f12d4b, []int{51} + return fileDescriptor_api_e9d594aefb977c8a, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2729,7 +2729,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_5f7f23e8e4f12d4b, []int{52} + return fileDescriptor_api_e9d594aefb977c8a, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2789,7 +2789,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_5f7f23e8e4f12d4b, []int{53} + return fileDescriptor_api_e9d594aefb977c8a, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2825,7 +2825,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_5f7f23e8e4f12d4b, []int{54} + return fileDescriptor_api_e9d594aefb977c8a, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2872,7 +2872,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_5f7f23e8e4f12d4b, []int{55} + return fileDescriptor_api_e9d594aefb977c8a, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2907,7 +2907,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_5f7f23e8e4f12d4b, []int{56} + return fileDescriptor_api_e9d594aefb977c8a, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2958,7 +2958,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_5f7f23e8e4f12d4b, []int{57} + return fileDescriptor_api_e9d594aefb977c8a, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2993,7 +2993,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_5f7f23e8e4f12d4b, []int{58} + return fileDescriptor_api_e9d594aefb977c8a, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3030,7 +3030,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_5f7f23e8e4f12d4b, []int{59} + return fileDescriptor_api_e9d594aefb977c8a, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3064,7 +3064,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_5f7f23e8e4f12d4b, []int{60} + return fileDescriptor_api_e9d594aefb977c8a, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3109,7 +3109,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_5f7f23e8e4f12d4b, []int{61} + return fileDescriptor_api_e9d594aefb977c8a, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3143,7 +3143,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_5f7f23e8e4f12d4b, []int{62} + return fileDescriptor_api_e9d594aefb977c8a, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3187,7 +3187,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_5f7f23e8e4f12d4b, []int{63} + return fileDescriptor_api_e9d594aefb977c8a, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3236,7 +3236,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_5f7f23e8e4f12d4b, []int{64} + return fileDescriptor_api_e9d594aefb977c8a, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3273,7 +3273,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_5f7f23e8e4f12d4b, []int{65} + return fileDescriptor_api_e9d594aefb977c8a, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3310,7 +3310,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_5f7f23e8e4f12d4b, []int{66} + return fileDescriptor_api_e9d594aefb977c8a, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3345,7 +3345,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_5f7f23e8e4f12d4b, []int{67} + return fileDescriptor_api_e9d594aefb977c8a, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3400,7 +3400,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_5f7f23e8e4f12d4b, []int{68} + return fileDescriptor_api_e9d594aefb977c8a, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3437,7 +3437,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_5f7f23e8e4f12d4b, []int{69} + return fileDescriptor_api_e9d594aefb977c8a, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3477,7 +3477,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_5f7f23e8e4f12d4b, []int{70} + return fileDescriptor_api_e9d594aefb977c8a, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3511,7 +3511,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_5f7f23e8e4f12d4b, []int{70, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3544,7 +3544,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_5f7f23e8e4f12d4b, []int{70, 1} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3586,7 +3586,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_5f7f23e8e4f12d4b, []int{70, 2} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3625,7 +3625,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_5f7f23e8e4f12d4b, []int{70, 3} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3661,7 +3661,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_5f7f23e8e4f12d4b, []int{70, 4} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3700,7 +3700,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_5f7f23e8e4f12d4b, []int{70, 5} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3742,7 +3742,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_5f7f23e8e4f12d4b, []int{70, 6} + return fileDescriptor_api_e9d594aefb977c8a, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3782,7 +3782,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_5f7f23e8e4f12d4b, []int{71} + return fileDescriptor_api_e9d594aefb977c8a, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3816,7 +3816,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_5f7f23e8e4f12d4b, []int{72} + return fileDescriptor_api_e9d594aefb977c8a, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3852,7 +3852,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_5f7f23e8e4f12d4b, []int{73} + return fileDescriptor_api_e9d594aefb977c8a, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3922,7 +3922,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_5f7f23e8e4f12d4b, []int{74} + return fileDescriptor_api_e9d594aefb977c8a, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3974,7 +3974,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_5f7f23e8e4f12d4b, []int{75} + return fileDescriptor_api_e9d594aefb977c8a, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4010,7 +4010,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_5f7f23e8e4f12d4b, []int{76} + return fileDescriptor_api_e9d594aefb977c8a, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4050,7 +4050,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_5f7f23e8e4f12d4b, []int{76, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4101,7 +4101,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_5f7f23e8e4f12d4b, []int{77} + return fileDescriptor_api_e9d594aefb977c8a, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4136,7 +4136,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_5f7f23e8e4f12d4b, []int{77, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4172,7 +4172,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_5f7f23e8e4f12d4b, []int{77, 1} + return fileDescriptor_api_e9d594aefb977c8a, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4207,7 +4207,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_5f7f23e8e4f12d4b, []int{78} + return fileDescriptor_api_e9d594aefb977c8a, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4245,7 +4245,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_5f7f23e8e4f12d4b, []int{79} + return fileDescriptor_api_e9d594aefb977c8a, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4282,7 +4282,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_5f7f23e8e4f12d4b, []int{80} + return fileDescriptor_api_e9d594aefb977c8a, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4315,7 +4315,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_5f7f23e8e4f12d4b, []int{80, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4360,7 +4360,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_5f7f23e8e4f12d4b, []int{81} + return fileDescriptor_api_e9d594aefb977c8a, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4398,7 +4398,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_5f7f23e8e4f12d4b, []int{82} + return fileDescriptor_api_e9d594aefb977c8a, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4451,7 +4451,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_5f7f23e8e4f12d4b, []int{83} + return fileDescriptor_api_e9d594aefb977c8a, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4485,7 +4485,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_5f7f23e8e4f12d4b, []int{84} + return fileDescriptor_api_e9d594aefb977c8a, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4529,7 +4529,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_5f7f23e8e4f12d4b, []int{85} + return fileDescriptor_api_e9d594aefb977c8a, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4563,7 +4563,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_5f7f23e8e4f12d4b, []int{86} + return fileDescriptor_api_e9d594aefb977c8a, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4602,7 +4602,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_5f7f23e8e4f12d4b, []int{87} + return fileDescriptor_api_e9d594aefb977c8a, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4636,7 +4636,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_5f7f23e8e4f12d4b, []int{88} + return fileDescriptor_api_e9d594aefb977c8a, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4685,7 +4685,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_5f7f23e8e4f12d4b, []int{89} + return fileDescriptor_api_e9d594aefb977c8a, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4734,7 +4734,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_5f7f23e8e4f12d4b, []int{90} + return fileDescriptor_api_e9d594aefb977c8a, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4769,7 +4769,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_5f7f23e8e4f12d4b, []int{91} + return fileDescriptor_api_e9d594aefb977c8a, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4811,7 +4811,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_5f7f23e8e4f12d4b, []int{92} + return fileDescriptor_api_e9d594aefb977c8a, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4854,7 +4854,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{93} + return fileDescriptor_api_e9d594aefb977c8a, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4888,7 +4888,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{94} + return fileDescriptor_api_e9d594aefb977c8a, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4972,7 +4972,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_5f7f23e8e4f12d4b, []int{95} + return fileDescriptor_api_e9d594aefb977c8a, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6453,7 +6453,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_5f7f23e8e4f12d4b, []int{96} + return fileDescriptor_api_e9d594aefb977c8a, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8018,7 +8018,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_5f7f23e8e4f12d4b, []int{97} + return fileDescriptor_api_e9d594aefb977c8a, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8056,7 +8056,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_5f7f23e8e4f12d4b, []int{98} + return fileDescriptor_api_e9d594aefb977c8a, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8092,7 +8092,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{99} + return fileDescriptor_api_e9d594aefb977c8a, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8129,7 +8129,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{100} + return fileDescriptor_api_e9d594aefb977c8a, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8208,7 +8208,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_5f7f23e8e4f12d4b, []int{100, 0} + return fileDescriptor_api_e9d594aefb977c8a, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8246,7 +8246,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_5f7f23e8e4f12d4b, []int{101} + return fileDescriptor_api_e9d594aefb977c8a, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8284,7 +8284,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_5f7f23e8e4f12d4b, []int{102} + return fileDescriptor_api_e9d594aefb977c8a, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8323,7 +8323,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_5f7f23e8e4f12d4b, []int{103} + return fileDescriptor_api_e9d594aefb977c8a, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8364,7 +8364,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_5f7f23e8e4f12d4b, []int{104} + return fileDescriptor_api_e9d594aefb977c8a, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8405,7 +8405,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_5f7f23e8e4f12d4b, []int{105} + return fileDescriptor_api_e9d594aefb977c8a, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8442,7 +8442,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_5f7f23e8e4f12d4b, []int{106} + return fileDescriptor_api_e9d594aefb977c8a, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8479,7 +8479,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_5f7f23e8e4f12d4b, []int{107} + return fileDescriptor_api_e9d594aefb977c8a, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8523,7 +8523,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{108} + return fileDescriptor_api_e9d594aefb977c8a, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8555,7 +8555,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{109} + return fileDescriptor_api_e9d594aefb977c8a, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8596,7 +8596,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_5f7f23e8e4f12d4b, []int{110} + return fileDescriptor_api_e9d594aefb977c8a, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8636,7 +8636,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_5f7f23e8e4f12d4b, []int{111} + return fileDescriptor_api_e9d594aefb977c8a, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8672,7 +8672,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_5f7f23e8e4f12d4b, []int{112} + return fileDescriptor_api_e9d594aefb977c8a, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8711,7 +8711,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_5f7f23e8e4f12d4b, []int{113} + return fileDescriptor_api_e9d594aefb977c8a, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8738,23 +8738,20 @@ var xxx_messageInfo_JoinNodeResponse proto.InternalMessageInfo // ContentionEvent is a message that will be attached to BatchResponses // indicating any conflicts with another transaction during replica evaluation. -// This message is currently not emitted and only exists for SQL Execution to -// have a protobuf to work with to build the higher-level infrastructure around -// contention observability while the work to emit these events is ongoing. type ContentionEvent struct { // Key is the key that this and the other transaction conflicted on. Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` - // Txn is the other transaction. - Txn Transaction `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` + // TxnMeta is the transaction conflicted + // with, i.e. the transaction holding a lock. + TxnMeta enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn_meta,json=txnMeta,proto3" json:"txn_meta"` // Duration spent contending against the other transaction. Duration time.Duration `protobuf:"bytes,3,opt,name=duration,proto3,stdduration" json:"duration"` } -func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } -func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } -func (*ContentionEvent) ProtoMessage() {} +func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } +func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_5f7f23e8e4f12d4b, []int{114} + return fileDescriptor_api_e9d594aefb977c8a, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -15942,8 +15939,8 @@ func (m *ContentionEvent) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x12 i++ - i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n281, err := m.Txn.MarshalTo(dAtA[i:]) + i = encodeVarintApi(dAtA, i, uint64(m.TxnMeta.Size())) + n281, err := m.TxnMeta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } @@ -19344,7 +19341,7 @@ func (m *ContentionEvent) Size() (n int) { if l > 0 { n += 1 + l + sovApi(uint64(l)) } - l = m.Txn.Size() + l = m.TxnMeta.Size() n += 1 + l + sovApi(uint64(l)) l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration) n += 1 + l + sovApi(uint64(l)) @@ -38985,7 +38982,7 @@ func (m *ContentionEvent) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Txn", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TxnMeta", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -39009,7 +39006,7 @@ func (m *ContentionEvent) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.Txn.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TxnMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -39169,525 +39166,525 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_5f7f23e8e4f12d4b) } - -var fileDescriptor_api_5f7f23e8e4f12d4b = []byte{ - // 8257 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, - 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0x38, 0xdc, 0xdd, 0xd1, 0x4c, - 0xcf, 0xff, 0x78, 0x57, 0xda, 0x99, 0xb9, 0xeb, 0x5d, 0xef, 0xac, 0xf7, 0x5a, 0xa2, 0x38, 0x43, - 0x4a, 0x23, 0x8d, 0xa6, 0x49, 0xcd, 0x64, 0xd7, 0xd7, 0xe9, 0xdb, 0xea, 0x2e, 0x51, 0x7d, 0x45, - 0x76, 0x73, 0xba, 0x9b, 0xfa, 0x59, 0x20, 0x40, 0x1c, 0x1b, 0xce, 0x05, 0x02, 0x18, 0xf7, 0xc1, - 0x41, 0xae, 0xe1, 0x24, 0xbe, 0x8e, 0x93, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x12, 0x04, 0x89, - 0xfd, 0x62, 0x20, 0x17, 0x81, 0x93, 0x5c, 0x3f, 0xc5, 0x08, 0x10, 0xc5, 0xd6, 0xcd, 0x4b, 0x90, - 0xc0, 0x08, 0x82, 0x00, 0x06, 0xf6, 0x21, 0x08, 0xea, 0xa7, 0xff, 0xc8, 0xe6, 0x8f, 0x66, 0xfb, - 0x26, 0x0b, 0xf8, 0x45, 0x62, 0x9f, 0xaa, 0x73, 0xba, 0xea, 0x54, 0xd5, 0xa9, 0xf3, 0x55, 0x9f, - 0xaa, 0x82, 0x79, 0xcb, 0x54, 0xd4, 0xfd, 0xee, 0xee, 0xb2, 0xd2, 0xd5, 0x97, 0xba, 0x96, 0xe9, - 0x98, 0x68, 0x5e, 0x35, 0xd5, 0x03, 0x4a, 0x5e, 0xe2, 0x89, 0xe5, 0xfb, 0x07, 0x87, 0xcb, 0x07, - 0x87, 0x36, 0xb6, 0x0e, 0xb1, 0xb5, 0xac, 0x9a, 0x86, 0xda, 0xb3, 0x2c, 0x6c, 0xa8, 0x27, 0xcb, - 0x6d, 0x53, 0x3d, 0xa0, 0x7f, 0x74, 0xa3, 0xc5, 0xd8, 0xcb, 0xc8, 0x95, 0xa8, 0x29, 0x8e, 0xc2, - 0x69, 0x17, 0x5c, 0x1a, 0xb6, 0x2c, 0xd3, 0xb2, 0x39, 0xf5, 0x92, 0x4b, 0xed, 0x60, 0x47, 0x09, - 0xe4, 0x7e, 0xcb, 0x76, 0x4c, 0x4b, 0x69, 0xe1, 0x65, 0x6c, 0xb4, 0x74, 0x03, 0x93, 0x0c, 0x87, - 0xaa, 0xca, 0x13, 0xdf, 0x8e, 0x4c, 0x7c, 0xc4, 0x53, 0x4b, 0x3d, 0x47, 0x6f, 0x2f, 0xef, 0xb7, - 0xd5, 0x65, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x9e, 0x72, 0x9f, 0xa6, 0x38, 0x96, 0xa2, - 0xea, 0x46, 0xcb, 0xfd, 0xdf, 0xdd, 0x5d, 0xb6, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xab, - 0x18, 0x6e, 0x71, 0x5b, 0x66, 0xcb, 0xa4, 0x3f, 0x97, 0xc9, 0x2f, 0x4e, 0xbd, 0xda, 0x32, 0xcd, - 0x56, 0x1b, 0x2f, 0xd3, 0xa7, 0xdd, 0xde, 0xde, 0xb2, 0xd6, 0xb3, 0x14, 0x47, 0x37, 0x39, 0x97, - 0xf8, 0xcf, 0x04, 0xc8, 0x49, 0xf8, 0x75, 0x0f, 0xdb, 0x4e, 0x0d, 0x2b, 0x1a, 0xb6, 0xd0, 0x15, - 0x48, 0x1e, 0xe0, 0x93, 0x52, 0xf2, 0x9a, 0x70, 0x77, 0x6e, 0x75, 0xf6, 0xcb, 0xd3, 0xc5, 0xe4, - 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x5d, 0x83, 0x59, 0x6c, 0x68, 0x32, 0x49, 0x4e, 0x85, 0x93, 0x67, - 0xb0, 0xa1, 0x6d, 0xe0, 0x13, 0xf4, 0x2d, 0x48, 0xdb, 0x44, 0x9a, 0xa1, 0xe2, 0xd2, 0xf4, 0x35, - 0xe1, 0xee, 0xf4, 0xea, 0xcf, 0x7d, 0x79, 0xba, 0xf8, 0x49, 0x4b, 0x77, 0xf6, 0x7b, 0xbb, 0x4b, - 0xaa, 0xd9, 0x59, 0xf6, 0xda, 0x49, 0xdb, 0xf5, 0x7f, 0x2f, 0x77, 0x0f, 0x5a, 0xcb, 0xfd, 0x3a, - 0x5a, 0x6a, 0x1e, 0x1b, 0x0d, 0xfc, 0x5a, 0xf2, 0x24, 0xae, 0xa7, 0xd2, 0x42, 0x31, 0xb1, 0x9e, - 0x4a, 0x27, 0x8a, 0x49, 0xf1, 0xb7, 0x93, 0x90, 0x97, 0xb0, 0xdd, 0x35, 0x0d, 0x1b, 0xf3, 0x92, - 0xbf, 0x0f, 0x49, 0xe7, 0xd8, 0xa0, 0x25, 0xcf, 0x3e, 0xbc, 0xba, 0x34, 0xd0, 0x23, 0x96, 0x9a, - 0x96, 0x62, 0xd8, 0x8a, 0x4a, 0xaa, 0x2f, 0x91, 0xac, 0xe8, 0x23, 0xc8, 0x5a, 0xd8, 0xee, 0x75, - 0x30, 0x55, 0x24, 0xad, 0x54, 0xf6, 0xe1, 0xe5, 0x08, 0xce, 0x46, 0x57, 0x31, 0x24, 0x60, 0x79, - 0xc9, 0x6f, 0x74, 0x05, 0xd2, 0x46, 0xaf, 0x43, 0x54, 0x61, 0xd3, 0x8a, 0x26, 0xa5, 0x59, 0xa3, - 0xd7, 0xd9, 0xc0, 0x27, 0x36, 0xfa, 0x0b, 0x70, 0x49, 0xc3, 0x5d, 0x0b, 0xab, 0x8a, 0x83, 0x35, - 0xd9, 0x52, 0x8c, 0x16, 0x96, 0x75, 0x63, 0xcf, 0xb4, 0x4b, 0x33, 0xd7, 0x92, 0x77, 0xb3, 0x0f, - 0xdf, 0x8e, 0x90, 0x2f, 0x91, 0x5c, 0x75, 0x63, 0xcf, 0x5c, 0x4d, 0xfd, 0xf0, 0x74, 0x71, 0x4a, - 0xba, 0xe0, 0x4b, 0xf0, 0x92, 0x6c, 0xd4, 0x80, 0x1c, 0x2f, 0xae, 0x85, 0x15, 0xdb, 0x34, 0x4a, - 0xb3, 0xd7, 0x84, 0xbb, 0xf9, 0x87, 0x4b, 0x51, 0x02, 0x43, 0xaa, 0x21, 0x8f, 0xbd, 0x0e, 0x96, - 0x28, 0x97, 0x34, 0x67, 0x05, 0x9e, 0xd0, 0x5b, 0x90, 0x21, 0x35, 0xd9, 0x3d, 0x71, 0xb0, 0x5d, - 0x4a, 0xd3, 0xaa, 0x90, 0xaa, 0xad, 0x92, 0x67, 0xf1, 0x53, 0x98, 0x0b, 0xb2, 0x22, 0x04, 0x79, - 0xa9, 0xda, 0xd8, 0xd9, 0xac, 0xca, 0x3b, 0x5b, 0x1b, 0x5b, 0xcf, 0x5f, 0x6d, 0x15, 0xa7, 0xd0, - 0x05, 0x28, 0x72, 0xda, 0x46, 0xf5, 0x33, 0xf9, 0x59, 0x7d, 0xb3, 0xde, 0x2c, 0x0a, 0xe5, 0xd4, - 0x77, 0x7f, 0xfb, 0xea, 0x94, 0xb8, 0x0d, 0xf0, 0x14, 0x3b, 0xbc, 0x83, 0xa1, 0x55, 0x98, 0xd9, - 0xa7, 0xe5, 0x29, 0x09, 0x54, 0xd3, 0xd7, 0x22, 0x0b, 0x1e, 0xe8, 0x8c, 0xab, 0x69, 0xa2, 0x8d, - 0x1f, 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0x7f, 0x4f, 0x80, 0x2c, 0x15, 0xc9, 0xea, 0x87, 0x2a, - 0x7d, 0x32, 0xaf, 0x8f, 0x55, 0xc6, 0xa0, 0x50, 0xb4, 0x04, 0xd3, 0x87, 0x4a, 0xbb, 0x87, 0x4b, - 0x09, 0x2a, 0xa3, 0x14, 0x21, 0xe3, 0x25, 0x49, 0x97, 0x58, 0x36, 0xf4, 0x18, 0xe6, 0x74, 0xc3, - 0xc1, 0x86, 0x23, 0x33, 0xb6, 0xe4, 0x18, 0xb6, 0x2c, 0xcb, 0x4d, 0x1f, 0xc4, 0x7f, 0x2a, 0x00, - 0x6c, 0xf7, 0xe2, 0x54, 0x0a, 0xfa, 0xc6, 0x84, 0xe5, 0xe7, 0xbd, 0x8b, 0xd7, 0xe2, 0x12, 0xcc, - 0xe8, 0x46, 0x5b, 0x37, 0x58, 0xf9, 0xd3, 0x12, 0x7f, 0x42, 0x17, 0x60, 0x7a, 0xb7, 0xad, 0x1b, - 0x1a, 0x1d, 0x0f, 0x69, 0x89, 0x3d, 0x88, 0x12, 0x64, 0x69, 0xa9, 0x63, 0xd4, 0xbb, 0x78, 0x9a, - 0x80, 0x8b, 0x15, 0xd3, 0xd0, 0x74, 0x32, 0x24, 0x95, 0xf6, 0xd7, 0x42, 0x2b, 0xeb, 0x10, 0x18, - 0x7c, 0x32, 0x3e, 0xee, 0x4e, 0xd8, 0xc6, 0xc8, 0xe7, 0xaa, 0x1e, 0x77, 0x29, 0x2d, 0x5a, 0x93, - 0xe8, 0x1b, 0x70, 0x59, 0x69, 0xb7, 0xcd, 0x23, 0x59, 0xdf, 0x93, 0x35, 0x13, 0xdb, 0xb2, 0x61, - 0x3a, 0x32, 0x3e, 0xd6, 0x6d, 0x87, 0x9a, 0x92, 0xb4, 0xb4, 0x40, 0x93, 0xeb, 0x7b, 0x6b, 0x26, - 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x92, 0xc8, 0x38, 0x25, 0x85, 0x61, 0xe3, 0x74, 0x86, 0x98, 0x5f, - 0x29, 0x8d, 0x8f, 0xbb, 0x74, 0x9c, 0x06, 0x9a, 0x72, 0x36, 0xd8, 0x94, 0xe2, 0x2f, 0xc0, 0xa5, - 0x7e, 0xfd, 0xc6, 0xd9, 0x7e, 0x7f, 0x20, 0x40, 0xbe, 0x6e, 0xe8, 0xce, 0xd7, 0xa2, 0xe1, 0x3c, - 0x65, 0x27, 0x83, 0xca, 0xbe, 0x0f, 0xc5, 0x3d, 0x45, 0x6f, 0x3f, 0x37, 0x9a, 0x66, 0x67, 0xd7, - 0x76, 0x4c, 0x03, 0xdb, 0xbc, 0x35, 0x06, 0xe8, 0xe2, 0x4b, 0x28, 0x78, 0xb5, 0x89, 0x53, 0x4d, - 0x0e, 0x14, 0xeb, 0x86, 0x6a, 0xe1, 0x0e, 0x36, 0x62, 0xd5, 0xd3, 0xdb, 0x90, 0xd1, 0x5d, 0xb9, - 0x54, 0x57, 0x49, 0xc9, 0x27, 0x88, 0x3d, 0x98, 0x0f, 0xbc, 0x35, 0x4e, 0x73, 0x49, 0xa6, 0x0c, - 0x7c, 0x24, 0xfb, 0x6d, 0x44, 0xa6, 0x0c, 0x7c, 0xc4, 0xcc, 0x5b, 0x03, 0x72, 0x6b, 0xb8, 0x8d, - 0x1d, 0x1c, 0xa7, 0xd5, 0xdf, 0x81, 0xbc, 0x2b, 0x34, 0xce, 0x86, 0xf9, 0x1b, 0x02, 0x20, 0x2e, - 0x97, 0xcc, 0xb2, 0x71, 0xb6, 0xcd, 0x22, 0x71, 0x2d, 0x9c, 0x9e, 0x65, 0x30, 0x1f, 0x81, 0xf5, - 0x49, 0x60, 0x24, 0xea, 0x26, 0xf8, 0x43, 0x36, 0x15, 0x1c, 0xb2, 0xdc, 0xbd, 0x39, 0x82, 0x85, - 0x50, 0xc1, 0xe2, 0x6d, 0xbe, 0x14, 0x2d, 0x53, 0xe2, 0x5a, 0x32, 0xe8, 0xc3, 0x51, 0xa2, 0xf8, - 0x7d, 0x01, 0xe6, 0x2b, 0x6d, 0xac, 0x58, 0xb1, 0x6b, 0xe4, 0x9b, 0x90, 0xd6, 0xb0, 0xa2, 0xd1, - 0x2a, 0xb3, 0x81, 0xfd, 0x4e, 0x40, 0x0a, 0xf1, 0x74, 0x97, 0xf6, 0xdb, 0xea, 0x52, 0xd3, 0xf5, - 0x81, 0xf9, 0xe8, 0xf6, 0x98, 0xc4, 0xcf, 0x00, 0x05, 0x4b, 0x16, 0x67, 0x47, 0xf8, 0xdf, 0x02, - 0x20, 0x09, 0x1f, 0x62, 0xcb, 0x89, 0xbd, 0xda, 0x6b, 0x90, 0x75, 0x14, 0xab, 0x85, 0x1d, 0x99, - 0x78, 0xf7, 0xe7, 0xa9, 0x39, 0x30, 0x3e, 0x42, 0x46, 0x4d, 0xb8, 0x83, 0x0d, 0x65, 0xb7, 0x8d, - 0xa9, 0x14, 0x79, 0xd7, 0xec, 0x19, 0x9a, 0xac, 0x3b, 0xd8, 0x52, 0x1c, 0xd3, 0x92, 0xcd, 0xae, - 0xa3, 0x77, 0xf4, 0x2f, 0xa8, 0x63, 0xcf, 0xbb, 0xda, 0x0d, 0x96, 0x9d, 0x30, 0xaf, 0x92, 0xcc, - 0x75, 0x9e, 0xf7, 0x79, 0x20, 0xab, 0xf8, 0x39, 0x2c, 0x84, 0x6a, 0x1d, 0xa7, 0x4a, 0xff, 0xa7, - 0x00, 0xd9, 0x86, 0xaa, 0x18, 0x71, 0xea, 0xf2, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, - 0xd5, 0x51, 0x1c, 0x3a, 0x70, 0xf2, 0x21, 0x5d, 0x7a, 0xfe, 0xba, 0xaa, 0x18, 0x4f, 0x68, 0x26, - 0x09, 0x6c, 0xef, 0x37, 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0xec, 0x47, 0x67, 0xdb, 0xfc, - 0xc3, 0xf7, 0x03, 0xfc, 0x07, 0x87, 0x4b, 0x2e, 0x64, 0x5c, 0x0a, 0x40, 0xc6, 0x25, 0xc2, 0xb1, - 0xd4, 0x70, 0x2c, 0x6c, 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x0d, 0xd7, - 0xf5, 0x54, 0x3a, 0x59, 0x4c, 0x89, 0x7f, 0x26, 0xc0, 0x1c, 0xab, 0x72, 0x9c, 0xc3, 0xf5, 0x03, - 0x48, 0x59, 0xe6, 0x11, 0x1b, 0xae, 0xd9, 0x87, 0x6f, 0x45, 0x88, 0xd8, 0xc0, 0x27, 0xc1, 0xf9, - 0x90, 0x66, 0x47, 0xab, 0xc0, 0xbd, 0x4e, 0x99, 0x72, 0x27, 0x27, 0xe5, 0x06, 0xc6, 0x25, 0x11, - 0x19, 0x77, 0xa0, 0xb0, 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x42, 0x92, 0xb9, 0x33, 0x79, 0x77, - 0x4e, 0xca, 0x53, 0xb2, 0x5b, 0x74, 0x9b, 0xd4, 0x9c, 0x8d, 0x1f, 0x1b, 0xff, 0x39, 0x6b, 0xf3, - 0xff, 0x23, 0xf0, 0x31, 0xe4, 0xd6, 0xfc, 0xcf, 0x5b, 0xd3, 0xff, 0x7a, 0x02, 0x2e, 0x57, 0xf6, - 0xb1, 0x7a, 0x50, 0x31, 0x0d, 0x5b, 0xb7, 0x1d, 0xa2, 0xbb, 0x38, 0xdb, 0xff, 0x2d, 0xc8, 0x1c, - 0xe9, 0xce, 0xbe, 0xac, 0xe9, 0x7b, 0x7b, 0xd4, 0x7a, 0xa6, 0xa5, 0x34, 0x21, 0xac, 0xe9, 0x7b, - 0x7b, 0xe8, 0x11, 0xa4, 0x3a, 0xa6, 0xc6, 0x9c, 0xf3, 0xfc, 0xc3, 0xc5, 0x08, 0xf1, 0xb4, 0x68, - 0x76, 0xaf, 0xb3, 0x69, 0x6a, 0x58, 0xa2, 0x99, 0xd1, 0x55, 0x00, 0x95, 0x50, 0xbb, 0xa6, 0x6e, - 0x38, 0x7c, 0xf6, 0x0d, 0x50, 0x50, 0x0d, 0x32, 0x0e, 0xb6, 0x3a, 0xba, 0xa1, 0x38, 0xb8, 0x34, - 0x4d, 0x95, 0x77, 0x33, 0xb2, 0xe0, 0xdd, 0xb6, 0xae, 0x2a, 0x6b, 0xd8, 0x56, 0x2d, 0xbd, 0xeb, - 0x98, 0x16, 0xd7, 0xa2, 0xcf, 0x2c, 0xfe, 0x6a, 0x0a, 0x4a, 0x83, 0xba, 0x89, 0xb3, 0x87, 0x6c, - 0xc3, 0x0c, 0x41, 0xf3, 0x6d, 0x87, 0xf7, 0x91, 0x87, 0xc3, 0x54, 0x10, 0x51, 0x02, 0xba, 0x2a, - 0xd0, 0x76, 0x78, 0xb1, 0xb9, 0x9c, 0xf2, 0xbf, 0x12, 0x60, 0x86, 0x25, 0xa0, 0x07, 0x90, 0xe6, - 0xcb, 0x17, 0x1a, 0x2d, 0x63, 0x72, 0xf5, 0xd2, 0xd9, 0xe9, 0xe2, 0x2c, 0x5b, 0x91, 0x58, 0xfb, - 0xd2, 0xff, 0x29, 0xcd, 0xd2, 0x7c, 0x75, 0x8d, 0xb4, 0x96, 0xed, 0x28, 0x96, 0x43, 0x17, 0x89, - 0x12, 0x0c, 0xa5, 0x50, 0xc2, 0x06, 0x3e, 0x41, 0xeb, 0x30, 0x63, 0x3b, 0x8a, 0xd3, 0xb3, 0x79, - 0x7b, 0x9d, 0xab, 0xb0, 0x0d, 0xca, 0x29, 0x71, 0x09, 0xc4, 0x7d, 0xd2, 0xb0, 0xa3, 0xe8, 0x6d, - 0xda, 0x80, 0x19, 0x89, 0x3f, 0x89, 0xbf, 0x21, 0xc0, 0x0c, 0xcb, 0x8a, 0x2e, 0xc3, 0x82, 0xb4, - 0xb2, 0xf5, 0xb4, 0x2a, 0xd7, 0xb7, 0xd6, 0xaa, 0xcd, 0xaa, 0xb4, 0x59, 0xdf, 0x5a, 0x69, 0x56, - 0x8b, 0x53, 0xe8, 0x12, 0x20, 0x37, 0xa1, 0xf2, 0x7c, 0xab, 0x51, 0x6f, 0x34, 0xab, 0x5b, 0xcd, - 0xa2, 0x40, 0x57, 0x32, 0x28, 0x3d, 0x40, 0x4d, 0xa0, 0x9b, 0x70, 0xad, 0x9f, 0x2a, 0x37, 0x9a, - 0x2b, 0xcd, 0x86, 0x5c, 0x6d, 0x34, 0xeb, 0x9b, 0x2b, 0xcd, 0xea, 0x5a, 0x31, 0x39, 0x22, 0x17, - 0x79, 0x89, 0x24, 0x55, 0x2b, 0xcd, 0x62, 0x4a, 0x74, 0xe0, 0xa2, 0x84, 0x55, 0xb3, 0xd3, 0xed, - 0x39, 0x98, 0x94, 0xd2, 0x8e, 0x73, 0xa4, 0x5c, 0x86, 0x59, 0xcd, 0x3a, 0x91, 0xad, 0x9e, 0xc1, - 0xc7, 0xc9, 0x8c, 0x66, 0x9d, 0x48, 0x3d, 0x43, 0xfc, 0xc7, 0x02, 0x5c, 0xea, 0x7f, 0x6d, 0x9c, - 0x9d, 0xf0, 0x05, 0x64, 0x15, 0x4d, 0xc3, 0x9a, 0xac, 0xe1, 0xb6, 0xa3, 0x70, 0x17, 0xe7, 0x7e, - 0x40, 0x12, 0x5f, 0xda, 0x5b, 0xf2, 0x96, 0xf6, 0x36, 0x5f, 0x56, 0x2a, 0xb4, 0x20, 0x6b, 0x84, - 0xc3, 0x35, 0x3f, 0x54, 0x08, 0xa5, 0x88, 0xff, 0x23, 0x05, 0xb9, 0xaa, 0xa1, 0x35, 0x8f, 0x63, - 0x9d, 0x4b, 0x2e, 0xc1, 0x8c, 0x6a, 0x76, 0x3a, 0xba, 0xe3, 0x2a, 0x88, 0x3d, 0xa1, 0x9f, 0x09, - 0xb8, 0xa6, 0xc9, 0x09, 0x1c, 0x34, 0xdf, 0x29, 0x45, 0xdf, 0x86, 0xcb, 0xc4, 0x6a, 0x5a, 0x86, - 0xd2, 0x96, 0x99, 0x34, 0xd9, 0xb1, 0xf4, 0x56, 0x0b, 0x5b, 0x7c, 0x39, 0xf1, 0x6e, 0x44, 0x39, - 0xeb, 0x9c, 0xa3, 0x42, 0x19, 0x9a, 0x2c, 0xbf, 0x74, 0x51, 0x8f, 0x22, 0xa3, 0x4f, 0x00, 0xc8, - 0x54, 0x44, 0x97, 0x28, 0x6d, 0x6e, 0x8f, 0x86, 0xad, 0x51, 0xba, 0x26, 0x88, 0x30, 0x90, 0x67, - 0x1b, 0x2d, 0x13, 0x1c, 0xf2, 0xba, 0xa7, 0x5b, 0x58, 0x7e, 0xd0, 0x55, 0xe9, 0xc2, 0x41, 0x7a, - 0x35, 0x7f, 0x76, 0xba, 0x08, 0x12, 0x23, 0x3f, 0xd8, 0xae, 0x10, 0x5c, 0xc2, 0x7e, 0x77, 0x55, - 0xf4, 0x0a, 0xee, 0x05, 0xd6, 0x3f, 0xc8, 0xcc, 0xcb, 0xab, 0xa5, 0x38, 0xf2, 0xbe, 0xde, 0xda, - 0xc7, 0x96, 0xec, 0x2d, 0x53, 0xd3, 0xf5, 0xc2, 0xb4, 0x74, 0xd3, 0x67, 0xa8, 0x28, 0x06, 0x2b, - 0xfd, 0x8a, 0x53, 0xa3, 0x99, 0x3d, 0x9d, 0x11, 0xe5, 0x77, 0x4d, 0xdd, 0x36, 0x8d, 0x52, 0x86, - 0x29, 0x9f, 0x3d, 0xa1, 0x7b, 0x50, 0x74, 0x8e, 0x0d, 0x79, 0x1f, 0x2b, 0x96, 0xb3, 0x8b, 0x15, - 0x87, 0xcc, 0xd2, 0x40, 0x73, 0x14, 0x9c, 0x63, 0xa3, 0x16, 0x20, 0xa3, 0x17, 0x50, 0xd4, 0x0d, - 0x79, 0xaf, 0xad, 0xb7, 0xf6, 0x1d, 0xf9, 0xc8, 0xd2, 0x1d, 0x6c, 0x97, 0xe6, 0xa9, 0x42, 0xa2, - 0xfa, 0x6d, 0x83, 0xaf, 0x1b, 0x6b, 0xaf, 0x48, 0x4e, 0xae, 0x9a, 0xbc, 0x6e, 0x3c, 0xa1, 0xfc, - 0x94, 0x68, 0xaf, 0xa7, 0xd2, 0xb3, 0xc5, 0xb4, 0xf8, 0x9f, 0x05, 0xc8, 0xbb, 0xdd, 0x2d, 0xce, - 0x91, 0x71, 0x17, 0x8a, 0xa6, 0x81, 0xe5, 0xee, 0xbe, 0x62, 0x63, 0xae, 0x47, 0x3e, 0xe1, 0xe4, - 0x4d, 0x03, 0x6f, 0x13, 0x32, 0x53, 0x17, 0xda, 0x86, 0x79, 0xdb, 0x51, 0x5a, 0xba, 0xd1, 0x0a, - 0xa8, 0x77, 0x7a, 0x72, 0xb0, 0x50, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa1, 0x00, 0xf3, - 0x2b, 0x5a, 0x47, 0x37, 0x1a, 0xdd, 0xb6, 0x1e, 0xeb, 0x1a, 0xc4, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, - 0x6f, 0xf0, 0x7d, 0x44, 0x99, 0xa6, 0x29, 0xc4, 0xf2, 0x3f, 0x83, 0x02, 0x3e, 0xee, 0xea, 0xec, - 0xd3, 0x03, 0x03, 0x42, 0xa9, 0xc9, 0xeb, 0x96, 0xf7, 0x79, 0x49, 0x12, 0xaf, 0xd3, 0x67, 0x80, - 0x82, 0x55, 0x8a, 0x13, 0xbb, 0x7c, 0x06, 0x0b, 0x54, 0xf4, 0x8e, 0x61, 0xc7, 0xac, 0x2f, 0xf1, - 0xe7, 0xe1, 0x42, 0x58, 0x74, 0x9c, 0xe5, 0x7e, 0xc5, 0x5b, 0x79, 0x13, 0x5b, 0xb1, 0x82, 0x58, - 0x4f, 0xd7, 0x5c, 0x70, 0x9c, 0x65, 0xfe, 0x65, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0xeb, 0xcc, 0x1e, - 0xb6, 0x9e, 0x61, 0xc5, 0x8e, 0x15, 0x81, 0xdf, 0x80, 0x19, 0x86, 0xa4, 0x69, 0xff, 0x9c, 0x5e, - 0xcd, 0x12, 0xcf, 0xa5, 0xe1, 0x98, 0x16, 0xf1, 0x5c, 0x78, 0x92, 0xa8, 0x40, 0x39, 0xaa, 0x14, - 0x71, 0xd6, 0xf4, 0x6f, 0x0b, 0x30, 0xcf, 0x9d, 0x46, 0xd2, 0x95, 0x2b, 0xfb, 0xc4, 0x67, 0x42, - 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, - 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, 0x4d, 0x14, 0x15, 0xa8, 0xe4, 0x48, 0x8f, 0x95, 0x8e, - 0x23, 0x9a, 0xd7, 0x75, 0xfd, 0xb8, 0x0e, 0xfe, 0x49, 0x92, 0x2b, 0x81, 0xbd, 0x83, 0x67, 0x8f, - 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0x71, 0x2c, 0x58, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, - 0xa9, 0xe8, 0x33, 0x08, 0xac, 0xc1, 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, - 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, - 0xfb, 0x8a, 0x37, 0x80, 0x07, 0x66, 0xf1, 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x99, 0xbd, 0x5c, 0x57, - 0x81, 0x16, 0xdb, 0x1e, 0x0f, 0x2f, 0xfc, 0x9e, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, 0x81, 0x89, 0x10, - 0x7f, 0x20, 0xc0, 0x5b, 0x91, 0xad, 0x16, 0xe7, 0x44, 0xf6, 0x09, 0xa4, 0x68, 0xe5, 0x13, 0xe7, - 0xac, 0x3c, 0xe5, 0x12, 0xbf, 0x9b, 0xe0, 0x63, 0x5c, 0xc2, 0x6d, 0x93, 0x28, 0x36, 0xf6, 0x55, - 0xb6, 0xe7, 0x90, 0x3b, 0x34, 0x1d, 0xe2, 0x9b, 0xf0, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, 0xe7, 0xa8, - 0x00, 0xb7, 0xc5, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xfb, 0x52, 0xc1, 0x30, - 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0xe9, 0xd3, 0x44, 0x9c, 0x76, 0xe6, 0x7b, 0x02, 0x2c, 0x78, - 0x6e, 0x53, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0xac, 0x62, 0x92, 0xfc, 0x64, - 0xd6, 0x0b, 0x97, 0x28, 0xce, 0xfa, 0xfe, 0xeb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0xac, 0xe5, 0x27, - 0x7c, 0x85, 0x9c, 0xb5, 0x77, 0x54, 0x6f, 0xf7, 0xde, 0xb7, 0xf4, 0xb4, 0xb2, 0x81, 0x4f, 0xdc, - 0xde, 0x4e, 0xb8, 0xd0, 0x0a, 0x64, 0x9c, 0x7d, 0x0b, 0xdb, 0xfb, 0x66, 0x5b, 0x3b, 0x8f, 0x9b, - 0xe3, 0x73, 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0x34, 0x86, 0x10, 0x11, 0x8d, 0x41, 0x5e, 0xe3, - 0x79, 0x8a, 0x89, 0xf3, 0xbc, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, 0xa4, 0x3a, - 0x71, 0x36, 0xc9, 0xaf, 0x24, 0x21, 0xbf, 0xdd, 0xb3, 0xf7, 0x63, 0xee, 0x7d, 0x15, 0x80, 0x6e, - 0xcf, 0xa6, 0x10, 0xe4, 0xd8, 0xe0, 0x75, 0x1e, 0x13, 0xe8, 0xe1, 0x56, 0x9a, 0xf1, 0x35, 0x8f, - 0x0d, 0x54, 0xe3, 0x42, 0xb0, 0xec, 0x47, 0x8b, 0xdc, 0x18, 0x05, 0x56, 0x9b, 0xc7, 0xc6, 0x26, - 0xf6, 0x50, 0x2a, 0x93, 0x84, 0x89, 0xa4, 0x4f, 0x60, 0x96, 0x3c, 0xc8, 0x8e, 0x79, 0x9e, 0x66, - 0x9e, 0x21, 0x3c, 0x4d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xb3, 0xdf, 0x0c, 0x9d, 0xfd, 0xa2, - 0xea, 0xc2, 0xd5, 0x48, 0xe7, 0xbd, 0x34, 0x65, 0x25, 0x73, 0xdd, 0x05, 0x98, 0xde, 0x33, 0x2d, - 0xd5, 0xfd, 0xde, 0xcb, 0x1e, 0x58, 0x7b, 0xae, 0xa7, 0xd2, 0xe9, 0x62, 0x66, 0x3d, 0x95, 0xce, - 0x14, 0x41, 0xfc, 0x0d, 0x01, 0x0a, 0x5e, 0x43, 0xc4, 0x39, 0x21, 0x54, 0x42, 0x5a, 0x3c, 0x7f, - 0x53, 0x10, 0x05, 0x8a, 0xff, 0x86, 0x7a, 0x44, 0xaa, 0x79, 0x48, 0x5b, 0x26, 0xce, 0x9e, 0xf2, - 0x98, 0xc5, 0x02, 0x25, 0xce, 0xdb, 0xba, 0x34, 0x2c, 0xe8, 0x01, 0x5c, 0xd0, 0x3b, 0xc4, 0x9e, - 0xeb, 0x4e, 0xfb, 0x84, 0xc3, 0x36, 0x07, 0xbb, 0x1f, 0x96, 0x17, 0xfc, 0xb4, 0x8a, 0x9b, 0x24, - 0xfe, 0x5d, 0xba, 0x00, 0xee, 0xd7, 0x24, 0x4e, 0x55, 0xd7, 0x21, 0x67, 0x31, 0xd1, 0xc4, 0xad, - 0x39, 0xa7, 0xb6, 0xe7, 0x3c, 0x56, 0xa2, 0xf0, 0xdf, 0x4a, 0x40, 0xe1, 0x45, 0x0f, 0x5b, 0x27, - 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0x91, 0xa2, 0x3b, 0xf2, 0x9e, 0x69, 0xc9, 0xbd, 0xae, 0xa6, - 0x38, 0x6e, 0x40, 0x4a, 0x8e, 0x90, 0x9f, 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xc0, 0x30, - 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf8, 0xaa, 0xf4, 0xea, 0x87, 0xff, 0xe9, 0x74, - 0xf1, 0xd1, 0x44, 0x61, 0x66, 0x34, 0xa4, 0xae, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, 0xfa, 0x9a, - 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x7e, 0x02, 0x8a, 0xbe, 0x8e, - 0xe2, 0x6c, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x33, 0x02, 0x67, 0x24, 0x66, - 0xe7, 0x73, 0x98, 0x0b, 0x69, 0x20, 0xf9, 0xd5, 0x34, 0x90, 0x3d, 0xf2, 0x2b, 0x8f, 0xee, 0xc3, - 0xbc, 0x73, 0x6c, 0xc8, 0x2c, 0xc0, 0x90, 0x05, 0xa5, 0xb8, 0xf1, 0x12, 0x05, 0x87, 0xe8, 0x83, - 0xd0, 0x69, 0x40, 0x8a, 0x2d, 0xfe, 0x9e, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, 0x7c, 0x5d, - 0xfa, 0xd3, 0x5d, 0x28, 0xd2, 0x90, 0x4d, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, 0x8b, - 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x4b, 0xb0, 0x10, 0xaa, 0x40, 0x9c, - 0x8d, 0x7d, 0x1d, 0xe6, 0xf6, 0xd8, 0x57, 0x5a, 0x2a, 0x9c, 0xaf, 0x38, 0x66, 0x29, 0x8d, 0xbd, - 0x4f, 0xfc, 0xef, 0x09, 0xb8, 0x20, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, 0x58, 0x03, 0xfe, - 0x39, 0x47, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x9b, 0xe6, 0xc2, 0xcb, 0xf6, 0x37, 0x47, 0xf7, - 0xd8, 0xc1, 0x85, 0x7a, 0xbe, 0xec, 0x97, 0x0a, 0x2d, 0xfb, 0x99, 0x50, 0xd0, 0x5b, 0x86, 0x49, - 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x0b, 0x86, 0x96, 0x46, 0x15, 0xb2, 0xce, 0x58, 0x1a, - 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xd5, 0x4b, 0xa4, 0xbc, 0x67, 0xa7, 0x8b, 0xf9, 0x50, - 0x9a, 0x2d, 0xe5, 0x75, 0xef, 0x99, 0x48, 0x17, 0xbf, 0x05, 0x17, 0xfb, 0x94, 0x1d, 0xa7, 0xc7, - 0xf3, 0x2f, 0x93, 0x70, 0x25, 0x2c, 0x3e, 0x6e, 0x88, 0xf3, 0x75, 0x6f, 0xd0, 0x1a, 0xe4, 0x3a, - 0xba, 0xf1, 0x66, 0xab, 0x97, 0x73, 0x1d, 0xdd, 0xf0, 0x57, 0x8a, 0x23, 0xba, 0xc6, 0xcc, 0x4f, - 0xb4, 0x6b, 0x28, 0x50, 0x8e, 0x6a, 0xbb, 0x38, 0xfb, 0xc7, 0x77, 0x05, 0x98, 0x8b, 0x7b, 0x59, - 0xee, 0xcd, 0x02, 0xe5, 0xc4, 0x26, 0xe4, 0x7e, 0x02, 0xeb, 0x78, 0xbf, 0x25, 0x00, 0x6a, 0x5a, - 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x6a, 0x5e, 0x80, 0x69, 0xdd, 0xd0, 0xf0, 0x31, - 0xad, 0x66, 0x4a, 0x62, 0x0f, 0xa1, 0xaf, 0x93, 0xc9, 0x89, 0xbe, 0x4e, 0x8a, 0x9f, 0xc3, 0x42, - 0xa8, 0x88, 0x71, 0xd6, 0xff, 0xbf, 0x25, 0x60, 0x81, 0x57, 0x24, 0xf6, 0x15, 0xcc, 0x6f, 0xc0, - 0x74, 0x9b, 0xc8, 0x1c, 0xd1, 0xce, 0xf4, 0x9d, 0x6e, 0x3b, 0xd3, 0xcc, 0xe8, 0x67, 0x01, 0xba, - 0x16, 0x3e, 0x94, 0x19, 0x6b, 0x72, 0x22, 0xd6, 0x0c, 0xe1, 0xa0, 0x04, 0xf4, 0x7d, 0x01, 0x0a, - 0x64, 0x40, 0x77, 0x2d, 0xb3, 0x6b, 0xda, 0xc4, 0x67, 0xb1, 0x27, 0x83, 0x39, 0x2f, 0xce, 0x4e, - 0x17, 0x73, 0x9b, 0xba, 0xb1, 0xcd, 0x19, 0x9b, 0x8d, 0x89, 0xf7, 0x00, 0xb8, 0x3b, 0x21, 0x96, - 0x2a, 0x6d, 0x53, 0x3d, 0xf0, 0xbf, 0xb7, 0x11, 0xcb, 0xe2, 0x89, 0xb3, 0xc5, 0xff, 0x20, 0xc0, - 0x85, 0x9f, 0xd8, 0x72, 0xf1, 0xff, 0x0f, 0x65, 0x8b, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, - 0x33, 0xce, 0x85, 0xfb, 0x5f, 0x15, 0x60, 0x3e, 0x20, 0x38, 0x4e, 0x07, 0xe7, 0x8d, 0xf4, 0x24, - 0xfe, 0x3c, 0x71, 0x79, 0x82, 0xa3, 0x24, 0xce, 0x31, 0xf8, 0xcf, 0x13, 0x70, 0xa9, 0xc2, 0x3e, - 0x73, 0xbb, 0x71, 0x1f, 0x71, 0xf6, 0x8c, 0x12, 0xcc, 0x1e, 0x62, 0xcb, 0xd6, 0x4d, 0x36, 0xfd, - 0xe6, 0x24, 0xf7, 0x11, 0x95, 0x21, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, 0xdd, 0xef, 0x7b, 0xde, - 0xb3, 0x17, 0xa3, 0x32, 0xfd, 0xe6, 0x31, 0x2a, 0x33, 0xa3, 0x63, 0x54, 0x66, 0xbf, 0x42, 0x8c, - 0x0a, 0xff, 0x98, 0xf6, 0x6f, 0x05, 0xb8, 0x3c, 0xa0, 0xb9, 0x38, 0x7b, 0xcb, 0x77, 0x20, 0xab, - 0x72, 0xc1, 0xc4, 0x60, 0xb3, 0x2f, 0x85, 0x75, 0x92, 0xed, 0x0d, 0x71, 0xcb, 0xd9, 0xe9, 0x22, - 0xb8, 0x45, 0xad, 0xaf, 0x71, 0xe5, 0x90, 0xdf, 0x9a, 0xf8, 0x4b, 0x39, 0x28, 0x54, 0x8f, 0xd9, - 0x2a, 0x79, 0x83, 0xb9, 0x09, 0xe8, 0x09, 0xa4, 0xbb, 0x96, 0x79, 0xa8, 0xbb, 0xd5, 0xc8, 0x87, - 0x02, 0x14, 0xdc, 0x6a, 0xf4, 0x71, 0x6d, 0x73, 0x0e, 0xc9, 0xe3, 0x45, 0x4d, 0xc8, 0x3c, 0x33, - 0x55, 0xa5, 0xfd, 0x44, 0x6f, 0xbb, 0x3d, 0xff, 0xfd, 0xf1, 0x82, 0x96, 0x3c, 0x9e, 0x6d, 0xc5, - 0xd9, 0x77, 0x1b, 0xc1, 0x23, 0xa2, 0x3a, 0xa4, 0x6b, 0x8e, 0xd3, 0x25, 0x89, 0xdc, 0x76, 0xdc, - 0x99, 0x40, 0x28, 0x61, 0x71, 0xa3, 0x64, 0x5d, 0x76, 0xd4, 0x84, 0xf9, 0xa7, 0x74, 0xcf, 0x57, - 0xa5, 0x6d, 0xf6, 0xb4, 0x8a, 0x69, 0xec, 0xe9, 0x2d, 0x6e, 0xb7, 0x6f, 0x4f, 0x20, 0xf3, 0x69, - 0xa5, 0x21, 0x0d, 0x0a, 0x40, 0x2b, 0x90, 0x6e, 0x3c, 0xe2, 0xc2, 0x98, 0x5f, 0x77, 0x6b, 0x02, - 0x61, 0x8d, 0x47, 0x92, 0xc7, 0x86, 0xd6, 0x21, 0xbb, 0xf2, 0x45, 0xcf, 0xc2, 0x5c, 0xca, 0xcc, - 0xd0, 0xe8, 0x88, 0x7e, 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x06, 0xe4, 0x5f, 0x99, 0xd6, 0x41, - 0xdb, 0x54, 0xdc, 0x1a, 0xce, 0x52, 0x71, 0x3f, 0x35, 0x81, 0x38, 0x97, 0x51, 0xea, 0x13, 0x81, - 0xbe, 0x05, 0x05, 0xd2, 0x18, 0x4d, 0x65, 0xb7, 0xed, 0x16, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, - 0x54, 0x8f, 0xd3, 0x5d, 0xf8, 0xef, 0x13, 0x55, 0x96, 0x20, 0x17, 0xea, 0x04, 0x08, 0x41, 0xaa, - 0x4b, 0xda, 0x5b, 0xa0, 0xf1, 0x4b, 0xf4, 0x37, 0x7a, 0x0f, 0x66, 0x0d, 0x53, 0xc3, 0xee, 0x08, - 0xc9, 0xad, 0x5e, 0x38, 0x3b, 0x5d, 0x9c, 0xd9, 0x32, 0x35, 0xe6, 0xd1, 0xf0, 0x5f, 0xd2, 0x0c, - 0xc9, 0x54, 0xd7, 0xca, 0xd7, 0x20, 0x45, 0xda, 0x9d, 0x18, 0xa6, 0x5d, 0xc5, 0xc6, 0x3b, 0x96, - 0xce, 0xa5, 0xb9, 0x8f, 0xe5, 0x7f, 0x94, 0x80, 0x44, 0xe3, 0x11, 0xf1, 0xd9, 0x77, 0x7b, 0xea, - 0x01, 0x76, 0x78, 0x3a, 0x7f, 0xa2, 0xbe, 0xbc, 0x85, 0xf7, 0x74, 0xe6, 0x5a, 0x65, 0x24, 0xfe, - 0x84, 0xde, 0x01, 0x50, 0x54, 0x15, 0xdb, 0xb6, 0xec, 0xee, 0x05, 0xcc, 0x48, 0x19, 0x46, 0xd9, - 0xc0, 0x27, 0x84, 0xcd, 0xc6, 0xaa, 0x85, 0x1d, 0x37, 0xf8, 0x8a, 0x3d, 0x11, 0x36, 0x07, 0x77, - 0xba, 0xb2, 0x63, 0x1e, 0x60, 0x83, 0xf6, 0x93, 0x0c, 0x31, 0x35, 0x9d, 0x6e, 0x93, 0x10, 0x88, - 0x95, 0xc4, 0x86, 0xe6, 0x9b, 0xb4, 0x8c, 0xe4, 0x3d, 0x13, 0x91, 0x16, 0x6e, 0xe9, 0x7c, 0x53, - 0x5b, 0x46, 0xe2, 0x4f, 0x44, 0x4b, 0x4a, 0xcf, 0xd9, 0xa7, 0x2d, 0x91, 0x91, 0xe8, 0x6f, 0x74, - 0x1b, 0x0a, 0x2c, 0x5e, 0x53, 0xc6, 0x86, 0x2a, 0x53, 0xe3, 0x9a, 0xa1, 0xc9, 0x39, 0x46, 0xae, - 0x1a, 0x2a, 0x31, 0xa5, 0xe8, 0x11, 0x70, 0x82, 0x7c, 0xd0, 0xb1, 0x89, 0x4e, 0x81, 0xe4, 0x5a, - 0x2d, 0x9c, 0x9d, 0x2e, 0x66, 0x1b, 0x34, 0x61, 0x63, 0xb3, 0x51, 0x5f, 0x93, 0xb2, 0x2c, 0xd7, - 0x46, 0xc7, 0xae, 0x6b, 0xe5, 0x5f, 0x13, 0x20, 0xf9, 0xb4, 0xd2, 0x38, 0xb7, 0xca, 0xdc, 0x82, - 0x26, 0x03, 0x05, 0xbd, 0x03, 0x85, 0x5d, 0xbd, 0xdd, 0xd6, 0x8d, 0x16, 0xf1, 0xa2, 0xbe, 0x83, - 0x55, 0x57, 0x61, 0x79, 0x4e, 0xde, 0x66, 0x54, 0x74, 0x0d, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, - 0xba, 0xd2, 0xb6, 0xb9, 0xe6, 0x82, 0xa4, 0xf2, 0x2f, 0x0a, 0x30, 0x4d, 0x47, 0x00, 0x7a, 0x1b, - 0x32, 0xaa, 0x69, 0x38, 0x8a, 0x6e, 0x70, 0x53, 0x96, 0x91, 0x7c, 0xc2, 0xd0, 0xe2, 0x5d, 0x87, - 0x39, 0x45, 0x55, 0xcd, 0x9e, 0xe1, 0xc8, 0x86, 0xd2, 0xc1, 0xbc, 0x98, 0x59, 0x4e, 0xdb, 0x52, - 0x3a, 0x18, 0x2d, 0x82, 0xfb, 0xe8, 0x6d, 0xf1, 0xcc, 0x48, 0xc0, 0x49, 0x1b, 0xf8, 0xa4, 0xfc, - 0xfb, 0x02, 0xa4, 0xdd, 0x31, 0x43, 0x8a, 0xd1, 0xc2, 0x06, 0x0b, 0x2a, 0x77, 0x8b, 0xe1, 0x11, - 0xfa, 0xa7, 0xca, 0x8c, 0x3f, 0x55, 0x5e, 0x80, 0x69, 0x87, 0x0c, 0x0b, 0x5e, 0x02, 0xf6, 0x40, - 0xd7, 0xb3, 0xdb, 0x4a, 0x8b, 0x2d, 0xe7, 0x65, 0x24, 0xf6, 0x40, 0x2a, 0xc3, 0xc3, 0x7e, 0x99, - 0x46, 0xf8, 0x13, 0x29, 0x29, 0x0b, 0x4e, 0xdd, 0xc5, 0x2d, 0xdd, 0xa0, 0x7d, 0x29, 0x29, 0x01, - 0x25, 0xad, 0x12, 0x0a, 0x7a, 0x0b, 0x32, 0x2c, 0x03, 0x36, 0x34, 0xda, 0xa1, 0x92, 0x52, 0x9a, - 0x12, 0xaa, 0x86, 0x56, 0xc6, 0x90, 0xf1, 0x06, 0x27, 0x69, 0xb6, 0x9e, 0xed, 0x29, 0x92, 0xfe, - 0x46, 0xef, 0xc3, 0x85, 0xd7, 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x95, 0x3a, 0x1a, 0x75, 0x4f, 0x75, - 0xc6, 0x6a, 0x82, 0xbc, 0x34, 0x2a, 0x81, 0xaa, 0xce, 0x1d, 0xcb, 0x49, 0x7f, 0x2c, 0x8b, 0xbf, - 0x23, 0xc0, 0x3c, 0x8b, 0x3c, 0x62, 0x01, 0xb3, 0xf1, 0xf9, 0x21, 0x1f, 0x43, 0x46, 0x53, 0x1c, - 0x85, 0x6d, 0x5a, 0x4d, 0x8c, 0xdc, 0xb4, 0xea, 0x6d, 0xa2, 0x50, 0x1c, 0x85, 0x6e, 0x5c, 0x45, - 0x90, 0x22, 0xbf, 0xd9, 0xfe, 0x5e, 0x89, 0xfe, 0x16, 0x3f, 0x03, 0x14, 0x2c, 0x68, 0x9c, 0x1e, - 0xd9, 0x3d, 0xb8, 0x48, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, - 0xb5, 0x51, 0x31, 0xf0, 0x61, 0x8b, 0x7e, 0xcf, 0x12, 0x7f, 0x77, 0x06, 0x72, 0xd5, 0xe3, 0xae, - 0x69, 0xc5, 0xba, 0x0c, 0xb6, 0x0a, 0xb3, 0x7c, 0xa5, 0x60, 0xc4, 0xb7, 0xeb, 0x3e, 0x63, 0xee, - 0x7e, 0xb8, 0xe7, 0x8c, 0x68, 0x15, 0x80, 0x05, 0xb5, 0xd2, 0xc0, 0xa5, 0xe4, 0x39, 0x3e, 0xb5, - 0x51, 0x36, 0xba, 0x81, 0x63, 0x0b, 0xb2, 0x9d, 0x43, 0x55, 0x95, 0xf7, 0xf4, 0xb6, 0xc3, 0x63, - 0x03, 0xa3, 0xc3, 0xd8, 0x37, 0x5f, 0x56, 0x2a, 0x4f, 0x68, 0x26, 0x16, 0xa6, 0xe7, 0x3f, 0x4b, - 0x40, 0x24, 0xb0, 0xdf, 0xe8, 0x5d, 0xe0, 0x9b, 0x89, 0x64, 0xdb, 0xdd, 0x37, 0xb8, 0x9a, 0x3b, - 0x3b, 0x5d, 0xcc, 0x48, 0x94, 0xda, 0x68, 0x34, 0xa5, 0x0c, 0xcb, 0xd0, 0xb0, 0x1d, 0x74, 0x03, - 0x72, 0x66, 0x47, 0x77, 0x64, 0xd7, 0x49, 0xe2, 0x1e, 0xe5, 0x1c, 0x21, 0xba, 0x4e, 0xd4, 0x79, - 0xf6, 0x98, 0xcc, 0x4e, 0xbc, 0xc7, 0x04, 0xfd, 0x55, 0x01, 0x2e, 0x71, 0x45, 0xca, 0xbb, 0x34, - 0x0e, 0x5f, 0x69, 0xeb, 0xce, 0x89, 0x7c, 0x70, 0x58, 0x4a, 0x53, 0xbf, 0xf5, 0x67, 0x22, 0x1b, - 0x24, 0xd0, 0x0f, 0x96, 0xdc, 0x66, 0x39, 0x79, 0xc6, 0x99, 0x37, 0x0e, 0xab, 0x86, 0x63, 0x9d, - 0xac, 0x5e, 0x3e, 0x3b, 0x5d, 0x5c, 0x18, 0x4c, 0x7d, 0x29, 0x2d, 0xd8, 0x83, 0x2c, 0xa8, 0x06, - 0x80, 0xbd, 0x7e, 0x48, 0x67, 0x8c, 0x68, 0xff, 0x23, 0xb2, 0xc3, 0x4a, 0x01, 0x5e, 0x74, 0x17, - 0x8a, 0x7c, 0x4f, 0xcf, 0x9e, 0xde, 0xc6, 0xb2, 0xad, 0x7f, 0x81, 0xe9, 0xdc, 0x92, 0x94, 0xf2, - 0x8c, 0x4e, 0x44, 0x34, 0xf4, 0x2f, 0x70, 0xf9, 0x3b, 0x50, 0x1a, 0x56, 0xfa, 0xe0, 0x10, 0xc8, - 0xb0, 0x4f, 0xba, 0x1f, 0x85, 0xd7, 0x73, 0x26, 0xe8, 0xaa, 0x7c, 0x4d, 0xe7, 0xe3, 0xc4, 0x47, - 0x82, 0xf8, 0x0f, 0x12, 0x90, 0x5b, 0xed, 0xb5, 0x0f, 0x9e, 0x77, 0x1b, 0xbd, 0x4e, 0x47, 0xb1, - 0x4e, 0x88, 0x19, 0x64, 0x86, 0x82, 0x14, 0x50, 0x60, 0x66, 0x90, 0x5a, 0x02, 0xfd, 0x0b, 0x4c, - 0x26, 0xa7, 0xe0, 0x3e, 0x75, 0xb6, 0xcf, 0x80, 0xd6, 0x21, 0xb0, 0xf9, 0xdc, 0x3c, 0xb2, 0xd1, - 0x47, 0x50, 0x0a, 0x64, 0xa4, 0x8b, 0x2f, 0x32, 0x36, 0x1c, 0x4b, 0xc7, 0x6c, 0x01, 0x31, 0x29, - 0x05, 0x62, 0x7a, 0xea, 0x24, 0xb9, 0xca, 0x52, 0x51, 0x13, 0xe6, 0x48, 0xc6, 0x13, 0x99, 0x4e, - 0x21, 0xee, 0x02, 0xef, 0x83, 0x88, 0x6a, 0x85, 0xca, 0xbd, 0x44, 0xf5, 0x53, 0xa1, 0x3c, 0xf4, - 0xa7, 0x94, 0xc5, 0x3e, 0xa5, 0xfc, 0x29, 0x14, 0xfb, 0x33, 0x04, 0x75, 0x99, 0x62, 0xba, 0xbc, - 0x10, 0xd4, 0x65, 0x32, 0xa0, 0xa7, 0xf5, 0x54, 0x3a, 0x55, 0x9c, 0x16, 0xff, 0x24, 0x09, 0x79, - 0xb7, 0x9b, 0xc5, 0x09, 0x74, 0x56, 0x61, 0x9a, 0x74, 0x0a, 0x37, 0x02, 0xe5, 0xf6, 0x88, 0xde, - 0xcd, 0x23, 0xdb, 0x49, 0x67, 0x71, 0x41, 0x32, 0x65, 0x8d, 0xc3, 0xe0, 0x94, 0x7f, 0x31, 0x01, - 0x29, 0x8a, 0x2d, 0x1e, 0x40, 0x8a, 0x4e, 0x14, 0xc2, 0x24, 0x13, 0x05, 0xcd, 0xea, 0x4d, 0x67, - 0x89, 0x80, 0x6b, 0x4a, 0x7c, 0xbe, 0x7d, 0xe5, 0x83, 0x07, 0x0f, 0xa9, 0xb1, 0x99, 0x93, 0xf8, - 0x13, 0x5a, 0xa5, 0xa1, 0x51, 0xa6, 0xe5, 0x60, 0x8d, 0xfb, 0xf4, 0xd7, 0xc6, 0xb5, 0xaf, 0x3b, - 0x29, 0xb9, 0x7c, 0xe8, 0x0a, 0x24, 0x89, 0x15, 0x9b, 0x65, 0x51, 0x0e, 0x67, 0xa7, 0x8b, 0x49, - 0x62, 0xbf, 0x08, 0x0d, 0x2d, 0x43, 0x36, 0x6c, 0x32, 0x88, 0x07, 0x47, 0x0d, 0x63, 0x60, 0xb8, - 0x43, 0xdb, 0x1b, 0x5a, 0x0c, 0xcf, 0xf2, 0x36, 0xfe, 0xd3, 0x14, 0xe4, 0xea, 0x9d, 0xb8, 0xa7, - 0x94, 0x95, 0x70, 0x0b, 0x47, 0x01, 0xa1, 0xd0, 0x4b, 0x23, 0x1a, 0x38, 0x34, 0x83, 0x27, 0xcf, - 0x37, 0x83, 0xd7, 0x89, 0xa7, 0xcc, 0x0f, 0x9e, 0x48, 0x0e, 0xc1, 0x3c, 0xe1, 0xf7, 0x53, 0x3f, - 0x45, 0x22, 0x3c, 0xfe, 0x5e, 0x0f, 0x1a, 0xa9, 0xf2, 0x29, 0x75, 0xc8, 0x59, 0x2f, 0x9b, 0x99, - 0xbc, 0x97, 0xcd, 0x62, 0x43, 0xa3, 0x93, 0x5a, 0xd8, 0xa2, 0xce, 0xbe, 0xb9, 0x45, 0x2d, 0x3b, - 0xbc, 0xb3, 0x7e, 0x0c, 0x49, 0x4d, 0x77, 0x1b, 0x67, 0xf2, 0xa9, 0x9a, 0x30, 0x8d, 0xe9, 0xb5, - 0xa9, 0x60, 0xaf, 0x65, 0xbd, 0xa4, 0x5c, 0x07, 0xf0, 0x75, 0x83, 0xae, 0xc1, 0x8c, 0xd9, 0xd6, - 0xdc, 0xcd, 0x2e, 0xb9, 0xd5, 0xcc, 0xd9, 0xe9, 0xe2, 0xf4, 0xf3, 0xb6, 0x56, 0x5f, 0x93, 0xa6, - 0xcd, 0xb6, 0x56, 0xd7, 0xe8, 0xa9, 0x1f, 0xf8, 0x48, 0xf6, 0x22, 0xe1, 0xe6, 0xa4, 0x59, 0x03, - 0x1f, 0xad, 0x61, 0x5b, 0xe5, 0x1d, 0xee, 0x37, 0x05, 0xc8, 0xbb, 0xba, 0x8f, 0xd7, 0xa8, 0xa4, - 0xf5, 0x0e, 0x1f, 0x64, 0xc9, 0xf3, 0x0d, 0x32, 0x97, 0x8f, 0x6f, 0x2c, 0xfe, 0x65, 0x81, 0xc7, - 0x36, 0x37, 0x54, 0xc5, 0x21, 0x4e, 0x45, 0x8c, 0x03, 0xe3, 0x1e, 0x14, 0x2d, 0xc5, 0xd0, 0xcc, - 0x8e, 0xfe, 0x05, 0x66, 0x0b, 0xa1, 0x36, 0xff, 0xec, 0x59, 0xf0, 0xe8, 0x74, 0xd5, 0xcf, 0x16, - 0x7f, 0x3f, 0xc1, 0xe3, 0xa0, 0xbd, 0x62, 0xc4, 0xa9, 0xae, 0x6f, 0xc3, 0x7c, 0xff, 0x11, 0x2c, - 0xee, 0x68, 0x7d, 0x2f, 0x42, 0x5e, 0x54, 0x41, 0x58, 0x3c, 0xa3, 0x1b, 0x5c, 0xdf, 0x77, 0x1c, - 0x8b, 0x8d, 0x2a, 0x90, 0x0d, 0x9e, 0xec, 0x92, 0x9c, 0xf8, 0x64, 0x17, 0xb0, 0xbc, 0xf3, 0x5c, - 0xca, 0x3f, 0x07, 0xd3, 0x34, 0xf9, 0x0d, 0x4c, 0x34, 0x6f, 0xcd, 0x3f, 0x4e, 0xc0, 0x4d, 0x5a, - 0xfa, 0x97, 0xd8, 0xd2, 0xf7, 0x4e, 0xb6, 0x2d, 0xd3, 0xc1, 0xaa, 0x83, 0x35, 0x7f, 0xe5, 0x3c, - 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x2f, 0x38, 0x57, 0xc0, 0x99, 0xc7, 0x85, 0x36, 0xa0, 0xc0, 0x43, - 0x0b, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0x39, 0xcf, 0xec, 0x96, 0x63, 0xbc, 0x2b, 0x84, 0x75, - 0xc5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0x07, 0x12, 0x3d, 0xfd, 0x6a, 0x0b, 0x8a, 0x69, - 0x16, 0xdf, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, 0xa3, 0x00, 0xb7, 0xc6, 0xa8, - 0x38, 0xce, 0xae, 0x5b, 0x86, 0xf4, 0x21, 0x79, 0x91, 0xce, 0x75, 0x9c, 0x96, 0xbc, 0x67, 0xb4, - 0x09, 0xb9, 0x3d, 0x45, 0x6f, 0xfb, 0x5d, 0x7a, 0x78, 0x7c, 0x62, 0x74, 0x34, 0xee, 0x1c, 0x63, - 0x67, 0x7d, 0x58, 0xfc, 0xf5, 0x04, 0xcc, 0xaf, 0x68, 0x5a, 0xa3, 0xc1, 0x6d, 0x60, 0x7c, 0x3d, - 0xc5, 0x05, 0x99, 0x09, 0x1f, 0x64, 0xa2, 0xf7, 0x00, 0x69, 0xba, 0xcd, 0x0e, 0x3e, 0xb1, 0xf7, - 0x15, 0xcd, 0x3c, 0xf2, 0xe3, 0x32, 0xe6, 0xdd, 0x94, 0x86, 0x9b, 0x80, 0x1a, 0x40, 0xd1, 0x8e, - 0x6c, 0x3b, 0x8a, 0xf7, 0xdd, 0xe9, 0xd6, 0x44, 0x5b, 0xca, 0x18, 0x0c, 0xf2, 0x1e, 0xa5, 0x0c, - 0x91, 0x43, 0x7f, 0x12, 0xbf, 0x5d, 0x27, 0x55, 0x77, 0x64, 0xc5, 0x76, 0xf7, 0x0f, 0xb1, 0x23, - 0x57, 0xf2, 0x8c, 0xbe, 0x62, 0xb3, 0x6d, 0x41, 0x6c, 0xc3, 0x83, 0xaf, 0x9a, 0x38, 0x21, 0xf1, - 0xdf, 0x11, 0x20, 0x2f, 0xe1, 0x3d, 0x0b, 0xdb, 0xb1, 0x2e, 0x0a, 0x3c, 0x81, 0x39, 0x8b, 0x49, - 0x95, 0xf7, 0x2c, 0xb3, 0x73, 0x9e, 0x71, 0x95, 0xe5, 0x8c, 0x4f, 0x2c, 0xb3, 0xc3, 0x0d, 0xcb, - 0x4b, 0x28, 0x78, 0x65, 0x8c, 0xb3, 0xf2, 0xbf, 0x43, 0xb7, 0x4b, 0x33, 0xc1, 0x71, 0x07, 0x48, - 0xc4, 0xab, 0x01, 0xfa, 0xa1, 0x2a, 0x58, 0xd0, 0x38, 0xd5, 0xf0, 0x5f, 0x05, 0xc8, 0x37, 0x7a, - 0xbb, 0xec, 0x64, 0xad, 0xf8, 0x34, 0x50, 0x85, 0x4c, 0x1b, 0xef, 0x39, 0xf2, 0x1b, 0x85, 0xea, - 0xa7, 0x09, 0x2b, 0xdd, 0xa8, 0xf0, 0x14, 0xc0, 0xa2, 0x5b, 0xec, 0xa8, 0x9c, 0xe4, 0x39, 0xe5, - 0x64, 0x28, 0x2f, 0x21, 0x8b, 0x7f, 0x9a, 0x80, 0x82, 0x57, 0xcd, 0x38, 0xad, 0xe4, 0xab, 0x90, - 0x75, 0x48, 0x9e, 0xc7, 0x3a, 0xcc, 0xf3, 0x98, 0x90, 0x68, 0x0b, 0xb1, 0x04, 0x0b, 0xd4, 0x71, - 0x91, 0x95, 0x6e, 0xb7, 0xad, 0xbb, 0x70, 0x97, 0xda, 0x9f, 0x94, 0x34, 0x4f, 0x93, 0x56, 0x58, - 0x0a, 0x05, 0xba, 0xe8, 0x57, 0x04, 0x98, 0xdb, 0xb3, 0x30, 0xfe, 0x02, 0xcb, 0x14, 0x7a, 0x4d, - 0x16, 0xf4, 0xb2, 0x46, 0xca, 0xf0, 0x95, 0x3f, 0x8a, 0x67, 0xd9, 0x8b, 0x1b, 0xe4, 0xbd, 0xbc, - 0xd3, 0xbe, 0x82, 0x79, 0xda, 0x28, 0x71, 0xef, 0x2d, 0x16, 0xff, 0x56, 0x02, 0x50, 0x50, 0xf2, - 0x4f, 0xae, 0x31, 0x13, 0xf1, 0x35, 0xe6, 0xbb, 0x80, 0x58, 0xd8, 0xa4, 0x2d, 0x77, 0xb1, 0x25, - 0xdb, 0x58, 0x35, 0xf9, 0xf1, 0x50, 0x82, 0x54, 0xe4, 0x29, 0xdb, 0xd8, 0x6a, 0x50, 0x3a, 0x7a, - 0x0c, 0xe0, 0xbb, 0x74, 0x7c, 0xc6, 0x19, 0xe9, 0xd1, 0x49, 0x19, 0xcf, 0x97, 0x13, 0xbf, 0x27, - 0x40, 0x7e, 0x53, 0x6f, 0x59, 0x4a, 0xac, 0xe7, 0x1e, 0xa1, 0x8f, 0xc3, 0xcb, 0xe8, 0xd9, 0x87, - 0xe5, 0xa8, 0x10, 0x1f, 0x96, 0xc3, 0x05, 0x67, 0x9c, 0x81, 0x98, 0x70, 0xaf, 0x44, 0x71, 0xda, - 0xae, 0x7f, 0x57, 0x86, 0x39, 0x5e, 0xee, 0x1d, 0x43, 0x37, 0x0d, 0xf4, 0x00, 0x92, 0x2d, 0xfe, - 0x99, 0x24, 0x1b, 0xb9, 0xa4, 0xe9, 0x1f, 0x01, 0x58, 0x9b, 0x92, 0x48, 0x5e, 0xc2, 0xd2, 0xed, - 0x39, 0x11, 0x4e, 0xa4, 0x1f, 0xf5, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, 0xea, 0x1f, - 0x65, 0x26, 0x13, 0xf6, 0xe4, 0x50, 0xc0, 0x19, 0x79, 0xa8, 0x5c, 0x6d, 0x4a, 0xca, 0xab, 0xa1, - 0x04, 0x54, 0x09, 0x9e, 0xa0, 0x95, 0x1a, 0x08, 0xa9, 0xf3, 0xf7, 0x6b, 0x87, 0x4f, 0xef, 0xaa, - 0x4d, 0x05, 0x0e, 0xda, 0x42, 0x1f, 0xc3, 0x8c, 0x46, 0xcf, 0x6a, 0xe2, 0x83, 0x3f, 0xaa, 0xa1, - 0x43, 0x47, 0x62, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x2e, 0x1d, 0x47, - 0xe1, 0xb7, 0x86, 0x4b, 0x08, 0x4c, 0x9a, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x0d, 0x48, - 0xd9, 0xaa, 0xe2, 0xe2, 0xf0, 0xab, 0x43, 0x8e, 0x48, 0xf1, 0x99, 0x69, 0x6e, 0xf4, 0x98, 0x1d, - 0xea, 0xe9, 0x1c, 0xbb, 0x4b, 0xa2, 0x51, 0xc5, 0x0f, 0x6d, 0xc5, 0x27, 0xc5, 0xc7, 0x94, 0x80, - 0x9e, 0x42, 0x56, 0x21, 0xbe, 0xb1, 0x4c, 0x37, 0xb3, 0xd2, 0x35, 0xd0, 0xe8, 0x40, 0x84, 0x81, - 0xcd, 0xc7, 0x35, 0xba, 0xdf, 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, 0x76, 0xb4, - 0xa0, 0x60, 0x20, 0x9d, 0x27, 0x88, 0x12, 0x89, 0x8f, 0xec, 0xed, 0x24, 0xa7, 0x95, 0x9a, 0x1b, - 0xfa, 0xe9, 0x3b, 0x62, 0x8f, 0x54, 0x6d, 0x4a, 0x9a, 0xdb, 0x0f, 0x90, 0xd1, 0x12, 0x24, 0x5a, - 0x6a, 0x29, 0x37, 0xd4, 0x18, 0x78, 0xfb, 0x80, 0x6a, 0x53, 0x52, 0xa2, 0xa5, 0xa2, 0x4f, 0x21, - 0xcd, 0x36, 0x75, 0x1c, 0x1b, 0xa5, 0xfc, 0xd0, 0x31, 0x16, 0xde, 0x1a, 0x53, 0x9b, 0x92, 0xe8, - 0x3e, 0x12, 0xf2, 0xbe, 0x6d, 0xc8, 0x5b, 0x2c, 0x12, 0xd1, 0x8d, 0x1b, 0x2e, 0x0e, 0x0d, 0x07, - 0x88, 0x0a, 0x1d, 0xae, 0x51, 0x94, 0x14, 0xa0, 0xa3, 0x6f, 0xc3, 0x85, 0xb0, 0x44, 0xde, 0xd3, - 0xe6, 0x87, 0x7e, 0xda, 0x1e, 0x1a, 0xc6, 0x5a, 0x9b, 0x92, 0x90, 0x35, 0x90, 0x88, 0x3e, 0x84, - 0x69, 0xd6, 0x6a, 0x88, 0x8a, 0x8c, 0x8a, 0x70, 0xe9, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, - 0x87, 0xe3, 0xc9, 0x6d, 0xb3, 0x55, 0x5a, 0x18, 0xda, 0xf9, 0x07, 0x03, 0x0b, 0x49, 0xe7, 0x77, - 0x7c, 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, 0x10, 0xac, 0x0b, 0x43, 0xdb, 0x3d, 0x22, 0x4a, 0xaf, - 0x46, 0x37, 0x4a, 0xf8, 0x64, 0x52, 0x34, 0x8b, 0x9d, 0xea, 0x23, 0xd3, 0x31, 0x75, 0x71, 0x68, - 0xd1, 0x06, 0x8f, 0x3d, 0xaa, 0x51, 0x7f, 0xd2, 0xa3, 0xa2, 0x97, 0x50, 0xe4, 0x67, 0x6f, 0xf8, - 0xdf, 0x5f, 0x2e, 0x51, 0x79, 0xf7, 0x22, 0x4d, 0x57, 0x54, 0xfc, 0x52, 0x6d, 0x4a, 0x2a, 0xa8, - 0xe1, 0x14, 0xf4, 0x19, 0xcc, 0x53, 0x79, 0xb2, 0xea, 0x1f, 0x97, 0x52, 0x2a, 0x0d, 0x1c, 0xbe, - 0x31, 0xfc, 0x64, 0x15, 0x57, 0x72, 0x51, 0xed, 0x4b, 0x22, 0xdd, 0x58, 0x37, 0x74, 0x87, 0x5a, - 0xd9, 0xf2, 0xd0, 0x6e, 0x1c, 0x3e, 0xfa, 0x91, 0x74, 0x63, 0x9d, 0x51, 0x48, 0x37, 0x76, 0x78, - 0x7c, 0x1e, 0x6f, 0x8e, 0xb7, 0x87, 0x76, 0xe3, 0xa8, 0x40, 0x3e, 0xd2, 0x8d, 0x9d, 0x20, 0x9d, - 0x74, 0x63, 0x66, 0x20, 0xfa, 0xe4, 0xbe, 0x33, 0xb4, 0x1b, 0x0f, 0xdd, 0x54, 0x4e, 0xba, 0xb1, - 0x32, 0x90, 0x88, 0xd6, 0x00, 0x98, 0xeb, 0x47, 0xe7, 0xff, 0xab, 0x43, 0x27, 0x83, 0xfe, 0x38, - 0x3d, 0x32, 0x19, 0xb4, 0x5d, 0x1a, 0x31, 0x64, 0x14, 0x58, 0xca, 0xf4, 0x73, 0x73, 0x69, 0x71, - 0xa8, 0x21, 0x1b, 0xf8, 0x34, 0x4c, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xb3, 0x0a, 0x5b, 0x1f, 0x2f, - 0x5d, 0x1b, 0x6e, 0x96, 0x83, 0x9f, 0xc9, 0xa8, 0x59, 0xa6, 0x04, 0xb4, 0x02, 0x19, 0xe2, 0xde, - 0x9c, 0x50, 0x33, 0x74, 0x7d, 0xa8, 0xff, 0xde, 0xb7, 0x13, 0xa8, 0x36, 0x25, 0xa5, 0x5f, 0x73, - 0x12, 0x79, 0x3d, 0x5b, 0x39, 0x2c, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x65, 0x26, 0xaf, 0x67, 0x1c, - 0x48, 0x85, 0x8b, 0xac, 0xad, 0xf8, 0xfe, 0x6e, 0x8b, 0x6f, 0x49, 0x2e, 0xdd, 0xa0, 0xa2, 0x86, - 0x2e, 0xc1, 0x45, 0x6e, 0x3b, 0xaf, 0x4d, 0x49, 0x0b, 0xca, 0x60, 0x2a, 0x19, 0xf0, 0x7c, 0xea, - 0x61, 0x0b, 0x77, 0xa5, 0x9b, 0x43, 0x07, 0x7c, 0xc4, 0x7a, 0x27, 0x19, 0xf0, 0x4a, 0x80, 0xcc, - 0x26, 0x20, 0x4d, 0xb6, 0x6d, 0x16, 0x9c, 0x70, 0x6b, 0xc4, 0x04, 0xd4, 0xb7, 0x62, 0xc2, 0x26, - 0x20, 0xad, 0xc1, 0x38, 0x89, 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, 0xdb, 0x43, 0x05, 0x0d, - 0x9c, 0xb2, 0x48, 0x04, 0xa9, 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0x68, 0x0f, 0xf7, 0x8d, 0xef, - 0x0c, 0x75, 0x78, 0x22, 0x4f, 0x15, 0x22, 0x0e, 0x8f, 0x15, 0x4a, 0x40, 0x3f, 0x0b, 0xb3, 0x1c, - 0xea, 0x96, 0xee, 0x8e, 0x70, 0x01, 0x83, 0xab, 0x13, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, - 0xc4, 0x66, 0xd5, 0xbb, 0x37, 0xc2, 0xca, 0x0e, 0xa0, 0x7c, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, - 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0xfe, 0x50, 0x2b, 0x3b, 0xb8, 0xcf, 0x88, 0x58, 0xd9, 0xd7, 0x3e, - 0x95, 0xd4, 0xcc, 0x66, 0x50, 0xb3, 0xf4, 0x53, 0x43, 0x6b, 0x16, 0xc6, 0xdc, 0xa4, 0x66, 0x9c, - 0x87, 0x34, 0x1b, 0xf3, 0xfe, 0x99, 0xa6, 0xdf, 0x1d, 0x7e, 0x88, 0x42, 0x3f, 0xbe, 0xaa, 0xb9, - 0x8b, 0xba, 0x4c, 0xc3, 0x9e, 0xa1, 0xb2, 0xf8, 0x0e, 0x6f, 0xae, 0xa9, 0xf7, 0x46, 0x1b, 0xaa, - 0xa8, 0x9d, 0xf1, 0x9e, 0xa1, 0x0a, 0x25, 0xd2, 0xa2, 0xb2, 0x4d, 0x7d, 0x74, 0x7c, 0x2f, 0x8d, - 0x38, 0xef, 0xa1, 0x6f, 0x6b, 0x25, 0x2d, 0xaa, 0x47, 0xf4, 0x87, 0x50, 0x8f, 0x1d, 0x46, 0x52, - 0x5a, 0x1e, 0x3d, 0x84, 0xc2, 0xc7, 0xa1, 0x78, 0x43, 0x88, 0x93, 0xbd, 0x39, 0xd3, 0xf5, 0x30, - 0xde, 0x1f, 0x3d, 0x67, 0xf6, 0xbb, 0x16, 0x6c, 0xce, 0xe4, 0x3e, 0xc5, 0x5f, 0x11, 0xe0, 0x1a, - 0x2b, 0x1b, 0x5d, 0xfd, 0x3c, 0x91, 0xbd, 0x35, 0xe4, 0xc0, 0x06, 0x93, 0x07, 0xf4, 0x05, 0x1f, - 0x0e, 0x2b, 0xee, 0x98, 0x35, 0xf1, 0xda, 0x94, 0xf4, 0x8e, 0x32, 0x2a, 0x1f, 0xe9, 0x52, 0x1d, - 0x86, 0xa0, 0x4a, 0x0f, 0x87, 0x76, 0xa9, 0x30, 0xea, 0x23, 0x5d, 0x8a, 0xf3, 0xac, 0xce, 0xf2, - 0x2f, 0xd0, 0xde, 0x86, 0xdc, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, 0x5f, 0x2e, 0x96, 0xd6, 0x53, 0xe9, - 0x2b, 0xc5, 0xf2, 0x7a, 0x2a, 0xfd, 0x56, 0xf1, 0x6d, 0xf1, 0x1f, 0x96, 0x21, 0xe7, 0x82, 0x2e, - 0x06, 0xa8, 0x1e, 0x06, 0x01, 0xd5, 0xd5, 0x61, 0x80, 0x8a, 0xc3, 0x34, 0x8e, 0xa8, 0x1e, 0x06, - 0x11, 0xd5, 0xd5, 0x61, 0x88, 0xca, 0xe7, 0x21, 0x90, 0xaa, 0x39, 0x0c, 0x52, 0xdd, 0x9b, 0x00, - 0x52, 0x79, 0xa2, 0xfa, 0x31, 0xd5, 0xda, 0x20, 0xa6, 0xba, 0x39, 0x1a, 0x53, 0x79, 0xa2, 0x02, - 0xa0, 0xea, 0x71, 0x1f, 0xa8, 0xba, 0x3e, 0x02, 0x54, 0x79, 0xfc, 0x2e, 0xaa, 0xda, 0x88, 0x44, - 0x55, 0xb7, 0xc7, 0xa1, 0x2a, 0x4f, 0x4e, 0x08, 0x56, 0x7d, 0x10, 0x82, 0x55, 0x8b, 0x43, 0x61, - 0x95, 0xc7, 0xcd, 0x70, 0xd5, 0x27, 0xfd, 0xb8, 0xea, 0xfa, 0x08, 0x5c, 0xe5, 0xd7, 0x80, 0x03, - 0xab, 0x5a, 0x14, 0xb0, 0xba, 0x35, 0x06, 0x58, 0x79, 0x52, 0x82, 0xc8, 0xaa, 0x16, 0x85, 0xac, - 0x6e, 0x8d, 0x41, 0x56, 0x7d, 0x92, 0x18, 0xb4, 0xda, 0x8a, 0x86, 0x56, 0x77, 0xc6, 0x42, 0x2b, - 0x4f, 0x5a, 0x18, 0x5b, 0x2d, 0x07, 0xb0, 0xd5, 0x3b, 0x43, 0xb0, 0x95, 0xc7, 0x4a, 0xc0, 0xd5, - 0x37, 0x07, 0xc0, 0x95, 0x38, 0x0a, 0x5c, 0x79, 0xbc, 0x1e, 0xba, 0x7a, 0x31, 0x04, 0x5d, 0xdd, - 0x1d, 0x8f, 0xae, 0x3c, 0x61, 0x7d, 0xf0, 0x4a, 0x19, 0x09, 0xaf, 0xde, 0x9b, 0x10, 0x5e, 0x79, - 0xd2, 0xa3, 0xf0, 0xd5, 0x47, 0x61, 0x7c, 0x75, 0x6d, 0x38, 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, - 0x11, 0x09, 0xb0, 0x6e, 0x8f, 0x03, 0x58, 0xfe, 0x38, 0x08, 0x22, 0xac, 0xad, 0x68, 0x84, 0x75, - 0x67, 0x2c, 0xc2, 0xf2, 0x9b, 0x3f, 0x04, 0xb1, 0x36, 0x22, 0x21, 0xd6, 0xed, 0x71, 0x10, 0xcb, - 0x2f, 0x5c, 0x10, 0x63, 0xbd, 0x1a, 0x8a, 0xb1, 0xee, 0x4f, 0x82, 0xb1, 0x3c, 0xa1, 0x03, 0x20, - 0xeb, 0xf3, 0xe1, 0x20, 0xeb, 0xa7, 0xce, 0x71, 0x7c, 0x65, 0x24, 0xca, 0xfa, 0xe6, 0x00, 0xca, - 0x12, 0x47, 0xa1, 0x2c, 0xbf, 0x3f, 0xbb, 0x30, 0x4b, 0x19, 0x09, 0x8a, 0xde, 0x9b, 0x10, 0x14, - 0xf9, 0x9d, 0x2f, 0x02, 0x15, 0x55, 0x23, 0x50, 0xd1, 0xcd, 0xd1, 0xa8, 0xc8, 0x37, 0xe7, 0x3e, - 0x2c, 0xaa, 0x45, 0xc1, 0xa2, 0x5b, 0x63, 0x60, 0x91, 0x6f, 0x85, 0x02, 0xb8, 0xe8, 0x71, 0x1f, - 0x2e, 0xba, 0x3e, 0x36, 0xc0, 0x2a, 0x00, 0x8c, 0x56, 0x07, 0x81, 0xd1, 0x8d, 0x91, 0xc0, 0xc8, - 0x93, 0xe0, 0x23, 0xa3, 0xc7, 0x7d, 0xc8, 0xe8, 0xfa, 0x08, 0x64, 0xe4, 0x17, 0x80, 0x43, 0x23, - 0x6d, 0x34, 0x34, 0x5a, 0x9a, 0x14, 0x1a, 0x79, 0x82, 0x23, 0xb1, 0xd1, 0x56, 0x34, 0x36, 0xba, - 0x33, 0x61, 0xec, 0xc3, 0x00, 0x38, 0xaa, 0x45, 0x81, 0xa3, 0x5b, 0x63, 0xc0, 0x51, 0x70, 0x0e, - 0xf1, 0xd0, 0x51, 0x2d, 0x0a, 0x1d, 0xdd, 0x1a, 0x83, 0x8e, 0x7c, 0x49, 0x01, 0x78, 0xd4, 0x1c, - 0x06, 0x8f, 0xee, 0x4d, 0x00, 0x8f, 0x7c, 0xe7, 0xa5, 0x0f, 0x1f, 0x7d, 0xda, 0x8f, 0x8f, 0xc4, - 0x51, 0xf8, 0xc8, 0x1f, 0x91, 0x2e, 0x40, 0xda, 0x8a, 0x06, 0x48, 0x77, 0xc6, 0x02, 0xa4, 0xa0, - 0x91, 0x0c, 0x20, 0xa4, 0x8d, 0x48, 0x84, 0x74, 0x7b, 0x1c, 0x42, 0xf2, 0x8d, 0x64, 0x10, 0x22, - 0x7d, 0xda, 0x0f, 0x91, 0xc4, 0x51, 0x10, 0xc9, 0xaf, 0x9c, 0x8b, 0x91, 0x6a, 0x51, 0x18, 0xe9, - 0xd6, 0x18, 0x8c, 0xe4, 0x37, 0x5e, 0x00, 0x24, 0x29, 0x23, 0x41, 0xd2, 0x7b, 0x13, 0x82, 0xa4, - 0x3e, 0xc3, 0x15, 0x46, 0x49, 0xb5, 0x28, 0x94, 0x74, 0x6b, 0x0c, 0x4a, 0x0a, 0x14, 0xd6, 0x87, - 0x49, 0x5b, 0xd1, 0x30, 0xe9, 0xce, 0x58, 0x98, 0xd4, 0x37, 0x9a, 0x5c, 0x9c, 0xb4, 0x11, 0x89, - 0x93, 0x6e, 0x8f, 0xc3, 0x49, 0x7d, 0x13, 0x1f, 0x77, 0x0e, 0x7e, 0x69, 0x72, 0xa0, 0xf4, 0xd1, - 0xf9, 0x81, 0x92, 0xf7, 0xce, 0x31, 0x48, 0xe9, 0xd3, 0x7e, 0xa4, 0x24, 0x8e, 0x42, 0x4a, 0x7e, - 0xcf, 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2e, 0xbe, 0x23, 0xfe, 0xb5, 0x59, 0x98, 0xa9, - 0x79, 0x21, 0x49, 0x7e, 0x2d, 0x85, 0x37, 0x39, 0x03, 0x0b, 0xad, 0x91, 0x11, 0x4f, 0xed, 0xe6, - 0xf8, 0x93, 0x13, 0x07, 0xcf, 0xf6, 0xe3, 0xac, 0x6f, 0xb0, 0x19, 0x1d, 0x7d, 0x00, 0xb9, 0x9e, - 0x8d, 0x2d, 0xb9, 0x6b, 0xe9, 0xa6, 0xa5, 0x3b, 0x6c, 0xc3, 0x8d, 0xb0, 0x5a, 0xfc, 0xf2, 0x74, - 0x71, 0x6e, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x73, 0xbd, 0xc0, 0x93, 0x7b, 0xcb, 0xd9, 0xf4, - 0xe4, 0xb7, 0x9c, 0xbd, 0x80, 0xa2, 0x85, 0x15, 0x2d, 0xe4, 0xc1, 0xb0, 0xf3, 0xa6, 0xa2, 0xfb, - 0x1c, 0xdd, 0x10, 0xe7, 0xe6, 0xa4, 0xe7, 0x4e, 0x15, 0xac, 0x30, 0x11, 0x3d, 0x80, 0x8b, 0x1d, - 0xe5, 0x98, 0x86, 0xaf, 0xca, 0xae, 0x53, 0x48, 0x43, 0x52, 0xd9, 0x05, 0x62, 0xa8, 0xa3, 0x1c, - 0xd3, 0x2b, 0xd3, 0x58, 0x12, 0xbd, 0xef, 0xe4, 0x16, 0xe4, 0x35, 0xdd, 0x76, 0x74, 0x43, 0x75, - 0xf8, 0x51, 0xc6, 0xec, 0x18, 0xe0, 0x9c, 0x4b, 0x65, 0xe7, 0x15, 0xdf, 0x87, 0x79, 0xbe, 0xaf, - 0x21, 0xf0, 0x31, 0x96, 0x1f, 0x07, 0xcc, 0x12, 0xbc, 0xef, 0xaf, 0xa8, 0x02, 0x85, 0x96, 0xe2, - 0xe0, 0x23, 0xe5, 0x44, 0x76, 0x77, 0xcd, 0x65, 0xe9, 0x09, 0x9f, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, - 0x9e, 0xb2, 0xa4, 0x81, 0xcd, 0x73, 0xb9, 0x56, 0x20, 0x41, 0x43, 0x77, 0xa0, 0xa0, 0xd8, 0x27, - 0x86, 0x4a, 0xd5, 0x83, 0x0d, 0xbb, 0x67, 0x53, 0x48, 0x92, 0x96, 0xf2, 0x94, 0x5c, 0x71, 0xa9, - 0xe8, 0x3a, 0xcc, 0xf1, 0xa0, 0x7f, 0x76, 0x07, 0x53, 0x81, 0x56, 0x95, 0x5f, 0xee, 0xc1, 0xae, - 0x61, 0x7a, 0x0c, 0x65, 0x7e, 0x55, 0xc1, 0x91, 0x62, 0x69, 0x32, 0xd5, 0xba, 0xdf, 0x3f, 0x8b, - 0x54, 0xec, 0x65, 0x76, 0x35, 0x01, 0xc9, 0x40, 0x54, 0xed, 0x8f, 0xa1, 0x2d, 0x98, 0x57, 0xdb, - 0xba, 0x87, 0x20, 0x58, 0xcd, 0xe7, 0x87, 0x8e, 0xa6, 0x0a, 0xcd, 0xeb, 0x7f, 0x8c, 0x2e, 0xa8, - 0x61, 0x02, 0x6a, 0x00, 0x3d, 0xd9, 0x47, 0xee, 0x9a, 0x6d, 0x5d, 0x3d, 0xa1, 0xe0, 0x21, 0x7c, - 0xe4, 0xfa, 0xc8, 0x8b, 0x0f, 0x5e, 0x29, 0xba, 0xb3, 0x4d, 0x39, 0x25, 0x38, 0xf2, 0x7e, 0xb3, - 0xe3, 0x92, 0xd7, 0x53, 0xe9, 0xb9, 0x62, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, 0x5f, 0x17, - 0xa0, 0xd0, 0x57, 0x16, 0x54, 0x83, 0x8b, 0x9a, 0x37, 0x54, 0x64, 0xbe, 0x2b, 0x4c, 0x37, 0x0d, - 0x7e, 0x8a, 0xfc, 0xc2, 0x97, 0xa7, 0x8b, 0x05, 0x9a, 0xfb, 0xa9, 0x97, 0x24, 0x5d, 0xf0, 0x39, - 0x7c, 0x2a, 0xfa, 0x08, 0xf2, 0xcc, 0xfd, 0xf4, 0xae, 0x15, 0xa4, 0xe1, 0xfc, 0xab, 0xf3, 0x5f, - 0x9e, 0x2e, 0xe6, 0xa8, 0xcf, 0xe9, 0x9e, 0xf2, 0x2c, 0xe5, 0xda, 0xc1, 0x47, 0xf1, 0xd7, 0x04, - 0x98, 0x0b, 0xed, 0xbc, 0x7a, 0xdc, 0xf7, 0xf1, 0xfb, 0x4a, 0x34, 0x6e, 0x1d, 0x16, 0xfb, 0x98, - 0xe6, 0xfd, 0xdc, 0x0d, 0x24, 0x5d, 0x1c, 0x8e, 0x7b, 0xe8, 0x2a, 0x8e, 0x1b, 0x55, 0xe3, 0xb2, - 0x7d, 0x9c, 0xfa, 0xfe, 0x0f, 0x16, 0xa7, 0xc4, 0x3f, 0x4b, 0x41, 0x2e, 0xbc, 0xcf, 0xaa, 0xde, - 0x57, 0xae, 0xa8, 0x79, 0x25, 0xc4, 0xb1, 0x34, 0xe2, 0xae, 0x99, 0x8c, 0x7f, 0x5d, 0x03, 0x2b, - 0xe6, 0xb5, 0x11, 0x9f, 0xf8, 0x83, 0xe5, 0xf4, 0x19, 0xcb, 0xff, 0x3e, 0xe9, 0xd9, 0xd7, 0x25, - 0x98, 0xa6, 0x47, 0x26, 0xf1, 0xa2, 0x95, 0xfa, 0x7b, 0x0f, 0xf1, 0xb5, 0x49, 0xba, 0xc4, 0xb2, - 0x11, 0x7b, 0xdc, 0x7c, 0xa3, 0x33, 0x09, 0xfd, 0x61, 0x70, 0xfe, 0x5b, 0x1c, 0x7b, 0xec, 0x4c, - 0xca, 0xff, 0x87, 0x91, 0x37, 0xe4, 0x7d, 0xe8, 0x17, 0xa0, 0xa0, 0x9a, 0xed, 0x36, 0x9b, 0x6b, - 0x99, 0x45, 0x1b, 0x3c, 0xa5, 0x86, 0x16, 0x81, 0x5f, 0xdc, 0xb9, 0xe4, 0x5d, 0xe0, 0xb9, 0x24, - 0xf1, 0x0b, 0x3c, 0x03, 0x61, 0xbd, 0x79, 0x4f, 0x18, 0x33, 0x84, 0x7d, 0x11, 0xc6, 0xb3, 0x6f, - 0x12, 0x61, 0xcc, 0xa2, 0xd2, 0x79, 0xcf, 0xfb, 0x43, 0x81, 0x87, 0xf0, 0x3c, 0x33, 0xcd, 0x83, - 0x9e, 0x17, 0x19, 0x5c, 0x0e, 0x9e, 0x30, 0x99, 0xfe, 0xf2, 0x74, 0x31, 0x25, 0x79, 0x47, 0x4c, - 0x46, 0xcd, 0x1c, 0x89, 0xaf, 0x36, 0x73, 0x5c, 0x87, 0xb9, 0xae, 0x85, 0xf7, 0xb0, 0xa3, 0xee, - 0xcb, 0x46, 0xaf, 0xc3, 0x37, 0x10, 0x65, 0x5d, 0xda, 0x56, 0xaf, 0x83, 0xee, 0x41, 0xd1, 0xcb, - 0xc2, 0x31, 0xbe, 0x7b, 0x04, 0x99, 0x4b, 0xe7, 0x2b, 0x02, 0xe2, 0xff, 0x12, 0x60, 0x21, 0x54, - 0x27, 0x3e, 0xa6, 0xd6, 0x21, 0xeb, 0x9b, 0x13, 0xbb, 0x24, 0x9c, 0x33, 0x4e, 0x36, 0xc8, 0x8c, - 0x64, 0xb8, 0xe4, 0xbe, 0x96, 0x5e, 0x71, 0xe0, 0x8b, 0x4d, 0x9c, 0x53, 0xec, 0x45, 0x5f, 0xce, - 0x5a, 0xe0, 0x05, 0xde, 0x20, 0x4b, 0x4e, 0x34, 0xc8, 0xc4, 0xdf, 0x14, 0xa0, 0x48, 0x5f, 0xf0, - 0x04, 0x63, 0x2d, 0x16, 0xeb, 0xe6, 0xc6, 0x9f, 0x27, 0x26, 0xdf, 0x22, 0x14, 0xba, 0x96, 0x25, - 0x19, 0xbe, 0x96, 0x45, 0xfc, 0x81, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x2a, 0xc4, 0x11, 0x07, 0x99, - 0xbe, 0xd9, 0x75, 0x7f, 0xee, 0x81, 0x2b, 0x13, 0xdd, 0xce, 0x18, 0x3c, 0x70, 0x85, 0x5d, 0x53, - 0xf7, 0x37, 0xdd, 0x9e, 0x43, 0x8a, 0x58, 0xf1, 0x0f, 0xd6, 0x78, 0x83, 0xdd, 0x52, 0x12, 0xbd, - 0x45, 0xd6, 0x6c, 0x1f, 0xb2, 0x33, 0x72, 0x26, 0x32, 0x7b, 0x88, 0x07, 0xae, 0x01, 0x5f, 0x0d, - 0xd4, 0x9a, 0x0d, 0x7a, 0xbf, 0x2c, 0xfb, 0x6d, 0x8b, 0x4f, 0x02, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, - 0x34, 0x91, 0x29, 0x76, 0xb5, 0xc4, 0xfa, 0xca, 0x1f, 0x04, 0x5b, 0xa2, 0x7a, 0x48, 0x50, 0xe0, - 0x23, 0x48, 0x1e, 0x2a, 0xed, 0x51, 0x11, 0x60, 0xa1, 0x96, 0x93, 0x48, 0x6e, 0xf4, 0x24, 0x74, - 0x1e, 0x49, 0x62, 0x38, 0x62, 0x19, 0x54, 0x69, 0xe8, 0xdc, 0x92, 0x0f, 0xc3, 0x7d, 0x7d, 0xe4, - 0xeb, 0x83, 0x9d, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, - 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0xcf, 0x76, 0xb9, 0xdd, 0x77, 0x4f, 0xcd, 0xf4, 0x6a, 0xf6, 0x2c, - 0xea, 0xf8, 0xa7, 0x8b, 0xb0, 0x10, 0xe2, 0x66, 0xc6, 0x42, 0xfc, 0x10, 0xae, 0x3c, 0x35, 0x6d, - 0x5b, 0xef, 0x12, 0xe8, 0x4b, 0x47, 0x25, 0x99, 0x5a, 0x3c, 0xf3, 0x98, 0xee, 0xd2, 0x45, 0x10, - 0x83, 0x99, 0x91, 0x8c, 0xe4, 0x3d, 0x8b, 0xbf, 0x2b, 0xc0, 0xe5, 0x41, 0x4e, 0xa6, 0xe5, 0xa8, - 0xcd, 0x9d, 0xb3, 0xaa, 0xe9, 0x9f, 0x05, 0x38, 0xbe, 0xb7, 0xba, 0xd9, 0x89, 0x8b, 0xca, 0xdf, - 0x29, 0x77, 0x14, 0x6a, 0x3e, 0xf8, 0x46, 0xf3, 0x3c, 0x27, 0x6f, 0x32, 0xaa, 0x6f, 0x49, 0x52, - 0x93, 0x59, 0x92, 0x26, 0x14, 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xb0, 0x5b, 0xdf, 0x15, 0xc8, 0xef, - 0xea, 0x86, 0x62, 0x9d, 0xc8, 0x6e, 0xe0, 0xa1, 0x30, 0x2e, 0xf0, 0x50, 0xca, 0x31, 0x0e, 0xfe, - 0x28, 0xfe, 0x48, 0x80, 0xa2, 0x2f, 0x96, 0x5b, 0xe4, 0x77, 0x01, 0xd4, 0x76, 0xcf, 0x76, 0xb0, - 0xe5, 0xb6, 0xd2, 0x1c, 0x0b, 0xd4, 0xaf, 0x30, 0x6a, 0x7d, 0x4d, 0xca, 0xf0, 0x0c, 0x75, 0x0d, - 0xdd, 0x08, 0x9f, 0x83, 0x31, 0xbd, 0x0a, 0x67, 0x03, 0xa7, 0x5f, 0x90, 0x66, 0xb7, 0x1d, 0xd3, - 0xf2, 0x30, 0x17, 0x6f, 0x76, 0xf7, 0x64, 0x7f, 0xba, 0x7d, 0x9b, 0xe4, 0x5b, 0x81, 0x3c, 0x71, - 0x17, 0x0e, 0xb1, 0x57, 0xa5, 0xd4, 0xf8, 0x2a, 0x31, 0x0e, 0xb7, 0x4a, 0x7f, 0x8f, 0x38, 0xb9, - 0xac, 0x35, 0xbc, 0x16, 0x1e, 0x61, 0xd1, 0x7e, 0x3a, 0x78, 0x7c, 0xe5, 0x64, 0x07, 0x7c, 0x52, - 0x4f, 0xe6, 0x9b, 0x90, 0x76, 0xef, 0xe7, 0xe6, 0x03, 0xe4, 0xca, 0x12, 0xbb, 0xc0, 0x7b, 0xc9, - 0xbd, 0xc0, 0x7b, 0x69, 0x8d, 0x67, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x65, 0x51, 0x90, 0x3c, 0xa6, - 0xfb, 0x0d, 0xd2, 0xc3, 0x07, 0x66, 0x61, 0x94, 0x07, 0x08, 0x5c, 0x69, 0xc4, 0xaf, 0x6c, 0x5e, - 0x59, 0x93, 0x77, 0xb6, 0x2a, 0xcf, 0x37, 0x37, 0xeb, 0xcd, 0x66, 0x75, 0xad, 0x28, 0xa0, 0x22, - 0xcc, 0x85, 0x2e, 0x44, 0x4a, 0xb0, 0x4b, 0x9c, 0xef, 0xff, 0x34, 0x80, 0x7f, 0xb7, 0x1a, 0x91, - 0xb5, 0x51, 0xfd, 0x4c, 0x7e, 0xb9, 0xf2, 0x6c, 0xa7, 0xda, 0x28, 0x4e, 0x21, 0x04, 0xf9, 0xd5, - 0x95, 0x66, 0xa5, 0x26, 0x4b, 0xd5, 0xc6, 0xf6, 0xf3, 0xad, 0x46, 0xd5, 0xbd, 0xfc, 0xf9, 0xfe, - 0x1a, 0xcc, 0x05, 0x4f, 0x36, 0x42, 0x0b, 0x50, 0xa8, 0xd4, 0xaa, 0x95, 0x0d, 0xf9, 0x65, 0x7d, - 0x45, 0x7e, 0xb1, 0x53, 0xdd, 0xa9, 0x16, 0xa7, 0x68, 0xd1, 0x28, 0xf1, 0xc9, 0xce, 0xb3, 0x67, - 0x45, 0x01, 0x15, 0x20, 0xcb, 0x9e, 0xe9, 0xe5, 0x49, 0xc5, 0xc4, 0xfd, 0x4d, 0xc8, 0x06, 0x8e, - 0x40, 0x26, 0xaf, 0xdb, 0xde, 0x69, 0xd4, 0xe4, 0x66, 0x7d, 0xb3, 0xda, 0x68, 0xae, 0x6c, 0x6e, - 0x33, 0x19, 0x94, 0xb6, 0xb2, 0xfa, 0x5c, 0x6a, 0x16, 0x05, 0xef, 0xb9, 0xf9, 0x7c, 0xa7, 0x52, - 0x73, 0xab, 0x21, 0xa6, 0xd2, 0xc9, 0x62, 0xf2, 0xfe, 0x5f, 0x16, 0xe0, 0xf2, 0x90, 0x53, 0x7e, - 0x50, 0x16, 0x66, 0x77, 0x0c, 0x7a, 0x1e, 0x6c, 0x71, 0x0a, 0xe5, 0x02, 0x07, 0xfd, 0x14, 0x05, - 0x94, 0x66, 0x47, 0xad, 0x14, 0x13, 0x68, 0x06, 0x12, 0x8d, 0x47, 0xc5, 0x24, 0x29, 0x69, 0xe0, - 0x9c, 0x9c, 0x62, 0x0a, 0x65, 0xf8, 0x09, 0x1d, 0xc5, 0x69, 0x34, 0xe7, 0x1f, 0x94, 0x51, 0x9c, - 0x21, 0xa2, 0xbc, 0x03, 0x27, 0x8a, 0xb3, 0xf7, 0xaf, 0x43, 0x60, 0x53, 0x3f, 0x02, 0x98, 0x79, - 0xa6, 0x38, 0xd8, 0x76, 0x8a, 0x53, 0x68, 0x16, 0x92, 0x2b, 0xed, 0x76, 0x51, 0x78, 0xf8, 0x2f, - 0x52, 0x90, 0x76, 0x2f, 0x09, 0x42, 0xcf, 0x60, 0x9a, 0x2d, 0x7d, 0x2f, 0x0e, 0x47, 0x06, 0x74, - 0xf0, 0x96, 0xaf, 0x8d, 0x83, 0x0e, 0xe2, 0x14, 0xfa, 0x8b, 0x90, 0x0d, 0x78, 0x4c, 0x68, 0xe8, - 0xf2, 0x5d, 0xc8, 0x4b, 0x2c, 0xdf, 0x1e, 0x97, 0xcd, 0x93, 0xff, 0x0a, 0x32, 0x9e, 0x05, 0x47, - 0x37, 0x46, 0xd9, 0x77, 0x57, 0xf6, 0xe8, 0x49, 0x80, 0x8c, 0x35, 0x71, 0xea, 0x7d, 0x01, 0x59, - 0x80, 0x06, 0x8d, 0x2d, 0x8a, 0x0a, 0xa8, 0x18, 0x6a, 0xcd, 0xcb, 0xf7, 0x27, 0xca, 0xed, 0xbf, - 0x93, 0x28, 0xcb, 0x9f, 0x31, 0xa2, 0x95, 0x35, 0x30, 0x1f, 0x45, 0x2b, 0x2b, 0x62, 0xe2, 0x99, - 0x42, 0x2f, 0x20, 0x45, 0x2c, 0x25, 0x8a, 0xf2, 0x21, 0xfb, 0x2c, 0x73, 0xf9, 0xc6, 0xc8, 0x3c, - 0xae, 0xc8, 0xd5, 0x7b, 0x3f, 0xfc, 0x93, 0xab, 0x53, 0x3f, 0x3c, 0xbb, 0x2a, 0xfc, 0xe8, 0xec, - 0xaa, 0xf0, 0x47, 0x67, 0x57, 0x85, 0x3f, 0x3e, 0xbb, 0x2a, 0x7c, 0xef, 0xc7, 0x57, 0xa7, 0x7e, - 0xf4, 0xe3, 0xab, 0x53, 0x7f, 0xf4, 0xe3, 0xab, 0x53, 0x9f, 0xcf, 0x72, 0xee, 0xdd, 0x19, 0x6a, - 0x54, 0x1e, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x04, 0x95, 0x46, 0x50, 0x81, 0x00, - 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e9d594aefb977c8a) } + +var fileDescriptor_api_e9d594aefb977c8a = []byte{ + // 8267 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x6b, 0x6c, 0x23, 0xc9, + 0x76, 0x1f, 0xae, 0x26, 0x29, 0x89, 0x3c, 0x14, 0x1f, 0x2a, 0xcd, 0x83, 0xc3, 0xdd, 0x1d, 0xcd, + 0xf4, 0xbc, 0xe7, 0xee, 0x4a, 0x3b, 0x33, 0x77, 0xff, 0xbb, 0xde, 0x59, 0xef, 0xb5, 0x44, 0x71, + 0x86, 0x94, 0x46, 0x1a, 0x4d, 0x93, 0x9a, 0xf9, 0xef, 0xfa, 0x3a, 0x7d, 0x5b, 0xdd, 0x25, 0xaa, + 0xaf, 0xc8, 0x6e, 0x4e, 0x77, 0x53, 0x8f, 0x05, 0x02, 0xc4, 0xb1, 0xe1, 0x5c, 0x20, 0x80, 0x71, + 0x3f, 0x38, 0xc8, 0x35, 0x9c, 0xc4, 0xd7, 0x71, 0x10, 0x7f, 0x08, 0x90, 0x04, 0x48, 0x90, 0x17, + 0x12, 0xfb, 0x8b, 0x81, 0x5c, 0x04, 0x4e, 0x72, 0xfd, 0x29, 0x46, 0x80, 0x28, 0xb6, 0x6e, 0xbe, + 0x04, 0x09, 0x8c, 0x20, 0x08, 0x60, 0x60, 0x3f, 0x04, 0x41, 0x3d, 0xfa, 0x45, 0x36, 0x1f, 0x9a, + 0xed, 0x9b, 0x2c, 0xe0, 0x2f, 0x12, 0xfb, 0x54, 0x9d, 0xd3, 0x55, 0xa7, 0xaa, 0x4e, 0x9d, 0x5f, + 0xf5, 0xa9, 0x2a, 0x98, 0xb7, 0x4c, 0x45, 0xdd, 0xef, 0xee, 0x2e, 0x2b, 0x5d, 0x7d, 0xa9, 0x6b, + 0x99, 0x8e, 0x89, 0xe6, 0x55, 0x53, 0x3d, 0xa0, 0xe4, 0x25, 0x9e, 0x58, 0xbe, 0x7f, 0x70, 0xb8, + 0x7c, 0x70, 0x68, 0x63, 0xeb, 0x10, 0x5b, 0xcb, 0xaa, 0x69, 0xa8, 0x3d, 0xcb, 0xc2, 0x86, 0x7a, + 0xb2, 0xdc, 0x36, 0xd5, 0x03, 0xfa, 0x47, 0x37, 0x5a, 0x8c, 0xbd, 0x8c, 0x5c, 0x89, 0x9a, 0xe2, + 0x28, 0x9c, 0x76, 0xc1, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0x2f, 0xb9, 0xd4, 0x0e, 0x76, + 0x94, 0x40, 0xee, 0xb7, 0x6c, 0xc7, 0xb4, 0x94, 0x16, 0x5e, 0xc6, 0x46, 0x4b, 0x37, 0x30, 0xc9, + 0x70, 0xa8, 0xaa, 0x3c, 0xf1, 0xed, 0xc8, 0xc4, 0x47, 0x3c, 0xb5, 0xd4, 0x73, 0xf4, 0xf6, 0xf2, + 0x7e, 0x5b, 0x5d, 0x76, 0xf4, 0x0e, 0xb6, 0x1d, 0xa5, 0xd3, 0xe5, 0x29, 0xf7, 0x69, 0x8a, 0x63, + 0x29, 0xaa, 0x6e, 0xb4, 0xdc, 0xff, 0xdd, 0xdd, 0x65, 0x0b, 0xab, 0xa6, 0xa5, 0x61, 0x4d, 0xb6, + 0xbb, 0x8a, 0xe1, 0x16, 0xb7, 0x65, 0xb6, 0x4c, 0xfa, 0x73, 0x99, 0xfc, 0xe2, 0xd4, 0xab, 0x2d, + 0xd3, 0x6c, 0xb5, 0xf1, 0x32, 0x7d, 0xda, 0xed, 0xed, 0x2d, 0x6b, 0x3d, 0x4b, 0x71, 0x74, 0x93, + 0x73, 0x89, 0xff, 0x44, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb0, 0xed, 0xd4, 0xb0, 0xa2, 0x61, 0x0b, + 0x5d, 0x81, 0xe4, 0x01, 0x3e, 0x29, 0x25, 0xaf, 0x09, 0x77, 0xe7, 0x56, 0x67, 0xbf, 0x3c, 0x5d, + 0x4c, 0x6e, 0xe0, 0x13, 0x89, 0xd0, 0xd0, 0x35, 0x98, 0xc5, 0x86, 0x26, 0x93, 0xe4, 0x54, 0x38, + 0x79, 0x06, 0x1b, 0xda, 0x06, 0x3e, 0x41, 0xdf, 0x86, 0xb4, 0x4d, 0xa4, 0x19, 0x2a, 0x2e, 0x4d, + 0x5f, 0x13, 0xee, 0x4e, 0xaf, 0xfe, 0xdc, 0x97, 0xa7, 0x8b, 0x9f, 0xb4, 0x74, 0x67, 0xbf, 0xb7, + 0xbb, 0xa4, 0x9a, 0x9d, 0x65, 0xaf, 0x9d, 0xb4, 0x5d, 0xff, 0xf7, 0x72, 0xf7, 0xa0, 0xb5, 0xdc, + 0xaf, 0xa3, 0xa5, 0xe6, 0xb1, 0xd1, 0xc0, 0xaf, 0x25, 0x4f, 0xe2, 0x7a, 0x2a, 0x2d, 0x14, 0x13, + 0xeb, 0xa9, 0x74, 0xa2, 0x98, 0x14, 0x7f, 0x3b, 0x09, 0x79, 0x09, 0xdb, 0x5d, 0xd3, 0xb0, 0x31, + 0x2f, 0xf9, 0xfb, 0x90, 0x74, 0x8e, 0x0d, 0x5a, 0xf2, 0xec, 0xc3, 0xab, 0x4b, 0x03, 0x3d, 0x62, + 0xa9, 0x69, 0x29, 0x86, 0xad, 0xa8, 0xa4, 0xfa, 0x12, 0xc9, 0x8a, 0x3e, 0x82, 0xac, 0x85, 0xed, + 0x5e, 0x07, 0x53, 0x45, 0xd2, 0x4a, 0x65, 0x1f, 0x5e, 0x8e, 0xe0, 0x6c, 0x74, 0x15, 0x43, 0x02, + 0x96, 0x97, 0xfc, 0x46, 0x57, 0x20, 0x6d, 0xf4, 0x3a, 0x44, 0x15, 0x36, 0xad, 0x68, 0x52, 0x9a, + 0x35, 0x7a, 0x9d, 0x0d, 0x7c, 0x62, 0xa3, 0xff, 0x1f, 0x2e, 0x69, 0xb8, 0x6b, 0x61, 0x55, 0x71, + 0xb0, 0x26, 0x5b, 0x8a, 0xd1, 0xc2, 0xb2, 0x6e, 0xec, 0x99, 0x76, 0x69, 0xe6, 0x5a, 0xf2, 0x6e, + 0xf6, 0xe1, 0xdb, 0x11, 0xf2, 0x25, 0x92, 0xab, 0x6e, 0xec, 0x99, 0xab, 0xa9, 0x1f, 0x9d, 0x2e, + 0x4e, 0x49, 0x17, 0x7c, 0x09, 0x5e, 0x92, 0x8d, 0x1a, 0x90, 0xe3, 0xc5, 0xb5, 0xb0, 0x62, 0x9b, + 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, 0xff, 0x70, 0x29, 0x4a, 0x60, 0x48, 0x35, 0xe4, 0xb1, 0xd7, + 0xc1, 0x12, 0xe5, 0x92, 0xe6, 0xac, 0xc0, 0x13, 0x7a, 0x0b, 0x32, 0xa4, 0x26, 0xbb, 0x27, 0x0e, + 0xb6, 0x4b, 0x69, 0x5a, 0x15, 0x52, 0xb5, 0x55, 0xf2, 0x2c, 0x7e, 0x0a, 0x73, 0x41, 0x56, 0x84, + 0x20, 0x2f, 0x55, 0x1b, 0x3b, 0x9b, 0x55, 0x79, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0xad, 0xe2, + 0x14, 0xba, 0x00, 0x45, 0x4e, 0xdb, 0xa8, 0x7e, 0x26, 0x3f, 0xab, 0x6f, 0xd6, 0x9b, 0x45, 0xa1, + 0x9c, 0xfa, 0xde, 0x6f, 0x5f, 0x9d, 0x12, 0xb7, 0x01, 0x9e, 0x62, 0x87, 0x77, 0x30, 0xb4, 0x0a, + 0x33, 0xfb, 0xb4, 0x3c, 0x25, 0x81, 0x6a, 0xfa, 0x5a, 0x64, 0xc1, 0x03, 0x9d, 0x71, 0x35, 0x4d, + 0xb4, 0xf1, 0xe3, 0xd3, 0x45, 0x41, 0xe2, 0x9c, 0xe2, 0xef, 0x09, 0x90, 0xa5, 0x22, 0x59, 0xfd, + 0x50, 0xa5, 0x4f, 0xe6, 0xf5, 0xb1, 0xca, 0x18, 0x14, 0x8a, 0x96, 0x60, 0xfa, 0x50, 0x69, 0xf7, + 0x70, 0x29, 0x41, 0x65, 0x94, 0x22, 0x64, 0xbc, 0x24, 0xe9, 0x12, 0xcb, 0x86, 0x1e, 0xc3, 0x9c, + 0x6e, 0x38, 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x1c, 0xc3, 0x96, 0x65, 0xb9, 0xe9, 0x83, 0xf8, 0x8f, + 0x05, 0x80, 0xed, 0x5e, 0x9c, 0x4a, 0x41, 0xdf, 0x9c, 0xb0, 0xfc, 0xbc, 0x77, 0xf1, 0x5a, 0x5c, + 0x82, 0x19, 0xdd, 0x68, 0xeb, 0x06, 0x2b, 0x7f, 0x5a, 0xe2, 0x4f, 0xe8, 0x02, 0x4c, 0xef, 0xb6, + 0x75, 0x43, 0xa3, 0xe3, 0x21, 0x2d, 0xb1, 0x07, 0x51, 0x82, 0x2c, 0x2d, 0x75, 0x8c, 0x7a, 0x17, + 0x4f, 0x13, 0x70, 0xb1, 0x62, 0x1a, 0x9a, 0x4e, 0x86, 0xa4, 0xd2, 0xfe, 0x5a, 0x68, 0x65, 0x1d, + 0x02, 0x83, 0x4f, 0xc6, 0xc7, 0xdd, 0x09, 0xdb, 0x18, 0xf9, 0x5c, 0xd5, 0xe3, 0x2e, 0xa5, 0x45, + 0x6b, 0x12, 0x7d, 0x13, 0x2e, 0x2b, 0xed, 0xb6, 0x79, 0x24, 0xeb, 0x7b, 0xb2, 0x66, 0x62, 0x5b, + 0x36, 0x4c, 0x47, 0xc6, 0xc7, 0xba, 0xed, 0x50, 0x53, 0x92, 0x96, 0x16, 0x68, 0x72, 0x7d, 0x6f, + 0xcd, 0xc4, 0xf6, 0x96, 0xe9, 0x54, 0x49, 0x12, 0x19, 0xa7, 0xa4, 0x30, 0x6c, 0x9c, 0xce, 0x10, + 0xf3, 0x2b, 0xa5, 0xf1, 0x71, 0x97, 0x8e, 0xd3, 0x40, 0x53, 0xce, 0x06, 0x9b, 0x52, 0xfc, 0x05, + 0xb8, 0xd4, 0xaf, 0xdf, 0x38, 0xdb, 0xef, 0x0f, 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x34, + 0x9c, 0xa7, 0xec, 0x64, 0x50, 0xd9, 0xf7, 0xa1, 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, + 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0xb7, 0xc6, 0x00, 0x5d, 0x7c, 0x09, 0x05, 0xaf, 0x36, 0x71, + 0xaa, 0xc9, 0x81, 0x62, 0xdd, 0x50, 0x2d, 0xdc, 0xc1, 0x46, 0xac, 0x7a, 0x7a, 0x1b, 0x32, 0xba, + 0x2b, 0x97, 0xea, 0x2a, 0x29, 0xf9, 0x04, 0xb1, 0x07, 0xf3, 0x81, 0xb7, 0xc6, 0x69, 0x2e, 0xc9, + 0x94, 0x81, 0x8f, 0x64, 0xbf, 0x8d, 0xc8, 0x94, 0x81, 0x8f, 0x98, 0x79, 0x6b, 0x40, 0x6e, 0x0d, + 0xb7, 0xb1, 0x83, 0xe3, 0xb4, 0xfa, 0x3b, 0x90, 0x77, 0x85, 0xc6, 0xd9, 0x30, 0x7f, 0x5d, 0x00, + 0xc4, 0xe5, 0x92, 0x59, 0x36, 0xce, 0xb6, 0x59, 0x24, 0xae, 0x85, 0xd3, 0xb3, 0x0c, 0xe6, 0x23, + 0xb0, 0x3e, 0x09, 0x8c, 0x44, 0xdd, 0x04, 0x7f, 0xc8, 0xa6, 0x82, 0x43, 0x96, 0xbb, 0x37, 0x47, + 0xb0, 0x10, 0x2a, 0x58, 0xbc, 0xcd, 0x97, 0xa2, 0x65, 0x4a, 0x5c, 0x4b, 0x06, 0x7d, 0x38, 0x4a, + 0x14, 0x7f, 0x20, 0xc0, 0x7c, 0xa5, 0x8d, 0x15, 0x2b, 0x76, 0x8d, 0x7c, 0x0b, 0xd2, 0x1a, 0x56, + 0x34, 0x5a, 0x65, 0x36, 0xb0, 0xdf, 0x09, 0x48, 0x21, 0x9e, 0xee, 0xd2, 0x7e, 0x5b, 0x5d, 0x6a, + 0xba, 0x3e, 0x30, 0x1f, 0xdd, 0x1e, 0x93, 0xf8, 0x19, 0xa0, 0x60, 0xc9, 0xe2, 0xec, 0x08, 0xff, + 0x4b, 0x00, 0x24, 0xe1, 0x43, 0x6c, 0x39, 0xb1, 0x57, 0x7b, 0x0d, 0xb2, 0x8e, 0x62, 0xb5, 0xb0, + 0x23, 0x13, 0xef, 0xfe, 0x3c, 0x35, 0x07, 0xc6, 0x47, 0xc8, 0xa8, 0x09, 0x77, 0xb0, 0xa1, 0xec, + 0xb6, 0x31, 0x95, 0x22, 0xef, 0x9a, 0x3d, 0x43, 0x93, 0x75, 0x07, 0x5b, 0x8a, 0x63, 0x5a, 0xb2, + 0xd9, 0x75, 0xf4, 0x8e, 0xfe, 0x05, 0x75, 0xec, 0x79, 0x57, 0xbb, 0xc1, 0xb2, 0x13, 0xe6, 0x55, + 0x92, 0xb9, 0xce, 0xf3, 0x3e, 0x0f, 0x64, 0x15, 0x3f, 0x87, 0x85, 0x50, 0xad, 0xe3, 0x54, 0xe9, + 0xff, 0x10, 0x20, 0xdb, 0x50, 0x15, 0x23, 0x4e, 0x5d, 0x7e, 0x0a, 0x59, 0x5b, 0x55, 0x0c, 0x79, + 0xcf, 0xb4, 0x3a, 0x8a, 0x43, 0x07, 0x4e, 0x3e, 0xa4, 0x4b, 0xcf, 0x5f, 0x57, 0x15, 0xe3, 0x09, + 0xcd, 0x24, 0x81, 0xed, 0xfd, 0x46, 0x2f, 0x20, 0x7b, 0x80, 0x4f, 0x64, 0x8e, 0xfd, 0xe8, 0x6c, + 0x9b, 0x7f, 0xf8, 0x7e, 0x80, 0xff, 0xe0, 0x70, 0xc9, 0x85, 0x8c, 0x4b, 0x01, 0xc8, 0xb8, 0x44, + 0x38, 0x96, 0x1a, 0x8e, 0x85, 0x8d, 0x96, 0xb3, 0x2f, 0xc1, 0x01, 0x3e, 0x79, 0xc6, 0x64, 0xb0, + 0xe1, 0xba, 0x9e, 0x4a, 0x27, 0x8b, 0x29, 0xf1, 0xcf, 0x04, 0x98, 0x63, 0x55, 0x8e, 0x73, 0xb8, + 0x7e, 0x00, 0x29, 0xcb, 0x3c, 0x62, 0xc3, 0x35, 0xfb, 0xf0, 0xad, 0x08, 0x11, 0x1b, 0xf8, 0x24, + 0x38, 0x1f, 0xd2, 0xec, 0x68, 0x15, 0xb8, 0xd7, 0x29, 0x53, 0xee, 0xe4, 0xa4, 0xdc, 0xc0, 0xb8, + 0x24, 0x22, 0xe3, 0x0e, 0x14, 0x76, 0x15, 0x47, 0xdd, 0x97, 0x2d, 0x5e, 0x48, 0x32, 0x77, 0x26, + 0xef, 0xce, 0x49, 0x79, 0x4a, 0x76, 0x8b, 0x6e, 0x93, 0x9a, 0xb3, 0xf1, 0x63, 0xe3, 0x3f, 0x67, + 0x6d, 0xfe, 0xbf, 0x05, 0x3e, 0x86, 0xdc, 0x9a, 0xff, 0x79, 0x6b, 0xfa, 0x5f, 0x4f, 0xc0, 0xe5, + 0xca, 0x3e, 0x56, 0x0f, 0x2a, 0xa6, 0x61, 0xeb, 0xb6, 0x43, 0x74, 0x17, 0x67, 0xfb, 0xbf, 0x05, + 0x99, 0x23, 0xdd, 0xd9, 0x97, 0x35, 0x7d, 0x6f, 0x8f, 0x5a, 0xcf, 0xb4, 0x94, 0x26, 0x84, 0x35, + 0x7d, 0x6f, 0x0f, 0x3d, 0x82, 0x54, 0xc7, 0xd4, 0x98, 0x73, 0x9e, 0x7f, 0xb8, 0x18, 0x21, 0x9e, + 0x16, 0xcd, 0xee, 0x75, 0x36, 0x4d, 0x0d, 0x4b, 0x34, 0x33, 0xba, 0x0a, 0xa0, 0x12, 0x6a, 0xd7, + 0xd4, 0x0d, 0x87, 0xcf, 0xbe, 0x01, 0x0a, 0xaa, 0x41, 0xc6, 0xc1, 0x56, 0x47, 0x37, 0x14, 0x07, + 0x97, 0xa6, 0xa9, 0xf2, 0x6e, 0x46, 0x16, 0xbc, 0xdb, 0xd6, 0x55, 0x65, 0x0d, 0xdb, 0xaa, 0xa5, + 0x77, 0x1d, 0xd3, 0xe2, 0x5a, 0xf4, 0x99, 0xc5, 0x5f, 0x4d, 0x41, 0x69, 0x50, 0x37, 0x71, 0xf6, + 0x90, 0x6d, 0x98, 0x21, 0x68, 0xbe, 0xed, 0xf0, 0x3e, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x4a, 0x40, + 0x57, 0x05, 0xda, 0x0e, 0x2f, 0x36, 0x97, 0x53, 0xfe, 0x97, 0x02, 0xcc, 0xb0, 0x04, 0xf4, 0x00, + 0xd2, 0x7c, 0xf9, 0x42, 0xa3, 0x65, 0x4c, 0xae, 0x5e, 0x3a, 0x3b, 0x5d, 0x9c, 0x65, 0x2b, 0x12, + 0x6b, 0x5f, 0xfa, 0x3f, 0xa5, 0x59, 0x9a, 0xaf, 0xae, 0x91, 0xd6, 0xb2, 0x1d, 0xc5, 0x72, 0xe8, + 0x22, 0x51, 0x82, 0xa1, 0x14, 0x4a, 0xd8, 0xc0, 0x27, 0x68, 0x1d, 0x66, 0x6c, 0x47, 0x71, 0x7a, + 0x36, 0x6f, 0xaf, 0x73, 0x15, 0xb6, 0x41, 0x39, 0x25, 0x2e, 0x81, 0xb8, 0x4f, 0x1a, 0x76, 0x14, + 0xbd, 0x4d, 0x1b, 0x30, 0x23, 0xf1, 0x27, 0xf1, 0x37, 0x04, 0x98, 0x61, 0x59, 0xd1, 0x65, 0x58, + 0x90, 0x56, 0xb6, 0x9e, 0x56, 0xe5, 0xfa, 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, 0x2b, + 0xcd, 0x6a, 0x71, 0x0a, 0x5d, 0x02, 0xe4, 0x26, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, + 0xab, 0x59, 0x14, 0xe8, 0x4a, 0x06, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0xae, 0xf5, 0x53, 0xe5, + 0x46, 0x73, 0xa5, 0xd9, 0x90, 0xab, 0x8d, 0x66, 0x7d, 0x73, 0xa5, 0x59, 0x5d, 0x2b, 0x26, 0x47, + 0xe4, 0x22, 0x2f, 0x91, 0xa4, 0x6a, 0xa5, 0x59, 0x4c, 0x89, 0x0e, 0x5c, 0x94, 0xb0, 0x6a, 0x76, + 0xba, 0x3d, 0x07, 0x93, 0x52, 0xda, 0x71, 0x8e, 0x94, 0xcb, 0x30, 0xab, 0x59, 0x27, 0xb2, 0xd5, + 0x33, 0xf8, 0x38, 0x99, 0xd1, 0xac, 0x13, 0xa9, 0x67, 0x88, 0xff, 0x50, 0x80, 0x4b, 0xfd, 0xaf, + 0x8d, 0xb3, 0x13, 0xbe, 0x80, 0xac, 0xa2, 0x69, 0x58, 0x93, 0x35, 0xdc, 0x76, 0x14, 0xee, 0xe2, + 0xdc, 0x0f, 0x48, 0xe2, 0x4b, 0x7b, 0x4b, 0xde, 0xd2, 0xde, 0xe6, 0xcb, 0x4a, 0x85, 0x16, 0x64, + 0x8d, 0x70, 0xb8, 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xf1, 0xbf, 0xa7, 0x20, 0x57, 0x35, 0xb4, 0xe6, + 0x71, 0xac, 0x73, 0xc9, 0x25, 0x98, 0x51, 0xcd, 0x4e, 0x47, 0x77, 0x5c, 0x05, 0xb1, 0x27, 0xf4, + 0x33, 0x01, 0xd7, 0x34, 0x39, 0x81, 0x83, 0xe6, 0x3b, 0xa5, 0xe8, 0x3b, 0x70, 0x99, 0x58, 0x4d, + 0xcb, 0x50, 0xda, 0x32, 0x93, 0x26, 0x3b, 0x96, 0xde, 0x6a, 0x61, 0x8b, 0x2f, 0x27, 0xde, 0x8d, + 0x28, 0x67, 0x9d, 0x73, 0x54, 0x28, 0x43, 0x93, 0xe5, 0x97, 0x2e, 0xea, 0x51, 0x64, 0xf4, 0x09, + 0x00, 0x99, 0x8a, 0xe8, 0x12, 0xa5, 0xcd, 0xed, 0xd1, 0xb0, 0x35, 0x4a, 0xd7, 0x04, 0x11, 0x06, + 0xf2, 0x6c, 0xa3, 0x65, 0x82, 0x43, 0x5e, 0xf7, 0x74, 0x0b, 0xcb, 0x0f, 0xba, 0x2a, 0x5d, 0x38, + 0x48, 0xaf, 0xe6, 0xcf, 0x4e, 0x17, 0x41, 0x62, 0xe4, 0x07, 0xdb, 0x15, 0x82, 0x4b, 0xd8, 0xef, + 0xae, 0x8a, 0x5e, 0xc1, 0xbd, 0xc0, 0xfa, 0x07, 0x99, 0x79, 0x79, 0xb5, 0x14, 0x47, 0xde, 0xd7, + 0x5b, 0xfb, 0xd8, 0x92, 0xbd, 0x65, 0x6a, 0xba, 0x5e, 0x98, 0x96, 0x6e, 0xfa, 0x0c, 0x15, 0xc5, + 0x60, 0xa5, 0x5f, 0x71, 0x6a, 0x34, 0xb3, 0xa7, 0x33, 0xa2, 0xfc, 0xae, 0xa9, 0xdb, 0xa6, 0x51, + 0xca, 0x30, 0xe5, 0xb3, 0x27, 0x74, 0x0f, 0x8a, 0xce, 0xb1, 0x21, 0xef, 0x63, 0xc5, 0x72, 0x76, + 0xb1, 0xe2, 0x90, 0x59, 0x1a, 0x68, 0x8e, 0x82, 0x73, 0x6c, 0xd4, 0x02, 0x64, 0xf4, 0x02, 0x8a, + 0xba, 0x21, 0xef, 0xb5, 0xf5, 0xd6, 0xbe, 0x23, 0x1f, 0x59, 0xba, 0x83, 0xed, 0xd2, 0x3c, 0x55, + 0x48, 0x54, 0xbf, 0x6d, 0xf0, 0x75, 0x63, 0xed, 0x15, 0xc9, 0xc9, 0x55, 0x93, 0xd7, 0x8d, 0x27, + 0x94, 0x9f, 0x12, 0xed, 0xf5, 0x54, 0x7a, 0xb6, 0x98, 0x16, 0xff, 0x93, 0x00, 0x79, 0xb7, 0xbb, + 0xc5, 0x39, 0x32, 0xee, 0x42, 0xd1, 0x34, 0xb0, 0xdc, 0xdd, 0x57, 0x6c, 0xcc, 0xf5, 0xc8, 0x27, + 0x9c, 0xbc, 0x69, 0xe0, 0x6d, 0x42, 0x66, 0xea, 0x42, 0xdb, 0x30, 0x6f, 0x3b, 0x4a, 0x4b, 0x37, + 0x5a, 0x01, 0xf5, 0x4e, 0x4f, 0x0e, 0x16, 0x8a, 0x9c, 0xdb, 0xa3, 0x87, 0xbc, 0x94, 0x3f, 0x14, + 0x60, 0x7e, 0x45, 0xeb, 0xe8, 0x46, 0xa3, 0xdb, 0xd6, 0x63, 0x5d, 0x83, 0xb8, 0x09, 0x19, 0x9b, + 0xc8, 0xf4, 0x0d, 0xbe, 0x8f, 0x28, 0xd3, 0x34, 0x85, 0x58, 0xfe, 0x67, 0x50, 0xc0, 0xc7, 0x5d, + 0x9d, 0x7d, 0x7a, 0x60, 0x40, 0x28, 0x35, 0x79, 0xdd, 0xf2, 0x3e, 0x2f, 0x49, 0xe2, 0x75, 0xfa, + 0x0c, 0x50, 0xb0, 0x4a, 0x71, 0x62, 0x97, 0xcf, 0x60, 0x81, 0x8a, 0xde, 0x31, 0xec, 0x98, 0xf5, + 0x25, 0xfe, 0x3c, 0x5c, 0x08, 0x8b, 0x8e, 0xb3, 0xdc, 0xaf, 0x78, 0x2b, 0x6f, 0x62, 0x2b, 0x56, + 0x10, 0xeb, 0xe9, 0x9a, 0x0b, 0x8e, 0xb3, 0xcc, 0xbf, 0x2c, 0xc0, 0x15, 0x2a, 0x9b, 0x7e, 0x9d, + 0xd9, 0xc3, 0xd6, 0x33, 0xac, 0xd8, 0xb1, 0x22, 0xf0, 0x1b, 0x30, 0xc3, 0x90, 0x34, 0xed, 0x9f, + 0xd3, 0xab, 0x59, 0xe2, 0xb9, 0x34, 0x1c, 0xd3, 0x22, 0x9e, 0x0b, 0x4f, 0x12, 0x15, 0x28, 0x47, + 0x95, 0x22, 0xce, 0x9a, 0xfe, 0x2d, 0x01, 0xe6, 0xb9, 0xd3, 0x48, 0xba, 0x72, 0x65, 0x9f, 0xf8, + 0x4c, 0xa8, 0x0a, 0x59, 0x95, 0xfe, 0x92, 0x9d, 0x93, 0x2e, 0xa6, 0xf2, 0xf3, 0xa3, 0xfc, 0x4d, + 0xc6, 0xd6, 0x3c, 0xe9, 0x62, 0xe2, 0xb4, 0xba, 0xbf, 0x89, 0xa2, 0x02, 0x95, 0x1c, 0xe9, 0xb1, + 0xd2, 0x71, 0x44, 0xf3, 0xba, 0xae, 0x1f, 0xd7, 0xc1, 0x3f, 0x4a, 0x72, 0x25, 0xb0, 0x77, 0xf0, + 0xec, 0xb1, 0xfa, 0x28, 0x9f, 0x87, 0x3e, 0x8e, 0x05, 0x2b, 0x9e, 0x38, 0x47, 0xc5, 0x03, 0x2b, + 0xf4, 0x3e, 0x15, 0x7d, 0x06, 0x81, 0x35, 0x78, 0x99, 0xd5, 0xc9, 0x45, 0x3f, 0xe7, 0x51, 0xc7, + 0xbc, 0x2f, 0x85, 0xd1, 0x6d, 0x54, 0x81, 0x34, 0x3e, 0xee, 0xca, 0x1a, 0xb6, 0x55, 0x6e, 0xb8, + 0xc4, 0x61, 0x5f, 0xf1, 0x06, 0xf0, 0xc0, 0x2c, 0x3e, 0xee, 0x12, 0x22, 0xda, 0x21, 0xb3, 0x97, + 0xeb, 0x2a, 0xd0, 0x62, 0xdb, 0xe3, 0xe1, 0x85, 0xdf, 0x53, 0xb8, 0xb8, 0x82, 0xe7, 0x25, 0x30, + 0x11, 0xe2, 0x0f, 0x05, 0x78, 0x2b, 0xb2, 0xd5, 0xe2, 0x9c, 0xc8, 0x3e, 0x81, 0x14, 0xad, 0x7c, + 0xe2, 0x9c, 0x95, 0xa7, 0x5c, 0xe2, 0xf7, 0x12, 0x7c, 0x8c, 0x4b, 0xb8, 0x6d, 0x12, 0xc5, 0xc6, + 0xbe, 0xca, 0xf6, 0x1c, 0x72, 0x87, 0xa6, 0x43, 0x7c, 0x13, 0xde, 0xec, 0x89, 0x73, 0x37, 0xfb, + 0x1c, 0x15, 0xe0, 0xb6, 0xf8, 0x4b, 0x98, 0x37, 0x4c, 0x43, 0x0e, 0x0b, 0x3d, 0x7f, 0x5f, 0x2a, + 0x18, 0xa6, 0xf1, 0x32, 0x20, 0xd7, 0xb3, 0x33, 0x7d, 0x9a, 0x88, 0xd3, 0xce, 0x7c, 0x5f, 0x80, + 0x05, 0xcf, 0x6d, 0x8a, 0xd9, 0x83, 0xfe, 0x00, 0x92, 0x86, 0x79, 0x74, 0x9e, 0x55, 0x4c, 0x92, + 0x9f, 0xcc, 0x7a, 0xe1, 0x12, 0xc5, 0x59, 0xdf, 0x7f, 0x95, 0x80, 0xcc, 0xd3, 0x4a, 0x9c, 0xb5, + 0xfc, 0x84, 0xaf, 0x90, 0xb3, 0xf6, 0x8e, 0xea, 0xed, 0xde, 0xfb, 0x96, 0x9e, 0x56, 0x36, 0xf0, + 0x89, 0xdb, 0xdb, 0x09, 0x17, 0x5a, 0x81, 0x8c, 0xb3, 0x6f, 0x61, 0x7b, 0xdf, 0x6c, 0x6b, 0xe7, + 0x71, 0x73, 0x7c, 0xae, 0x32, 0x86, 0x69, 0x2a, 0xd7, 0x8d, 0xc6, 0x10, 0x22, 0xa2, 0x31, 0xc8, + 0x6b, 0x3c, 0x4f, 0x31, 0x71, 0x9e, 0xd7, 0x04, 0x5c, 0xc4, 0xe9, 0xe2, 0x8c, 0xf8, 0x02, 0x80, + 0x54, 0x27, 0xce, 0x26, 0xf9, 0x95, 0x24, 0xe4, 0xb7, 0x7b, 0xf6, 0x7e, 0xcc, 0xbd, 0xaf, 0x02, + 0xd0, 0xed, 0xd9, 0x14, 0x82, 0x1c, 0x1b, 0xbc, 0xce, 0x63, 0x02, 0x3d, 0xdc, 0x4a, 0x33, 0xbe, + 0xe6, 0xb1, 0x81, 0x6a, 0x5c, 0x08, 0x96, 0xfd, 0x68, 0x91, 0x1b, 0xa3, 0xc0, 0x6a, 0xf3, 0xd8, + 0xd8, 0xc4, 0x1e, 0x4a, 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0xcc, 0x92, 0x07, 0xd9, 0x31, 0xcf, + 0xd3, 0xcc, 0x33, 0x84, 0xa7, 0x69, 0xa2, 0xc7, 0x90, 0x61, 0xdc, 0x64, 0xf6, 0x9b, 0xa1, 0xb3, + 0x5f, 0x54, 0x5d, 0xb8, 0x1a, 0xe9, 0xbc, 0x97, 0xa6, 0xac, 0x64, 0xae, 0xbb, 0x00, 0xd3, 0x7b, + 0xa6, 0xa5, 0xba, 0xdf, 0x7b, 0xd9, 0x03, 0x6b, 0xcf, 0xf5, 0x54, 0x3a, 0x5d, 0xcc, 0xac, 0xa7, + 0xd2, 0x99, 0x22, 0x88, 0xbf, 0x21, 0x40, 0xc1, 0x6b, 0x88, 0x38, 0x27, 0x84, 0x4a, 0x48, 0x8b, + 0xe7, 0x6f, 0x0a, 0xa2, 0x40, 0xf1, 0x5f, 0x53, 0x8f, 0x48, 0x35, 0x0f, 0x69, 0xcb, 0xc4, 0xd9, + 0x53, 0x1e, 0xb3, 0x58, 0xa0, 0xc4, 0x79, 0x5b, 0x97, 0x86, 0x05, 0x3d, 0x80, 0x0b, 0x7a, 0x87, + 0xd8, 0x73, 0xdd, 0x69, 0x9f, 0x70, 0xd8, 0xe6, 0x60, 0xf7, 0xc3, 0xf2, 0x82, 0x9f, 0x56, 0x71, + 0x93, 0xc4, 0xbf, 0x43, 0x17, 0xc0, 0xfd, 0x9a, 0xc4, 0xa9, 0xea, 0x3a, 0xe4, 0x2c, 0x26, 0x9a, + 0xb8, 0x35, 0xe7, 0xd4, 0xf6, 0x9c, 0xc7, 0x4a, 0x14, 0xfe, 0x5b, 0x09, 0x28, 0xbc, 0xe8, 0x61, + 0xeb, 0xe4, 0xeb, 0xa4, 0xee, 0xdb, 0x50, 0x38, 0x52, 0x74, 0x47, 0xde, 0x33, 0x2d, 0xb9, 0xd7, + 0xd5, 0x14, 0xc7, 0x0d, 0x48, 0xc9, 0x11, 0xf2, 0x13, 0xd3, 0xda, 0xa1, 0x44, 0x84, 0x01, 0x1d, + 0x18, 0xe6, 0x91, 0x21, 0x13, 0x32, 0x05, 0xca, 0xc7, 0x06, 0x5f, 0x95, 0x5e, 0xfd, 0xf0, 0x3f, + 0x9e, 0x2e, 0x3e, 0x9a, 0x28, 0xcc, 0x8c, 0x86, 0xd4, 0xf5, 0x7a, 0xba, 0xb6, 0xb4, 0xb3, 0x53, + 0x5f, 0x93, 0x8a, 0x54, 0xe4, 0x2b, 0x26, 0xb1, 0x79, 0x6c, 0xd8, 0xe2, 0xdf, 0x4d, 0x40, 0xd1, + 0xd7, 0x51, 0x9c, 0x0d, 0x59, 0x85, 0xec, 0xeb, 0x1e, 0xb6, 0xf4, 0x37, 0x68, 0x46, 0xe0, 0x8c, + 0xc4, 0xec, 0x7c, 0x0e, 0x73, 0x21, 0x0d, 0x24, 0xbf, 0x9a, 0x06, 0xb2, 0x47, 0x7e, 0xe5, 0xd1, + 0x7d, 0x98, 0x77, 0x8e, 0x0d, 0x99, 0x05, 0x18, 0xb2, 0xa0, 0x14, 0x37, 0x5e, 0xa2, 0xe0, 0x10, + 0x7d, 0x10, 0x3a, 0x0d, 0x48, 0xb1, 0xc5, 0xdf, 0x13, 0x00, 0x51, 0x45, 0xd5, 0xd9, 0x67, 0x83, + 0xaf, 0x4b, 0x7f, 0xba, 0x0b, 0x45, 0x1a, 0xb2, 0x29, 0xeb, 0x7b, 0x72, 0x47, 0xb7, 0x6d, 0xdd, + 0x68, 0xf1, 0x0e, 0x95, 0xa7, 0xf4, 0xfa, 0xde, 0x26, 0xa3, 0x8a, 0x7f, 0x11, 0x16, 0x42, 0x15, + 0x88, 0xb3, 0xb1, 0xaf, 0xc3, 0xdc, 0x1e, 0xfb, 0x4a, 0x4b, 0x85, 0xf3, 0x15, 0xc7, 0x2c, 0xa5, + 0xb1, 0xf7, 0x89, 0xff, 0x2d, 0x01, 0x17, 0x24, 0x6c, 0x9b, 0xed, 0x43, 0x1c, 0xbf, 0x0a, 0x6b, + 0xc0, 0x3f, 0xe7, 0xc8, 0x6f, 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xd3, 0x5c, 0x78, 0xd9, 0xfe, 0xe6, + 0xe8, 0x1e, 0x3b, 0xb8, 0x50, 0xcf, 0x97, 0xfd, 0x52, 0xa1, 0x65, 0x3f, 0x13, 0x0a, 0x7a, 0xcb, + 0x30, 0x89, 0x4d, 0xb3, 0xf1, 0x6b, 0xa3, 0xd7, 0x71, 0xc1, 0xd0, 0xd2, 0xa8, 0x42, 0xd6, 0x19, + 0x4b, 0x03, 0xbf, 0xde, 0xea, 0x75, 0xa8, 0xef, 0xbc, 0x7a, 0x89, 0x94, 0xf7, 0xec, 0x74, 0x31, + 0x1f, 0x4a, 0xb3, 0xa5, 0xbc, 0xee, 0x3d, 0x13, 0xe9, 0xe2, 0xb7, 0xe1, 0x62, 0x9f, 0xb2, 0xe3, + 0xf4, 0x78, 0xfe, 0x45, 0x12, 0xae, 0x84, 0xc5, 0xc7, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x83, + 0x5c, 0x47, 0x37, 0xde, 0x6c, 0xf5, 0x72, 0xae, 0xa3, 0x1b, 0xfe, 0x4a, 0x71, 0x44, 0xd7, 0x98, + 0xf9, 0xa9, 0x76, 0x0d, 0x05, 0xca, 0x51, 0x6d, 0x17, 0x67, 0xff, 0xf8, 0x9e, 0x00, 0x73, 0x71, + 0x2f, 0xcb, 0xbd, 0x59, 0xa0, 0x9c, 0xd8, 0x84, 0xdc, 0x4f, 0x61, 0x1d, 0xef, 0xb7, 0x04, 0x40, + 0x4d, 0xab, 0x67, 0x10, 0x50, 0xfb, 0xcc, 0x6c, 0xc5, 0x59, 0xcd, 0x0b, 0x30, 0xad, 0x1b, 0x1a, + 0x3e, 0xa6, 0xd5, 0x4c, 0x49, 0xec, 0x21, 0xf4, 0x75, 0x32, 0x39, 0xd1, 0xd7, 0x49, 0xf1, 0x73, + 0x58, 0x08, 0x15, 0x31, 0xce, 0xfa, 0xff, 0xd7, 0x04, 0x2c, 0xf0, 0x8a, 0xc4, 0xbe, 0x82, 0xf9, + 0x4d, 0x98, 0x6e, 0x13, 0x99, 0x23, 0xda, 0x99, 0xbe, 0xd3, 0x6d, 0x67, 0x9a, 0x19, 0xfd, 0x2c, + 0x40, 0xd7, 0xc2, 0x87, 0x32, 0x63, 0x4d, 0x4e, 0xc4, 0x9a, 0x21, 0x1c, 0x94, 0x80, 0x7e, 0x20, + 0x40, 0x81, 0x0c, 0xe8, 0xae, 0x65, 0x76, 0x4d, 0x9b, 0xf8, 0x2c, 0xf6, 0x64, 0x30, 0xe7, 0xc5, + 0xd9, 0xe9, 0x62, 0x6e, 0x53, 0x37, 0xb6, 0x39, 0x63, 0xb3, 0x31, 0xf1, 0x1e, 0x00, 0x77, 0x27, + 0xc4, 0x52, 0xa5, 0x6d, 0xaa, 0x07, 0xfe, 0xf7, 0x36, 0x62, 0x59, 0x3c, 0x71, 0xb6, 0xf8, 0xef, + 0x05, 0xb8, 0xf0, 0x53, 0x5b, 0x2e, 0xfe, 0x7f, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, + 0xb1, 0x67, 0xc6, 0xb9, 0x70, 0xff, 0xab, 0x02, 0xcc, 0x07, 0x04, 0xc7, 0xe9, 0xe0, 0xbc, 0x91, + 0x9e, 0xc4, 0x9f, 0x27, 0x2e, 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x69, 0x02, 0x2e, 0x55, + 0xd8, 0x67, 0x6e, 0x37, 0xee, 0x23, 0xce, 0x9e, 0x51, 0x82, 0xd9, 0x43, 0x6c, 0xd9, 0xba, 0xc9, + 0xa6, 0xdf, 0x9c, 0xe4, 0x3e, 0xa2, 0x32, 0xa4, 0x6d, 0x43, 0xe9, 0xda, 0xfb, 0xa6, 0xfb, 0x7d, + 0xcf, 0x7b, 0xf6, 0x62, 0x54, 0xa6, 0xdf, 0x3c, 0x46, 0x65, 0x66, 0x74, 0x8c, 0xca, 0xec, 0x57, + 0x88, 0x51, 0xe1, 0x1f, 0xd3, 0xfe, 0x8d, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, 0xf9, 0x2e, + 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xf6, 0xa5, 0xb0, 0x4e, 0xb2, 0xbd, 0x21, 0x6e, 0x39, 0x3b, + 0x5d, 0x04, 0xb7, 0xa8, 0xf5, 0x35, 0xae, 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x29, 0x07, 0x85, 0xea, + 0x31, 0x5b, 0x25, 0x6f, 0x30, 0x37, 0x01, 0x3d, 0x81, 0x74, 0xd7, 0x32, 0x0f, 0x75, 0xb7, 0x1a, + 0xf9, 0x50, 0x80, 0x82, 0x5b, 0x8d, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x09, 0x99, + 0x67, 0xa6, 0xaa, 0xb4, 0x9f, 0xe8, 0x6d, 0xb7, 0xe7, 0xbf, 0x3f, 0x5e, 0xd0, 0x92, 0xc7, 0xb3, + 0xad, 0x38, 0xfb, 0x6e, 0x23, 0x78, 0x44, 0x54, 0x87, 0x74, 0xcd, 0x71, 0xba, 0x24, 0x91, 0xdb, + 0x8e, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, 0x94, 0xac, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0xee, + 0xf9, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xf6, 0xed, 0x09, 0x64, + 0x3e, 0xad, 0x34, 0xa4, 0x41, 0x01, 0x68, 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0xf3, 0xeb, 0x6e, + 0x4d, 0x20, 0xac, 0xf1, 0x48, 0xf2, 0xd8, 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, + 0x99, 0x19, 0x1a, 0x1d, 0xd1, 0x2f, 0x85, 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, + 0x3a, 0x68, 0x9b, 0x8a, 0x5b, 0xc3, 0x59, 0x2a, 0xee, 0x1b, 0x13, 0x88, 0x73, 0x19, 0xa5, 0x3e, + 0x11, 0xe8, 0xdb, 0x50, 0x20, 0x8d, 0xd1, 0x54, 0x76, 0xdb, 0x6e, 0x21, 0xd3, 0x54, 0xea, 0xbb, + 0x13, 0x48, 0xf5, 0x38, 0xdd, 0x85, 0xff, 0x3e, 0x51, 0x65, 0x09, 0x72, 0xa1, 0x4e, 0x80, 0x10, + 0xa4, 0xba, 0xa4, 0xbd, 0x05, 0x1a, 0xbf, 0x44, 0x7f, 0xa3, 0xf7, 0x60, 0xd6, 0x30, 0x35, 0xec, + 0x8e, 0x90, 0xdc, 0xea, 0x85, 0xb3, 0xd3, 0xc5, 0x99, 0x2d, 0x53, 0x63, 0x1e, 0x0d, 0xff, 0x25, + 0xcd, 0x90, 0x4c, 0x75, 0xad, 0x7c, 0x0d, 0x52, 0xa4, 0xdd, 0x89, 0x61, 0xda, 0x55, 0x6c, 0xbc, + 0x63, 0xe9, 0x5c, 0x9a, 0xfb, 0x58, 0xfe, 0x07, 0x09, 0x48, 0x34, 0x1e, 0x11, 0x9f, 0x7d, 0xb7, + 0xa7, 0x1e, 0x60, 0x87, 0xa7, 0xf3, 0x27, 0xea, 0xcb, 0x5b, 0x78, 0x4f, 0x67, 0xae, 0x55, 0x46, + 0xe2, 0x4f, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, 0xcb, 0xee, 0x5e, 0xc0, 0x8c, 0x94, 0x61, + 0x94, 0x0d, 0x7c, 0x42, 0xd8, 0x6c, 0xac, 0x5a, 0xd8, 0x71, 0x83, 0xaf, 0xd8, 0x13, 0x61, 0x73, + 0x70, 0xa7, 0x2b, 0x3b, 0xe6, 0x01, 0x36, 0x68, 0x3f, 0xc9, 0x10, 0x53, 0xd3, 0xe9, 0x36, 0x09, + 0x81, 0x58, 0x49, 0x6c, 0x68, 0xbe, 0x49, 0xcb, 0x48, 0xde, 0x33, 0x11, 0x69, 0xe1, 0x96, 0xce, + 0x37, 0xb5, 0x65, 0x24, 0xfe, 0x44, 0xb4, 0xa4, 0xf4, 0x9c, 0x7d, 0xda, 0x12, 0x19, 0x89, 0xfe, + 0x46, 0xb7, 0xa1, 0xc0, 0xe2, 0x35, 0x65, 0x6c, 0xa8, 0x32, 0x35, 0xae, 0x19, 0x9a, 0x9c, 0x63, + 0xe4, 0xaa, 0xa1, 0x12, 0x53, 0x8a, 0x1e, 0x01, 0x27, 0xc8, 0x07, 0x1d, 0x9b, 0xe8, 0x14, 0x48, + 0xae, 0xd5, 0xc2, 0xd9, 0xe9, 0x62, 0xb6, 0x41, 0x13, 0x36, 0x36, 0x1b, 0xf5, 0x35, 0x29, 0xcb, + 0x72, 0x6d, 0x74, 0xec, 0xba, 0x56, 0xfe, 0x35, 0x01, 0x92, 0x4f, 0x2b, 0x8d, 0x73, 0xab, 0xcc, + 0x2d, 0x68, 0x32, 0x50, 0xd0, 0x3b, 0x50, 0xd8, 0xd5, 0xdb, 0x6d, 0xdd, 0x68, 0x11, 0x2f, 0xea, + 0xbb, 0x58, 0x75, 0x15, 0x96, 0xe7, 0xe4, 0x6d, 0x46, 0x45, 0xd7, 0x20, 0xab, 0x5a, 0x58, 0xc3, + 0x86, 0xa3, 0x2b, 0x6d, 0x9b, 0x6b, 0x2e, 0x48, 0x2a, 0xff, 0xa2, 0x00, 0xd3, 0x74, 0x04, 0xa0, + 0xb7, 0x21, 0xa3, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0x37, 0x65, 0x19, 0xc9, 0x27, 0x0c, 0x2d, 0xde, + 0x75, 0x98, 0x53, 0x54, 0xd5, 0xec, 0x19, 0x8e, 0x6c, 0x28, 0x1d, 0xcc, 0x8b, 0x99, 0xe5, 0xb4, + 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x8f, 0xde, 0x16, 0xcf, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, + 0xca, 0xbf, 0x2f, 0x40, 0xda, 0x1d, 0x33, 0xa4, 0x18, 0x2d, 0x6c, 0xb0, 0xa0, 0x72, 0xb7, 0x18, + 0x1e, 0xa1, 0x7f, 0xaa, 0xcc, 0xf8, 0x53, 0xe5, 0x05, 0x98, 0x76, 0xc8, 0xb0, 0xe0, 0x25, 0x60, + 0x0f, 0x74, 0x3d, 0xbb, 0xad, 0xb4, 0xd8, 0x72, 0x5e, 0x46, 0x62, 0x0f, 0xa4, 0x32, 0x3c, 0xec, + 0x97, 0x69, 0x84, 0x3f, 0x91, 0x92, 0xb2, 0xe0, 0xd4, 0x5d, 0xdc, 0xd2, 0x0d, 0xda, 0x97, 0x92, + 0x12, 0x50, 0xd2, 0x2a, 0xa1, 0xa0, 0xb7, 0x20, 0xc3, 0x32, 0x60, 0x43, 0xa3, 0x1d, 0x2a, 0x29, + 0xa5, 0x29, 0xa1, 0x6a, 0x68, 0x65, 0x0c, 0x19, 0x6f, 0x70, 0x92, 0x66, 0xeb, 0xd9, 0x9e, 0x22, + 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, 0xdd, 0x53, 0xda, 0xfa, 0x1e, 0x5d, 0xa9, 0xa3, 0x51, 0xf7, + 0x54, 0x67, 0xac, 0x26, 0xc8, 0x4b, 0xa3, 0x12, 0xa8, 0xea, 0xdc, 0xb1, 0x9c, 0xf4, 0xc7, 0xb2, + 0xf8, 0x3b, 0x02, 0xcc, 0xb3, 0xc8, 0x23, 0x16, 0x30, 0x1b, 0x9f, 0x1f, 0xf2, 0x31, 0x64, 0x34, + 0xc5, 0x51, 0xd8, 0xa6, 0xd5, 0xc4, 0xc8, 0x4d, 0xab, 0xde, 0x26, 0x0a, 0xc5, 0x51, 0xe8, 0xc6, + 0x55, 0x04, 0x29, 0xf2, 0x9b, 0xed, 0xef, 0x95, 0xe8, 0x6f, 0xf1, 0x33, 0x40, 0xc1, 0x82, 0xc6, + 0xe9, 0x91, 0xdd, 0x83, 0x8b, 0x44, 0xd7, 0x55, 0x43, 0xb5, 0x4e, 0xba, 0x8e, 0x6e, 0x1a, 0xcf, + 0xe9, 0x5f, 0x1b, 0x15, 0x03, 0x1f, 0xb6, 0xe8, 0xf7, 0x2c, 0xf1, 0x77, 0x67, 0x20, 0x57, 0x3d, + 0xee, 0x9a, 0x56, 0xac, 0xcb, 0x60, 0xab, 0x30, 0xcb, 0x57, 0x0a, 0x46, 0x7c, 0xbb, 0xee, 0x33, + 0xe6, 0xee, 0x87, 0x7b, 0xce, 0x88, 0x56, 0x01, 0x58, 0x50, 0x2b, 0x0d, 0x5c, 0x4a, 0x9e, 0xe3, + 0x53, 0x1b, 0x65, 0xa3, 0x1b, 0x38, 0xb6, 0x20, 0xdb, 0x39, 0x54, 0x55, 0x79, 0x4f, 0x6f, 0x3b, + 0x3c, 0x36, 0x30, 0x3a, 0x8c, 0x7d, 0xf3, 0x65, 0xa5, 0xf2, 0x84, 0x66, 0x62, 0x61, 0x7a, 0xfe, + 0xb3, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0xde, 0x05, 0xbe, 0x99, 0x48, 0xb6, 0xdd, 0x7d, 0x83, 0xab, + 0xb9, 0xb3, 0xd3, 0xc5, 0x8c, 0x44, 0xa9, 0x8d, 0x46, 0x53, 0xca, 0xb0, 0x0c, 0x0d, 0xdb, 0x41, + 0x37, 0x20, 0x67, 0x76, 0x74, 0x47, 0x76, 0x9d, 0x24, 0xee, 0x51, 0xce, 0x11, 0xa2, 0xeb, 0x44, + 0x9d, 0x67, 0x8f, 0xc9, 0xec, 0xc4, 0x7b, 0x4c, 0xd0, 0x5f, 0x11, 0xe0, 0x12, 0x57, 0xa4, 0xbc, + 0x4b, 0xe3, 0xf0, 0x95, 0xb6, 0xee, 0x9c, 0xc8, 0x07, 0x87, 0xa5, 0x34, 0xf5, 0x5b, 0x7f, 0x26, + 0xb2, 0x41, 0x02, 0xfd, 0x60, 0xc9, 0x6d, 0x96, 0x93, 0x67, 0x9c, 0x79, 0xe3, 0xb0, 0x6a, 0x38, + 0xd6, 0xc9, 0xea, 0xe5, 0xb3, 0xd3, 0xc5, 0x85, 0xc1, 0xd4, 0x97, 0xd2, 0x82, 0x3d, 0xc8, 0x82, + 0x6a, 0x00, 0xd8, 0xeb, 0x87, 0x74, 0xc6, 0x88, 0xf6, 0x3f, 0x22, 0x3b, 0xac, 0x14, 0xe0, 0x45, + 0x77, 0xa1, 0xc8, 0xf7, 0xf4, 0xec, 0xe9, 0x6d, 0x2c, 0xdb, 0xfa, 0x17, 0x98, 0xce, 0x2d, 0x49, + 0x29, 0xcf, 0xe8, 0x44, 0x44, 0x43, 0xff, 0x02, 0x97, 0xbf, 0x0b, 0xa5, 0x61, 0xa5, 0x0f, 0x0e, + 0x81, 0x0c, 0xfb, 0xa4, 0xfb, 0x51, 0x78, 0x3d, 0x67, 0x82, 0xae, 0xca, 0xd7, 0x74, 0x3e, 0x4e, + 0x7c, 0x24, 0x88, 0x7f, 0x2f, 0x01, 0xb9, 0xd5, 0x5e, 0xfb, 0xe0, 0x79, 0xb7, 0xd1, 0xeb, 0x74, + 0x14, 0xeb, 0x84, 0x98, 0x41, 0x66, 0x28, 0x48, 0x01, 0x05, 0x66, 0x06, 0xa9, 0x25, 0xd0, 0xbf, + 0xc0, 0x64, 0x72, 0x0a, 0xee, 0x53, 0x67, 0xfb, 0x0c, 0x68, 0x1d, 0x02, 0x9b, 0xcf, 0xcd, 0x23, + 0x1b, 0x7d, 0x04, 0xa5, 0x40, 0x46, 0xba, 0xf8, 0x22, 0x63, 0xc3, 0xb1, 0x74, 0xcc, 0x16, 0x10, + 0x93, 0x52, 0x20, 0xa6, 0xa7, 0x4e, 0x92, 0xab, 0x2c, 0x15, 0x35, 0x61, 0x8e, 0x64, 0x3c, 0x91, + 0xe9, 0x14, 0xe2, 0x2e, 0xf0, 0x3e, 0x88, 0xa8, 0x56, 0xa8, 0xdc, 0x4b, 0x54, 0x3f, 0x15, 0xca, + 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0xca, 0x9f, 0x42, 0xb1, 0x3f, 0x43, 0x50, 0x97, 0x29, 0xa6, + 0xcb, 0x0b, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, 0x69, 0xf1, 0x4f, 0x92, + 0x90, 0x77, 0xbb, 0x59, 0x9c, 0x40, 0x67, 0x15, 0xa6, 0x49, 0xa7, 0x70, 0x23, 0x50, 0x6e, 0x8f, + 0xe8, 0xdd, 0x3c, 0xb2, 0x9d, 0x74, 0x16, 0x17, 0x24, 0x53, 0xd6, 0x38, 0x0c, 0x4e, 0xf9, 0x17, + 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x44, 0x21, 0x4c, 0x32, 0x51, 0xd0, 0xac, 0xde, + 0x74, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, 0x90, 0x1a, 0x9b, 0x39, + 0x89, 0x3f, 0xa1, 0x55, 0x1a, 0x1a, 0x65, 0x5a, 0x0e, 0xd6, 0xb8, 0x4f, 0x7f, 0x6d, 0x5c, 0xfb, + 0xba, 0x93, 0x92, 0xcb, 0x87, 0xae, 0x40, 0x92, 0x58, 0xb1, 0x59, 0x16, 0xe5, 0x70, 0x76, 0xba, + 0x98, 0x24, 0xf6, 0x8b, 0xd0, 0xd0, 0x32, 0x64, 0xc3, 0x26, 0x83, 0x78, 0x70, 0xd4, 0x30, 0x06, + 0x86, 0x3b, 0xb4, 0xbd, 0xa1, 0xc5, 0xf0, 0x2c, 0x6f, 0xe3, 0x3f, 0x4d, 0x41, 0xae, 0xde, 0x89, + 0x7b, 0x4a, 0x59, 0x09, 0xb7, 0x70, 0x14, 0x10, 0x0a, 0xbd, 0x34, 0xa2, 0x81, 0x43, 0x33, 0x78, + 0xf2, 0x7c, 0x33, 0x78, 0x9d, 0x78, 0xca, 0xfc, 0xe0, 0x89, 0xe4, 0x10, 0xcc, 0x13, 0x7e, 0x3f, + 0xf5, 0x53, 0x24, 0xc2, 0xe3, 0xef, 0xf5, 0xa0, 0x91, 0x2a, 0x9f, 0x52, 0x87, 0x9c, 0xf5, 0xb2, + 0x99, 0xc9, 0x7b, 0xd9, 0x2c, 0x36, 0x34, 0x3a, 0xa9, 0x85, 0x2d, 0xea, 0xec, 0x9b, 0x5b, 0xd4, + 0xb2, 0xc3, 0x3b, 0xeb, 0xc7, 0x90, 0xd4, 0x74, 0xb7, 0x71, 0x26, 0x9f, 0xaa, 0x09, 0xd3, 0x98, + 0x5e, 0x9b, 0x0a, 0xf6, 0x5a, 0xd6, 0x4b, 0xca, 0x75, 0x00, 0x5f, 0x37, 0xe8, 0x1a, 0xcc, 0x98, + 0x6d, 0xcd, 0xdd, 0xec, 0x92, 0x5b, 0xcd, 0x9c, 0x9d, 0x2e, 0x4e, 0x3f, 0x6f, 0x6b, 0xf5, 0x35, + 0x69, 0xda, 0x6c, 0x6b, 0x75, 0x8d, 0x9e, 0xfa, 0x81, 0x8f, 0x64, 0x2f, 0x12, 0x6e, 0x4e, 0x9a, + 0x35, 0xf0, 0xd1, 0x1a, 0xb6, 0x55, 0xde, 0xe1, 0x7e, 0x53, 0x80, 0xbc, 0xab, 0xfb, 0x78, 0x8d, + 0x4a, 0x5a, 0xef, 0xf0, 0x41, 0x96, 0x3c, 0xdf, 0x20, 0x73, 0xf9, 0xf8, 0xc6, 0xe2, 0x5f, 0x16, + 0x78, 0x6c, 0x73, 0x43, 0x55, 0x1c, 0xe2, 0x54, 0xc4, 0x38, 0x30, 0xee, 0x41, 0xd1, 0x52, 0x0c, + 0xcd, 0xec, 0xe8, 0x5f, 0x60, 0xb6, 0x10, 0x6a, 0xf3, 0xcf, 0x9e, 0x05, 0x8f, 0x4e, 0x57, 0xfd, + 0x6c, 0xf1, 0xf7, 0x13, 0x3c, 0x0e, 0xda, 0x2b, 0x46, 0x9c, 0xea, 0xfa, 0x0e, 0xcc, 0xf7, 0x1f, + 0xc1, 0xe2, 0x8e, 0xd6, 0xf7, 0x22, 0xe4, 0x45, 0x15, 0x84, 0xc5, 0x33, 0xba, 0xc1, 0xf5, 0x7d, + 0xc7, 0xb1, 0xd8, 0xa8, 0x02, 0xd9, 0xe0, 0xc9, 0x2e, 0xc9, 0x89, 0x4f, 0x76, 0x01, 0xcb, 0x3b, + 0xcf, 0xa5, 0xfc, 0x73, 0x30, 0x4d, 0x93, 0xdf, 0xc0, 0x44, 0xf3, 0xd6, 0xfc, 0xe3, 0x04, 0xdc, + 0xa4, 0xa5, 0x7f, 0x89, 0x2d, 0x7d, 0xef, 0x64, 0xdb, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, 0x57, + 0xce, 0x63, 0xb5, 0x7b, 0x99, 0xae, 0xfb, 0x82, 0x73, 0x05, 0x9c, 0x79, 0x5c, 0x68, 0x03, 0x0a, + 0x3c, 0xb4, 0x40, 0x69, 0xeb, 0x87, 0x58, 0x56, 0x9c, 0xf3, 0xcc, 0x6e, 0x39, 0xc6, 0xbb, 0x42, + 0x58, 0x57, 0x1c, 0xa4, 0x41, 0x86, 0x0b, 0xd3, 0x35, 0x7e, 0x20, 0xd1, 0xd3, 0xaf, 0xb6, 0xa0, + 0x98, 0x66, 0xf1, 0x0d, 0xf5, 0x35, 0x29, 0xcd, 0x24, 0xd7, 0x35, 0xf1, 0x3f, 0x08, 0x70, 0x6b, + 0x8c, 0x8a, 0xe3, 0xec, 0xba, 0x65, 0x48, 0x1f, 0x92, 0x17, 0xe9, 0x5c, 0xc7, 0x69, 0xc9, 0x7b, + 0x46, 0x9b, 0x90, 0xdb, 0x53, 0xf4, 0xb6, 0xdf, 0xa5, 0x87, 0xc7, 0x27, 0x46, 0x47, 0xe3, 0xce, + 0x31, 0x76, 0xd6, 0x87, 0xc5, 0x5f, 0x4f, 0xc0, 0xfc, 0x8a, 0xa6, 0x35, 0x1a, 0xdc, 0x06, 0xc6, + 0xd7, 0x53, 0x5c, 0x90, 0x99, 0xf0, 0x41, 0x26, 0x7a, 0x0f, 0x90, 0xa6, 0xdb, 0xec, 0xe0, 0x13, + 0x7b, 0x5f, 0xd1, 0xcc, 0x23, 0x3f, 0x2e, 0x63, 0xde, 0x4d, 0x69, 0xb8, 0x09, 0xa8, 0x01, 0x14, + 0xed, 0xc8, 0xb6, 0xa3, 0x78, 0xdf, 0x9d, 0x6e, 0x4d, 0xb4, 0xa5, 0x8c, 0xc1, 0x20, 0xef, 0x51, + 0xca, 0x10, 0x39, 0xf4, 0x27, 0xf1, 0xdb, 0x75, 0x52, 0x75, 0x47, 0x56, 0x6c, 0x77, 0xff, 0x10, + 0x3b, 0x72, 0x25, 0xcf, 0xe8, 0x2b, 0x36, 0xdb, 0x16, 0xc4, 0x36, 0x3c, 0xf8, 0xaa, 0x89, 0x13, + 0x12, 0xff, 0x6d, 0x01, 0xf2, 0x12, 0xde, 0xb3, 0xb0, 0x1d, 0xeb, 0xa2, 0xc0, 0x13, 0x98, 0xb3, + 0x98, 0x54, 0x79, 0xcf, 0x32, 0x3b, 0xe7, 0x19, 0x57, 0x59, 0xce, 0xf8, 0xc4, 0x32, 0x3b, 0xdc, + 0xb0, 0xbc, 0x84, 0x82, 0x57, 0xc6, 0x38, 0x2b, 0xff, 0x3b, 0x74, 0xbb, 0x34, 0x13, 0x1c, 0x77, + 0x80, 0x44, 0xbc, 0x1a, 0xa0, 0x1f, 0xaa, 0x82, 0x05, 0x8d, 0x53, 0x0d, 0xff, 0x45, 0x80, 0x7c, + 0xa3, 0xb7, 0xcb, 0x4e, 0xd6, 0x8a, 0x4f, 0x03, 0x55, 0xc8, 0xb4, 0xf1, 0x9e, 0x23, 0xbf, 0x51, + 0xa8, 0x7e, 0x9a, 0xb0, 0xd2, 0x8d, 0x0a, 0x4f, 0x01, 0x2c, 0xba, 0xc5, 0x8e, 0xca, 0x49, 0x9e, + 0x53, 0x4e, 0x86, 0xf2, 0x12, 0xb2, 0xf8, 0xa7, 0x09, 0x28, 0x78, 0xd5, 0x8c, 0xd3, 0x4a, 0xbe, + 0x0a, 0x59, 0x87, 0xe4, 0x79, 0xac, 0xc3, 0x3c, 0x8f, 0x09, 0x89, 0xb6, 0x10, 0x4b, 0xb0, 0x40, + 0x1d, 0x17, 0x59, 0xe9, 0x76, 0xdb, 0xba, 0x0b, 0x77, 0xa9, 0xfd, 0x49, 0x49, 0xf3, 0x34, 0x69, + 0x85, 0xa5, 0x50, 0xa0, 0x8b, 0x7e, 0x45, 0x80, 0xb9, 0x3d, 0x0b, 0xe3, 0x2f, 0xb0, 0x4c, 0xa1, + 0xd7, 0x64, 0x41, 0x2f, 0x6b, 0xa4, 0x0c, 0x5f, 0xf9, 0xa3, 0x78, 0x96, 0xbd, 0xb8, 0x41, 0xde, + 0xcb, 0x3b, 0xed, 0x2b, 0x98, 0xa7, 0x8d, 0x12, 0xf7, 0xde, 0x62, 0xf1, 0x6f, 0x26, 0x00, 0x05, + 0x25, 0xff, 0xf4, 0x1a, 0x33, 0x11, 0x5f, 0x63, 0xbe, 0x0b, 0x88, 0x85, 0x4d, 0xda, 0x72, 0x17, + 0x5b, 0xb2, 0x8d, 0x55, 0x93, 0x1f, 0x0f, 0x25, 0x48, 0x45, 0x9e, 0xb2, 0x8d, 0xad, 0x06, 0xa5, + 0xa3, 0xc7, 0x00, 0xbe, 0x4b, 0xc7, 0x67, 0x9c, 0x91, 0x1e, 0x9d, 0x94, 0xf1, 0x7c, 0x39, 0xf1, + 0xfb, 0x02, 0xe4, 0x37, 0xf5, 0x96, 0xa5, 0xc4, 0x7a, 0xee, 0x11, 0xfa, 0x38, 0xbc, 0x8c, 0x9e, + 0x7d, 0x58, 0x8e, 0x0a, 0xf1, 0x61, 0x39, 0x5c, 0x70, 0xc6, 0x19, 0x88, 0x09, 0xf7, 0x4a, 0x14, + 0xa7, 0xed, 0xfa, 0xb7, 0x65, 0x98, 0xe3, 0xe5, 0xde, 0x31, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9, + 0xe2, 0x9f, 0x49, 0xb2, 0x91, 0x4b, 0x9a, 0xfe, 0x11, 0x80, 0xb5, 0x29, 0x89, 0xe4, 0x25, 0x2c, + 0xdd, 0x9e, 0x13, 0xe1, 0x44, 0xfa, 0x51, 0xef, 0x41, 0x96, 0x6e, 0xcf, 0x41, 0x0d, 0x28, 0xa8, + 0xfe, 0x51, 0x66, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x9c, 0x91, 0x87, 0xca, 0xd5, 0xa6, 0xa4, 0xbc, + 0x1a, 0x4a, 0x40, 0x95, 0xe0, 0x09, 0x5a, 0xa9, 0x81, 0x90, 0x3a, 0x7f, 0xbf, 0x76, 0xf8, 0xf4, + 0xae, 0xda, 0x54, 0xe0, 0xa0, 0x2d, 0xf4, 0x31, 0xcc, 0x68, 0xf4, 0xac, 0x26, 0x3e, 0xf8, 0xa3, + 0x1a, 0x3a, 0x74, 0x24, 0x56, 0x6d, 0x4a, 0xe2, 0x1c, 0x68, 0x1d, 0xe6, 0xd8, 0x2f, 0xe6, 0xd2, + 0x71, 0x14, 0x7e, 0x6b, 0xb8, 0x84, 0xc0, 0xa4, 0x59, 0x9b, 0x92, 0xb2, 0x9a, 0x4f, 0x45, 0xdf, + 0x84, 0x94, 0xad, 0x2a, 0x2e, 0x0e, 0xbf, 0x3a, 0xe4, 0x88, 0x14, 0x9f, 0x99, 0xe6, 0x46, 0x8f, + 0xd9, 0xa1, 0x9e, 0xce, 0xb1, 0xbb, 0x24, 0x1a, 0x55, 0xfc, 0xd0, 0x56, 0x7c, 0x52, 0x7c, 0x4c, + 0x09, 0xe8, 0x29, 0x64, 0x15, 0xe2, 0x1b, 0xcb, 0x74, 0x33, 0x2b, 0x5d, 0x03, 0x8d, 0x0e, 0x44, + 0x18, 0xd8, 0x7c, 0x5c, 0xa3, 0xfb, 0xfd, 0x5d, 0xa2, 0x2f, 0xa8, 0x83, 0xad, 0x16, 0x2e, 0x65, + 0x47, 0x0b, 0x0a, 0x06, 0xd2, 0x79, 0x82, 0x28, 0x91, 0xf8, 0xc8, 0xde, 0x4e, 0x72, 0x5a, 0xa9, + 0xb9, 0xa1, 0x9f, 0xbe, 0x23, 0xf6, 0x48, 0xd5, 0xa6, 0xa4, 0xb9, 0xfd, 0x00, 0x19, 0x2d, 0x41, + 0xa2, 0xa5, 0x96, 0x72, 0x43, 0x8d, 0x81, 0xb7, 0x0f, 0xa8, 0x36, 0x25, 0x25, 0x5a, 0x2a, 0xfa, + 0x14, 0xd2, 0x6c, 0x53, 0xc7, 0xb1, 0x51, 0xca, 0x0f, 0x1d, 0x63, 0xe1, 0xad, 0x31, 0xb5, 0x29, + 0x89, 0xee, 0x23, 0x21, 0xef, 0xdb, 0x86, 0xbc, 0xc5, 0x22, 0x11, 0xdd, 0xb8, 0xe1, 0xe2, 0xd0, + 0x70, 0x80, 0xa8, 0xd0, 0xe1, 0x1a, 0x45, 0x49, 0x01, 0x3a, 0xfa, 0x0e, 0x5c, 0x08, 0x4b, 0xe4, + 0x3d, 0x6d, 0x7e, 0xe8, 0xa7, 0xed, 0xa1, 0x61, 0xac, 0xb5, 0x29, 0x09, 0x59, 0x03, 0x89, 0xe8, + 0x43, 0x98, 0x66, 0xad, 0x86, 0xa8, 0xc8, 0xa8, 0x08, 0x97, 0xbe, 0x06, 0x63, 0xf9, 0x49, 0xe7, + 0x77, 0x78, 0x38, 0x9e, 0xdc, 0x36, 0x5b, 0xa5, 0x85, 0xa1, 0x9d, 0x7f, 0x30, 0xb0, 0x90, 0x74, + 0x7e, 0xc7, 0xa7, 0x92, 0x76, 0xb7, 0x58, 0x0a, 0x0f, 0xc1, 0xba, 0x30, 0xb4, 0xdd, 0x23, 0xa2, + 0xf4, 0x6a, 0x74, 0xa3, 0x84, 0x4f, 0x26, 0x45, 0xb3, 0xd8, 0xa9, 0x3e, 0x32, 0x1d, 0x53, 0x17, + 0x87, 0x16, 0x6d, 0xf0, 0xd8, 0xa3, 0x1a, 0xf5, 0x27, 0x3d, 0x2a, 0x7a, 0x09, 0x45, 0x7e, 0xf6, + 0x86, 0xff, 0xfd, 0xe5, 0x12, 0x95, 0x77, 0x2f, 0xd2, 0x74, 0x45, 0xc5, 0x2f, 0xd5, 0xa6, 0xa4, + 0x82, 0x1a, 0x4e, 0x41, 0x9f, 0xc1, 0x3c, 0x95, 0x27, 0xab, 0xfe, 0x71, 0x29, 0xa5, 0xd2, 0xc0, + 0xe1, 0x1b, 0xc3, 0x4f, 0x56, 0x71, 0x25, 0x17, 0xd5, 0xbe, 0x24, 0xd2, 0x8d, 0x75, 0x43, 0x77, + 0xa8, 0x95, 0x2d, 0x0f, 0xed, 0xc6, 0xe1, 0xa3, 0x1f, 0x49, 0x37, 0xd6, 0x19, 0x85, 0x74, 0x63, + 0x87, 0xc7, 0xe7, 0xf1, 0xe6, 0x78, 0x7b, 0x68, 0x37, 0x8e, 0x0a, 0xe4, 0x23, 0xdd, 0xd8, 0x09, + 0xd2, 0x49, 0x37, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, 0xbb, 0xf1, 0xd0, 0x4d, 0xe5, 0xa4, + 0x1b, 0x2b, 0x03, 0x89, 0x68, 0x0d, 0x80, 0xb9, 0x7e, 0x74, 0xfe, 0xbf, 0x3a, 0x74, 0x32, 0xe8, + 0x8f, 0xd3, 0x23, 0x93, 0x41, 0xdb, 0xa5, 0x11, 0x43, 0x46, 0x81, 0xa5, 0x4c, 0x3f, 0x37, 0x97, + 0x16, 0x87, 0x1a, 0xb2, 0x81, 0x4f, 0xc3, 0xc4, 0x90, 0x1d, 0x79, 0x44, 0x32, 0xab, 0xb0, 0xf5, + 0xf1, 0xd2, 0xb5, 0xe1, 0x66, 0x39, 0xf8, 0x99, 0x8c, 0x9a, 0x65, 0x4a, 0x40, 0x2b, 0x90, 0x21, + 0xee, 0xcd, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xef, 0x7d, 0x3b, 0x81, 0x6a, 0x53, 0x52, 0xfa, + 0x35, 0x27, 0x91, 0xd7, 0xb3, 0x95, 0xc3, 0x92, 0x38, 0xf4, 0xf5, 0xa1, 0x55, 0x66, 0xf2, 0x7a, + 0xc6, 0x81, 0x54, 0xb8, 0xc8, 0xda, 0x8a, 0xef, 0xef, 0xb6, 0xf8, 0x96, 0xe4, 0xd2, 0x0d, 0x2a, + 0x6a, 0xe8, 0x12, 0x5c, 0xe4, 0xb6, 0xf3, 0xda, 0x94, 0xb4, 0xa0, 0x0c, 0xa6, 0x92, 0x01, 0xcf, + 0xa7, 0x1e, 0xb6, 0x70, 0x57, 0xba, 0x39, 0x74, 0xc0, 0x47, 0xac, 0x77, 0x92, 0x01, 0xaf, 0x04, + 0xc8, 0x6c, 0x02, 0xd2, 0x64, 0xdb, 0x66, 0xc1, 0x09, 0xb7, 0x46, 0x4c, 0x40, 0x7d, 0x2b, 0x26, + 0x6c, 0x02, 0xd2, 0x1a, 0x8c, 0x93, 0x08, 0x52, 0xdb, 0x58, 0xb1, 0xb8, 0x99, 0xbd, 0x3d, 0x54, + 0xd0, 0xc0, 0x29, 0x8b, 0x44, 0x90, 0xea, 0x11, 0x89, 0xc3, 0x63, 0xb9, 0x87, 0xf6, 0x70, 0xdf, + 0xf8, 0xce, 0x50, 0x87, 0x27, 0xf2, 0x54, 0x21, 0xe2, 0xf0, 0x58, 0xa1, 0x04, 0xf4, 0xb3, 0x30, + 0xcb, 0xa1, 0x6e, 0xe9, 0xee, 0x08, 0x17, 0x30, 0xb8, 0x3a, 0x41, 0xc6, 0x35, 0xe7, 0x61, 0x56, + 0x96, 0x41, 0x6c, 0x56, 0xbd, 0x7b, 0x23, 0xac, 0xec, 0x00, 0xca, 0x67, 0x56, 0xd6, 0x27, 0x13, + 0x2b, 0xcb, 0xfa, 0x29, 0x9f, 0xeb, 0xee, 0x0f, 0xb5, 0xb2, 0x83, 0xfb, 0x8c, 0x88, 0x95, 0x7d, + 0xed, 0x53, 0x49, 0xcd, 0x6c, 0x06, 0x35, 0x4b, 0xdf, 0x18, 0x5a, 0xb3, 0x30, 0xe6, 0x26, 0x35, + 0xe3, 0x3c, 0xa4, 0xd9, 0x98, 0xf7, 0xcf, 0x34, 0xfd, 0xee, 0xf0, 0x43, 0x14, 0xfa, 0xf1, 0x55, + 0xcd, 0x5d, 0xd4, 0x65, 0x1a, 0xf6, 0x0c, 0x95, 0xc5, 0x77, 0x78, 0x73, 0x4d, 0xbd, 0x37, 0xda, + 0x50, 0x45, 0xed, 0x8c, 0xf7, 0x0c, 0x55, 0x28, 0x91, 0x16, 0x95, 0x6d, 0xea, 0xa3, 0xe3, 0x7b, + 0x69, 0xc4, 0x79, 0x0f, 0x7d, 0x5b, 0x2b, 0x69, 0x51, 0x3d, 0xa2, 0x3f, 0x84, 0x7a, 0xec, 0x30, + 0x92, 0xd2, 0xf2, 0xe8, 0x21, 0x14, 0x3e, 0x0e, 0xc5, 0x1b, 0x42, 0x9c, 0xec, 0xcd, 0x99, 0xae, + 0x87, 0xf1, 0xfe, 0xe8, 0x39, 0xb3, 0xdf, 0xb5, 0x60, 0x73, 0x26, 0xf7, 0x29, 0xfe, 0xb2, 0x00, + 0xd7, 0x58, 0xd9, 0xe8, 0xea, 0xe7, 0x89, 0xec, 0xad, 0x21, 0x07, 0x36, 0x98, 0x3c, 0xa0, 0x2f, + 0xf8, 0x70, 0x58, 0x71, 0xc7, 0xac, 0x89, 0xd7, 0xa6, 0xa4, 0x77, 0x94, 0x51, 0xf9, 0x48, 0x97, + 0xea, 0x30, 0x04, 0x55, 0x7a, 0x38, 0xb4, 0x4b, 0x85, 0x51, 0x1f, 0xe9, 0x52, 0x9c, 0x67, 0x75, + 0x96, 0x7f, 0x81, 0xf6, 0x36, 0xe4, 0x16, 0x8a, 0xc5, 0xf5, 0x54, 0xfa, 0x72, 0xb1, 0xb4, 0x9e, + 0x4a, 0x5f, 0x29, 0x96, 0xd7, 0x53, 0xe9, 0xb7, 0x8a, 0x6f, 0x8b, 0x7f, 0xbf, 0x0c, 0x39, 0x17, + 0x74, 0x31, 0x40, 0xf5, 0x30, 0x08, 0xa8, 0xae, 0x0e, 0x03, 0x54, 0x1c, 0xa6, 0x71, 0x44, 0xf5, + 0x30, 0x88, 0xa8, 0xae, 0x0e, 0x43, 0x54, 0x3e, 0x0f, 0x81, 0x54, 0xcd, 0x61, 0x90, 0xea, 0xde, + 0x04, 0x90, 0xca, 0x13, 0xd5, 0x8f, 0xa9, 0xd6, 0x06, 0x31, 0xd5, 0xcd, 0xd1, 0x98, 0xca, 0x13, + 0x15, 0x00, 0x55, 0x8f, 0xfb, 0x40, 0xd5, 0xf5, 0x11, 0xa0, 0xca, 0xe3, 0x77, 0x51, 0xd5, 0x46, + 0x24, 0xaa, 0xba, 0x3d, 0x0e, 0x55, 0x79, 0x72, 0x42, 0xb0, 0xea, 0x83, 0x10, 0xac, 0x5a, 0x1c, + 0x0a, 0xab, 0x3c, 0x6e, 0x86, 0xab, 0x3e, 0xe9, 0xc7, 0x55, 0xd7, 0x47, 0xe0, 0x2a, 0xbf, 0x06, + 0x1c, 0x58, 0xd5, 0xa2, 0x80, 0xd5, 0xad, 0x31, 0xc0, 0xca, 0x93, 0x12, 0x44, 0x56, 0xb5, 0x28, + 0x64, 0x75, 0x6b, 0x0c, 0xb2, 0xea, 0x93, 0xc4, 0xa0, 0xd5, 0x56, 0x34, 0xb4, 0xba, 0x33, 0x16, + 0x5a, 0x79, 0xd2, 0xc2, 0xd8, 0x6a, 0x39, 0x80, 0xad, 0xde, 0x19, 0x82, 0xad, 0x3c, 0x56, 0x02, + 0xae, 0xbe, 0x35, 0x00, 0xae, 0xc4, 0x51, 0xe0, 0xca, 0xe3, 0xf5, 0xd0, 0xd5, 0x8b, 0x21, 0xe8, + 0xea, 0xee, 0x78, 0x74, 0xe5, 0x09, 0xeb, 0x83, 0x57, 0xca, 0x48, 0x78, 0xf5, 0xde, 0x84, 0xf0, + 0xca, 0x93, 0x1e, 0x85, 0xaf, 0x3e, 0x0a, 0xe3, 0xab, 0x6b, 0xc3, 0xf1, 0x95, 0x27, 0x86, 0x03, + 0xac, 0x8d, 0x48, 0x80, 0x75, 0x7b, 0x1c, 0xc0, 0xf2, 0xc7, 0x41, 0x10, 0x61, 0x6d, 0x45, 0x23, + 0xac, 0x3b, 0x63, 0x11, 0x96, 0xdf, 0xfc, 0x21, 0x88, 0xb5, 0x11, 0x09, 0xb1, 0x6e, 0x8f, 0x83, + 0x58, 0x7e, 0xe1, 0x82, 0x18, 0xeb, 0xd5, 0x50, 0x8c, 0x75, 0x7f, 0x12, 0x8c, 0xe5, 0x09, 0x1d, + 0x00, 0x59, 0x9f, 0x0f, 0x07, 0x59, 0xdf, 0x38, 0xc7, 0xf1, 0x95, 0x91, 0x28, 0xeb, 0x5b, 0x03, + 0x28, 0x4b, 0x1c, 0x85, 0xb2, 0xfc, 0xfe, 0xec, 0xc2, 0x2c, 0x65, 0x24, 0x28, 0x7a, 0x6f, 0x42, + 0x50, 0xe4, 0x77, 0xbe, 0x08, 0x54, 0x54, 0x8d, 0x40, 0x45, 0x37, 0x47, 0xa3, 0x22, 0xdf, 0x9c, + 0xfb, 0xb0, 0xa8, 0x16, 0x05, 0x8b, 0x6e, 0x8d, 0x81, 0x45, 0xbe, 0x15, 0x0a, 0xe0, 0xa2, 0xc7, + 0x7d, 0xb8, 0xe8, 0xfa, 0xd8, 0x00, 0xab, 0x00, 0x30, 0x5a, 0x1d, 0x04, 0x46, 0x37, 0x46, 0x02, + 0x23, 0x4f, 0x82, 0x8f, 0x8c, 0x1e, 0xf7, 0x21, 0xa3, 0xeb, 0x23, 0x90, 0x91, 0x5f, 0x00, 0x0e, + 0x8d, 0xb4, 0xd1, 0xd0, 0x68, 0x69, 0x52, 0x68, 0xe4, 0x09, 0x8e, 0xc4, 0x46, 0x5b, 0xd1, 0xd8, + 0xe8, 0xce, 0x84, 0xb1, 0x0f, 0x03, 0xe0, 0xa8, 0x16, 0x05, 0x8e, 0x6e, 0x8d, 0x01, 0x47, 0xc1, + 0x39, 0xc4, 0x43, 0x47, 0xb5, 0x28, 0x74, 0x74, 0x6b, 0x0c, 0x3a, 0xf2, 0x25, 0x05, 0xe0, 0x51, + 0x73, 0x18, 0x3c, 0xba, 0x37, 0x01, 0x3c, 0xf2, 0x9d, 0x97, 0x3e, 0x7c, 0xf4, 0x69, 0x3f, 0x3e, + 0x12, 0x47, 0xe1, 0x23, 0x7f, 0x44, 0xba, 0x00, 0x69, 0x2b, 0x1a, 0x20, 0xdd, 0x19, 0x0b, 0x90, + 0x82, 0x46, 0x32, 0x80, 0x90, 0x36, 0x22, 0x11, 0xd2, 0xed, 0x71, 0x08, 0xc9, 0x37, 0x92, 0x41, + 0x88, 0xf4, 0x69, 0x3f, 0x44, 0x12, 0x47, 0x41, 0x24, 0xbf, 0x72, 0x2e, 0x46, 0xaa, 0x45, 0x61, + 0xa4, 0x5b, 0x63, 0x30, 0x92, 0xdf, 0x78, 0x01, 0x90, 0xa4, 0x8c, 0x04, 0x49, 0xef, 0x4d, 0x08, + 0x92, 0xfa, 0x0c, 0x57, 0x18, 0x25, 0xd5, 0xa2, 0x50, 0xd2, 0xad, 0x31, 0x28, 0x29, 0x50, 0x58, + 0x1f, 0x26, 0x6d, 0x45, 0xc3, 0xa4, 0x3b, 0x63, 0x61, 0x52, 0xdf, 0x68, 0x72, 0x71, 0xd2, 0x46, + 0x24, 0x4e, 0xba, 0x3d, 0x0e, 0x27, 0xf5, 0x4d, 0x7c, 0xdc, 0x39, 0xf8, 0xa5, 0xc9, 0x81, 0xd2, + 0x47, 0xe7, 0x07, 0x4a, 0xde, 0x3b, 0xc7, 0x20, 0xa5, 0x4f, 0xfb, 0x91, 0x92, 0x38, 0x0a, 0x29, + 0xf9, 0x3d, 0xeb, 0x7c, 0x50, 0x69, 0x3d, 0x95, 0x7e, 0xbb, 0xf8, 0x8e, 0xf8, 0x57, 0x67, 0x61, + 0xa6, 0xe6, 0x85, 0x24, 0xf9, 0xb5, 0x14, 0xde, 0xe4, 0x0c, 0x2c, 0xb4, 0x46, 0x46, 0x3c, 0xb5, + 0x9b, 0xe3, 0x4f, 0x4e, 0x1c, 0x3c, 0xdb, 0x8f, 0xb3, 0xbe, 0xc1, 0x66, 0x74, 0xf4, 0x01, 0xe4, + 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x0d, 0x37, 0xc2, 0x6a, 0xf1, 0xcb, + 0xd3, 0xc5, 0xb9, 0x1d, 0x1b, 0x5b, 0xdb, 0x9c, 0x2e, 0xcd, 0xf5, 0x02, 0x4f, 0xee, 0x2d, 0x67, + 0xd3, 0x93, 0xdf, 0x72, 0xf6, 0x02, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc3, 0xce, 0x9b, 0x8a, + 0xee, 0x73, 0x74, 0x43, 0x9c, 0x9b, 0x93, 0x9e, 0x3b, 0x55, 0xb0, 0xc2, 0x44, 0xf4, 0x00, 0x2e, + 0x76, 0x94, 0x63, 0x1a, 0xbe, 0x2a, 0xbb, 0x4e, 0x21, 0x0d, 0x49, 0x65, 0x17, 0x88, 0xa1, 0x8e, + 0x72, 0x4c, 0xaf, 0x4c, 0x63, 0x49, 0xf4, 0xbe, 0x93, 0x5b, 0x90, 0xd7, 0x74, 0xdb, 0xd1, 0x0d, + 0xd5, 0xe1, 0x47, 0x19, 0xb3, 0x63, 0x80, 0x73, 0x2e, 0x95, 0x9d, 0x57, 0x7c, 0x1f, 0xe6, 0xf9, + 0xbe, 0x86, 0xc0, 0xc7, 0x58, 0x7e, 0x1c, 0x30, 0x4b, 0xf0, 0xbe, 0xbf, 0xa2, 0x0a, 0x14, 0x5a, + 0x8a, 0x83, 0x8f, 0x94, 0x13, 0xd9, 0xdd, 0x35, 0x97, 0xa5, 0x27, 0x7c, 0xbe, 0x75, 0x76, 0xba, + 0x98, 0x7b, 0xca, 0x92, 0x06, 0x36, 0xcf, 0xe5, 0x5a, 0x81, 0x04, 0x0d, 0xdd, 0x81, 0x82, 0x62, + 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, 0xec, 0x9e, 0x4d, 0x21, 0x49, 0x5a, 0xca, 0x53, 0x72, 0xc5, + 0xa5, 0xa2, 0xeb, 0x30, 0xc7, 0x83, 0xfe, 0xd9, 0x1d, 0x4c, 0x05, 0x5a, 0x55, 0x7e, 0xb9, 0x07, + 0xbb, 0x86, 0xe9, 0x31, 0x94, 0xf9, 0x55, 0x05, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0x7e, 0xff, + 0x2c, 0x52, 0xb1, 0x97, 0xd9, 0xd5, 0x04, 0x24, 0x03, 0x51, 0xb5, 0x3f, 0x86, 0xb6, 0x60, 0x5e, + 0x6d, 0xeb, 0x1e, 0x82, 0x60, 0x35, 0x9f, 0x1f, 0x3a, 0x9a, 0x2a, 0x34, 0xaf, 0xff, 0x31, 0xba, + 0xa0, 0x86, 0x09, 0xa8, 0x01, 0xf4, 0x64, 0x1f, 0xb9, 0x6b, 0xb6, 0x75, 0xf5, 0x84, 0x82, 0x87, + 0xf0, 0x91, 0xeb, 0x23, 0x2f, 0x3e, 0x78, 0xa5, 0xe8, 0xce, 0x36, 0xe5, 0x94, 0xe0, 0xc8, 0xfb, + 0xcd, 0x8e, 0x4b, 0x5e, 0x4f, 0xa5, 0xe7, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, + 0x4d, 0x80, 0x42, 0x5f, 0x59, 0x50, 0x0d, 0x2e, 0x6a, 0xde, 0x50, 0x91, 0xf9, 0xae, 0x30, 0xdd, + 0x34, 0xf8, 0x29, 0xf2, 0x0b, 0x5f, 0x9e, 0x2e, 0x16, 0x68, 0xee, 0xa7, 0x5e, 0x92, 0x74, 0xc1, + 0xe7, 0xf0, 0xa9, 0xe8, 0x23, 0xc8, 0x33, 0xf7, 0xd3, 0xbb, 0x56, 0x90, 0x86, 0xf3, 0xaf, 0xce, + 0x7f, 0x79, 0xba, 0x98, 0xa3, 0x3e, 0xa7, 0x7b, 0xca, 0xb3, 0x94, 0x6b, 0x07, 0x1f, 0xc5, 0x5f, + 0x13, 0x60, 0x2e, 0xb4, 0xf3, 0xea, 0x71, 0xdf, 0xc7, 0xef, 0x2b, 0xd1, 0xb8, 0x75, 0x58, 0xec, + 0x63, 0x9a, 0xf7, 0x73, 0x37, 0x90, 0x74, 0x71, 0x38, 0xee, 0xa1, 0xab, 0x38, 0x6e, 0x54, 0x8d, + 0xcb, 0xf6, 0x71, 0xea, 0x07, 0x3f, 0x5c, 0x9c, 0x12, 0xff, 0x2c, 0x05, 0xb9, 0xf0, 0x3e, 0xab, + 0x7a, 0x5f, 0xb9, 0xa2, 0xe6, 0x95, 0x10, 0xc7, 0xd2, 0x88, 0xbb, 0x66, 0x32, 0xfe, 0x75, 0x0d, + 0xac, 0x98, 0xd7, 0x46, 0x7c, 0xe2, 0x0f, 0x96, 0xd3, 0x67, 0x2c, 0xff, 0xbb, 0xa4, 0x67, 0x5f, + 0x97, 0x60, 0x9a, 0x1e, 0x99, 0xc4, 0x8b, 0x56, 0xea, 0xef, 0x3d, 0xc4, 0xd7, 0x26, 0xe9, 0x12, + 0xcb, 0x46, 0xec, 0x71, 0xf3, 0x8d, 0xce, 0x24, 0xf4, 0x87, 0xc1, 0xf9, 0x6f, 0x71, 0xec, 0xb1, + 0x33, 0x29, 0xff, 0x2f, 0x46, 0xde, 0x90, 0xf7, 0xa1, 0x5f, 0x80, 0x82, 0x6a, 0xb6, 0xdb, 0x6c, + 0xae, 0x65, 0x16, 0x6d, 0xf0, 0x94, 0x1a, 0x5a, 0x04, 0x7e, 0x71, 0xe7, 0x92, 0x77, 0x81, 0xe7, + 0x92, 0xc4, 0x2f, 0xf0, 0x0c, 0x84, 0xf5, 0xe6, 0x3d, 0x61, 0xcc, 0x10, 0xf6, 0x45, 0x18, 0xcf, + 0xbe, 0x49, 0x84, 0x31, 0x8b, 0x4a, 0xe7, 0x3d, 0xef, 0x0f, 0x05, 0x1e, 0xc2, 0xf3, 0xcc, 0x34, + 0x0f, 0x7a, 0x5e, 0x64, 0x70, 0x39, 0x78, 0xc2, 0x64, 0xfa, 0xcb, 0xd3, 0xc5, 0x94, 0xe4, 0x1d, + 0x31, 0x19, 0x35, 0x73, 0x24, 0xbe, 0xda, 0xcc, 0x71, 0x1d, 0xe6, 0xba, 0x16, 0xde, 0xc3, 0x8e, + 0xba, 0x2f, 0x1b, 0xbd, 0x0e, 0xdf, 0x40, 0x94, 0x75, 0x69, 0x5b, 0xbd, 0x0e, 0xba, 0x07, 0x45, + 0x2f, 0x0b, 0xc7, 0xf8, 0xee, 0x11, 0x64, 0x2e, 0x9d, 0xaf, 0x08, 0x88, 0xff, 0x53, 0x80, 0x85, + 0x50, 0x9d, 0xf8, 0x98, 0x5a, 0x87, 0xac, 0x6f, 0x4e, 0xec, 0x92, 0x70, 0xce, 0x38, 0xd9, 0x20, + 0x33, 0x92, 0xe1, 0x92, 0xfb, 0x5a, 0x7a, 0xc5, 0x81, 0x2f, 0x36, 0x71, 0x4e, 0xb1, 0x17, 0x7d, + 0x39, 0x6b, 0x81, 0x17, 0x78, 0x83, 0x2c, 0x39, 0xd1, 0x20, 0x13, 0x7f, 0x53, 0x80, 0x22, 0x7d, + 0xc1, 0x13, 0x8c, 0xb5, 0x58, 0xac, 0x9b, 0x1b, 0x7f, 0x9e, 0x98, 0x7c, 0x8b, 0x50, 0xe8, 0x5a, + 0x96, 0x64, 0xf8, 0x5a, 0x16, 0xf1, 0x87, 0x02, 0xe4, 0xbd, 0x12, 0xb2, 0xab, 0x10, 0x47, 0x1c, + 0x64, 0xfa, 0x66, 0xd7, 0xfd, 0xb9, 0x07, 0xae, 0x4c, 0x74, 0x3b, 0x63, 0xf0, 0xc0, 0x15, 0x76, + 0x4d, 0xdd, 0xdf, 0x70, 0x7b, 0x0e, 0x29, 0x62, 0xc5, 0x3f, 0x58, 0xe3, 0x0d, 0x76, 0x4b, 0x49, + 0xf4, 0x16, 0x59, 0xb3, 0x7d, 0xc8, 0xce, 0xc8, 0x99, 0xc8, 0xec, 0x21, 0x1e, 0xb8, 0x06, 0x7c, + 0x35, 0x50, 0x6b, 0x36, 0xe8, 0xfd, 0xb2, 0xec, 0xb7, 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x36, 0x3e, + 0xd1, 0xd2, 0x44, 0xa6, 0xd8, 0xd5, 0x12, 0xeb, 0x2b, 0x7f, 0x10, 0x6c, 0x89, 0xea, 0x21, 0x41, + 0x81, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x45, 0x80, 0x85, 0x5a, 0x4e, 0x22, 0xb9, 0xd1, 0x93, + 0xd0, 0x79, 0x24, 0x89, 0xe1, 0x88, 0x65, 0x50, 0xa5, 0xa1, 0x73, 0x4b, 0x3e, 0x0c, 0xf7, 0xf5, + 0x91, 0xaf, 0x0f, 0x76, 0xfa, 0x8f, 0x53, 0x3f, 0xfa, 0xe1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, + 0x36, 0x76, 0x5e, 0xf4, 0x4c, 0xcb, 0x3f, 0xdb, 0xe5, 0x76, 0xdf, 0x3d, 0x35, 0xd3, 0xab, 0xd9, + 0xb3, 0xa8, 0xe3, 0x9f, 0x2e, 0xc2, 0x42, 0x88, 0x9b, 0x19, 0x0b, 0xf1, 0x43, 0xb8, 0xf2, 0xd4, + 0xb4, 0x6d, 0xbd, 0x4b, 0xa0, 0x2f, 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xcc, 0x63, 0xba, 0x4b, 0x17, + 0x41, 0x0c, 0x66, 0x46, 0x32, 0x92, 0xf7, 0x2c, 0xfe, 0xae, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, + 0xa3, 0x36, 0x77, 0xce, 0xaa, 0xa6, 0x7f, 0x16, 0xe0, 0xf8, 0xde, 0xea, 0x66, 0x27, 0x2e, 0x2a, + 0x7f, 0xa7, 0xdc, 0x51, 0xa8, 0xf9, 0xe0, 0x1b, 0xcd, 0xf3, 0x9c, 0xbc, 0xc9, 0xa8, 0xbe, 0x25, + 0x49, 0x4d, 0x66, 0x49, 0x9a, 0x50, 0x58, 0x37, 0x75, 0x83, 0x78, 0xc2, 0x6e, 0x7d, 0x57, 0x20, + 0xbf, 0xab, 0x1b, 0x8a, 0x75, 0x22, 0xbb, 0x81, 0x87, 0xc2, 0xb8, 0xc0, 0x43, 0x29, 0xc7, 0x38, + 0xf8, 0xa3, 0xf8, 0x63, 0x01, 0x8a, 0xbe, 0x58, 0x6e, 0x91, 0xdf, 0x05, 0x50, 0xdb, 0x3d, 0xdb, + 0xc1, 0x96, 0xdb, 0x4a, 0x73, 0x2c, 0x50, 0xbf, 0xc2, 0xa8, 0xf5, 0x35, 0x29, 0xc3, 0x33, 0xd4, + 0x35, 0x74, 0x23, 0x7c, 0x0e, 0xc6, 0xf4, 0x2a, 0x9c, 0x0d, 0x9c, 0x7e, 0x41, 0x9a, 0xdd, 0x76, + 0x4c, 0xcb, 0xc3, 0x5c, 0xbc, 0xd9, 0xdd, 0x93, 0xfd, 0xe9, 0xf6, 0x6d, 0x92, 0x6f, 0x05, 0xf2, + 0xc4, 0x5d, 0x38, 0xc4, 0x5e, 0x95, 0x52, 0xe3, 0xab, 0xc4, 0x38, 0xdc, 0x2a, 0xfd, 0x73, 0xe2, + 0xe4, 0xb2, 0xd6, 0xf0, 0x5a, 0x78, 0x84, 0x45, 0x5b, 0x83, 0xb4, 0x73, 0x6c, 0xc8, 0x1d, 0xec, + 0xdd, 0x86, 0x73, 0x8e, 0x83, 0xfa, 0x66, 0x1d, 0xf6, 0x48, 0x2f, 0x4c, 0xe4, 0xb7, 0x75, 0xf3, + 0xe1, 0x72, 0x65, 0x89, 0x5d, 0xe7, 0xbd, 0xe4, 0x5e, 0xe7, 0xbd, 0xb4, 0xc6, 0x33, 0x30, 0xa3, + 0xfe, 0x83, 0xff, 0xbc, 0x28, 0x48, 0x1e, 0x13, 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, + 0x99, 0x51, 0x1e, 0x20, 0x70, 0xcd, 0x11, 0xbf, 0xc6, 0x79, 0x65, 0x4d, 0xde, 0xd9, 0xaa, 0x3c, + 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x17, 0xba, 0x24, 0x29, 0xc1, + 0x2e, 0x76, 0xbe, 0xff, 0xff, 0x01, 0xf8, 0xf7, 0xad, 0x11, 0x59, 0x1b, 0xd5, 0xcf, 0xe4, 0x97, + 0x2b, 0xcf, 0x76, 0xaa, 0x8d, 0xe2, 0x14, 0x42, 0x90, 0x5f, 0x5d, 0x69, 0x56, 0x6a, 0xb2, 0x54, + 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0x0b, 0xa1, 0xef, 0xaf, 0xc1, 0x5c, 0xf0, 0xb4, 0x23, + 0xb4, 0x00, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe4, 0x17, 0x3b, 0xd5, 0x9d, + 0x6a, 0x71, 0x8a, 0x16, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, 0xec, + 0x99, 0x5e, 0xa8, 0x54, 0x4c, 0xdc, 0xdf, 0x84, 0x6c, 0xe0, 0x58, 0x64, 0xf2, 0xba, 0xed, 0x9d, + 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xca, 0xe6, 0x36, 0x93, 0x41, 0x69, 0x2b, 0xab, + 0xcf, 0xa5, 0x66, 0x51, 0xf0, 0x9e, 0x9b, 0xcf, 0x77, 0x2a, 0x35, 0xb7, 0x1a, 0x62, 0x2a, 0x9d, + 0x2c, 0x26, 0xef, 0xff, 0x25, 0x01, 0x2e, 0x0f, 0x39, 0xf9, 0x07, 0x65, 0x61, 0x76, 0xc7, 0xa0, + 0x67, 0xc4, 0x16, 0xa7, 0x50, 0x2e, 0x70, 0xf8, 0x4f, 0x51, 0x40, 0x69, 0x76, 0xfc, 0x4a, 0x31, + 0x81, 0x66, 0x20, 0xd1, 0x78, 0x54, 0x4c, 0x92, 0x92, 0x06, 0xce, 0xce, 0x29, 0xa6, 0x50, 0x86, + 0x9f, 0xda, 0x51, 0x9c, 0x46, 0x73, 0xfe, 0xe1, 0x19, 0xc5, 0x19, 0x22, 0xca, 0x3b, 0x84, 0xa2, + 0x38, 0x7b, 0xff, 0x3a, 0x04, 0x36, 0xfa, 0x23, 0x80, 0x99, 0x67, 0x8a, 0x83, 0x6d, 0xa7, 0x38, + 0x85, 0x66, 0x21, 0xb9, 0xd2, 0x6e, 0x17, 0x85, 0x87, 0xff, 0x2c, 0x05, 0x69, 0xf7, 0xe2, 0x20, + 0xf4, 0x0c, 0xa6, 0xd9, 0x72, 0xf8, 0xe2, 0x70, 0xb4, 0x40, 0x07, 0x74, 0xf9, 0xda, 0x38, 0x38, + 0x21, 0x4e, 0xa1, 0xbf, 0x00, 0xd9, 0x80, 0x17, 0x85, 0x86, 0x2e, 0xe9, 0x85, 0x3c, 0xc7, 0xf2, + 0xed, 0x71, 0xd9, 0x3c, 0xf9, 0xaf, 0x20, 0xe3, 0x59, 0x75, 0x74, 0x63, 0x94, 0xcd, 0x77, 0x65, + 0x8f, 0x9e, 0x18, 0xc8, 0xf8, 0x13, 0xa7, 0xde, 0x17, 0x90, 0x05, 0x68, 0xd0, 0x00, 0xa3, 0xa8, + 0x20, 0x8b, 0xa1, 0x16, 0xbe, 0x7c, 0x7f, 0xa2, 0xdc, 0xfe, 0x3b, 0x89, 0xb2, 0xfc, 0x59, 0x24, + 0x5a, 0x59, 0x03, 0x73, 0x54, 0xb4, 0xb2, 0x22, 0x26, 0xa3, 0x29, 0xf4, 0x02, 0x52, 0xc4, 0x7a, + 0xa2, 0x28, 0xbf, 0xb2, 0xcf, 0x5a, 0x97, 0x6f, 0x8c, 0xcc, 0xe3, 0x8a, 0x5c, 0xbd, 0xf7, 0xa3, + 0x3f, 0xb9, 0x3a, 0xf5, 0xa3, 0xb3, 0xab, 0xc2, 0x8f, 0xcf, 0xae, 0x0a, 0x7f, 0x74, 0x76, 0x55, + 0xf8, 0xe3, 0xb3, 0xab, 0xc2, 0xf7, 0x7f, 0x72, 0x75, 0xea, 0xc7, 0x3f, 0xb9, 0x3a, 0xf5, 0x47, + 0x3f, 0xb9, 0x3a, 0xf5, 0xf9, 0x2c, 0xe7, 0xde, 0x9d, 0xa1, 0xa6, 0xe5, 0xd1, 0xff, 0x09, 0x00, + 0x00, 0xff, 0xff, 0xaf, 0x6d, 0x62, 0x22, 0x64, 0x81, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 066f158b2980..63421d4fdf7e 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2203,14 +2203,14 @@ service Internal { // ContentionEvent is a message that will be attached to BatchResponses // indicating any conflicts with another transaction during replica evaluation. -// This message is currently not emitted and only exists for SQL Execution to -// have a protobuf to work with to build the higher-level infrastructure around -// contention observability while the work to emit these events is ongoing. message ContentionEvent { + option (gogoproto.goproto_stringer) = false; + // Key is the key that this and the other transaction conflicted on. bytes key = 1 [(gogoproto.casttype) = "Key"]; - // Txn is the other transaction. - Transaction txn = 2 [(gogoproto.nullable) = false]; + // TxnMeta is the transaction conflicted + // with, i.e. the transaction holding a lock. + cockroach.storage.enginepb.TxnMeta txn_meta = 2 [(gogoproto.nullable) = false]; // Duration spent contending against the other transaction. google.protobuf.Duration duration = 3 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index 6c26f6d638ac..cbaf8af27b50 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -14,6 +14,9 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -237,3 +240,12 @@ func TestMustSetInner(t *testing.T) { t.Fatalf("unexpected response union: %+v", res) } } + +func TestContentionEvent_SafeFormat(t *testing.T) { + ce := &ContentionEvent{ + Key: Key("foo"), + TxnMeta: enginepb.TxnMeta{ID: uuid.FromStringOrNil("51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27")}, + } + const exp = redact.RedactableString(`conflicted with ‹51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27› on ‹"foo"› for 0.00s`) + require.Equal(t, exp, redact.Sprint(ce)) +} diff --git a/pkg/sql/contention/registry.go b/pkg/sql/contention/registry.go index 6a666fd30057..42de040f65be 100644 --- a/pkg/sql/contention/registry.go +++ b/pkg/sql/contention/registry.go @@ -116,7 +116,7 @@ type indexMapValue struct { // initialized with that event's data. func newIndexMapValue(c roachpb.ContentionEvent) *indexMapValue { txnCache := cache.NewUnorderedCache(txnCacheCfg) - txnCache.Add(c.Txn.TxnMeta.ID, 1) + txnCache.Add(c.TxnMeta.ID, 1) keyMap := cache.NewOrderedCache(orderedKeyMapCfg) keyMap.Add(comparableKey(c.Key), txnCache) return &indexMapValue{ @@ -134,7 +134,7 @@ func (v *indexMapValue) addContentionEvent(c roachpb.ContentionEvent) { var numTimesThisTxnWasEncountered int txnCache, ok := v.orderedKeyMap.Get(comparableKey(c.Key)) if ok { - if txnVal, ok := txnCache.(*cache.UnorderedCache).Get(c.Txn.TxnMeta.ID); ok { + if txnVal, ok := txnCache.(*cache.UnorderedCache).Get(c.TxnMeta.ID); ok { numTimesThisTxnWasEncountered = txnVal.(int) } } else { @@ -142,7 +142,7 @@ func (v *indexMapValue) addContentionEvent(c roachpb.ContentionEvent) { txnCache = cache.NewUnorderedCache(txnCacheCfg) v.orderedKeyMap.Add(comparableKey(c.Key), txnCache) } - txnCache.(*cache.UnorderedCache).Add(c.Txn.TxnMeta.ID, numTimesThisTxnWasEncountered+1) + txnCache.(*cache.UnorderedCache).Add(c.TxnMeta.ID, numTimesThisTxnWasEncountered+1) } // indexMap is a helper struct that wraps an LRU cache sized up to diff --git a/pkg/sql/contention/registry_test.go b/pkg/sql/contention/registry_test.go index 2254e78f2c70..654a16eb2f17 100644 --- a/pkg/sql/contention/registry_test.go +++ b/pkg/sql/contention/registry_test.go @@ -107,10 +107,8 @@ func TestRegistry(t *testing.T) { keyBytes = encoding.EncodeStringAscending(keyBytes, key) if err := registry.AddContentionEvent(roachpb.ContentionEvent{ Key: keyBytes, - Txn: roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - ID: contendingTxnID, - }, + TxnMeta: enginepb.TxnMeta{ + ID: contendingTxnID, }, Duration: time.Duration(contentionDuration), }); err != nil { diff --git a/pkg/sql/logictest/testdata/logic_test/contention_event b/pkg/sql/logictest/testdata/logic_test/contention_event new file mode 100644 index 000000000000..9b46cec3953f --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/contention_event @@ -0,0 +1,63 @@ +# Verify that ContentionEvents are emitted. This is mostly a sanity check - look +# for the datadriven tests in `pkg/kv/kvserver/concurrency` for the actual events +# that do get emitted in various contention scenarios. + +statement ok +GRANT ADMIN TO testuser + +statement ok +CREATE TABLE kv (k VARCHAR PRIMARY KEY, v VARCHAR) + + +query TT +SELECT * FROM kv +---- + +user testuser + +statement ok +BEGIN; + +statement ok +INSERT INTO kv VALUES('k', 'v') + +user root + +statement ok +SET tracing=on + +statement ok +SET TRACING=on; +BEGIN; +SET TRANSACTION PRIORITY HIGH; +INSERT INTO kv VALUES('k', 'my v'); +COMMIT; +SET TRACING=off; + +# Check for the message emitted by (roachpb.ContentionEvent).String() as proof that a ContentionEvent +# payload was added to the trace. +# +# NB: if the matcher here ever gets updated, make sure to update throughout this file. +query I +SELECT count(message) FROM [ SHOW TRACE FOR SESSION ] WHERE message LIKE '%conflicted with % on % for %' +---- +1 + +# Clean up. +user testuser + +statement ok +ROLLBACK + +# Do a straightforward non-conflicting insert; we want to check that there's *no* ContentionEvent +# emitted for those. There's no reason to believe it would be - this is a pure sanity check. + +statement ok +SET TRACING=on; +INSERT INTO kv VALUES ('l', 'lll'); +SET TRACING=off; + +query I +SELECT count(message) FROM [ SHOW TRACE FOR SESSION ] WHERE message LIKE '%conflicted with % on % for %' +---- +0 diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index 9ce2a3ac8126..6c3e399a20df 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -12,6 +12,7 @@ package tracing import ( "fmt" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -335,6 +336,11 @@ func (s *Span) LogStructured(item Structured) { return } s.crdb.logStructured(item) + if s.hasVerboseSink() { + // NB: TrimSpace avoids the trailing whitespace + // generated by the protobuf stringers. + s.Record(strings.TrimSpace(item.String())) + } } // Record provides a way to record free-form text into verbose spans.