diff --git a/pkg/cmd/roachtest/kv.go b/pkg/cmd/roachtest/kv.go index d864c6eaa46e..a52b57832e53 100644 --- a/pkg/cmd/roachtest/kv.go +++ b/pkg/cmd/roachtest/kv.go @@ -35,13 +35,17 @@ func registerKV(r *testRegistry) { nodes int cpus int readPercent int - batchSize int - blockSize int - splits int // 0 implies default, negative implies 0 - encryption bool - sequential bool - duration time.Duration - tags []string + // If true, the reads are limited reads over the full span of the table. + // Currently this also enables SFU writes on the workload since this is + // geared towards testing optimistic locking and latching. + spanReads bool + batchSize int + blockSize int + splits int // 0 implies default, negative implies 0 + encryption bool + sequential bool + duration time.Duration + tags []string } computeNumSplits := func(opts kvOptions) int { // TODO(ajwerner): set this default to a more sane value or remove it and @@ -62,6 +66,16 @@ func registerKV(r *testRegistry) { c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes), startArgs(fmt.Sprintf("--encrypt=%t", opts.encryption))) + if opts.splits < 0 { + // In addition to telling the workload to not split, disable load-based + // splitting. + db := c.Conn(ctx, 1) + defer db.Close() + if _, err := db.ExecContext(ctx, "SET CLUSTER SETTING kv.range_split.by_load_enabled = 'false'"); err != nil { + t.Fatalf("failed to disable load based splitting: %v", err) + } + } + t.Status("running workload") m := newMonitor(ctx, c, c.Range(1, nodes)) m.Go(func(ctx context.Context) error { @@ -72,7 +86,17 @@ func registerKV(r *testRegistry) { opts.duration = 10 * time.Minute } duration := " --duration=" + ifLocal("10s", opts.duration.String()) - readPercent := fmt.Sprintf(" --read-percent=%d", opts.readPercent) + var readPercent string + if opts.spanReads { + // SFU makes sense only if we repeat writes to the same key. Here + // we've arbitrarily picked a cycle-length of 1000, so 1 in 1000 + // writes will contend with the limited scan wrt locking. + readPercent = + fmt.Sprintf(" --span-percent=%d --span-limit=1 --sfu-writes=true --cycle-length=1000", + opts.readPercent) + } else { + readPercent = fmt.Sprintf(" --read-percent=%d", opts.readPercent) + } histograms := " --histograms=" + perfArtifactsDir + "/stats.json" var batchSize string if opts.batchSize > 0 { @@ -143,6 +167,10 @@ func registerKV(r *testRegistry) { {nodes: 3, cpus: 32, readPercent: 0, sequential: true}, {nodes: 3, cpus: 32, readPercent: 95, sequential: true}, + // Configs with reads, that are of limited spans, along with SFU writes. + {nodes: 1, cpus: 8, readPercent: 95, spanReads: true, splits: -1 /* no splits */, sequential: true}, + {nodes: 1, cpus: 32, readPercent: 95, spanReads: true, splits: -1 /* no splits */, sequential: true}, + // Weekly larger scale configurations. {nodes: 32, cpus: 8, readPercent: 0, tags: []string{"weekly"}, duration: time.Hour}, {nodes: 32, cpus: 8, readPercent: 95, tags: []string{"weekly"}, duration: time.Hour}, @@ -150,7 +178,11 @@ func registerKV(r *testRegistry) { opts := opts var nameParts []string - nameParts = append(nameParts, fmt.Sprintf("kv%d", opts.readPercent)) + var limitedSpanStr string + if opts.spanReads { + limitedSpanStr = "limited-spans" + } + nameParts = append(nameParts, fmt.Sprintf("kv%d%s", opts.readPercent, limitedSpanStr)) if len(opts.tags) > 0 { nameParts = append(nameParts, strings.Join(opts.tags, "/")) } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 58da6d76bcdc..3b8b8dff57bf 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -53,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/kr/pretty" @@ -3786,3 +3787,196 @@ func TestRaftSchedulerPrioritizesNodeLiveness(t *testing.T) { priorityID := store.RaftSchedulerPriorityID() require.Equal(t, livenessRangeID, priorityID) } + +func setupDBAndWriteAAndB(t *testing.T) (serverutils.TestServerInterface, *kv.DB) { + ctx := context.Background() + args := base.TestServerArgs{} + s, _, db := serverutils.StartServer(t, args) + + require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + if err := txn.Put(ctx, "a", "a"); err != nil { + return err + } + if err := txn.Put(ctx, "b", "b"); err != nil { + return err + } + return txn.Commit(ctx) + })) + tup, err := db.Get(ctx, "a") + require.NoError(t, err) + require.NotNil(t, tup.Value) + tup, err = db.Get(ctx, "b") + require.NoError(t, err) + require.NotNil(t, tup.Value) + return s, db +} + +// TestOptimisticEvalRetry tests the case where an optimistically evaluated +// scan encounters contention from a concurrent txn holding unreplicated +// exclusive locks, and therefore re-evaluates pessimistically, and eventually +// succeeds once the contending txn commits. +func TestOptimisticEvalRetry(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, db := setupDBAndWriteAAndB(t) + defer s.Stopper().Stop(ctx) + + txn1 := db.NewTxn(ctx, "locking txn") + _, err := txn1.ScanForUpdate(ctx, "a", "c", 0) + require.NoError(t, err) + + readDone := make(chan error) + go func() { + readDone <- db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + // We can't actually prove that the optimistic evaluation path was + // taken, but it should happen based on the fact that this is a limited + // scan with a limit of 1 row, and the replica has 2 rows. + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + return err + } + return txn.Commit(ctx) + }) + }() + removedLocks := false + timer := timeutil.NewTimer() + timer.Reset(time.Second * 2) + defer timer.Stop() + done := false + for !done { + select { + case err := <-readDone: + if !removedLocks { + t.Fatal("read completed before exclusive locks were released") + } + require.NoError(t, err) + require.True(t, removedLocks) + done = true + case <-timer.C: + require.NoError(t, txn1.Commit(ctx)) + removedLocks = true + } + } +} + +// TestOptimisticEvalNoContention tests the case where an optimistically +// evaluated scan has a span that overlaps with a concurrent txn holding +// unreplicated exclusive locks, but the actual span that is read does not +// overlap, and therefore the scan succeeds before the lock holding txn +// commits. +func TestOptimisticEvalNoContention(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, db := setupDBAndWriteAAndB(t) + defer s.Stopper().Stop(ctx) + + txn1 := db.NewTxn(ctx, "locking txn") + _, err := txn1.ScanForUpdate(ctx, "b", "c", 0) + require.NoError(t, err) + + readDone := make(chan error) + go func() { + readDone <- db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + // There is no contention when doing optimistic evaluation, since it can read a + // which is not locked. + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + return err + } + return txn.Commit(ctx) + }) + }() + err = <-readDone + require.NoError(t, err) + require.NoError(t, txn1.Commit(ctx)) +} + +func BenchmarkOptimisticEval(b *testing.B) { + defer log.Scope(b).Close(b) + ctx := context.Background() + args := base.TestServerArgs{} + s, _, db := serverutils.StartServer(b, args) + defer s.Stopper().Stop(ctx) + + require.NoError(b, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + b.Log(err) + }() + if err := txn.Put(ctx, "a", "a"); err != nil { + return err + } + if err := txn.Put(ctx, "b", "b"); err != nil { + return err + } + return txn.Commit(ctx) + })) + tup, err := db.Get(ctx, "a") + require.NoError(b, err) + require.NotNil(b, tup.Value) + tup, err = db.Get(ctx, "b") + require.NoError(b, err) + require.NotNil(b, tup.Value) + + for _, realContention := range []bool{false, true} { + b.Run(fmt.Sprintf("real-contention=%t", realContention), + func(b *testing.B) { + lockStart := "b" + if realContention { + lockStart = "a" + } + finishWrites := make(chan struct{}) + var writers sync.WaitGroup + for i := 0; i < 1; i++ { + writers.Add(1) + go func() { + for { + txn := db.NewTxn(ctx, "locking txn") + _, err = txn.ScanForUpdate(ctx, lockStart, "c", 0) + require.NoError(b, err) + time.Sleep(5 * time.Millisecond) + // Normally, it would do a write here, but we don't bother. + require.NoError(b, txn.Commit(ctx)) + select { + case _, recv := <-finishWrites: + if !recv { + writers.Done() + return + } + default: + } + } + }() + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + _ = db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + panic(err) + } + err = txn.Commit(ctx) + if err != nil { + panic(err) + } + return err + }) + } + b.StopTimer() + close(finishWrites) + writers.Wait() + }) + } +} diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index 7603c9a13312..ca1723691bd7 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -160,15 +160,25 @@ type RequestSequencer interface { // SequenceReq acquires latches, checks for locks, and queues behind and/or // pushes other transactions to resolve any conflicts. Once sequenced, the // request is guaranteed sufficient isolation for the duration of its - // evaluation, until the returned request guard is released. - // NOTE: this last part will not be true until replicated locks are pulled - // into the concurrency manager. + // evaluation, until the returned request guard is released. NOTE: this last + // part will not be true until replicated locks are pulled into the + // concurrency manager. This is the normal behavior for a request marked as + // PessimisticEval. For OptimisticEval, it can optimize by not acquiring + // locks, and the request must call Guard.CheckOptimisticNoConflicts after + // evaluation. OptimisticEval is only permitted in the first call to + // SequenceReq for a Request. A failed OptimisticEval must use + // PessimisticAfterFailedOptimisticEval, for the immediately following + // SequenceReq call, if the latches are already held. + // TODO(sumeer): change OptimisticEval to only queue the latches and not + // wait for them, so PessimisticAfterFailedOptimisticEval will wait for + // them. // // An optional existing request guard can be provided to SequenceReq. This // allows the request's position in lock wait-queues to be retained across // sequencing attempts. If provided, the guard should not be holding latches - // already. The expected usage of this parameter is that it will only be - // provided after acquiring a Guard from a ContentionHandler method. + // already (for PessimisticEval). The expected usage of this parameter is + // that it will only be provided after acquiring a Guard from a + // ContentionHandler method. // // If the method returns a non-nil request guard then the caller must ensure // that the guard is eventually released by passing it to FinishReq. @@ -176,7 +186,7 @@ type RequestSequencer interface { // Alternatively, the concurrency manager may be able to serve the request // directly, in which case it will return a Response for the request. If it // does so, it will not return a request guard. - SequenceReq(context.Context, *Guard, Request) (*Guard, Response, *Error) + SequenceReq(context.Context, *Guard, Request, RequestEvalKind) (*Guard, Response, *Error) // FinishReq marks the request as complete, releasing any protection // the request had against conflicting requests and allowing conflicting @@ -308,6 +318,32 @@ type MetricExporter interface { // External API Type Definitions // /////////////////////////////////// +// RequestEvalKind informs the manager of the evaluation kind for the current +// evaluation attempt. Optimistic evaluation is used for requests involving +// limited scans, where the checking of locks and latches may be (partially) +// postponed until after evaluation, once the limit has been applied and the +// key spans have been constrained, using Guard.CheckOptimisticNoConflicts. +// Note that intents (replicated single-key locks) will still be observed +// during evaluation. +// +// The setting can change across different calls to SequenceReq. The +// permissible sequences are: +// - OptimisticEval: when optimistic evaluation succeeds. +// - OptimisticEval, PessimisticAfterFailedOptimisticEval, PessimisticEval*: +// when optimistic evaluation failed. +// - PessimisticEval+: when only pessimistic evaluation was attempted. +type RequestEvalKind int + +const ( + // PessimisticEval represents pessimistic locking. + PessimisticEval RequestEvalKind = iota + // OptimisticEval represents optimistic locking. + OptimisticEval + // PessimisticAfterFailedOptimisticEval represents a request sequencing + // attempt immediately following a failed OptimisticEval. + PessimisticAfterFailedOptimisticEval +) + // Request is the input to Manager.SequenceReq. The struct contains all of the // information necessary to sequence a KV request and determine which locks and // other in-flight requests it conflicts with. @@ -359,6 +395,8 @@ type Guard struct { Req Request lg latchGuard ltg lockTableGuard + // The latest RequestEvalKind passed to SequenceReq. + EvalKind RequestEvalKind } // Response is a slice of responses to requests in a batch. This type is used @@ -460,6 +498,13 @@ type lockTable interface { // function. ScanAndEnqueue(Request, lockTableGuard) lockTableGuard + // ScanOptimistic takes a snapshot of the lock table for later checking for + // conflicts, and returns a guard. It is for optimistic evaluation of + // requests that will typically scan a small subset of the spans mentioned + // in the Request. After Request evaluation, CheckOptimisticNoConflicts + // must be called on the guard. + ScanOptimistic(Request) lockTableGuard + // Dequeue removes the request from its lock wait-queues. It should be // called when the request is finished, whether it evaluated or not. The // guard should not be used after being dequeued. @@ -608,6 +653,15 @@ type lockTableGuard interface { // - if locks were discovered during evaluation, it must be called after all // the discovered locks have been added. ResolveBeforeScanning() []roachpb.LockUpdate + + // CheckOptimisticNoConflicts uses the SpanSet representing the spans that + // were actually read, to check for conflicting locks, after an optimistic + // evaluation. It returns true if there were no conflicts. See + // lockTable.ScanOptimistic for context. Note that the evaluation has + // already seen any intents (replicated single-key locks) that conflicted, + // so this checking is practically only going to find unreplicated locks + // that conflict. + CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) } // lockTableWaiter is concerned with waiting in lock wait-queues for locks held diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index c4bad6ce5e67..3b671218588d 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" ) // DiscoveredLocksThresholdToConsultFinalizedTxnCache sets a threshold as @@ -129,18 +130,35 @@ func NewManager(cfg Config) Manager { // SequenceReq implements the RequestSequencer interface. func (m *managerImpl) SequenceReq( - ctx context.Context, prev *Guard, req Request, + ctx context.Context, prev *Guard, req Request, evalKind RequestEvalKind, ) (*Guard, Response, *Error) { var g *Guard if prev == nil { + switch evalKind { + case PessimisticEval: + log.Event(ctx, "sequencing request") + case OptimisticEval: + log.Event(ctx, "optimistically sequencing request") + case PessimisticAfterFailedOptimisticEval: + panic("retry should have non-nil guard") + } g = newGuard(req) - log.Event(ctx, "sequencing request") } else { g = prev - g.AssertNoLatches() - log.Event(ctx, "re-sequencing request") + switch evalKind { + case PessimisticEval: + g.AssertNoLatches() + log.Event(ctx, "re-sequencing request") + case OptimisticEval: + panic("optimistic eval cannot happen when re-sequencing") + case PessimisticAfterFailedOptimisticEval: + if shouldAcquireLatches(req) { + g.AssertLatches() + } + log.Event(ctx, "re-sequencing request after optimistic sequencing failed") + } } - + g.EvalKind = evalKind resp, err := m.sequenceReqWithGuard(ctx, g, req) if resp != nil || err != nil { // Ensure that we release the guard if we return a response or an error. @@ -150,6 +168,8 @@ func (m *managerImpl) SequenceReq( return g, nil, nil } +// TODO(sumeer): we are using both g.Req and req, when the former should +// suffice. Remove the req parameter. func (m *managerImpl) sequenceReqWithGuard( ctx context.Context, g *Guard, req Request, ) (Response, *Error) { @@ -167,13 +187,26 @@ func (m *managerImpl) sequenceReqWithGuard( return resp, err } + // Only the first iteration can sometimes already be holding latches -- we + // use this to assert below. + first := true for { - // Acquire latches for the request. This synchronizes the request - // with all conflicting in-flight requests. - log.Event(ctx, "acquiring latches") - g.lg, err = m.lm.Acquire(ctx, req) - if err != nil { - return nil, err + if !first { + g.AssertNoLatches() + } + first = false + if !g.HoldingLatches() { + // TODO(sumeer): optimistic requests could register their need for + // latches, but not actually wait until acquisition. + // https://github.com/cockroachdb/cockroach/issues/9521 + + // Acquire latches for the request. This synchronizes the request + // with all conflicting in-flight requests. + log.Event(ctx, "acquiring latches") + g.lg, err = m.lm.Acquire(ctx, req) + if err != nil { + return nil, err + } } // Some requests don't want the wait on locks. @@ -181,9 +214,17 @@ func (m *managerImpl) sequenceReqWithGuard( return nil, nil } - // Scan for conflicting locks. - log.Event(ctx, "scanning lock table for conflicting locks") - g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg) + if g.EvalKind == OptimisticEval { + if g.ltg != nil { + panic("Optimistic locking should not have a non-nil lockTableGuard") + } + log.Event(ctx, "optimistically scanning lock table for conflicting locks") + g.ltg = m.lt.ScanOptimistic(g.Req) + } else { + // Scan for conflicting locks. + log.Event(ctx, "scanning lock table for conflicting locks") + g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg) + } // Wait on conflicting locks, if necessary. if g.ltg.ShouldWait() { @@ -490,6 +531,18 @@ func (g *Guard) AssertNoLatches() { } } +// CheckOptimisticNoConflicts checks that the lockSpansRead do not have a +// conflicting lock. +func (g *Guard) CheckOptimisticNoConflicts(lockSpansRead *spanset.SpanSet) (ok bool) { + if g.EvalKind != OptimisticEval { + panic(errors.AssertionFailedf("unexpected EvalKind: %d", g.EvalKind)) + } + if g.ltg == nil { + return true + } + return g.ltg.CheckOptimisticNoConflicts(lockSpansRead) +} + func (g *Guard) moveLatchGuard() latchGuard { lg := g.lg g.lg = nil diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index 9c5649b86bc7..9bcb1f21bddf 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -55,12 +55,14 @@ import ( // new-txn name= ts=[,] epoch= [uncertainty-limit=[,]] // new-request name= txn=|none ts=[,] [priority] [inconsistent] [wait-policy=] // [=...] (hint: see scanSingleRequest) -// sequence req= +// sequence req= [eval-kind= // // handle-write-intent-error req= txn= key= lease-seq= // handle-txn-push-error req= txn= key= TODO(nvanbenschoten): implement this // +// check-opt-no-conflicts req= +// // on-lock-acquired req= key= [seq=] [dur=r|u] // on-lock-updated req= txn= key= status=[committed|aborted|pending] [ts=[,]] // on-txn-updated txn= status=[committed|aborted|pending] [ts=[,]] @@ -153,16 +155,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { waitPolicy := scanWaitPolicy(t, d, false /* required */) // Each roachpb.Request is provided on an indented line. - var reqs []roachpb.Request - singleReqLines := strings.Split(d.Input, "\n") - for _, line := range singleReqLines { - req := scanSingleRequest(t, d, line, c.txnsByName) - reqs = append(reqs, req) - } - reqUnions := make([]roachpb.RequestUnion, len(reqs)) - for i, req := range reqs { - reqUnions[i].MustSetInner(req) - } + reqs, reqUnions := scanRequests(t, d, c) latchSpans, lockSpans := c.collectSpans(t, txn, ts, reqs) c.requestsByName[reqName] = testReq{ @@ -185,7 +178,21 @@ func TestConcurrencyManagerBasic(t *testing.T) { if !ok { d.Fatalf(t, "unknown request: %s", reqName) } - + evalKind := concurrency.PessimisticEval + if d.HasArg("eval-kind") { + var kind string + d.ScanArgs(t, "eval-kind", &kind) + switch kind { + case "pess": + evalKind = concurrency.PessimisticEval + case "opt": + evalKind = concurrency.OptimisticEval + case "pess-after-opt": + evalKind = concurrency.PessimisticAfterFailedOptimisticEval + default: + d.Fatalf(t, "unknown eval-kind: %s", kind) + } + } c.mu.Lock() prev := c.guardsByReqName[reqName] delete(c.guardsByReqName, reqName) @@ -193,7 +200,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { opName := fmt.Sprintf("sequence %s", reqName) cancel := mon.runAsync(opName, func(ctx context.Context) { - guard, resp, err := m.SequenceReq(ctx, prev, req.Request) + guard, resp, err := m.SequenceReq(ctx, prev, req.Request, evalKind) if err != nil { log.Eventf(ctx, "sequencing complete, returned error: %v", err) } else if resp != nil { @@ -285,6 +292,17 @@ func TestConcurrencyManagerBasic(t *testing.T) { }) return c.waitAndCollect(t, mon) + case "check-opt-no-conflicts": + var reqName string + d.ScanArgs(t, "req", &reqName) + g, ok := c.guardsByReqName[reqName] + if !ok { + d.Fatalf(t, "unknown request: %s", reqName) + } + reqs, _ := scanRequests(t, d, c) + _, lockSpans := c.collectSpans(t, g.Req.Txn, g.Req.Timestamp, reqs) + return fmt.Sprintf("no-conflicts: %t", g.CheckOptimisticNoConflicts(lockSpans)) + case "on-lock-acquired": var reqName string d.ScanArgs(t, "req", &reqName) @@ -498,6 +516,23 @@ func TestConcurrencyManagerBasic(t *testing.T) { }) } +func scanRequests( + t *testing.T, d *datadriven.TestData, c *cluster, +) ([]roachpb.Request, []roachpb.RequestUnion) { + // Each roachpb.Request is provided on an indented line. + var reqs []roachpb.Request + singleReqLines := strings.Split(d.Input, "\n") + for _, line := range singleReqLines { + req := scanSingleRequest(t, d, line, c.txnsByName) + reqs = append(reqs, req) + } + reqUnions := make([]roachpb.RequestUnion, len(reqs)) + for i, req := range reqs { + reqUnions[i].MustSetInner(req) + } + return reqs, reqUnions +} + type testReq struct { cancel func() concurrency.Request diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index efed086f595d..9622f268aecb 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -485,6 +485,33 @@ func (g *lockTableGuardImpl) CurState() waitingState { return g.mu.state } +func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet) (ok bool) { + // Temporarily replace the SpanSet in the guard. + originalSpanSet := g.spans + g.spans = spanSet + g.sa = spanset.NumSpanAccess - 1 + g.ss = spanset.SpanScope(0) + g.index = -1 + defer func() { + g.spans = originalSpanSet + }() + span := stepToNextSpan(g) + for span != nil { + startKey := span.Key + tree := g.tableSnapshot[g.ss] + iter := tree.MakeIter() + ltRange := &lockState{key: startKey, endKey: span.EndKey} + for iter.FirstOverlap(ltRange); iter.Valid(); iter.NextOverlap(ltRange) { + l := iter.Cur() + if !l.isNonConflictingLock(g, g.sa) { + return false + } + } + span = stepToNextSpan(g) + } + return true +} + func (g *lockTableGuardImpl) notify() { select { case g.mu.signal <- struct{}{}: @@ -1447,6 +1474,45 @@ func (l *lockState) tryActiveWait( return true, false } +func (l *lockState) isNonConflictingLock(g *lockTableGuardImpl, sa spanset.SpanAccess) bool { + l.mu.Lock() + defer l.mu.Unlock() + + // It is possible that this lock is empty and has not yet been deleted. + if l.isEmptyLock() { + return true + } + // Lock is not empty. + lockHolderTxn, lockHolderTS := l.getLockHolder() + if lockHolderTxn == nil { + // Reservation holders are non-conflicting. + // + // When optimistic evaluation holds latches, there cannot be a conflicting + // reservation holder that is also holding latches (reservation holder + // without latches can happen due to lock discovery). So after this + // optimistic evaluation succeeds and releases latches, the reservation + // holder will acquire latches and scan the lock table again. When + // optimistic evaluation does not hold latches, it will check for + // conflicting latches before declaring success and a reservation holder + // that holds latches will be discovered, and the optimistic evaluation + // will retry as pessimistic. + return true + } + if g.isSameTxn(lockHolderTxn) { + // Already locked by this txn. + return true + } + // NB: We do not look at the finalizedTxnCache in this optimistic evaluation + // path. A conflict with a finalized txn will be noticed when retrying + // pessimistically. + + if sa == spanset.SpanReadOnly && g.ts.Less(lockHolderTS) { + return true + } + // Conflicts. + return false +} + // Acquires this lock. Returns the list of guards that are done actively // waiting at this key -- these will be requests from the same transaction // that is acquiring the lock. @@ -2067,6 +2133,12 @@ func (t *treeMu) nextLockSeqNum() (seqNum uint64, checkMaxLocks bool) { return t.lockIDSeqNum, checkMaxLocks } +func (t *lockTableImpl) ScanOptimistic(req Request) lockTableGuard { + g := t.newGuardForReq(req) + t.doSnapshotForGuard(g) + return g +} + // ScanAndEnqueue implements the lockTable interface. func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTableGuard { // NOTE: there is no need to synchronize with enabledMu here. ScanAndEnqueue @@ -2077,14 +2149,7 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa var g *lockTableGuardImpl if guard == nil { - g = newLockTableGuardImpl() - g.seqNum = atomic.AddUint64(&t.seqNum, 1) - g.lt = t - g.txn = req.txnMeta() - g.ts = req.Timestamp - g.spans = req.LockSpans - g.sa = spanset.NumSpanAccess - 1 - g.index = -1 + g = t.newGuardForReq(req) } else { g = guard.(*lockTableGuardImpl) g.key = nil @@ -2097,6 +2162,30 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa g.mu.Unlock() g.toResolve = g.toResolve[:0] } + t.doSnapshotForGuard(g) + g.findNextLockAfter(true /* notify */) + if g.notRemovableLock != nil { + // Either waiting at the notRemovableLock, or elsewhere. Either way we are + // making forward progress, which ensures liveness. + g.notRemovableLock.decrementNotRemovable() + g.notRemovableLock = nil + } + return g +} + +func (t *lockTableImpl) newGuardForReq(req Request) *lockTableGuardImpl { + g := newLockTableGuardImpl() + g.seqNum = atomic.AddUint64(&t.seqNum, 1) + g.lt = t + g.txn = req.txnMeta() + g.ts = req.Timestamp + g.spans = req.LockSpans + g.sa = spanset.NumSpanAccess - 1 + g.index = -1 + return g +} + +func (t *lockTableImpl) doSnapshotForGuard(g *lockTableGuardImpl) { for ss := spanset.SpanScope(0); ss < spanset.NumSpanScope; ss++ { for sa := spanset.SpanAccess(0); sa < spanset.NumSpanAccess; sa++ { if len(g.spans.GetSpans(sa, ss)) > 0 { @@ -2112,14 +2201,6 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa } } } - g.findNextLockAfter(true /* notify */) - if g.notRemovableLock != nil { - // Either waiting at the notRemovableLock, or elsewhere. Either way we are - // making forward progress, which ensures liveness. - g.notRemovableLock.decrementNotRemovable() - g.notRemovableLock = nil - } - return g } // Dequeue implements the lockTable interface. @@ -2279,7 +2360,12 @@ func (t *lockTableImpl) AcquireLock( checkMaxLocks := false if !iter.Valid() { if durability == lock.Replicated { - // Don't remember uncontended replicated locks. + // Don't remember uncontended replicated locks. The downside is that + // sometimes contention won't be noticed until when the request + // evaluates. Remembering here would be better, but our behavior when + // running into the maxLocks limit is somewhat crude. Treating the + // data-structure as a bounded cache with eviction guided by contention + // would be better. tree.mu.Unlock() return nil } @@ -2297,6 +2383,9 @@ func (t *lockTableImpl) AcquireLock( // case where the lock is initially added as replicated, we drop // replicated locks from the lockTable when being upgraded from // Unreplicated to Replicated, whenever possible. + // TODO(sumeer): now that limited scans evaluate optimistically, we + // should consider removing this hack. But see the comment in the + // preceding block about maxLocks. tree.Delete(l) tree.mu.Unlock() atomic.AddInt64(&tree.numLocks, -1) diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 25b76ed23f07..9fca4622bab0 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -61,11 +61,18 @@ new-request r= txn=|none ts=[,] spans=r|w@[,]+ scan r= ---- -|start-waiting: +start-waiting: Calls lockTable.ScanAndEnqueue. If the request has an existing guard, uses it. If a guard is returned, stores it for later use. +scan-opt r= +---- +start-waiting: + + Calls lockTable.ScanOptimistic. The request must not have an existing guard. + If a guard is returned, stores it for later use. + acquire r= k= durability=r|u ---- @@ -95,6 +102,12 @@ add-discovered r= k= txn= [lease-seq=] [consult-finalized- Adds a discovered lock that is discovered by the named request. +check-opt-no-conflicts r= spans=r|w@[,]+... +---- +no-conflicts: + + Checks whether the request, which previously called ScanOptimistic, has no lock conflicts. + dequeue r= ---- @@ -262,6 +275,21 @@ func TestLockTableBasic(t *testing.T) { guardsByReqName[reqName] = g return fmt.Sprintf("start-waiting: %t", g.ShouldWait()) + case "scan-opt": + var reqName string + d.ScanArgs(t, "r", &reqName) + req, ok := requestsByName[reqName] + if !ok { + d.Fatalf(t, "unknown request: %s", reqName) + } + _, ok = guardsByReqName[reqName] + if ok { + d.Fatalf(t, "request has an existing guard: %s", reqName) + } + g := lt.ScanOptimistic(req) + guardsByReqName[reqName] = g + return fmt.Sprintf("start-waiting: %t", g.ShouldWait()) + case "acquire": var reqName string d.ScanArgs(t, "r", &reqName) @@ -384,6 +412,20 @@ func TestLockTableBasic(t *testing.T) { } return lt.(*lockTableImpl).String() + case "check-opt-no-conflicts": + var reqName string + d.ScanArgs(t, "r", &reqName) + req, ok := requestsByName[reqName] + if !ok { + d.Fatalf(t, "unknown request: %s", reqName) + } + g := guardsByReqName[reqName] + if g == nil { + d.Fatalf(t, "unknown guard: %s", reqName) + } + spans := scanSpans(t, d, req.Timestamp) + return fmt.Sprintf("no-conflicts: %t", g.CheckOptimisticNoConflicts(spans)) + case "dequeue": var reqName string d.ScanArgs(t, "r", &reqName) diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 9cfd9d90f967..676a63a12885 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -65,6 +65,8 @@ type mockLockTableGuard struct { toResolve []roachpb.LockUpdate } +var _ lockTableGuard = &mockLockTableGuard{} + // mockLockTableGuard implements the lockTableGuard interface. func (g *mockLockTableGuard) ShouldWait() bool { return true } func (g *mockLockTableGuard) NewStateChan() chan struct{} { return g.signal } @@ -78,6 +80,9 @@ func (g *mockLockTableGuard) CurState() waitingState { func (g *mockLockTableGuard) ResolveBeforeScanning() []roachpb.LockUpdate { return g.toResolve } +func (g *mockLockTableGuard) CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) { + return true +} func (g *mockLockTableGuard) notify() { g.signal <- struct{}{} } // mockLockTable overrides TransactionIsFinalized, which is the only LockTable diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic new file mode 100644 index 000000000000..073e596a03df --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic @@ -0,0 +1,88 @@ +new-txn name=txn1 ts=10,1 epoch=0 +---- + +new-txn name=txn2 ts=12,1 epoch=0 +---- + +new-request name=req1 txn=txn1 ts=10,1 + put key=d value=d +---- + +sequence req=req1 +---- +[1] sequence req1: sequencing request +[1] sequence req1: acquiring latches +[1] sequence req1: scanning lock table for conflicting locks +[1] sequence req1: sequencing complete, returned guard + +on-lock-acquired req=req1 key=d +---- +[-] acquire lock: txn 00000001 @ d + +debug-lock-table +---- +global: num=1 + lock: "d" + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +finish req=req1 +---- +[-] finish req1: finishing request + +new-request name=req2 txn=txn2 ts=12,1 + scan key=a endkey=e +---- + +# Optimistic locking for req2 +sequence req=req2 eval-kind=opt +---- +[2] sequence req2: optimistically sequencing request +[2] sequence req2: acquiring latches +[2] sequence req2: optimistically scanning lock table for conflicting locks +[2] sequence req2: sequencing complete, returned guard + +debug-lock-table +---- +global: num=1 + lock: "d" + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +# When checking with a span that does not include the existing lock, there is +# no conflict. +check-opt-no-conflicts req=req2 + scan key=a endkey=c +---- +no-conflicts: true + +# Wider span has a conflict. +check-opt-no-conflicts req=req2 + scan key=a endkey=e +---- +no-conflicts: false + +# Sequence again -- latches are already held. +sequence req=req2 eval-kind=pess-after-opt +---- +[3] sequence req2: re-sequencing request after optimistic sequencing failed +[3] sequence req2: scanning lock table for conflicting locks +[3] sequence req2: waiting in lock wait-queues +[3] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key "d" (queuedWriters: 0, queuedReaders: 1) +[3] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 +[3] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction + +# Conflicting transaction commits. +on-txn-updated txn=txn1 status=committed +---- +[-] update txn: committing txn1 +[3] sequence req2: resolving intent "d" for txn 00000001 with COMMITTED status +[3] sequence req2: lock wait-queue event: done waiting +[3] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "d" for 1.234s +[3] sequence req2: acquiring latches +[3] sequence req2: scanning lock table for conflicting locks +[3] sequence req2: sequencing complete, returned guard + +finish req=req2 +---- +[-] finish req2: finishing request diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic new file mode 100644 index 000000000000..40337a795935 --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic @@ -0,0 +1,111 @@ +new-lock-table maxlocks=10000 +---- + +new-txn txn=txn1 ts=10,1 epoch=0 +---- + +new-txn txn=txn2 ts=11,1 epoch=0 +---- + +# req1 will acquire locks for txn1 + +new-request r=req1 txn=txn1 ts=10,1 spans=w@c,h +---- + +scan r=req1 +---- +start-waiting: false + +should-wait r=req1 +---- +false + +acquire r=req1 k=c durability=u +---- +global: num=1 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +acquire r=req1 k=g durability=u +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +dequeue r=req1 +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +new-request r=req2 txn=txn2 ts=11,1 spans=r@a,d +---- + +scan r=req2 +---- +start-waiting: true + +should-wait r=req2 +---- +true + +dequeue r=req2 +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +new-request r=req3 txn=txn2 ts=11,1 spans=r@a,d+r@f,i +---- + +scan-opt r=req3 +---- +start-waiting: false + +should-wait r=req3 +---- +false + +print +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +check-opt-no-conflicts r=req3 spans=r@a,c +---- +no-conflicts: true + +check-opt-no-conflicts r=req3 spans=r@a,d +---- +no-conflicts: false + +check-opt-no-conflicts r=req3 spans=r@a,c+r@f,g +---- +no-conflicts: true + +check-opt-no-conflicts r=req3 spans=r@a,c+r@f,h +---- +no-conflicts: false + +dequeue r=req3 +---- +global: num=2 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 5b77e3caad10..e3a51cbdfdb5 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -71,7 +71,13 @@ func (r *Replica) executeReadOnlyBatch( // turn means that we can bump the timestamp cache *before* evaluation // without risk of starving writes. Once we start doing that, we're free to // release latches immediately after we acquire an engine iterator as long - // as we're performing a non-locking read. + // as we're performing a non-locking read. Note that this also requires that + // the request is not being optimistically evaluated (optimistic evaluation + // does not check locks). It would also be nice, but not required for + // correctness, that the read-only engine eagerly create an iterator (that + // is later cloned) while the latches are held, so that this request does + // not "see" the effect of any later requests that happen after the latches + // are released. var result result.Result br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes( @@ -79,12 +85,27 @@ func (r *Replica) executeReadOnlyBatch( ) // If the request hit a server-side concurrency retry error, immediately - // proagate the error. Don't assume ownership of the concurrency guard. + // propagate the error. Don't assume ownership of the concurrency guard. if isConcurrencyRetryError(pErr) { pErr = maybeAttachLease(pErr, &st.Lease) return nil, g, pErr } + if pErr == nil && g.EvalKind == concurrency.OptimisticEval { + // Gather the spans that were read -- we distinguish the spans in the + // request from the spans that were actually read, using resume spans in + // the response. For now we ignore the latch spans, but when we stop + // waiting for latches in optimistic evaluation we will use these to check + // latches first. + _, lockSpansRead, err := r.collectSpansRead(ba, br) + if err != nil { + return nil, g, roachpb.NewError(err) + } + if ok := g.CheckOptimisticNoConflicts(lockSpansRead); !ok { + return nil, g, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError()) + } + } + // Handle any local (leaseholder-only) side-effects of the request. intents := result.Local.DetachEncounteredIntents() if pErr == nil { @@ -92,6 +113,13 @@ func (r *Replica) executeReadOnlyBatch( } // Otherwise, update the timestamp cache and release the concurrency guard. + // Note: + // - The update to the timestamp cache is not gated on pErr == nil, + // since certain semantic errors (e.g. ConditionFailedError on CPut) + // require updating the timestamp cache (see updatesTSCacheOnErr). + // - For optimistic evaluation, used for limited scans, the update to the + // timestamp cache limits itself to the spans that were read, by using + // the ResumeSpans. ec, g := endCmds{repl: r, g: g, st: st}, nil ec.done(ctx, ba, br, pErr) @@ -185,3 +213,65 @@ func (r *Replica) handleReadOnlyLocalEvalResult( } return nil } + +// collectSpansRead uses the spans declared in the requests, and the resume +// spans in the responses, to construct the effective spans that were read, +// and uses that to compute the latch and lock spans. +func (r *Replica) collectSpansRead( + ba *roachpb.BatchRequest, br *roachpb.BatchResponse, +) (latchSpans, lockSpans *spanset.SpanSet, _ error) { + baCopy := *ba + baCopy.Requests = make([]roachpb.RequestUnion, len(baCopy.Requests)) + j := 0 + for i := 0; i < len(baCopy.Requests); i++ { + baReq := ba.Requests[i] + req := baReq.GetInner() + header := req.Header() + + resp := br.Responses[i].GetInner() + if resp.Header().ResumeSpan == nil { + // Fully evaluated. + baCopy.Requests[j] = baReq + j++ + continue + } + + switch t := resp.(type) { + case *roachpb.ScanResponse: + if header.Key.Equal(t.ResumeSpan.Key) { + // The request did not evaluate. Ignore it. + continue + } + // The scan will resume at the inclusive ResumeSpan.Key. So + // ResumeSpan.Key has not been read and becomes the exclusive end key of + // what was read. + header.EndKey = t.ResumeSpan.Key + case *roachpb.ReverseScanResponse: + if header.EndKey.Equal(t.ResumeSpan.EndKey) { + // The request did not evaluate. Ignore it. + continue + } + // The scan will resume at the exclusive ResumeSpan.EndKey and proceed + // towards the current header.Key. So ResumeSpan.EndKey has been read + // and becomes the inclusive start key of what was read. + header.Key = t.ResumeSpan.EndKey + default: + // Consider it fully evaluated, which is safe. + baCopy.Requests[j] = baReq + j++ + continue + } + // The ResumeSpan has changed the header. + req = req.ShallowCopy() + req.SetHeader(header) + baCopy.Requests[j].MustSetInner(req) + j++ + } + baCopy.Requests = baCopy.Requests[:j] + + // Collect the batch's declared spans again, this time with the + // span bounds constrained to what was read. + var err error + latchSpans, lockSpans, _, err = r.collectSpans(&baCopy) + return latchSpans, lockSpans, err +} diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 1233bbe2db4c..00dbfdd47831 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -20,11 +20,20 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) +var optimisticEvalLimitedScans = settings.RegisterBoolSetting( + "kv.concurrency.optimistic_eval_limited_scans.enabled", + "when true, limited scans are optimistically evaluated in the sense of not checking for "+ + "conflicting locks up front for the full key range of the scan, and instead subsequently "+ + "checking for conflicts only over the key range that was read", + true, +) + // Send executes a command on this range, dispatching it to the // read-only, read-write, or admin execution path as appropriate. // ctx should contain the log tags from the store (and up). @@ -318,7 +327,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( ) (br *roachpb.BatchResponse, pErr *roachpb.Error) { // Determine the maximal set of key spans that the batch will operate on. // This is used below to sequence the request in the concurrency manager. - latchSpans, lockSpans, err := r.collectSpans(ba) + latchSpans, lockSpans, requestEvalKind, err := r.collectSpans(ba) if err != nil { return nil, roachpb.NewError(err) } @@ -354,7 +363,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( Requests: ba.Requests, LatchSpans: latchSpans, LockSpans: lockSpans, - }) + }, requestEvalKind) if pErr != nil { return nil, pErr } else if resp != nil { @@ -379,6 +388,14 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if filter := r.store.cfg.TestingKnobs.TestingConcurrencyRetryFilter; filter != nil { filter(ctx, *ba, pErr) } + + // Typically, retries are marked PessimisticEval. The one exception is a + // pessimistic retry immediately after an optimistic eval which failed + // when checking for conflicts, which is handled below. Note that an + // optimistic eval failure for any other reason will also retry as + // PessimisticEval. + requestEvalKind = concurrency.PessimisticEval + switch t := pErr.GetDetail().(type) { case *roachpb.WriteIntentError: // Drop latches, but retain lock wait-queues. @@ -416,6 +433,11 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } + case *roachpb.OptimisticEvalConflictsError: + // We are deliberately not dropping latches. The next iteration will + // pessimistically check for locks while holding these latches, and will + // find them again and queue up, and then release latches. + requestEvalKind = concurrency.PessimisticAfterFailedOptimisticEval default: log.Fatalf(ctx, "unexpected concurrency retry error %T", t) } @@ -424,11 +446,13 @@ func (r *Replica) executeBatchWithConcurrencyRetries( } // isConcurrencyRetryError returns whether or not the provided error is a -// "server-side concurrency retry error" that will be captured and retried by -// executeBatchWithConcurrencyRetries. Server-side concurrency retry errors are -// handled by dropping a request's latches, waiting for and/or ensuring that the -// condition which caused the error is handled, re-sequencing through the -// concurrency manager, and executing the request again. +// "concurrency retry error" that will be captured and retried by +// executeBatchWithConcurrencyRetries. Most concurrency retry errors are +// handled by dropping a request's latches, waiting for and/or ensuring that +// the condition which caused the error is handled, re-sequencing through the +// concurrency manager, and executing the request again. The one exception is +// OptimisticEvalConflictsError, where there is no need to drop latches, and +// the request can immediately proceed to retrying pessimistically. func isConcurrencyRetryError(pErr *roachpb.Error) bool { switch pErr.GetDetail().(type) { case *roachpb.WriteIntentError: @@ -455,11 +479,14 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { // NotLeaseHolderError) to the new leaseholder. case *roachpb.MergeInProgressError: // If a request hits a MergeInProgressError, the replica it is being - // evaluted against is in the process of being merged into its left-hand + // evaluated against is in the process of being merged into its left-hand // neighbor. The request cannot proceed until the range merge completes, // either successfully or unsuccessfully, so it waits before retrying. // If the merge does complete successfully, the retry will be rejected // with an error that will propagate back to the client. + case *roachpb.OptimisticEvalConflictsError: + // Optimistic evaluation encountered a conflict. The request will + // immediately retry pessimistically. default: return false } @@ -750,8 +777,14 @@ func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) e func (r *Replica) collectSpans( ba *roachpb.BatchRequest, -) (latchSpans, lockSpans *spanset.SpanSet, _ error) { +) (latchSpans, lockSpans *spanset.SpanSet, requestEvalKind concurrency.RequestEvalKind, _ error) { latchSpans, lockSpans = new(spanset.SpanSet), new(spanset.SpanSet) + isReadOnly := ba.IsReadOnly() + r.mu.RLock() + desc := r.descRLocked() + liveCount := r.mu.state.Stats.LiveCount + r.mu.RUnlock() + // TODO(bdarnell): need to make this less global when local // latches are used more heavily. For example, a split will // have a large read-only span but also a write (see #10084). @@ -775,6 +808,13 @@ func (r *Replica) collectSpans( lockSpans.Reserve(spanset.SpanReadOnly, spanset.SpanGlobal, len(ba.Requests)) } + considerOptEval := isReadOnly && ba.Header.MaxSpanRequestKeys > 0 && + optimisticEvalLimitedScans.Get(&r.ClusterSettings().SV) + // When considerOptEval, these are computed below and used to decide whether + // to actually do optimistic evaluation. + hasScans := false + numGets := 0 + // For non-local, MVCC spans we annotate them with the request timestamp // during declaration. This is the timestamp used during latch acquisitions. // For read requests this works as expected, reads are performed at the same @@ -782,14 +822,21 @@ func (r *Replica) collectSpans( // than the request timestamp, and may have to retry at a higher timestamp. // This is still safe as we're only ever writing at timestamps higher than the // timestamp any write latch would be declared at. - desc := r.Desc() batcheval.DeclareKeysForBatch(desc, ba.Header, latchSpans) for _, union := range ba.Requests { inner := union.GetInner() if cmd, ok := batcheval.LookupCommand(inner.Method()); ok { cmd.DeclareKeys(desc, ba.Header, inner, latchSpans, lockSpans) + if considerOptEval { + switch inner.(type) { + case *roachpb.ScanRequest, *roachpb.ReverseScanRequest: + hasScans = true + case *roachpb.GetRequest: + numGets++ + } + } } else { - return nil, nil, errors.Errorf("unrecognized command %s", inner.Method()) + return nil, nil, concurrency.PessimisticEval, errors.Errorf("unrecognized command %s", inner.Method()) } } @@ -801,11 +848,44 @@ func (r *Replica) collectSpans( // If any command gave us spans that are invalid, bail out early // (before passing them to the spanlatch manager, which may panic). if err := s.Validate(); err != nil { - return nil, nil, err + return nil, nil, concurrency.PessimisticEval, err + } + } + + requestEvalKind = concurrency.PessimisticEval + if considerOptEval { + // Evaluate batches optimistically if they have a key limit which is less + // than the upper bound on number of keys that can be returned for this + // batch. For scans, the upper bound is the number of live keys on the + // Range. For gets, it is the minimum of he number of live keys on the + // Range and the number of gets. Ignoring write latches and locks can be + // beneficial because it can help avoid waiting on writes to keys that the + // batch will never actually need to read due to the overestimate of its + // key bounds. Only after it is clear exactly what spans were read do we + // verify whether there were any conflicts with concurrent writes. + // + // This case is not uncommon; for example, a Scan which requests the entire + // range but has a limit of 1 result. We want to avoid allowing overly broad + // spans from backing up the latch manager, or encountering too much contention + // in the lock table. + // + // The heuristic is upper bound = k * liveCount, where k <= 1. The use of + // k=1 below is an un-tuned setting. + // + // This heuristic is crude in that it looks at the live count for the + // whole range, which may be much wider than the spans requested. + // Additionally, it does not consider TargetBytes. + const k = 1 + upperBoundKeys := k * liveCount + if !hasScans && int64(numGets) < upperBoundKeys { + upperBoundKeys = int64(numGets) + } + if ba.Header.MaxSpanRequestKeys < upperBoundKeys { + requestEvalKind = concurrency.OptimisticEval } } - return latchSpans, lockSpans, nil + return latchSpans, lockSpans, requestEvalKind, nil } // endCmds holds necessary information to end a batch after command processing, diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index be6b4c6f1656..37f52ffbcb5c 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -6183,6 +6183,7 @@ type Header struct { // - ScanRequest // - ReverseScanRequest // - DeleteRangeRequest + // - GetRequest // - RevertRangeRequest // - ResolveIntentRangeRequest // diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 03b1b11790c8..755d2f007bb9 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1932,6 +1932,7 @@ message Header { // - ScanRequest // - ReverseScanRequest // - DeleteRangeRequest + // - GetRequest // - RevertRangeRequest // - ResolveIntentRangeRequest // diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 26556adee661..a6b27ceb9b15 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -70,6 +70,8 @@ func (ru ErrorDetail) GetInner() error { return t.IndeterminateCommit case *ErrorDetail_InvalidLeaseError: return t.InvalidLeaseError + case *ErrorDetail_OptimisticEvalConflicts: + return t.OptimisticEvalConflicts default: return nil } @@ -328,6 +330,8 @@ func (ru *ErrorDetail) MustSetInner(r error) { union = &ErrorDetail_IndeterminateCommit{t} case *InvalidLeaseError: union = &ErrorDetail_InvalidLeaseError{t} + case *OptimisticEvalConflictsError: + union = &ErrorDetail_OptimisticEvalConflicts{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } diff --git a/pkg/roachpb/errordetailtype_string.go b/pkg/roachpb/errordetailtype_string.go index 0ddcaacd392e..977c51a9601b 100644 --- a/pkg/roachpb/errordetailtype_string.go +++ b/pkg/roachpb/errordetailtype_string.go @@ -37,6 +37,7 @@ func _() { _ = x[RangeFeedRetryErrType-38] _ = x[IndeterminateCommitErrType-39] _ = x[InvalidLeaseErrType-40] + _ = x[OptimisticEvalConflictsErrType-41] _ = x[CommunicationErrType-22] _ = x[InternalErrType-25] } @@ -47,7 +48,7 @@ const ( _ErrorDetailType_name_2 = "CommunicationErrType" _ErrorDetailType_name_3 = "InternalErrTypeAmbiguousResultErrTypeStoreNotFoundErrTypeTransactionRetryWithProtoRefreshErrType" _ErrorDetailType_name_4 = "IntegerOverflowErrTypeUnsupportedRequestErrType" - _ErrorDetailType_name_5 = "BatchTimestampBeforeGCErrTypeTxnAlreadyEncounteredErrTypeIntentMissingErrTypeMergeInProgressErrTypeRangeFeedRetryErrTypeIndeterminateCommitErrTypeInvalidLeaseErrType" + _ErrorDetailType_name_5 = "BatchTimestampBeforeGCErrTypeTxnAlreadyEncounteredErrTypeIntentMissingErrTypeMergeInProgressErrTypeRangeFeedRetryErrTypeIndeterminateCommitErrTypeInvalidLeaseErrTypeOptimisticEvalConflictsErrType" ) var ( @@ -55,7 +56,7 @@ var ( _ErrorDetailType_index_1 = [...]uint8{0, 23, 47, 67} _ErrorDetailType_index_3 = [...]uint8{0, 15, 37, 57, 96} _ErrorDetailType_index_4 = [...]uint8{0, 22, 47} - _ErrorDetailType_index_5 = [...]uint8{0, 29, 57, 77, 99, 120, 146, 165} + _ErrorDetailType_index_5 = [...]uint8{0, 29, 57, 77, 99, 120, 146, 165, 195} ) func (i ErrorDetailType) String() string { @@ -74,7 +75,7 @@ func (i ErrorDetailType) String() string { case 31 <= i && i <= 32: i -= 31 return _ErrorDetailType_name_4[_ErrorDetailType_index_4[i]:_ErrorDetailType_index_4[i+1]] - case 34 <= i && i <= 40: + case 34 <= i && i <= 41: i -= 34 return _ErrorDetailType_name_5[_ErrorDetailType_index_5[i]:_ErrorDetailType_index_5[i+1]] default: diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 2109aa00f789..90522e0219df 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -318,6 +318,7 @@ const ( RangeFeedRetryErrType ErrorDetailType = 38 IndeterminateCommitErrType ErrorDetailType = 39 InvalidLeaseErrType ErrorDetailType = 40 + OptimisticEvalConflictsErrType ErrorDetailType = 41 // When adding new error types, don't forget to update NumErrors below. // CommunicationErrType indicates a gRPC error; this is not an ErrorDetail. @@ -327,7 +328,7 @@ const ( // detail. The value 25 is chosen because it's reserved in the errors proto. InternalErrType ErrorDetailType = 25 - NumErrors int = 41 + NumErrors int = 42 ) // GoError returns a Go error converted from Error. If the error is a transaction @@ -1256,3 +1257,24 @@ func (e *InvalidLeaseError) Type() ErrorDetailType { } var _ ErrorDetailInterface = &InvalidLeaseError{} + +// NewOptimisticEvalConflictsError initializes a new +// OptimisticEvalConflictsError. +func NewOptimisticEvalConflictsError() *OptimisticEvalConflictsError { + return &OptimisticEvalConflictsError{} +} + +func (e *OptimisticEvalConflictsError) Error() string { + return e.message(nil) +} + +func (e *OptimisticEvalConflictsError) message(pErr *Error) string { + return "optimistic eval encountered conflict" +} + +// Type is part of the ErrorDetailInterface. +func (e *OptimisticEvalConflictsError) Type() ErrorDetailType { + return OptimisticEvalConflictsErrType +} + +var _ ErrorDetailInterface = &OptimisticEvalConflictsError{} diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 6895c7bbb79f..76e99acbdb4c 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -1545,6 +1545,42 @@ func (m *InvalidLeaseError) XXX_DiscardUnknown() { var xxx_messageInfo_InvalidLeaseError proto.InternalMessageInfo +// OptimisticEvalConflictsError indicates that the optimistic evaluation of a +// BatchRequest failed when checking locks or latches after evaluation. This +// is never sent over-the-wire, and is a proto only to fit the existing +// plumbing that passes a roachpb.Error. +type OptimisticEvalConflictsError struct { +} + +func (m *OptimisticEvalConflictsError) Reset() { *m = OptimisticEvalConflictsError{} } +func (m *OptimisticEvalConflictsError) String() string { return proto.CompactTextString(m) } +func (*OptimisticEvalConflictsError) ProtoMessage() {} +func (*OptimisticEvalConflictsError) Descriptor() ([]byte, []int) { + return fileDescriptor_123941c6716fd549, []int{30} +} +func (m *OptimisticEvalConflictsError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OptimisticEvalConflictsError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *OptimisticEvalConflictsError) XXX_Merge(src proto.Message) { + xxx_messageInfo_OptimisticEvalConflictsError.Merge(m, src) +} +func (m *OptimisticEvalConflictsError) XXX_Size() int { + return m.Size() +} +func (m *OptimisticEvalConflictsError) XXX_DiscardUnknown() { + xxx_messageInfo_OptimisticEvalConflictsError.DiscardUnknown(m) +} + +var xxx_messageInfo_OptimisticEvalConflictsError proto.InternalMessageInfo + // ErrorDetail is a union type containing all available errors. type ErrorDetail struct { // Types that are valid to be assigned to Value: @@ -1577,6 +1613,7 @@ type ErrorDetail struct { // *ErrorDetail_RangefeedRetry // *ErrorDetail_IndeterminateCommit // *ErrorDetail_InvalidLeaseError + // *ErrorDetail_OptimisticEvalConflicts Value isErrorDetail_Value `protobuf_oneof:"value"` } @@ -1584,7 +1621,7 @@ func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } func (*ErrorDetail) ProtoMessage() {} func (*ErrorDetail) Descriptor() ([]byte, []int) { - return fileDescriptor_123941c6716fd549, []int{30} + return fileDescriptor_123941c6716fd549, []int{31} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1702,6 +1739,9 @@ type ErrorDetail_IndeterminateCommit struct { type ErrorDetail_InvalidLeaseError struct { InvalidLeaseError *InvalidLeaseError `protobuf:"bytes,40,opt,name=invalid_lease_error,json=invalidLeaseError,oneof" json:"invalid_lease_error,omitempty"` } +type ErrorDetail_OptimisticEvalConflicts struct { + OptimisticEvalConflicts *OptimisticEvalConflictsError `protobuf:"bytes,41,opt,name=optimistic_eval_conflicts,json=optimisticEvalConflicts,oneof" json:"optimistic_eval_conflicts,omitempty"` +} func (*ErrorDetail_NotLeaseHolder) isErrorDetail_Value() {} func (*ErrorDetail_RangeNotFound) isErrorDetail_Value() {} @@ -1732,6 +1772,7 @@ func (*ErrorDetail_MergeInProgress) isErrorDetail_Value() {} func (*ErrorDetail_RangefeedRetry) isErrorDetail_Value() {} func (*ErrorDetail_IndeterminateCommit) isErrorDetail_Value() {} func (*ErrorDetail_InvalidLeaseError) isErrorDetail_Value() {} +func (*ErrorDetail_OptimisticEvalConflicts) isErrorDetail_Value() {} func (m *ErrorDetail) GetValue() isErrorDetail_Value { if m != nil { @@ -1943,6 +1984,13 @@ func (m *ErrorDetail) GetInvalidLeaseError() *InvalidLeaseError { return nil } +func (m *ErrorDetail) GetOptimisticEvalConflicts() *OptimisticEvalConflictsError { + if x, ok := m.GetValue().(*ErrorDetail_OptimisticEvalConflicts); ok { + return x.OptimisticEvalConflicts + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*ErrorDetail) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -1975,6 +2023,7 @@ func (*ErrorDetail) XXX_OneofWrappers() []interface{} { (*ErrorDetail_RangefeedRetry)(nil), (*ErrorDetail_IndeterminateCommit)(nil), (*ErrorDetail_InvalidLeaseError)(nil), + (*ErrorDetail_OptimisticEvalConflicts)(nil), } } @@ -1989,7 +2038,7 @@ func (m *ErrPosition) Reset() { *m = ErrPosition{} } func (m *ErrPosition) String() string { return proto.CompactTextString(m) } func (*ErrPosition) ProtoMessage() {} func (*ErrPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_123941c6716fd549, []int{31} + return fileDescriptor_123941c6716fd549, []int{32} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2051,7 +2100,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_123941c6716fd549, []int{32} + return fileDescriptor_123941c6716fd549, []int{33} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2112,6 +2161,7 @@ func init() { proto.RegisterType((*RangeFeedRetryError)(nil), "cockroach.kv.kvpb.RangeFeedRetryError") proto.RegisterType((*IndeterminateCommitError)(nil), "cockroach.kv.kvpb.IndeterminateCommitError") proto.RegisterType((*InvalidLeaseError)(nil), "cockroach.kv.kvpb.InvalidLeaseError") + proto.RegisterType((*OptimisticEvalConflictsError)(nil), "cockroach.kv.kvpb.OptimisticEvalConflictsError") proto.RegisterType((*ErrorDetail)(nil), "cockroach.kv.kvpb.ErrorDetail") proto.RegisterType((*ErrPosition)(nil), "cockroach.kv.kvpb.ErrPosition") proto.RegisterType((*Error)(nil), "cockroach.kv.kvpb.Error") @@ -2120,200 +2170,203 @@ func init() { func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_123941c6716fd549) } var fileDescriptor_123941c6716fd549 = []byte{ - // 3073 bytes of a gzipped FileDescriptorProto + // 3125 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcb, 0x6f, 0x1b, 0xd7, - 0xb9, 0x27, 0x25, 0x4a, 0xa2, 0x3e, 0x3d, 0x3c, 0x3a, 0x7a, 0x8d, 0xe4, 0x98, 0x52, 0xc6, 0x76, - 0x62, 0x3b, 0x88, 0x74, 0xe1, 0xdc, 0x00, 0x37, 0xb9, 0xce, 0x82, 0x8f, 0x91, 0x48, 0x89, 0x0f, - 0xdd, 0x21, 0xe5, 0x47, 0x8c, 0xe0, 0x64, 0xc4, 0x39, 0xa2, 0xe6, 0x7a, 0x38, 0xc3, 0x9c, 0x19, - 0xca, 0x12, 0x70, 0x17, 0x17, 0xf7, 0x6e, 0x82, 0x16, 0x28, 0xda, 0x55, 0xbb, 0x0c, 0x10, 0x74, - 0x51, 0xa0, 0x28, 0xfa, 0x17, 0x74, 0x9d, 0x65, 0x96, 0x41, 0x51, 0x18, 0xad, 0xdd, 0xbf, 0x22, - 0xab, 0xe2, 0x3c, 0x86, 0x1c, 0x92, 0x43, 0x59, 0xc9, 0x8e, 0xf3, 0xbd, 0xce, 0x77, 0x5e, 0xbf, - 0xef, 0xf7, 0x1d, 0xc2, 0x0a, 0xf5, 0xcc, 0xe6, 0x59, 0xe7, 0x64, 0x97, 0x50, 0xea, 0x51, 0x7f, - 0xa7, 0x43, 0xbd, 0xc0, 0x43, 0x4b, 0x4d, 0xaf, 0xf9, 0x82, 0x6b, 0x76, 0x5e, 0x9c, 0xef, 0xbc, - 0x38, 0xef, 0x9c, 0x6c, 0xae, 0x0a, 0x83, 0x21, 0xcb, 0x4d, 0x14, 0xfa, 0x5b, 0x66, 0x60, 0x4a, - 0xd9, 0x5a, 0x28, 0x6b, 0x93, 0xc0, 0x8c, 0xc8, 0xd5, 0x6e, 0x60, 0x3b, 0xbb, 0x67, 0x4e, 0x73, - 0x37, 0xb0, 0xdb, 0xc4, 0x0f, 0xcc, 0x76, 0x47, 0x6a, 0x56, 0x5a, 0x5e, 0xcb, 0xe3, 0x3f, 0x77, - 0xd9, 0x2f, 0x21, 0xd5, 0xfe, 0x3c, 0x01, 0xcb, 0x55, 0x2f, 0x28, 0x13, 0xd3, 0x27, 0x45, 0xcf, - 0xb1, 0x08, 0xd5, 0xd9, 0xd0, 0xa8, 0x00, 0x33, 0x94, 0x74, 0x1c, 0xbb, 0x69, 0xaa, 0xc9, 0xed, - 0xe4, 0xbd, 0xb9, 0x87, 0x77, 0x76, 0xfa, 0xf9, 0xca, 0xb1, 0x77, 0x0c, 0x61, 0x51, 0x20, 0x7e, - 0x93, 0xda, 0x9d, 0xc0, 0xa3, 0xb9, 0xd4, 0x77, 0xaf, 0xb6, 0x12, 0x46, 0xe8, 0x8a, 0xf6, 0x61, - 0xde, 0x61, 0x91, 0xf1, 0x19, 0x0f, 0xad, 0x4e, 0x5c, 0x3f, 0x94, 0x31, 0xe7, 0xf4, 0x73, 0x42, - 0x3b, 0x30, 0xc5, 0x3f, 0xd5, 0x14, 0x8f, 0xa0, 0xc6, 0x44, 0xe0, 0x53, 0x30, 0x84, 0x19, 0xfa, - 0x18, 0xd2, 0xd4, 0x74, 0x5b, 0x04, 0xdb, 0x96, 0x3a, 0xb9, 0x9d, 0xbc, 0x37, 0x99, 0xdb, 0x64, - 0x99, 0xbd, 0x7e, 0xb5, 0x35, 0x63, 0x30, 0x79, 0xa9, 0xf0, 0x63, 0xff, 0xa7, 0x31, 0xc3, 0x6d, - 0x4b, 0x16, 0xba, 0x0d, 0xd0, 0xec, 0xfa, 0x81, 0xd7, 0xc6, 0x6d, 0xbf, 0xa5, 0x4e, 0x6d, 0x27, - 0xef, 0xcd, 0xca, 0x29, 0xcd, 0x0a, 0x79, 0xc5, 0x6f, 0x69, 0x6b, 0xb0, 0x52, 0xf5, 0x2c, 0x72, - 0xec, 0x9a, 0xe7, 0xa6, 0xed, 0x98, 0x27, 0x0e, 0xe1, 0x4b, 0xa6, 0x6d, 0xc0, 0xfa, 0xb1, 0xeb, - 0x77, 0x3b, 0x1d, 0x8f, 0x06, 0xc4, 0x32, 0xc8, 0x57, 0x5d, 0xe2, 0x07, 0x42, 0xf5, 0x7f, 0x49, - 0x40, 0x7c, 0xb0, 0xaa, 0x17, 0xec, 0x79, 0x5d, 0xd7, 0x12, 0x8b, 0x1c, 0xcd, 0x32, 0x79, 0xfd, - 0x2c, 0x3f, 0x86, 0xb4, 0x1f, 0x78, 0x94, 0xbb, 0x4d, 0x0c, 0xba, 0xd5, 0x99, 0x5c, 0xb8, 0xc9, - 0x9f, 0xc6, 0x0c, 0xb7, 0x2d, 0x59, 0xda, 0x6f, 0x27, 0x61, 0x95, 0xc7, 0x3a, 0x24, 0x97, 0x15, - 0xdb, 0x6f, 0x9b, 0x41, 0xf3, 0x4c, 0xe4, 0xf1, 0x11, 0x2c, 0x51, 0x91, 0x2e, 0xf6, 0x03, 0x93, - 0x06, 0xf8, 0x05, 0xb9, 0xe4, 0x09, 0xcd, 0xe7, 0x66, 0x7e, 0x7c, 0xb5, 0x35, 0x79, 0x48, 0x2e, - 0x8d, 0x1b, 0xd2, 0xa2, 0xce, 0x0c, 0x0e, 0xc9, 0x25, 0xda, 0x85, 0x50, 0x84, 0x89, 0x6b, 0x71, - 0x97, 0x89, 0x41, 0x97, 0x05, 0xa9, 0xd7, 0x5d, 0x8b, 0x39, 0x9c, 0xc1, 0x4d, 0x8b, 0x74, 0x28, - 0x69, 0x9a, 0x01, 0xb1, 0x70, 0x5b, 0x66, 0x40, 0x2c, 0xcc, 0xe7, 0xc5, 0xb7, 0x69, 0xee, 0xa1, - 0x16, 0x77, 0x36, 0x98, 0x7e, 0xe4, 0x90, 0x6d, 0xf4, 0x83, 0x55, 0x7a, 0xb1, 0xb8, 0x29, 0xfa, - 0x12, 0x36, 0x23, 0x23, 0xf9, 0xdd, 0x56, 0x8b, 0xf8, 0x41, 0x6f, 0xa0, 0xd4, 0x75, 0x07, 0x32, - 0xd4, 0x7e, 0x94, 0x7a, 0x18, 0x44, 0x8c, 0x50, 0x86, 0x69, 0x1e, 0xcc, 0x57, 0xa7, 0xb6, 0x27, - 0xef, 0xcd, 0x3d, 0x7c, 0x67, 0x5c, 0xb4, 0x92, 0x7b, 0xea, 0xe5, 0xd6, 0xe4, 0xf6, 0x2c, 0x0a, - 0x9f, 0x92, 0x1b, 0x10, 0xea, 0x9a, 0x8e, 0x21, 0x63, 0x68, 0x6f, 0x26, 0x41, 0x33, 0x88, 0x69, - 0x3d, 0xb1, 0x83, 0x33, 0xdb, 0x3d, 0x76, 0x9b, 0x84, 0x06, 0xa6, 0xed, 0x06, 0x97, 0xdc, 0xf2, - 0xdc, 0x74, 0xc4, 0x36, 0x1d, 0xc0, 0x22, 0x25, 0xa6, 0x85, 0x7b, 0x37, 0x5b, 0x5e, 0xcd, 0x5b, - 0x91, 0xc1, 0xd9, 0xf5, 0xdf, 0x39, 0x73, 0x9a, 0x3b, 0x8d, 0xd0, 0x48, 0x2e, 0xd7, 0x02, 0x73, - 0xed, 0x09, 0x91, 0x01, 0x88, 0x5c, 0xd8, 0x7e, 0x60, 0xbb, 0xad, 0x48, 0xbc, 0x89, 0xeb, 0xc7, - 0x5b, 0x0a, 0xdd, 0xfb, 0x31, 0x9f, 0xc3, 0xba, 0xe3, 0x35, 0x4d, 0x07, 0x77, 0xfb, 0x33, 0xc0, - 0x8e, 0xdd, 0xb6, 0x03, 0x7e, 0x95, 0xae, 0x19, 0x78, 0x95, 0xc7, 0x88, 0x2c, 0x42, 0x99, 0x45, - 0x40, 0x5f, 0x80, 0xda, 0x72, 0xbc, 0x93, 0xd8, 0xe8, 0x93, 0xd7, 0x8f, 0xbe, 0x26, 0x82, 0x8c, - 0x84, 0x7f, 0x0e, 0xcb, 0xde, 0x89, 0x4f, 0xe8, 0x39, 0x89, 0xac, 0xaf, 0xaf, 0xa6, 0xf8, 0xee, - 0xc6, 0x01, 0x56, 0x4d, 0x5a, 0x0f, 0x0f, 0x80, 0xbc, 0x61, 0x85, 0xff, 0x69, 0xea, 0x77, 0xdf, - 0x6c, 0x25, 0x34, 0x0b, 0xd6, 0x1b, 0xd4, 0x74, 0x7d, 0xb3, 0x19, 0xd8, 0x9e, 0x9b, 0x3d, 0xe1, - 0x30, 0x21, 0x76, 0xb6, 0x04, 0xd3, 0x94, 0x98, 0xbe, 0xe7, 0xf2, 0x1d, 0x5d, 0x7c, 0xf8, 0xc1, - 0xce, 0x48, 0x71, 0xd8, 0x19, 0xf5, 0x35, 0xb8, 0x8b, 0x1c, 0x57, 0x06, 0xd0, 0x9e, 0xc3, 0x4a, - 0xc4, 0xf2, 0xa8, 0xeb, 0xcb, 0x3b, 0x9e, 0x07, 0xe8, 0x74, 0xfd, 0x33, 0x42, 0x70, 0x70, 0xe1, - 0xca, 0x83, 0x93, 0x89, 0x99, 0x57, 0xc4, 0x39, 0x84, 0x3e, 0xe1, 0xd7, 0xb8, 0x70, 0xb5, 0xff, - 0x4f, 0xc2, 0x6a, 0xc4, 0xc0, 0x20, 0x01, 0xbd, 0x14, 0xe1, 0xf7, 0x87, 0x66, 0x70, 0xff, 0xea, - 0x19, 0x70, 0xcf, 0xb8, 0xfc, 0xd1, 0xbb, 0x30, 0x4b, 0x2e, 0x02, 0x6a, 0x72, 0x04, 0x9e, 0x88, - 0x20, 0x70, 0x9a, 0x8b, 0x19, 0x00, 0xff, 0x25, 0x09, 0x6b, 0x91, 0x58, 0xf5, 0xc0, 0x0c, 0xba, - 0xbe, 0x48, 0x63, 0x0d, 0x26, 0x99, 0x5f, 0x32, 0xe2, 0xc7, 0x04, 0xa8, 0xda, 0x4b, 0x6f, 0x82, - 0xa7, 0xf7, 0x6f, 0x57, 0xa7, 0x17, 0x09, 0xb9, 0x13, 0xbb, 0xca, 0x8f, 0x60, 0x5a, 0xc8, 0x11, - 0x82, 0x45, 0x43, 0xcf, 0xd6, 0x6b, 0x55, 0x7c, 0x5c, 0x3d, 0xac, 0xd6, 0x9e, 0x54, 0x95, 0x04, - 0x52, 0x61, 0x45, 0xca, 0x1a, 0x4f, 0xab, 0x38, 0x5f, 0xab, 0x54, 0x4a, 0x8d, 0x86, 0x5e, 0x50, - 0x26, 0xb4, 0x54, 0x3a, 0xa9, 0x24, 0xb5, 0xdf, 0x24, 0x41, 0x79, 0x42, 0xed, 0x80, 0xb0, 0xfb, - 0xed, 0x8a, 0x1a, 0x81, 0x3e, 0x81, 0x19, 0x9b, 0x7f, 0xfa, 0x6a, 0x92, 0x9f, 0xba, 0x8d, 0x98, - 0xdd, 0x11, 0x0e, 0x61, 0x99, 0x95, 0xf6, 0xe8, 0x11, 0x2c, 0x8a, 0x32, 0xeb, 0x33, 0xc0, 0x75, - 0x9b, 0x44, 0xd6, 0xbc, 0x55, 0x66, 0xf6, 0xe3, 0xab, 0xad, 0x05, 0x5e, 0x1b, 0xeb, 0x52, 0x69, - 0x2c, 0x38, 0xd1, 0xcf, 0x83, 0x54, 0x7a, 0x42, 0x99, 0xd4, 0x7e, 0x1f, 0xe6, 0xd4, 0xf0, 0xbc, - 0x9a, 0x23, 0xcf, 0x65, 0x16, 0x66, 0x7f, 0x16, 0xd8, 0xf4, 0xbd, 0x50, 0x15, 0x14, 0xb3, 0x19, - 0x74, 0x4d, 0xe7, 0xe7, 0xc1, 0xcc, 0x0d, 0xe1, 0xdc, 0x13, 0x6b, 0x2b, 0x80, 0x6a, 0x1d, 0x56, - 0x5c, 0x6d, 0x4a, 0xfc, 0xc6, 0x85, 0x2b, 0x0a, 0x6c, 0x1d, 0x56, 0xf2, 0x9e, 0x6b, 0xd9, 0x6c, - 0xf3, 0xf6, 0x4c, 0xdb, 0x09, 0x2f, 0xd6, 0x7f, 0xc2, 0xbc, 0x1c, 0xfd, 0xdc, 0x74, 0xba, 0x44, - 0xce, 0x21, 0x8e, 0x3e, 0x3c, 0x66, 0x7a, 0x63, 0x4e, 0x58, 0xf3, 0x0f, 0xed, 0x4f, 0x49, 0x40, - 0x82, 0x55, 0x90, 0xff, 0x26, 0xcd, 0xde, 0x65, 0xcd, 0xc0, 0x4c, 0x9b, 0xf8, 0xbe, 0xd9, 0x22, - 0x03, 0xe7, 0x2c, 0x14, 0xa2, 0x47, 0x30, 0x2b, 0x0b, 0x1f, 0xb1, 0xe4, 0x54, 0xc7, 0xf2, 0x95, - 0x70, 0xbd, 0x7a, 0x0e, 0xe8, 0x53, 0x48, 0x87, 0xc8, 0x2a, 0x71, 0xed, 0x6d, 0xce, 0x3d, 0x7b, - 0xed, 0x2b, 0x58, 0xc9, 0xb6, 0x4f, 0xec, 0x56, 0xd7, 0xeb, 0xfa, 0x06, 0xf1, 0xbb, 0x4e, 0x70, - 0xbd, 0x8c, 0x3f, 0x81, 0xb9, 0x97, 0xd4, 0xec, 0x74, 0x88, 0x85, 0x09, 0xa5, 0x31, 0x39, 0x87, - 0x57, 0x84, 0x87, 0x33, 0x40, 0x1a, 0xeb, 0x94, 0x6a, 0xeb, 0x8c, 0x53, 0x9c, 0x06, 0xfb, 0xd4, - 0xeb, 0x76, 0x0a, 0xc4, 0x21, 0xe1, 0x2a, 0x69, 0x18, 0xd6, 0x24, 0xa7, 0xcb, 0x7b, 0x94, 0x76, - 0x3b, 0x6c, 0x67, 0x44, 0x36, 0xec, 0x86, 0xb3, 0x1f, 0x78, 0xf8, 0xa6, 0xa6, 0xb9, 0xb8, 0xe2, - 0xb7, 0x90, 0x06, 0xb3, 0x1d, 0xea, 0x35, 0x89, 0xef, 0xcb, 0x25, 0x4c, 0xf7, 0xb0, 0x28, 0x14, - 0x6b, 0x75, 0x40, 0x72, 0x80, 0xe8, 0x89, 0xfd, 0x0c, 0x40, 0x92, 0xcf, 0x90, 0x54, 0x4d, 0xe5, - 0x32, 0xb2, 0xfc, 0xce, 0x4a, 0x7b, 0xce, 0x8f, 0xfa, 0x1f, 0x6c, 0xf5, 0xc5, 0x4f, 0x4b, 0x3b, - 0x04, 0xc4, 0x79, 0xd3, 0x08, 0x4f, 0xeb, 0x11, 0xae, 0xe4, 0xf5, 0x09, 0x57, 0x9d, 0x11, 0xc2, - 0x33, 0xd3, 0xb5, 0x1c, 0x86, 0xd5, 0x01, 0xbd, 0xec, 0x71, 0x45, 0xf4, 0x10, 0x52, 0x1d, 0x9d, - 0xd2, 0x98, 0xf3, 0x38, 0xb0, 0xd4, 0x72, 0xd6, 0xdc, 0x56, 0x56, 0x91, 0x7f, 0x26, 0xe1, 0xee, - 0x30, 0x90, 0x32, 0xde, 0x70, 0xc4, 0xe8, 0xbc, 0x41, 0x4e, 0x29, 0x09, 0x11, 0x7f, 0x1c, 0x16, - 0x3e, 0x87, 0xe9, 0xe0, 0xc2, 0x0d, 0xc9, 0xe3, 0x7c, 0xae, 0xc0, 0x54, 0x7f, 0x7d, 0xb5, 0xf5, - 0x51, 0xcb, 0x0e, 0xce, 0xba, 0x27, 0x3b, 0x4d, 0xaf, 0xbd, 0xdb, 0xcb, 0xc7, 0x3a, 0xe9, 0xff, - 0xde, 0xed, 0xbc, 0x68, 0xed, 0xf2, 0xfe, 0xa2, 0xdb, 0xb5, 0xad, 0x9d, 0xe3, 0xe3, 0x52, 0xe1, - 0xf5, 0xab, 0xad, 0xa9, 0xc6, 0x85, 0x5b, 0x2a, 0x18, 0x53, 0xc1, 0x85, 0x5b, 0xb2, 0xd0, 0x1e, - 0xcc, 0x05, 0xfd, 0xec, 0xe4, 0x09, 0xbe, 0x5e, 0x9d, 0x89, 0x3a, 0x6a, 0x7b, 0xb0, 0xd5, 0xb8, - 0x70, 0xb3, 0x0e, 0x63, 0x2d, 0x97, 0xba, 0xdb, 0xf4, 0xba, 0x8c, 0x0a, 0xc9, 0xc3, 0x25, 0xe6, - 0x77, 0x1b, 0xa0, 0x43, 0xc9, 0x39, 0xe6, 0xa7, 0x66, 0x60, 0x9a, 0xb3, 0x4c, 0x2e, 0x8e, 0xe1, - 0x2f, 0x93, 0xb0, 0xc2, 0x40, 0xb3, 0x45, 0x68, 0xed, 0x9c, 0xd0, 0x53, 0xc7, 0x7b, 0x29, 0xbc, - 0x37, 0x60, 0x32, 0x86, 0xe5, 0x32, 0x19, 0xba, 0x0f, 0x0b, 0xcd, 0x2e, 0xa5, 0xc4, 0x0d, 0x24, - 0x6a, 0x08, 0x92, 0x2d, 0x62, 0xcf, 0x4b, 0x15, 0x87, 0x08, 0xf4, 0x21, 0xdc, 0xb0, 0xdd, 0x26, - 0x25, 0xed, 0xbe, 0xf1, 0x64, 0xc4, 0x78, 0xb1, 0xa7, 0x14, 0x88, 0xf2, 0x6d, 0x12, 0x6e, 0xe6, - 0x18, 0x53, 0xed, 0xc3, 0x1c, 0x39, 0xf5, 0x28, 0xd9, 0xcf, 0xf7, 0xf0, 0xb6, 0xf1, 0xb3, 0xf0, - 0xb6, 0x4f, 0xc2, 0x58, 0x88, 0x33, 0x76, 0x08, 0x3c, 0xc7, 0xfa, 0x29, 0x40, 0xdb, 0xf7, 0xd2, - 0xda, 0x80, 0x44, 0x9d, 0xa9, 0xd8, 0xbe, 0x6f, 0xbb, 0x2d, 0x91, 0xdb, 0x23, 0x98, 0x7f, 0x49, - 0x3d, 0xb7, 0x85, 0x45, 0xd5, 0x91, 0xe9, 0x8d, 0x2f, 0x52, 0xc6, 0x1c, 0x37, 0x17, 0x1f, 0xe1, - 0x72, 0x4f, 0x8c, 0x2e, 0x37, 0xeb, 0xa7, 0x2a, 0x84, 0x32, 0xaa, 0x7c, 0x44, 0xbd, 0x16, 0x25, - 0xbe, 0x28, 0xbc, 0xda, 0xaf, 0x26, 0x60, 0x99, 0x73, 0xe8, 0x3d, 0x22, 0xef, 0x8f, 0x48, 0xe4, - 0x70, 0x88, 0x6a, 0x7c, 0x18, 0x73, 0x7b, 0x62, 0xfc, 0xe2, 0x0b, 0xf9, 0x1f, 0x92, 0xbd, 0x4a, - 0xbe, 0x09, 0x6b, 0xb2, 0x6a, 0x1b, 0xfa, 0x51, 0xb9, 0x94, 0xcf, 0x62, 0x43, 0xaf, 0xd4, 0x1e, - 0xeb, 0x05, 0x25, 0x81, 0xd6, 0x00, 0x85, 0xba, 0x6c, 0x75, 0x5f, 0xc7, 0xf5, 0xa3, 0x72, 0xa9, - 0xa1, 0x24, 0xd1, 0x3a, 0x2c, 0x0f, 0xc8, 0x2b, 0xba, 0xb1, 0xcf, 0x0a, 0x7d, 0x84, 0x02, 0x18, - 0xd9, 0xbd, 0x06, 0xae, 0x57, 0xb3, 0x47, 0xf5, 0x62, 0xad, 0xa1, 0x4c, 0xa2, 0x0c, 0x6c, 0x4a, - 0x4d, 0xb9, 0xb6, 0x5f, 0xca, 0x67, 0xcb, 0xb8, 0x76, 0x54, 0xc7, 0x95, 0x52, 0xbd, 0x5e, 0xaa, - 0xee, 0x2b, 0xa9, 0x88, 0x67, 0xbd, 0x5c, 0x7b, 0x82, 0xf3, 0xb5, 0x6a, 0xfd, 0xb8, 0xa2, 0x1b, - 0xca, 0x94, 0x66, 0x82, 0x5a, 0x72, 0x2d, 0x12, 0x10, 0xda, 0xb6, 0x5d, 0x33, 0x20, 0x79, 0xaf, - 0xdd, 0xb6, 0x25, 0xc4, 0xeb, 0x30, 0xe7, 0x07, 0x66, 0x8b, 0xd3, 0xf9, 0x9f, 0xc8, 0xef, 0x40, - 0x3a, 0x32, 0x82, 0xb7, 0x0c, 0x4b, 0x25, 0xf7, 0xdc, 0x74, 0x6c, 0x8b, 0x57, 0x18, 0xb1, 0x11, - 0xdf, 0xac, 0xc2, 0x1c, 0xff, 0x55, 0x20, 0x81, 0x69, 0x3b, 0xc8, 0x00, 0xc5, 0xf5, 0x02, 0x3c, - 0xd0, 0xd9, 0x8b, 0x01, 0xdf, 0x8b, 0xd9, 0x8a, 0x98, 0xd7, 0x85, 0x62, 0xc2, 0x58, 0x74, 0x07, - 0xc4, 0xa8, 0x06, 0x37, 0x44, 0x2b, 0xcc, 0x22, 0x9f, 0x32, 0xe8, 0x95, 0x87, 0xf7, 0xee, 0xb8, - 0xdd, 0x1d, 0x80, 0xe8, 0x22, 0x6b, 0x70, 0xa2, 0x52, 0xf4, 0x14, 0x90, 0x08, 0xf8, 0x82, 0x5c, - 0xf6, 0x9a, 0x4d, 0x89, 0x47, 0xf7, 0xc6, 0xc5, 0x1c, 0xee, 0x8c, 0x8b, 0x09, 0x43, 0xa1, 0x43, - 0x0a, 0xf4, 0xbf, 0x49, 0xd8, 0xe6, 0x7d, 0xd8, 0x4b, 0xde, 0xae, 0x0d, 0xf4, 0x23, 0xb6, 0x6c, - 0xd8, 0x64, 0x93, 0xf9, 0x71, 0xdc, 0x40, 0x6f, 0x6d, 0xf4, 0x8a, 0x09, 0xe3, 0x16, 0xbd, 0xca, - 0x0a, 0x7d, 0x01, 0xcb, 0x11, 0xb0, 0xc4, 0xa6, 0xe8, 0x07, 0x64, 0x97, 0xf5, 0xe0, 0x5a, 0xcd, - 0x43, 0x38, 0x12, 0x0a, 0x46, 0x54, 0xa8, 0x01, 0x4a, 0x34, 0x3c, 0xe3, 0xff, 0xea, 0x34, 0x8f, - 0xfd, 0xfe, 0xd5, 0xb1, 0x7b, 0xed, 0x46, 0x31, 0x61, 0xdc, 0x08, 0x06, 0xe5, 0xe8, 0x09, 0x2c, - 0x45, 0xa3, 0x52, 0x76, 0x33, 0xd5, 0x99, 0xb1, 0x1b, 0x12, 0xdb, 0x67, 0xb0, 0x0d, 0x09, 0x86, - 0x14, 0xe8, 0x73, 0x88, 0x4e, 0x02, 0xfb, 0x9c, 0xbc, 0xab, 0x69, 0x1e, 0xf9, 0xfe, 0xb5, 0x89, - 0x7e, 0x31, 0x61, 0x44, 0xf3, 0x13, 0x1a, 0x54, 0x64, 0xa8, 0x67, 0x07, 0x24, 0x44, 0xbd, 0x59, - 0x1e, 0xf5, 0x76, 0x4c, 0xd4, 0x61, 0x42, 0x5f, 0x4c, 0x30, 0x04, 0xec, 0xc9, 0x50, 0x09, 0x16, - 0x44, 0xa4, 0xc0, 0xf3, 0x30, 0x03, 0x67, 0xb8, 0x3a, 0x54, 0x84, 0xd5, 0xf4, 0x42, 0x09, 0x19, - 0xbb, 0x2c, 0x5e, 0x07, 0x53, 0x49, 0x82, 0xf9, 0x85, 0x9f, 0x1b, 0x7b, 0x59, 0x46, 0xd9, 0x32, - 0xbb, 0x2c, 0x5e, 0x54, 0xca, 0x36, 0xbc, 0x19, 0xd2, 0x67, 0x7c, 0xca, 0xf9, 0xb3, 0x3a, 0x3f, - 0x76, 0xc3, 0xe3, 0x98, 0x36, 0xdb, 0xf0, 0xe6, 0xa0, 0x1c, 0x55, 0xc3, 0xb6, 0x84, 0x4a, 0xfe, - 0xac, 0x2e, 0x8c, 0xcd, 0x72, 0x94, 0x67, 0xb3, 0x2c, 0x9d, 0xa8, 0x94, 0x65, 0xe9, 0x7a, 0x16, - 0xc1, 0xdd, 0xfe, 0xcb, 0x9b, 0xba, 0x38, 0x36, 0xcb, 0xb8, 0x37, 0x3a, 0x96, 0xa5, 0x3b, 0x28, - 0x17, 0x40, 0x71, 0x1a, 0xe0, 0x16, 0xa3, 0xb0, 0xd8, 0x12, 0x1c, 0x56, 0x55, 0xae, 0x00, 0x8a, - 0x18, 0xba, 0x2b, 0x80, 0x62, 0x50, 0xc1, 0xce, 0x65, 0xc8, 0x45, 0x9b, 0x3d, 0x0e, 0xac, 0x2e, - 0x8d, 0x3d, 0x97, 0xf1, 0x7c, 0x99, 0x9d, 0x4b, 0x3a, 0xac, 0xe1, 0x78, 0x29, 0x63, 0x87, 0xe7, - 0x09, 0x8d, 0xc7, 0xcb, 0x11, 0x9e, 0xcc, 0xf1, 0x32, 0x2a, 0x65, 0x8b, 0x6b, 0x86, 0xbd, 0x03, - 0xa6, 0xbc, 0x79, 0x50, 0x37, 0xc7, 0x2e, 0x6e, 0x5c, 0x9b, 0xc1, 0x16, 0xd7, 0x1c, 0x94, 0xb3, - 0x34, 0x05, 0x73, 0xee, 0xc3, 0xfa, 0xcd, 0xb1, 0x69, 0x8e, 0x32, 0x6f, 0x96, 0xa6, 0x1f, 0x95, - 0xa2, 0x5f, 0x24, 0xe1, 0xce, 0x08, 0x8a, 0x70, 0x24, 0xc6, 0xfc, 0x41, 0x1b, 0x53, 0x41, 0x81, - 0xd5, 0x77, 0xf8, 0x30, 0xff, 0x71, 0x0d, 0x60, 0x89, 0x65, 0xcf, 0xc5, 0x84, 0xb1, 0x1d, 0xbc, - 0xc5, 0x90, 0xad, 0x99, 0x2d, 0xb8, 0x25, 0xf6, 0x24, 0xb9, 0x54, 0xb7, 0xc6, 0xae, 0x59, 0x1c, - 0x0d, 0x65, 0x6b, 0x66, 0x0f, 0xca, 0x19, 0xb8, 0x77, 0xfb, 0xef, 0xc8, 0x58, 0xb6, 0x86, 0xea, - 0xf6, 0x58, 0x70, 0x1f, 0xf3, 0xea, 0xcc, 0xc0, 0xbd, 0x3b, 0xa2, 0x42, 0xcf, 0x41, 0xe9, 0x75, - 0xe2, 0xf8, 0x84, 0xd3, 0x4f, 0x55, 0xe3, 0xb1, 0x77, 0x62, 0x62, 0x5f, 0xc1, 0x56, 0x39, 0xc6, - 0x0f, 0x6a, 0xd0, 0x4b, 0xb8, 0xc5, 0x7a, 0x0b, 0x53, 0xf0, 0x76, 0x4c, 0xfa, 0xc4, 0x5d, 0xd2, - 0xf4, 0xdb, 0x7c, 0xa4, 0x87, 0x71, 0xdb, 0x72, 0x35, 0xdd, 0x2f, 0x26, 0x8c, 0xcd, 0x60, 0xac, - 0x09, 0xc3, 0x1a, 0x81, 0xd0, 0xac, 0xd6, 0x33, 0xd2, 0xaa, 0xde, 0x19, 0x7b, 0xce, 0x46, 0xc9, - 0x2d, 0x3b, 0x67, 0x76, 0x54, 0x8a, 0x8e, 0x61, 0xa9, 0xcd, 0x48, 0x29, 0xb6, 0x5d, 0x76, 0xb0, - 0x38, 0x2d, 0x55, 0xef, 0x8e, 0xdd, 0xdb, 0x38, 0x02, 0xcb, 0xd6, 0xa7, 0x3d, 0x28, 0x47, 0xff, - 0x25, 0x69, 0xce, 0x29, 0xe1, 0x3b, 0xcb, 0x2a, 0xe0, 0x7b, 0x63, 0x99, 0x53, 0x0c, 0x89, 0x65, - 0xcc, 0xa9, 0x17, 0x40, 0x54, 0xbf, 0x2f, 0x61, 0xc5, 0x8e, 0xb2, 0x42, 0xdc, 0xe4, 0xb4, 0x50, - 0x7d, 0x9f, 0xc7, 0xfd, 0x20, 0x76, 0xfe, 0xf1, 0x24, 0xb2, 0x98, 0x30, 0x96, 0xed, 0x51, 0x1d, - 0x7a, 0x0c, 0xcb, 0xb6, 0x20, 0x85, 0x92, 0xf3, 0x89, 0xad, 0xbc, 0x37, 0xf2, 0x67, 0x4e, 0x7f, - 0x80, 0x21, 0x0a, 0xc9, 0x30, 0xcc, 0x1e, 0x16, 0xe6, 0x66, 0x60, 0x8a, 0xb7, 0x4c, 0x07, 0xa9, - 0xf4, 0x0d, 0x45, 0x39, 0x48, 0xa5, 0x97, 0x95, 0x95, 0x83, 0x54, 0x7a, 0x45, 0x59, 0x3d, 0x48, - 0xa5, 0x57, 0x95, 0xb5, 0x83, 0x54, 0x7a, 0x4d, 0x59, 0x3f, 0x48, 0xa5, 0xd7, 0x15, 0xf5, 0x20, - 0x95, 0x56, 0x95, 0x8d, 0x83, 0x54, 0x7a, 0x43, 0xd9, 0x3c, 0x48, 0xa5, 0x6f, 0x29, 0x99, 0x83, - 0x54, 0x3a, 0xa3, 0x6c, 0x1d, 0xa4, 0xd2, 0xef, 0x2a, 0x9a, 0x76, 0x9f, 0x33, 0xd4, 0x23, 0xcf, - 0xe7, 0xf5, 0x07, 0x6d, 0xc2, 0x14, 0x9b, 0xc8, 0x85, 0x7c, 0x00, 0x10, 0x3c, 0x57, 0x88, 0xb4, - 0xaf, 0xa7, 0x60, 0x2a, 0xfc, 0xdb, 0x63, 0xe8, 0x59, 0x64, 0x43, 0x76, 0xf5, 0x4b, 0x91, 0xbf, - 0x16, 0x84, 0x41, 0xff, 0xad, 0xe4, 0x7f, 0x06, 0xa9, 0x17, 0x25, 0xfc, 0x1f, 0x13, 0x4e, 0x2c, - 0x17, 0x63, 0x4f, 0xdb, 0x00, 0xdc, 0x70, 0xe3, 0xdc, 0x1d, 0x39, 0xce, 0x3b, 0xfd, 0x71, 0x46, - 0xad, 0x06, 0x98, 0x99, 0x94, 0xa1, 0x3c, 0x2c, 0x74, 0x5d, 0x72, 0xd1, 0xf1, 0x7c, 0x62, 0xf1, - 0xba, 0x9f, 0xba, 0x0e, 0xd1, 0x37, 0xe6, 0x7b, 0x4e, 0xac, 0xda, 0xef, 0xc2, 0x9c, 0x47, 0xed, - 0x96, 0xed, 0x62, 0x56, 0x0b, 0x39, 0x6b, 0x9c, 0xca, 0x2d, 0xca, 0xb7, 0xc2, 0x69, 0x56, 0x37, - 0x4b, 0x05, 0x03, 0x84, 0x09, 0xfb, 0x42, 0x47, 0x30, 0x6d, 0x71, 0xea, 0x2f, 0x59, 0x60, 0x66, - 0xdc, 0x7b, 0x85, 0x68, 0x10, 0x72, 0xaa, 0x9c, 0x9f, 0xd2, 0x9f, 0x9f, 0xd0, 0x18, 0x32, 0x0e, - 0x3a, 0x84, 0x05, 0x86, 0x0d, 0x56, 0x0f, 0x17, 0x04, 0xaf, 0xda, 0x8e, 0x04, 0x0e, 0xff, 0x0b, - 0xdd, 0xd1, 0x85, 0x61, 0xf4, 0x41, 0x64, 0x9e, 0x44, 0x64, 0xe8, 0xdf, 0xc3, 0xdd, 0x9e, 0xb9, - 0x2a, 0xbb, 0xf0, 0x70, 0xc8, 0x73, 0x80, 0xba, 0x30, 0xe9, 0x7a, 0x2f, 0x25, 0x4d, 0x7c, 0x4b, - 0x8b, 0x5c, 0x90, 0x8b, 0xf3, 0xe8, 0xfa, 0x4f, 0x24, 0x2c, 0x40, 0xde, 0xf1, 0x9a, 0x2f, 0x7a, - 0x51, 0x0c, 0x36, 0x9e, 0x78, 0xc5, 0x11, 0x6f, 0xae, 0x0f, 0xfe, 0x36, 0x01, 0xea, 0xb8, 0x67, - 0x7d, 0xd6, 0x07, 0x66, 0x73, 0x35, 0xa3, 0x81, 0x47, 0x9e, 0x97, 0xef, 0xc2, 0xbb, 0x03, 0x1a, - 0xfe, 0xa1, 0x17, 0xb0, 0xa1, 0xe7, 0x6b, 0x46, 0x01, 0xef, 0xd5, 0x8e, 0xab, 0x05, 0x25, 0xc9, - 0x1a, 0xcd, 0x01, 0xb3, 0x7c, 0xb9, 0xa4, 0x57, 0xd9, 0xd7, 0x81, 0x9e, 0x67, 0x8d, 0xe8, 0x16, - 0xdc, 0x1c, 0xd0, 0x1f, 0x1d, 0xd7, 0x8b, 0xba, 0x11, 0x46, 0x53, 0x52, 0xe8, 0x26, 0xac, 0x8f, - 0x8e, 0x83, 0xeb, 0x47, 0xd9, 0xaa, 0x32, 0x85, 0xb2, 0xf0, 0xd9, 0xa0, 0xb2, 0x6c, 0xe8, 0xd9, - 0xc2, 0xb3, 0xfe, 0x6b, 0x37, 0xae, 0x19, 0xd8, 0xa8, 0x95, 0xcb, 0x7a, 0x01, 0xe7, 0xb2, 0xf9, - 0x43, 0x7c, 0x54, 0xab, 0xd7, 0x4b, 0xb9, 0xb2, 0xce, 0xbb, 0xeb, 0xec, 0x33, 0x65, 0x1a, 0xbd, - 0x0f, 0xb7, 0x07, 0x42, 0x54, 0xf5, 0x27, 0xb8, 0xac, 0x67, 0xeb, 0x3a, 0x3e, 0x32, 0xf4, 0xc7, - 0x7a, 0xb5, 0x51, 0xc7, 0x8d, 0xa7, 0x55, 0x25, 0x8d, 0xee, 0xc3, 0xdd, 0x01, 0xc3, 0x46, 0xa9, - 0xa2, 0xd7, 0x1b, 0xd9, 0xca, 0x11, 0xce, 0x67, 0xf3, 0x45, 0x5d, 0x4e, 0x49, 0x2f, 0x28, 0x33, - 0x9b, 0xa9, 0xaf, 0xbf, 0xcd, 0x24, 0x34, 0xb6, 0xbc, 0x13, 0x0f, 0xfe, 0x38, 0xf8, 0x3f, 0x41, - 0xe4, 0x3f, 0x07, 0xd1, 0x64, 0x37, 0x8c, 0x67, 0xa3, 0x8b, 0xcb, 0x3b, 0x7a, 0xa6, 0x79, 0x62, - 0x94, 0x1a, 0x3a, 0x6e, 0xd4, 0x6a, 0xb8, 0x56, 0x66, 0xcb, 0xc9, 0x9f, 0x00, 0x98, 0xa2, 0xae, - 0x1b, 0xa5, 0x6c, 0xb9, 0xf4, 0x79, 0x36, 0x57, 0xd6, 0x95, 0x49, 0x74, 0x0b, 0x36, 0x84, 0x3c, - 0x5b, 0x7f, 0x56, 0xcd, 0x4b, 0xb7, 0xbd, 0x6c, 0xa9, 0x7c, 0x6c, 0xe8, 0xca, 0x14, 0xd2, 0x20, - 0x23, 0xd4, 0x62, 0x61, 0x70, 0x41, 0xcf, 0x16, 0xca, 0xa5, 0xaa, 0x8e, 0xf5, 0xa7, 0x79, 0x5d, - 0x2f, 0xe8, 0x05, 0x65, 0x5a, 0x24, 0xfd, 0xe0, 0x53, 0x40, 0xa3, 0x28, 0x80, 0xd2, 0x90, 0xaa, - 0xd6, 0xaa, 0xba, 0x92, 0x40, 0x73, 0x30, 0xc3, 0x16, 0xb2, 0xb6, 0xb7, 0xa7, 0x24, 0xd1, 0x02, - 0xcc, 0x96, 0x2a, 0x15, 0xbd, 0x50, 0xca, 0x36, 0x74, 0x65, 0x22, 0x77, 0xff, 0xbb, 0x7f, 0x64, - 0x12, 0xdf, 0xbd, 0xce, 0x24, 0xbf, 0x7f, 0x9d, 0x49, 0xfe, 0xf0, 0x3a, 0x93, 0xfc, 0xfb, 0xeb, - 0x4c, 0xf2, 0xd7, 0x6f, 0x32, 0x89, 0xef, 0xdf, 0x64, 0x12, 0x3f, 0xbc, 0xc9, 0x24, 0x3e, 0x9f, - 0x91, 0xb8, 0xf0, 0xaf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x72, 0xe3, 0x19, 0x66, 0x8e, 0x20, 0x00, - 0x00, + 0xb9, 0x27, 0x25, 0x4a, 0xa2, 0x3e, 0x3d, 0x3c, 0x3a, 0x92, 0xa5, 0x91, 0x6c, 0x53, 0xca, 0xd8, + 0x4e, 0x6c, 0x07, 0x91, 0x2e, 0x9c, 0x1b, 0xe0, 0x26, 0xd7, 0x59, 0xf0, 0x31, 0x12, 0x29, 0xf1, + 0xa1, 0x3b, 0xa4, 0xfc, 0x88, 0x11, 0x9c, 0x8c, 0x38, 0x47, 0xd4, 0x5c, 0x0f, 0x67, 0x98, 0x33, + 0x43, 0x59, 0x02, 0xba, 0x28, 0xda, 0x4d, 0xd0, 0x02, 0x7d, 0xac, 0xda, 0x65, 0x81, 0xa0, 0x8b, + 0x02, 0x45, 0xd1, 0xbf, 0xa0, 0xeb, 0x2c, 0xb3, 0x0c, 0x8a, 0xc2, 0x68, 0xed, 0xfe, 0x15, 0x59, + 0x15, 0xe7, 0x31, 0xe4, 0x90, 0x1c, 0xca, 0x4a, 0x76, 0x9c, 0xef, 0x75, 0xbe, 0xf3, 0xfa, 0x7d, + 0xbf, 0xef, 0x10, 0x56, 0xa8, 0x67, 0x36, 0x4f, 0x3b, 0xc7, 0x3b, 0x84, 0x52, 0x8f, 0xfa, 0xdb, + 0x1d, 0xea, 0x05, 0x1e, 0x5a, 0x6a, 0x7a, 0xcd, 0x17, 0x5c, 0xb3, 0xfd, 0xe2, 0x6c, 0xfb, 0xc5, + 0x59, 0xe7, 0x78, 0xe3, 0xba, 0x30, 0x18, 0xb2, 0xdc, 0x40, 0xa1, 0xbf, 0x65, 0x06, 0xa6, 0x94, + 0xad, 0x86, 0xb2, 0x36, 0x09, 0xcc, 0x88, 0x5c, 0xed, 0x06, 0xb6, 0xb3, 0x73, 0xea, 0x34, 0x77, + 0x02, 0xbb, 0x4d, 0xfc, 0xc0, 0x6c, 0x77, 0xa4, 0x66, 0xa5, 0xe5, 0xb5, 0x3c, 0xfe, 0x73, 0x87, + 0xfd, 0x12, 0x52, 0xed, 0xaf, 0x13, 0xb0, 0x5c, 0xf5, 0x82, 0x32, 0x31, 0x7d, 0x52, 0xf4, 0x1c, + 0x8b, 0x50, 0x9d, 0x0d, 0x8d, 0x0a, 0x30, 0x43, 0x49, 0xc7, 0xb1, 0x9b, 0xa6, 0x9a, 0xdc, 0x4a, + 0xde, 0x9b, 0x7b, 0x78, 0x67, 0xbb, 0x9f, 0xaf, 0x1c, 0x7b, 0xdb, 0x10, 0x16, 0x05, 0xe2, 0x37, + 0xa9, 0xdd, 0x09, 0x3c, 0x9a, 0x4b, 0x7d, 0xf3, 0x6a, 0x33, 0x61, 0x84, 0xae, 0x68, 0x0f, 0xe6, + 0x1d, 0x16, 0x19, 0x9f, 0xf2, 0xd0, 0xea, 0xc4, 0xd5, 0x43, 0x19, 0x73, 0x4e, 0x3f, 0x27, 0xb4, + 0x0d, 0x53, 0xfc, 0x53, 0x4d, 0xf1, 0x08, 0x6a, 0x4c, 0x04, 0x3e, 0x05, 0x43, 0x98, 0xa1, 0x8f, + 0x20, 0x4d, 0x4d, 0xb7, 0x45, 0xb0, 0x6d, 0xa9, 0x93, 0x5b, 0xc9, 0x7b, 0x93, 0xb9, 0x0d, 0x96, + 0xd9, 0xeb, 0x57, 0x9b, 0x33, 0x06, 0x93, 0x97, 0x0a, 0xdf, 0xf7, 0x7f, 0x1a, 0x33, 0xdc, 0xb6, + 0x64, 0xa1, 0xdb, 0x00, 0xcd, 0xae, 0x1f, 0x78, 0x6d, 0xdc, 0xf6, 0x5b, 0xea, 0xd4, 0x56, 0xf2, + 0xde, 0xac, 0x9c, 0xd2, 0xac, 0x90, 0x57, 0xfc, 0x96, 0xb6, 0x0a, 0x2b, 0x55, 0xcf, 0x22, 0x47, + 0xae, 0x79, 0x66, 0xda, 0x8e, 0x79, 0xec, 0x10, 0xbe, 0x64, 0xda, 0x3a, 0xac, 0x1d, 0xb9, 0x7e, + 0xb7, 0xd3, 0xf1, 0x68, 0x40, 0x2c, 0x83, 0x7c, 0xd9, 0x25, 0x7e, 0x20, 0x54, 0x3f, 0x4b, 0x02, + 0xe2, 0x83, 0x55, 0xbd, 0x60, 0xd7, 0xeb, 0xba, 0x96, 0x58, 0xe4, 0x68, 0x96, 0xc9, 0xab, 0x67, + 0xf9, 0x11, 0xa4, 0xfd, 0xc0, 0xa3, 0xdc, 0x6d, 0x62, 0xd0, 0xad, 0xce, 0xe4, 0xc2, 0x4d, 0xfe, + 0x34, 0x66, 0xb8, 0x6d, 0xc9, 0xd2, 0x7e, 0x37, 0x09, 0xd7, 0x79, 0xac, 0x03, 0x72, 0x51, 0xb1, + 0xfd, 0xb6, 0x19, 0x34, 0x4f, 0x45, 0x1e, 0x1f, 0xc2, 0x12, 0x15, 0xe9, 0x62, 0x3f, 0x30, 0x69, + 0x80, 0x5f, 0x90, 0x0b, 0x9e, 0xd0, 0x7c, 0x6e, 0xe6, 0xfb, 0x57, 0x9b, 0x93, 0x07, 0xe4, 0xc2, + 0xb8, 0x26, 0x2d, 0xea, 0xcc, 0xe0, 0x80, 0x5c, 0xa0, 0x1d, 0x08, 0x45, 0x98, 0xb8, 0x16, 0x77, + 0x99, 0x18, 0x74, 0x59, 0x90, 0x7a, 0xdd, 0xb5, 0x98, 0xc3, 0x29, 0xdc, 0xb0, 0x48, 0x87, 0x92, + 0xa6, 0x19, 0x10, 0x0b, 0xb7, 0x65, 0x06, 0xc4, 0xc2, 0x7c, 0x5e, 0x7c, 0x9b, 0xe6, 0x1e, 0x6a, + 0x71, 0x67, 0x83, 0xe9, 0x47, 0x0e, 0xd9, 0x7a, 0x3f, 0x58, 0xa5, 0x17, 0x8b, 0x9b, 0xa2, 0x2f, + 0x60, 0x23, 0x32, 0x92, 0xdf, 0x6d, 0xb5, 0x88, 0x1f, 0xf4, 0x06, 0x4a, 0x5d, 0x75, 0x20, 0x43, + 0xed, 0x47, 0xa9, 0x87, 0x41, 0xc4, 0x08, 0x65, 0x98, 0xe6, 0xc1, 0x7c, 0x75, 0x6a, 0x6b, 0xf2, + 0xde, 0xdc, 0xc3, 0x9b, 0xe3, 0xa2, 0x95, 0xdc, 0x13, 0x2f, 0xb7, 0x2a, 0xb7, 0x67, 0x51, 0xf8, + 0x94, 0xdc, 0x80, 0x50, 0xd7, 0x74, 0x0c, 0x19, 0x43, 0x7b, 0x33, 0x09, 0x9a, 0x41, 0x4c, 0xeb, + 0x89, 0x1d, 0x9c, 0xda, 0xee, 0x91, 0xdb, 0x24, 0x34, 0x30, 0x6d, 0x37, 0xb8, 0xe0, 0x96, 0x67, + 0xa6, 0x23, 0xb6, 0x69, 0x1f, 0x16, 0x29, 0x31, 0x2d, 0xdc, 0xbb, 0xd9, 0xf2, 0x6a, 0xde, 0x8a, + 0x0c, 0xce, 0xae, 0xff, 0xf6, 0xa9, 0xd3, 0xdc, 0x6e, 0x84, 0x46, 0x72, 0xb9, 0x16, 0x98, 0x6b, + 0x4f, 0x88, 0x0c, 0x40, 0xe4, 0xdc, 0xf6, 0x03, 0xdb, 0x6d, 0x45, 0xe2, 0x4d, 0x5c, 0x3d, 0xde, + 0x52, 0xe8, 0xde, 0x8f, 0xf9, 0x1c, 0xd6, 0x1c, 0xaf, 0x69, 0x3a, 0xb8, 0xdb, 0x9f, 0x01, 0x76, + 0xec, 0xb6, 0x1d, 0xf0, 0xab, 0x74, 0xc5, 0xc0, 0xd7, 0x79, 0x8c, 0xc8, 0x22, 0x94, 0x59, 0x04, + 0xf4, 0x39, 0xa8, 0x2d, 0xc7, 0x3b, 0x8e, 0x8d, 0x3e, 0x79, 0xf5, 0xe8, 0xab, 0x22, 0xc8, 0x48, + 0xf8, 0xe7, 0xb0, 0xec, 0x1d, 0xfb, 0x84, 0x9e, 0x91, 0xc8, 0xfa, 0xfa, 0x6a, 0x8a, 0xef, 0x6e, + 0x1c, 0x60, 0xd5, 0xa4, 0xf5, 0xf0, 0x00, 0xc8, 0x1b, 0x56, 0xf8, 0x9f, 0xa4, 0x7e, 0xff, 0x87, + 0xcd, 0x84, 0x66, 0xc1, 0x5a, 0x83, 0x9a, 0xae, 0x6f, 0x36, 0x03, 0xdb, 0x73, 0xb3, 0xc7, 0x1c, + 0x26, 0xc4, 0xce, 0x96, 0x60, 0x9a, 0x12, 0xd3, 0xf7, 0x5c, 0xbe, 0xa3, 0x8b, 0x0f, 0xdf, 0xdf, + 0x1e, 0x29, 0x0e, 0xdb, 0xa3, 0xbe, 0x06, 0x77, 0x91, 0xe3, 0xca, 0x00, 0xda, 0x73, 0x58, 0x89, + 0x58, 0x1e, 0x76, 0x7d, 0x79, 0xc7, 0xf3, 0x00, 0x9d, 0xae, 0x7f, 0x4a, 0x08, 0x0e, 0xce, 0x5d, + 0x79, 0x70, 0x32, 0x31, 0xf3, 0x8a, 0x38, 0x87, 0xd0, 0x27, 0xfc, 0x1a, 0xe7, 0xae, 0xf6, 0xf3, + 0x24, 0x5c, 0x8f, 0x18, 0x18, 0x24, 0xa0, 0x17, 0x22, 0xfc, 0xde, 0xd0, 0x0c, 0xee, 0x5f, 0x3e, + 0x03, 0xee, 0x19, 0x97, 0x3f, 0x7a, 0x07, 0x66, 0xc9, 0x79, 0x40, 0x4d, 0x8e, 0xc0, 0x13, 0x11, + 0x04, 0x4e, 0x73, 0x31, 0x03, 0xe0, 0xbf, 0x25, 0x61, 0x35, 0x12, 0xab, 0x1e, 0x98, 0x41, 0xd7, + 0x17, 0x69, 0xac, 0xc2, 0x24, 0xf3, 0x4b, 0x46, 0xfc, 0x98, 0x00, 0x55, 0x7b, 0xe9, 0x4d, 0xf0, + 0xf4, 0xfe, 0xeb, 0xf2, 0xf4, 0x22, 0x21, 0xb7, 0x63, 0x57, 0xf9, 0x11, 0x4c, 0x0b, 0x39, 0x42, + 0xb0, 0x68, 0xe8, 0xd9, 0x7a, 0xad, 0x8a, 0x8f, 0xaa, 0x07, 0xd5, 0xda, 0x93, 0xaa, 0x92, 0x40, + 0x2a, 0xac, 0x48, 0x59, 0xe3, 0x69, 0x15, 0xe7, 0x6b, 0x95, 0x4a, 0xa9, 0xd1, 0xd0, 0x0b, 0xca, + 0x84, 0x96, 0x4a, 0x27, 0x95, 0xa4, 0xf6, 0xdb, 0x24, 0x28, 0x4f, 0xa8, 0x1d, 0x10, 0x76, 0xbf, + 0x5d, 0x51, 0x23, 0xd0, 0xc7, 0x30, 0x63, 0xf3, 0x4f, 0x5f, 0x4d, 0xf2, 0x53, 0xb7, 0x1e, 0xb3, + 0x3b, 0xc2, 0x21, 0x2c, 0xb3, 0xd2, 0x1e, 0x3d, 0x82, 0x45, 0x51, 0x66, 0x7d, 0x06, 0xb8, 0x6e, + 0x93, 0xc8, 0x9a, 0x77, 0x9d, 0x99, 0x7d, 0xff, 0x6a, 0x73, 0x81, 0xd7, 0xc6, 0xba, 0x54, 0x1a, + 0x0b, 0x4e, 0xf4, 0x73, 0x3f, 0x95, 0x9e, 0x50, 0x26, 0xb5, 0x3f, 0x86, 0x39, 0x35, 0x3c, 0xaf, + 0xe6, 0xc8, 0x73, 0x99, 0x85, 0xd9, 0x1f, 0x05, 0x36, 0x7d, 0x2f, 0x54, 0x05, 0xc5, 0x6c, 0x06, + 0x5d, 0xd3, 0xf9, 0x71, 0x30, 0x73, 0x4d, 0x38, 0xf7, 0xc4, 0xda, 0x0a, 0xa0, 0x5a, 0x87, 0x15, + 0x57, 0x9b, 0x12, 0xbf, 0x71, 0xee, 0x8a, 0x02, 0x5b, 0x87, 0x95, 0xbc, 0xe7, 0x5a, 0x36, 0xdb, + 0xbc, 0x5d, 0xd3, 0x76, 0xc2, 0x8b, 0xf5, 0xbf, 0x30, 0x2f, 0x47, 0x3f, 0x33, 0x9d, 0x2e, 0x91, + 0x73, 0x88, 0xa3, 0x0f, 0x8f, 0x99, 0xde, 0x98, 0x13, 0xd6, 0xfc, 0x43, 0xfb, 0x4b, 0x12, 0x90, + 0x60, 0x15, 0xe4, 0xff, 0x49, 0xb3, 0x77, 0x59, 0x33, 0x30, 0xd3, 0x26, 0xbe, 0x6f, 0xb6, 0xc8, + 0xc0, 0x39, 0x0b, 0x85, 0xe8, 0x11, 0xcc, 0xca, 0xc2, 0x47, 0x2c, 0x39, 0xd5, 0xb1, 0x7c, 0x25, + 0x5c, 0xaf, 0x9e, 0x03, 0xfa, 0x04, 0xd2, 0x21, 0xb2, 0x4a, 0x5c, 0x7b, 0x9b, 0x73, 0xcf, 0x5e, + 0xfb, 0x12, 0x56, 0xb2, 0xed, 0x63, 0xbb, 0xd5, 0xf5, 0xba, 0xbe, 0x41, 0xfc, 0xae, 0x13, 0x5c, + 0x2d, 0xe3, 0x8f, 0x61, 0xee, 0x25, 0x35, 0x3b, 0x1d, 0x62, 0x61, 0x42, 0x69, 0x4c, 0xce, 0xe1, + 0x15, 0xe1, 0xe1, 0x0c, 0x90, 0xc6, 0x3a, 0xa5, 0xda, 0x1a, 0xe3, 0x14, 0x27, 0xc1, 0x1e, 0xf5, + 0xba, 0x9d, 0x02, 0x71, 0x48, 0xb8, 0x4a, 0x1a, 0x86, 0x55, 0xc9, 0xe9, 0xf2, 0x1e, 0xa5, 0xdd, + 0x0e, 0xdb, 0x19, 0x91, 0x0d, 0xbb, 0xe1, 0xec, 0x07, 0x1e, 0xbe, 0xa9, 0x69, 0x2e, 0xae, 0xf8, + 0x2d, 0xa4, 0xc1, 0x6c, 0x87, 0x7a, 0x4d, 0xe2, 0xfb, 0x72, 0x09, 0xd3, 0x3d, 0x2c, 0x0a, 0xc5, + 0x5a, 0x1d, 0x90, 0x1c, 0x20, 0x7a, 0x62, 0x3f, 0x05, 0x90, 0xe4, 0x33, 0x24, 0x55, 0x53, 0xb9, + 0x8c, 0x2c, 0xbf, 0xb3, 0xd2, 0x9e, 0xf3, 0xa3, 0xfe, 0x07, 0x5b, 0x7d, 0xf1, 0xd3, 0xd2, 0x0e, + 0x00, 0x71, 0xde, 0x34, 0xc2, 0xd3, 0x7a, 0x84, 0x2b, 0x79, 0x75, 0xc2, 0x55, 0x67, 0x84, 0xf0, + 0xd4, 0x74, 0x2d, 0x87, 0x61, 0x75, 0x40, 0x2f, 0x7a, 0x5c, 0x11, 0x3d, 0x84, 0x54, 0x47, 0xa7, + 0x34, 0xe6, 0x3c, 0x0e, 0x2c, 0xb5, 0x9c, 0x35, 0xb7, 0x95, 0x55, 0xe4, 0xdf, 0x49, 0xb8, 0x3b, + 0x0c, 0xa4, 0x8c, 0x37, 0x1c, 0x32, 0x3a, 0x6f, 0x90, 0x13, 0x4a, 0x42, 0xc4, 0x1f, 0x87, 0x85, + 0xcf, 0x61, 0x3a, 0x38, 0x77, 0x43, 0xf2, 0x38, 0x9f, 0x2b, 0x30, 0xd5, 0xdf, 0x5f, 0x6d, 0x7e, + 0xd8, 0xb2, 0x83, 0xd3, 0xee, 0xf1, 0x76, 0xd3, 0x6b, 0xef, 0xf4, 0xf2, 0xb1, 0x8e, 0xfb, 0xbf, + 0x77, 0x3a, 0x2f, 0x5a, 0x3b, 0xbc, 0xbf, 0xe8, 0x76, 0x6d, 0x6b, 0xfb, 0xe8, 0xa8, 0x54, 0x78, + 0xfd, 0x6a, 0x73, 0xaa, 0x71, 0xee, 0x96, 0x0a, 0xc6, 0x54, 0x70, 0xee, 0x96, 0x2c, 0xb4, 0x0b, + 0x73, 0x41, 0x3f, 0x3b, 0x79, 0x82, 0xaf, 0x56, 0x67, 0xa2, 0x8e, 0xda, 0x2e, 0x6c, 0x36, 0xce, + 0xdd, 0xac, 0xc3, 0x58, 0xcb, 0x85, 0xee, 0x36, 0xbd, 0x2e, 0xa3, 0x42, 0xf2, 0x70, 0x89, 0xf9, + 0xdd, 0x06, 0xe8, 0x50, 0x72, 0x86, 0xf9, 0xa9, 0x19, 0x98, 0xe6, 0x2c, 0x93, 0x8b, 0x63, 0xf8, + 0xcb, 0x24, 0xac, 0x30, 0xd0, 0x6c, 0x11, 0x5a, 0x3b, 0x23, 0xf4, 0xc4, 0xf1, 0x5e, 0x0a, 0xef, + 0x75, 0x98, 0x8c, 0x61, 0xb9, 0x4c, 0x86, 0xee, 0xc3, 0x42, 0xb3, 0x4b, 0x29, 0x71, 0x03, 0x89, + 0x1a, 0x82, 0x64, 0x8b, 0xd8, 0xf3, 0x52, 0xc5, 0x21, 0x02, 0x7d, 0x00, 0xd7, 0x6c, 0xb7, 0x49, + 0x49, 0xbb, 0x6f, 0x3c, 0x19, 0x31, 0x5e, 0xec, 0x29, 0x05, 0xa2, 0x7c, 0x9d, 0x84, 0x1b, 0x39, + 0xc6, 0x54, 0xfb, 0x30, 0x47, 0x4e, 0x3c, 0x4a, 0xf6, 0xf2, 0x3d, 0xbc, 0x6d, 0xfc, 0x28, 0xbc, + 0xed, 0x93, 0x30, 0x16, 0xe2, 0x94, 0x1d, 0x02, 0xcf, 0xb1, 0x7e, 0x08, 0xd0, 0xf6, 0xbd, 0xb4, + 0x36, 0x20, 0x51, 0x67, 0x2a, 0xb6, 0xef, 0xdb, 0x6e, 0x4b, 0xe4, 0xf6, 0x08, 0xe6, 0x5f, 0x52, + 0xcf, 0x6d, 0x61, 0x51, 0x75, 0x64, 0x7a, 0xe3, 0x8b, 0x94, 0x31, 0xc7, 0xcd, 0xc5, 0x47, 0xb8, + 0xdc, 0x13, 0xa3, 0xcb, 0xcd, 0xfa, 0xa9, 0x0a, 0xa1, 0x8c, 0x2a, 0x1f, 0x52, 0xaf, 0x45, 0x89, + 0x2f, 0x0a, 0xaf, 0xf6, 0xab, 0x09, 0x58, 0xe6, 0x1c, 0x7a, 0x97, 0xc8, 0xfb, 0x23, 0x12, 0x39, + 0x18, 0xa2, 0x1a, 0x1f, 0xc4, 0xdc, 0x9e, 0x18, 0xbf, 0xf8, 0x42, 0xfe, 0xa7, 0x64, 0xaf, 0x92, + 0x6f, 0xc0, 0xaa, 0xac, 0xda, 0x86, 0x7e, 0x58, 0x2e, 0xe5, 0xb3, 0xd8, 0xd0, 0x2b, 0xb5, 0xc7, + 0x7a, 0x41, 0x49, 0xa0, 0x55, 0x40, 0xa1, 0x2e, 0x5b, 0xdd, 0xd3, 0x71, 0xfd, 0xb0, 0x5c, 0x6a, + 0x28, 0x49, 0xb4, 0x06, 0xcb, 0x03, 0xf2, 0x8a, 0x6e, 0xec, 0xb1, 0x42, 0x1f, 0xa1, 0x00, 0x46, + 0x76, 0xb7, 0x81, 0xeb, 0xd5, 0xec, 0x61, 0xbd, 0x58, 0x6b, 0x28, 0x93, 0x28, 0x03, 0x1b, 0x52, + 0x53, 0xae, 0xed, 0x95, 0xf2, 0xd9, 0x32, 0xae, 0x1d, 0xd6, 0x71, 0xa5, 0x54, 0xaf, 0x97, 0xaa, + 0x7b, 0x4a, 0x2a, 0xe2, 0x59, 0x2f, 0xd7, 0x9e, 0xe0, 0x7c, 0xad, 0x5a, 0x3f, 0xaa, 0xe8, 0x86, + 0x32, 0xa5, 0x99, 0xa0, 0x96, 0x5c, 0x8b, 0x04, 0x84, 0xb6, 0x6d, 0xd7, 0x0c, 0x48, 0xde, 0x6b, + 0xb7, 0x6d, 0x09, 0xf1, 0x3a, 0xcc, 0xf9, 0x81, 0xd9, 0xe2, 0x74, 0xfe, 0x07, 0xf2, 0x3b, 0x90, + 0x8e, 0x8c, 0xe0, 0x2d, 0xc3, 0x52, 0xc9, 0x3d, 0x33, 0x1d, 0xdb, 0xe2, 0x15, 0x46, 0x6c, 0x44, + 0x06, 0x6e, 0xd6, 0x3a, 0x81, 0xdd, 0x66, 0x55, 0xa6, 0xa9, 0x9f, 0x99, 0x4e, 0xde, 0x73, 0x4f, + 0x1c, 0xbb, 0x19, 0xc8, 0x8d, 0xfa, 0xf5, 0x2a, 0xcc, 0xf1, 0x5f, 0x05, 0x12, 0x98, 0xb6, 0x83, + 0x0c, 0x50, 0x5c, 0x2f, 0xc0, 0x03, 0x9d, 0xbf, 0x48, 0xe8, 0xdd, 0x98, 0xad, 0x8a, 0x79, 0x7d, + 0x28, 0x26, 0x8c, 0x45, 0x77, 0x40, 0x8c, 0x6a, 0x70, 0x4d, 0xb4, 0xca, 0x2c, 0xf2, 0x09, 0x83, + 0x66, 0x79, 0xb8, 0xef, 0x8e, 0xdb, 0xfd, 0x01, 0x08, 0x2f, 0xb2, 0x06, 0x28, 0x2a, 0x45, 0x4f, + 0x01, 0x89, 0x80, 0x2f, 0xc8, 0x45, 0xaf, 0x19, 0x95, 0x78, 0x75, 0x6f, 0x5c, 0xcc, 0xe1, 0xce, + 0xb9, 0x98, 0x30, 0x14, 0x3a, 0xa4, 0x40, 0x3f, 0x4d, 0xc2, 0x16, 0xef, 0xd3, 0x5e, 0xf2, 0x76, + 0x6e, 0xa0, 0x5f, 0xb1, 0x65, 0x43, 0x27, 0x9b, 0xd0, 0x8f, 0xe2, 0x06, 0x7a, 0x6b, 0x23, 0x58, + 0x4c, 0x18, 0xb7, 0xe8, 0x65, 0x56, 0xe8, 0x73, 0x58, 0x8e, 0x80, 0x29, 0x36, 0x45, 0xbf, 0x20, + 0xbb, 0xb0, 0x07, 0x57, 0x6a, 0x2e, 0xc2, 0x91, 0x50, 0x30, 0xa2, 0x42, 0x0d, 0x50, 0xa2, 0xe1, + 0x59, 0x7f, 0xa0, 0x4e, 0xf3, 0xd8, 0xef, 0x5d, 0x1e, 0xbb, 0xd7, 0x8e, 0x14, 0x13, 0xc6, 0xb5, + 0x60, 0x50, 0x8e, 0x9e, 0xc0, 0x52, 0x34, 0x2a, 0x65, 0x37, 0x57, 0x9d, 0x19, 0xbb, 0x21, 0xb1, + 0x7d, 0x08, 0xdb, 0x90, 0x60, 0x48, 0x81, 0x3e, 0x83, 0xe8, 0x24, 0xb0, 0xcf, 0xc9, 0xbd, 0x9a, + 0xe6, 0x91, 0xef, 0x5f, 0xb9, 0x11, 0x28, 0x26, 0x8c, 0x68, 0x7e, 0x42, 0x83, 0x8a, 0x0c, 0x15, + 0xed, 0x80, 0x84, 0xa8, 0x38, 0xcb, 0xa3, 0xde, 0x8e, 0x89, 0x3a, 0x4c, 0xf8, 0x8b, 0x09, 0x86, + 0x90, 0x3d, 0x19, 0x2a, 0xc1, 0x82, 0x88, 0x14, 0x78, 0x1e, 0x66, 0xe0, 0x0d, 0x97, 0x87, 0x8a, + 0xb0, 0x9e, 0x5e, 0x28, 0x21, 0x63, 0x97, 0xc5, 0xeb, 0x60, 0x2a, 0x49, 0x32, 0x07, 0x84, 0xb9, + 0xb1, 0x97, 0x65, 0x94, 0x4d, 0xb3, 0xcb, 0xe2, 0x45, 0xa5, 0x6c, 0xc3, 0x9b, 0x21, 0xbd, 0xc6, + 0x27, 0x9c, 0x5f, 0xab, 0xf3, 0x63, 0x37, 0x3c, 0x8e, 0x89, 0xb3, 0x0d, 0x6f, 0x0e, 0xca, 0x51, + 0x35, 0x6c, 0x5b, 0xa8, 0xe4, 0xd7, 0xea, 0xc2, 0xd8, 0x2c, 0x47, 0x79, 0x38, 0xcb, 0xd2, 0x89, + 0x4a, 0x59, 0x96, 0xae, 0x67, 0x11, 0xdc, 0xed, 0xbf, 0xcc, 0xa9, 0x8b, 0x63, 0xb3, 0x8c, 0x7b, + 0xc3, 0x63, 0x59, 0xba, 0x83, 0x72, 0x01, 0x14, 0x27, 0x01, 0x6e, 0x31, 0x8a, 0x8b, 0x2d, 0xc1, + 0x71, 0x55, 0xe5, 0x12, 0xa0, 0x88, 0xa1, 0xc3, 0x02, 0x28, 0x06, 0x15, 0xec, 0x5c, 0x86, 0x5c, + 0xb5, 0xd9, 0xe3, 0xc8, 0xea, 0xd2, 0xd8, 0x73, 0x19, 0xcf, 0xa7, 0xd9, 0xb9, 0xa4, 0xc3, 0x1a, + 0x8e, 0x97, 0x32, 0x76, 0x78, 0x9e, 0xd0, 0x78, 0xbc, 0x1c, 0xe1, 0xd1, 0x1c, 0x2f, 0xa3, 0x52, + 0xb6, 0xb8, 0x66, 0xd8, 0x5b, 0x60, 0xca, 0x9b, 0x0b, 0x75, 0x63, 0xec, 0xe2, 0xc6, 0xb5, 0x21, + 0x6c, 0x71, 0xcd, 0x41, 0x39, 0x4b, 0x53, 0x30, 0xeb, 0x3e, 0xac, 0xdf, 0x18, 0x9b, 0xe6, 0x28, + 0x33, 0x67, 0x69, 0xfa, 0x51, 0x29, 0xfa, 0x45, 0x12, 0xee, 0x8c, 0xa0, 0x08, 0x47, 0x62, 0xcc, + 0x1f, 0xbc, 0x31, 0x15, 0x14, 0x59, 0xbd, 0xc9, 0x87, 0xf9, 0x9f, 0x2b, 0x00, 0x4b, 0x2c, 0xbb, + 0x2e, 0x26, 0x8c, 0xad, 0xe0, 0x2d, 0x86, 0x6c, 0xcd, 0x6c, 0xc1, 0x3d, 0xb1, 0x27, 0xc9, 0xa7, + 0xba, 0x39, 0x76, 0xcd, 0xe2, 0x68, 0x2a, 0x5b, 0x33, 0x7b, 0x50, 0xce, 0xc0, 0xbd, 0xdb, 0x7f, + 0x67, 0xc6, 0xb2, 0x75, 0x54, 0xb7, 0xc6, 0x82, 0xfb, 0x98, 0x57, 0x69, 0x06, 0xee, 0xdd, 0x11, + 0x15, 0x7a, 0x0e, 0x4a, 0xaf, 0x53, 0xc7, 0xc7, 0x9c, 0x9e, 0xaa, 0x1a, 0x8f, 0xbd, 0x1d, 0x13, + 0xfb, 0x12, 0x36, 0xcb, 0x31, 0x7e, 0x50, 0x83, 0x5e, 0xc2, 0x2d, 0xd6, 0x7b, 0x98, 0x82, 0xd7, + 0x63, 0xd2, 0x27, 0xf6, 0x92, 0xc6, 0xdf, 0xe6, 0x23, 0x3d, 0x8c, 0xdb, 0x96, 0xcb, 0xdb, 0x81, + 0x62, 0xc2, 0xd8, 0x08, 0xc6, 0x9a, 0x30, 0xac, 0x11, 0x08, 0xcd, 0x6a, 0x3d, 0x23, 0xb5, 0xea, + 0x9d, 0xb1, 0xe7, 0x6c, 0x94, 0xfc, 0xb2, 0x73, 0x66, 0x47, 0xa5, 0xe8, 0x08, 0x96, 0xda, 0x8c, + 0xb4, 0x62, 0xdb, 0x65, 0x07, 0x8b, 0xd3, 0x56, 0xf5, 0xee, 0xd8, 0xbd, 0x8d, 0x23, 0xb8, 0x6c, + 0x7d, 0xda, 0x83, 0x72, 0xf4, 0x7f, 0x92, 0xe6, 0x9c, 0x10, 0xbe, 0xb3, 0xac, 0x02, 0xbe, 0x3b, + 0x96, 0x39, 0xc5, 0x90, 0x5c, 0xc6, 0x9c, 0x7a, 0x01, 0x44, 0xf5, 0xfb, 0x02, 0x56, 0xec, 0x28, + 0x6b, 0xc4, 0x4d, 0x4e, 0x1b, 0xd5, 0xf7, 0x78, 0xdc, 0xf7, 0x63, 0xe7, 0x1f, 0x4f, 0x32, 0x8b, + 0x09, 0x63, 0xd9, 0x1e, 0xd5, 0xa1, 0xc7, 0xb0, 0x6c, 0x0b, 0xd2, 0x28, 0x39, 0x9f, 0xd8, 0xca, + 0x7b, 0x23, 0x7f, 0xf6, 0xf4, 0x07, 0x18, 0xa2, 0x98, 0x0c, 0xc3, 0xec, 0x61, 0x21, 0x6a, 0xc3, + 0xba, 0xd7, 0xe3, 0x9d, 0x98, 0x9c, 0x99, 0x0e, 0x6e, 0x86, 0xcc, 0x53, 0xbd, 0xcf, 0xa3, 0xef, + 0xc4, 0x16, 0xb4, 0xf1, 0x5c, 0xb5, 0x98, 0x30, 0xd6, 0xbc, 0x78, 0x7d, 0x6e, 0x06, 0xa6, 0x78, + 0x07, 0xb7, 0x9f, 0x4a, 0x5f, 0x53, 0x94, 0xfd, 0x54, 0x7a, 0x59, 0x59, 0xd9, 0x4f, 0xa5, 0x57, + 0x94, 0xeb, 0xfb, 0xa9, 0xf4, 0x75, 0x65, 0x75, 0x3f, 0x95, 0x5e, 0x55, 0xd6, 0xf6, 0x53, 0xe9, + 0x35, 0x45, 0xdd, 0x4f, 0xa5, 0x55, 0x65, 0x7d, 0x3f, 0x95, 0x5e, 0x57, 0x36, 0xf6, 0x53, 0xe9, + 0x5b, 0x4a, 0x66, 0x3f, 0x95, 0xce, 0x28, 0x9b, 0xfb, 0xa9, 0xf4, 0x3b, 0x8a, 0xa6, 0xdd, 0xe7, + 0x84, 0xf8, 0xd0, 0xf3, 0x79, 0xb9, 0x43, 0x1b, 0x30, 0xc5, 0xd6, 0xed, 0x5c, 0xbe, 0x47, 0x08, + 0xda, 0x2d, 0x44, 0xda, 0x57, 0x53, 0x30, 0x15, 0xfe, 0x0b, 0x33, 0xf4, 0x4a, 0xb3, 0x2e, 0x1f, + 0x19, 0x96, 0x22, 0xff, 0x74, 0x08, 0x83, 0xfe, 0xd3, 0xcd, 0x4f, 0x06, 0x99, 0x1e, 0x25, 0xfc, + 0x0f, 0x1c, 0xce, 0x63, 0x17, 0x63, 0x0f, 0xf7, 0x00, 0xba, 0x71, 0xe3, 0xdc, 0x1d, 0x39, 0xce, + 0xcd, 0xfe, 0x38, 0xa3, 0x56, 0x03, 0x44, 0x50, 0xca, 0x50, 0x1e, 0x16, 0xba, 0x2e, 0x39, 0xef, + 0x78, 0x3e, 0xb1, 0x38, 0xcd, 0x48, 0x5d, 0xa5, 0xef, 0x30, 0xe6, 0x7b, 0x4e, 0x8c, 0x5c, 0xec, + 0xc0, 0x9c, 0x47, 0xed, 0x96, 0xed, 0x62, 0x56, 0x7a, 0x39, 0x49, 0x9d, 0xca, 0x2d, 0xca, 0xa7, + 0xcb, 0x69, 0x56, 0xa6, 0x4b, 0x05, 0x03, 0x84, 0x09, 0xfb, 0x42, 0x87, 0x30, 0x6d, 0xf1, 0x4e, + 0x43, 0x92, 0xce, 0xcc, 0xb8, 0xe7, 0x13, 0xd1, 0x8f, 0xe4, 0x54, 0x39, 0x3f, 0xa5, 0x3f, 0x3f, + 0xa1, 0x31, 0x64, 0x1c, 0x74, 0x00, 0x0b, 0x0c, 0x8a, 0xac, 0x1e, 0x0c, 0x09, 0x1a, 0xb7, 0x15, + 0x09, 0x1c, 0xfe, 0x35, 0xbb, 0xad, 0x0b, 0xc3, 0xe8, 0xfb, 0xcc, 0x3c, 0x89, 0xc8, 0xd0, 0x7f, + 0x87, 0xbb, 0x3d, 0x73, 0x59, 0x76, 0xe1, 0xe1, 0x90, 0xe7, 0x00, 0x75, 0x61, 0xd2, 0xf5, 0x5e, + 0x4a, 0x56, 0xfa, 0x96, 0x8e, 0xbd, 0x20, 0x17, 0xe7, 0xd1, 0xd5, 0x5f, 0x6c, 0x58, 0x80, 0xbc, + 0xe3, 0x35, 0x5f, 0xf4, 0xa2, 0x18, 0x6c, 0x3c, 0xf1, 0xa8, 0x24, 0x9e, 0x80, 0x1f, 0xfc, 0x63, + 0x02, 0xd4, 0x71, 0xff, 0x32, 0xb0, 0xb6, 0x34, 0x9b, 0xab, 0x19, 0x0d, 0x3c, 0xf2, 0xda, 0x7d, + 0x17, 0xde, 0x19, 0xd0, 0xf0, 0x0f, 0xbd, 0x80, 0x0d, 0x3d, 0x5f, 0x33, 0x0a, 0x78, 0xb7, 0x76, + 0x54, 0x2d, 0x28, 0x49, 0xd6, 0xf7, 0x0e, 0x98, 0xe5, 0xcb, 0x25, 0xbd, 0xca, 0xbe, 0xf6, 0xf5, + 0x3c, 0xeb, 0x8b, 0x37, 0xe1, 0xc6, 0x80, 0xfe, 0xf0, 0xa8, 0x5e, 0xd4, 0x8d, 0x30, 0x9a, 0x92, + 0x42, 0x37, 0x60, 0x6d, 0x74, 0x1c, 0x5c, 0x3f, 0xcc, 0x56, 0x95, 0x29, 0x94, 0x85, 0x4f, 0x07, + 0x95, 0x65, 0x43, 0xcf, 0x16, 0x9e, 0xf5, 0x1f, 0xdf, 0x71, 0xcd, 0xc0, 0x46, 0xad, 0x5c, 0xd6, + 0x0b, 0x38, 0x97, 0xcd, 0x1f, 0xe0, 0xc3, 0x5a, 0xbd, 0x5e, 0xca, 0x95, 0x75, 0xde, 0xec, 0x67, + 0x9f, 0x29, 0xd3, 0xe8, 0x3d, 0xb8, 0x3d, 0x10, 0xa2, 0xaa, 0x3f, 0xc1, 0x65, 0x3d, 0x5b, 0xd7, + 0xf1, 0xa1, 0xa1, 0x3f, 0xd6, 0xab, 0x8d, 0x3a, 0x6e, 0x3c, 0xad, 0x2a, 0x69, 0x74, 0x1f, 0xee, + 0x0e, 0x18, 0x36, 0x4a, 0x15, 0xbd, 0xde, 0xc8, 0x56, 0x0e, 0x71, 0x3e, 0x9b, 0x2f, 0xea, 0x72, + 0x4a, 0x7a, 0x41, 0x99, 0xd9, 0x48, 0x7d, 0xf5, 0x75, 0x26, 0xa1, 0xb1, 0xe5, 0x9d, 0x78, 0xf0, + 0xe7, 0xc1, 0xbf, 0x2d, 0x22, 0x7f, 0x81, 0x88, 0x9e, 0xbf, 0x61, 0x3c, 0x1b, 0x5d, 0x5c, 0xfe, + 0xc0, 0xc0, 0x34, 0x4f, 0x8c, 0x52, 0x43, 0xc7, 0x8d, 0x5a, 0x0d, 0xd7, 0xca, 0x6c, 0x39, 0xf9, + 0x8b, 0x04, 0x53, 0xd4, 0x75, 0xa3, 0x94, 0x2d, 0x97, 0x3e, 0xcb, 0xe6, 0xca, 0xba, 0x32, 0x89, + 0x6e, 0xc1, 0xba, 0x90, 0x67, 0xeb, 0xcf, 0xaa, 0x79, 0xe9, 0xb6, 0x9b, 0x2d, 0x95, 0x8f, 0x0c, + 0x5d, 0x99, 0x42, 0x1a, 0x64, 0x84, 0x5a, 0x2c, 0x0c, 0x2e, 0xe8, 0xd9, 0x42, 0xb9, 0x54, 0xd5, + 0xb1, 0xfe, 0x34, 0xaf, 0xeb, 0x05, 0xbd, 0xa0, 0x4c, 0x8b, 0xa4, 0x1f, 0x7c, 0x02, 0x68, 0x14, + 0x05, 0x50, 0x1a, 0x52, 0xd5, 0x5a, 0x55, 0x57, 0x12, 0x68, 0x0e, 0x66, 0xd8, 0x42, 0xd6, 0x76, + 0x77, 0x95, 0x24, 0x5a, 0x80, 0xd9, 0x52, 0xa5, 0xa2, 0x17, 0x4a, 0xd9, 0x86, 0xae, 0x4c, 0xe4, + 0xee, 0x7f, 0xf3, 0xaf, 0x4c, 0xe2, 0x9b, 0xd7, 0x99, 0xe4, 0xb7, 0xaf, 0x33, 0xc9, 0xef, 0x5e, + 0x67, 0x92, 0xff, 0x7c, 0x9d, 0x49, 0xfe, 0xe6, 0x4d, 0x26, 0xf1, 0xed, 0x9b, 0x4c, 0xe2, 0xbb, + 0x37, 0x99, 0xc4, 0x67, 0x33, 0x12, 0x17, 0xfe, 0x13, 0x00, 0x00, 0xff, 0xff, 0xf9, 0xa4, 0x7e, + 0xc7, 0x1d, 0x21, 0x00, 0x00, } func (m *NotLeaseHolderError) Marshal() (dAtA []byte, err error) { @@ -3402,6 +3455,29 @@ func (m *InvalidLeaseError) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *OptimisticEvalConflictsError) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *OptimisticEvalConflictsError) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OptimisticEvalConflictsError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + func (m *ErrorDetail) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4073,6 +4149,29 @@ func (m *ErrorDetail_InvalidLeaseError) MarshalToSizedBuffer(dAtA []byte) (int, } return len(dAtA) - i, nil } +func (m *ErrorDetail_OptimisticEvalConflicts) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ErrorDetail_OptimisticEvalConflicts) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.OptimisticEvalConflicts != nil { + { + size, err := m.OptimisticEvalConflicts.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintErrors(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} func (m *ErrPosition) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4589,6 +4688,15 @@ func (m *InvalidLeaseError) Size() (n int) { return n } +func (m *OptimisticEvalConflictsError) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *ErrorDetail) Size() (n int) { if m == nil { return 0 @@ -4949,6 +5057,18 @@ func (m *ErrorDetail_InvalidLeaseError) Size() (n int) { } return n } +func (m *ErrorDetail_OptimisticEvalConflicts) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.OptimisticEvalConflicts != nil { + l = m.OptimisticEvalConflicts.Size() + n += 2 + l + sovErrors(uint64(l)) + } + return n +} func (m *ErrPosition) Size() (n int) { if m == nil { return 0 @@ -8011,6 +8131,56 @@ func (m *InvalidLeaseError) Unmarshal(dAtA []byte) error { } return nil } +func (m *OptimisticEvalConflictsError) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: OptimisticEvalConflictsError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OptimisticEvalConflictsError: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipErrors(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthErrors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ErrorDetail) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -9055,6 +9225,41 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error { } m.Value = &ErrorDetail_InvalidLeaseError{v} iNdEx = postIndex + case 41: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OptimisticEvalConflicts", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthErrors + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &OptimisticEvalConflictsError{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ErrorDetail_OptimisticEvalConflicts{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index ee5919afe605..3675573003ea 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -486,6 +486,13 @@ message IndeterminateCommitError { message InvalidLeaseError { } +// OptimisticEvalConflictsError indicates that the optimistic evaluation of a +// BatchRequest failed when checking locks or latches after evaluation. This +// is never sent over-the-wire, and is a proto only to fit the existing +// plumbing that passes a roachpb.Error. +message OptimisticEvalConflictsError { +} + // ErrorDetail is a union type containing all available errors. message ErrorDetail { reserved 15, 19, 20, 21, 22, 23, 24, 25, 29, 30, 33; @@ -524,6 +531,7 @@ message ErrorDetail { RangeFeedRetryError rangefeed_retry = 38; IndeterminateCommitError indeterminate_commit = 39; InvalidLeaseError invalid_lease_error = 40; + OptimisticEvalConflictsError optimistic_eval_conflicts = 41; } } diff --git a/pkg/sql/logictest/testdata/logic_test/select_for_update b/pkg/sql/logictest/testdata/logic_test/select_for_update index 5c697d8fcf12..251eab1eac1b 100644 --- a/pkg/sql/logictest/testdata/logic_test/select_for_update +++ b/pkg/sql/logictest/testdata/logic_test/select_for_update @@ -433,7 +433,7 @@ SELECT * FROM p1_0 FOR SHARE NOWAIT query error pgcode 55P03 could not obtain lock on row in interleaved table SELECT * FROM p1_1 FOR SHARE NOWAIT -query error pgcode 55P03 could not obtain lock on row \(i\)=\(7\) in p2@primary +query error pgcode 55P03 could not obtain lock on row (\(i\)=\(7\) in p2@primary|in interleaved table) SELECT s, s1 FROM p2 JOIN p1_0 USING (i) ORDER BY s FOR SHARE NOWAIT user root @@ -456,7 +456,7 @@ SELECT * FROM p1_0 FOR SHARE NOWAIT query error pgcode 55P03 could not obtain lock on row in interleaved table SELECT * FROM p1_1 FOR SHARE NOWAIT -query error pgcode 55P03 could not obtain lock on row in interleaved table +query error pgcode 55P03 could not obtain lock on row (\(i\)=\(5\) in p1_0@primary|in interleaved table) SELECT s, s1 FROM p2 JOIN p1_0 USING (i) ORDER BY s FOR SHARE NOWAIT user root diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index d597943a92ad..7846ba9ea1e9 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -354,6 +354,7 @@ var charts = []sectionDescription{ "distsender.rpc.err.nodeunavailableerrtype", "distsender.rpc.err.notleaseholdererrtype", "distsender.rpc.err.oprequirestxnerrtype", + "distsender.rpc.err.optimisticevalconflictserrtype", "distsender.rpc.err.raftgroupdeletederrtype", "distsender.rpc.err.rangefeedretryerrtype", "distsender.rpc.err.rangekeymismatcherrtype", diff --git a/pkg/workload/kv/BUILD.bazel b/pkg/workload/kv/BUILD.bazel index 7e5157fdddc8..d1b68edb06d7 100644 --- a/pkg/workload/kv/BUILD.bazel +++ b/pkg/workload/kv/BUILD.bazel @@ -9,10 +9,12 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/workload/kv", visibility = ["//visibility:public"], deps = [ + "//pkg/sql/pgwire/pgcode", "//pkg/util/timeutil", "//pkg/workload", "//pkg/workload/histogram", "@com_github_cockroachdb_errors//:errors", + "@com_github_jackc_pgx//:pgx", "@com_github_spf13_pflag//:pflag", ], ) diff --git a/pkg/workload/kv/kv.go b/pkg/workload/kv/kv.go index 40eac4a4378e..b255f0b142c4 100644 --- a/pkg/workload/kv/kv.go +++ b/pkg/workload/kv/kv.go @@ -22,11 +22,14 @@ import ( "strconv" "strings" "sync/atomic" + "time" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/histogram" "github.com/cockroachdb/errors" + "github.com/jackc/pgx" "github.com/spf13/pflag" ) @@ -66,6 +69,8 @@ type kv struct { cycleLength int64 readPercent int spanPercent int + spanLimit int + writesUseSelectForUpdate bool seed int64 writeSeq string sequential bool @@ -114,6 +119,10 @@ var kvMeta = workload.Meta{ `Percent (0-100) of operations that are reads of existing keys.`) g.flags.IntVar(&g.spanPercent, `span-percent`, 0, `Percent (0-100) of operations that are spanning queries of all ranges.`) + g.flags.IntVar(&g.spanLimit, `span-limit`, 0, + `LIMIT count for each spanning query, or 0 for no limit`) + g.flags.BoolVar(&g.writesUseSelectForUpdate, `sfu-writes`, false, + `Use SFU and transactional writes with a sleep after SFU.`) g.flags.Int64Var(&g.seed, `seed`, 1, `Key hash seed.`) g.flags.BoolVar(&g.zipfian, `zipfian`, false, `Pick keys in a zipfian distribution instead of randomly.`) @@ -301,8 +310,32 @@ func (w *kv) Ops( } writeStmtStr := buf.String() + // Select for update statement + var sfuStmtStr string + if w.writesUseSelectForUpdate { + if w.shards != 0 { + return workload.QueryLoad{}, fmt.Errorf("select for update in kv requires shard=0") + } + buf.Reset() + buf.WriteString(`SELECT k, v FROM kv WHERE k IN (`) + for i := 0; i < w.batchSize; i++ { + if i > 0 { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, `$%d`, i+1) + } + buf.WriteString(`) FOR UPDATE`) + sfuStmtStr = buf.String() + } + // Span statement - spanStmtStr := "SELECT count(v) FROM kv" + buf.Reset() + buf.WriteString(`SELECT count(v) FROM [SELECT v FROM kv`) + if w.spanLimit > 0 { + fmt.Fprintf(&buf, ` ORDER BY k LIMIT %d`, w.spanLimit) + } + buf.WriteString(`]`) + spanStmtStr := buf.String() ql := workload.QueryLoad{SQLDatabase: sqlDatabase} seq := &sequence{config: w, val: int64(writeSeq)} @@ -315,10 +348,14 @@ func (w *kv) Ops( } op.readStmt = op.sr.Define(readStmtStr) op.writeStmt = op.sr.Define(writeStmtStr) + if len(sfuStmtStr) > 0 { + op.sfuStmt = op.sr.Define(sfuStmtStr) + } op.spanStmt = op.sr.Define(spanStmtStr) if err := op.sr.Init(ctx, "kv", mcp, w.connFlags); err != nil { return workload.QueryLoad{}, err } + op.mcp = mcp if w.sequential { op.g = newSequentialGenerator(seq) } else if w.zipfian { @@ -336,9 +373,11 @@ type kvOp struct { config *kv hists *histogram.Histograms sr workload.SQLRunner + mcp *workload.MultiConnPool readStmt workload.StmtHandle writeStmt workload.StmtHandle spanStmt workload.StmtHandle + sfuStmt workload.StmtHandle g keyGenerator numEmptyResults *int64 // accessed atomically } @@ -377,19 +416,75 @@ func (o *kvOp) run(ctx context.Context) error { return err } const argCount = 2 - args := make([]interface{}, argCount*o.config.batchSize) + writeArgs := make([]interface{}, argCount*o.config.batchSize) + var sfuArgs []interface{} + if o.config.writesUseSelectForUpdate { + sfuArgs = make([]interface{}, o.config.batchSize) + } for i := 0; i < o.config.batchSize; i++ { j := i * argCount - args[j+0] = o.g.writeKey() - args[j+1] = randomBlock(o.config, o.g.rand()) + writeArgs[j+0] = o.g.writeKey() + if sfuArgs != nil { + sfuArgs[i] = writeArgs[j] + } + writeArgs[j+1] = randomBlock(o.config, o.g.rand()) } start := timeutil.Now() - _, err := o.writeStmt.Exec(ctx, args...) + var err error + if o.config.writesUseSelectForUpdate { + // We could use crdb.ExecuteTx, but we avoid retries in this workload so + // that each run call makes 1 attempt, so that rate limiting in workerRun + // behaves as expected. + var tx *pgx.Tx + if tx, err = o.mcp.Get().Begin(); err != nil { + return err + } + defer func() { + _ = tx.Rollback() + }() + rows, err := o.sfuStmt.QueryTx(ctx, tx, sfuArgs...) + if err != nil { + return err + } + rows.Close() + if err = rows.Err(); err != nil { + return err + } + // Simulate a transaction that does other work between the SFU and write. + // TODO(sumeer): this should be configurable. + time.Sleep(10 * time.Millisecond) + if _, err = o.writeStmt.ExecTx(ctx, tx, writeArgs...); err != nil { + // Multiple write transactions can contend and encounter + // a serialization failure. We swallow such an error. + return o.tryHandleWriteErr("write-write-err", start, err) + } + if err = tx.Commit(); err != nil { + return o.tryHandleWriteErr("write-commit-err", start, err) + } + } else { + _, err = o.writeStmt.Exec(ctx, writeArgs...) + } elapsed := timeutil.Since(start) o.hists.Get(`write`).Record(elapsed) return err } +func (o *kvOp) tryHandleWriteErr(name string, start time.Time, err error) error { + // If the error not an instance of pgx.PgError, then it is unexpected. + pgErr := pgx.PgError{} + if !errors.As(err, &pgErr) { + return err + } + // Transaction retry errors are acceptable. Allow the transaction + // to rollback. + if pgcode.MakeCode(pgErr.Code) == pgcode.SerializationFailure { + elapsed := timeutil.Since(start) + o.hists.Get(name).Record(elapsed) + return nil + } + return err +} + func (o *kvOp) close(context.Context) { if empty := atomic.LoadInt64(o.numEmptyResults); empty != 0 { fmt.Printf("Number of reads that didn't return any results: %d.\n", empty)