From dc689ae9bdef495533c5aef2cc294485186201d3 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 5 Jan 2021 11:50:36 +0100 Subject: [PATCH] concurrency: emit structured contention information to trace This change attaches a protobuf payload to the current Span whenever a request conflicts with another transaction. The payload contains the contending txn (i.e. the pushee) at the time at which it was first encountered, the key on which the conflict occurred (note that this is not necessarily the key at which the pushee is anchored) and the time spent waiting on the conflict (excluding intent resolution). This enables #57114. Touches #55583. I am not closing that issue yet because we also want to potentially track the outcome of the conflict. Release note: None --- pkg/kv/kvserver/concurrency/BUILD.bazel | 1 + .../concurrency/concurrency_manager.go | 2 + .../concurrency/concurrency_manager_test.go | 68 +- pkg/kv/kvserver/concurrency/lock_table.go | 16 +- .../kvserver/concurrency/lock_table_waiter.go | 87 ++ .../concurrency/lock_table_waiter_test.go | 53 +- .../testdata/concurrency_manager/basic | 4 + .../clear_abandoned_intents | 7 + .../testdata/concurrency_manager/deadlocks | 22 + .../discover_lock_after_lease_race | 2 + .../concurrency_manager/discovered_lock | 1 + .../concurrency_manager/range_state_listener | 7 + .../testdata/concurrency_manager/uncertainty | 3 + .../testdata/concurrency_manager/update | 4 + .../concurrency_manager/wait_policy_error | 5 + .../testdata/concurrency_manager/wait_self | 128 ++ pkg/roachpb/api.go | 11 + pkg/roachpb/api.pb.go | 1339 ++++++++--------- pkg/roachpb/api.proto | 10 +- pkg/roachpb/api_test.go | 12 + pkg/sql/contention/registry.go | 6 +- pkg/sql/contention/registry_test.go | 6 +- .../testdata/logic_test/contention_event | 63 + 23 files changed, 1151 insertions(+), 706 deletions(-) create mode 100644 pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self create mode 100644 pkg/sql/logictest/testdata/logic_test/contention_event 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