diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 0d10e1ea5acb..a46fcc36e2f9 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -146,4 +146,4 @@ trace.datadog.project string CockroachDB the project under which traces will be trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -version version 21.1-124 set the active cluster version in the format '.' +version version 21.1-126 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 422d6c60c69b..508c0b18a364 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -150,6 +150,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -versionversion21.1-124set the active cluster version in the format '.' +versionversion21.1-126set the active cluster version in the format '.' diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 3e330863b7e1..381f9f4af3ae 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -24,6 +24,8 @@ import ( "sort" "strconv" "strings" + "sync" + "sync/atomic" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -1307,6 +1309,99 @@ func runDebugMergeLogs(cmd *cobra.Command, args []string) error { return writeLogStream(s, cmd.OutOrStdout(), o.filter, o.prefix, o.keepRedactable) } +var debugIntentCount = &cobra.Command{ + Use: "intent-count ", + Short: "return a count of intents in directory", + Long: ` +Returns a count of interleaved and separated intents in the store directory. +Used to investigate stores with lots of unresolved intents, or to confirm +if the migration away from interleaved intents was successful. +`, + Args: cobra.MinimumNArgs(1), + RunE: runDebugIntentCount, +} + +func runDebugIntentCount(cmd *cobra.Command, args []string) error { + stopper := stop.NewStopper() + ctx := context.Background() + defer stopper.Stop(ctx) + + db, err := OpenExistingStore(args[0], stopper, true /* readOnly */) + if err != nil { + return err + } + defer db.Close() + + var interleavedIntentCount, separatedIntentCount int + var keysCount uint64 + var wg sync.WaitGroup + closer := make(chan bool) + + wg.Add(1) + _ = stopper.RunAsyncTask(ctx, "intent-count-progress-indicator", func(ctx context.Context) { + defer wg.Done() + ctx, cancel := stopper.WithCancelOnQuiesce(ctx) + defer cancel() + + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + + select { + case <-ticker.C: + fmt.Printf("scanned %d keys\n", atomic.LoadUint64(&keysCount)) + case <-ctx.Done(): + return + case <-closer: + return + } + }) + + iter := db.NewEngineIterator(storage.IterOptions{ + LowerBound: roachpb.KeyMin, + UpperBound: roachpb.KeyMax, + }) + defer iter.Close() + valid, err := iter.SeekEngineKeyGE(storage.EngineKey{Key: roachpb.KeyMin}) + var meta enginepb.MVCCMetadata + for ; valid && err == nil; valid, err = iter.NextEngineKey() { + key, err := iter.EngineKey() + if err != nil { + return err + } + atomic.AddUint64(&keysCount, 1) + if key.IsLockTableKey() { + separatedIntentCount++ + continue + } + if !key.IsMVCCKey() { + continue + } + mvccKey, err := key.ToMVCCKey() + if err != nil { + return err + } + if !mvccKey.Timestamp.IsEmpty() { + continue + } + val := iter.UnsafeValue() + if err := protoutil.Unmarshal(val, &meta); err != nil { + return err + } + if meta.IsInline() { + continue + } + interleavedIntentCount++ + } + if err != nil { + return err + } + close(closer) + wg.Wait() + fmt.Printf("interleaved intents: %d\nseparated intents: %d\n", + interleavedIntentCount, separatedIntentCount) + return nil +} + // DebugCmdsForRocksDB lists debug commands that access rocksdb through the engine // and need encryption flags (injected by CCL code). // Note: do NOT include commands that just call rocksdb code without setting up an engine. @@ -1318,6 +1413,7 @@ var DebugCmdsForRocksDB = []*cobra.Command{ debugRaftLogCmd, debugRangeDataCmd, debugRangeDescriptorsCmd, + debugIntentCount, } // All other debug commands go here. diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 71332b69af66..0c918901d0e5 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -271,6 +271,12 @@ const ( SQLInstancesTable // Can return new retryable rangefeed errors without crashing the client NewRetryableRangefeedErrors + // SeparatedIntentsMigration adds the migration to move over all remaining + // intents to the separated lock table space. + SeparatedIntentsMigration + // PostSeparatedIntentsMigration runs a cleanup migration after the main + // SeparatedIntentsMigration. + PostSeparatedIntentsMigration // Step (1): Add new versions here. ) @@ -434,12 +440,20 @@ var versionsSingleton = keyedVersions{ }, { Key: SQLInstancesTable, - Version: roachpb.Version{Major: 21, Minor: 1, Internal: 122}, + Version: roachpb.Version{Major: 21, Minor: 1, Internal: 120}, }, { Key: NewRetryableRangefeedErrors, + Version: roachpb.Version{Major: 21, Minor: 1, Internal: 122}, + }, + { + Key: SeparatedIntentsMigration, Version: roachpb.Version{Major: 21, Minor: 1, Internal: 124}, }, + { + Key: PostSeparatedIntentsMigration, + Version: roachpb.Version{Major: 21, Minor: 1, Internal: 126}, + }, // Step (2): Add new versions here. } diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 8136f7389248..7e7a3eb9ad13 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -41,11 +41,13 @@ func _() { _ = x[TenantUsageTable-30] _ = x[SQLInstancesTable-31] _ = x[NewRetryableRangefeedErrors-32] + _ = x[SeparatedIntentsMigration-33] + _ = x[PostSeparatedIntentsMigration-34] } -const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrors" +const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsSeparatedIntentsMigrationPostSeparatedIntentsMigration" -var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684} +var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 709, 738} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index b748fafd9297..3811e0550843 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -271,6 +271,8 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.AddSSTableRequest: case *roachpb.MigrateRequest: case *roachpb.QueryResolvedTimestampRequest: + case *roachpb.BarrierRequest: + case *roachpb.ScanInterleavedIntentsRequest: default: if result.Err == nil { result.Err = errors.Errorf("unsupported reply: %T for %T", @@ -843,3 +845,45 @@ func (b *Batch) queryResolvedTimestamp(s, e interface{}) { b.appendReqs(req) b.initResult(1, 0, notRaw, nil) } + +func (b *Batch) scanInterleavedIntents(s, e interface{}) { + begin, err := marshalKey(s) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + end, err := marshalKey(e) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + req := &roachpb.ScanInterleavedIntentsRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: begin, + EndKey: end, + }, + } + b.appendReqs(req) + b.initResult(1, 0, notRaw, nil) +} + +func (b *Batch) barrier(s, e interface{}) { + begin, err := marshalKey(s) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + end, err := marshalKey(e) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + req := &roachpb.BarrierRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: begin, + EndKey: end, + }, + } + b.appendReqs(req) + b.initResult(1, 0, notRaw, nil) +} diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 674a18889c07..02ec41bfbde7 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -725,6 +725,51 @@ func (db *DB) QueryResolvedTimestamp( return r.ResolvedTS, nil } +// ScanInterleavedIntents is a command that returns all interleaved intents +// encountered in the request span. A resume span is returned if the entirety +// of the request span was not scanned. +func (db *DB) ScanInterleavedIntents( + ctx context.Context, begin, end interface{}, ts hlc.Timestamp, +) ([]roachpb.Intent, *roachpb.Span, error) { + b := &Batch{Header: roachpb.Header{Timestamp: ts}} + b.scanInterleavedIntents(begin, end) + result, err := getOneResult(db.Run(ctx, b), b) + if err != nil { + return nil, nil, err + } + responses := b.response.Responses + if len(responses) == 0 { + return nil, nil, errors.Errorf("unexpected empty response for ScanInterleavedIntents") + } + resp, ok := responses[0].GetInner().(*roachpb.ScanInterleavedIntentsResponse) + if !ok { + return nil, nil, errors.Errorf("unexpected response of type %T for ScanInterleavedIntents", + responses[0].GetInner()) + } + return resp.Intents, result.ResumeSpan, nil +} + +// Barrier is a command that waits for conflicting operations such as earlier +// writes on the specified key range to finish. +func (db *DB) Barrier(ctx context.Context, begin, end interface{}) (hlc.Timestamp, error) { + b := &Batch{} + b.barrier(begin, end) + err := getOneErr(db.Run(ctx, b), b) + if err != nil { + return hlc.Timestamp{}, err + } + responses := b.response.Responses + if len(responses) == 0 { + return hlc.Timestamp{}, errors.Errorf("unexpected empty response for Barrier") + } + resp, ok := responses[0].GetInner().(*roachpb.BarrierResponse) + if !ok { + return hlc.Timestamp{}, errors.Errorf("unexpected response of type %T for Barrier", + responses[0].GetInner()) + } + return resp.Timestamp, nil +} + // sendAndFill is a helper which sends the given batch and fills its results, // returning the appropriate error which is either from the first failing call, // or an "internal" error. diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 1d9ecbb2e00d..5e5cc68f1933 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "batcheval", srcs = [ "cmd_add_sstable.go", + "cmd_barrier.go", "cmd_clear_range.go", "cmd_compute_checksum.go", "cmd_conditional_put.go", @@ -36,6 +37,7 @@ go_library( "cmd_reverse_scan.go", "cmd_revert_range.go", "cmd_scan.go", + "cmd_scan_interleaved_intents.go", "cmd_subsume.go", "cmd_truncate_log.go", "command.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_barrier.go b/pkg/kv/kvserver/batcheval/cmd_barrier.go new file mode 100644 index 000000000000..c5e52533e35e --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_barrier.go @@ -0,0 +1,47 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" +) + +func init() { + RegisterReadWriteCommand(roachpb.Barrier, declareKeysBarrier, Barrier) +} + +func declareKeysBarrier( + rs ImmutableRangeState, + h roachpb.Header, + req roachpb.Request, + latchSpans, lockSpans *spanset.SpanSet, +) { + // Barrier is special-cased in the concurrency manager to *not* actually + // grab these latches. Instead, any conflicting latches with these are waited + // on, but new latches aren't inserted. + latchSpans.AddMVCC(spanset.SpanReadWrite, req.Header().Span(), h.Timestamp) +} + +// Barrier evaluation is a no-op, as all the latch waiting happens in +// the latch manager. +func Barrier( + _ context.Context, _ storage.ReadWriter, cArgs CommandArgs, response roachpb.Response, +) (result.Result, error) { + resp := response.(*roachpb.BarrierResponse) + resp.Timestamp = cArgs.EvalCtx.Clock().Now() + + return result.Result{}, nil +} diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go index 51d48d1d91f5..288d8330c911 100644 --- a/pkg/kv/kvserver/batcheval/cmd_migrate.go +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -55,6 +55,7 @@ type migration func(context.Context, storage.ReadWriter, CommandArgs) (result.Re func init() { registerMigration(clusterversion.TruncatedAndRangeAppliedStateMigration, truncatedAndAppliedStateMigration) + registerMigration(clusterversion.PostSeparatedIntentsMigration, postSeparatedIntentsMigration) } func registerMigration(key clusterversion.Key, migration migration) { @@ -130,6 +131,16 @@ func truncatedAndAppliedStateMigration( return pd, nil } +// postSeparatedIntentsMigration is the below-raft part of the migration for +// interleaved to separated intents. It is a no-op as the only purpose of +// running the Migrate command here is to clear out any orphaned replicas with +// interleaved intents. +func postSeparatedIntentsMigration( + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, +) (result.Result, error) { + return result.Result{}, nil +} + // TestingRegisterMigrationInterceptor is used in tests to register an // interceptor for a below-raft migration. // diff --git a/pkg/kv/kvserver/batcheval/cmd_scan_interleaved_intents.go b/pkg/kv/kvserver/batcheval/cmd_scan_interleaved_intents.go new file mode 100644 index 000000000000..60ee3ba725d1 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_scan_interleaved_intents.go @@ -0,0 +1,106 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +func init() { + RegisterReadOnlyCommand(roachpb.ScanInterleavedIntents, declareKeysScanInterleavedIntents, ScanInterleavedIntents) +} + +func declareKeysScanInterleavedIntents( + rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet, +) { + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) +} + +// ScanInterleavedIntents returns intents encountered in the provided span. +// These intents are then resolved in the separated intents migration, the +// usual caller for this request. +func ScanInterleavedIntents( + ctx context.Context, reader storage.Reader, cArgs CommandArgs, response roachpb.Response, +) (result.Result, error) { + req := cArgs.Args.(*roachpb.ScanInterleavedIntentsRequest) + resp := response.(*roachpb.ScanInterleavedIntentsResponse) + + // Put a limit on memory usage by scanning for at least maxIntentCount + // intents or maxIntentBytes in intent values, whichever is reached first, + // then returning those. + const maxIntentCount = 1000 + const maxIntentBytes = 1 << 20 // 1MB + iter := reader.NewEngineIterator(storage.IterOptions{ + LowerBound: req.Key, + UpperBound: req.EndKey, + }) + defer iter.Close() + valid, err := iter.SeekEngineKeyGE(storage.EngineKey{Key: req.Key}) + intentCount := 0 + intentBytes := 0 + + for ; valid && err == nil; valid, err = iter.NextEngineKey() { + key, err := iter.EngineKey() + if err != nil { + return result.Result{}, err + } + if !key.IsMVCCKey() { + // This should never happen, as the only non-MVCC keys are lock table + // keys and those are in the local keyspace. Return an error. + return result.Result{}, errors.New("encountered non-MVCC key during lock table migration") + } + mvccKey, err := key.ToMVCCKey() + if err != nil { + return result.Result{}, err + } + if !mvccKey.Timestamp.IsEmpty() { + // Versioned value - not an intent. + // + // TODO(bilal): Explore seeking here in case there are keys with lots of + // versioned values. + continue + } + + val := iter.Value() + meta := enginepb.MVCCMetadata{} + if err := protoutil.Unmarshal(val, &meta); err != nil { + return result.Result{}, err + } + if meta.IsInline() { + // Inlined value - not an intent. + continue + } + + if intentCount >= maxIntentCount || intentBytes >= maxIntentBytes { + // Batch limit reached - cut short this batch here. This kv + // will be added to txnIntents on the next iteration of the outer loop. + resp.ResumeSpan = &roachpb.Span{ + Key: mvccKey.Key, + EndKey: req.EndKey, + } + break + } + resp.Intents = append(resp.Intents, roachpb.MakeIntent(meta.Txn, mvccKey.Key)) + intentCount++ + intentBytes += len(val) + } + + return result.Result{}, nil +} diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index 0df0da0e06c4..c2d98a04ab02 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -455,6 +455,11 @@ type latchManager interface { // causing this request to switch to pessimistic latching. WaitUntilAcquired(ctx context.Context, lg latchGuard) (latchGuard, *Error) + // WaitFor waits for conflicting latches on the specified spans without adding + // any latches itself. Fast path for operations that only require flushing out + // old operations without blocking any new ones. + WaitFor(ctx context.Context, spans *spanset.SpanSet) *Error + // Releases latches, relinquish its protection from conflicting requests. Release(latchGuard) diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index e43c81dd4e2e..eec701513a06 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -200,7 +200,7 @@ func (m *managerImpl) SequenceReq( case OptimisticEval: panic("optimistic eval cannot happen when re-sequencing") case PessimisticAfterFailedOptimisticEval: - if shouldAcquireLatches(g.Req) { + if !shouldIgnoreLatches(g.Req) { g.AssertLatches() } log.Event(ctx, "re-sequencing request after optimistic sequencing failed") @@ -218,11 +218,18 @@ func (m *managerImpl) SequenceReq( func (m *managerImpl) sequenceReqWithGuard(ctx context.Context, g *Guard) (Response, *Error) { // Some requests don't need to acquire latches at all. - if !shouldAcquireLatches(g.Req) { + if shouldIgnoreLatches(g.Req) { log.Event(ctx, "not acquiring latches") return nil, nil } + // Check if this is a request that waits on latches, but does not acquire + // them. + if shouldWaitOnLatchesWithoutAcquiring(g.Req) { + log.Event(ctx, "waiting on latches without acquiring") + return nil, m.lm.WaitFor(ctx, g.Req.LatchSpans) + } + // Provide the manager with an opportunity to intercept the request. It // may be able to serve the request directly, and even if not, it may be // able to update its internal state based on the request. @@ -346,22 +353,30 @@ func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Respo return nil, nil } -// shouldAcquireLatches determines whether the request should acquire latches +// shouldIgnoreLatches determines whether the request should ignore latches // before proceeding to evaluate. Latches are used to synchronize with other // conflicting requests, based on the Spans collected for the request. Most -// request types will want to acquire latches. -func shouldAcquireLatches(req Request) bool { +// request types will want to acquire latches. Requests that return true for +// shouldWaitOnLatchesWithoutAcquiring will not completely ignore latches as +// they could wait on them, even if they don't acquire latches. +func shouldIgnoreLatches(req Request) bool { switch { case req.ReadConsistency != roachpb.CONSISTENT: // Only acquire latches for consistent operations. - return false + return true case req.isSingle(roachpb.RequestLease): - // Do not acquire latches for lease requests. These requests are run on - // replicas that do not hold the lease, so acquiring latches wouldn't - // help synchronize with other requests. - return false + // Ignore latches for lease requests. These requests are run on replicas + // that do not hold the lease, so acquiring latches wouldn't help + // synchronize with other requests. + return true } - return true + return false +} + +// shouldWaitOnLatchesWithoutAcquiring determines if this is a request that +// only waits on existing latches without acquiring any new ones. +func shouldWaitOnLatchesWithoutAcquiring(req Request) bool { + return req.isSingle(roachpb.Barrier) } // FinishReq implements the RequestSequencer interface. @@ -628,7 +643,7 @@ func (g *Guard) HoldingLatches() bool { // AssertLatches asserts that the guard is non-nil and holding latches, if the // request is supposed to hold latches while evaluating in the first place. func (g *Guard) AssertLatches() { - if shouldAcquireLatches(g.Req) && !g.HoldingLatches() { + if !shouldIgnoreLatches(g.Req) && !shouldWaitOnLatchesWithoutAcquiring(g.Req) && !g.HoldingLatches() { panic("expected latches held, found none") } } diff --git a/pkg/kv/kvserver/concurrency/datadriven_util_test.go b/pkg/kv/kvserver/concurrency/datadriven_util_test.go index cd713b1dea20..e9e8f9483a33 100644 --- a/pkg/kv/kvserver/concurrency/datadriven_util_test.go +++ b/pkg/kv/kvserver/concurrency/datadriven_util_test.go @@ -145,6 +145,14 @@ func scanSingleRequest( var r roachpb.RequestLeaseRequest return &r + case "barrier": + var r roachpb.BarrierRequest + r.Key = roachpb.Key(mustGetField("key")) + if v, ok := fields["endkey"]; ok { + r.EndKey = roachpb.Key(v) + } + return &r + default: d.Fatalf(t, "unknown request type: %s", cmd) return nil diff --git a/pkg/kv/kvserver/concurrency/latch_manager.go b/pkg/kv/kvserver/concurrency/latch_manager.go index bec2112be357..b0a4b8eb1073 100644 --- a/pkg/kv/kvserver/concurrency/latch_manager.go +++ b/pkg/kv/kvserver/concurrency/latch_manager.go @@ -50,6 +50,14 @@ func (m *latchManagerImpl) WaitUntilAcquired( return lg, nil } +func (m *latchManagerImpl) WaitFor(ctx context.Context, ss *spanset.SpanSet) *Error { + err := m.m.WaitFor(ctx, ss) + if err != nil { + return roachpb.NewError(err) + } + return nil +} + func (m *latchManagerImpl) Release(lg latchGuard) { m.m.Release(lg.(*spanlatch.Guard)) } diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/barrier b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/barrier new file mode 100644 index 000000000000..a6d7ad585dbe --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/barrier @@ -0,0 +1,180 @@ + +# ------------------------------------------------------------- +# Barrier requests do not acquire latches +# ------------------------------------------------------------- + +new-request name=barrier1 txn=none ts=10,1 + barrier key=a endkey=f +---- + +sequence req=barrier1 +---- +[1] sequence barrier1: sequencing request +[1] sequence barrier1: waiting on latches without acquiring +[1] sequence barrier1: sequencing complete, returned guard + +debug-latch-manager +---- +write count: 0 + read count: 0 + +finish req=barrier1 +---- +[-] finish barrier1: finishing request + +reset +---- + +# ------------------------------------------------------------- +# Barrier requests wait for conflicting reads +# ------------------------------------------------------------- + +new-request name=barrier2 txn=none ts=10,1 + barrier key=a endkey=f +---- + +new-request name=read1 txn=none ts=15,1 + get key=c +---- + +sequence req=read1 +---- +[1] sequence read1: sequencing request +[1] sequence read1: acquiring latches +[1] sequence read1: scanning lock table for conflicting locks +[1] sequence read1: sequencing complete, returned guard + +debug-latch-manager +---- +write count: 0 + read count: 1 + +sequence req=barrier2 +---- +[2] sequence barrier2: sequencing request +[2] sequence barrier2: waiting on latches without acquiring +[2] sequence barrier2: waiting to acquire write latch {a-f}@10.000000000,1, held by read latch c@15.000000000,1 +[2] sequence barrier2: blocked on select in spanlatch.(*Manager).waitForSignal + +finish req=read1 +---- +[-] finish read1: finishing request +[2] sequence barrier2: sequencing complete, returned guard + +finish req=barrier2 +---- +[-] finish barrier2: finishing request + +reset +---- + +# ------------------------------------------------------------- +# Barrier requests wait for conflicting writes +# ------------------------------------------------------------- + +new-txn name=txn1 ts=10,1 epoch=0 +---- + +new-request name=barrier3 txn=none ts=15,1 + barrier key=a endkey=f +---- + +new-request name=write1 txn=txn1 ts=10,1 + put key=c value=v +---- + +sequence req=write1 +---- +[1] sequence write1: sequencing request +[1] sequence write1: acquiring latches +[1] sequence write1: scanning lock table for conflicting locks +[1] sequence write1: sequencing complete, returned guard + + +debug-latch-manager +---- +write count: 1 + read count: 0 + +sequence req=barrier3 +---- +[2] sequence barrier3: sequencing request +[2] sequence barrier3: waiting on latches without acquiring +[2] sequence barrier3: waiting to acquire write latch {a-f}@15.000000000,1, held by write latch c@10.000000000,1 +[2] sequence barrier3: blocked on select in spanlatch.(*Manager).waitForSignal + +debug-latch-manager +---- +write count: 1 + read count: 0 + +finish req=write1 +---- +[-] finish write1: finishing request +[2] sequence barrier3: sequencing complete, returned guard + +finish req=barrier3 +---- +[-] finish barrier3: finishing request + +reset +---- + +# ------------------------------------------------------------- +# Barrier requests do not wait for older reads, and do not block future writes. +# ------------------------------------------------------------- + +new-txn name=txn2 ts=15,1 epoch=0 +---- + +new-request name=read2 txn=none ts=10,1 + get key=d +---- + +new-request name=barrier4 txn=none ts=12,1 + barrier key=a endkey=f +---- + +new-request name=write2 txn=txn2 ts=15,1 + put key=c value=v +---- + +sequence req=read2 +---- +[1] sequence read2: sequencing request +[1] sequence read2: acquiring latches +[1] sequence read2: scanning lock table for conflicting locks +[1] sequence read2: sequencing complete, returned guard + +sequence req=barrier4 +---- +[2] sequence barrier4: sequencing request +[2] sequence barrier4: waiting on latches without acquiring +[2] sequence barrier4: sequencing complete, returned guard + +sequence req=write2 +---- +[3] sequence write2: sequencing request +[3] sequence write2: acquiring latches +[3] sequence write2: scanning lock table for conflicting locks +[3] sequence write2: sequencing complete, returned guard + +debug-latch-manager +---- +write count: 1 + read count: 1 + +finish req=barrier4 +---- +[-] finish barrier4: finishing request + +finish req=write2 +---- +[-] finish write2: finishing request + +finish req=read2 +---- +[-] finish read2: finishing request + +reset +---- diff --git a/pkg/kv/kvserver/spanlatch/manager.go b/pkg/kv/kvserver/spanlatch/manager.go index a5c9370b1d0c..92a67b87d8f3 100644 --- a/pkg/kv/kvserver/spanlatch/manager.go +++ b/pkg/kv/kvserver/spanlatch/manager.go @@ -233,6 +233,22 @@ func (m *Manager) AcquireOptimistic(spans *spanset.SpanSet) *Guard { return lg } +// WaitFor waits for conflicting latches on the spans without adding +// any latches itself. Fast path for operations that only require past latches +// to be released without blocking new latches. +func (m *Manager) WaitFor(ctx context.Context, spans *spanset.SpanSet) error { + // The guard is only used to store latches by this request. These latches + // are not actually inserted using insertLocked. + lg := newGuard(spans) + + m.mu.Lock() + snap := m.snapshotLocked(spans) + defer snap.close() + m.mu.Unlock() + + return m.wait(ctx, lg, snap) +} + // CheckOptimisticNoConflicts returns true iff the spans in the provided // spanset do not conflict with any existing latches (in the snapshot created // in AcquireOptimistic). It must only be called after AcquireOptimistic, and diff --git a/pkg/kv/kvserver/spanlatch/manager_test.go b/pkg/kv/kvserver/spanlatch/manager_test.go index c8f461c82d2b..0a36746c10c7 100644 --- a/pkg/kv/kvserver/spanlatch/manager_test.go +++ b/pkg/kv/kvserver/spanlatch/manager_test.go @@ -586,6 +586,40 @@ func TestLatchManagerOptimistic(t *testing.T) { m.Release(lg4) } +func TestLatchManagerWaitFor(t *testing.T) { + defer leaktest.AfterTest(t)() + var m Manager + + // Acquire latches, no conflict. + lg1 := m.AcquireOptimistic(spans("d", "f", write, zeroTS)) + require.True(t, m.CheckOptimisticNoConflicts(lg1, spans("d", "f", write, zeroTS))) + lg1, err := m.WaitUntilAcquired(context.Background(), lg1) + require.NoError(t, err) + + // See if WaitFor waits for above latch. + waitForCh := func() <-chan *Guard { + ch := make(chan *Guard) + go func() { + err := m.WaitFor(context.Background(), spans("a", "e", read, zeroTS)) + require.NoError(t, err) + ch <- &Guard{} + }() + return ch + } + ch2 := waitForCh() + testLatchBlocks(t, ch2) + m.Release(lg1) + testLatchSucceeds(t, ch2) + + // Optimistic acquire should _not_ encounter conflict - as WaitFor should + // not lay any latches. + lg3 := m.AcquireOptimistic(spans("a", "e", write, zeroTS)) + require.True(t, m.CheckOptimisticNoConflicts(lg3, spans("a", "e", write, zeroTS))) + lg3, err = m.WaitUntilAcquired(context.Background(), lg3) + require.NoError(t, err) + m.Release(lg3) +} + func BenchmarkLatchManagerReadOnlyMix(b *testing.B) { for _, size := range []int{1, 4, 16, 64, 128, 256} { b.Run(fmt.Sprintf("size=%d", size), func(b *testing.B) { diff --git a/pkg/kv/mock_transactional_sender.go b/pkg/kv/mock_transactional_sender.go index eacffe47115b..1e0726204d6b 100644 --- a/pkg/kv/mock_transactional_sender.go +++ b/pkg/kv/mock_transactional_sender.go @@ -218,6 +218,7 @@ func (m *MockTransactionalSender) DeferCommitWait(ctx context.Context) func(cont type MockTxnSenderFactory struct { senderFunc func(context.Context, *roachpb.Transaction, roachpb.BatchRequest) ( *roachpb.BatchResponse, *roachpb.Error) + nonTxnSenderFunc Sender } var _ TxnSenderFactory = MockTxnSenderFactory{} @@ -235,6 +236,21 @@ func MakeMockTxnSenderFactory( } } +// MakeMockTxnSenderFactoryWithNonTxnSender creates a MockTxnSenderFactory from +// two sender functions: one for transactional and one for non-transactional +// requests. +func MakeMockTxnSenderFactoryWithNonTxnSender( + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error), + nonTxnSenderFunc SenderFunc, +) MockTxnSenderFactory { + return MockTxnSenderFactory{ + senderFunc: senderFunc, + nonTxnSenderFunc: nonTxnSenderFunc, + } +} + // RootTransactionalSender is part of TxnSenderFactory. func (f MockTxnSenderFactory) RootTransactionalSender( txn *roachpb.Transaction, _ roachpb.UserPriority, @@ -249,5 +265,5 @@ func (f MockTxnSenderFactory) LeafTransactionalSender(tis *roachpb.LeafTxnInputS // NonTransactionalSender is part of TxnSenderFactory. func (f MockTxnSenderFactory) NonTransactionalSender() Sender { - return nil + return f.nonTxnSenderFunc } diff --git a/pkg/kv/mock_transactional_sender.go.orig b/pkg/kv/mock_transactional_sender.go.orig new file mode 100644 index 000000000000..4885932dae15 --- /dev/null +++ b/pkg/kv/mock_transactional_sender.go.orig @@ -0,0 +1,269 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kv + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +// MockTransactionalSender allows a function to be used as a TxnSender. +type MockTransactionalSender struct { + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) + txn roachpb.Transaction +} + +// NewMockTransactionalSender creates a MockTransactionalSender. +// The passed in txn is cloned. +func NewMockTransactionalSender( + f func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error), + txn *roachpb.Transaction, +) *MockTransactionalSender { + return &MockTransactionalSender{senderFunc: f, txn: *txn} +} + +// Send is part of the TxnSender interface. +func (m *MockTransactionalSender) Send( + ctx context.Context, ba roachpb.BatchRequest, +) (*roachpb.BatchResponse, *roachpb.Error) { + return m.senderFunc(ctx, &m.txn, ba) +} + +// GetLeafTxnInputState is part of the TxnSender interface. +func (m *MockTransactionalSender) GetLeafTxnInputState( + context.Context, TxnStatusOpt, +) (roachpb.LeafTxnInputState, error) { + panic("unimplemented") +} + +// GetLeafTxnFinalState is part of the TxnSender interface. +func (m *MockTransactionalSender) GetLeafTxnFinalState( + context.Context, TxnStatusOpt, +) (roachpb.LeafTxnFinalState, error) { + panic("unimplemented") +} + +// UpdateRootWithLeafFinalState is part of the TxnSender interface. +func (m *MockTransactionalSender) UpdateRootWithLeafFinalState( + context.Context, *roachpb.LeafTxnFinalState, +) { + panic("unimplemented") +} + +// AnchorOnSystemConfigRange is part of the TxnSender interface. +func (m *MockTransactionalSender) AnchorOnSystemConfigRange() error { + return errors.New("unimplemented") +} + +// TxnStatus is part of the TxnSender interface. +func (m *MockTransactionalSender) TxnStatus() roachpb.TransactionStatus { + return m.txn.Status +} + +// SetUserPriority is part of the TxnSender interface. +func (m *MockTransactionalSender) SetUserPriority(pri roachpb.UserPriority) error { + m.txn.Priority = roachpb.MakePriority(pri) + return nil +} + +// SetDebugName is part of the TxnSender interface. +func (m *MockTransactionalSender) SetDebugName(name string) { + m.txn.Name = name +} + +// String is part of the TxnSender interface. +func (m *MockTransactionalSender) String() string { + return m.txn.String() +} + +// ReadTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) ReadTimestamp() hlc.Timestamp { + return m.txn.ReadTimestamp +} + +// ProvisionalCommitTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) ProvisionalCommitTimestamp() hlc.Timestamp { + return m.txn.WriteTimestamp +} + +// CommitTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) CommitTimestamp() hlc.Timestamp { + return m.txn.ReadTimestamp +} + +// CommitTimestampFixed is part of the TxnSender interface. +func (m *MockTransactionalSender) CommitTimestampFixed() bool { + return m.txn.CommitTimestampFixed +} + +// SetFixedTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) SetFixedTimestamp(_ context.Context, ts hlc.Timestamp) error { + m.txn.WriteTimestamp = ts + m.txn.ReadTimestamp = ts + m.txn.GlobalUncertaintyLimit = ts + m.txn.CommitTimestampFixed = true + + // Set the MinTimestamp to the minimum of the existing MinTimestamp and the fixed + // timestamp. This ensures that the MinTimestamp is always <= the other timestamps. + m.txn.MinTimestamp.Backward(ts) + return nil +} + +// RequiredFrontier is part of the TxnSender interface. +func (m *MockTransactionalSender) RequiredFrontier() hlc.Timestamp { + return m.txn.RequiredFrontier() +} + +// ManualRestart is part of the TxnSender interface. +func (m *MockTransactionalSender) ManualRestart( + ctx context.Context, pri roachpb.UserPriority, ts hlc.Timestamp, +) { + m.txn.Restart(pri, 0 /* upgradePriority */, ts) +} + +// IsSerializablePushAndRefreshNotPossible is part of the TxnSender interface. +func (m *MockTransactionalSender) IsSerializablePushAndRefreshNotPossible() bool { + return false +} + +// CreateSavepoint is part of the client.TxnSender interface. +func (m *MockTransactionalSender) CreateSavepoint(context.Context) (SavepointToken, error) { + panic("unimplemented") +} + +// RollbackToSavepoint is part of the client.TxnSender interface. +func (m *MockTransactionalSender) RollbackToSavepoint(context.Context, SavepointToken) error { + panic("unimplemented") +} + +// ReleaseSavepoint is part of the client.TxnSender interface. +func (m *MockTransactionalSender) ReleaseSavepoint(context.Context, SavepointToken) error { + panic("unimplemented") +} + +// Epoch is part of the TxnSender interface. +func (m *MockTransactionalSender) Epoch() enginepb.TxnEpoch { panic("unimplemented") } + +// TestingCloneTxn is part of the TxnSender interface. +func (m *MockTransactionalSender) TestingCloneTxn() *roachpb.Transaction { + return m.txn.Clone() +} + +// Active is part of the TxnSender interface. +func (m *MockTransactionalSender) Active() bool { + panic("unimplemented") +} + +// UpdateStateOnRemoteRetryableErr is part of the TxnSender interface. +func (m *MockTransactionalSender) UpdateStateOnRemoteRetryableErr( + ctx context.Context, pErr *roachpb.Error, +) *roachpb.Error { + panic("unimplemented") +} + +// DisablePipelining is part of the client.TxnSender interface. +func (m *MockTransactionalSender) DisablePipelining() error { return nil } + +// PrepareRetryableError is part of the client.TxnSender interface. +func (m *MockTransactionalSender) PrepareRetryableError(ctx context.Context, msg string) error { + return roachpb.NewTransactionRetryWithProtoRefreshError(msg, m.txn.ID, *m.txn.Clone()) +} + +// Step is part of the TxnSender interface. +func (m *MockTransactionalSender) Step(_ context.Context) error { + // At least one test (e.g sql/TestPortalsDestroyedOnTxnFinish) requires + // the ability to run simple statements that do not access storage, + // and that requires a non-panicky Step(). + return nil +} + +// ConfigureStepping is part of the TxnSender interface. +func (m *MockTransactionalSender) ConfigureStepping(context.Context, SteppingMode) SteppingMode { + // See Step() above. + return SteppingDisabled +} + +// GetSteppingMode is part of the TxnSender interface. +func (m *MockTransactionalSender) GetSteppingMode(context.Context) SteppingMode { + return SteppingDisabled +} + +// ManualRefresh is part of the TxnSender interface. +func (m *MockTransactionalSender) ManualRefresh(ctx context.Context) error { + panic("unimplemented") +} + +// DeferCommitWait is part of the TxnSender interface. +func (m *MockTransactionalSender) DeferCommitWait(ctx context.Context) func(context.Context) error { + panic("unimplemented") +} + +// MockTxnSenderFactory is a TxnSenderFactory producing MockTxnSenders. +type MockTxnSenderFactory struct { + senderFunc func(context.Context, *roachpb.Transaction, roachpb.BatchRequest) ( + *roachpb.BatchResponse, *roachpb.Error) + nonTxnSenderFunc Sender +} + +var _ TxnSenderFactory = MockTxnSenderFactory{} + +// MakeMockTxnSenderFactory creates a MockTxnSenderFactory from a sender +// function that receives the transaction in addition to the request. The +// function is responsible for putting the txn inside the batch, if needed. +func MakeMockTxnSenderFactory( + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error), +) MockTxnSenderFactory { + return MockTxnSenderFactory{ + senderFunc: senderFunc, + } +} + +// MakeMockTxnSenderFactoryWithNonTxnSender creates a MockTxnSenderFactory from +// two sender functions: one for transactional and one for non-transactional +// requests. +func MakeMockTxnSenderFactoryWithNonTxnSender( + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, +) (*roachpb.BatchResponse, *roachpb.Error), + nonTxnSenderFunc SenderFunc, +) MockTxnSenderFactory { + return MockTxnSenderFactory{ + senderFunc: senderFunc, + nonTxnSenderFunc: nonTxnSenderFunc, + } +} + +// RootTransactionalSender is part of TxnSenderFactory. +func (f MockTxnSenderFactory) RootTransactionalSender( + txn *roachpb.Transaction, _ roachpb.UserPriority, +) TxnSender { + return NewMockTransactionalSender(f.senderFunc, txn) +} + +// LeafTransactionalSender is part of TxnSenderFactory. +func (f MockTxnSenderFactory) LeafTransactionalSender(tis *roachpb.LeafTxnInputState) TxnSender { + return NewMockTransactionalSender(f.senderFunc, &tis.Txn) +} + +// NonTransactionalSender is part of TxnSenderFactory. +func (f MockTxnSenderFactory) NonTransactionalSender() Sender { + return f.nonTxnSenderFunc +} diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 61877bb4145d..483b7b03c8fd 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/roachpb:with-mocks", "//pkg/server/serverpb", "//pkg/settings/cluster", @@ -21,6 +22,7 @@ go_library( "//pkg/sql/catalog/lease", "//pkg/sql/sqlutil", "//pkg/util/log", + "//pkg/util/stop", "@com_github_cockroachdb_logtags//:logtags", ], ) diff --git a/pkg/migration/migration.go b/pkg/migration/migration.go index 2c2f55f1f634..db102c7ea359 100644 --- a/pkg/migration/migration.go +++ b/pkg/migration/migration.go @@ -71,8 +71,8 @@ type JobDeps interface { // if one exists. GetMigration(key clusterversion.ClusterVersion) (Migration, bool) - // Cluster returns a handle to the cluster on a system tenant. - Cluster() Cluster + // SystemDeps returns a handle to migration dependencies on a system tenant. + SystemDeps() SystemDeps } type migration struct { diff --git a/pkg/migration/migrationcluster/BUILD.bazel b/pkg/migration/migrationcluster/BUILD.bazel index 65b18e76898a..c7bf58b5b06d 100644 --- a/pkg/migration/migrationcluster/BUILD.bazel +++ b/pkg/migration/migrationcluster/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb:with-mocks", "//pkg/rpc", @@ -19,6 +20,7 @@ go_library( "//pkg/util/ctxgroup", "//pkg/util/log", "//pkg/util/quotapool", + "//pkg/util/stop", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@org_golang_google_grpc//:go_default_library", diff --git a/pkg/migration/migrationcluster/cluster.go b/pkg/migration/migrationcluster/cluster.go index 6a5aac6ddfba..18b2595576d9 100644 --- a/pkg/migration/migrationcluster/cluster.go +++ b/pkg/migration/migrationcluster/cluster.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -23,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "google.golang.org/grpc" @@ -48,6 +50,13 @@ type ClusterConfig struct { // to expose only relevant, vetted bits of kv.DB. It'll make our tests less // "integration-ey". DB *kv.DB + + // Stopper is a reference to a stop.Stopper for spawning tasks. + Stopper *stop.Stopper + + // DistSender provides access to a DistSender for accessing the distributed kv + // store. + DistSender *kvcoord.DistSender } // NodeDialer abstracts connecting to other nodes in the cluster. @@ -186,3 +195,13 @@ func (c *Cluster) IterateRangeDescriptors( func (c *Cluster) DB() *kv.DB { return c.c.DB } + +// DistSender exposes the underlying *kvcoord.DistSender instance. +func (c *Cluster) DistSender() *kvcoord.DistSender { + return c.c.DistSender +} + +// Stopper exposes the stored Stopper instance. +func (c *Cluster) Stopper() *stop.Stopper { + return c.c.Stopper +} diff --git a/pkg/migration/migrationcluster/tenant_cluster.go b/pkg/migration/migrationcluster/tenant_cluster.go index 090527d568e8..9fbec92fe528 100644 --- a/pkg/migration/migrationcluster/tenant_cluster.go +++ b/pkg/migration/migrationcluster/tenant_cluster.go @@ -14,8 +14,10 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/errors" ) @@ -132,6 +134,16 @@ func (t *TenantCluster) DB() *kv.DB { return t.db } +// DistSender is part of the migration.Cluster interface. +func (t *TenantCluster) DistSender() *kvcoord.DistSender { + return nil +} + +// Stopper is part of the migration.Cluster interface. +func (t *TenantCluster) Stopper() *stop.Stopper { + return nil +} + // ForEveryNode is part of the migration.Cluster interface. func (t *TenantCluster) ForEveryNode( ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, diff --git a/pkg/migration/migrationjob/migration_job.go b/pkg/migration/migrationjob/migration_job.go index eb9c77889b5e..6cb74a4c45bf 100644 --- a/pkg/migration/migrationjob/migration_job.go +++ b/pkg/migration/migrationjob/migration_job.go @@ -79,7 +79,7 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error { } switch m := m.(type) { case *migration.SystemMigration: - err = m.Run(ctx, cv, mc.Cluster()) + err = m.Run(ctx, cv, mc.SystemDeps()) case *migration.TenantMigration: err = m.Run(ctx, cv, migration.TenantDeps{ DB: execCtx.ExecCfg().DB, diff --git a/pkg/migration/migrationmanager/manager.go b/pkg/migration/migrationmanager/manager.go index 4bfb47de5b3b..f62f24c435b2 100644 --- a/pkg/migration/migrationmanager/manager.go +++ b/pkg/migration/migrationmanager/manager.go @@ -39,7 +39,7 @@ import ( // Manager is the instance responsible for executing migrations across the // cluster. type Manager struct { - c migration.Cluster + deps migration.SystemDeps ie sqlutil.InternalExecutor jr *jobs.Registry codec keys.SQLCodec @@ -57,16 +57,16 @@ func (m *Manager) GetMigration(key clusterversion.ClusterVersion) (migration.Mig return migrations.GetMigration(key) } -// Cluster returns the cluster associated with this manager. It may be nil -// in a secondary tenant. -func (m *Manager) Cluster() migration.Cluster { - return m.c +// SystemDeps returns dependencies to run system migrations for the cluster +// associated with this manager. It may be the zero value in a secondary tenant. +func (m *Manager) SystemDeps() migration.SystemDeps { + return m.deps } -// NewManager constructs a new Manager. The Cluster parameter may be nil in +// NewManager constructs a new Manager. The SystemDeps parameter may be zero in // secondary tenants. The testingKnobs parameter may be nil. func NewManager( - c migration.Cluster, + deps migration.SystemDeps, ie sqlutil.InternalExecutor, jr *jobs.Registry, codec keys.SQLCodec, @@ -78,7 +78,7 @@ func NewManager( knobs = *testingKnobs } return &Manager{ - c: c, + deps: deps, ie: ie, jr: jr, codec: codec, @@ -114,7 +114,7 @@ func (m *Manager) Migrate( // that might be doomed to fail. { finalVersion := clusterVersions[len(clusterVersions)-1] - if err := validateTargetClusterVersion(ctx, m.c, finalVersion); err != nil { + if err := validateTargetClusterVersion(ctx, m.deps.Cluster, finalVersion); err != nil { return err } } @@ -206,19 +206,19 @@ func (m *Manager) Migrate( // version, and by design also supports the actual version (which is // the direct successor of the fence). fenceVersion := migration.FenceVersionFor(ctx, clusterVersion) - if err := bumpClusterVersion(ctx, m.c, fenceVersion); err != nil { + if err := bumpClusterVersion(ctx, m.deps.Cluster, fenceVersion); err != nil { return err } } // Now sanity check that we'll actually be able to perform the real // cluster version bump, cluster-wide. - if err := validateTargetClusterVersion(ctx, m.c, clusterVersion); err != nil { + if err := validateTargetClusterVersion(ctx, m.deps.Cluster, clusterVersion); err != nil { return err } // Finally, bump the real version cluster-wide. - if err := bumpClusterVersion(ctx, m.c, clusterVersion); err != nil { + if err := bumpClusterVersion(ctx, m.deps.Cluster, clusterVersion); err != nil { return err } } @@ -289,7 +289,7 @@ func (m *Manager) getOrCreateMigrationJob( ctx context.Context, user security.SQLUsername, version clusterversion.ClusterVersion, ) (alreadyCompleted bool, jobID jobspb.JobID, _ error) { newJobID := m.jr.MakeJobID() - if err := m.c.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + if err := m.deps.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { alreadyCompleted, err = migrationjob.CheckIfMigrationCompleted(ctx, txn, m.ie, version) if err != nil && ctx.Err() == nil { log.Warningf(ctx, "failed to check if migration already completed: %v", err) diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 80f27c972005..557f580b1fda 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "fix_descriptor_migration.go", "join_tokens.go", "migrations.go", + "separated_intents.go", "sql_instances.go", "sql_stats.go", "tenant_usage.go", @@ -19,6 +20,8 @@ go_library( "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", + "//pkg/kv/kvserver/intentresolver", "//pkg/migration", "//pkg/roachpb:with-mocks", "//pkg/server/serverpb", @@ -30,10 +33,14 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", "//pkg/startupmigrations", + "//pkg/storage/enginepb", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/protoutil", + "//pkg/util/stop", + "//pkg/util/syncutil", + "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", ], ) @@ -44,15 +51,22 @@ go_test( "delete_deprecated_namespace_tabledesc_external_test.go", "fix_descriptor_migration_external_test.go", "main_test.go", + "separated_intents_external_test.go", + "separated_intents_test.go", "truncated_state_external_test.go", ], + data = glob(["testdata/**"]), + embed = [":migrations"], deps = [ "//pkg/base", "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/intentresolver", "//pkg/kv/kvserver/stateloader", + "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", @@ -61,12 +75,22 @@ go_test( "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/systemschema", + "//pkg/storage", + "//pkg/storage/enginepb", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/protoutil", + "//pkg/util/stop", + "//pkg/util/syncutil", + "//pkg/util/tracing", + "//pkg/util/uint128", + "//pkg/util/uuid", + "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 3f4306ec1a91..7608543d205c 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -82,6 +82,14 @@ var migrations = []migration.Migration{ toCV(clusterversion.SQLInstancesTable), sqlInstancesTableMigration, ), + migration.NewSystemMigration( + "move over all intents to separate lock table", + toCV(clusterversion.SeparatedIntentsMigration), + separatedIntentsMigration), + migration.NewSystemMigration( + "run no-op migrate command on all ranges after lock table migration", + toCV(clusterversion.PostSeparatedIntentsMigration), + postSeparatedIntentsMigration), } func init() { diff --git a/pkg/migration/migrations/separated_intents.go b/pkg/migration/migrations/separated_intents.go new file mode 100644 index 000000000000..5f70aeacea10 --- /dev/null +++ b/pkg/migration/migrations/separated_intents.go @@ -0,0 +1,429 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migrations + +import ( + "bytes" + "context" + "fmt" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" +) + +// The number of concurrent migrateLockTableRequests requests to run. This +// is effectively a cluster-wide setting as the actual legwork of the migration +// happens when the destination replica(s) are sending replies back to the +// original node. +// +// TODO(bilal): Add logic to make this concurrency limit a per-leaseholder limit +// as opposed to a cluster-wide limit. That way, we could limit +// migrateLockTableRequests to 1 per leaseholder as opposed to 4 for the entire +// cluster, avoiding the case where all 4 ranges at a time could have the same node +// as their leaseholder. +const concurrentMigrateLockTableRequests = 4 + +// The maximum number of times to retry a migrateLockTableRequest before failing +// the migration. +const migrateLockTableRetries = 3 + +// migrateLockTableRequest represents migration of one slice of the keyspace. As +// part of this request, multiple non-transactional requests would need to be +// run: a Barrier, a ScanInterleavedIntents, then multiple txn pushes and intent +// resolutions. +// +// One request will correspond to one range at the time of running the +// IterateRangeDescriptors command. If range boundaries change during the +// course of the migration, that is okay as the migration logic does not rely on +// that assumption. The debugRangeID is the range ID for this range at the time +// of the range descriptor iteration, and is +// present solely for observability / logging purposes. +type migrateLockTableRequest struct { + start, end roachpb.Key + debugRangeID roachpb.RangeID + barrierDone bool + barrierTS hlc.Timestamp +} + +type intentResolver interface { + PushTransaction( + ctx context.Context, pushTxn *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, + ) (*roachpb.Transaction, *roachpb.Error) + + ResolveIntents( + ctx context.Context, intents []roachpb.LockUpdate, opts intentresolver.ResolveOptions, + ) (pErr *roachpb.Error) +} + +type migrateLockTablePool struct { + requests chan migrateLockTableRequest + wg sync.WaitGroup + stopper *stop.Stopper + ir intentResolver + db *kv.DB + clock *hlc.Clock + done chan bool + status [concurrentMigrateLockTableRequests]int64 + + mu struct { + syncutil.Mutex + + errorCount int + combinedErr error + } +} + +func (m *migrateLockTablePool) attemptMigrateRequest( + ctx context.Context, req *migrateLockTableRequest, +) (nextReq *migrateLockTableRequest, err error) { + // The barrier command needs to be invoked if it hasn't been invoked on this + // key range yet. This command does not return a resume span, so once it has + // returned successfully, it doesn't need to be called again unless there's + // an error. + barrierTS := req.barrierTS + if !req.barrierDone { + var err error + barrierTS, err = m.db.Barrier(ctx, req.start, req.end) + if err != nil { + return nil, errors.Wrap(err, "error when invoking Barrier command") + } + } + barrierTS.Forward(m.clock.Now()) + req.barrierDone = true + + intents, resumeSpan, err := m.db.ScanInterleavedIntents(ctx, req.start, req.end, barrierTS) + if err != nil { + return nil, errors.Wrap(err, "error when invoking ScanInterleavedIntents command") + } + + txnIntents := make(map[uuid.UUID][]roachpb.Intent) + for _, intent := range intents { + txnIntents[intent.Txn.ID] = append(txnIntents[intent.Txn.ID], intent) + } + for _, intents := range txnIntents { + txn := &intents[0].Txn + + // Create a request for a PushTxn request of type PUSH_ABORT. If this + // transaction is still running, it will abort. The retry of that + // transaction will then write separated intents. + h := roachpb.Header{ + Timestamp: m.clock.Now(), + UserPriority: roachpb.MinUserPriority, + } + pushedTxn, err := m.ir.PushTransaction(ctx, txn, h, roachpb.PUSH_ABORT) + if err != nil { + return nil, err.GoError() + } + lockUpdates := make([]roachpb.LockUpdate, 0, len(intents)) + for _, intent := range intents { + resolve := roachpb.MakeLockUpdate(pushedTxn, roachpb.Span{Key: intent.Key}) + lockUpdates = append(lockUpdates, resolve) + } + opts := intentresolver.ResolveOptions{Poison: true} + if err := m.ir.ResolveIntents(ctx, lockUpdates, opts); err != nil { + return nil, err.GoError() + } + } + if resumeSpan != nil { + nextReq = req + nextReq.start = resumeSpan.Key + nextReq.end = resumeSpan.EndKey + nextReq.barrierDone = true + nextReq.barrierTS = barrierTS + } + return nextReq, nil +} + +func (m *migrateLockTablePool) run(ctx context.Context, workerIdx int) { + defer m.wg.Done() + ctx, cancel := m.stopper.WithCancelOnQuiesce(ctx) + defer cancel() + + var retryRequest *migrateLockTableRequest + retryAttempt := 0 + statusSlot := &m.status[workerIdx] + atomic.StoreInt64(statusSlot, 0) + + for { + if retryRequest == nil { + // Pull a new request out of the channel. + select { + case r, ok := <-m.requests: + if !ok { + return + } + retryRequest = &r + retryAttempt = 0 + case <-ctx.Done(): + log.Warningf(ctx, "lock table migration canceled") + return + } + } + + if ctx.Err() != nil { + log.Warningf(ctx, "lock table migration canceled on range r%d", retryRequest.debugRangeID) + return + } + + atomic.StoreInt64(statusSlot, int64(retryRequest.debugRangeID)) + handleError := func(err error) { + log.Errorf(ctx, "error when running migrate lock table command for range r%d: %s", + retryRequest.debugRangeID, err) + retryAttempt++ + if retryAttempt >= migrateLockTableRetries { + // Report this error to the migration manager. This will cause the + // whole migration to be retried later. In the meantime, continue + // migrating any other ranges in the queue, instead of stalling the + // pipeline. + m.mu.Lock() + // Limit the number of errors chained. This prevents excessive memory + // usage in case of error blowup (rangeCount * migrateLockTableRetries). + if m.mu.errorCount < 16 { + m.mu.combinedErr = errors.CombineErrors(m.mu.combinedErr, err) + } + m.mu.errorCount++ + m.mu.Unlock() + + retryAttempt = 0 + retryRequest = nil + atomic.StoreInt64(statusSlot, 0) + } + } + + nextReq, err := m.attemptMigrateRequest(ctx, retryRequest) + + if err != nil { + handleError(err) + continue + } else { + retryRequest = nextReq + retryAttempt = 0 + atomic.StoreInt64(statusSlot, 0) + } + } +} + +func (m *migrateLockTablePool) startStatusLogger(ctx context.Context) { + m.done = make(chan bool) + _ = m.stopper.RunAsyncTask(ctx, "migrate-lock-table-status", m.runStatusLogger) +} + +func (m *migrateLockTablePool) stopStatusLogger() { + close(m.done) +} + +func (m *migrateLockTablePool) runStatusLogger(ctx context.Context) { + ctx, cancel := m.stopper.WithCancelOnQuiesce(ctx) + defer cancel() + + const statusTickDuration = 5 * time.Second + ticker := time.NewTicker(statusTickDuration) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + var ranges strings.Builder + for i := 0; i < concurrentMigrateLockTableRequests; i++ { + rangeID := atomic.LoadInt64(&m.status[i]) + if rangeID == 0 { + continue + } + if ranges.Len() != 0 { + fmt.Fprintf(&ranges, ", ") + } + fmt.Fprintf(&ranges, "%s", roachpb.RangeID(rangeID)) + } + + if ranges.Len() > 0 { + log.Infof(ctx, "currently migrating lock table on ranges %s", ranges.String()) + } + + case <-m.done: + return + case <-ctx.Done(): + return + } + } +} + +// rangeIterator provides a not-necessarily-transactional view of KV ranges +// spanning a key range. +type rangeIterator interface { + Desc() *roachpb.RangeDescriptor + Error() error + NeedAnother(rs roachpb.RSpan) bool + Next(ctx context.Context) + Seek(ctx context.Context, key roachpb.RKey, scanDir kvcoord.ScanDirection) + Valid() bool +} + +// ignoreSeparatedIntentsMigrationForRange returns true if the migration should +// be skipped for this range. Only returns true for range containing timeseries +// keys; those ranges are guaranteed to not contain intents. +func ignoreSeparatedIntentsMigrationForRange(start, end roachpb.RKey) bool { + return bytes.HasPrefix(start, keys.TimeseriesPrefix) && bytes.HasPrefix(end, keys.TimeseriesPrefix) +} + +func runSeparatedIntentsMigration( + ctx context.Context, + clock *hlc.Clock, + stopper *stop.Stopper, + db *kv.DB, + ri rangeIterator, + ir intentResolver, +) error { + var numMigratedRanges int + workerPool := migrateLockTablePool{ + requests: make(chan migrateLockTableRequest, concurrentMigrateLockTableRequests), + stopper: stopper, + db: db, + ir: ir, + clock: clock, + } + + workerPool.wg.Add(concurrentMigrateLockTableRequests) + for i := 0; i < concurrentMigrateLockTableRequests; i++ { + idx := i // Copy for closure below. + taskName := fmt.Sprintf("migrate-lock-table-%d", i) + if err := stopper.RunAsyncTask(ctx, taskName, func(ctx context.Context) { + workerPool.run(ctx, idx) + }); err != nil { + return err + } + } + rs := roachpb.RSpan{Key: roachpb.RKeyMin, EndKey: roachpb.RKeyMax} + for ri.Seek(ctx, roachpb.RKeyMin, kvcoord.Ascending); ri.Valid(); ri.Next(ctx) { + desc := ri.Desc() + start, end := desc.StartKey, desc.EndKey + if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 { + start, _ = keys.Addr(keys.LocalMax) + } + if ignoreSeparatedIntentsMigrationForRange(start, end) { + continue + } + request := migrateLockTableRequest{ + start: start.AsRawKey(), + end: end.AsRawKey(), + debugRangeID: desc.RangeID, + } + select { + case workerPool.requests <- request: + case <-ctx.Done(): + return errors.New("lock table migration canceled") + } + + // Also enqueue a request for range local keys. + rangeKeyStart := keys.MakeRangeKeyPrefix(start) + rangeKeyEnd := keys.MakeRangeKeyPrefix(end) + + request2 := migrateLockTableRequest{ + start: rangeKeyStart, + end: rangeKeyEnd, + debugRangeID: desc.RangeID, + } + select { + case workerPool.requests <- request2: + case <-ctx.Done(): + return errors.New("lock table migration canceled") + } + numMigratedRanges++ + + if !ri.NeedAnother(rs) { + break + } + } + if err := ri.Error(); err != nil { + log.Errorf(ctx, "error when iterating through ranges in lock table migration: %s", err) + close(workerPool.requests) + workerPool.wg.Wait() + return err + } + workerPool.startStatusLogger(ctx) + + close(workerPool.requests) + workerPool.wg.Wait() + workerPool.stopStatusLogger() + + if workerPool.mu.combinedErr != nil { + return workerPool.mu.combinedErr + } + + log.Infof(ctx, "finished lock table migrations for %d ranges", numMigratedRanges) + return nil +} + +func separatedIntentsMigration( + ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, +) error { + ir := intentresolver.New(intentresolver.Config{ + Clock: deps.DB.Clock(), + Stopper: deps.Stopper, + RangeDescriptorCache: deps.DistSender.RangeDescriptorCache(), + DB: deps.DB, + }) + ri := kvcoord.NewRangeIterator(deps.DistSender) + + return runSeparatedIntentsMigration(ctx, deps.DB.Clock(), deps.Stopper, deps.DB, ri, ir) +} + +func postSeparatedIntentsMigration( + ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, +) error { + var batchIdx, numMigratedRanges int + init := func() { batchIdx, numMigratedRanges = 1, 0 } + + // Issue no-op Migrate commands to all ranges. This has the only + // purpose of clearing out any orphaned replicas, preventing interleaved + // intents in them from resurfacing. + if err := deps.Cluster.IterateRangeDescriptors(ctx, defaultPageSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + for _, desc := range descriptors { + start, end := desc.StartKey, desc.EndKey + if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 { + start, _ = keys.Addr(keys.LocalMax) + } + // Check if this range is a timeseries range. If it is, we can just skip + // it - it will not contain any intents. + if bytes.HasPrefix(start, keys.TimeseriesPrefix) && bytes.HasPrefix(end, keys.TimeseriesPrefix) { + continue + } + if err := deps.DB.Migrate(ctx, start, end, cv.Version); err != nil { + return err + } + } + numMigratedRanges += len(descriptors) + log.Infof(ctx, "[batch %d/??] started no-op migrations for %d ranges", batchIdx, numMigratedRanges) + batchIdx++ + return nil + }); err != nil { + return err + } + + log.Infof(ctx, "[batch %d/%d] finished no-op migrations for %d ranges", batchIdx, batchIdx, numMigratedRanges) + + return nil +} diff --git a/pkg/migration/migrations/separated_intents_external_test.go b/pkg/migration/migrations/separated_intents_external_test.go new file mode 100644 index 000000000000..3a844b262493 --- /dev/null +++ b/pkg/migration/migrations/separated_intents_external_test.go @@ -0,0 +1,177 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migrations_test + +import ( + "context" + "strconv" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestSeparatedIntentsMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + settings := cluster.MakeTestingClusterSettingsWithVersions( + clusterversion.ByKey(clusterversion.PostSeparatedIntentsMigration), + clusterversion.ByKey(clusterversion.SeparatedIntentsMigration-1), + false, /* initializeVersion */ + ) + storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, false) + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + const numServers int = 5 + stickyServerArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + Settings: settings, + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey(clusterversion.SeparatedIntentsMigration - 1), + StickyEngineRegistry: stickyEngineRegistry, + }, + }, + } + } + + tc := testcluster.StartTestCluster(t, numServers, base.TestClusterArgs{ + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) + require.NoError(t, tc.WaitForFullReplication()) + + // Create tables and databases, and get their IDs. + tdb := tc.ServerConn(0) + + getIntentCount := func(s *kvserver.Store) int { + db := s.Engine() + count := 0 + + iter := db.NewEngineIterator(storage.IterOptions{ + LowerBound: roachpb.KeyMin, + UpperBound: roachpb.KeyMax, + }) + defer iter.Close() + valid, err := iter.SeekEngineKeyGE(storage.EngineKey{Key: roachpb.KeyMin}) + for ; valid && err == nil; valid, err = iter.NextEngineKey() { + key, err := iter.EngineKey() + if err != nil { + t.Fatal(err) + } + if !key.IsMVCCKey() { + continue + } + mvccKey, err := key.ToMVCCKey() + if err != nil { + t.Fatal(err) + } + if !mvccKey.Timestamp.IsEmpty() { + continue + } + val := iter.Value() + meta := enginepb.MVCCMetadata{} + if err := protoutil.Unmarshal(val, &meta); err != nil { + t.Fatal(err) + } + if meta.IsInline() { + continue + } + count++ + } + + return count + } + + _, err := tdb.Exec("CREATE DATABASE test;") + require.NoError(t, err) + _, err = tdb.Exec("CREATE TABLE test.kv (key INTEGER PRIMARY KEY, val STRING NOT NULL);") + require.NoError(t, err) + + // Create a transaction, write a lot of rows (intents), then kill the + // gateway node before + txn, err := tdb.BeginTx(ctx, nil) + require.NoError(t, err) + _, err = txn.Exec("INSERT INTO test.kv SELECT generate_series(1, 100), 'test123';") + require.NoError(t, err) + + interleavedIntentCount := 0 + for i := 0; i < numServers; i++ { + err := tc.Server(i).GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + interleavedIntentCount += getIntentCount(s) + return nil + }) + require.NoError(t, err) + if interleavedIntentCount > 0 { + // This is all we care about; no need to waste cycles. + break + } + } + require.Greater(t, interleavedIntentCount, 0) + + storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, true) + require.NoError(t, tc.Restart()) + time.Sleep(10 * time.Second) + require.NoError(t, tc.WaitForFullReplication()) + tdb = tc.ServerConn(0) + + _, err = tdb.Exec(`SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.PostSeparatedIntentsMigration).String()) + require.NoError(t, err) + + time.Sleep(5 * time.Second) + testutils.SucceedsSoon(t, func() error { + interleavedIntentCount := 0 + for i := 0; i < 5; i++ { + err := tc.Server(i).GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + s.WaitForInit() + if !s.IsStarted() { + return errors.New("store not started") + } + interleavedIntentCount += getIntentCount(s) + return nil + }) + if err != nil { + return err + } + if interleavedIntentCount > 0 { + // This is all we care about; no need to waste cycles. + break + } + } + if interleavedIntentCount > 0 { + return errors.Errorf("expected 0 interleaved intents, got %d", interleavedIntentCount) + } + return nil + }) +} diff --git a/pkg/migration/migrations/separated_intents_test.go b/pkg/migration/migrations/separated_intents_test.go new file mode 100644 index 000000000000..70e2c28d9ff7 --- /dev/null +++ b/pkg/migration/migrations/separated_intents_test.go @@ -0,0 +1,291 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migrations + +import ( + "bytes" + "context" + "fmt" + "sort" + "strconv" + "strings" + "sync/atomic" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/uint128" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type mockRange struct { + desc *roachpb.RangeDescriptor + intents []roachpb.Intent + localIntents []roachpb.Intent +} + +type mockRangeIterator struct { + ranges []mockRange + idx int + err error +} + +func (m *mockRangeIterator) Desc() *roachpb.RangeDescriptor { + if m.idx >= len(m.ranges) || m.idx < 0 { + return nil + } + return m.ranges[m.idx].desc +} + +func (m *mockRangeIterator) Error() error { + return m.err +} + +func (m *mockRangeIterator) NeedAnother(rs roachpb.RSpan) bool { + return true +} + +func (m *mockRangeIterator) Next(ctx context.Context) { + m.idx++ +} + +func (m *mockRangeIterator) Seek( + ctx context.Context, key roachpb.RKey, scanDir kvcoord.ScanDirection, +) { + m.idx = 0 +} + +func (m *mockRangeIterator) Valid() bool { + return m.idx < len(m.ranges) && m.idx >= 0 +} + +type mockIntentResolver struct { + syncutil.Mutex + + pushedTxns []uuid.UUID + resolvedIntents []roachpb.Key +} + +func (m *mockIntentResolver) PushTransaction( + ctx context.Context, pushTxn *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, +) (*roachpb.Transaction, *roachpb.Error) { + m.Lock() + m.pushedTxns = append(m.pushedTxns, pushTxn.ID) + m.Unlock() + return &roachpb.Transaction{ + TxnMeta: *pushTxn, + }, nil +} + +func (m *mockIntentResolver) ResolveIntents( + ctx context.Context, intents []roachpb.LockUpdate, opts intentresolver.ResolveOptions, +) (pErr *roachpb.Error) { + m.Lock() + defer m.Unlock() + for i := range intents { + keyCopy := append([]byte(nil), intents[i].Key...) + m.resolvedIntents = append(m.resolvedIntents, keyCopy) + } + return nil +} + +func TestRunSeparatedIntentsMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + + ri := &mockRangeIterator{} + ir := &mockIntentResolver{} + hlcClock := hlc.NewClock(hlc.UnixNano, base.DefaultMaxClockOffset) + var numIntentsPerResumeSpan, errorPerNCalls int + var barrierCalls, scanIntentsCalls int64 + + mockSender := kv.MakeMockTxnSenderFactoryWithNonTxnSender(nil, + func(ctx context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + for i, req := range ba.Requests { + args := req.GetInner() + switch request := args.(type) { + case *roachpb.BarrierRequest: + newVal := atomic.AddInt64(&barrierCalls, 1) + if errorPerNCalls > 0 && int(newVal)%errorPerNCalls == 0 { + return nil, roachpb.NewError(errors.New("injected")) + } + br.Responses[i].MustSetInner(&roachpb.BarrierResponse{ + ResponseHeader: roachpb.ResponseHeader{}, + Timestamp: hlcClock.Now(), + }) + case *roachpb.ScanInterleavedIntentsRequest: + newVal := atomic.AddInt64(&scanIntentsCalls, 1) + if errorPerNCalls > 0 && int(newVal)%errorPerNCalls == 0 { + return nil, roachpb.NewError(errors.New("injected")) + } + + var resumeSpan *roachpb.Span + var r *mockRange + var intentsToScan []roachpb.Intent + for i := range ri.ranges { + if bytes.HasPrefix(request.Key, keys.LocalRangePrefix) { + rangeKeyRange := roachpb.RSpan{ + Key: roachpb.RKey(keys.MakeRangeKeyPrefix(ri.ranges[i].desc.StartKey)), + EndKey: roachpb.RKey(keys.MakeRangeKeyPrefix(ri.ranges[i].desc.EndKey)), + } + if rangeKeyRange.ContainsKeyRange(roachpb.RKey(request.Key), roachpb.RKey(request.EndKey)) { + r = &ri.ranges[i] + intentsToScan = r.localIntents + break + } + } + if ri.ranges[i].desc.ContainsKeyRange(roachpb.RKey(request.Key), roachpb.RKey(request.EndKey)) { + r = &ri.ranges[i] + intentsToScan = r.intents + break + } + } + if r == nil { + t.Fatal("ScanInterleavedIntents request issued outside range bounds") + } + startIdx := sort.Search(len(intentsToScan), func(i int) bool { + return bytes.Compare(request.Key, intentsToScan[i].Key) <= 0 + }) + endIdx := sort.Search(len(intentsToScan), func(i int) bool { + return bytes.Compare(request.EndKey, intentsToScan[i].Key) < 0 + }) + if endIdx-startIdx > numIntentsPerResumeSpan && numIntentsPerResumeSpan > 0 { + endIdx = startIdx + numIntentsPerResumeSpan + // Set the resume span. + if endIdx < len(intentsToScan) { + resumeSpan = &roachpb.Span{ + Key: intentsToScan[endIdx].Key, + EndKey: request.EndKey, + } + } + } + br.Responses[i].MustSetInner(&roachpb.ScanInterleavedIntentsResponse{ + ResponseHeader: roachpb.ResponseHeader{ResumeSpan: resumeSpan}, + Intents: intentsToScan[startIdx:endIdx], + }) + } + } + return br, nil + }) + + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + + db := kv.NewDB(log.AmbientContext{Tracer: tracing.NewTracer()}, mockSender, hlcClock, stopper) + + datadriven.RunTest(t, "testdata/separated_intents", + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "add-range": + var startKey, endKey string + var rangeID int + d.ScanArgs(t, "id", &rangeID) + d.ScanArgs(t, "key", &startKey) + d.ScanArgs(t, "endkey", &endKey) + + desc := &roachpb.RangeDescriptor{ + RangeID: roachpb.RangeID(rangeID), + StartKey: roachpb.RKey(startKey), + EndKey: roachpb.RKey(endKey), + } + var intents, localIntents []roachpb.Intent + localKeys := false + for _, line := range strings.Split(d.Input, "\n") { + if line == "local" { + localKeys = true + continue + } + fields := strings.Fields(line) + key := fields[0] + txnIDInt, err := strconv.Atoi(fields[1]) + require.NoError(t, err) + txnID := uuid.FromUint128(uint128.FromInts(0, uint64(txnIDInt))) + intentKey := []byte(key) + if localKeys { + intentKey = keys.MakeRangeKey(roachpb.RKey(startKey), keys.LocalQueueLastProcessedSuffix, roachpb.RKey(key)) + } + intent := roachpb.MakeIntent(&enginepb.TxnMeta{ID: txnID}, intentKey) + if localKeys { + localIntents = append(localIntents, intent) + } else { + intents = append(intents, intent) + } + } + ri.ranges = append(ri.ranges, mockRange{ + desc: desc, + intents: intents, + localIntents: localIntents, + }) + case "set-max-intent-count": + var err error + numIntentsPerResumeSpan, err = strconv.Atoi(d.Input) + require.NoError(t, err) + case "error-per-n-calls": + var err error + errorPerNCalls, err = strconv.Atoi(d.Input) + require.NoError(t, err) + case "run-migration": + err := runSeparatedIntentsMigration(ctx, hlcClock, stopper, db, ri, ir) + if err == nil { + return "ok" + } + return err.Error() + case "pushed-txns": + var builder strings.Builder + sort.Slice(ir.pushedTxns, func(i, j int) bool { + return ir.pushedTxns[i].ToUint128().Compare(ir.pushedTxns[j].ToUint128()) < 0 + }) + for i := range ir.pushedTxns { + fmt.Fprintf(&builder, "%d\n", ir.pushedTxns[i].ToUint128().Lo) + } + return builder.String() + case "resolved-intents": + var builder strings.Builder + sort.Slice(ir.resolvedIntents, func(i, j int) bool { + return ir.resolvedIntents[i].Compare(ir.resolvedIntents[j]) < 0 + }) + for i := range ir.resolvedIntents { + fmt.Fprintf(&builder, "%s\n", ir.resolvedIntents[i].String()) + } + return builder.String() + case "reset": + ri.ranges = nil + ri.idx = 0 + *ir = mockIntentResolver{} + barrierCalls = 0 + scanIntentsCalls = 0 + numIntentsPerResumeSpan = 0 + errorPerNCalls = 0 + case "count-calls": + return fmt.Sprintf("barrier: %d\nscanInterleavedIntents: %d\n", barrierCalls, scanIntentsCalls) + default: + return "unexpected command" + } + return "" + }) +} diff --git a/pkg/migration/migrations/separated_intents_test.go.orig b/pkg/migration/migrations/separated_intents_test.go.orig new file mode 100644 index 000000000000..6426f30dc662 --- /dev/null +++ b/pkg/migration/migrations/separated_intents_test.go.orig @@ -0,0 +1,291 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migrations + +import ( + "bytes" + "context" + "fmt" + "sort" + "strconv" + "strings" + "sync/atomic" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/uint128" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type mockRange struct { + desc *roachpb.RangeDescriptor + intents []roachpb.Intent + localIntents []roachpb.Intent +} + +type mockRangeIterator struct { + ranges []mockRange + idx int + err error +} + +func (m *mockRangeIterator) Desc() *roachpb.RangeDescriptor { + if m.idx >= len(m.ranges) || m.idx < 0 { + return nil + } + return m.ranges[m.idx].desc +} + +func (m *mockRangeIterator) Error() error { + return m.err +} + +func (m *mockRangeIterator) NeedAnother(rs roachpb.RSpan) bool { + return true +} + +func (m *mockRangeIterator) Next(ctx context.Context) { + m.idx++ +} + +func (m *mockRangeIterator) Seek( + ctx context.Context, key roachpb.RKey, scanDir kvcoord.ScanDirection, +) { + m.idx = 0 +} + +func (m *mockRangeIterator) Valid() bool { + return m.idx < len(m.ranges) && m.idx >= 0 +} + +type mockIntentResolver struct { + syncutil.Mutex + + pushedTxns []uuid.UUID + resolvedIntents []roachpb.Key +} + +func (m *mockIntentResolver) PushTransaction( + ctx context.Context, pushTxn *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, +) (*roachpb.Transaction, *roachpb.Error) { + m.Lock() + m.pushedTxns = append(m.pushedTxns, pushTxn.ID) + m.Unlock() + return &roachpb.Transaction{ + TxnMeta: *pushTxn, + }, nil +} + +func (m *mockIntentResolver) ResolveIntents( + ctx context.Context, intents []roachpb.LockUpdate, opts intentresolver.ResolveOptions, +) (pErr *roachpb.Error) { + m.Lock() + defer m.Unlock() + for i := range intents { + keyCopy := append([]byte(nil), intents[i].Key...) + m.resolvedIntents = append(m.resolvedIntents, keyCopy) + } + return nil +} + +func TestRunSeparatedIntentsMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx, cancel := context.WithTimeout(context.Background(), 15 * time.Second) + defer cancel() + + ri := &mockRangeIterator{} + ir := &mockIntentResolver{} + hlcClock := hlc.NewClock(hlc.UnixNano, base.DefaultMaxClockOffset) + var numIntentsPerResumeSpan, errorPerNCalls int + var barrierCalls, scanIntentsCalls int64 + + mockSender := kv.MakeMockTxnSenderFactoryWithNonTxnSender(nil, + func(ctx context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + for i, req := range ba.Requests { + args := req.GetInner() + switch request := args.(type) { + case *roachpb.BarrierRequest: + newVal := atomic.AddInt64(&barrierCalls, 1) + if errorPerNCalls > 0 && int(newVal) % errorPerNCalls == 0 { + return nil, roachpb.NewError(errors.New("injected")) + } + br.Responses[i].MustSetInner(&roachpb.BarrierResponse{ + ResponseHeader: roachpb.ResponseHeader{}, + Timestamp: hlcClock.Now(), + }) + case *roachpb.ScanInterleavedIntentsRequest: + newVal := atomic.AddInt64(&scanIntentsCalls, 1) + if errorPerNCalls > 0 && int(newVal) % errorPerNCalls == 0 { + return nil, roachpb.NewError(errors.New("injected")) + } + + var resumeSpan *roachpb.Span + var r *mockRange + var intentsToScan []roachpb.Intent + for i := range ri.ranges { + if bytes.HasPrefix(request.Key, keys.LocalRangePrefix) { + rangeKeyRange := roachpb.RSpan{ + Key: roachpb.RKey(keys.MakeRangeKeyPrefix(ri.ranges[i].desc.StartKey)), + EndKey: roachpb.RKey(keys.MakeRangeKeyPrefix(ri.ranges[i].desc.EndKey)), + } + if rangeKeyRange.ContainsKeyRange(roachpb.RKey(request.Key), roachpb.RKey(request.EndKey)) { + r = &ri.ranges[i] + intentsToScan = r.localIntents + break + } + } + if ri.ranges[i].desc.ContainsKeyRange(roachpb.RKey(request.Key), roachpb.RKey(request.EndKey)) { + r = &ri.ranges[i] + intentsToScan = r.intents + break + } + } + if r == nil { + t.Fatal("ScanInterleavedIntents request issued outside range bounds") + } + startIdx := sort.Search(len(intentsToScan), func(i int) bool { + return bytes.Compare(request.Key, intentsToScan[i].Key) <= 0 + }) + endIdx := sort.Search(len(intentsToScan), func(i int) bool { + return bytes.Compare(request.EndKey, intentsToScan[i].Key) < 0 + }) + if endIdx-startIdx > numIntentsPerResumeSpan && numIntentsPerResumeSpan > 0 { + endIdx = startIdx + numIntentsPerResumeSpan + // Set the resume span. + if endIdx < len(intentsToScan) { + resumeSpan = &roachpb.Span{ + Key: intentsToScan[endIdx].Key, + EndKey: request.EndKey, + } + } + } + br.Responses[i].MustSetInner(&roachpb.ScanInterleavedIntentsResponse{ + ResponseHeader: roachpb.ResponseHeader{ResumeSpan: resumeSpan}, + Intents: intentsToScan[startIdx:endIdx], + }) + } + } + return br, nil + }) + + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + + db := kv.NewDB(log.AmbientContext{Tracer: tracing.NewTracer()}, mockSender, hlcClock, stopper) + + datadriven.RunTest(t, "testdata/separated_intents", + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "add-range": + var startKey, endKey string + var rangeID int + d.ScanArgs(t, "id", &rangeID) + d.ScanArgs(t, "key", &startKey) + d.ScanArgs(t, "endkey", &endKey) + + desc := &roachpb.RangeDescriptor{ + RangeID: roachpb.RangeID(rangeID), + StartKey: roachpb.RKey(startKey), + EndKey: roachpb.RKey(endKey), + } + var intents, localIntents []roachpb.Intent + localKeys := false + for _, line := range strings.Split(d.Input, "\n") { + if line == "local" { + localKeys = true + continue + } + fields := strings.Fields(line) + key := fields[0] + txnIDInt, err := strconv.Atoi(fields[1]) + require.NoError(t, err) + txnID := uuid.FromUint128(uint128.FromInts(0, uint64(txnIDInt))) + intentKey := []byte(key) + if localKeys { + intentKey = keys.MakeRangeKey(roachpb.RKey(startKey), keys.LocalQueueLastProcessedSuffix, roachpb.RKey(key)) + } + intent := roachpb.MakeIntent(&enginepb.TxnMeta{ID: txnID}, intentKey) + if localKeys { + localIntents = append(localIntents, intent) + } else { + intents = append(intents, intent) + } + } + ri.ranges = append(ri.ranges, mockRange{ + desc: desc, + intents: intents, + localIntents: localIntents, + }) + case "set-max-intent-count": + var err error + numIntentsPerResumeSpan, err = strconv.Atoi(d.Input) + require.NoError(t, err) + case "error-per-n-calls": + var err error + errorPerNCalls, err = strconv.Atoi(d.Input) + require.NoError(t, err) + case "run-migration": + err := runSeparatedIntentsMigration(ctx, hlcClock, stopper, db, ri, ir) + if err == nil { + return "ok" + } + return err.Error() + case "pushed-txns": + var builder strings.Builder + sort.Slice(ir.pushedTxns, func(i, j int) bool { + return ir.pushedTxns[i].ToUint128().Compare(ir.pushedTxns[j].ToUint128()) < 0 + }) + for i := range ir.pushedTxns { + fmt.Fprintf(&builder, "%d\n", ir.pushedTxns[i].ToUint128().Lo) + } + return builder.String() + case "resolved-intents": + var builder strings.Builder + sort.Slice(ir.resolvedIntents, func(i, j int) bool { + return ir.resolvedIntents[i].Compare(ir.resolvedIntents[j]) < 0 + }) + for i := range ir.resolvedIntents { + fmt.Fprintf(&builder, "%s\n", ir.resolvedIntents[i].String()) + } + return builder.String() + case "reset": + ri.ranges = nil + ri.idx = 0 + *ir = mockIntentResolver{} + barrierCalls = 0 + scanIntentsCalls = 0 + numIntentsPerResumeSpan = 0 + errorPerNCalls = 0 + case "count-calls": + return fmt.Sprintf("barrier: %d\nscanInterleavedIntents: %d\n", barrierCalls, scanIntentsCalls) + default: + return "unexpected command" + } + return "" + }) +} diff --git a/pkg/migration/migrations/testdata/separated_intents b/pkg/migration/migrations/testdata/separated_intents new file mode 100644 index 000000000000..09e669009ac7 --- /dev/null +++ b/pkg/migration/migrations/testdata/separated_intents @@ -0,0 +1,403 @@ + +# Simple case - no resume spans (unlimited intents returned per scan), +# one range, no range local keys. + +reset +---- + +add-range id=3 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +---- + +run-migration +---- +ok + +count-calls +---- +barrier: 2 +scanInterleavedIntents: 2 + +pushed-txns +---- +1 +2 + +resolved-intents +---- +"a" +"b" +"c" +"d" +"e" +"f" + +reset +---- + +# Same case as above, but with use of resume spans. There should be 4 +# scanInterleavedIntents requests, 3 for the six range replicates keys, and 1 +# for range local keys. + +add-range id=3 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +---- + +set-max-intent-count +2 +---- + +run-migration +---- +ok + +count-calls +---- +barrier: 2 +scanInterleavedIntents: 4 + +pushed-txns +---- +1 +1 +2 +2 + +resolved-intents +---- +"a" +"b" +"c" +"d" +"e" +"f" + +reset +---- + +# Add range local keys, while maintaining the same limit of two intents returned +# per scanInterleavedIntents call. There should be 5 calls to +# scanInterleavedIntents in all - 3 for replicated keys, 2 for range local keys. + +add-range id=3 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +local + a 4 + b 3 + c 3 +---- + +set-max-intent-count +2 +---- + +run-migration +---- +ok + +count-calls +---- +barrier: 2 +scanInterleavedIntents: 5 + +pushed-txns +---- +1 +1 +2 +2 +3 +3 +4 + +resolved-intents +---- +/Local/Range"a"/QueueLastProcessed/"a" +/Local/Range"a"/QueueLastProcessed/"b" +/Local/Range"a"/QueueLastProcessed/"c" +"a" +"b" +"c" +"d" +"e" +"f" + +reset +---- + +# Case where one request is fired per intent. There are 15 keys across 2 ranges, +# so there should be 4 barrier calls (2x per range, 1x per intent), and 16 +# scanInterleavedIntents calls (1 per intent, 1 for the empty range local keys +# of range 2). + +add-range id=1 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +local + a 4 + b 3 + c 3 +---- + +add-range id=2 key=g endkey=k + g 5 + gg 6 + h 7 + hh 8 + i 9 + ii 10 +---- + +set-max-intent-count +1 +---- + +run-migration +---- +ok + +count-calls +---- +barrier: 4 +scanInterleavedIntents: 16 + +pushed-txns +---- +1 +1 +1 +2 +2 +2 +3 +3 +4 +5 +6 +7 +8 +9 +10 + +resolved-intents +---- +/Local/Range"a"/QueueLastProcessed/"a" +/Local/Range"a"/QueueLastProcessed/"b" +/Local/Range"a"/QueueLastProcessed/"c" +"a" +"b" +"c" +"d" +"e" +"f" +"g" +"gg" +"h" +"hh" +"i" +"ii" + +reset +---- + +# Same case as above, but with no resume span usage. + +add-range id=1 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +local + a 4 + b 3 + c 3 +---- + +add-range id=2 key=g endkey=k + g 5 + gg 6 + h 7 + hh 8 + i 9 + ii 10 +---- + +set-max-intent-count +0 +---- + +run-migration +---- +ok + +count-calls +---- +barrier: 4 +scanInterleavedIntents: 4 + +pushed-txns +---- +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 + +resolved-intents +---- +/Local/Range"a"/QueueLastProcessed/"a" +/Local/Range"a"/QueueLastProcessed/"b" +/Local/Range"a"/QueueLastProcessed/"c" +"a" +"b" +"c" +"d" +"e" +"f" +"g" +"gg" +"h" +"hh" +"i" +"ii" + +reset +---- + +# Take the case above, and add an error injection rate of 33% (1/3). The number +# of calls will go up, but the end result (pushed txns, resolved intents) will +# be the same. count-calls isn't added here as the count itself could be +# possibly non-deterministic. + +add-range id=1 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +local + a 4 + b 3 + c 3 +---- + +add-range id=2 key=g endkey=k + g 5 + gg 6 + h 7 + hh 8 + i 9 + ii 10 +---- + +error-per-n-calls +3 +---- + +run-migration +---- +ok + + +pushed-txns +---- +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 + +resolved-intents +---- +/Local/Range"a"/QueueLastProcessed/"a" +/Local/Range"a"/QueueLastProcessed/"b" +/Local/Range"a"/QueueLastProcessed/"c" +"a" +"b" +"c" +"d" +"e" +"f" +"g" +"gg" +"h" +"hh" +"i" +"ii" + +reset +---- + +# Take the case above, and add an error injection rate of 100% (1/1). The +# migration should error out and quit gracefully. + +add-range id=1 key=a endkey=g + a 1 + b 1 + c 1 + d 2 + e 2 + f 2 +local + a 4 + b 3 + c 3 +---- + +add-range id=2 key=g endkey=k + g 5 + gg 6 + h 7 + hh 8 + i 9 + ii 10 +---- + +error-per-n-calls +1 +---- + +run-migration +---- +error when invoking Barrier command: injected + +pushed-txns +---- + +resolved-intents +---- + +reset +---- diff --git a/pkg/migration/migrations/truncated_state.go b/pkg/migration/migrations/truncated_state.go index 3129643ea6b7..ff1033e055c2 100644 --- a/pkg/migration/migrations/truncated_state.go +++ b/pkg/migration/migrations/truncated_state.go @@ -37,11 +37,11 @@ import ( const defaultPageSize = 200 func truncatedStateMigration( - ctx context.Context, cv clusterversion.ClusterVersion, h migration.Cluster, + ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, ) error { var batchIdx, numMigratedRanges int init := func() { batchIdx, numMigratedRanges = 1, 0 } - if err := h.IterateRangeDescriptors(ctx, defaultPageSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + if err := deps.Cluster.IterateRangeDescriptors(ctx, defaultPageSize, init, func(descriptors ...roachpb.RangeDescriptor) error { for _, desc := range descriptors { // NB: This is a bit of a wart. We want to reach the first range, // but we can't address the (local) StartKey. However, keys.LocalMax @@ -50,7 +50,7 @@ func truncatedStateMigration( if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 { start, _ = keys.Addr(keys.LocalMax) } - if err := h.DB().Migrate(ctx, start, end, cv.Version); err != nil { + if err := deps.DB.Migrate(ctx, start, end, cv.Version); err != nil { return err } } @@ -73,21 +73,21 @@ func truncatedStateMigration( // migrations, this ensures that the applied state is flushed to disk. req := &serverpb.SyncAllEnginesRequest{} op := "flush-stores" - return h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + return deps.Cluster.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { _, err := client.SyncAllEngines(ctx, req) return err }) } func postTruncatedStateMigration( - ctx context.Context, cv clusterversion.ClusterVersion, h migration.Cluster, + ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, ) error { // Purge all replicas that haven't been migrated to use the unreplicated // truncated state and the range applied state. truncStateVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) req := &serverpb.PurgeOutdatedReplicasRequest{Version: &truncStateVersion} op := fmt.Sprintf("purge-outdated-replicas=%s", req.Version) - return h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + return deps.Cluster.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { _, err := client.PurgeOutdatedReplicas(ctx, req) return err }) diff --git a/pkg/migration/system_migration.go b/pkg/migration/system_migration.go index aa972d38e70e..4b0935c7352d 100644 --- a/pkg/migration/system_migration.go +++ b/pkg/migration/system_migration.go @@ -16,18 +16,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/logtags" ) // Cluster abstracts a physical KV cluster and can be utilized by a long-running // migration. type Cluster interface { - - // DB returns access to the kv. - DB() *kv.DB - // ForEveryNode is a short hand to execute the given closure (named by the // informational parameter op) against every node in the cluster at a given // point in time. Given it's possible for nodes to join or leave the cluster @@ -123,6 +121,15 @@ type Cluster interface { ) error } +// SystemDeps are the dependencies of migrations which perform actions at the +// KV layer on behalf of the system tenant. +type SystemDeps struct { + Cluster Cluster + DB *kv.DB + DistSender *kvcoord.DistSender + Stopper *stop.Stopper +} + // SystemMigration is an implementation of Migration for system-level // migrations. It is only to be run on the system tenant. These migrations // tend to touch the kv layer. @@ -133,7 +140,7 @@ type SystemMigration struct { // SystemMigrationFunc is used to perform kv-level migrations. It should only be // run from the system tenant. -type SystemMigrationFunc func(context.Context, clusterversion.ClusterVersion, Cluster) error +type SystemMigrationFunc func(context.Context, clusterversion.ClusterVersion, SystemDeps) error // NewSystemMigration constructs a SystemMigration. func NewSystemMigration( @@ -150,8 +157,8 @@ func NewSystemMigration( // Run kickstarts the actual migration process for system-level migrations. func (m *SystemMigration) Run( - ctx context.Context, cv clusterversion.ClusterVersion, h Cluster, + ctx context.Context, cv clusterversion.ClusterVersion, d SystemDeps, ) error { ctx = logtags.AddTag(ctx, fmt.Sprintf("migration=%s", cv), nil) - return m.fn(ctx, cv, h) + return m.fn(ctx, cv, d) } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 7657296cafec..facf96d47db4 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -678,6 +678,12 @@ func (*AdminVerifyProtectedTimestampRequest) Method() Method { return AdminVerif // Method implements the Request interface. func (*QueryResolvedTimestampRequest) Method() Method { return QueryResolvedTimestamp } +// Method implements the Request interface. +func (*ScanInterleavedIntentsRequest) Method() Method { return ScanInterleavedIntents } + +// Method implements the Request interface. +func (*BarrierRequest) Method() Method { return Barrier } + // ShallowCopy implements the Request interface. func (gr *GetRequest) ShallowCopy() Request { shallowCopy := *gr @@ -942,6 +948,18 @@ func (r *QueryResolvedTimestampRequest) ShallowCopy() Request { return &shallowCopy } +// ShallowCopy implements the Request interface. +func (r *ScanInterleavedIntentsRequest) ShallowCopy() Request { + shallowCopy := *r + return &shallowCopy +} + +// ShallowCopy implements the Request interface. +func (r *BarrierRequest) ShallowCopy() Request { + shallowCopy := *r + return &shallowCopy +} + // NewGet returns a Request initialized to get the value at key. If // forUpdate is true, an unreplicated, exclusive lock is acquired on on // the key, if it exists. @@ -1286,6 +1304,8 @@ func (r *RefreshRangeRequest) flags() int { func (*SubsumeRequest) flags() int { return isRead | isAlone | updatesTSCache } func (*RangeStatsRequest) flags() int { return isRead } func (*QueryResolvedTimestampRequest) flags() int { return isRead | isRange } +func (*ScanInterleavedIntentsRequest) flags() int { return isRead | isRange } +func (*BarrierRequest) flags() int { return isWrite | isRange } // IsParallelCommit returns whether the EndTxn request is attempting to perform // a parallel commit. See txn_interceptor_committer.go for a discussion about diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 694eb69ef45e..424011dc6245 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -402,7 +402,7 @@ func (x AdmissionHeader_Source) String() string { } func (AdmissionHeader_Source) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{97, 0} + return fileDescriptor_e08772acc330f58b, []int{101, 0} } // SourceLocation specifies physically where the call originated. LOCAL @@ -431,7 +431,7 @@ func (x AdmissionHeader_SourceLocation) String() string { } func (AdmissionHeader_SourceLocation) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{97, 1} + return fileDescriptor_e08772acc330f58b, []int{101, 1} } // RequestHeader is supplied with every storage node request. @@ -5017,6 +5017,153 @@ func (m *QueryResolvedTimestampResponse) XXX_DiscardUnknown() { var xxx_messageInfo_QueryResolvedTimestampResponse proto.InternalMessageInfo +// ScanInterleavedIntentsRequest is the request for a ScanInterleavedIntents operation. +// This is a read-only operation that returns all interleaved (non-separated) +// intents found over the request range. +type ScanInterleavedIntentsRequest struct { + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` +} + +func (m *ScanInterleavedIntentsRequest) Reset() { *m = ScanInterleavedIntentsRequest{} } +func (m *ScanInterleavedIntentsRequest) String() string { return proto.CompactTextString(m) } +func (*ScanInterleavedIntentsRequest) ProtoMessage() {} +func (*ScanInterleavedIntentsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{94} +} +func (m *ScanInterleavedIntentsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScanInterleavedIntentsRequest) 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 *ScanInterleavedIntentsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScanInterleavedIntentsRequest.Merge(m, src) +} +func (m *ScanInterleavedIntentsRequest) XXX_Size() int { + return m.Size() +} +func (m *ScanInterleavedIntentsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ScanInterleavedIntentsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ScanInterleavedIntentsRequest proto.InternalMessageInfo + +// ScanInterleavedIntentsResponse is the response to a ScanInterleavedIntents operation. +type ScanInterleavedIntentsResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + // The intents encountered in the part of the request span that was + // evaluated. A resume span is set in the response header if the entirety of + // the request span was not evaluated. + Intents []Intent `protobuf:"bytes,3,rep,name=intents,proto3" json:"intents"` +} + +func (m *ScanInterleavedIntentsResponse) Reset() { *m = ScanInterleavedIntentsResponse{} } +func (m *ScanInterleavedIntentsResponse) String() string { return proto.CompactTextString(m) } +func (*ScanInterleavedIntentsResponse) ProtoMessage() {} +func (*ScanInterleavedIntentsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{95} +} +func (m *ScanInterleavedIntentsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScanInterleavedIntentsResponse) 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 *ScanInterleavedIntentsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScanInterleavedIntentsResponse.Merge(m, src) +} +func (m *ScanInterleavedIntentsResponse) XXX_Size() int { + return m.Size() +} +func (m *ScanInterleavedIntentsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ScanInterleavedIntentsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ScanInterleavedIntentsResponse proto.InternalMessageInfo + +// BarrierRequest is the request for a Barrier operation. This goes through Raft +// and has the purpose of waiting until all conflicting in-flight operations on +// this range have completed, without blocking any new operations. +type BarrierRequest struct { + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` +} + +func (m *BarrierRequest) Reset() { *m = BarrierRequest{} } +func (m *BarrierRequest) String() string { return proto.CompactTextString(m) } +func (*BarrierRequest) ProtoMessage() {} +func (*BarrierRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{96} +} +func (m *BarrierRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BarrierRequest) 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 *BarrierRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BarrierRequest.Merge(m, src) +} +func (m *BarrierRequest) XXX_Size() int { + return m.Size() +} +func (m *BarrierRequest) XXX_DiscardUnknown() { + xxx_messageInfo_BarrierRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_BarrierRequest proto.InternalMessageInfo + +// BarrierResponse is the response for a Barrier operation. +type BarrierResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + // Timestamp at which this Barrier was evaluated. Can be used to guarantee + // future operations happen on the same or newer leaseholders. + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` +} + +func (m *BarrierResponse) Reset() { *m = BarrierResponse{} } +func (m *BarrierResponse) String() string { return proto.CompactTextString(m) } +func (*BarrierResponse) ProtoMessage() {} +func (*BarrierResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{97} +} +func (m *BarrierResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BarrierResponse) 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 *BarrierResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BarrierResponse.Merge(m, src) +} +func (m *BarrierResponse) XXX_Size() int { + return m.Size() +} +func (m *BarrierResponse) XXX_DiscardUnknown() { + xxx_messageInfo_BarrierResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_BarrierResponse proto.InternalMessageInfo + // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. // @@ -5068,6 +5215,8 @@ type RequestUnion struct { // *RequestUnion_AdminVerifyProtectedTimestamp // *RequestUnion_Migrate // *RequestUnion_QueryResolvedTimestamp + // *RequestUnion_ScanInterleavedIntents + // *RequestUnion_Barrier Value isRequestUnion_Value `protobuf_oneof:"value"` } @@ -5075,7 +5224,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{94} + return fileDescriptor_e08772acc330f58b, []int{98} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5238,6 +5387,12 @@ type RequestUnion_Migrate struct { type RequestUnion_QueryResolvedTimestamp struct { QueryResolvedTimestamp *QueryResolvedTimestampRequest `protobuf:"bytes,51,opt,name=query_resolved_timestamp,json=queryResolvedTimestamp,proto3,oneof" json:"query_resolved_timestamp,omitempty"` } +type RequestUnion_ScanInterleavedIntents struct { + ScanInterleavedIntents *ScanInterleavedIntentsRequest `protobuf:"bytes,52,opt,name=scan_interleaved_intents,json=scanInterleavedIntents,proto3,oneof" json:"scan_interleaved_intents,omitempty"` +} +type RequestUnion_Barrier struct { + Barrier *BarrierRequest `protobuf:"bytes,53,opt,name=barrier,proto3,oneof" json:"barrier,omitempty"` +} func (*RequestUnion_Get) isRequestUnion_Value() {} func (*RequestUnion_Put) isRequestUnion_Value() {} @@ -5283,6 +5438,8 @@ func (*RequestUnion_RangeStats) isRequestUnion_Value() {} func (*RequestUnion_AdminVerifyProtectedTimestamp) isRequestUnion_Value() {} func (*RequestUnion_Migrate) isRequestUnion_Value() {} func (*RequestUnion_QueryResolvedTimestamp) isRequestUnion_Value() {} +func (*RequestUnion_ScanInterleavedIntents) isRequestUnion_Value() {} +func (*RequestUnion_Barrier) isRequestUnion_Value() {} func (m *RequestUnion) GetValue() isRequestUnion_Value { if m != nil { @@ -5599,6 +5756,20 @@ func (m *RequestUnion) GetQueryResolvedTimestamp() *QueryResolvedTimestampReques return nil } +func (m *RequestUnion) GetScanInterleavedIntents() *ScanInterleavedIntentsRequest { + if x, ok := m.GetValue().(*RequestUnion_ScanInterleavedIntents); ok { + return x.ScanInterleavedIntents + } + return nil +} + +func (m *RequestUnion) GetBarrier() *BarrierRequest { + if x, ok := m.GetValue().(*RequestUnion_Barrier); ok { + return x.Barrier + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*RequestUnion) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -5646,6 +5817,8 @@ func (*RequestUnion) XXX_OneofWrappers() []interface{} { (*RequestUnion_AdminVerifyProtectedTimestamp)(nil), (*RequestUnion_Migrate)(nil), (*RequestUnion_QueryResolvedTimestamp)(nil), + (*RequestUnion_ScanInterleavedIntents)(nil), + (*RequestUnion_Barrier)(nil), } } @@ -5696,6 +5869,8 @@ type ResponseUnion struct { // *ResponseUnion_AdminVerifyProtectedTimestamp // *ResponseUnion_Migrate // *ResponseUnion_QueryResolvedTimestamp + // *ResponseUnion_ScanInterleavedIntents + // *ResponseUnion_Barrier Value isResponseUnion_Value `protobuf_oneof:"value"` } @@ -5703,7 +5878,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{95} + return fileDescriptor_e08772acc330f58b, []int{99} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5863,6 +6038,12 @@ type ResponseUnion_Migrate struct { type ResponseUnion_QueryResolvedTimestamp struct { QueryResolvedTimestamp *QueryResolvedTimestampResponse `protobuf:"bytes,51,opt,name=query_resolved_timestamp,json=queryResolvedTimestamp,proto3,oneof" json:"query_resolved_timestamp,omitempty"` } +type ResponseUnion_ScanInterleavedIntents struct { + ScanInterleavedIntents *ScanInterleavedIntentsResponse `protobuf:"bytes,52,opt,name=scan_interleaved_intents,json=scanInterleavedIntents,proto3,oneof" json:"scan_interleaved_intents,omitempty"` +} +type ResponseUnion_Barrier struct { + Barrier *BarrierResponse `protobuf:"bytes,53,opt,name=barrier,proto3,oneof" json:"barrier,omitempty"` +} func (*ResponseUnion_Get) isResponseUnion_Value() {} func (*ResponseUnion_Put) isResponseUnion_Value() {} @@ -5907,6 +6088,8 @@ func (*ResponseUnion_RangeStats) isResponseUnion_Value() {} func (*ResponseUnion_AdminVerifyProtectedTimestamp) isResponseUnion_Value() {} func (*ResponseUnion_Migrate) isResponseUnion_Value() {} func (*ResponseUnion_QueryResolvedTimestamp) isResponseUnion_Value() {} +func (*ResponseUnion_ScanInterleavedIntents) isResponseUnion_Value() {} +func (*ResponseUnion_Barrier) isResponseUnion_Value() {} func (m *ResponseUnion) GetValue() isResponseUnion_Value { if m != nil { @@ -6216,6 +6399,20 @@ func (m *ResponseUnion) GetQueryResolvedTimestamp() *QueryResolvedTimestampRespo return nil } +func (m *ResponseUnion) GetScanInterleavedIntents() *ScanInterleavedIntentsResponse { + if x, ok := m.GetValue().(*ResponseUnion_ScanInterleavedIntents); ok { + return x.ScanInterleavedIntents + } + return nil +} + +func (m *ResponseUnion) GetBarrier() *BarrierResponse { + if x, ok := m.GetValue().(*ResponseUnion_Barrier); ok { + return x.Barrier + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*ResponseUnion) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -6262,6 +6459,8 @@ func (*ResponseUnion) XXX_OneofWrappers() []interface{} { (*ResponseUnion_AdminVerifyProtectedTimestamp)(nil), (*ResponseUnion_Migrate)(nil), (*ResponseUnion_QueryResolvedTimestamp)(nil), + (*ResponseUnion_ScanInterleavedIntents)(nil), + (*ResponseUnion_Barrier)(nil), } } @@ -6446,7 +6645,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{96} + return fileDescriptor_e08772acc330f58b, []int{100} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6506,7 +6705,7 @@ func (m *AdmissionHeader) Reset() { *m = AdmissionHeader{} } func (m *AdmissionHeader) String() string { return proto.CompactTextString(m) } func (*AdmissionHeader) ProtoMessage() {} func (*AdmissionHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{97} + return fileDescriptor_e08772acc330f58b, []int{101} } func (m *AdmissionHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6543,7 +6742,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{98} + return fileDescriptor_e08772acc330f58b, []int{102} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6580,7 +6779,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{99} + return fileDescriptor_e08772acc330f58b, []int{103} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6656,7 +6855,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{99, 0} + return fileDescriptor_e08772acc330f58b, []int{103, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6694,7 +6893,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{100} + return fileDescriptor_e08772acc330f58b, []int{104} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6732,7 +6931,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{101} + return fileDescriptor_e08772acc330f58b, []int{105} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6774,7 +6973,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{102} + return fileDescriptor_e08772acc330f58b, []int{106} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6815,7 +7014,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{103} + return fileDescriptor_e08772acc330f58b, []int{107} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6856,7 +7055,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{104} + return fileDescriptor_e08772acc330f58b, []int{108} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6893,7 +7092,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{105} + return fileDescriptor_e08772acc330f58b, []int{109} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6930,7 +7129,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{106} + return fileDescriptor_e08772acc330f58b, []int{110} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6974,7 +7173,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{107} + return fileDescriptor_e08772acc330f58b, []int{111} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7006,7 +7205,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{108} + return fileDescriptor_e08772acc330f58b, []int{112} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7047,7 +7246,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{109} + return fileDescriptor_e08772acc330f58b, []int{113} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7087,7 +7286,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{110} + return fileDescriptor_e08772acc330f58b, []int{114} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7127,7 +7326,7 @@ func (m *TenantConsumption) Reset() { *m = TenantConsumption{} } func (m *TenantConsumption) String() string { return proto.CompactTextString(m) } func (*TenantConsumption) ProtoMessage() {} func (*TenantConsumption) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{111} + return fileDescriptor_e08772acc330f58b, []int{115} } func (m *TenantConsumption) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7170,7 +7369,7 @@ func (m *TokenBucketRequest) Reset() { *m = TokenBucketRequest{} } func (m *TokenBucketRequest) String() string { return proto.CompactTextString(m) } func (*TokenBucketRequest) ProtoMessage() {} func (*TokenBucketRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{112} + return fileDescriptor_e08772acc330f58b, []int{116} } func (m *TokenBucketRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7206,7 +7405,7 @@ func (m *TokenBucketResponse) Reset() { *m = TokenBucketResponse{} } func (m *TokenBucketResponse) String() string { return proto.CompactTextString(m) } func (*TokenBucketResponse) ProtoMessage() {} func (*TokenBucketResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{113} + return fileDescriptor_e08772acc330f58b, []int{117} } func (m *TokenBucketResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7242,7 +7441,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{114} + return fileDescriptor_e08772acc330f58b, []int{118} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7281,7 +7480,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{115} + return fileDescriptor_e08772acc330f58b, []int{119} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7321,7 +7520,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{116} + return fileDescriptor_e08772acc330f58b, []int{120} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7465,6 +7664,10 @@ func init() { proto.RegisterType((*MigrateResponse)(nil), "cockroach.roachpb.MigrateResponse") proto.RegisterType((*QueryResolvedTimestampRequest)(nil), "cockroach.roachpb.QueryResolvedTimestampRequest") proto.RegisterType((*QueryResolvedTimestampResponse)(nil), "cockroach.roachpb.QueryResolvedTimestampResponse") + proto.RegisterType((*ScanInterleavedIntentsRequest)(nil), "cockroach.roachpb.ScanInterleavedIntentsRequest") + proto.RegisterType((*ScanInterleavedIntentsResponse)(nil), "cockroach.roachpb.ScanInterleavedIntentsResponse") + proto.RegisterType((*BarrierRequest)(nil), "cockroach.roachpb.BarrierRequest") + proto.RegisterType((*BarrierResponse)(nil), "cockroach.roachpb.BarrierResponse") proto.RegisterType((*RequestUnion)(nil), "cockroach.roachpb.RequestUnion") proto.RegisterType((*ResponseUnion)(nil), "cockroach.roachpb.ResponseUnion") proto.RegisterType((*Header)(nil), "cockroach.roachpb.Header") @@ -7494,551 +7697,560 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 8701 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x24, 0x59, - 0x96, 0x96, 0x23, 0x33, 0x6d, 0x67, 0x9e, 0x74, 0x66, 0x86, 0xaf, 0xeb, 0xc7, 0xed, 0xea, 0xb6, - 0xab, 0xa2, 0xba, 0x7e, 0x77, 0x3a, 0xdd, 0x55, 0x35, 0xc3, 0xf4, 0x76, 0xf7, 0xf4, 0xac, 0x33, - 0x9d, 0x55, 0x99, 0xfe, 0x2b, 0x57, 0x64, 0xba, 0x4a, 0xdd, 0x3b, 0x4b, 0x6c, 0x38, 0xe2, 0xda, - 0x8e, 0x71, 0x66, 0x44, 0x56, 0x44, 0xa4, 0xcb, 0x6e, 0x89, 0x07, 0x60, 0x35, 0xf4, 0x13, 0x1a, - 0x24, 0xa4, 0xed, 0xd1, 0x22, 0x34, 0xfb, 0x23, 0x78, 0xe0, 0x01, 0x04, 0x08, 0xb4, 0x08, 0x58, - 0x89, 0x17, 0x46, 0x68, 0xc4, 0xcc, 0xbe, 0xad, 0x90, 0x30, 0x8b, 0x87, 0x07, 0x56, 0x08, 0x81, - 0x00, 0x69, 0x51, 0x4b, 0x20, 0x74, 0x7f, 0xe2, 0x2f, 0x33, 0x32, 0x9d, 0xae, 0x8e, 0x66, 0x5b, - 0xda, 0x17, 0x3b, 0xe3, 0xdc, 0x7b, 0x4e, 0xdc, 0x7b, 0xee, 0xdf, 0xf9, 0xee, 0x3d, 0xe7, 0x06, - 0xcc, 0xda, 0x96, 0xaa, 0x1d, 0x74, 0x77, 0x97, 0xd5, 0xae, 0x51, 0xee, 0xda, 0x96, 0x6b, 0xa1, - 0x59, 0xcd, 0xd2, 0x0e, 0x29, 0xb9, 0xcc, 0x13, 0x17, 0x2e, 0x63, 0xdb, 0xb6, 0x6c, 0xa7, 0xbb, - 0xbb, 0xcc, 0x7e, 0xb0, 0x9c, 0x0b, 0xf7, 0x0f, 0x8f, 0x96, 0x0f, 0x8f, 0x1c, 0x6c, 0x1f, 0x61, - 0x7b, 0x59, 0xb3, 0x4c, 0xad, 0x67, 0xdb, 0xd8, 0xd4, 0x4e, 0x96, 0xdb, 0x96, 0x76, 0x48, 0xff, - 0x18, 0xe6, 0x7e, 0x5c, 0x5e, 0x1b, 0xab, 0xba, 0xd3, 0xeb, 0x74, 0x54, 0xfb, 0x64, 0x99, 0x8a, - 0xe5, 0x0f, 0x3c, 0x2f, 0xf2, 0x0a, 0xa5, 0xab, 0xae, 0xca, 0x69, 0x97, 0x3c, 0x5a, 0xa4, 0x04, - 0x57, 0x3c, 0x6a, 0x07, 0xbb, 0x6a, 0x28, 0xf7, 0x35, 0xc7, 0xb5, 0x6c, 0x75, 0x1f, 0x2f, 0x63, - 0x73, 0xdf, 0x30, 0x31, 0xc9, 0x70, 0xa4, 0x69, 0x3c, 0xf1, 0xcd, 0xd8, 0xc4, 0x47, 0x3c, 0x75, - 0xbe, 0xe7, 0x1a, 0xed, 0xe5, 0x83, 0xb6, 0xb6, 0xec, 0x1a, 0x1d, 0xec, 0xb8, 0x6a, 0xa7, 0xeb, - 0x55, 0x81, 0xa6, 0xb8, 0xb6, 0xaa, 0x19, 0xe6, 0xbe, 0xf7, 0xbf, 0xbb, 0xbb, 0x6c, 0x63, 0xcd, - 0xb2, 0x75, 0xac, 0x2b, 0x4e, 0x57, 0x35, 0xbd, 0xe2, 0xee, 0x5b, 0xfb, 0x16, 0xfd, 0xb9, 0x4c, - 0x7e, 0x71, 0xea, 0xe2, 0xbe, 0x65, 0xed, 0xb7, 0xf1, 0x32, 0x7d, 0xda, 0xed, 0xed, 0x2d, 0xeb, - 0x3d, 0x5b, 0x75, 0x0d, 0x8b, 0x73, 0x49, 0xff, 0x44, 0x80, 0x82, 0x8c, 0x5f, 0xf6, 0xb0, 0xe3, - 0xd6, 0xb1, 0xaa, 0x63, 0x1b, 0xbd, 0x01, 0xe9, 0x43, 0x7c, 0x32, 0x9f, 0xbe, 0x2e, 0xdc, 0x9d, - 0xa9, 0x4c, 0x7f, 0x71, 0xba, 0x94, 0x5e, 0xc7, 0x27, 0x32, 0xa1, 0xa1, 0xeb, 0x30, 0x8d, 0x4d, - 0x5d, 0x21, 0xc9, 0x99, 0x68, 0xf2, 0x14, 0x36, 0xf5, 0x75, 0x7c, 0x82, 0xbe, 0x07, 0x59, 0x87, - 0x48, 0x33, 0x35, 0x3c, 0x3f, 0x79, 0x5d, 0xb8, 0x3b, 0x59, 0xf9, 0x95, 0x2f, 0x4e, 0x97, 0x3e, - 0xdc, 0x37, 0xdc, 0x83, 0xde, 0x6e, 0x59, 0xb3, 0x3a, 0xcb, 0x7e, 0x53, 0xeb, 0xbb, 0xc1, 0xef, - 0xe5, 0xee, 0xe1, 0xfe, 0x72, 0xbf, 0x8e, 0xca, 0xad, 0x63, 0xb3, 0x89, 0x5f, 0xca, 0xbe, 0xc4, - 0xb5, 0x4c, 0x56, 0x10, 0x53, 0x6b, 0x99, 0x6c, 0x4a, 0x4c, 0x4b, 0x3f, 0x4b, 0x41, 0x51, 0xc6, - 0x4e, 0xd7, 0x32, 0x1d, 0xcc, 0x4b, 0xfe, 0x2e, 0xa4, 0xdd, 0x63, 0x93, 0x96, 0x3c, 0xff, 0x70, - 0xb1, 0x3c, 0xd0, 0xa9, 0xca, 0x2d, 0x5b, 0x35, 0x1d, 0x55, 0x23, 0xd5, 0x97, 0x49, 0x56, 0xf4, - 0x1e, 0xe4, 0x6d, 0xec, 0xf4, 0x3a, 0x98, 0x2a, 0x92, 0x56, 0x2a, 0xff, 0xf0, 0x6a, 0x0c, 0x67, - 0xb3, 0xab, 0x9a, 0x32, 0xb0, 0xbc, 0xe4, 0x37, 0x6a, 0x42, 0x81, 0x73, 0xda, 0x58, 0x75, 0x2c, - 0x73, 0x7e, 0xfa, 0xba, 0x70, 0xb7, 0xf8, 0xb0, 0x1c, 0xc3, 0x1b, 0x2d, 0x25, 0x79, 0xec, 0x75, - 0xb0, 0x4c, 0xb9, 0xe4, 0x19, 0x3b, 0xf4, 0x84, 0xde, 0x80, 0xac, 0xd9, 0xeb, 0x10, 0xfd, 0x3a, - 0x54, 0x7b, 0x69, 0x79, 0xda, 0xec, 0x75, 0xd6, 0xf1, 0x89, 0x83, 0xae, 0x41, 0x8e, 0x24, 0xed, - 0x9e, 0xb8, 0xd8, 0x99, 0xcf, 0xd2, 0x34, 0x92, 0xb7, 0x42, 0x9e, 0xa5, 0x8f, 0x60, 0x26, 0x2c, - 0x15, 0x21, 0x28, 0xca, 0xb5, 0xe6, 0xce, 0x66, 0x4d, 0xd9, 0xd9, 0x5a, 0xdf, 0x7a, 0xfa, 0x62, - 0x4b, 0x9c, 0x40, 0x97, 0x40, 0xe4, 0xb4, 0xf5, 0xda, 0xc7, 0xca, 0x46, 0x63, 0xb3, 0xd1, 0x12, - 0x85, 0x85, 0xcc, 0x67, 0xbf, 0xbb, 0x38, 0xb1, 0x96, 0xc9, 0x4e, 0x89, 0xd3, 0xd2, 0xef, 0x0a, - 0x00, 0x4f, 0xb0, 0xcb, 0x7b, 0x03, 0xaa, 0xc0, 0xd4, 0x01, 0x2d, 0xf1, 0xbc, 0x40, 0xd5, 0x72, - 0x3d, 0xb6, 0x6a, 0xa1, 0x9e, 0x53, 0xc9, 0xfe, 0xe4, 0x74, 0x69, 0xe2, 0xe7, 0xa7, 0x4b, 0x82, - 0xcc, 0x39, 0xd1, 0x33, 0xc8, 0x1f, 0xe2, 0x13, 0x85, 0x8f, 0xcb, 0xf9, 0x14, 0xd5, 0xd1, 0xbb, - 0x21, 0x41, 0x87, 0x47, 0x65, 0x6f, 0x88, 0x96, 0x43, 0xc3, 0xb9, 0x4c, 0x38, 0xca, 0x4d, 0xd7, - 0xc6, 0xe6, 0xbe, 0x7b, 0x20, 0xc3, 0x21, 0x3e, 0xd9, 0x60, 0x32, 0xa4, 0x3f, 0x10, 0x20, 0x4f, - 0x4b, 0xc9, 0x94, 0x8a, 0xaa, 0x7d, 0xc5, 0xbc, 0x71, 0x6e, 0x0b, 0xc4, 0x94, 0xb3, 0x0c, 0x93, - 0x47, 0x6a, 0xbb, 0x87, 0x69, 0x09, 0xf3, 0x0f, 0xe7, 0x63, 0x64, 0x3c, 0x27, 0xe9, 0x32, 0xcb, - 0x86, 0x3e, 0x80, 0x19, 0xc3, 0x74, 0xb1, 0xe9, 0x2a, 0x8c, 0x2d, 0x7d, 0x0e, 0x5b, 0x9e, 0xe5, - 0xa6, 0x0f, 0xd2, 0x3f, 0x16, 0x00, 0xb6, 0x7b, 0x89, 0xea, 0xf9, 0x9b, 0x63, 0x96, 0xbf, 0x92, - 0x21, 0xac, 0x5e, 0x2d, 0xae, 0xc0, 0x94, 0x61, 0xb6, 0x0d, 0x93, 0x95, 0x3f, 0x2b, 0xf3, 0x27, - 0x74, 0x09, 0x26, 0x77, 0xdb, 0x86, 0xa9, 0xd3, 0xf1, 0x90, 0x95, 0xd9, 0x83, 0x24, 0x43, 0x9e, - 0x96, 0x3a, 0x41, 0xbd, 0x4b, 0xa7, 0x29, 0xb8, 0x5c, 0xb5, 0x4c, 0xdd, 0x20, 0x43, 0x52, 0x6d, - 0x7f, 0x2d, 0xb4, 0xb2, 0x06, 0x97, 0x74, 0xdc, 0xb5, 0xb1, 0xa6, 0xba, 0x58, 0x57, 0xf0, 0x71, - 0x77, 0xcc, 0x36, 0x46, 0x01, 0x57, 0xed, 0xb8, 0x4b, 0x69, 0x64, 0xd4, 0x12, 0x01, 0x6c, 0xd4, - 0x4e, 0x91, 0x29, 0x53, 0xce, 0xe2, 0xe3, 0x2e, 0x1d, 0xb5, 0xf1, 0x6a, 0x46, 0xdf, 0x84, 0xab, - 0x6a, 0xbb, 0x6d, 0xbd, 0x52, 0x8c, 0x3d, 0x45, 0xb7, 0xb0, 0xa3, 0x98, 0x96, 0xab, 0xe0, 0x63, - 0xc3, 0x71, 0xe9, 0x94, 0x90, 0x95, 0xe7, 0x68, 0x72, 0x63, 0x6f, 0xd5, 0xc2, 0xce, 0x96, 0xe5, - 0xd6, 0x48, 0x52, 0xa8, 0x29, 0xa7, 0xc3, 0x4d, 0x29, 0xfd, 0x1a, 0x5c, 0xe9, 0xd7, 0x6f, 0x92, - 0xed, 0xf7, 0x53, 0x01, 0x8a, 0x0d, 0xd3, 0x70, 0xbf, 0x16, 0x0d, 0xe7, 0xeb, 0x33, 0x1d, 0xd6, - 0xe7, 0x7d, 0x10, 0xf7, 0x54, 0xa3, 0xfd, 0xd4, 0x6c, 0x59, 0x9d, 0x5d, 0xc7, 0xb5, 0x4c, 0xec, - 0x70, 0x85, 0x0f, 0xd0, 0xa5, 0xe7, 0x50, 0xf2, 0x6b, 0x93, 0xa4, 0x9a, 0x5c, 0x10, 0x1b, 0xa6, - 0x66, 0xe3, 0x0e, 0x36, 0x13, 0xd5, 0xd3, 0x9b, 0x90, 0x33, 0x3c, 0xb9, 0x54, 0x57, 0x69, 0x39, - 0x20, 0x48, 0x3d, 0x98, 0x0d, 0xbd, 0x35, 0xc9, 0xe9, 0x92, 0x2c, 0x46, 0xf8, 0x95, 0x12, 0xb4, - 0x11, 0x59, 0x8c, 0xf0, 0x2b, 0x36, 0xbd, 0x35, 0xa1, 0xb0, 0x8a, 0xdb, 0xd8, 0xc5, 0x09, 0xd6, - 0x54, 0xda, 0x81, 0xa2, 0x27, 0x34, 0xc9, 0x86, 0xf9, 0x4d, 0x01, 0x10, 0x97, 0xab, 0x9a, 0xfb, - 0x49, 0x96, 0x18, 0x2d, 0x11, 0xd3, 0xc2, 0xed, 0xd9, 0x26, 0x5b, 0xce, 0x59, 0x9f, 0x04, 0x46, - 0xa2, 0x2b, 0x7a, 0x30, 0x64, 0x33, 0xe1, 0x21, 0xcb, 0xcd, 0x9b, 0x57, 0x30, 0x17, 0x29, 0x58, - 0xb2, 0xcd, 0x97, 0xa1, 0x65, 0x4a, 0x5d, 0x4f, 0x87, 0x6d, 0x38, 0x4a, 0x94, 0x3e, 0x17, 0x60, - 0xb6, 0xda, 0xc6, 0xaa, 0x9d, 0xb8, 0x46, 0xbe, 0x0b, 0x59, 0x1d, 0xab, 0x3a, 0xad, 0x32, 0x1b, - 0xd8, 0x6f, 0x85, 0xa4, 0x10, 0x4b, 0xb7, 0x7c, 0xd0, 0xd6, 0xca, 0x2d, 0xcf, 0x06, 0xe6, 0xa3, - 0xdb, 0x67, 0x92, 0x3e, 0x06, 0x14, 0x2e, 0x59, 0x92, 0x1d, 0xe1, 0xf7, 0x52, 0x80, 0x64, 0x7c, - 0x84, 0x6d, 0x37, 0xf1, 0x6a, 0xaf, 0x42, 0xde, 0x55, 0xed, 0x7d, 0xec, 0x2a, 0xc4, 0xba, 0xbf, - 0x48, 0xcd, 0x81, 0xf1, 0x11, 0x32, 0x6a, 0xc1, 0x1d, 0x6c, 0xaa, 0xbb, 0x6d, 0x4c, 0xa5, 0x28, - 0xbb, 0x56, 0xcf, 0xd4, 0x15, 0xc3, 0xc5, 0xb6, 0xea, 0x5a, 0xb6, 0x62, 0x75, 0x5d, 0xa3, 0x63, - 0x7c, 0x4a, 0x0d, 0x7b, 0xde, 0xd5, 0x6e, 0xb2, 0xec, 0x84, 0xb9, 0x42, 0x32, 0x37, 0x78, 0xde, - 0xa7, 0xa1, 0xac, 0xa8, 0x0c, 0x73, 0xc6, 0xbe, 0x69, 0xd9, 0x58, 0xd9, 0xd7, 0x14, 0xf7, 0xc0, - 0xc6, 0xce, 0x81, 0xd5, 0xf6, 0x16, 0xa4, 0x59, 0x96, 0xf4, 0x44, 0x6b, 0x79, 0x09, 0xd2, 0x27, - 0x30, 0x17, 0xd1, 0x52, 0x92, 0x4d, 0xf0, 0xdf, 0x05, 0xc8, 0x37, 0x35, 0xd5, 0x4c, 0x52, 0xf7, - 0x1f, 0x41, 0xde, 0xd1, 0x54, 0x53, 0xd9, 0xb3, 0xec, 0x8e, 0xea, 0xd2, 0x7a, 0x15, 0x23, 0xba, - 0xf7, 0xed, 0x7b, 0x4d, 0x35, 0x1f, 0xd3, 0x4c, 0x32, 0x38, 0xfe, 0xef, 0x7e, 0xfb, 0x75, 0xf2, - 0xcb, 0xdb, 0xaf, 0x6c, 0x78, 0xaf, 0x65, 0xb2, 0x69, 0x31, 0x23, 0xfd, 0xa9, 0x00, 0x33, 0xac, - 0xca, 0x49, 0x0e, 0xef, 0x6f, 0x41, 0xc6, 0xb6, 0x5e, 0xb1, 0xe1, 0x9d, 0x7f, 0x78, 0x2d, 0x46, - 0xc4, 0x3a, 0x3e, 0x09, 0xaf, 0x9f, 0x34, 0x3b, 0xaa, 0x00, 0xb7, 0x52, 0x15, 0xca, 0x9d, 0x1e, - 0x97, 0x1b, 0x18, 0x97, 0x4c, 0x64, 0xdc, 0x81, 0xd2, 0xae, 0xea, 0x6a, 0x07, 0x8a, 0xcd, 0x0b, - 0x49, 0xd6, 0xda, 0xf4, 0xdd, 0x19, 0xb9, 0x48, 0xc9, 0x5e, 0xd1, 0x1d, 0x52, 0x73, 0x36, 0xde, - 0x1c, 0xfc, 0xe7, 0xac, 0xcd, 0xff, 0xaf, 0xc0, 0xc7, 0x90, 0x57, 0xf3, 0x3f, 0x6f, 0x4d, 0xff, - 0xa3, 0x14, 0x5c, 0xad, 0x1e, 0x60, 0xed, 0xb0, 0x6a, 0x99, 0x8e, 0xe1, 0xb8, 0x44, 0x77, 0x49, - 0xb6, 0xff, 0x35, 0xc8, 0xbd, 0x32, 0xdc, 0x03, 0x45, 0x37, 0xf6, 0xf6, 0xe8, 0x6c, 0x9b, 0x95, - 0xb3, 0x84, 0xb0, 0x6a, 0xec, 0xed, 0xa1, 0x47, 0x90, 0xe9, 0x58, 0x3a, 0x33, 0xe6, 0x8b, 0x0f, - 0x97, 0x62, 0xc4, 0xd3, 0xa2, 0x39, 0xbd, 0xce, 0xa6, 0xa5, 0x63, 0x99, 0x66, 0x46, 0x8b, 0x00, - 0x1a, 0xa1, 0x76, 0x2d, 0xc3, 0x74, 0xf9, 0xe4, 0x18, 0xa2, 0xa0, 0x3a, 0xe4, 0x5c, 0x6c, 0x77, - 0x0c, 0x53, 0x75, 0xf1, 0xfc, 0x24, 0x55, 0xde, 0xdb, 0xb1, 0x05, 0xef, 0xb6, 0x0d, 0x4d, 0x5d, - 0xc5, 0x8e, 0x66, 0x1b, 0x5d, 0xd7, 0xb2, 0xb9, 0x16, 0x03, 0x66, 0xe9, 0xaf, 0x67, 0x60, 0x7e, - 0x50, 0x37, 0x49, 0xf6, 0x90, 0x6d, 0x98, 0xb2, 0xb1, 0xd3, 0x6b, 0xbb, 0xbc, 0x8f, 0x3c, 0x1c, - 0xa6, 0x82, 0x98, 0x12, 0xd0, 0xad, 0x8b, 0xb6, 0xcb, 0x8b, 0xcd, 0xe5, 0x2c, 0xfc, 0x73, 0x01, - 0xa6, 0x58, 0x02, 0x7a, 0x00, 0x59, 0x9b, 0x2c, 0x0c, 0x8a, 0xa1, 0xd3, 0x32, 0xa6, 0x2b, 0x57, - 0xce, 0x4e, 0x97, 0xa6, 0xe9, 0x62, 0xd1, 0x58, 0xfd, 0x22, 0xf8, 0x29, 0x4f, 0xd3, 0x7c, 0x0d, - 0x9d, 0xb4, 0x96, 0xe3, 0xaa, 0xb6, 0x4b, 0x37, 0x95, 0x52, 0x0c, 0x21, 0x51, 0xc2, 0x3a, 0x3e, - 0x41, 0x6b, 0x30, 0xe5, 0xb8, 0xaa, 0xdb, 0x73, 0x78, 0x7b, 0x5d, 0xa8, 0xb0, 0x4d, 0xca, 0x29, - 0x73, 0x09, 0xc4, 0xdc, 0xd2, 0xb1, 0xab, 0x1a, 0x6d, 0xda, 0x80, 0x39, 0x99, 0x3f, 0x49, 0xbf, - 0x25, 0xc0, 0x14, 0xcb, 0x8a, 0xae, 0xc2, 0x9c, 0xbc, 0xb2, 0xf5, 0xa4, 0xa6, 0x34, 0xb6, 0x56, - 0x6b, 0xad, 0x9a, 0xbc, 0xd9, 0xd8, 0x5a, 0x69, 0xd5, 0xc4, 0x09, 0x74, 0x05, 0x90, 0x97, 0x50, - 0x7d, 0xba, 0xd5, 0x6c, 0x34, 0x5b, 0xb5, 0xad, 0x96, 0x28, 0xd0, 0x3d, 0x15, 0x4a, 0x0f, 0x51, - 0x53, 0xe8, 0x6d, 0xb8, 0xde, 0x4f, 0x55, 0x9a, 0xad, 0x95, 0x56, 0x53, 0xa9, 0x35, 0x5b, 0x8d, - 0xcd, 0x95, 0x56, 0x6d, 0x55, 0x4c, 0x8f, 0xc8, 0x45, 0x5e, 0x22, 0xcb, 0xb5, 0x6a, 0x4b, 0xcc, - 0x48, 0x2e, 0x5c, 0x96, 0xb1, 0x66, 0x75, 0xba, 0x3d, 0x17, 0x93, 0x52, 0x3a, 0x49, 0x8e, 0x94, - 0xab, 0x30, 0xad, 0xdb, 0x27, 0x8a, 0xdd, 0x33, 0xf9, 0x38, 0x99, 0xd2, 0xed, 0x13, 0xb9, 0x67, - 0x4a, 0xff, 0x40, 0x80, 0x2b, 0xfd, 0xaf, 0x4d, 0xb2, 0x13, 0x3e, 0x83, 0xbc, 0xaa, 0xeb, 0x58, - 0x57, 0x74, 0xdc, 0x76, 0x55, 0x6e, 0x12, 0xdd, 0x0f, 0x49, 0xe2, 0x5b, 0x81, 0x65, 0x7f, 0x2b, - 0x70, 0xf3, 0x79, 0xb5, 0x4a, 0x0b, 0xb2, 0x4a, 0x38, 0xbc, 0xe9, 0x87, 0x0a, 0xa1, 0x14, 0xe9, - 0x47, 0x19, 0x28, 0xd4, 0x4c, 0xbd, 0x75, 0x9c, 0xe8, 0x5a, 0x72, 0x05, 0xa6, 0x34, 0xab, 0xd3, - 0x31, 0x5c, 0x4f, 0x41, 0xec, 0x09, 0xfd, 0x72, 0xc8, 0x94, 0x4d, 0x8f, 0x61, 0xd0, 0x05, 0x46, - 0x2c, 0xfa, 0x75, 0xb8, 0x4a, 0x66, 0x4d, 0xdb, 0x54, 0xdb, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xb1, - 0xbf, 0x8f, 0x6d, 0xbe, 0xfd, 0x78, 0x37, 0xa6, 0x9c, 0x0d, 0xce, 0x51, 0xa5, 0x0c, 0x2d, 0x96, - 0x5f, 0xbe, 0x6c, 0xc4, 0x91, 0xd1, 0x87, 0x00, 0x64, 0x29, 0xa2, 0x5b, 0x9a, 0x0e, 0x9f, 0x8f, - 0x86, 0xed, 0x69, 0x7a, 0x53, 0x10, 0x61, 0x20, 0xcf, 0x0e, 0x7a, 0x06, 0xa2, 0x61, 0x2a, 0x7b, - 0x6d, 0x63, 0xff, 0xc0, 0x55, 0x5e, 0xd9, 0x86, 0x8b, 0x9d, 0xf9, 0x59, 0x2a, 0x23, 0xae, 0xa9, - 0x9b, 0x7c, 0x6b, 0x56, 0x7f, 0x41, 0x72, 0x72, 0x69, 0x45, 0xc3, 0x7c, 0x4c, 0xf9, 0x29, 0xd1, - 0x41, 0xcb, 0x04, 0x0a, 0xbd, 0xec, 0x19, 0x36, 0x56, 0x1e, 0x74, 0x35, 0xba, 0x0f, 0x92, 0xad, - 0x14, 0xcf, 0x4e, 0x97, 0x40, 0x66, 0xe4, 0x07, 0xdb, 0x55, 0x02, 0x8d, 0xd8, 0xef, 0xae, 0x46, - 0xd4, 0xde, 0xb5, 0x0c, 0xc7, 0x32, 0xe7, 0x73, 0x4c, 0xed, 0xec, 0x09, 0xdd, 0x03, 0xd1, 0x3d, - 0x36, 0x95, 0x03, 0xac, 0xda, 0xee, 0x2e, 0x56, 0x5d, 0xb2, 0x3e, 0x03, 0xcd, 0x51, 0x72, 0x8f, - 0xcd, 0x7a, 0x88, 0xbc, 0x96, 0xc9, 0x4e, 0x8b, 0xd9, 0xb5, 0x4c, 0x36, 0x2b, 0xe6, 0xa4, 0x7f, - 0x2f, 0x40, 0xd1, 0xeb, 0x1b, 0x49, 0x76, 0xe3, 0xbb, 0x20, 0x5a, 0x26, 0x56, 0xba, 0x07, 0xaa, - 0x83, 0x79, 0x5b, 0xf2, 0xd5, 0xa1, 0x68, 0x99, 0x78, 0x9b, 0x90, 0x59, 0xcb, 0xa0, 0x6d, 0x98, - 0x75, 0x5c, 0x75, 0xdf, 0x30, 0xf7, 0x15, 0x7f, 0x8b, 0x9f, 0x5a, 0x16, 0x63, 0x22, 0x01, 0x91, - 0x73, 0xfb, 0xf4, 0x88, 0x49, 0xf1, 0x87, 0x02, 0xcc, 0xae, 0xe8, 0x1d, 0xc3, 0x6c, 0x76, 0xdb, - 0x46, 0xa2, 0x1b, 0x0c, 0x6f, 0x43, 0xce, 0x21, 0x32, 0x83, 0xd9, 0x39, 0x80, 0x8b, 0x59, 0x9a, - 0x42, 0xa6, 0xe9, 0x0d, 0x28, 0xe1, 0xe3, 0xae, 0xc1, 0xce, 0x15, 0x18, 0xca, 0xc9, 0x8c, 0x5f, - 0xb7, 0x62, 0xc0, 0x4b, 0x92, 0x78, 0x9d, 0x3e, 0x06, 0x14, 0xae, 0x52, 0x92, 0x40, 0xe3, 0x63, - 0x98, 0xa3, 0xa2, 0x77, 0x4c, 0x27, 0x61, 0x7d, 0x49, 0xbf, 0x0a, 0x97, 0xa2, 0xa2, 0x93, 0x2c, - 0xf7, 0x0b, 0xde, 0xca, 0x9b, 0xd8, 0x4e, 0x14, 0xa1, 0xfa, 0xba, 0xe6, 0x82, 0x93, 0x2c, 0xf3, - 0x6f, 0x08, 0xf0, 0x06, 0x95, 0x4d, 0x8f, 0x5e, 0xf6, 0xb0, 0xbd, 0x81, 0x55, 0x27, 0x51, 0x78, - 0x7d, 0x13, 0xa6, 0x18, 0x4c, 0xa6, 0xfd, 0x73, 0xb2, 0x92, 0x27, 0x66, 0x46, 0xd3, 0xb5, 0x6c, - 0x62, 0x66, 0xf0, 0x24, 0x49, 0x85, 0x85, 0xb8, 0x52, 0x24, 0x59, 0xd3, 0xbf, 0x2d, 0xc0, 0x2c, - 0xb7, 0xf0, 0x48, 0x57, 0xae, 0x1e, 0x10, 0x03, 0x07, 0xd5, 0x20, 0xaf, 0xd1, 0x5f, 0x8a, 0x7b, - 0xd2, 0xc5, 0x54, 0x7e, 0x71, 0x94, 0x71, 0xc8, 0xd8, 0x5a, 0x27, 0x5d, 0x4c, 0x2c, 0x4c, 0xef, - 0x37, 0x51, 0x54, 0xa8, 0x92, 0x23, 0xcd, 0x4b, 0x3a, 0x8e, 0x68, 0x5e, 0xcf, 0x4e, 0xe3, 0x3a, - 0xf8, 0x47, 0x69, 0xae, 0x04, 0xf6, 0x0e, 0x9e, 0x3d, 0x51, 0x83, 0xe2, 0x13, 0xb8, 0x12, 0xda, - 0x3a, 0x0f, 0x57, 0x3c, 0x75, 0x81, 0x8a, 0x87, 0xb6, 0xdf, 0x03, 0x2a, 0xfa, 0x18, 0x42, 0x1b, - 0xec, 0x0a, 0xab, 0x93, 0x07, 0x55, 0x2e, 0xa2, 0x8e, 0xd9, 0x40, 0x0a, 0xa3, 0x3b, 0xa8, 0x0a, - 0x59, 0x7c, 0xdc, 0x55, 0x74, 0xec, 0x68, 0x7c, 0xe2, 0x92, 0xe2, 0x04, 0x92, 0xa2, 0x0c, 0x18, - 0xef, 0xd3, 0xf8, 0xb8, 0x4b, 0x88, 0x68, 0x87, 0xac, 0x9b, 0xde, 0xba, 0x4e, 0x8b, 0xed, 0x9c, - 0x8f, 0x05, 0x82, 0x9e, 0xc2, 0xc5, 0x95, 0xfc, 0x25, 0x9d, 0x89, 0x90, 0x7e, 0x2c, 0xc0, 0xb5, - 0xd8, 0x56, 0x4b, 0x72, 0x21, 0xfb, 0x10, 0x32, 0xb4, 0xf2, 0xa9, 0x0b, 0x56, 0x9e, 0x72, 0x49, - 0x9f, 0xa5, 0xf8, 0x18, 0x97, 0x71, 0xdb, 0x22, 0x8a, 0x4d, 0x7c, 0x0b, 0xed, 0x29, 0x14, 0x8e, - 0x2c, 0x17, 0xdb, 0x7e, 0xb3, 0xa7, 0x2e, 0xdc, 0xec, 0x33, 0x54, 0x80, 0xd7, 0xe2, 0xcf, 0x61, - 0xd6, 0xb4, 0x4c, 0x25, 0x2a, 0xf4, 0xe2, 0x7d, 0xa9, 0x64, 0x5a, 0xe6, 0xf3, 0x90, 0x5c, 0x7f, - 0x9e, 0xe9, 0xd3, 0x44, 0x92, 0xf3, 0xcc, 0x0f, 0x05, 0x98, 0xf3, 0x2d, 0x9d, 0x84, 0xcd, 0xdd, - 0x6f, 0x41, 0xda, 0xb4, 0x5e, 0x5d, 0x64, 0x8b, 0x92, 0xe4, 0x27, 0xab, 0x5e, 0xb4, 0x44, 0x49, - 0xd6, 0xf7, 0x5f, 0xa4, 0x20, 0xf7, 0xa4, 0x9a, 0x64, 0x2d, 0x3f, 0xe4, 0xdb, 0xdf, 0xac, 0xbd, - 0xe3, 0x7a, 0xbb, 0xff, 0xbe, 0xf2, 0x93, 0xea, 0x3a, 0x3e, 0xf1, 0x7a, 0x3b, 0xe1, 0x42, 0x2b, - 0x90, 0x8b, 0x6e, 0x94, 0x8e, 0xa9, 0xa9, 0x80, 0x6b, 0x01, 0xc3, 0x24, 0x95, 0xeb, 0xb9, 0x5a, - 0x08, 0x31, 0xae, 0x16, 0xe4, 0x35, 0xbe, 0xa5, 0x98, 0xba, 0xc8, 0x6b, 0x42, 0x26, 0xe2, 0xa4, - 0x38, 0x25, 0x3d, 0x03, 0x20, 0xd5, 0x49, 0xb2, 0x49, 0x7e, 0x90, 0x86, 0xe2, 0x76, 0xcf, 0x39, - 0x48, 0xb8, 0xf7, 0x55, 0x01, 0xba, 0x3d, 0xe7, 0x80, 0x8c, 0xc8, 0x63, 0x93, 0xd7, 0xf9, 0x1c, - 0x2f, 0x0e, 0xaf, 0xd2, 0x8c, 0xaf, 0x75, 0x6c, 0xa2, 0x3a, 0x17, 0x82, 0x95, 0xc0, 0x15, 0xe4, - 0xe6, 0x28, 0x64, 0xd9, 0x3a, 0x36, 0x37, 0xb1, 0x0f, 0x29, 0x99, 0x24, 0x4c, 0x24, 0x7d, 0x08, - 0xd3, 0xe4, 0x41, 0x71, 0xad, 0x8b, 0x34, 0xf3, 0x14, 0xe1, 0x69, 0x59, 0xe8, 0x03, 0xc8, 0x31, - 0x6e, 0xb2, 0xfa, 0x4d, 0xd1, 0xd5, 0x2f, 0xae, 0x2e, 0x5c, 0x8d, 0x74, 0xdd, 0xcb, 0x52, 0x56, - 0xb2, 0xd6, 0x5d, 0x82, 0xc9, 0x3d, 0xcb, 0xd6, 0xbc, 0xc3, 0x5c, 0xf6, 0xc0, 0xda, 0x93, 0x41, - 0x9a, 0xb5, 0x4c, 0x36, 0x27, 0x82, 0xf4, 0x5b, 0x02, 0x94, 0xfc, 0x86, 0x48, 0x72, 0x41, 0xa8, - 0x46, 0xb4, 0x78, 0xf1, 0xa6, 0x20, 0x0a, 0x94, 0xfe, 0x35, 0xb5, 0x88, 0x34, 0xeb, 0x88, 0xb6, - 0x4c, 0x92, 0x3d, 0xe5, 0x03, 0xe6, 0xe8, 0x93, 0xba, 0x68, 0xeb, 0x52, 0x9f, 0x9f, 0x07, 0x70, - 0xc9, 0xe8, 0x90, 0xf9, 0xdc, 0x70, 0xdb, 0x27, 0x1c, 0xb6, 0xb9, 0xd8, 0x3b, 0x35, 0x9e, 0x0b, - 0xd2, 0xaa, 0x5e, 0x92, 0xf4, 0x7b, 0x74, 0xb7, 0x3a, 0xa8, 0x49, 0x92, 0xaa, 0x6e, 0x40, 0xc1, - 0x66, 0xa2, 0x89, 0x59, 0x73, 0x41, 0x6d, 0xcf, 0xf8, 0xac, 0x44, 0xe1, 0xbf, 0x9d, 0x82, 0xd2, - 0xb3, 0x1e, 0xb6, 0x4f, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0xa5, 0x57, 0xaa, 0xe1, 0x2a, 0x7b, 0x96, - 0xad, 0xf4, 0xba, 0xba, 0xea, 0x7a, 0xde, 0x26, 0x05, 0x42, 0x7e, 0x6c, 0xd9, 0x3b, 0x94, 0x88, - 0x30, 0xa0, 0x43, 0xd3, 0x7a, 0x65, 0x2a, 0x84, 0x4c, 0x81, 0xf2, 0xb1, 0xc9, 0xb7, 0x90, 0x2b, - 0xdf, 0xfe, 0x77, 0xa7, 0x4b, 0x8f, 0xc6, 0xf2, 0x21, 0xa3, 0xfe, 0x72, 0xbd, 0x9e, 0xa1, 0x97, - 0x77, 0x76, 0x1a, 0xab, 0xb2, 0x48, 0x45, 0xbe, 0x60, 0x12, 0x5b, 0xc7, 0xa6, 0x23, 0xfd, 0x9d, - 0x14, 0x88, 0x81, 0x8e, 0x92, 0x6c, 0xc8, 0x1a, 0xe4, 0x5f, 0xf6, 0xb0, 0x6d, 0xbc, 0x46, 0x33, - 0x02, 0x67, 0x24, 0xd3, 0xce, 0x7d, 0x98, 0x75, 0x8f, 0x4d, 0x85, 0x79, 0xf8, 0x31, 0xc7, 0x0f, - 0xcf, 0x61, 0xa1, 0xe4, 0x92, 0x32, 0x13, 0x3a, 0x75, 0xfa, 0x70, 0xd0, 0x27, 0x30, 0x13, 0xd1, - 0x56, 0xfa, 0xcb, 0x69, 0x2b, 0xff, 0x2a, 0xa4, 0xa8, 0x3f, 0x10, 0x00, 0x51, 0x45, 0x35, 0xd8, - 0x1e, 0xff, 0xd7, 0xa5, 0x3f, 0xdd, 0x05, 0x91, 0xfa, 0x63, 0x2a, 0xc6, 0x9e, 0xd2, 0x31, 0x1c, - 0xc7, 0x30, 0xf7, 0x79, 0x87, 0x2a, 0x52, 0x7a, 0x63, 0x6f, 0x93, 0x51, 0xa5, 0xbf, 0x04, 0x73, - 0x91, 0x0a, 0x24, 0xd9, 0xd8, 0x37, 0x60, 0x66, 0x8f, 0x1d, 0xc1, 0x52, 0xe1, 0x7c, 0x7b, 0x30, - 0x4f, 0x69, 0xec, 0x7d, 0xd2, 0x7f, 0x49, 0xc1, 0x25, 0x19, 0x3b, 0x56, 0xfb, 0x08, 0x27, 0xaf, - 0xc2, 0x3a, 0xf0, 0xb3, 0x17, 0xe5, 0xb5, 0x34, 0x99, 0x63, 0xcc, 0x6c, 0x99, 0x8b, 0xee, 0xb1, - 0xbf, 0x3d, 0xba, 0xc7, 0x0e, 0xee, 0xaa, 0xf3, 0x9d, 0xba, 0x4c, 0x64, 0xa7, 0xce, 0x82, 0x12, - 0x3b, 0x3d, 0xd6, 0x15, 0x07, 0xbf, 0x34, 0x7b, 0x1d, 0x0f, 0x0c, 0x95, 0x47, 0x15, 0xb2, 0xc1, - 0x58, 0x9a, 0xf8, 0xe5, 0x56, 0xaf, 0x43, 0x6d, 0xe7, 0xca, 0x15, 0x52, 0xde, 0xb3, 0xd3, 0xa5, - 0x62, 0x24, 0xcd, 0x91, 0x8b, 0x86, 0xff, 0x4c, 0xa4, 0x4b, 0xdf, 0x83, 0xcb, 0x7d, 0xca, 0x4e, - 0xd2, 0xe2, 0xf9, 0x67, 0x69, 0x78, 0x23, 0x2a, 0x3e, 0x69, 0x88, 0xf3, 0x75, 0x6f, 0xd0, 0x3a, - 0x14, 0x3a, 0x86, 0xf9, 0x7a, 0xbb, 0x97, 0x33, 0x1d, 0xc3, 0xf4, 0x69, 0x71, 0x5d, 0x63, 0xea, - 0x2b, 0xed, 0x1a, 0x2a, 0x2c, 0xc4, 0xb5, 0x5d, 0x92, 0xfd, 0xe3, 0x33, 0x01, 0x66, 0x92, 0xde, - 0x96, 0x7b, 0x3d, 0x2f, 0x38, 0xa9, 0x05, 0x85, 0xaf, 0x60, 0x1f, 0xef, 0xb7, 0x05, 0x40, 0x2d, - 0xbb, 0x67, 0x12, 0x50, 0xbb, 0x61, 0xed, 0x27, 0x59, 0xcd, 0x4b, 0x30, 0x69, 0x98, 0x3a, 0x3e, - 0xa6, 0xd5, 0xcc, 0xc8, 0xec, 0x21, 0x72, 0x94, 0x98, 0x1e, 0xeb, 0x28, 0x51, 0xfa, 0x04, 0xe6, - 0x22, 0x45, 0x4c, 0xb2, 0xfe, 0x7f, 0x92, 0x82, 0x39, 0x5e, 0x91, 0xc4, 0x77, 0x30, 0xbf, 0x09, - 0x93, 0x6d, 0x22, 0x73, 0x44, 0x3b, 0xd3, 0x77, 0x7a, 0xed, 0x4c, 0x33, 0xa3, 0xef, 0x00, 0x74, - 0x6d, 0x7c, 0xa4, 0x30, 0xd6, 0xf4, 0x58, 0xac, 0x39, 0xc2, 0x41, 0x09, 0xe8, 0x73, 0x01, 0x4a, - 0x64, 0x40, 0x77, 0x6d, 0xab, 0x6b, 0x39, 0xc4, 0x66, 0x71, 0xc6, 0x83, 0x39, 0xcf, 0xce, 0x4e, - 0x97, 0x0a, 0x9b, 0x86, 0xb9, 0xcd, 0x19, 0x5b, 0xcd, 0xb1, 0x1d, 0xfc, 0xbd, 0x30, 0x87, 0x72, - 0xb5, 0x6d, 0x69, 0x87, 0xc1, 0xe1, 0x18, 0x99, 0x59, 0x7c, 0x71, 0x8e, 0xf4, 0x33, 0x01, 0x2e, - 0x7d, 0x65, 0xdb, 0xc5, 0x7f, 0x16, 0xca, 0x96, 0x9e, 0x83, 0x48, 0x7f, 0x34, 0xcc, 0x3d, 0x2b, - 0xc9, 0x8d, 0xfb, 0xff, 0x23, 0xc0, 0x6c, 0x48, 0x70, 0x92, 0x06, 0xce, 0xeb, 0xea, 0xa9, 0xc0, - 0xdc, 0x61, 0xdc, 0xf1, 0x54, 0x25, 0xcf, 0xf0, 0xec, 0xac, 0x53, 0x96, 0x61, 0x06, 0x93, 0x59, - 0x8c, 0x6e, 0xf1, 0xee, 0xb2, 0x20, 0x93, 0xbe, 0x1d, 0xfd, 0xbc, 0x9f, 0xa1, 0x72, 0x22, 0xfd, - 0x2a, 0xb1, 0xb0, 0xc2, 0x83, 0x32, 0xc9, 0x21, 0xff, 0x4f, 0x53, 0x70, 0xa5, 0xca, 0x8e, 0xc0, - 0x3d, 0x9f, 0x90, 0x24, 0x3b, 0xe2, 0x3c, 0x4c, 0x1f, 0x61, 0xdb, 0x31, 0x2c, 0xb6, 0xda, 0x17, - 0x64, 0xef, 0x11, 0x2d, 0x40, 0xd6, 0x31, 0xd5, 0xae, 0x73, 0x60, 0x79, 0xc7, 0x89, 0xfe, 0xb3, - 0xef, 0xbf, 0x32, 0xf9, 0xfa, 0xfe, 0x2b, 0x53, 0xa3, 0xfd, 0x57, 0xa6, 0xbf, 0x84, 0xff, 0x0a, - 0x3f, 0xbb, 0xfb, 0x37, 0x02, 0x5c, 0x1d, 0xd0, 0x5c, 0x92, 0x9d, 0xf3, 0xfb, 0x90, 0xd7, 0xb8, - 0x60, 0xb2, 0x3e, 0xb0, 0x83, 0xc9, 0x06, 0xc9, 0xf6, 0x9a, 0xd0, 0xe7, 0xec, 0x74, 0x09, 0xbc, - 0xa2, 0x36, 0x56, 0xb9, 0x72, 0xc8, 0x6f, 0x5d, 0xfa, 0xaf, 0x00, 0xa5, 0xda, 0x31, 0xdb, 0x94, - 0x6f, 0x32, 0xab, 0x04, 0x3d, 0x86, 0x6c, 0xd7, 0xb6, 0x8e, 0x0c, 0xaf, 0x1a, 0xc5, 0x88, 0xf3, - 0x82, 0x57, 0x8d, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0xfb, 0xbc, 0xa8, 0x05, 0xb9, 0x0d, 0x4b, 0x53, - 0xdb, 0x8f, 0x8d, 0xb6, 0x37, 0xd0, 0xde, 0x3d, 0x5f, 0x50, 0xd9, 0xe7, 0xd9, 0x56, 0xdd, 0x03, - 0xaf, 0x11, 0x7c, 0x22, 0x6a, 0x40, 0xb6, 0xee, 0xba, 0x5d, 0x92, 0xc8, 0xc7, 0xdf, 0x9d, 0x31, - 0x84, 0x12, 0x16, 0xcf, 0xe3, 0xd6, 0x63, 0x47, 0x2d, 0x98, 0x7d, 0x42, 0xe3, 0xc7, 0xaa, 0x6d, - 0xab, 0xa7, 0x57, 0x2d, 0x73, 0xcf, 0xd8, 0xe7, 0xcb, 0xc4, 0xed, 0x31, 0x64, 0x3e, 0xa9, 0x36, - 0xe5, 0x41, 0x01, 0x68, 0x05, 0xb2, 0xcd, 0x47, 0x5c, 0x18, 0x33, 0x23, 0x6f, 0x8d, 0x21, 0xac, - 0xf9, 0x48, 0xf6, 0xd9, 0xd0, 0x1a, 0xe4, 0x57, 0x3e, 0xed, 0xd9, 0x98, 0x4b, 0x99, 0x1a, 0xea, - 0x39, 0xd1, 0x2f, 0x85, 0x72, 0xc9, 0x61, 0x66, 0xf4, 0x3d, 0x28, 0x11, 0xbd, 0xb5, 0xd4, 0xdd, - 0xb6, 0x27, 0x2f, 0x4b, 0xe5, 0x7d, 0x63, 0x0c, 0x79, 0x3e, 0xa7, 0x77, 0x24, 0xd0, 0x27, 0x6a, - 0x41, 0x86, 0x42, 0xa4, 0xbd, 0x10, 0x82, 0x4c, 0x97, 0x34, 0x8d, 0x40, 0xdd, 0x90, 0xe8, 0x6f, - 0xf4, 0x0e, 0x4c, 0x9b, 0x96, 0x8e, 0xbd, 0xce, 0x5c, 0xa8, 0x5c, 0x3a, 0x3b, 0x5d, 0x9a, 0xda, - 0xb2, 0x74, 0x66, 0xeb, 0xf0, 0x5f, 0xf2, 0x14, 0xc9, 0xd4, 0xd0, 0x17, 0xae, 0x43, 0x86, 0x34, - 0x11, 0x99, 0x43, 0x76, 0x55, 0x07, 0xef, 0xd8, 0x06, 0x97, 0xe6, 0x3d, 0x2e, 0xfc, 0xfd, 0x14, - 0xa4, 0x9a, 0x8f, 0x88, 0x35, 0xbf, 0xdb, 0xd3, 0x0e, 0xb1, 0xcb, 0xd3, 0xf9, 0x13, 0xb5, 0xf2, - 0x6d, 0xbc, 0x67, 0x30, 0xa3, 0x2b, 0x27, 0xf3, 0x27, 0xf4, 0x16, 0x80, 0xaa, 0x69, 0xd8, 0x71, - 0x14, 0x2f, 0x04, 0x30, 0x27, 0xe7, 0x18, 0x65, 0x1d, 0x9f, 0x10, 0x36, 0x07, 0x6b, 0x36, 0x76, - 0x3d, 0x1f, 0x2a, 0xf6, 0x44, 0xd8, 0x5c, 0xdc, 0xe9, 0x2a, 0xae, 0x75, 0x88, 0x4d, 0xda, 0xa4, - 0x39, 0x32, 0x2b, 0x74, 0xba, 0x2d, 0x42, 0x20, 0x13, 0x1a, 0x36, 0xf5, 0x60, 0xf6, 0xc9, 0xc9, - 0xfe, 0x33, 0x11, 0x69, 0xe3, 0x7d, 0x83, 0x07, 0xd0, 0xe5, 0x64, 0xfe, 0x44, 0xb4, 0xa4, 0xf6, - 0xdc, 0x03, 0xda, 0x12, 0x39, 0x99, 0xfe, 0x46, 0xb7, 0xa1, 0xc4, 0xdc, 0x2e, 0x15, 0x6c, 0x6a, - 0x0a, 0x9d, 0x07, 0x73, 0x34, 0xb9, 0xc0, 0xc8, 0x35, 0x53, 0x23, 0xb3, 0x1e, 0x7a, 0x04, 0x9c, - 0xa0, 0x1c, 0x76, 0x1c, 0xa2, 0x53, 0x20, 0xb9, 0x2a, 0xa5, 0xb3, 0xd3, 0xa5, 0x7c, 0x93, 0x26, - 0xac, 0x6f, 0x36, 0xc9, 0x5a, 0xc2, 0x72, 0xad, 0x77, 0x9c, 0x86, 0xbe, 0xf0, 0x37, 0x05, 0x48, - 0x3f, 0xa9, 0x36, 0x2f, 0xac, 0x32, 0xaf, 0xa0, 0xe9, 0x50, 0x41, 0xef, 0x40, 0x69, 0xd7, 0x68, - 0xb7, 0x0d, 0x73, 0x9f, 0xd8, 0x57, 0xdf, 0xc7, 0x9a, 0xa7, 0xb0, 0x22, 0x27, 0x6f, 0x33, 0x2a, - 0xba, 0x0e, 0x79, 0xcd, 0xc6, 0x3a, 0x36, 0x5d, 0x43, 0x6d, 0x3b, 0x5c, 0x73, 0x61, 0xd2, 0xc2, - 0x5f, 0x16, 0x60, 0x92, 0x76, 0x56, 0xf4, 0x26, 0xe4, 0x34, 0xcb, 0x74, 0x55, 0xc3, 0xe4, 0xb3, - 0x4e, 0x4e, 0x0e, 0x08, 0x43, 0x8b, 0x77, 0x03, 0x66, 0x54, 0x4d, 0xb3, 0x7a, 0xa6, 0xab, 0x98, - 0x6a, 0x07, 0xf3, 0x62, 0xe6, 0x39, 0x6d, 0x4b, 0xed, 0x60, 0xb4, 0x04, 0xde, 0xa3, 0x1f, 0xd9, - 0x99, 0x93, 0x81, 0x93, 0xd6, 0xf1, 0xc9, 0x02, 0x86, 0x9c, 0xdf, 0xab, 0x49, 0x7d, 0x7b, 0x8e, - 0x5f, 0x02, 0xfa, 0x1b, 0xbd, 0x0b, 0x97, 0x5e, 0xf6, 0xd4, 0xb6, 0xb1, 0x47, 0x37, 0xbf, 0xa8, - 0x97, 0x3a, 0x7d, 0x19, 0x2b, 0x0a, 0xf2, 0xd3, 0xa8, 0x04, 0xfa, 0x4e, 0x6f, 0x10, 0xa4, 0x83, - 0x41, 0xc0, 0x5c, 0x76, 0xa4, 0x13, 0x98, 0x95, 0xb1, 0x6b, 0x9f, 0xb4, 0x58, 0xb0, 0x6b, 0xed, - 0x08, 0x9b, 0x2e, 0xa9, 0xbb, 0xd5, 0xc5, 0xcc, 0x49, 0xc4, 0xab, 0xbb, 0x4f, 0x40, 0xb7, 0xa0, - 0xa8, 0xba, 0xa4, 0xbb, 0xb9, 0x8a, 0xd9, 0xeb, 0xec, 0x62, 0x9b, 0xb9, 0x02, 0xc8, 0x05, 0x4e, - 0xdd, 0xa2, 0x44, 0x1e, 0x91, 0x61, 0x9f, 0x28, 0x74, 0x9f, 0x88, 0xbf, 0x1a, 0x28, 0xa9, 0x46, - 0x28, 0xd2, 0x3d, 0xb8, 0x4c, 0xea, 0x59, 0x33, 0x35, 0xfb, 0xa4, 0x4b, 0x24, 0x3f, 0xa5, 0x7f, - 0x1d, 0x24, 0x86, 0xce, 0x69, 0xe8, 0xf1, 0x8c, 0xf4, 0xd3, 0x29, 0x28, 0xd4, 0x8e, 0xbb, 0x96, - 0x9d, 0xe8, 0xae, 0x4e, 0x05, 0xa6, 0x39, 0xf0, 0x1d, 0x71, 0x14, 0xdb, 0x37, 0x03, 0x79, 0xe7, - 0xd0, 0x9c, 0x11, 0x55, 0x00, 0x98, 0x43, 0x25, 0xf5, 0xc3, 0x49, 0x5f, 0xe0, 0xe4, 0x88, 0xb2, - 0xd1, 0x60, 0x83, 0x2d, 0xc8, 0x77, 0x8e, 0x34, 0x4d, 0xd9, 0x33, 0xda, 0x2e, 0xf7, 0x4b, 0x8b, - 0x77, 0xa1, 0xde, 0x7c, 0x5e, 0xad, 0x3e, 0xa6, 0x99, 0x98, 0x3f, 0x57, 0xf0, 0x2c, 0x03, 0x91, - 0xc0, 0x7e, 0xa3, 0x6f, 0x00, 0x0f, 0x7c, 0x51, 0x1c, 0x2f, 0x8c, 0xad, 0x52, 0x38, 0x3b, 0x5d, - 0xca, 0xc9, 0x94, 0xda, 0x6c, 0xb6, 0xe4, 0x1c, 0xcb, 0xd0, 0x74, 0xdc, 0x8b, 0x84, 0x3a, 0x4c, - 0x8f, 0x1f, 0xea, 0xf0, 0xd7, 0x04, 0xb8, 0xc2, 0x75, 0xa4, 0xec, 0x52, 0xf7, 0x6e, 0xb5, 0x6d, - 0xb8, 0x27, 0xca, 0xe1, 0xd1, 0x7c, 0x96, 0x9a, 0x3c, 0xbf, 0x1c, 0xab, 0xeb, 0x50, 0x13, 0x97, - 0x3d, 0x8d, 0x9f, 0x6c, 0x70, 0xe6, 0xf5, 0xa3, 0x9a, 0xe9, 0xda, 0x27, 0x95, 0xab, 0x67, 0xa7, - 0x4b, 0x73, 0x83, 0xa9, 0xcf, 0xe5, 0x39, 0x67, 0x90, 0x05, 0xd5, 0x01, 0xb0, 0xdf, 0xc5, 0xe8, - 0x0c, 0x16, 0xbf, 0x74, 0xc5, 0xf6, 0x45, 0x39, 0xc4, 0x8b, 0xee, 0x82, 0xc8, 0x43, 0x4b, 0xf6, - 0x8c, 0x36, 0x56, 0x1c, 0xe3, 0x53, 0x4c, 0xe7, 0xba, 0xb4, 0x5c, 0x64, 0x74, 0x22, 0xa2, 0x69, - 0x7c, 0x8a, 0xd1, 0x03, 0xb8, 0x1c, 0xb4, 0x80, 0xb2, 0x8b, 0xdb, 0xd6, 0x2b, 0x96, 0x3d, 0x4f, - 0xb3, 0x23, 0x5f, 0xfb, 0x15, 0x92, 0x44, 0x58, 0x16, 0xbe, 0x0f, 0xf3, 0xc3, 0x2a, 0x1c, 0x1e, - 0x10, 0x39, 0x76, 0x5e, 0xf9, 0x5e, 0x74, 0xb3, 0x62, 0x8c, 0x8e, 0xcb, 0x37, 0x2c, 0xde, 0x4f, - 0xbd, 0x27, 0x48, 0x7f, 0x2f, 0x05, 0x85, 0x4a, 0xaf, 0x7d, 0xf8, 0xb4, 0xdb, 0x64, 0x61, 0xf9, - 0xe8, 0x1a, 0xe4, 0x74, 0xd5, 0x55, 0x59, 0x21, 0x05, 0x16, 0x62, 0x46, 0x08, 0xb4, 0x36, 0x77, - 0xa0, 0x14, 0xf2, 0x05, 0xe1, 0x1e, 0xef, 0xb4, 0xda, 0x01, 0x99, 0x3a, 0xa5, 0xbf, 0x07, 0xf3, - 0xa1, 0x8c, 0x74, 0x67, 0x41, 0xc1, 0xa6, 0x6b, 0x1b, 0x98, 0xed, 0x8e, 0xa5, 0xe5, 0x90, 0xc3, - 0x4a, 0x83, 0x24, 0xd7, 0x58, 0x2a, 0x6a, 0xc1, 0x0c, 0xc9, 0x78, 0xa2, 0xd0, 0x59, 0xd0, 0xdb, - 0xbd, 0x7c, 0x10, 0x53, 0xad, 0x48, 0xb9, 0xcb, 0x54, 0x3f, 0x55, 0xca, 0x43, 0x7f, 0xca, 0x79, - 0x1c, 0x50, 0x16, 0x3e, 0x02, 0xb1, 0x3f, 0x43, 0x58, 0x97, 0x19, 0xa6, 0xcb, 0x4b, 0x61, 0x5d, - 0xa6, 0x43, 0x7a, 0x5a, 0xcb, 0x64, 0x33, 0xe2, 0xa4, 0xf4, 0xb3, 0x34, 0x14, 0xbd, 0x9e, 0x99, - 0xa4, 0x59, 0x5d, 0x81, 0x49, 0xd2, 0x8f, 0x3c, 0xf7, 0x8a, 0xdb, 0x23, 0x06, 0x04, 0xf7, 0xb1, - 0x26, 0xfd, 0xcb, 0x43, 0x80, 0x94, 0x35, 0x89, 0xe9, 0x67, 0xe1, 0xbf, 0x09, 0x90, 0xa1, 0x96, - 0xec, 0x03, 0xc8, 0xd0, 0xb8, 0x7c, 0x61, 0x64, 0x5c, 0xbe, 0x77, 0x3c, 0x4f, 0xb2, 0xfa, 0x0b, - 0x4b, 0x2a, 0x64, 0x5d, 0x55, 0xa8, 0x7f, 0x8f, 0x65, 0xbb, 0x58, 0xe7, 0x96, 0xe2, 0xf5, 0xf3, - 0xda, 0xd1, 0xb3, 0x84, 0x3d, 0x3e, 0xf4, 0x06, 0xa4, 0xc9, 0xdc, 0x35, 0xcd, 0x8e, 0xea, 0xcf, - 0x4e, 0x97, 0xd2, 0x64, 0xd6, 0x22, 0x34, 0xb4, 0x0c, 0xf9, 0xe8, 0x6c, 0x42, 0x8c, 0x0d, 0x3a, - 0x1d, 0x86, 0x66, 0x02, 0x68, 0xfb, 0x43, 0x88, 0xa1, 0x24, 0xd6, 0x96, 0xfc, 0x90, 0xfe, 0x37, - 0x04, 0xee, 0x93, 0xd8, 0xd4, 0xc8, 0x9a, 0x65, 0x27, 0xb9, 0xa8, 0xdc, 0x03, 0xd1, 0x56, 0x4d, - 0xdd, 0xea, 0x18, 0x9f, 0x62, 0x86, 0xca, 0x1d, 0x7e, 0x5c, 0x51, 0xf2, 0xe9, 0x14, 0x3e, 0x3b, - 0xd2, 0x7f, 0x16, 0xb8, 0xff, 0xa2, 0x5f, 0x8c, 0x64, 0x0f, 0x95, 0xf3, 0x7c, 0x4b, 0xcf, 0xdc, - 0xb3, 0x3c, 0xf7, 0x8b, 0x37, 0x87, 0x39, 0x1b, 0x35, 0xcc, 0x3d, 0xcb, 0x3b, 0x1e, 0xb3, 0x3d, - 0x82, 0xb3, 0xf0, 0x2b, 0x30, 0x49, 0x93, 0x5f, 0xa3, 0x6f, 0xf8, 0x3e, 0xb3, 0x29, 0x31, 0x2d, - 0xfd, 0x71, 0x0a, 0xde, 0xa6, 0x55, 0x7d, 0x8e, 0x6d, 0x63, 0xef, 0x64, 0xdb, 0xb6, 0x5c, 0xac, - 0xb9, 0x58, 0x0f, 0x76, 0xa5, 0x12, 0x6c, 0x02, 0x1d, 0x72, 0xfc, 0x3c, 0xcf, 0xd0, 0xf9, 0xcd, - 0x19, 0x4f, 0xbe, 0x1c, 0x5a, 0xcd, 0xb2, 0x73, 0xc0, 0xc6, 0xaa, 0x9c, 0x65, 0x92, 0x1b, 0x3a, - 0x5a, 0x81, 0x5c, 0xd7, 0xab, 0xc6, 0x85, 0x5c, 0x46, 0x7c, 0x2e, 0xb4, 0x0e, 0x25, 0x5e, 0x50, - 0xb5, 0x6d, 0x1c, 0x61, 0x45, 0x75, 0x2f, 0x32, 0x84, 0x0b, 0x8c, 0x77, 0x85, 0xb0, 0xae, 0xb8, - 0xd2, 0xdf, 0xc8, 0xc0, 0xad, 0x73, 0x54, 0x9c, 0x64, 0xf7, 0x5a, 0x80, 0xec, 0x11, 0x79, 0x91, - 0xc1, 0x6b, 0x9f, 0x95, 0xfd, 0x67, 0xb4, 0x1b, 0x59, 0x07, 0xf6, 0x54, 0xa3, 0x4d, 0xd6, 0x0d, - 0xe6, 0xa4, 0x37, 0xdc, 0x0d, 0x28, 0xde, 0xe9, 0x2d, 0xb4, 0x62, 0x3c, 0xa6, 0x82, 0x68, 0x36, - 0x07, 0x7d, 0x26, 0xc0, 0x02, 0x7b, 0x21, 0xf3, 0x14, 0xeb, 0x7b, 0x4d, 0x86, 0xbe, 0x66, 0x35, - 0xe6, 0x35, 0x63, 0xe9, 0xa8, 0x1c, 0x7a, 0x17, 0x2f, 0xc8, 0x7c, 0xf8, 0x6d, 0xe1, 0xa2, 0x2c, - 0xfc, 0xa6, 0x00, 0xf9, 0x10, 0x01, 0xdd, 0x1e, 0x88, 0xcb, 0xc9, 0x9f, 0xc5, 0x05, 0xe3, 0xdc, - 0x1a, 0x08, 0xc6, 0xa9, 0x64, 0xbf, 0x38, 0x5d, 0xca, 0xc8, 0xcc, 0xdf, 0xdb, 0x0b, 0xcb, 0xb9, - 0x11, 0x5c, 0x03, 0x93, 0xee, 0xcb, 0xe4, 0xdd, 0x03, 0x43, 0x61, 0x9d, 0xea, 0x1d, 0x23, 0x51, - 0x58, 0x47, 0x9e, 0xa4, 0x1f, 0xa5, 0x60, 0x76, 0x45, 0xd7, 0x9b, 0x4d, 0x0a, 0x05, 0x92, 0x1c, - 0x63, 0x08, 0x32, 0xc4, 0x3e, 0xe0, 0x31, 0x44, 0xf4, 0x37, 0x7a, 0x07, 0x90, 0x6e, 0x38, 0xec, - 0x3a, 0x05, 0xe7, 0x40, 0xd5, 0xad, 0x57, 0xc1, 0x69, 0xf1, 0xac, 0x97, 0xd2, 0xf4, 0x12, 0x50, - 0x13, 0xa8, 0xd1, 0xaa, 0x38, 0xae, 0xea, 0xef, 0x86, 0xdf, 0x1a, 0x2b, 0x2a, 0x85, 0x59, 0xb3, - 0xfe, 0xa3, 0x9c, 0x23, 0x72, 0xe8, 0x4f, 0x62, 0xa3, 0x19, 0xa4, 0x51, 0x5c, 0x45, 0x75, 0xbc, - 0x78, 0x0a, 0x76, 0x91, 0x43, 0x91, 0xd1, 0x57, 0x1c, 0x16, 0x26, 0xc1, 0xdc, 0xb0, 0x03, 0xd5, - 0x24, 0xb9, 0x97, 0xf9, 0x3b, 0x02, 0x14, 0x65, 0xbc, 0x67, 0x63, 0xe7, 0x20, 0x49, 0x9d, 0x3f, - 0x86, 0x19, 0x9b, 0x49, 0x55, 0xf6, 0x6c, 0xab, 0x73, 0x91, 0xb9, 0x22, 0xcf, 0x19, 0x1f, 0xdb, - 0x56, 0x87, 0x4f, 0xc9, 0xcf, 0xa1, 0xe4, 0x97, 0x31, 0xc9, 0xca, 0xff, 0x5d, 0x1a, 0x71, 0xc9, - 0x04, 0x27, 0x7d, 0x6c, 0x9b, 0xac, 0x06, 0xe8, 0x7e, 0x76, 0xb8, 0xa0, 0x49, 0xaa, 0xe1, 0x3f, - 0x09, 0x50, 0x6c, 0xf6, 0x76, 0xd9, 0x35, 0x41, 0xc9, 0x69, 0xa0, 0x06, 0xb9, 0x36, 0xde, 0x73, - 0x95, 0xd7, 0x72, 0x20, 0xce, 0x12, 0x56, 0xea, 0x3e, 0xfd, 0x04, 0xc0, 0xa6, 0x21, 0x47, 0x54, - 0x4e, 0xfa, 0x82, 0x72, 0x72, 0x94, 0x97, 0x90, 0xc9, 0xaa, 0x53, 0xf2, 0xab, 0x99, 0xe4, 0xfa, - 0xf2, 0x22, 0x32, 0x3b, 0xa4, 0x2f, 0x32, 0x3b, 0xcc, 0xf2, 0x93, 0xea, 0xf8, 0x19, 0xa2, 0x0c, - 0x73, 0xd4, 0x2c, 0x53, 0xd4, 0x6e, 0xb7, 0x6d, 0x78, 0x38, 0x85, 0xce, 0x3f, 0x19, 0x79, 0x96, - 0x26, 0xad, 0xb0, 0x14, 0x8a, 0x50, 0xd0, 0x0f, 0x04, 0x98, 0xd9, 0xb3, 0x31, 0xfe, 0x14, 0x2b, - 0x74, 0x4a, 0x1e, 0xef, 0x28, 0x7e, 0x95, 0x94, 0xe1, 0x4b, 0x1f, 0xd5, 0xe5, 0xd9, 0x8b, 0x9b, - 0xe4, 0xbd, 0x68, 0x0b, 0x44, 0xad, 0xcd, 0x0e, 0x0f, 0x7d, 0xb7, 0x80, 0xa9, 0xf1, 0x07, 0x40, - 0x89, 0x31, 0x07, 0x9e, 0x01, 0xcf, 0xc8, 0x60, 0x52, 0x75, 0x85, 0x5f, 0xcd, 0x46, 0x8d, 0xed, - 0xa8, 0x5b, 0x40, 0x38, 0xf4, 0x3a, 0x74, 0xa3, 0x5b, 0x59, 0xc6, 0xaa, 0xce, 0x2d, 0x77, 0x32, - 0xae, 0xfc, 0x07, 0x3e, 0xae, 0x5e, 0xc0, 0x2c, 0xed, 0x37, 0x49, 0x47, 0x50, 0x4a, 0xff, 0x30, - 0x0d, 0x28, 0x2c, 0xf9, 0xab, 0xeb, 0x6f, 0xa9, 0xe4, 0xfa, 0xdb, 0x1a, 0x48, 0x21, 0x63, 0xa8, - 0xad, 0x3a, 0xae, 0xc2, 0xfc, 0xcf, 0x1c, 0xa5, 0x8b, 0x6d, 0xc5, 0xc1, 0x9a, 0xc5, 0x2f, 0xd1, - 0x11, 0xe4, 0xc5, 0x20, 0xe7, 0x86, 0xea, 0xb8, 0xcf, 0x58, 0xbe, 0x6d, 0x6c, 0x37, 0x69, 0x2e, - 0xf4, 0x08, 0xae, 0x74, 0xd4, 0xe3, 0x38, 0xfe, 0x49, 0xca, 0x3f, 0xd7, 0x51, 0x8f, 0x07, 0x98, - 0xde, 0x87, 0x85, 0x78, 0x26, 0xc5, 0xc1, 0xde, 0xf9, 0xd4, 0x95, 0x18, 0xc6, 0x26, 0x76, 0xd1, - 0x0a, 0x40, 0x00, 0x22, 0xf8, 0x1a, 0x3d, 0x0e, 0x86, 0xc8, 0xf9, 0x18, 0x42, 0xfa, 0xa1, 0x00, - 0xc5, 0x4d, 0x63, 0xdf, 0x56, 0x13, 0xbd, 0xa2, 0x06, 0xbd, 0x1f, 0x3d, 0xd0, 0xcb, 0x3f, 0x5c, - 0x88, 0x73, 0xd8, 0x60, 0x39, 0xbc, 0x4d, 0x3b, 0xce, 0x40, 0x96, 0x3e, 0xbf, 0x44, 0x49, 0xce, - 0xf9, 0x1a, 0xbc, 0x45, 0x5d, 0xe0, 0xb8, 0xff, 0xcb, 0x57, 0x82, 0x6e, 0xa4, 0xdf, 0x17, 0x60, - 0x71, 0xd8, 0x5b, 0x92, 0x1c, 0x10, 0x32, 0xbd, 0xac, 0x8f, 0xbe, 0x41, 0xf1, 0x47, 0xc4, 0x39, - 0x33, 0x0d, 0xe2, 0x23, 0x01, 0xfc, 0xb2, 0x35, 0xe9, 0x35, 0x7e, 0xec, 0xb7, 0x23, 0xfd, 0xab, - 0x05, 0x98, 0xe1, 0xf5, 0xdb, 0x31, 0x0d, 0xcb, 0x44, 0x0f, 0x20, 0xbd, 0xcf, 0xf7, 0xfe, 0xf3, - 0xb1, 0x5b, 0x9e, 0xc1, 0x0d, 0x79, 0xf5, 0x09, 0x99, 0xe4, 0x25, 0x2c, 0xdd, 0x9e, 0x1b, 0x53, - 0x9e, 0xc0, 0xc9, 0x3b, 0xcc, 0xd2, 0xed, 0xb9, 0xa8, 0x09, 0x25, 0x2d, 0xb8, 0x96, 0x4b, 0x21, - 0xec, 0xe9, 0xa1, 0xfb, 0x80, 0xb1, 0x17, 0xa4, 0xd5, 0x27, 0xe4, 0xa2, 0x16, 0x49, 0x40, 0xd5, - 0xf0, 0x6d, 0x50, 0x99, 0x01, 0x0f, 0xb2, 0x20, 0x96, 0x38, 0x7a, 0x13, 0x55, 0x7d, 0x22, 0x74, - 0x69, 0x14, 0x7a, 0x1f, 0xa6, 0x74, 0x7a, 0xef, 0x10, 0x5f, 0x55, 0xe2, 0x3a, 0x44, 0xe4, 0x7a, - 0xa7, 0xfa, 0x84, 0xcc, 0x39, 0xd0, 0x1a, 0xcc, 0xb0, 0x5f, 0x0c, 0xf3, 0xf0, 0xb5, 0xe0, 0xd6, - 0x70, 0x09, 0x21, 0x6b, 0xac, 0x3e, 0x21, 0xe7, 0xf5, 0x80, 0x8a, 0x9e, 0x40, 0x5e, 0x6b, 0x63, - 0xd5, 0xe6, 0xa2, 0x6e, 0x0f, 0x0d, 0x7b, 0x1b, 0xb8, 0xab, 0xa8, 0x3e, 0x21, 0x83, 0xe6, 0x13, - 0x49, 0xa1, 0x6c, 0x7a, 0x65, 0x0d, 0x97, 0xf4, 0xee, 0xd0, 0x42, 0x0d, 0xde, 0xff, 0x53, 0xa7, - 0x56, 0x9a, 0x4f, 0x45, 0xdf, 0x84, 0x8c, 0xa3, 0xa9, 0x26, 0x5f, 0x98, 0x16, 0x87, 0xdc, 0x29, - 0x12, 0x30, 0xd3, 0xdc, 0xe8, 0x03, 0x06, 0x97, 0xdc, 0x63, 0x6f, 0xb3, 0x37, 0x4e, 0xa7, 0x91, - 0xd8, 0x75, 0xa2, 0x53, 0x4c, 0x09, 0x44, 0x0f, 0x2a, 0xc1, 0x87, 0x0a, 0x0d, 0x28, 0xa5, 0xbb, - 0xbb, 0xf1, 0x7a, 0x18, 0x08, 0x00, 0xae, 0xd3, 0x00, 0x79, 0x8f, 0x88, 0x36, 0xa1, 0xc0, 0x04, - 0xf5, 0x58, 0x6c, 0xea, 0xfc, 0xf2, 0xd0, 0x63, 0xdc, 0x98, 0xe8, 0xd8, 0xfa, 0x84, 0x3c, 0xa3, - 0x86, 0xc8, 0x41, 0xb9, 0x3a, 0xd8, 0xde, 0x67, 0xdb, 0xc8, 0x23, 0xca, 0x15, 0xf6, 0x8d, 0xf3, - 0xcb, 0x45, 0x89, 0xe8, 0xd7, 0xe1, 0x12, 0x13, 0xe4, 0x72, 0x97, 0x1f, 0xee, 0x39, 0xf2, 0xd6, - 0xd0, 0x23, 0xd8, 0xa1, 0xf1, 0xa4, 0xf5, 0x09, 0x19, 0xa9, 0x03, 0x89, 0x48, 0x83, 0xcb, 0xec, - 0x0d, 0x3c, 0x20, 0xd1, 0xe6, 0x31, 0x74, 0xf3, 0x37, 0xe9, 0x2b, 0xde, 0x19, 0xf6, 0x8a, 0xd8, - 0x38, 0xc9, 0xfa, 0x84, 0x3c, 0xa7, 0x0e, 0xa6, 0x06, 0xd5, 0xb0, 0x79, 0xe8, 0x17, 0xef, 0x6e, - 0xef, 0x8c, 0xae, 0x46, 0x5c, 0xc8, 0x9c, 0x5f, 0x8d, 0x48, 0x22, 0x69, 0x40, 0x3f, 0xf0, 0x9d, - 0x76, 0xa6, 0x99, 0xa1, 0x0d, 0x18, 0x13, 0x1f, 0x46, 0x1a, 0xf0, 0x20, 0x44, 0x46, 0x65, 0x48, - 0xed, 0x6b, 0xf3, 0x85, 0xa1, 0x0b, 0xa8, 0x1f, 0x03, 0x55, 0x9f, 0x90, 0x53, 0xfb, 0x1a, 0xfa, - 0x08, 0xb2, 0x2c, 0xa0, 0xe5, 0xd8, 0x9c, 0x2f, 0x0e, 0x9d, 0xc4, 0xa3, 0x61, 0x41, 0xf5, 0x09, - 0x99, 0xc6, 0xd0, 0xf0, 0x8e, 0xcc, 0x83, 0x15, 0xa8, 0x88, 0xf2, 0x88, 0x38, 0xd6, 0xbe, 0x90, - 0x11, 0xd2, 0x61, 0x6c, 0x9f, 0x88, 0xb6, 0xa1, 0xc8, 0x27, 0x70, 0xcf, 0xf9, 0x5a, 0x1c, 0xea, - 0xe4, 0x10, 0xe7, 0x7f, 0x5d, 0xa7, 0x1b, 0x55, 0x21, 0x3a, 0x69, 0xbb, 0xa8, 0x44, 0xde, 0x76, - 0xb3, 0x43, 0xdb, 0x6e, 0xa8, 0x2f, 0x30, 0x69, 0x3b, 0x7b, 0x20, 0x11, 0x7d, 0x1b, 0x26, 0xd9, - 0x38, 0x41, 0x54, 0x64, 0x9c, 0xdf, 0x4e, 0xdf, 0x10, 0x61, 0xf9, 0xc9, 0xec, 0xe5, 0x72, 0x9f, - 0x46, 0xa5, 0x6d, 0xed, 0xcf, 0xcf, 0x0d, 0x9d, 0xbd, 0x06, 0xbd, 0x33, 0xc9, 0xec, 0xe5, 0x06, - 0x54, 0xd2, 0x81, 0x6c, 0x96, 0xc2, 0x87, 0xd8, 0xa5, 0xa1, 0x1d, 0x28, 0xc6, 0xd5, 0xb1, 0x4e, - 0xa3, 0x4d, 0x02, 0xb2, 0x3f, 0xb1, 0x3a, 0x58, 0xa1, 0x93, 0xe2, 0xe5, 0xd1, 0x13, 0x6b, 0xe4, - 0xa2, 0x27, 0x7f, 0x62, 0x65, 0x54, 0xf4, 0x1c, 0x44, 0x7e, 0xdb, 0x88, 0xe2, 0xb9, 0xde, 0xcc, - 0x5f, 0xa1, 0xf2, 0xee, 0xc5, 0x2e, 0x88, 0x71, 0x5e, 0x59, 0x75, 0x82, 0x28, 0xa2, 0x29, 0xe8, - 0x63, 0x98, 0xa5, 0xf2, 0x14, 0x2d, 0xb8, 0x20, 0x66, 0x7e, 0x7e, 0xe0, 0xba, 0x91, 0xe1, 0x77, - 0xc9, 0x78, 0x92, 0x45, 0xad, 0x2f, 0x89, 0x8c, 0x07, 0xc3, 0x34, 0x5c, 0xba, 0x76, 0x2f, 0x0c, - 0x1d, 0x0f, 0xd1, 0xcb, 0x31, 0xc9, 0x78, 0x30, 0x18, 0x85, 0x74, 0xe3, 0xbe, 0x19, 0xef, 0xcd, - 0xa1, 0xdd, 0x78, 0xc8, 0x64, 0x57, 0x70, 0x23, 0xf3, 0xdc, 0x2a, 0x00, 0xc3, 0x91, 0xd4, 0x34, - 0x5e, 0x1c, 0x6a, 0x00, 0xf4, 0xbb, 0x22, 0x12, 0x03, 0xa0, 0xed, 0xd1, 0x88, 0x01, 0xc0, 0x0e, - 0x3d, 0xe6, 0xaf, 0x0f, 0x5f, 0xac, 0xc2, 0xc7, 0xa2, 0x74, 0xb1, 0xa2, 0x04, 0xb4, 0x02, 0x39, - 0x62, 0xd4, 0x9f, 0xd0, 0x11, 0x7e, 0x63, 0x28, 0x86, 0xef, 0x8b, 0x51, 0xaa, 0x4f, 0xc8, 0xd9, - 0x97, 0x9c, 0x44, 0x7a, 0x15, 0x13, 0xc1, 0xc7, 0xf6, 0xfd, 0xa1, 0xbd, 0x6a, 0x30, 0x38, 0x85, - 0xf4, 0xaa, 0x97, 0x01, 0x35, 0x58, 0xf2, 0x1c, 0x76, 0x9c, 0x31, 0xff, 0xf6, 0xe8, 0x25, 0x2f, - 0x7a, 0xf8, 0xe2, 0x2f, 0x79, 0x9c, 0xcc, 0x96, 0x3c, 0x5d, 0x71, 0x1c, 0xea, 0xe1, 0x30, 0x7f, - 0x6b, 0xc4, 0x92, 0xd7, 0xb7, 0xc1, 0xc9, 0x96, 0x3c, 0xbd, 0xc9, 0x38, 0x89, 0xf5, 0x67, 0x7b, - 0xb7, 0xeb, 0x70, 0x78, 0x77, 0x67, 0xa8, 0xf5, 0x17, 0x7b, 0xfd, 0x0f, 0xb1, 0xfe, 0xec, 0x48, - 0x02, 0xfa, 0x0e, 0x4c, 0xf3, 0x0d, 0xa5, 0xf9, 0xbb, 0x23, 0x6c, 0xec, 0xf0, 0x1e, 0x20, 0xe9, - 0x8e, 0x9c, 0x87, 0x4d, 0x0e, 0x6c, 0x23, 0x8b, 0x4d, 0x7e, 0xf7, 0x46, 0x4c, 0x0e, 0x03, 0x7b, - 0x69, 0x6c, 0x72, 0x08, 0xc8, 0xa4, 0x34, 0x0e, 0xdb, 0x84, 0x99, 0xff, 0xa5, 0xa1, 0xa5, 0x89, - 0xee, 0x46, 0x91, 0xd2, 0x70, 0x1e, 0xba, 0x58, 0xd0, 0xb5, 0x9a, 0x69, 0xe7, 0x1b, 0xc3, 0x17, - 0x8b, 0x7e, 0x58, 0x5f, 0xf7, 0x8e, 0x8b, 0x98, 0x56, 0xfe, 0x8a, 0x00, 0xd7, 0x59, 0x1f, 0xa0, - 0x9b, 0xe5, 0x27, 0x8a, 0x7f, 0xd6, 0x11, 0xda, 0xb3, 0x78, 0x40, 0xc5, 0x7f, 0xfb, 0xe2, 0x5b, - 0xf3, 0xde, 0x1b, 0xdf, 0x52, 0x47, 0xe5, 0x23, 0xca, 0xe8, 0x30, 0x74, 0x37, 0xff, 0x70, 0xa8, - 0x32, 0xa2, 0x88, 0x94, 0x28, 0x83, 0xf3, 0xa0, 0x36, 0xcc, 0xb3, 0x21, 0x11, 0xa0, 0x1f, 0xbf, - 0xe8, 0x8f, 0x86, 0x3a, 0x0d, 0x8e, 0xc4, 0x7d, 0xf5, 0x09, 0xf9, 0xca, 0xcb, 0xd8, 0x0c, 0x95, - 0x69, 0x7e, 0xf8, 0xec, 0x07, 0x9a, 0x96, 0x44, 0x71, 0x2d, 0x93, 0xbd, 0x2a, 0xce, 0xaf, 0x65, - 0xb2, 0x6f, 0x88, 0x0b, 0x6b, 0x99, 0xec, 0x35, 0xf1, 0xcd, 0xb5, 0x4c, 0x76, 0x49, 0xbc, 0xbe, - 0x96, 0xc9, 0x4a, 0xe2, 0x4d, 0xe9, 0x77, 0x16, 0xa0, 0xe0, 0xa1, 0x37, 0x86, 0xa2, 0x1e, 0x86, - 0x51, 0xd4, 0xe2, 0x30, 0x14, 0xc5, 0xf1, 0x1e, 0x87, 0x51, 0x0f, 0xc3, 0x30, 0x6a, 0x71, 0x18, - 0x8c, 0x0a, 0x78, 0x08, 0x8e, 0x6a, 0x0d, 0xc3, 0x51, 0xf7, 0xc6, 0xc0, 0x51, 0xbe, 0xa8, 0x7e, - 0x20, 0xb5, 0x3a, 0x08, 0xa4, 0xde, 0x1e, 0x0d, 0xa4, 0x7c, 0x51, 0x21, 0x24, 0xf5, 0x41, 0x1f, - 0x92, 0xba, 0x31, 0x02, 0x49, 0xf9, 0xfc, 0x1e, 0x94, 0x5a, 0x8f, 0x85, 0x52, 0xb7, 0xcf, 0x83, - 0x52, 0xbe, 0x9c, 0x08, 0x96, 0xaa, 0xc7, 0x61, 0xa9, 0x5b, 0xe7, 0x60, 0x29, 0x5f, 0x54, 0x18, - 0x4c, 0xad, 0xc7, 0x82, 0xa9, 0xdb, 0xe7, 0x81, 0xa9, 0xa0, 0x58, 0x61, 0x34, 0xf5, 0xad, 0x08, - 0x9a, 0x5a, 0x1a, 0x8a, 0xa6, 0x7c, 0x6e, 0x06, 0xa7, 0x3e, 0xec, 0x87, 0x53, 0x37, 0x46, 0xc0, - 0xa9, 0x40, 0xb1, 0x1c, 0x4f, 0xd5, 0xe3, 0xf0, 0xd4, 0xad, 0x73, 0xf0, 0x54, 0xa0, 0x8b, 0x10, - 0xa0, 0xda, 0x8a, 0x07, 0x54, 0x77, 0xce, 0x05, 0x54, 0xbe, 0xb4, 0x28, 0xa2, 0xaa, 0xc7, 0x21, - 0xaa, 0x5b, 0xe7, 0x20, 0xaa, 0xbe, 0x92, 0x31, 0x48, 0xa5, 0x8e, 0x84, 0x54, 0xef, 0x8c, 0x09, - 0xa9, 0x7c, 0xd1, 0x71, 0x98, 0x4a, 0x1f, 0x8d, 0xa9, 0xca, 0xe3, 0x62, 0x2a, 0xff, 0x25, 0xb1, - 0xa0, 0x4a, 0x1d, 0x09, 0xaa, 0xde, 0x19, 0x13, 0x54, 0xf5, 0x55, 0x24, 0x8a, 0xaa, 0xb6, 0xe2, - 0x51, 0xd5, 0x9d, 0x73, 0x51, 0x55, 0xd0, 0x8a, 0x11, 0x58, 0xb5, 0x1c, 0x82, 0x55, 0x6f, 0x0d, - 0x81, 0x55, 0x3e, 0x2b, 0xc1, 0x55, 0xdf, 0x1d, 0xc0, 0x55, 0xd2, 0x28, 0x5c, 0xe5, 0xf3, 0xfa, - 0xc0, 0xaa, 0x1e, 0x07, 0xac, 0x6e, 0x9d, 0x03, 0xac, 0x82, 0x7e, 0x13, 0x42, 0x56, 0xcf, 0x86, - 0x20, 0xab, 0xbb, 0xe7, 0x23, 0x2b, 0x5f, 0x5e, 0x1f, 0xb4, 0x52, 0x47, 0x42, 0xab, 0x77, 0xc6, - 0x84, 0x56, 0x41, 0x0b, 0xc6, 0x60, 0xab, 0xf7, 0xa2, 0xd8, 0xea, 0xfa, 0x70, 0x6c, 0xe5, 0x8b, - 0xe1, 0xe0, 0x6a, 0x3d, 0x16, 0x5c, 0xdd, 0x3e, 0x0f, 0x5c, 0x05, 0xb3, 0x59, 0x18, 0x5d, 0x6d, - 0xc5, 0xa3, 0xab, 0x3b, 0xe7, 0xa2, 0xab, 0xa0, 0x23, 0x45, 0xe0, 0xd5, 0x7a, 0x2c, 0xbc, 0xba, - 0x7d, 0x1e, 0xbc, 0xea, 0x9b, 0x6a, 0x39, 0xbe, 0x7a, 0x31, 0x14, 0x5f, 0xdd, 0x1f, 0x07, 0x5f, - 0xf9, 0x42, 0x07, 0x00, 0xd6, 0x27, 0xc3, 0x01, 0xd6, 0x2f, 0x5d, 0xe0, 0xb2, 0xce, 0x58, 0x84, - 0xf5, 0xdd, 0x01, 0x84, 0x25, 0x8d, 0x42, 0x58, 0xc1, 0xc8, 0xf0, 0x20, 0x56, 0x2d, 0x06, 0x10, - 0xbd, 0x3d, 0x1a, 0x10, 0x05, 0x0b, 0x79, 0x80, 0x88, 0x3e, 0xe8, 0x43, 0x44, 0x37, 0xce, 0xf5, - 0x8b, 0x0b, 0x41, 0xa2, 0xca, 0x20, 0x24, 0xba, 0x39, 0x12, 0x12, 0xf9, 0x12, 0x02, 0x4c, 0xb4, - 0x1e, 0x8b, 0x89, 0x6e, 0x9f, 0x87, 0x89, 0x82, 0xae, 0x10, 0x06, 0x45, 0x5b, 0xf1, 0xa0, 0xe8, - 0xce, 0xb9, 0xa0, 0xa8, 0x6f, 0xd9, 0xf2, 0x50, 0x51, 0x3d, 0x0e, 0x15, 0xdd, 0x3a, 0x07, 0x15, - 0x85, 0x97, 0x2d, 0x1f, 0x16, 0xb5, 0x86, 0xc1, 0xa2, 0x7b, 0x63, 0xc0, 0xa2, 0xc0, 0x98, 0xeb, - 0xc3, 0x45, 0x1f, 0xf5, 0xe3, 0x22, 0x69, 0x14, 0x2e, 0x0a, 0x3a, 0x91, 0x07, 0x8c, 0xb6, 0xe2, - 0x81, 0xd1, 0x9d, 0x73, 0x81, 0x51, 0x78, 0x5c, 0x87, 0x90, 0xd1, 0x47, 0xfd, 0xc8, 0x48, 0x1a, - 0x85, 0x8c, 0x82, 0xf2, 0x78, 0xd0, 0xa8, 0x1e, 0x07, 0x8d, 0x6e, 0x9d, 0x03, 0x8d, 0x42, 0xd3, - 0x7d, 0x80, 0x8d, 0xfe, 0xea, 0xf8, 0xd8, 0xe8, 0xbd, 0xd7, 0x75, 0x5b, 0x3a, 0x1f, 0x1c, 0x7d, - 0xd4, 0x0f, 0x8e, 0xa4, 0x51, 0xe0, 0x28, 0xd0, 0x87, 0x87, 0x8e, 0x3a, 0xe7, 0xa2, 0xa3, 0x07, - 0x17, 0x40, 0x47, 0xbe, 0xfc, 0x2f, 0x0d, 0x8f, 0xde, 0x14, 0xdf, 0x8a, 0x80, 0xa4, 0xff, 0x39, - 0x0d, 0x53, 0xfc, 0x3b, 0x55, 0x91, 0xbb, 0x9d, 0x84, 0xd7, 0xb9, 0xdb, 0x09, 0x7d, 0x07, 0xae, - 0xf9, 0x0f, 0xd4, 0xf3, 0x44, 0xe1, 0x11, 0x2f, 0x5a, 0xdb, 0xd2, 0x0e, 0xe9, 0xda, 0x93, 0x95, - 0xe7, 0xfd, 0x2c, 0x8f, 0x6d, 0xab, 0xc3, 0x22, 0x5f, 0xe8, 0x79, 0x3e, 0x5a, 0x25, 0x83, 0x82, - 0x1a, 0x59, 0xe7, 0x5f, 0x28, 0x38, 0x78, 0xe5, 0x1d, 0x67, 0x7d, 0x8d, 0x18, 0x6d, 0xf4, 0x2d, - 0x28, 0xf4, 0x1c, 0x6c, 0x2b, 0x5d, 0xdb, 0xb0, 0x6c, 0xc3, 0x65, 0xd1, 0x26, 0x42, 0x45, 0xfc, - 0xe2, 0x74, 0x69, 0x66, 0xc7, 0xc1, 0xf6, 0x36, 0xa7, 0xcb, 0x33, 0xbd, 0xd0, 0x93, 0xf7, 0x65, - 0xaf, 0xc9, 0xf1, 0xbf, 0xec, 0xf5, 0x0c, 0x44, 0xea, 0x4b, 0x10, 0x5e, 0x97, 0xd8, 0x35, 0x4c, - 0xf1, 0x4b, 0xa8, 0xaa, 0x87, 0x96, 0x1e, 0x7a, 0x1d, 0x53, 0xc9, 0x8e, 0x12, 0xd1, 0x13, 0x28, - 0xda, 0x56, 0x8f, 0xde, 0xb6, 0xd2, 0xb5, 0xda, 0x86, 0x76, 0x42, 0x0d, 0x86, 0x62, 0xfc, 0x99, - 0x29, 0xcb, 0xb8, 0x4d, 0xf3, 0xc9, 0x05, 0x3b, 0xfc, 0x88, 0x9a, 0x40, 0x6f, 0x5a, 0xf1, 0xa4, - 0xa0, 0x81, 0xbb, 0xad, 0x47, 0xde, 0x30, 0xff, 0x42, 0x35, 0x5c, 0x2e, 0x17, 0x5e, 0xf9, 0xbf, - 0xd1, 0x03, 0xb8, 0xdc, 0x51, 0x8f, 0xe9, 0xa5, 0xbf, 0x8a, 0x67, 0x88, 0xd0, 0x6b, 0xce, 0xd8, - 0xc7, 0xc2, 0x50, 0x47, 0x3d, 0xa6, 0x1f, 0x31, 0x63, 0x49, 0xf4, 0x0b, 0x24, 0x37, 0x60, 0x86, - 0x87, 0x0f, 0xb0, 0x0f, 0x14, 0x95, 0x68, 0x4e, 0xfe, 0xb5, 0x0a, 0xf6, 0x8d, 0xa2, 0x5b, 0x50, - 0xd4, 0x0d, 0xc7, 0x35, 0x4c, 0xcd, 0xe5, 0xf7, 0x09, 0xb3, 0x1b, 0x79, 0x0b, 0x1e, 0x95, 0x5d, - 0x1a, 0xdc, 0x82, 0x59, 0xad, 0x6d, 0xf8, 0xe6, 0x1d, 0x5b, 0x70, 0x67, 0x87, 0x0e, 0xe7, 0x2a, - 0xcd, 0xdb, 0x7f, 0x44, 0x5f, 0xd2, 0xa2, 0x64, 0x54, 0x85, 0xd2, 0xbe, 0xea, 0xe2, 0x57, 0xea, - 0x89, 0xe2, 0x45, 0xc7, 0xe5, 0x69, 0x44, 0xf0, 0xb5, 0xb3, 0xd3, 0xa5, 0xc2, 0x13, 0x96, 0x34, - 0x10, 0x24, 0x57, 0xd8, 0x0f, 0x25, 0xe8, 0xe8, 0x0e, 0x94, 0x54, 0xe7, 0xc4, 0xd4, 0x68, 0x4f, - 0xc0, 0xa6, 0xd3, 0x73, 0xa8, 0x75, 0x9e, 0x95, 0x8b, 0x94, 0x5c, 0xf5, 0xa8, 0xe8, 0x03, 0x58, - 0xe0, 0x9f, 0x0d, 0x78, 0xa5, 0xda, 0xba, 0x42, 0x7b, 0x4f, 0x30, 0x4c, 0x45, 0xca, 0x73, 0x95, - 0x7d, 0x26, 0x80, 0x64, 0x20, 0x5d, 0x26, 0x7c, 0x1d, 0x2f, 0xbb, 0x6e, 0x18, 0xc4, 0xfc, 0x5a, - 0x26, 0x3b, 0x23, 0x16, 0xd6, 0x32, 0xd9, 0xa2, 0x58, 0x92, 0x7e, 0x90, 0x86, 0x12, 0x99, 0x28, - 0x1d, 0xc7, 0xb0, 0xcc, 0xba, 0xef, 0xa8, 0xea, 0x77, 0x7f, 0x81, 0x06, 0x2a, 0xf9, 0xcf, 0x68, - 0x89, 0x06, 0x84, 0x11, 0x83, 0xd4, 0xff, 0x58, 0x48, 0x5a, 0x06, 0x46, 0xa2, 0xa1, 0x39, 0x2b, - 0x30, 0xe5, 0x58, 0x3d, 0x5b, 0xf3, 0xae, 0xb0, 0xbf, 0x37, 0x64, 0x66, 0x0e, 0xbd, 0xb0, 0xdc, - 0xa4, 0x0c, 0x32, 0x67, 0x44, 0x9f, 0x40, 0x89, 0xfd, 0xa2, 0x41, 0x30, 0x34, 0x08, 0x85, 0x45, - 0xf8, 0x3c, 0x18, 0x5b, 0xd6, 0x06, 0x67, 0x94, 0x8b, 0x4e, 0xe4, 0x19, 0x7d, 0x04, 0x6f, 0x9a, - 0x96, 0xd2, 0xc1, 0x1d, 0x8b, 0xcd, 0xc8, 0xa4, 0x0f, 0xeb, 0x8a, 0xea, 0x2a, 0xbc, 0xd0, 0xcc, - 0xf3, 0x71, 0xde, 0xb4, 0x36, 0x69, 0x16, 0x99, 0xe7, 0x58, 0x71, 0x99, 0x5c, 0xa9, 0x0c, 0x53, - 0xec, 0x17, 0xca, 0xc1, 0xe4, 0xd3, 0x56, 0xbd, 0x26, 0x8b, 0x13, 0x68, 0x06, 0xb2, 0x8f, 0xe5, - 0xa7, 0x9b, 0x4a, 0xf3, 0xd9, 0x86, 0x28, 0xa0, 0x3c, 0x4c, 0xcb, 0x4f, 0x9f, 0xb6, 0x94, 0xf5, - 0xe7, 0x62, 0x4a, 0xba, 0x03, 0xc5, 0x68, 0x89, 0x10, 0xc0, 0x94, 0x5c, 0xdb, 0x7c, 0x4a, 0xef, - 0x6d, 0xcf, 0xc1, 0xe4, 0xc6, 0xd3, 0xea, 0xca, 0x86, 0x28, 0x48, 0x7f, 0x22, 0xc0, 0x4c, 0x85, - 0x7d, 0x88, 0x80, 0x79, 0x3e, 0x7c, 0xd0, 0xe7, 0x92, 0xf0, 0x46, 0x3c, 0xe2, 0x8b, 0x77, 0x45, - 0x58, 0x81, 0x2c, 0x1f, 0x63, 0x5e, 0xb0, 0xc4, 0xd2, 0x70, 0x3b, 0x9f, 0x6e, 0x89, 0x79, 0x0e, - 0x6f, 0x1e, 0x1b, 0x6a, 0x82, 0xa8, 0x7a, 0xba, 0x55, 0x78, 0x49, 0x86, 0xbb, 0xbd, 0xf5, 0x35, - 0x83, 0x37, 0x62, 0xd4, 0x28, 0xf9, 0xfd, 0xcc, 0xe7, 0x3f, 0x5e, 0x9a, 0x90, 0xfe, 0x34, 0x03, - 0x85, 0x4a, 0xf8, 0xa3, 0x0b, 0xa8, 0xd1, 0x57, 0xd9, 0x38, 0xeb, 0x25, 0xc2, 0x51, 0x1e, 0xf1, - 0x39, 0x9b, 0x5c, 0xf0, 0x85, 0x07, 0x56, 0xf7, 0xeb, 0x23, 0xbc, 0x39, 0xc2, 0x95, 0x0f, 0x18, - 0x17, 0xfe, 0x6d, 0xda, 0x5f, 0x0d, 0xcb, 0x30, 0xc9, 0x02, 0xf2, 0x84, 0x81, 0xbb, 0x02, 0xe8, - 0x3c, 0x48, 0xac, 0x6b, 0x92, 0x2e, 0xb3, 0x6c, 0x64, 0xf5, 0x6c, 0xbd, 0xd6, 0xcd, 0x88, 0x81, - 0xcd, 0x71, 0xf1, 0x0f, 0x45, 0xf6, 0xd8, 0xcd, 0x98, 0xff, 0x1f, 0x3d, 0xed, 0xc8, 0xfb, 0xd0, - 0xaf, 0x41, 0x49, 0xb3, 0xda, 0x6d, 0x66, 0x94, 0xb1, 0xf9, 0x77, 0xf0, 0xae, 0x1c, 0x5a, 0x04, - 0xfe, 0x6d, 0xd0, 0xb2, 0xff, 0x8d, 0xd0, 0xb2, 0xcc, 0xbf, 0x11, 0x1a, 0x0a, 0x83, 0x28, 0xfa, - 0xc2, 0xd8, 0xb4, 0xdd, 0x17, 0x91, 0x31, 0xfd, 0x3a, 0x11, 0x19, 0x2c, 0x8e, 0x85, 0xf7, 0xbc, - 0x3f, 0x14, 0xb8, 0x3f, 0xdc, 0x86, 0x65, 0x1d, 0xf6, 0x7c, 0x2f, 0xa3, 0x85, 0xf0, 0x3d, 0x97, - 0x81, 0xb3, 0x38, 0x0d, 0x76, 0x8a, 0x5b, 0xa8, 0x53, 0x5f, 0x6e, 0xa1, 0xbe, 0x01, 0x33, 0x5d, - 0x1b, 0xef, 0x61, 0x57, 0x3b, 0x50, 0xcc, 0x5e, 0x87, 0x47, 0x7a, 0xe5, 0x3d, 0xda, 0x56, 0xaf, - 0x83, 0xee, 0x81, 0xe8, 0x67, 0xe1, 0x40, 0xd9, 0xbb, 0x64, 0xcd, 0xa3, 0x73, 0x58, 0x2d, 0xfd, - 0x0f, 0x01, 0xe6, 0x22, 0x75, 0xe2, 0x63, 0x6a, 0x0d, 0xf2, 0xba, 0x6f, 0x1a, 0x39, 0xf3, 0xc2, - 0x05, 0xc3, 0x08, 0xc2, 0xcc, 0x48, 0x81, 0x2b, 0xde, 0x6b, 0xe9, 0x57, 0x11, 0x02, 0xb1, 0xa9, - 0x0b, 0x8a, 0xbd, 0x1c, 0xc8, 0x59, 0x0d, 0xbd, 0xc0, 0x1f, 0x64, 0xe9, 0xb1, 0x06, 0x99, 0xf4, - 0xbf, 0x04, 0x10, 0xe9, 0x0b, 0x1e, 0x63, 0xac, 0x27, 0x32, 0x65, 0x7a, 0xf1, 0x3a, 0xa9, 0xf1, - 0x63, 0xb9, 0x22, 0x5f, 0x72, 0x49, 0xf7, 0x7d, 0xc9, 0x25, 0x6e, 0xfe, 0xcc, 0x7c, 0xc9, 0xf9, - 0x53, 0xfa, 0xb1, 0x00, 0x45, 0xbf, 0xda, 0xec, 0x13, 0x8e, 0x23, 0xee, 0x68, 0x7d, 0xbd, 0xcf, - 0x14, 0x7a, 0x77, 0xc9, 0x8c, 0xf5, 0x55, 0xc9, 0xf0, 0x5d, 0x32, 0xec, 0xf3, 0x7a, 0x7f, 0xcb, - 0xeb, 0x8e, 0xa4, 0x88, 0xd5, 0xe0, 0x12, 0x8f, 0xd7, 0x88, 0x95, 0xfb, 0x2a, 0x1c, 0xea, 0x1e, - 0x87, 0x14, 0x48, 0x7b, 0x14, 0xd1, 0xd2, 0x58, 0xf3, 0xbb, 0xa7, 0x25, 0xd6, 0x01, 0x7f, 0x1a, - 0x6e, 0x09, 0x16, 0x04, 0xfe, 0x08, 0xd2, 0x47, 0x6a, 0x7b, 0x94, 0x07, 0x61, 0xa4, 0xe5, 0x64, - 0x92, 0x1b, 0x3d, 0x8e, 0xdc, 0x7d, 0x92, 0x1a, 0xbe, 0x89, 0x36, 0xa8, 0xd2, 0xc8, 0x1d, 0x29, - 0xdf, 0x8e, 0x0e, 0xa0, 0x91, 0xaf, 0x0f, 0x8f, 0xa4, 0xf7, 0x33, 0x3f, 0xf9, 0xf1, 0x92, 0x20, - 0x7d, 0x08, 0x88, 0xd8, 0x3a, 0xee, 0xb3, 0x9e, 0x65, 0x07, 0xf7, 0xc8, 0xf4, 0xc7, 0xe5, 0x4c, - 0xc6, 0xc7, 0xe5, 0x48, 0x97, 0x61, 0x2e, 0xc2, 0xcd, 0x66, 0x20, 0xe9, 0xdb, 0xf0, 0xc6, 0x13, - 0xcb, 0x71, 0x8c, 0x6e, 0xb3, 0xb7, 0xcb, 0x86, 0x3a, 0x59, 0xaf, 0xfc, 0x39, 0x37, 0xdb, 0xa5, - 0x3b, 0x34, 0x26, 0x9b, 0x9b, 0x72, 0xb2, 0xff, 0x2c, 0xfd, 0x4b, 0x01, 0xae, 0x0e, 0x72, 0x32, - 0x2d, 0xc7, 0x85, 0xf6, 0x4e, 0x6b, 0x56, 0x70, 0xcd, 0xe1, 0xf9, 0xbd, 0xd5, 0xcb, 0x4e, 0x6c, - 0x6f, 0xfe, 0x4e, 0xa5, 0xa3, 0xd2, 0x39, 0x89, 0x47, 0xdd, 0x17, 0x39, 0x79, 0x93, 0x51, 0x83, - 0xe9, 0x29, 0x33, 0xde, 0xf4, 0xf4, 0xbf, 0x05, 0x98, 0x6d, 0x61, 0x53, 0x35, 0x5d, 0x32, 0xef, - 0xf7, 0x3a, 0x2c, 0x1c, 0xba, 0x04, 0x69, 0x5b, 0xe9, 0xd1, 0xa2, 0x0b, 0x72, 0xca, 0xde, 0x41, - 0x37, 0xa1, 0x40, 0xd7, 0x96, 0x90, 0xad, 0x26, 0xdc, 0xcd, 0xc8, 0xd4, 0xcb, 0x5c, 0xf6, 0x0c, - 0xb1, 0xb7, 0x00, 0x68, 0x26, 0x86, 0x81, 0xd2, 0x34, 0x47, 0x8e, 0x50, 0x7c, 0x04, 0x44, 0xa3, - 0x76, 0x02, 0x21, 0xcc, 0x31, 0xbf, 0x40, 0xa9, 0xbe, 0x94, 0x25, 0xc8, 0xb3, 0x6c, 0x4c, 0xcc, - 0x24, 0xcd, 0x03, 0x94, 0xc4, 0xe4, 0x3c, 0x86, 0x4b, 0xce, 0xcb, 0xb6, 0xd2, 0xb5, 0x74, 0x47, - 0xd1, 0xba, 0x3d, 0xee, 0xf1, 0xcc, 0xbe, 0x0a, 0x2b, 0x54, 0x2e, 0x9f, 0x9d, 0x2e, 0xcd, 0x36, - 0x9f, 0x6d, 0x6c, 0x5b, 0xba, 0x53, 0xdd, 0xde, 0x61, 0xfe, 0xce, 0x8e, 0x3c, 0xeb, 0xbc, 0x6c, - 0x53, 0x52, 0xb7, 0xc7, 0x49, 0xd2, 0x67, 0x29, 0x40, 0xf4, 0x5a, 0x8d, 0x0a, 0xbd, 0x97, 0xc2, - 0x6b, 0x6e, 0x0b, 0x16, 0xb5, 0x40, 0x15, 0x8a, 0x63, 0x98, 0xc4, 0xbe, 0x57, 0x1d, 0xd7, 0x2b, - 0x37, 0x1f, 0x37, 0xb1, 0xb7, 0x01, 0xf6, 0x6b, 0x92, 0xb7, 0xe3, 0xb5, 0x90, 0xc4, 0x26, 0x11, - 0xb8, 0xa1, 0x3a, 0xfe, 0x0b, 0xef, 0x41, 0xce, 0xa5, 0x7c, 0xde, 0xa5, 0x25, 0x99, 0xca, 0xcc, - 0xd9, 0xe9, 0x52, 0x96, 0x09, 0x6b, 0xac, 0xca, 0x59, 0x96, 0xdc, 0xd0, 0xd1, 0x32, 0xe4, 0x0d, - 0xd3, 0x71, 0x55, 0x52, 0x24, 0xbe, 0x55, 0x50, 0x60, 0x01, 0xb2, 0x0d, 0x4e, 0x6e, 0xac, 0xca, - 0xe0, 0x65, 0xa1, 0x71, 0x68, 0x45, 0x9f, 0x81, 0x6d, 0xa6, 0xd3, 0xa0, 0x49, 0xb9, 0xe0, 0x51, - 0xd9, 0xb5, 0x5a, 0x4d, 0x98, 0x8b, 0x68, 0x82, 0x2f, 0xcc, 0x1f, 0x46, 0x27, 0x9c, 0xb0, 0x75, - 0xea, 0x7d, 0x54, 0xbe, 0x5c, 0x33, 0x35, 0x4b, 0xe7, 0x23, 0x35, 0x3a, 0xf1, 0xb4, 0xa0, 0xb4, - 0x66, 0x19, 0x26, 0x41, 0x8f, 0x5e, 0x55, 0x57, 0xa0, 0xb8, 0x6b, 0x98, 0xaa, 0x7d, 0xa2, 0x78, - 0x0e, 0xde, 0xc2, 0x79, 0x0e, 0xde, 0x72, 0x81, 0x71, 0xf0, 0x47, 0xe9, 0xe7, 0x02, 0x88, 0x81, - 0x58, 0x5e, 0xd0, 0x6f, 0x00, 0x68, 0xed, 0x9e, 0xe3, 0x62, 0xdb, 0x9b, 0x00, 0x66, 0x58, 0x20, - 0x59, 0x95, 0x51, 0x1b, 0xab, 0x72, 0x8e, 0x67, 0x68, 0xe8, 0xe8, 0x66, 0xf4, 0x8e, 0x98, 0xc9, - 0x0a, 0x9c, 0x0d, 0xdc, 0x0c, 0x43, 0x66, 0x14, 0xc7, 0xb5, 0x6c, 0x5f, 0xcf, 0x7c, 0x46, 0xf1, - 0x6e, 0xcf, 0xa2, 0xb7, 0x44, 0x60, 0x1a, 0x2b, 0x5a, 0x24, 0xe6, 0xed, 0x11, 0xf6, 0xab, 0x94, - 0x39, 0xbf, 0x4a, 0x8c, 0xc3, 0xab, 0xd2, 0xef, 0x0b, 0x50, 0xaa, 0xb2, 0x81, 0xee, 0x4f, 0x1e, - 0x23, 0x16, 0xcb, 0x55, 0xc8, 0xba, 0xc7, 0xa6, 0xd2, 0xc1, 0xfe, 0x07, 0x9f, 0x2e, 0x70, 0xbd, - 0xe5, 0xb4, 0xcb, 0x1e, 0xe9, 0x37, 0x44, 0xf9, 0x07, 0xec, 0xf9, 0x4c, 0xfc, 0x46, 0x99, 0x7d, - 0xe1, 0xbe, 0xec, 0x7d, 0xe1, 0xbe, 0xbc, 0xca, 0x33, 0x30, 0x23, 0xe4, 0xf3, 0xff, 0xb0, 0x24, - 0xc8, 0x3e, 0x13, 0xb3, 0x53, 0xef, 0x37, 0xc9, 0x84, 0x3a, 0x60, 0x49, 0xa2, 0x22, 0x40, 0xe8, - 0x4b, 0x5e, 0xfc, 0x9b, 0xe9, 0x2b, 0xab, 0xca, 0xce, 0x56, 0xf5, 0xe9, 0xe6, 0x66, 0xa3, 0xd5, - 0xaa, 0xad, 0x8a, 0x02, 0x12, 0x61, 0x26, 0xf2, 0x1d, 0xb0, 0x14, 0xfb, 0x8a, 0xfa, 0xfd, 0x77, - 0xa0, 0x10, 0xd9, 0xf6, 0x41, 0x25, 0xc8, 0x6f, 0xd4, 0x56, 0x9a, 0xb5, 0xfa, 0xd3, 0x8d, 0x55, - 0x0a, 0x64, 0xf3, 0x30, 0xbd, 0x55, 0x5b, 0x91, 0x6b, 0xcd, 0x96, 0x28, 0xdc, 0xff, 0x0b, 0x00, - 0xc1, 0x17, 0x08, 0xc9, 0xab, 0xd7, 0x6b, 0x1f, 0x2b, 0xcf, 0x57, 0x36, 0x76, 0x6a, 0x4d, 0x71, - 0x02, 0x21, 0x28, 0x56, 0x56, 0x5a, 0xd5, 0xba, 0x22, 0xd7, 0x9a, 0xdb, 0x4f, 0xb7, 0x9a, 0x35, - 0xef, 0x63, 0xed, 0xf7, 0x57, 0x61, 0x26, 0x7c, 0xc7, 0x17, 0x9a, 0x83, 0x52, 0xb5, 0x5e, 0xab, - 0xae, 0x2b, 0xcf, 0x1b, 0x2b, 0xca, 0xb3, 0x9d, 0xda, 0x0e, 0x41, 0xbe, 0xa4, 0x26, 0x94, 0xf8, - 0x78, 0x67, 0x83, 0x80, 0xe6, 0x12, 0xe4, 0xd9, 0x33, 0xfd, 0xc4, 0x98, 0x98, 0xba, 0xbf, 0x09, - 0xf9, 0xd0, 0xdd, 0xe3, 0xe4, 0x75, 0xdb, 0x3b, 0xcd, 0xba, 0xd2, 0x6a, 0x6c, 0xd6, 0x9a, 0xad, - 0x95, 0xcd, 0x6d, 0x26, 0x83, 0xd2, 0x56, 0x2a, 0x4f, 0xe5, 0x96, 0x28, 0xf8, 0xcf, 0xad, 0xa7, - 0x3b, 0xd5, 0xba, 0x57, 0x6b, 0x29, 0x93, 0x4d, 0x8b, 0xe9, 0xfb, 0xc7, 0x70, 0x75, 0xc8, 0x75, - 0x57, 0xa4, 0xd2, 0x3b, 0x26, 0xbd, 0x87, 0x59, 0x9c, 0x40, 0x05, 0xc8, 0x91, 0x9e, 0x4a, 0x83, - 0xe1, 0x45, 0x01, 0x65, 0x21, 0x73, 0xe0, 0xba, 0x5d, 0x31, 0x85, 0xa6, 0x20, 0xe5, 0x3c, 0x12, - 0xd3, 0xe4, 0xff, 0xbe, 0x23, 0x66, 0x08, 0x74, 0x57, 0x3f, 0xed, 0xd9, 0x58, 0x9c, 0x24, 0xf0, - 0xbf, 0xe7, 0x60, 0x7b, 0xcf, 0x68, 0x63, 0x71, 0x9a, 0xb0, 0x98, 0xbd, 0x76, 0x5b, 0xcc, 0x4a, - 0x99, 0xec, 0x94, 0x38, 0x75, 0xff, 0x06, 0x84, 0x6e, 0x1d, 0x21, 0xe8, 0x7f, 0x43, 0x75, 0xb1, - 0xe3, 0x8a, 0x13, 0x68, 0x1a, 0xd2, 0x2b, 0xed, 0xb6, 0x28, 0x3c, 0xfc, 0x7c, 0x12, 0xb2, 0xde, - 0x17, 0xb4, 0xd0, 0x06, 0x4c, 0x52, 0xa4, 0x8b, 0x96, 0x86, 0x63, 0x60, 0x3a, 0xec, 0x17, 0xae, - 0x9f, 0x07, 0x92, 0xa5, 0x09, 0xf4, 0x17, 0x21, 0x1f, 0xc2, 0x06, 0x68, 0xe8, 0x3e, 0x7c, 0x04, - 0x0f, 0x2d, 0xdc, 0x3e, 0x2f, 0x9b, 0x2f, 0xff, 0x05, 0xe4, 0x7c, 0xb3, 0x02, 0xdd, 0x1c, 0x65, - 0x74, 0x78, 0xb2, 0x47, 0x5b, 0x26, 0x64, 0x94, 0x4a, 0x13, 0xef, 0x0a, 0xc8, 0x06, 0x34, 0x68, - 0x01, 0xa0, 0x38, 0x5f, 0xd7, 0xa1, 0x26, 0xc6, 0xc2, 0xfd, 0xb1, 0x72, 0x07, 0xef, 0x24, 0xca, - 0x0a, 0xcc, 0x98, 0x78, 0x65, 0x0d, 0x18, 0x49, 0xf1, 0xca, 0x8a, 0xb1, 0x86, 0x68, 0x63, 0x84, - 0xd6, 0x83, 0x58, 0xf9, 0x83, 0x2b, 0x67, 0xac, 0xfc, 0x98, 0x65, 0x45, 0x9a, 0x40, 0xcf, 0x20, - 0x43, 0xe6, 0x70, 0x14, 0x07, 0x30, 0xfa, 0xd6, 0x8c, 0x85, 0x9b, 0x23, 0xf3, 0x78, 0x22, 0x2b, - 0xf7, 0x7e, 0xf2, 0x1f, 0x17, 0x27, 0x7e, 0x72, 0xb6, 0x28, 0xfc, 0xfc, 0x6c, 0x51, 0xf8, 0xa3, - 0xb3, 0x45, 0xe1, 0x8f, 0xcf, 0x16, 0x85, 0x1f, 0xfe, 0x62, 0x71, 0xe2, 0xe7, 0xbf, 0x58, 0x9c, - 0xf8, 0xa3, 0x5f, 0x2c, 0x4e, 0x7c, 0x32, 0xcd, 0xb9, 0x77, 0xa7, 0xe8, 0x04, 0xf7, 0xe8, 0xff, - 0x05, 0x00, 0x00, 0xff, 0xff, 0xe5, 0x42, 0x49, 0xd7, 0x40, 0x85, 0x00, 0x00, + // 8837 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x57, + 0x96, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x24, 0x4b, 0x57, 0xfd, 0x43, 0xab, 0x6d, 0xa9, 0xbb, + 0xda, 0xfd, 0xbb, 0x63, 0xca, 0xdd, 0x3d, 0x8e, 0x3d, 0xb6, 0xc7, 0xb3, 0x22, 0xc5, 0x6e, 0x52, + 0xff, 0x5d, 0xa4, 0xba, 0x61, 0xef, 0x6c, 0x6a, 0x4b, 0x55, 0x57, 0x52, 0x8d, 0xc8, 0x2a, 0x76, + 0x55, 0x51, 0x2d, 0x19, 0xc8, 0x43, 0x92, 0xc5, 0xc4, 0x4f, 0xc1, 0x04, 0x08, 0xb0, 0x1e, 0x6c, + 0x10, 0x78, 0x33, 0x8b, 0xe4, 0x21, 0x0f, 0x09, 0x92, 0x20, 0xc1, 0x06, 0x49, 0xf6, 0x31, 0x83, + 0x60, 0x90, 0x99, 0x79, 0x5b, 0x04, 0x88, 0xb2, 0xd1, 0xe4, 0x21, 0x8b, 0x20, 0xd8, 0x20, 0x09, + 0xb0, 0x81, 0x81, 0x04, 0xc1, 0xfd, 0xa9, 0x1f, 0x92, 0x45, 0x8a, 0x6a, 0x97, 0x13, 0x03, 0xfb, + 0x22, 0xb1, 0xce, 0xbd, 0xe7, 0xd4, 0xbd, 0xe7, 0xfe, 0x9d, 0xef, 0xde, 0x73, 0x6e, 0xc1, 0xac, + 0x6d, 0xa9, 0xda, 0x41, 0x67, 0x77, 0x49, 0xed, 0x18, 0xa5, 0x8e, 0x6d, 0xb9, 0x16, 0x9a, 0xd5, + 0x2c, 0xed, 0x90, 0x92, 0x4b, 0x3c, 0x71, 0xfe, 0x32, 0xb6, 0x6d, 0xcb, 0x76, 0x3a, 0xbb, 0x4b, + 0xec, 0x07, 0xcb, 0x39, 0x7f, 0xff, 0xf0, 0x68, 0xe9, 0xf0, 0xc8, 0xc1, 0xf6, 0x11, 0xb6, 0x97, + 0x34, 0xcb, 0xd4, 0xba, 0xb6, 0x8d, 0x4d, 0xed, 0x64, 0xa9, 0x65, 0x69, 0x87, 0xf4, 0x8f, 0x61, + 0xee, 0x47, 0xe5, 0xb5, 0xb1, 0xaa, 0x3b, 0xdd, 0x76, 0x5b, 0xb5, 0x4f, 0x96, 0xa8, 0x58, 0xfe, + 0xc0, 0xf3, 0x22, 0xaf, 0x50, 0xba, 0xea, 0xaa, 0x9c, 0x76, 0xc9, 0xa3, 0xf5, 0x94, 0xe0, 0x8a, + 0x47, 0x6d, 0x63, 0x57, 0x0d, 0xe5, 0xbe, 0xe6, 0xb8, 0x96, 0xad, 0xee, 0xe3, 0x25, 0x6c, 0xee, + 0x1b, 0x26, 0x26, 0x19, 0x8e, 0x34, 0x8d, 0x27, 0xbe, 0x1e, 0x99, 0xf8, 0x88, 0xa7, 0x16, 0xbb, + 0xae, 0xd1, 0x5a, 0x3a, 0x68, 0x69, 0x4b, 0xae, 0xd1, 0xc6, 0x8e, 0xab, 0xb6, 0x3b, 0x5e, 0x15, + 0x68, 0x8a, 0x6b, 0xab, 0x9a, 0x61, 0xee, 0x7b, 0xff, 0x3b, 0xbb, 0x4b, 0x36, 0xd6, 0x2c, 0x5b, + 0xc7, 0xba, 0xe2, 0x74, 0x54, 0xd3, 0x2b, 0xee, 0xbe, 0xb5, 0x6f, 0xd1, 0x9f, 0x4b, 0xe4, 0x17, + 0xa7, 0x2e, 0xec, 0x5b, 0xd6, 0x7e, 0x0b, 0x2f, 0xd1, 0xa7, 0xdd, 0xee, 0xde, 0x92, 0xde, 0xb5, + 0x55, 0xd7, 0xb0, 0x38, 0x97, 0xf4, 0x4f, 0x05, 0xc8, 0xc9, 0xf8, 0x45, 0x17, 0x3b, 0x6e, 0x0d, + 0xab, 0x3a, 0xb6, 0xd1, 0x6b, 0x90, 0x3c, 0xc4, 0x27, 0xc5, 0xe4, 0x75, 0xe1, 0xee, 0x4c, 0x79, + 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, 0x1a, 0x3e, 0x91, 0x09, 0x0d, 0x5d, 0x87, 0x69, 0x6c, 0xea, 0x0a, + 0x49, 0x4e, 0xf5, 0x26, 0x4f, 0x61, 0x53, 0x5f, 0xc3, 0x27, 0xe8, 0xfb, 0x90, 0x76, 0x88, 0x34, + 0x53, 0xc3, 0xc5, 0xc9, 0xeb, 0xc2, 0xdd, 0xc9, 0xf2, 0xaf, 0x7f, 0x79, 0xba, 0xf8, 0xe1, 0xbe, + 0xe1, 0x1e, 0x74, 0x77, 0x4b, 0x9a, 0xd5, 0x5e, 0xf2, 0x9b, 0x5a, 0xdf, 0x0d, 0x7e, 0x2f, 0x75, + 0x0e, 0xf7, 0x97, 0xfa, 0x75, 0x54, 0x6a, 0x1e, 0x9b, 0x0d, 0xfc, 0x42, 0xf6, 0x25, 0xae, 0xa6, + 0xd2, 0x82, 0x98, 0x58, 0x4d, 0xa5, 0x13, 0x62, 0x52, 0xfa, 0x79, 0x02, 0xf2, 0x32, 0x76, 0x3a, + 0x96, 0xe9, 0x60, 0x5e, 0xf2, 0xb7, 0x21, 0xe9, 0x1e, 0x9b, 0xb4, 0xe4, 0xd9, 0x87, 0x0b, 0xa5, + 0x81, 0x4e, 0x55, 0x6a, 0xda, 0xaa, 0xe9, 0xa8, 0x1a, 0xa9, 0xbe, 0x4c, 0xb2, 0xa2, 0xf7, 0x20, + 0x6b, 0x63, 0xa7, 0xdb, 0xc6, 0x54, 0x91, 0xb4, 0x52, 0xd9, 0x87, 0x57, 0x23, 0x38, 0x1b, 0x1d, + 0xd5, 0x94, 0x81, 0xe5, 0x25, 0xbf, 0x51, 0x03, 0x72, 0x9c, 0xd3, 0xc6, 0xaa, 0x63, 0x99, 0xc5, + 0xe9, 0xeb, 0xc2, 0xdd, 0xfc, 0xc3, 0x52, 0x04, 0x6f, 0x6f, 0x29, 0xc9, 0x63, 0xb7, 0x8d, 0x65, + 0xca, 0x25, 0xcf, 0xd8, 0xa1, 0x27, 0xf4, 0x1a, 0xa4, 0xcd, 0x6e, 0x9b, 0xe8, 0xd7, 0xa1, 0xda, + 0x4b, 0xca, 0xd3, 0x66, 0xb7, 0xbd, 0x86, 0x4f, 0x1c, 0x74, 0x0d, 0x32, 0x24, 0x69, 0xf7, 0xc4, + 0xc5, 0x4e, 0x31, 0x4d, 0xd3, 0x48, 0xde, 0x32, 0x79, 0x96, 0x3e, 0x82, 0x99, 0xb0, 0x54, 0x84, + 0x20, 0x2f, 0x57, 0x1b, 0x3b, 0x1b, 0x55, 0x65, 0x67, 0x73, 0x6d, 0x73, 0xeb, 0xf9, 0xa6, 0x38, + 0x81, 0x2e, 0x81, 0xc8, 0x69, 0x6b, 0xd5, 0x8f, 0x95, 0xf5, 0xfa, 0x46, 0xbd, 0x29, 0x0a, 0xf3, + 0xa9, 0xcf, 0x7e, 0xb2, 0x30, 0xb1, 0x9a, 0x4a, 0x4f, 0x89, 0xd3, 0xd2, 0x4f, 0x04, 0x80, 0x27, + 0xd8, 0xe5, 0xbd, 0x01, 0x95, 0x61, 0xea, 0x80, 0x96, 0xb8, 0x28, 0x50, 0xb5, 0x5c, 0x8f, 0xac, + 0x5a, 0xa8, 0xe7, 0x94, 0xd3, 0x3f, 0x3d, 0x5d, 0x9c, 0xf8, 0xc5, 0xe9, 0xa2, 0x20, 0x73, 0x4e, + 0xf4, 0x14, 0xb2, 0x87, 0xf8, 0x44, 0xe1, 0xe3, 0xb2, 0x98, 0xa0, 0x3a, 0x7a, 0x3b, 0x24, 0xe8, + 0xf0, 0xa8, 0xe4, 0x0d, 0xd1, 0x52, 0x68, 0x38, 0x97, 0x08, 0x47, 0xa9, 0xe1, 0xda, 0xd8, 0xdc, + 0x77, 0x0f, 0x64, 0x38, 0xc4, 0x27, 0xeb, 0x4c, 0x86, 0xf4, 0x87, 0x02, 0x64, 0x69, 0x29, 0x99, + 0x52, 0x51, 0xa5, 0xaf, 0x98, 0x37, 0xce, 0x6d, 0x81, 0x88, 0x72, 0x96, 0x60, 0xf2, 0x48, 0x6d, + 0x75, 0x31, 0x2d, 0x61, 0xf6, 0x61, 0x31, 0x42, 0xc6, 0x33, 0x92, 0x2e, 0xb3, 0x6c, 0xe8, 0x03, + 0x98, 0x31, 0x4c, 0x17, 0x9b, 0xae, 0xc2, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x6e, 0xfa, 0x20, + 0xfd, 0x13, 0x01, 0x60, 0xbb, 0x1b, 0xab, 0x9e, 0xbf, 0x3d, 0x66, 0xf9, 0xcb, 0x29, 0xc2, 0xea, + 0xd5, 0xe2, 0x0a, 0x4c, 0x19, 0x66, 0xcb, 0x30, 0x59, 0xf9, 0xd3, 0x32, 0x7f, 0x42, 0x97, 0x60, + 0x72, 0xb7, 0x65, 0x98, 0x3a, 0x1d, 0x0f, 0x69, 0x99, 0x3d, 0x48, 0x32, 0x64, 0x69, 0xa9, 0x63, + 0xd4, 0xbb, 0x74, 0x9a, 0x80, 0xcb, 0x15, 0xcb, 0xd4, 0x0d, 0x32, 0x24, 0xd5, 0xd6, 0x37, 0x42, + 0x2b, 0xab, 0x70, 0x49, 0xc7, 0x1d, 0x1b, 0x6b, 0xaa, 0x8b, 0x75, 0x05, 0x1f, 0x77, 0xc6, 0x6c, + 0x63, 0x14, 0x70, 0x55, 0x8f, 0x3b, 0x94, 0x46, 0x46, 0x2d, 0x11, 0xc0, 0x46, 0xed, 0x14, 0x99, + 0x32, 0xe5, 0x34, 0x3e, 0xee, 0xd0, 0x51, 0x1b, 0xad, 0x66, 0xf4, 0x6d, 0xb8, 0xaa, 0xb6, 0x5a, + 0xd6, 0x4b, 0xc5, 0xd8, 0x53, 0x74, 0x0b, 0x3b, 0x8a, 0x69, 0xb9, 0x0a, 0x3e, 0x36, 0x1c, 0x97, + 0x4e, 0x09, 0x69, 0x79, 0x8e, 0x26, 0xd7, 0xf7, 0x56, 0x2c, 0xec, 0x6c, 0x5a, 0x6e, 0x95, 0x24, + 0x85, 0x9a, 0x72, 0x3a, 0xdc, 0x94, 0xd2, 0x6f, 0xc2, 0x95, 0x7e, 0xfd, 0xc6, 0xd9, 0x7e, 0x3f, + 0x13, 0x20, 0x5f, 0x37, 0x0d, 0xf7, 0x1b, 0xd1, 0x70, 0xbe, 0x3e, 0x93, 0x61, 0x7d, 0xde, 0x07, + 0x71, 0x4f, 0x35, 0x5a, 0x5b, 0x66, 0xd3, 0x6a, 0xef, 0x3a, 0xae, 0x65, 0x62, 0x87, 0x2b, 0x7c, + 0x80, 0x2e, 0x3d, 0x83, 0x82, 0x5f, 0x9b, 0x38, 0xd5, 0xe4, 0x82, 0x58, 0x37, 0x35, 0x1b, 0xb7, + 0xb1, 0x19, 0xab, 0x9e, 0x5e, 0x87, 0x8c, 0xe1, 0xc9, 0xa5, 0xba, 0x4a, 0xca, 0x01, 0x41, 0xea, + 0xc2, 0x6c, 0xe8, 0xad, 0x71, 0x4e, 0x97, 0x64, 0x31, 0xc2, 0x2f, 0x95, 0xa0, 0x8d, 0xc8, 0x62, + 0x84, 0x5f, 0xb2, 0xe9, 0xad, 0x01, 0xb9, 0x15, 0xdc, 0xc2, 0x2e, 0x8e, 0xb1, 0xa6, 0xd2, 0x0e, + 0xe4, 0x3d, 0xa1, 0x71, 0x36, 0xcc, 0xef, 0x08, 0x80, 0xb8, 0x5c, 0xd5, 0xdc, 0x8f, 0xb3, 0xc4, + 0x68, 0x91, 0x98, 0x16, 0x6e, 0xd7, 0x36, 0xd9, 0x72, 0xce, 0xfa, 0x24, 0x30, 0x12, 0x5d, 0xd1, + 0x83, 0x21, 0x9b, 0x0a, 0x0f, 0x59, 0x6e, 0xde, 0xbc, 0x84, 0xb9, 0x9e, 0x82, 0xc5, 0xdb, 0x7c, + 0x29, 0x5a, 0xa6, 0xc4, 0xf5, 0x64, 0xd8, 0x86, 0xa3, 0x44, 0xe9, 0x73, 0x01, 0x66, 0x2b, 0x2d, + 0xac, 0xda, 0xb1, 0x6b, 0xe4, 0x7b, 0x90, 0xd6, 0xb1, 0xaa, 0xd3, 0x2a, 0xb3, 0x81, 0xfd, 0x46, + 0x48, 0x0a, 0xb1, 0x74, 0x4b, 0x07, 0x2d, 0xad, 0xd4, 0xf4, 0x6c, 0x60, 0x3e, 0xba, 0x7d, 0x26, + 0xe9, 0x63, 0x40, 0xe1, 0x92, 0xc5, 0xd9, 0x11, 0x7e, 0x3f, 0x01, 0x48, 0xc6, 0x47, 0xd8, 0x76, + 0x63, 0xaf, 0xf6, 0x0a, 0x64, 0x5d, 0xd5, 0xde, 0xc7, 0xae, 0x42, 0xac, 0xfb, 0x8b, 0xd4, 0x1c, + 0x18, 0x1f, 0x21, 0xa3, 0x26, 0xdc, 0xc1, 0xa6, 0xba, 0xdb, 0xc2, 0x54, 0x8a, 0xb2, 0x6b, 0x75, + 0x4d, 0x5d, 0x31, 0x5c, 0x6c, 0xab, 0xae, 0x65, 0x2b, 0x56, 0xc7, 0x35, 0xda, 0xc6, 0xa7, 0xd4, + 0xb0, 0xe7, 0x5d, 0xed, 0x26, 0xcb, 0x4e, 0x98, 0xcb, 0x24, 0x73, 0x9d, 0xe7, 0xdd, 0x0a, 0x65, + 0x45, 0x25, 0x98, 0x33, 0xf6, 0x4d, 0xcb, 0xc6, 0xca, 0xbe, 0xa6, 0xb8, 0x07, 0x36, 0x76, 0x0e, + 0xac, 0x96, 0xb7, 0x20, 0xcd, 0xb2, 0xa4, 0x27, 0x5a, 0xd3, 0x4b, 0x90, 0x3e, 0x81, 0xb9, 0x1e, + 0x2d, 0xc5, 0xd9, 0x04, 0xff, 0x4d, 0x80, 0x6c, 0x43, 0x53, 0xcd, 0x38, 0x75, 0xff, 0x11, 0x64, + 0x1d, 0x4d, 0x35, 0x95, 0x3d, 0xcb, 0x6e, 0xab, 0x2e, 0xad, 0x57, 0xbe, 0x47, 0xf7, 0xbe, 0x7d, + 0xaf, 0xa9, 0xe6, 0x63, 0x9a, 0x49, 0x06, 0xc7, 0xff, 0xdd, 0x6f, 0xbf, 0x4e, 0x7e, 0x75, 0xfb, + 0x95, 0x0d, 0xef, 0xd5, 0x54, 0x3a, 0x29, 0xa6, 0xa4, 0x3f, 0x13, 0x60, 0x86, 0x55, 0x39, 0xce, + 0xe1, 0xfd, 0x0e, 0xa4, 0x6c, 0xeb, 0x25, 0x1b, 0xde, 0xd9, 0x87, 0xd7, 0x22, 0x44, 0xac, 0xe1, + 0x93, 0xf0, 0xfa, 0x49, 0xb3, 0xa3, 0x32, 0x70, 0x2b, 0x55, 0xa1, 0xdc, 0xc9, 0x71, 0xb9, 0x81, + 0x71, 0xc9, 0x44, 0xc6, 0x1d, 0x28, 0xec, 0xaa, 0xae, 0x76, 0xa0, 0xd8, 0xbc, 0x90, 0x64, 0xad, + 0x4d, 0xde, 0x9d, 0x91, 0xf3, 0x94, 0xec, 0x15, 0xdd, 0x21, 0x35, 0x67, 0xe3, 0xcd, 0xc1, 0x7f, + 0xce, 0xda, 0xfc, 0xff, 0x08, 0x7c, 0x0c, 0x79, 0x35, 0xff, 0xf3, 0xd6, 0xf4, 0x3f, 0x4e, 0xc0, + 0xd5, 0xca, 0x01, 0xd6, 0x0e, 0x2b, 0x96, 0xe9, 0x18, 0x8e, 0x4b, 0x74, 0x17, 0x67, 0xfb, 0x5f, + 0x83, 0xcc, 0x4b, 0xc3, 0x3d, 0x50, 0x74, 0x63, 0x6f, 0x8f, 0xce, 0xb6, 0x69, 0x39, 0x4d, 0x08, + 0x2b, 0xc6, 0xde, 0x1e, 0x7a, 0x04, 0xa9, 0xb6, 0xa5, 0x33, 0x63, 0x3e, 0xff, 0x70, 0x31, 0x42, + 0x3c, 0x2d, 0x9a, 0xd3, 0x6d, 0x6f, 0x58, 0x3a, 0x96, 0x69, 0x66, 0xb4, 0x00, 0xa0, 0x11, 0x6a, + 0xc7, 0x32, 0x4c, 0x97, 0x4f, 0x8e, 0x21, 0x0a, 0xaa, 0x41, 0xc6, 0xc5, 0x76, 0xdb, 0x30, 0x55, + 0x17, 0x17, 0x27, 0xa9, 0xf2, 0xde, 0x8c, 0x2c, 0x78, 0xa7, 0x65, 0x68, 0xea, 0x0a, 0x76, 0x34, + 0xdb, 0xe8, 0xb8, 0x96, 0xcd, 0xb5, 0x18, 0x30, 0x4b, 0x7f, 0x3d, 0x05, 0xc5, 0x41, 0xdd, 0xc4, + 0xd9, 0x43, 0xb6, 0x61, 0xca, 0xc6, 0x4e, 0xb7, 0xe5, 0xf2, 0x3e, 0xf2, 0x70, 0x98, 0x0a, 0x22, + 0x4a, 0x40, 0xb7, 0x2e, 0x5a, 0x2e, 0x2f, 0x36, 0x97, 0x33, 0xff, 0x2f, 0x04, 0x98, 0x62, 0x09, + 0xe8, 0x01, 0xa4, 0x6d, 0xb2, 0x30, 0x28, 0x86, 0x4e, 0xcb, 0x98, 0x2c, 0x5f, 0x39, 0x3b, 0x5d, + 0x9c, 0xa6, 0x8b, 0x45, 0x7d, 0xe5, 0xcb, 0xe0, 0xa7, 0x3c, 0x4d, 0xf3, 0xd5, 0x75, 0xd2, 0x5a, + 0x8e, 0xab, 0xda, 0x2e, 0xdd, 0x54, 0x4a, 0x30, 0x84, 0x44, 0x09, 0x6b, 0xf8, 0x04, 0xad, 0xc2, + 0x94, 0xe3, 0xaa, 0x6e, 0xd7, 0xe1, 0xed, 0x75, 0xa1, 0xc2, 0x36, 0x28, 0xa7, 0xcc, 0x25, 0x10, + 0x73, 0x4b, 0xc7, 0xae, 0x6a, 0xb4, 0x68, 0x03, 0x66, 0x64, 0xfe, 0x24, 0xfd, 0xae, 0x00, 0x53, + 0x2c, 0x2b, 0xba, 0x0a, 0x73, 0xf2, 0xf2, 0xe6, 0x93, 0xaa, 0x52, 0xdf, 0x5c, 0xa9, 0x36, 0xab, + 0xf2, 0x46, 0x7d, 0x73, 0xb9, 0x59, 0x15, 0x27, 0xd0, 0x15, 0x40, 0x5e, 0x42, 0x65, 0x6b, 0xb3, + 0x51, 0x6f, 0x34, 0xab, 0x9b, 0x4d, 0x51, 0xa0, 0x7b, 0x2a, 0x94, 0x1e, 0xa2, 0x26, 0xd0, 0x9b, + 0x70, 0xbd, 0x9f, 0xaa, 0x34, 0x9a, 0xcb, 0xcd, 0x86, 0x52, 0x6d, 0x34, 0xeb, 0x1b, 0xcb, 0xcd, + 0xea, 0x8a, 0x98, 0x1c, 0x91, 0x8b, 0xbc, 0x44, 0x96, 0xab, 0x95, 0xa6, 0x98, 0x92, 0x5c, 0xb8, + 0x2c, 0x63, 0xcd, 0x6a, 0x77, 0xba, 0x2e, 0x26, 0xa5, 0x74, 0xe2, 0x1c, 0x29, 0x57, 0x61, 0x5a, + 0xb7, 0x4f, 0x14, 0xbb, 0x6b, 0xf2, 0x71, 0x32, 0xa5, 0xdb, 0x27, 0x72, 0xd7, 0x94, 0xfe, 0xa1, + 0x00, 0x57, 0xfa, 0x5f, 0x1b, 0x67, 0x27, 0x7c, 0x0a, 0x59, 0x55, 0xd7, 0xb1, 0xae, 0xe8, 0xb8, + 0xe5, 0xaa, 0xdc, 0x24, 0xba, 0x1f, 0x92, 0xc4, 0xb7, 0x02, 0x4b, 0xfe, 0x56, 0xe0, 0xc6, 0xb3, + 0x4a, 0x85, 0x16, 0x64, 0x85, 0x70, 0x78, 0xd3, 0x0f, 0x15, 0x42, 0x29, 0xd2, 0x8f, 0x53, 0x90, + 0xab, 0x9a, 0x7a, 0xf3, 0x38, 0xd6, 0xb5, 0xe4, 0x0a, 0x4c, 0x69, 0x56, 0xbb, 0x6d, 0xb8, 0x9e, + 0x82, 0xd8, 0x13, 0xfa, 0x4e, 0xc8, 0x94, 0x4d, 0x8e, 0x61, 0xd0, 0x05, 0x46, 0x2c, 0xfa, 0x2d, + 0xb8, 0x4a, 0x66, 0x4d, 0xdb, 0x54, 0x5b, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xb1, 0xbf, 0x8f, 0x6d, + 0xbe, 0xfd, 0x78, 0x37, 0xa2, 0x9c, 0x75, 0xce, 0x51, 0xa1, 0x0c, 0x4d, 0x96, 0x5f, 0xbe, 0x6c, + 0x44, 0x91, 0xd1, 0x87, 0x00, 0x64, 0x29, 0xa2, 0x5b, 0x9a, 0x0e, 0x9f, 0x8f, 0x86, 0xed, 0x69, + 0x7a, 0x53, 0x10, 0x61, 0x20, 0xcf, 0x0e, 0x7a, 0x0a, 0xa2, 0x61, 0x2a, 0x7b, 0x2d, 0x63, 0xff, + 0xc0, 0x55, 0x5e, 0xda, 0x86, 0x8b, 0x9d, 0xe2, 0x2c, 0x95, 0x11, 0xd5, 0xd4, 0x0d, 0xbe, 0x35, + 0xab, 0x3f, 0x27, 0x39, 0xb9, 0xb4, 0xbc, 0x61, 0x3e, 0xa6, 0xfc, 0x94, 0xe8, 0xa0, 0x25, 0x02, + 0x85, 0x5e, 0x74, 0x0d, 0x1b, 0x2b, 0x0f, 0x3a, 0x1a, 0xdd, 0x07, 0x49, 0x97, 0xf3, 0x67, 0xa7, + 0x8b, 0x20, 0x33, 0xf2, 0x83, 0xed, 0x0a, 0x81, 0x46, 0xec, 0x77, 0x47, 0x23, 0x6a, 0xef, 0x58, + 0x86, 0x63, 0x99, 0xc5, 0x0c, 0x53, 0x3b, 0x7b, 0x42, 0xf7, 0x40, 0x74, 0x8f, 0x4d, 0xe5, 0x00, + 0xab, 0xb6, 0xbb, 0x8b, 0x55, 0x97, 0xac, 0xcf, 0x40, 0x73, 0x14, 0xdc, 0x63, 0xb3, 0x16, 0x22, + 0xaf, 0xa6, 0xd2, 0xd3, 0x62, 0x7a, 0x35, 0x95, 0x4e, 0x8b, 0x19, 0xe9, 0xdf, 0x0b, 0x90, 0xf7, + 0xfa, 0x46, 0x9c, 0xdd, 0xf8, 0x2e, 0x88, 0x96, 0x89, 0x95, 0xce, 0x81, 0xea, 0x60, 0xde, 0x96, + 0x7c, 0x75, 0xc8, 0x5b, 0x26, 0xde, 0x26, 0x64, 0xd6, 0x32, 0x68, 0x1b, 0x66, 0x1d, 0x57, 0xdd, + 0x37, 0xcc, 0x7d, 0xc5, 0xdf, 0xe2, 0xa7, 0x96, 0xc5, 0x98, 0x48, 0x40, 0xe4, 0xdc, 0x3e, 0xbd, + 0xc7, 0xa4, 0xf8, 0xa5, 0x00, 0xb3, 0xcb, 0x7a, 0xdb, 0x30, 0x1b, 0x9d, 0x96, 0x11, 0xeb, 0x06, + 0xc3, 0x9b, 0x90, 0x71, 0x88, 0xcc, 0x60, 0x76, 0x0e, 0xe0, 0x62, 0x9a, 0xa6, 0x90, 0x69, 0x7a, + 0x1d, 0x0a, 0xf8, 0xb8, 0x63, 0xb0, 0x73, 0x05, 0x86, 0x72, 0x52, 0xe3, 0xd7, 0x2d, 0x1f, 0xf0, + 0x92, 0x24, 0x5e, 0xa7, 0x8f, 0x01, 0x85, 0xab, 0x14, 0x27, 0xd0, 0xf8, 0x18, 0xe6, 0xa8, 0xe8, + 0x1d, 0xd3, 0x89, 0x59, 0x5f, 0xd2, 0x6f, 0xc0, 0xa5, 0x5e, 0xd1, 0x71, 0x96, 0xfb, 0x39, 0x6f, + 0xe5, 0x0d, 0x6c, 0xc7, 0x8a, 0x50, 0x7d, 0x5d, 0x73, 0xc1, 0x71, 0x96, 0xf9, 0xb7, 0x05, 0x78, + 0x8d, 0xca, 0xa6, 0x47, 0x2f, 0x7b, 0xd8, 0x5e, 0xc7, 0xaa, 0x13, 0x2b, 0xbc, 0xbe, 0x09, 0x53, + 0x0c, 0x26, 0xd3, 0xfe, 0x39, 0x59, 0xce, 0x12, 0x33, 0xa3, 0xe1, 0x5a, 0x36, 0x31, 0x33, 0x78, + 0x92, 0xa4, 0xc2, 0x7c, 0x54, 0x29, 0xe2, 0xac, 0xe9, 0xdf, 0x16, 0x60, 0x96, 0x5b, 0x78, 0xa4, + 0x2b, 0x57, 0x0e, 0x88, 0x81, 0x83, 0xaa, 0x90, 0xd5, 0xe8, 0x2f, 0xc5, 0x3d, 0xe9, 0x60, 0x2a, + 0x3f, 0x3f, 0xca, 0x38, 0x64, 0x6c, 0xcd, 0x93, 0x0e, 0x26, 0x16, 0xa6, 0xf7, 0x9b, 0x28, 0x2a, + 0x54, 0xc9, 0x91, 0xe6, 0x25, 0x1d, 0x47, 0x34, 0xaf, 0x67, 0xa7, 0x71, 0x1d, 0xfc, 0xe3, 0x24, + 0x57, 0x02, 0x7b, 0x07, 0xcf, 0x1e, 0xab, 0x41, 0xf1, 0x09, 0x5c, 0x09, 0x6d, 0x9d, 0x87, 0x2b, + 0x9e, 0xb8, 0x40, 0xc5, 0x43, 0xdb, 0xef, 0x01, 0x15, 0x7d, 0x0c, 0xa1, 0x0d, 0x76, 0x85, 0xd5, + 0xc9, 0x83, 0x2a, 0x17, 0x51, 0xc7, 0x6c, 0x20, 0x85, 0xd1, 0x1d, 0x54, 0x81, 0x34, 0x3e, 0xee, + 0x28, 0x3a, 0x76, 0x34, 0x3e, 0x71, 0x49, 0x51, 0x02, 0x49, 0x51, 0x06, 0x8c, 0xf7, 0x69, 0x7c, + 0xdc, 0x21, 0x44, 0xb4, 0x43, 0xd6, 0x4d, 0x6f, 0x5d, 0xa7, 0xc5, 0x76, 0xce, 0xc7, 0x02, 0x41, + 0x4f, 0xe1, 0xe2, 0x0a, 0xfe, 0x92, 0xce, 0x44, 0x48, 0x5f, 0x08, 0x70, 0x2d, 0xb2, 0xd5, 0xe2, + 0x5c, 0xc8, 0x3e, 0x84, 0x14, 0xad, 0x7c, 0xe2, 0x82, 0x95, 0xa7, 0x5c, 0xd2, 0x67, 0x09, 0x3e, + 0xc6, 0x65, 0xdc, 0xb2, 0x88, 0x62, 0x63, 0xdf, 0x42, 0xdb, 0x82, 0xdc, 0x91, 0xe5, 0x62, 0xdb, + 0x6f, 0xf6, 0xc4, 0x85, 0x9b, 0x7d, 0x86, 0x0a, 0xf0, 0x5a, 0xfc, 0x19, 0xcc, 0x9a, 0x96, 0xa9, + 0xf4, 0x0a, 0xbd, 0x78, 0x5f, 0x2a, 0x98, 0x96, 0xf9, 0x2c, 0x24, 0xd7, 0x9f, 0x67, 0xfa, 0x34, + 0x11, 0xe7, 0x3c, 0xf3, 0x23, 0x01, 0xe6, 0x7c, 0x4b, 0x27, 0x66, 0x73, 0xf7, 0x1d, 0x48, 0x9a, + 0xd6, 0xcb, 0x8b, 0x6c, 0x51, 0x92, 0xfc, 0x64, 0xd5, 0xeb, 0x2d, 0x51, 0x9c, 0xf5, 0xfd, 0x97, + 0x09, 0xc8, 0x3c, 0xa9, 0xc4, 0x59, 0xcb, 0x0f, 0xf9, 0xf6, 0x37, 0x6b, 0xef, 0xa8, 0xde, 0xee, + 0xbf, 0xaf, 0xf4, 0xa4, 0xb2, 0x86, 0x4f, 0xbc, 0xde, 0x4e, 0xb8, 0xd0, 0x32, 0x64, 0x7a, 0x37, + 0x4a, 0xc7, 0xd4, 0x54, 0xc0, 0x35, 0x8f, 0x61, 0x92, 0xca, 0xf5, 0x5c, 0x2d, 0x84, 0x08, 0x57, + 0x0b, 0xf2, 0x1a, 0xdf, 0x52, 0x4c, 0x5c, 0xe4, 0x35, 0x21, 0x13, 0x71, 0x52, 0x9c, 0x92, 0x9e, + 0x02, 0x90, 0xea, 0xc4, 0xd9, 0x24, 0x3f, 0x4c, 0x42, 0x7e, 0xbb, 0xeb, 0x1c, 0xc4, 0xdc, 0xfb, + 0x2a, 0x00, 0x9d, 0xae, 0x73, 0x40, 0x46, 0xe4, 0xb1, 0xc9, 0xeb, 0x7c, 0x8e, 0x17, 0x87, 0x57, + 0x69, 0xc6, 0xd7, 0x3c, 0x36, 0x51, 0x8d, 0x0b, 0xc1, 0x4a, 0xe0, 0x0a, 0x72, 0x73, 0x14, 0xb2, + 0x6c, 0x1e, 0x9b, 0x1b, 0xd8, 0x87, 0x94, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x84, 0x69, 0xf2, 0xa0, + 0xb8, 0xd6, 0x45, 0x9a, 0x79, 0x8a, 0xf0, 0x34, 0x2d, 0xf4, 0x01, 0x64, 0x18, 0x37, 0x59, 0xfd, + 0xa6, 0xe8, 0xea, 0x17, 0x55, 0x17, 0xae, 0x46, 0xba, 0xee, 0xa5, 0x29, 0x2b, 0x59, 0xeb, 0x2e, + 0xc1, 0xe4, 0x9e, 0x65, 0x6b, 0xde, 0x61, 0x2e, 0x7b, 0x60, 0xed, 0xc9, 0x20, 0xcd, 0x6a, 0x2a, + 0x9d, 0x11, 0x41, 0xfa, 0x5d, 0x01, 0x0a, 0x7e, 0x43, 0xc4, 0xb9, 0x20, 0x54, 0x7a, 0xb4, 0x78, + 0xf1, 0xa6, 0x20, 0x0a, 0x94, 0xfe, 0x35, 0xb5, 0x88, 0x34, 0xeb, 0x88, 0xb6, 0x4c, 0x9c, 0x3d, + 0xe5, 0x03, 0xe6, 0xe8, 0x93, 0xb8, 0x68, 0xeb, 0x52, 0x9f, 0x9f, 0x07, 0x70, 0xc9, 0x68, 0x93, + 0xf9, 0xdc, 0x70, 0x5b, 0x27, 0x1c, 0xb6, 0xb9, 0xd8, 0x3b, 0x35, 0x9e, 0x0b, 0xd2, 0x2a, 0x5e, + 0x92, 0xf4, 0xfb, 0x74, 0xb7, 0x3a, 0xa8, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x66, 0xa2, 0x89, + 0x59, 0x73, 0x41, 0x6d, 0xcf, 0xf8, 0xac, 0x44, 0xe1, 0xbf, 0x97, 0x80, 0xc2, 0xd3, 0x2e, 0xb6, + 0x4f, 0xbe, 0x49, 0xea, 0xbe, 0x0d, 0x85, 0x97, 0xaa, 0xe1, 0x2a, 0x7b, 0x96, 0xad, 0x74, 0x3b, + 0xba, 0xea, 0x7a, 0xde, 0x26, 0x39, 0x42, 0x7e, 0x6c, 0xd9, 0x3b, 0x94, 0x88, 0x30, 0xa0, 0x43, + 0xd3, 0x7a, 0x69, 0x2a, 0x84, 0x4c, 0x81, 0xf2, 0xb1, 0xc9, 0xb7, 0x90, 0xcb, 0xef, 0xfe, 0xbb, + 0xd3, 0xc5, 0x47, 0x63, 0xf9, 0x90, 0x51, 0x7f, 0xb9, 0x6e, 0xd7, 0xd0, 0x4b, 0x3b, 0x3b, 0xf5, + 0x15, 0x59, 0xa4, 0x22, 0x9f, 0x33, 0x89, 0xcd, 0x63, 0xd3, 0x91, 0xfe, 0x6e, 0x02, 0xc4, 0x40, + 0x47, 0x71, 0x36, 0x64, 0x15, 0xb2, 0x2f, 0xba, 0xd8, 0x36, 0x5e, 0xa1, 0x19, 0x81, 0x33, 0x92, + 0x69, 0xe7, 0x3e, 0xcc, 0xba, 0xc7, 0xa6, 0xc2, 0x3c, 0xfc, 0x98, 0xe3, 0x87, 0xe7, 0xb0, 0x50, + 0x70, 0x49, 0x99, 0x09, 0x9d, 0x3a, 0x7d, 0x38, 0xe8, 0x13, 0x98, 0xe9, 0xd1, 0x56, 0xf2, 0xab, + 0x69, 0x2b, 0xfb, 0x32, 0xa4, 0xa8, 0x3f, 0x14, 0x00, 0x51, 0x45, 0xd5, 0xd9, 0x1e, 0xff, 0x37, + 0xa5, 0x3f, 0xdd, 0x05, 0x91, 0xfa, 0x63, 0x2a, 0xc6, 0x9e, 0xd2, 0x36, 0x1c, 0xc7, 0x30, 0xf7, + 0x79, 0x87, 0xca, 0x53, 0x7a, 0x7d, 0x6f, 0x83, 0x51, 0xa5, 0xbf, 0x04, 0x73, 0x3d, 0x15, 0x88, + 0xb3, 0xb1, 0x6f, 0xc0, 0xcc, 0x1e, 0x3b, 0x82, 0xa5, 0xc2, 0xf9, 0xf6, 0x60, 0x96, 0xd2, 0xd8, + 0xfb, 0xa4, 0xff, 0x92, 0x80, 0x4b, 0x32, 0x76, 0xac, 0xd6, 0x11, 0x8e, 0x5f, 0x85, 0x35, 0xe0, + 0x67, 0x2f, 0xca, 0x2b, 0x69, 0x32, 0xc3, 0x98, 0xd9, 0x32, 0xd7, 0xbb, 0xc7, 0xfe, 0xe6, 0xe8, + 0x1e, 0x3b, 0xb8, 0xab, 0xce, 0x77, 0xea, 0x52, 0x3d, 0x3b, 0x75, 0x16, 0x14, 0xd8, 0xe9, 0xb1, + 0xae, 0x38, 0xf8, 0x85, 0xd9, 0x6d, 0x7b, 0x60, 0xa8, 0x34, 0xaa, 0x90, 0x75, 0xc6, 0xd2, 0xc0, + 0x2f, 0x36, 0xbb, 0x6d, 0x6a, 0x3b, 0x97, 0xaf, 0x90, 0xf2, 0x9e, 0x9d, 0x2e, 0xe6, 0x7b, 0xd2, + 0x1c, 0x39, 0x6f, 0xf8, 0xcf, 0x44, 0xba, 0xf4, 0x7d, 0xb8, 0xdc, 0xa7, 0xec, 0x38, 0x2d, 0x9e, + 0x7f, 0x9e, 0x84, 0xd7, 0x7a, 0xc5, 0xc7, 0x0d, 0x71, 0xbe, 0xe9, 0x0d, 0x5a, 0x83, 0x5c, 0xdb, + 0x30, 0x5f, 0x6d, 0xf7, 0x72, 0xa6, 0x6d, 0x98, 0x3e, 0x2d, 0xaa, 0x6b, 0x4c, 0x7d, 0xad, 0x5d, + 0x43, 0x85, 0xf9, 0xa8, 0xb6, 0x8b, 0xb3, 0x7f, 0x7c, 0x26, 0xc0, 0x4c, 0xdc, 0xdb, 0x72, 0xaf, + 0xe6, 0x05, 0x27, 0x35, 0x21, 0xf7, 0x35, 0xec, 0xe3, 0xfd, 0x9e, 0x00, 0xa8, 0x69, 0x77, 0x4d, + 0x02, 0x6a, 0xd7, 0xad, 0xfd, 0x38, 0xab, 0x79, 0x09, 0x26, 0x0d, 0x53, 0xc7, 0xc7, 0xb4, 0x9a, + 0x29, 0x99, 0x3d, 0xf4, 0x1c, 0x25, 0x26, 0xc7, 0x3a, 0x4a, 0x94, 0x3e, 0x81, 0xb9, 0x9e, 0x22, + 0xc6, 0x59, 0xff, 0x3f, 0x49, 0xc0, 0x1c, 0xaf, 0x48, 0xec, 0x3b, 0x98, 0xdf, 0x86, 0xc9, 0x16, + 0x91, 0x39, 0xa2, 0x9d, 0xe9, 0x3b, 0xbd, 0x76, 0xa6, 0x99, 0xd1, 0x77, 0x01, 0x3a, 0x36, 0x3e, + 0x52, 0x18, 0x6b, 0x72, 0x2c, 0xd6, 0x0c, 0xe1, 0xa0, 0x04, 0xf4, 0xb9, 0x00, 0x05, 0x32, 0xa0, + 0x3b, 0xb6, 0xd5, 0xb1, 0x1c, 0x62, 0xb3, 0x38, 0xe3, 0xc1, 0x9c, 0xa7, 0x67, 0xa7, 0x8b, 0xb9, + 0x0d, 0xc3, 0xdc, 0xe6, 0x8c, 0xcd, 0xc6, 0xd8, 0x0e, 0xfe, 0x5e, 0x98, 0x43, 0xa9, 0xd2, 0xb2, + 0xb4, 0xc3, 0xe0, 0x70, 0x8c, 0xcc, 0x2c, 0xbe, 0x38, 0x47, 0xfa, 0xb9, 0x00, 0x97, 0xbe, 0xb6, + 0xed, 0xe2, 0xff, 0x1f, 0xca, 0x96, 0x9e, 0x81, 0x48, 0x7f, 0xd4, 0xcd, 0x3d, 0x2b, 0xce, 0x8d, + 0xfb, 0xff, 0x2d, 0xc0, 0x6c, 0x48, 0x70, 0x9c, 0x06, 0xce, 0xab, 0xea, 0x29, 0xc7, 0xdc, 0x61, + 0xdc, 0xf1, 0x54, 0x25, 0xcf, 0xf0, 0xec, 0xac, 0x53, 0x96, 0x60, 0x06, 0x93, 0x59, 0x8c, 0x6e, + 0xf1, 0xee, 0xb2, 0x20, 0x93, 0xbe, 0x1d, 0xfd, 0xac, 0x9f, 0xa1, 0x7c, 0x22, 0xfd, 0x06, 0xb1, + 0xb0, 0xc2, 0x83, 0x32, 0xce, 0x21, 0xff, 0xcf, 0x12, 0x70, 0xa5, 0xc2, 0x8e, 0xc0, 0x3d, 0x9f, + 0x90, 0x38, 0x3b, 0x62, 0x11, 0xa6, 0x8f, 0xb0, 0xed, 0x18, 0x16, 0x5b, 0xed, 0x73, 0xb2, 0xf7, + 0x88, 0xe6, 0x21, 0xed, 0x98, 0x6a, 0xc7, 0x39, 0xb0, 0xbc, 0xe3, 0x44, 0xff, 0xd9, 0xf7, 0x5f, + 0x99, 0x7c, 0x75, 0xff, 0x95, 0xa9, 0xd1, 0xfe, 0x2b, 0xd3, 0x5f, 0xc1, 0x7f, 0x85, 0x9f, 0xdd, + 0xfd, 0x1b, 0x01, 0xae, 0x0e, 0x68, 0x2e, 0xce, 0xce, 0xf9, 0x03, 0xc8, 0x6a, 0x5c, 0x30, 0x59, + 0x1f, 0xd8, 0xc1, 0x64, 0x9d, 0x64, 0x7b, 0x45, 0xe8, 0x73, 0x76, 0xba, 0x08, 0x5e, 0x51, 0xeb, + 0x2b, 0x5c, 0x39, 0xe4, 0xb7, 0x2e, 0xfd, 0x57, 0x80, 0x42, 0xf5, 0x98, 0x6d, 0xca, 0x37, 0x98, + 0x55, 0x82, 0x1e, 0x43, 0xba, 0x63, 0x5b, 0x47, 0x86, 0x57, 0x8d, 0x7c, 0x8f, 0xf3, 0x82, 0x57, + 0x8d, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0xfb, 0xbc, 0xa8, 0x09, 0x99, 0x75, 0x4b, 0x53, 0x5b, 0x8f, + 0x8d, 0x96, 0x37, 0xd0, 0xde, 0x3e, 0x5f, 0x50, 0xc9, 0xe7, 0xd9, 0x56, 0xdd, 0x03, 0xaf, 0x11, + 0x7c, 0x22, 0xaa, 0x43, 0xba, 0xe6, 0xba, 0x1d, 0x92, 0xc8, 0xc7, 0xdf, 0x9d, 0x31, 0x84, 0x12, + 0x16, 0xcf, 0xe3, 0xd6, 0x63, 0x47, 0x4d, 0x98, 0x7d, 0x42, 0xe3, 0xc7, 0x2a, 0x2d, 0xab, 0xab, + 0x57, 0x2c, 0x73, 0xcf, 0xd8, 0xe7, 0xcb, 0xc4, 0xed, 0x31, 0x64, 0x3e, 0xa9, 0x34, 0xe4, 0x41, + 0x01, 0x68, 0x19, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0x33, 0x23, 0x6f, 0x8d, 0x21, 0xac, 0xf1, 0x48, + 0xf6, 0xd9, 0xd0, 0x2a, 0x64, 0x97, 0x3f, 0xed, 0xda, 0x98, 0x4b, 0x99, 0x1a, 0xea, 0x39, 0xd1, + 0x2f, 0x85, 0x72, 0xc9, 0x61, 0x66, 0xf4, 0x7d, 0x28, 0x10, 0xbd, 0x35, 0xd5, 0xdd, 0x96, 0x27, + 0x2f, 0x4d, 0xe5, 0x7d, 0x6b, 0x0c, 0x79, 0x3e, 0xa7, 0x77, 0x24, 0xd0, 0x27, 0x6a, 0x5e, 0x86, + 0x5c, 0x4f, 0x7b, 0x21, 0x04, 0xa9, 0x0e, 0x69, 0x1a, 0x81, 0xba, 0x21, 0xd1, 0xdf, 0xe8, 0x2d, + 0x98, 0x36, 0x2d, 0x1d, 0x7b, 0x9d, 0x39, 0x57, 0xbe, 0x74, 0x76, 0xba, 0x38, 0xb5, 0x69, 0xe9, + 0xcc, 0xd6, 0xe1, 0xbf, 0xe4, 0x29, 0x92, 0xa9, 0xae, 0xcf, 0x5f, 0x87, 0x14, 0x69, 0x22, 0x32, + 0x87, 0xec, 0xaa, 0x0e, 0xde, 0xb1, 0x0d, 0x2e, 0xcd, 0x7b, 0x9c, 0xff, 0x07, 0x09, 0x48, 0x34, + 0x1e, 0x11, 0x6b, 0x7e, 0xb7, 0xab, 0x1d, 0x62, 0x97, 0xa7, 0xf3, 0x27, 0x6a, 0xe5, 0xdb, 0x78, + 0xcf, 0x60, 0x46, 0x57, 0x46, 0xe6, 0x4f, 0xe8, 0x0d, 0x00, 0x55, 0xd3, 0xb0, 0xe3, 0x28, 0x5e, + 0x08, 0x60, 0x46, 0xce, 0x30, 0xca, 0x1a, 0x3e, 0x21, 0x6c, 0x0e, 0xd6, 0x6c, 0xec, 0x7a, 0x3e, + 0x54, 0xec, 0x89, 0xb0, 0xb9, 0xb8, 0xdd, 0x51, 0x5c, 0xeb, 0x10, 0x9b, 0xb4, 0x49, 0x33, 0x64, + 0x56, 0x68, 0x77, 0x9a, 0x84, 0x40, 0x26, 0x34, 0x6c, 0xea, 0xc1, 0xec, 0x93, 0x91, 0xfd, 0x67, + 0x22, 0xd2, 0xc6, 0xfb, 0x06, 0x0f, 0xa0, 0xcb, 0xc8, 0xfc, 0x89, 0x68, 0x49, 0xed, 0xba, 0x07, + 0xb4, 0x25, 0x32, 0x32, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xb9, 0x5d, 0x2a, 0xd8, 0xd4, 0x14, 0x3a, + 0x0f, 0x66, 0x68, 0x72, 0x8e, 0x91, 0xab, 0xa6, 0x46, 0x66, 0x3d, 0xf4, 0x08, 0x38, 0x41, 0x39, + 0x6c, 0x3b, 0x44, 0xa7, 0x40, 0x72, 0x95, 0x0b, 0x67, 0xa7, 0x8b, 0xd9, 0x06, 0x4d, 0x58, 0xdb, + 0x68, 0x90, 0xb5, 0x84, 0xe5, 0x5a, 0x6b, 0x3b, 0x75, 0x7d, 0xfe, 0x6f, 0x0a, 0x90, 0x7c, 0x52, + 0x69, 0x5c, 0x58, 0x65, 0x5e, 0x41, 0x93, 0xa1, 0x82, 0xde, 0x81, 0xc2, 0xae, 0xd1, 0x6a, 0x19, + 0xe6, 0x3e, 0xb1, 0xaf, 0x7e, 0x80, 0x35, 0x4f, 0x61, 0x79, 0x4e, 0xde, 0x66, 0x54, 0x74, 0x1d, + 0xb2, 0x9a, 0x8d, 0x75, 0x6c, 0xba, 0x86, 0xda, 0x72, 0xb8, 0xe6, 0xc2, 0xa4, 0xf9, 0xbf, 0x2c, + 0xc0, 0x24, 0xed, 0xac, 0xe8, 0x75, 0xc8, 0x68, 0x96, 0xe9, 0xaa, 0x86, 0xc9, 0x67, 0x9d, 0x8c, + 0x1c, 0x10, 0x86, 0x16, 0xef, 0x06, 0xcc, 0xa8, 0x9a, 0x66, 0x75, 0x4d, 0x57, 0x31, 0xd5, 0x36, + 0xe6, 0xc5, 0xcc, 0x72, 0xda, 0xa6, 0xda, 0xc6, 0x68, 0x11, 0xbc, 0x47, 0x3f, 0xb2, 0x33, 0x23, + 0x03, 0x27, 0xad, 0xe1, 0x93, 0x79, 0x0c, 0x19, 0xbf, 0x57, 0x93, 0xfa, 0x76, 0x1d, 0xbf, 0x04, + 0xf4, 0x37, 0x7a, 0x1b, 0x2e, 0xbd, 0xe8, 0xaa, 0x2d, 0x63, 0x8f, 0x6e, 0x7e, 0x51, 0x2f, 0x75, + 0xfa, 0x32, 0x56, 0x14, 0xe4, 0xa7, 0x51, 0x09, 0xf4, 0x9d, 0xde, 0x20, 0x48, 0x06, 0x83, 0x80, + 0xb9, 0xec, 0x48, 0x27, 0x30, 0x2b, 0x63, 0xd7, 0x3e, 0x69, 0xb2, 0x60, 0xd7, 0xea, 0x11, 0x36, + 0x5d, 0x52, 0x77, 0xab, 0x83, 0x99, 0x93, 0x88, 0x57, 0x77, 0x9f, 0x80, 0x6e, 0x41, 0x5e, 0x75, + 0x49, 0x77, 0x73, 0x15, 0xb3, 0xdb, 0xde, 0xc5, 0x36, 0x73, 0x05, 0x90, 0x73, 0x9c, 0xba, 0x49, + 0x89, 0x3c, 0x22, 0xc3, 0x3e, 0x51, 0xe8, 0x3e, 0x11, 0x7f, 0x35, 0x50, 0x52, 0x95, 0x50, 0xa4, + 0x7b, 0x70, 0x99, 0xd4, 0xb3, 0x6a, 0x6a, 0xf6, 0x49, 0x87, 0x48, 0xde, 0xa2, 0x7f, 0x1d, 0x24, + 0x86, 0xce, 0x69, 0xe8, 0xf1, 0x8c, 0xf4, 0xb3, 0x29, 0xc8, 0x55, 0x8f, 0x3b, 0x96, 0x1d, 0xeb, + 0xae, 0x4e, 0x19, 0xa6, 0x39, 0xf0, 0x1d, 0x71, 0x14, 0xdb, 0x37, 0x03, 0x79, 0xe7, 0xd0, 0x9c, + 0x11, 0x95, 0x01, 0x98, 0x43, 0x25, 0xf5, 0xc3, 0x49, 0x5e, 0xe0, 0xe4, 0x88, 0xb2, 0xd1, 0x60, + 0x83, 0x4d, 0xc8, 0xb6, 0x8f, 0x34, 0x4d, 0xd9, 0x33, 0x5a, 0x2e, 0xf7, 0x4b, 0x8b, 0x76, 0xa1, + 0xde, 0x78, 0x56, 0xa9, 0x3c, 0xa6, 0x99, 0x98, 0x3f, 0x57, 0xf0, 0x2c, 0x03, 0x91, 0xc0, 0x7e, + 0xa3, 0x6f, 0x01, 0x0f, 0x7c, 0x51, 0x1c, 0x2f, 0x8c, 0xad, 0x9c, 0x3b, 0x3b, 0x5d, 0xcc, 0xc8, + 0x94, 0xda, 0x68, 0x34, 0xe5, 0x0c, 0xcb, 0xd0, 0x70, 0xdc, 0x8b, 0x84, 0x3a, 0x4c, 0x8f, 0x1f, + 0xea, 0xf0, 0xd7, 0x04, 0xb8, 0xc2, 0x75, 0xa4, 0xec, 0x52, 0xf7, 0x6e, 0xb5, 0x65, 0xb8, 0x27, + 0xca, 0xe1, 0x51, 0x31, 0x4d, 0x4d, 0x9e, 0xef, 0x44, 0xea, 0x3a, 0xd4, 0xc4, 0x25, 0x4f, 0xe3, + 0x27, 0xeb, 0x9c, 0x79, 0xed, 0xa8, 0x6a, 0xba, 0xf6, 0x49, 0xf9, 0xea, 0xd9, 0xe9, 0xe2, 0xdc, + 0x60, 0xea, 0x33, 0x79, 0xce, 0x19, 0x64, 0x41, 0x35, 0x00, 0xec, 0x77, 0x31, 0x3a, 0x83, 0x45, + 0x2f, 0x5d, 0x91, 0x7d, 0x51, 0x0e, 0xf1, 0xa2, 0xbb, 0x20, 0xf2, 0xd0, 0x92, 0x3d, 0xa3, 0x85, + 0x15, 0xc7, 0xf8, 0x14, 0xd3, 0xb9, 0x2e, 0x29, 0xe7, 0x19, 0x9d, 0x88, 0x68, 0x18, 0x9f, 0x62, + 0xf4, 0x00, 0x2e, 0x07, 0x2d, 0xa0, 0xec, 0xe2, 0x96, 0xf5, 0x92, 0x65, 0xcf, 0xd2, 0xec, 0xc8, + 0xd7, 0x7e, 0x99, 0x24, 0x11, 0x96, 0xf9, 0x1f, 0x40, 0x71, 0x58, 0x85, 0xc3, 0x03, 0x22, 0xc3, + 0xce, 0x2b, 0xdf, 0xeb, 0xdd, 0xac, 0x18, 0xa3, 0xe3, 0xf2, 0x0d, 0x8b, 0xf7, 0x13, 0xef, 0x09, + 0xd2, 0xdf, 0x4f, 0x40, 0xae, 0xdc, 0x6d, 0x1d, 0x6e, 0x75, 0x1a, 0x2c, 0x2c, 0x1f, 0x5d, 0x83, + 0x8c, 0xae, 0xba, 0x2a, 0x2b, 0xa4, 0xc0, 0x42, 0xcc, 0x08, 0x81, 0xd6, 0xe6, 0x0e, 0x14, 0x42, + 0xbe, 0x20, 0xdc, 0xe3, 0x9d, 0x56, 0x3b, 0x20, 0x53, 0xa7, 0xf4, 0xf7, 0xa0, 0x18, 0xca, 0x48, + 0x77, 0x16, 0x14, 0x6c, 0xba, 0xb6, 0x81, 0xd9, 0xee, 0x58, 0x52, 0x0e, 0x39, 0xac, 0xd4, 0x49, + 0x72, 0x95, 0xa5, 0xa2, 0x26, 0xcc, 0x90, 0x8c, 0x27, 0x0a, 0x9d, 0x05, 0xbd, 0xdd, 0xcb, 0x07, + 0x11, 0xd5, 0xea, 0x29, 0x77, 0x89, 0xea, 0xa7, 0x42, 0x79, 0xe8, 0x4f, 0x39, 0x8b, 0x03, 0xca, + 0xfc, 0x47, 0x20, 0xf6, 0x67, 0x08, 0xeb, 0x32, 0xc5, 0x74, 0x79, 0x29, 0xac, 0xcb, 0x64, 0x48, + 0x4f, 0xab, 0xa9, 0x74, 0x4a, 0x9c, 0x94, 0x7e, 0x9e, 0x84, 0xbc, 0xd7, 0x33, 0xe3, 0x34, 0xab, + 0xcb, 0x30, 0x49, 0xfa, 0x91, 0xe7, 0x5e, 0x71, 0x7b, 0xc4, 0x80, 0xe0, 0x3e, 0xd6, 0xa4, 0x7f, + 0x79, 0x08, 0x90, 0xb2, 0xc6, 0x31, 0xfd, 0xcc, 0xff, 0xa9, 0x00, 0x29, 0x6a, 0xc9, 0x3e, 0x80, + 0x14, 0x8d, 0xcb, 0x17, 0x46, 0xc6, 0xe5, 0x7b, 0xc7, 0xf3, 0x24, 0xab, 0xbf, 0xb0, 0x24, 0x42, + 0xd6, 0x55, 0x99, 0xfa, 0xf7, 0x58, 0xb6, 0x8b, 0x75, 0x6e, 0x29, 0x5e, 0x3f, 0xaf, 0x1d, 0x3d, + 0x4b, 0xd8, 0xe3, 0x43, 0xaf, 0x41, 0x92, 0xcc, 0x5d, 0xd3, 0xec, 0xa8, 0xfe, 0xec, 0x74, 0x31, + 0x49, 0x66, 0x2d, 0x42, 0x43, 0x4b, 0x90, 0xed, 0x9d, 0x4d, 0x88, 0xb1, 0x41, 0xa7, 0xc3, 0xd0, + 0x4c, 0x00, 0x2d, 0x7f, 0x08, 0x31, 0x94, 0xc4, 0xda, 0x92, 0x1f, 0xd2, 0xff, 0xb6, 0xc0, 0x7d, + 0x12, 0x1b, 0x1a, 0x59, 0xb3, 0xec, 0x38, 0x17, 0x95, 0x7b, 0x20, 0xda, 0xaa, 0xa9, 0x5b, 0x6d, + 0xe3, 0x53, 0xcc, 0x50, 0xb9, 0xc3, 0x8f, 0x2b, 0x0a, 0x3e, 0x9d, 0xc2, 0x67, 0x47, 0xfa, 0xcf, + 0x02, 0xf7, 0x5f, 0xf4, 0x8b, 0x11, 0xef, 0xa1, 0x72, 0x96, 0x6f, 0xe9, 0x99, 0x7b, 0x96, 0xe7, + 0x7e, 0xf1, 0xfa, 0x30, 0x67, 0xa3, 0xba, 0xb9, 0x67, 0x79, 0xc7, 0x63, 0xb6, 0x47, 0x70, 0xe6, + 0x7f, 0x1d, 0x26, 0x69, 0xf2, 0x2b, 0xf4, 0x0d, 0xdf, 0x67, 0x36, 0x21, 0x26, 0xa5, 0x3f, 0x4e, + 0xc0, 0x9b, 0xb4, 0xaa, 0xcf, 0xb0, 0x6d, 0xec, 0x9d, 0x6c, 0xdb, 0x96, 0x8b, 0x35, 0x17, 0xeb, + 0xc1, 0xae, 0x54, 0x8c, 0x4d, 0xa0, 0x43, 0x86, 0x9f, 0xe7, 0x19, 0x3a, 0xbf, 0x39, 0xe3, 0xc9, + 0x57, 0x43, 0xab, 0x69, 0x76, 0x0e, 0x58, 0x5f, 0x91, 0xd3, 0x4c, 0x72, 0x5d, 0x47, 0xcb, 0x90, + 0xe9, 0x78, 0xd5, 0xb8, 0x90, 0xcb, 0x88, 0xcf, 0x85, 0xd6, 0xa0, 0xc0, 0x0b, 0xaa, 0xb6, 0x8c, + 0x23, 0xac, 0xa8, 0xee, 0x45, 0x86, 0x70, 0x8e, 0xf1, 0x2e, 0x13, 0xd6, 0x65, 0x57, 0xfa, 0x1b, + 0x29, 0xb8, 0x75, 0x8e, 0x8a, 0xe3, 0xec, 0x5e, 0xf3, 0x90, 0x3e, 0x22, 0x2f, 0x32, 0x78, 0xed, + 0xd3, 0xb2, 0xff, 0x8c, 0x76, 0x7b, 0xd6, 0x81, 0x3d, 0xd5, 0x68, 0x91, 0x75, 0x83, 0x39, 0xe9, + 0x0d, 0x77, 0x03, 0x8a, 0x76, 0x7a, 0x0b, 0xad, 0x18, 0x8f, 0xa9, 0x20, 0x9a, 0xcd, 0x41, 0x9f, + 0x09, 0x30, 0xcf, 0x5e, 0xc8, 0x3c, 0xc5, 0xfa, 0x5e, 0x93, 0xa2, 0xaf, 0x59, 0x89, 0x78, 0xcd, + 0x58, 0x3a, 0x2a, 0x85, 0xde, 0xc5, 0x0b, 0x52, 0x0c, 0xbf, 0x2d, 0x5c, 0x94, 0xf9, 0xdf, 0x11, + 0x20, 0x1b, 0x22, 0xa0, 0xdb, 0x03, 0x71, 0x39, 0xd9, 0xb3, 0xa8, 0x60, 0x9c, 0x5b, 0x03, 0xc1, + 0x38, 0xe5, 0xf4, 0x97, 0xa7, 0x8b, 0x29, 0x99, 0xf9, 0x7b, 0x7b, 0x61, 0x39, 0x37, 0x82, 0x6b, + 0x60, 0x92, 0x7d, 0x99, 0xbc, 0x7b, 0x60, 0x28, 0xac, 0x53, 0xbd, 0x63, 0x24, 0x0a, 0xeb, 0xc8, + 0x93, 0xf4, 0xe3, 0x04, 0xcc, 0x2e, 0xeb, 0x7a, 0xa3, 0x41, 0xa1, 0x40, 0x9c, 0x63, 0x0c, 0x41, + 0x8a, 0xd8, 0x07, 0x3c, 0x86, 0x88, 0xfe, 0x46, 0x6f, 0x01, 0xd2, 0x0d, 0x87, 0x5d, 0xa7, 0xe0, + 0x1c, 0xa8, 0xba, 0xf5, 0x32, 0x38, 0x2d, 0x9e, 0xf5, 0x52, 0x1a, 0x5e, 0x02, 0x6a, 0x00, 0x35, + 0x5a, 0x15, 0xc7, 0x55, 0xfd, 0xdd, 0xf0, 0x5b, 0x63, 0x45, 0xa5, 0x30, 0x6b, 0xd6, 0x7f, 0x94, + 0x33, 0x44, 0x0e, 0xfd, 0x49, 0x6c, 0x34, 0x83, 0x34, 0x8a, 0xab, 0xa8, 0x8e, 0x17, 0x4f, 0xc1, + 0x2e, 0x72, 0xc8, 0x33, 0xfa, 0xb2, 0xc3, 0xc2, 0x24, 0x98, 0x1b, 0x76, 0xa0, 0x9a, 0x38, 0xf7, + 0x32, 0xff, 0x8e, 0x00, 0x79, 0x19, 0xef, 0xd9, 0xd8, 0x39, 0x88, 0x53, 0xe7, 0x8f, 0x61, 0xc6, + 0x66, 0x52, 0x95, 0x3d, 0xdb, 0x6a, 0x5f, 0x64, 0xae, 0xc8, 0x72, 0xc6, 0xc7, 0xb6, 0xd5, 0xe6, + 0x53, 0xf2, 0x33, 0x28, 0xf8, 0x65, 0x8c, 0xb3, 0xf2, 0x7f, 0x8f, 0x46, 0x5c, 0x32, 0xc1, 0x71, + 0x1f, 0xdb, 0xc6, 0xab, 0x01, 0xba, 0x9f, 0x1d, 0x2e, 0x68, 0x9c, 0x6a, 0xf8, 0x4f, 0x02, 0xe4, + 0x1b, 0xdd, 0x5d, 0x76, 0x4d, 0x50, 0x7c, 0x1a, 0xa8, 0x42, 0xa6, 0x85, 0xf7, 0x5c, 0xe5, 0x95, + 0x1c, 0x88, 0xd3, 0x84, 0x95, 0xba, 0x4f, 0x3f, 0x01, 0xb0, 0x69, 0xc8, 0x11, 0x95, 0x93, 0xbc, + 0xa0, 0x9c, 0x0c, 0xe5, 0x25, 0x64, 0xb2, 0xea, 0x14, 0xfc, 0x6a, 0xc6, 0xb9, 0xbe, 0x3c, 0xef, + 0x99, 0x1d, 0x92, 0x17, 0x99, 0x1d, 0x66, 0xf9, 0x49, 0x75, 0xf4, 0x0c, 0x51, 0x82, 0x39, 0x6a, + 0x96, 0x29, 0x6a, 0xa7, 0xd3, 0x32, 0x3c, 0x9c, 0x42, 0xe7, 0x9f, 0x94, 0x3c, 0x4b, 0x93, 0x96, + 0x59, 0x0a, 0x45, 0x28, 0xe8, 0x87, 0x02, 0xcc, 0xec, 0xd9, 0x18, 0x7f, 0x8a, 0x15, 0x3a, 0x25, + 0x8f, 0x77, 0x14, 0xbf, 0x42, 0xca, 0xf0, 0x95, 0x8f, 0xea, 0xb2, 0xec, 0xc5, 0x0d, 0xf2, 0x5e, + 0xb4, 0x09, 0xa2, 0xd6, 0x62, 0x87, 0x87, 0xbe, 0x5b, 0xc0, 0xd4, 0xf8, 0x03, 0xa0, 0xc0, 0x98, + 0x03, 0xcf, 0x80, 0xa7, 0x64, 0x30, 0xa9, 0xba, 0xc2, 0xaf, 0x66, 0xa3, 0xc6, 0x76, 0xaf, 0x5b, + 0x40, 0x38, 0xf4, 0x3a, 0x74, 0xa3, 0x5b, 0x49, 0xc6, 0xaa, 0xce, 0x2d, 0x77, 0x32, 0xae, 0xfc, + 0x07, 0x3e, 0xae, 0x9e, 0xc3, 0x2c, 0xed, 0x37, 0x71, 0x47, 0x50, 0x4a, 0xff, 0x28, 0x09, 0x28, + 0x2c, 0xf9, 0xeb, 0xeb, 0x6f, 0x89, 0xf8, 0xfa, 0xdb, 0x2a, 0x48, 0x21, 0x63, 0xa8, 0xa5, 0x3a, + 0xae, 0xc2, 0xfc, 0xcf, 0x1c, 0xa5, 0x83, 0x6d, 0xc5, 0xc1, 0x9a, 0xc5, 0x2f, 0xd1, 0x11, 0xe4, + 0x85, 0x20, 0xe7, 0xba, 0xea, 0xb8, 0x4f, 0x59, 0xbe, 0x6d, 0x6c, 0x37, 0x68, 0x2e, 0xf4, 0x08, + 0xae, 0xb4, 0xd5, 0xe3, 0x28, 0xfe, 0x49, 0xca, 0x3f, 0xd7, 0x56, 0x8f, 0x07, 0x98, 0xde, 0x87, + 0xf9, 0x68, 0x26, 0xc5, 0xc1, 0xde, 0xf9, 0xd4, 0x95, 0x08, 0xc6, 0x06, 0x76, 0xd1, 0x32, 0x40, + 0x00, 0x22, 0xf8, 0x1a, 0x3d, 0x0e, 0x86, 0xc8, 0xf8, 0x18, 0x42, 0xfa, 0x91, 0x00, 0xf9, 0x0d, + 0x63, 0xdf, 0x56, 0x63, 0xbd, 0xa2, 0x06, 0xbd, 0xdf, 0x7b, 0xa0, 0x97, 0x7d, 0x38, 0x1f, 0xe5, + 0xb0, 0xc1, 0x72, 0x78, 0x9b, 0x76, 0x9c, 0x81, 0x2c, 0x7d, 0x7e, 0x89, 0xe2, 0x9c, 0xf3, 0x35, + 0x78, 0x83, 0xba, 0xc0, 0x71, 0xff, 0x97, 0xaf, 0x05, 0xdd, 0x48, 0x7f, 0x20, 0xc0, 0xc2, 0xb0, + 0xb7, 0xc4, 0x39, 0x20, 0x64, 0x7a, 0x59, 0x1f, 0x7d, 0x83, 0xe2, 0x8f, 0x88, 0x73, 0x66, 0x1a, + 0xc4, 0x47, 0x02, 0xf8, 0x65, 0x6b, 0xd0, 0x6b, 0xfc, 0xd8, 0x6f, 0x87, 0x28, 0xa8, 0xa1, 0xa9, + 0x26, 0x8d, 0xaf, 0x6d, 0x61, 0xf5, 0x08, 0x73, 0xef, 0xbd, 0x58, 0x67, 0x89, 0x2f, 0x04, 0x58, + 0x18, 0xf6, 0x96, 0x38, 0x15, 0xf4, 0x1d, 0x98, 0x66, 0x6e, 0x60, 0x1e, 0xa8, 0x79, 0x6d, 0x48, + 0x28, 0xb1, 0xe9, 0x05, 0xb0, 0x78, 0xf9, 0xa5, 0x26, 0xe4, 0xcb, 0xaa, 0x6d, 0x1b, 0xb1, 0x6e, + 0x3d, 0x48, 0x3f, 0x16, 0xa0, 0xe0, 0x8b, 0x8d, 0xb3, 0xa6, 0x5f, 0x3d, 0x3a, 0x42, 0xfa, 0xd3, + 0x6b, 0x30, 0xc3, 0xcb, 0xbf, 0x63, 0x1a, 0x96, 0x89, 0x1e, 0x40, 0x72, 0x9f, 0x9f, 0xfa, 0x64, + 0x23, 0x37, 0xbb, 0x83, 0xbb, 0x11, 0x6b, 0x13, 0x32, 0xc9, 0x4b, 0x58, 0x3a, 0x5d, 0x37, 0xa2, + 0x00, 0x81, 0x7b, 0x7f, 0x98, 0xa5, 0xd3, 0x75, 0x51, 0x03, 0x0a, 0x5a, 0x70, 0x21, 0x9b, 0x42, + 0xd8, 0x93, 0x43, 0x77, 0x80, 0x23, 0xaf, 0xc6, 0xab, 0x4d, 0xc8, 0x79, 0xad, 0x27, 0x01, 0x55, + 0xc2, 0xf7, 0x80, 0xa5, 0x06, 0x7c, 0x07, 0x83, 0xa6, 0xef, 0xbd, 0x83, 0xac, 0x36, 0x11, 0xba, + 0x2e, 0x0c, 0xbd, 0x0f, 0x53, 0x3a, 0xbd, 0x71, 0x8a, 0xdb, 0x13, 0x51, 0x0d, 0xde, 0x73, 0xb1, + 0x57, 0x6d, 0x42, 0xe6, 0x1c, 0x68, 0x15, 0x66, 0xd8, 0x2f, 0x86, 0x76, 0xb9, 0x15, 0x70, 0x6b, + 0xb8, 0x84, 0x90, 0x1d, 0x5e, 0x9b, 0x90, 0xb3, 0x7a, 0x40, 0x45, 0x4f, 0x20, 0xab, 0xb5, 0xb0, + 0x6a, 0x73, 0x51, 0xb7, 0x87, 0x06, 0x3c, 0x0e, 0xdc, 0x52, 0x55, 0x9b, 0x90, 0x41, 0xf3, 0x89, + 0xa4, 0x50, 0x36, 0xbd, 0xac, 0x88, 0x4b, 0x7a, 0x7b, 0x68, 0xa1, 0x06, 0x6f, 0x7e, 0xaa, 0x51, + 0xfb, 0xdc, 0xa7, 0xa2, 0x6f, 0x43, 0xca, 0xd1, 0x54, 0x93, 0x9b, 0x24, 0x0b, 0x43, 0x6e, 0x93, + 0x09, 0x98, 0x69, 0x6e, 0xf4, 0x01, 0x03, 0xca, 0xee, 0xb1, 0xb7, 0xcd, 0x1f, 0xa5, 0xd3, 0x9e, + 0x5b, 0x0b, 0x88, 0x4e, 0x31, 0x25, 0x10, 0x3d, 0xa8, 0x7a, 0xdb, 0x30, 0x15, 0x1a, 0x4a, 0x4c, + 0xf7, 0xf5, 0xa3, 0xf5, 0x30, 0x10, 0xfa, 0x5d, 0xa3, 0x57, 0x23, 0x78, 0x44, 0xb4, 0x01, 0x39, + 0x26, 0xa8, 0xcb, 0xa2, 0x92, 0x8b, 0x4b, 0x43, 0x0f, 0xf0, 0x23, 0xe2, 0xa2, 0x6b, 0x13, 0xf2, + 0x8c, 0x1a, 0x22, 0x07, 0xe5, 0x6a, 0x63, 0x7b, 0x9f, 0x1d, 0x20, 0x8c, 0x28, 0x57, 0xd8, 0x2b, + 0xd2, 0x2f, 0x17, 0x25, 0xa2, 0xdf, 0x82, 0x4b, 0x4c, 0x90, 0xcb, 0x9d, 0xbd, 0xb8, 0xcf, 0xd0, + 0x1b, 0x43, 0x0f, 0xdf, 0x87, 0x46, 0x12, 0xd7, 0x26, 0x64, 0xa4, 0x0e, 0x24, 0x22, 0x0d, 0x2e, + 0xb3, 0x37, 0xf0, 0x50, 0x54, 0x9b, 0x47, 0x4f, 0x16, 0x6f, 0xd2, 0x57, 0xbc, 0x35, 0xec, 0x15, + 0x91, 0x11, 0xb2, 0xb5, 0x09, 0x79, 0x4e, 0x1d, 0x4c, 0x0d, 0xaa, 0x61, 0xf3, 0xa0, 0x3f, 0xde, + 0xdd, 0xde, 0x1a, 0x5d, 0x8d, 0xa8, 0x60, 0x49, 0xbf, 0x1a, 0x3d, 0x89, 0xa4, 0x01, 0xfd, 0x2b, + 0x0f, 0x68, 0x67, 0x9a, 0x19, 0xda, 0x80, 0x11, 0x91, 0x81, 0xa4, 0x01, 0x0f, 0x42, 0x64, 0x54, + 0x82, 0xc4, 0xbe, 0x56, 0xcc, 0x0d, 0x35, 0x9d, 0xfc, 0xe8, 0xb7, 0xda, 0x84, 0x9c, 0xd8, 0xd7, + 0xd0, 0x47, 0x90, 0x66, 0xa1, 0x4c, 0xc7, 0x66, 0x31, 0x3f, 0x74, 0xce, 0xee, 0x0d, 0x08, 0xab, + 0x4d, 0xc8, 0x34, 0x7a, 0x8a, 0x77, 0x64, 0x1e, 0xa6, 0x42, 0x45, 0x94, 0x46, 0x44, 0x30, 0xf7, + 0x05, 0x0b, 0x91, 0x0e, 0x63, 0xfb, 0x44, 0xb4, 0x0d, 0x79, 0xbe, 0x74, 0x7b, 0x6e, 0xf7, 0xe2, + 0x50, 0xf7, 0x96, 0x28, 0xcf, 0xfb, 0x1a, 0xdd, 0xa2, 0x0c, 0xd1, 0x49, 0xdb, 0xf5, 0x4a, 0xe4, + 0x6d, 0x37, 0x3b, 0xb4, 0xed, 0x86, 0x7a, 0x81, 0x93, 0xb6, 0xb3, 0x07, 0x12, 0xd1, 0xbb, 0x30, + 0xc9, 0xc6, 0x09, 0xa2, 0x22, 0xa3, 0x3c, 0xb6, 0xfa, 0x86, 0x08, 0xcb, 0x4f, 0x66, 0x2f, 0x97, + 0x7b, 0xb3, 0x2a, 0x2d, 0x6b, 0xbf, 0x38, 0x37, 0x74, 0xf6, 0x1a, 0xf4, 0xcb, 0x25, 0xb3, 0x97, + 0x1b, 0x50, 0x49, 0x07, 0xb2, 0x59, 0x0a, 0x1f, 0x62, 0x97, 0x86, 0x76, 0xa0, 0x08, 0x27, 0xd7, + 0x1a, 0x8d, 0x33, 0x0a, 0xc8, 0xfe, 0xc4, 0xea, 0x60, 0x85, 0x4e, 0x8a, 0x97, 0x47, 0x4f, 0xac, + 0x3d, 0x57, 0x7c, 0xf9, 0x13, 0x2b, 0xa3, 0xa2, 0x67, 0x20, 0xf2, 0x7b, 0x66, 0x14, 0xcf, 0xe9, + 0xaa, 0x78, 0x85, 0xca, 0xbb, 0x17, 0xb9, 0x20, 0x46, 0xf9, 0xe3, 0xd5, 0x08, 0x96, 0xec, 0x4d, + 0x41, 0x1f, 0xc3, 0x2c, 0x95, 0xa7, 0x68, 0xc1, 0xd5, 0x40, 0xc5, 0xe2, 0xc0, 0x45, 0x33, 0xc3, + 0x6f, 0x11, 0xf2, 0x24, 0x8b, 0x5a, 0x5f, 0x12, 0x19, 0x0f, 0x86, 0x69, 0xb8, 0x74, 0xed, 0x9e, + 0x1f, 0x3a, 0x1e, 0x7a, 0xaf, 0x45, 0xad, 0x51, 0x5b, 0x8b, 0x52, 0x48, 0x37, 0xee, 0x9b, 0xf1, + 0x5e, 0x1f, 0xda, 0x8d, 0x87, 0x4c, 0x76, 0x39, 0xb7, 0x67, 0x9e, 0x5b, 0x01, 0x60, 0x3b, 0x08, + 0x14, 0x14, 0x2d, 0x0c, 0x35, 0x00, 0xfa, 0x9d, 0x50, 0x89, 0x01, 0xd0, 0xf2, 0x68, 0xc4, 0x00, + 0x60, 0xc7, 0x5d, 0xc5, 0xeb, 0xc3, 0x17, 0xab, 0xf0, 0x81, 0x38, 0x5d, 0xac, 0x28, 0x81, 0x18, + 0x64, 0x04, 0xce, 0x9d, 0xd0, 0x11, 0x7e, 0x63, 0xe8, 0xee, 0x4d, 0x5f, 0x74, 0x5a, 0x6d, 0x42, + 0x4e, 0xbf, 0xe0, 0x24, 0xd2, 0xab, 0x98, 0x08, 0x3e, 0xb6, 0xef, 0x0f, 0xed, 0x55, 0x83, 0x61, + 0x49, 0xa4, 0x57, 0xbd, 0x08, 0xa8, 0xc1, 0x92, 0xe7, 0xb0, 0x83, 0xac, 0xe2, 0x9b, 0xa3, 0x97, + 0xbc, 0xde, 0x63, 0x37, 0x7f, 0xc9, 0xe3, 0x64, 0xb6, 0xe4, 0xe9, 0x8a, 0xe3, 0x50, 0xdf, 0x96, + 0xe2, 0xad, 0x11, 0x4b, 0x5e, 0xdf, 0xd6, 0x36, 0x5b, 0xf2, 0xf4, 0x06, 0xe3, 0x24, 0xd6, 0x9f, + 0xed, 0xdd, 0xab, 0xc4, 0x81, 0xfd, 0x9d, 0xa1, 0xd6, 0x5f, 0xe4, 0xc5, 0x4f, 0xc4, 0xfa, 0xb3, + 0x7b, 0x12, 0xd0, 0x77, 0x61, 0x9a, 0x6f, 0x25, 0x16, 0xef, 0x8e, 0x30, 0xa9, 0xc3, 0xbb, 0xbf, + 0xa4, 0x3b, 0x72, 0x1e, 0x36, 0x39, 0xb0, 0x2d, 0x4c, 0x36, 0xf9, 0xdd, 0x1b, 0x31, 0x39, 0x0c, + 0xec, 0xa2, 0xb2, 0xc9, 0x21, 0x20, 0x93, 0xd2, 0x38, 0x6c, 0xfb, 0xad, 0xf8, 0x6b, 0x43, 0x4b, + 0xd3, 0xbb, 0x0f, 0x49, 0x4a, 0xc3, 0x79, 0xe8, 0x62, 0x41, 0xd7, 0x6a, 0xa6, 0x9d, 0x6f, 0x0d, + 0x5f, 0x2c, 0xfa, 0x37, 0x74, 0x6a, 0xde, 0x41, 0x21, 0xd3, 0xca, 0x5f, 0x11, 0xe0, 0x3a, 0xeb, + 0x03, 0xf4, 0x98, 0xe4, 0x44, 0xf1, 0x4f, 0xb9, 0x42, 0xbb, 0x55, 0x0f, 0xa8, 0xf8, 0x77, 0x2f, + 0x7e, 0x28, 0xe3, 0xbd, 0xf1, 0x0d, 0x75, 0x54, 0x3e, 0xa2, 0x8c, 0x36, 0xc3, 0xf5, 0xc5, 0x87, + 0x43, 0x95, 0xd1, 0xbb, 0x17, 0x41, 0x94, 0xc1, 0x79, 0x50, 0x0b, 0x8a, 0x6c, 0x48, 0x04, 0xb8, + 0xd7, 0x2f, 0xfa, 0xa3, 0xa1, 0xee, 0xa2, 0x23, 0x11, 0x7f, 0x6d, 0x42, 0xbe, 0xf2, 0x22, 0x32, + 0x03, 0x79, 0x1b, 0xbd, 0x38, 0xd1, 0x08, 0x60, 0xaa, 0xe2, 0xe1, 0xc9, 0x6f, 0x0f, 0x7d, 0xdb, + 0x48, 0xf8, 0x4c, 0xde, 0xe6, 0x44, 0x66, 0x20, 0xaa, 0xd9, 0x65, 0xd0, 0xb0, 0xf8, 0xce, 0x50, + 0xd5, 0xf4, 0x62, 0x52, 0xa2, 0x1a, 0xce, 0x53, 0x9e, 0xe6, 0x3e, 0x12, 0x7e, 0x3c, 0x74, 0x41, + 0x14, 0x57, 0x53, 0xe9, 0xab, 0x62, 0x71, 0x35, 0x95, 0x7e, 0x4d, 0x9c, 0x5f, 0x4d, 0xa5, 0xaf, + 0x89, 0xaf, 0xaf, 0xa6, 0xd2, 0x8b, 0xe2, 0xf5, 0xd5, 0x54, 0x5a, 0x12, 0x6f, 0x4a, 0xbf, 0xbc, + 0x06, 0x39, 0x0f, 0x59, 0x32, 0xc8, 0xf7, 0x30, 0x0c, 0xf9, 0x16, 0x86, 0x41, 0x3e, 0x8e, 0x45, + 0x39, 0xe6, 0x7b, 0x18, 0xc6, 0x7c, 0x0b, 0xc3, 0x30, 0x5f, 0xc0, 0x43, 0x40, 0x5f, 0x73, 0x18, + 0xe8, 0xbb, 0x37, 0x06, 0xe8, 0xf3, 0x45, 0xf5, 0xa3, 0xbe, 0x95, 0x41, 0xd4, 0xf7, 0xe6, 0x68, + 0xd4, 0xe7, 0x8b, 0x0a, 0xc1, 0xbe, 0x0f, 0xfa, 0x60, 0xdf, 0x8d, 0x11, 0xb0, 0xcf, 0xe7, 0xf7, + 0x70, 0xdf, 0x5a, 0x24, 0xee, 0xbb, 0x7d, 0x1e, 0xee, 0xf3, 0xe5, 0xf4, 0x00, 0xbf, 0x5a, 0x14, + 0xf0, 0xbb, 0x75, 0x0e, 0xf0, 0xf3, 0x45, 0x85, 0x91, 0xdf, 0x5a, 0x24, 0xf2, 0xbb, 0x7d, 0x1e, + 0xf2, 0x0b, 0x8a, 0x15, 0x86, 0x7e, 0xef, 0xf4, 0x40, 0xbf, 0xc5, 0xa1, 0xd0, 0xcf, 0xe7, 0x66, + 0xd8, 0xef, 0xc3, 0x7e, 0xec, 0x77, 0x63, 0x04, 0xf6, 0x0b, 0x14, 0xcb, 0xc1, 0x5f, 0x2d, 0x0a, + 0xfc, 0xdd, 0x3a, 0x07, 0xfc, 0x05, 0xba, 0x08, 0xa1, 0xbf, 0xcd, 0x68, 0xf4, 0x77, 0xe7, 0x5c, + 0xf4, 0xe7, 0x4b, 0xeb, 0x85, 0x7f, 0xb5, 0x28, 0xf8, 0x77, 0xeb, 0x1c, 0xf8, 0xd7, 0x57, 0x32, + 0x86, 0xff, 0xd4, 0x91, 0xf8, 0xef, 0xad, 0x31, 0xf1, 0x9f, 0x2f, 0x3a, 0x0a, 0x00, 0xea, 0xa3, + 0x01, 0x60, 0x69, 0x5c, 0x00, 0xe8, 0xbf, 0x24, 0x12, 0x01, 0xaa, 0x23, 0x11, 0xe0, 0x5b, 0x63, + 0x22, 0xc0, 0xbe, 0x8a, 0xf4, 0x42, 0xc0, 0xcd, 0x68, 0x08, 0x78, 0xe7, 0x5c, 0x08, 0x18, 0xb4, + 0x62, 0x0f, 0x06, 0x5c, 0x0a, 0x61, 0xc0, 0x37, 0x86, 0x60, 0x40, 0x9f, 0x95, 0x80, 0xc0, 0xef, + 0x0d, 0x80, 0x40, 0x69, 0x14, 0x08, 0xf4, 0x79, 0x7d, 0x14, 0x58, 0x8b, 0x42, 0x81, 0xb7, 0xce, + 0x41, 0x81, 0x41, 0xbf, 0x09, 0xc1, 0xc0, 0xa7, 0x43, 0x60, 0xe0, 0xdd, 0xf3, 0x61, 0xa0, 0x2f, + 0xaf, 0x0f, 0x07, 0xaa, 0x23, 0x71, 0xe0, 0x5b, 0x63, 0xe2, 0xc0, 0xa0, 0x05, 0x23, 0x80, 0xe0, + 0x7b, 0xbd, 0x40, 0xf0, 0xfa, 0x70, 0x20, 0xe8, 0x8b, 0xe1, 0x48, 0x70, 0x2d, 0x12, 0x09, 0xde, + 0x3e, 0x0f, 0x09, 0x06, 0xb3, 0x59, 0x18, 0x0a, 0x6e, 0x46, 0x43, 0xc1, 0x3b, 0xe7, 0x42, 0xc1, + 0xa0, 0x23, 0xf5, 0x60, 0xc1, 0xb5, 0x48, 0x2c, 0x78, 0xfb, 0x3c, 0x2c, 0xd8, 0x37, 0xd5, 0x72, + 0x30, 0xf8, 0x7c, 0x28, 0x18, 0xbc, 0x3f, 0x0e, 0x18, 0xf4, 0x85, 0x0e, 0xa0, 0xc1, 0x4f, 0x86, + 0xa3, 0xc1, 0x5f, 0xbb, 0xc0, 0x9d, 0xb2, 0x91, 0x70, 0xf0, 0x7b, 0x03, 0x70, 0x50, 0x1a, 0x05, + 0x07, 0x83, 0x91, 0xe1, 0xe1, 0xc1, 0x6a, 0x04, 0x7a, 0x7b, 0x73, 0x34, 0x7a, 0x0b, 0x16, 0xf2, + 0x00, 0xbe, 0x7d, 0xd0, 0x07, 0xdf, 0x6e, 0x9c, 0xeb, 0xbe, 0x19, 0xc2, 0x6f, 0xe5, 0x41, 0xfc, + 0x76, 0x73, 0x24, 0x7e, 0xf3, 0x25, 0x04, 0x00, 0x6e, 0x2d, 0x12, 0xc0, 0xdd, 0x3e, 0x0f, 0xc0, + 0x05, 0x5d, 0x21, 0x8c, 0xe0, 0x36, 0xa3, 0x11, 0xdc, 0x9d, 0x73, 0x11, 0x5c, 0xdf, 0xb2, 0xe5, + 0x41, 0xb8, 0x5a, 0x14, 0x84, 0xbb, 0x75, 0x0e, 0x84, 0x0b, 0x2f, 0x5b, 0x3e, 0x86, 0x6b, 0x0e, + 0xc3, 0x70, 0xf7, 0xc6, 0xc0, 0x70, 0x81, 0x31, 0xd7, 0x07, 0xe2, 0x3e, 0xea, 0x07, 0x71, 0xd2, + 0x28, 0x10, 0x17, 0x74, 0x22, 0x0f, 0xc5, 0x6d, 0x46, 0xa3, 0xb8, 0x3b, 0xe7, 0xa2, 0xb8, 0xf0, + 0xb8, 0x0e, 0xc1, 0xb8, 0x8f, 0xfa, 0x61, 0x9c, 0x34, 0x0a, 0xc6, 0x05, 0xe5, 0xf1, 0x70, 0x5c, + 0x2d, 0x0a, 0xc7, 0xdd, 0x3a, 0x07, 0xc7, 0x85, 0xa6, 0xfb, 0x00, 0xc8, 0xfd, 0xd5, 0xf1, 0x81, + 0xdc, 0x7b, 0xaf, 0xea, 0x5d, 0x77, 0x3e, 0x92, 0xfb, 0xa8, 0x1f, 0xc9, 0x49, 0xa3, 0x90, 0x5c, + 0xa0, 0x0f, 0x0f, 0xca, 0xb5, 0xcf, 0x85, 0x72, 0x0f, 0x2e, 0x00, 0xe5, 0x7c, 0xf9, 0xc3, 0xb0, + 0x5c, 0xfb, 0x5c, 0x2c, 0xf7, 0xe0, 0x02, 0x58, 0x2e, 0x78, 0xdd, 0x10, 0x30, 0xf7, 0x51, 0x3f, + 0x98, 0x93, 0x46, 0x81, 0xb9, 0x40, 0x3b, 0x17, 0x46, 0x73, 0xaf, 0x8b, 0x6f, 0xf4, 0x60, 0xba, + 0xff, 0x31, 0x0d, 0x53, 0xb5, 0x88, 0x33, 0x41, 0xe1, 0x55, 0xce, 0x04, 0xd1, 0x77, 0xe1, 0x9a, + 0xff, 0x40, 0xfd, 0xb9, 0x14, 0x1e, 0x47, 0xa6, 0xb5, 0x2c, 0xed, 0x90, 0x2e, 0x95, 0x69, 0xb9, + 0xe8, 0x67, 0x79, 0x6c, 0x5b, 0x6d, 0x16, 0x4f, 0x46, 0xbd, 0x64, 0xd0, 0x0a, 0x19, 0xc3, 0xd4, + 0x26, 0x3c, 0xff, 0x9a, 0xce, 0xc1, 0x8b, 0x24, 0x39, 0xeb, 0x2b, 0xdc, 0x7c, 0x80, 0xde, 0x81, + 0x5c, 0xd7, 0xc1, 0xb6, 0xd2, 0xb1, 0x0d, 0xcb, 0x36, 0x5c, 0x16, 0xc3, 0x25, 0x94, 0xc5, 0x2f, + 0x4f, 0x17, 0x67, 0x76, 0x1c, 0x6c, 0x6f, 0x73, 0xba, 0x3c, 0xd3, 0x0d, 0x3d, 0x79, 0xdf, 0xcb, + 0x9b, 0x1c, 0xff, 0x7b, 0x79, 0x4f, 0x41, 0xa4, 0x1e, 0x3a, 0xe1, 0x65, 0x94, 0x5d, 0x6e, 0x16, + 0xbd, 0xe2, 0xab, 0x7a, 0x68, 0xa5, 0xa4, 0x97, 0x9c, 0x15, 0xec, 0x5e, 0x22, 0x7a, 0x02, 0x79, + 0xdb, 0xea, 0xd2, 0x3b, 0x8c, 0x3a, 0x56, 0xcb, 0xd0, 0x4e, 0xa8, 0x7d, 0x93, 0x8f, 0x3e, 0x6f, + 0x66, 0x19, 0xb7, 0x69, 0x3e, 0x39, 0x67, 0x87, 0x1f, 0x51, 0x03, 0xe8, 0xfd, 0x45, 0x9e, 0x14, + 0x34, 0x70, 0x63, 0xfc, 0xc8, 0xef, 0x36, 0x3c, 0x57, 0x0d, 0x97, 0xcb, 0x85, 0x97, 0xfe, 0x6f, + 0xf4, 0x00, 0x2e, 0xb7, 0xd5, 0x63, 0x7a, 0x95, 0xb6, 0xe2, 0xd9, 0x4d, 0xf4, 0xf2, 0x40, 0xf6, + 0x09, 0x3e, 0xd4, 0x56, 0x8f, 0xe9, 0xa7, 0x01, 0x59, 0x12, 0xfd, 0xae, 0xcf, 0x0d, 0x98, 0xe1, + 0x41, 0x39, 0xec, 0xb3, 0x5f, 0x05, 0x9a, 0x93, 0x7f, 0x03, 0x86, 0x7d, 0xf9, 0xeb, 0x16, 0xe4, + 0x75, 0xc3, 0x71, 0x0d, 0x53, 0x73, 0xf9, 0x2d, 0xdd, 0xec, 0x9e, 0xeb, 0x9c, 0x47, 0x65, 0x57, + 0x71, 0x37, 0x61, 0x56, 0x6b, 0x19, 0xbe, 0x35, 0xca, 0xec, 0x83, 0xd9, 0xa1, 0xe3, 0xab, 0x42, + 0xf3, 0xf6, 0x3b, 0xbe, 0x14, 0xb4, 0x5e, 0x32, 0xaa, 0x40, 0x61, 0x5f, 0x75, 0xf1, 0x4b, 0xf5, + 0x44, 0xf1, 0x62, 0x4e, 0xb3, 0x34, 0xce, 0xfe, 0xda, 0xd9, 0xe9, 0x62, 0xee, 0x09, 0x4b, 0x1a, + 0x08, 0x3d, 0xcd, 0xed, 0x87, 0x12, 0x74, 0x74, 0x07, 0x0a, 0xaa, 0x73, 0x62, 0x6a, 0xb4, 0x27, + 0x60, 0xd3, 0xe9, 0x3a, 0x14, 0x4c, 0xa4, 0xe5, 0x3c, 0x25, 0x57, 0x3c, 0x2a, 0xfa, 0x00, 0xe6, + 0xf9, 0xc7, 0x38, 0x5e, 0xaa, 0xb6, 0xae, 0xd0, 0xde, 0x13, 0x0c, 0x53, 0x91, 0xf2, 0x5c, 0x65, + 0x1f, 0xdf, 0x20, 0x19, 0x48, 0x97, 0x09, 0x5f, 0x72, 0xcd, 0x2e, 0xf1, 0x06, 0x31, 0xbb, 0x9a, + 0x4a, 0xcf, 0x88, 0xb9, 0xd5, 0x54, 0x3a, 0x2f, 0x16, 0xa4, 0x1f, 0x26, 0xa1, 0x40, 0xe6, 0x75, + 0xc7, 0x31, 0x2c, 0xb3, 0xe6, 0xbb, 0x7f, 0xfb, 0xdd, 0x5f, 0xa0, 0xe1, 0x7f, 0xfe, 0x33, 0x5a, + 0xa4, 0x61, 0x96, 0xc4, 0x7e, 0xf6, 0x3f, 0xc1, 0x93, 0x94, 0x81, 0x91, 0x68, 0xc0, 0xdb, 0x32, + 0x4c, 0x39, 0x56, 0xd7, 0xd6, 0xbc, 0x0f, 0x43, 0xdc, 0x1b, 0xb2, 0x90, 0x84, 0x5e, 0x58, 0x6a, + 0x50, 0x06, 0x99, 0x33, 0xa2, 0x4f, 0xa0, 0xc0, 0x7e, 0xd1, 0xd0, 0x32, 0x1a, 0xda, 0xc5, 0xe2, + 0xe6, 0x1e, 0x8c, 0x2d, 0x6b, 0x9d, 0x33, 0xca, 0x79, 0xa7, 0xe7, 0x19, 0x7d, 0x04, 0xaf, 0x9b, + 0x96, 0xd2, 0xc6, 0x6d, 0x8b, 0x2d, 0x20, 0xa4, 0x0f, 0xeb, 0x8a, 0xea, 0x2a, 0xbc, 0xd0, 0xcc, + 0x9f, 0xb8, 0x68, 0x5a, 0x1b, 0x34, 0x8b, 0xcc, 0x73, 0x2c, 0xbb, 0x4c, 0xae, 0x54, 0x82, 0x29, + 0xf6, 0x0b, 0x65, 0x60, 0x72, 0xab, 0x59, 0xab, 0xca, 0xe2, 0x04, 0x9a, 0x81, 0xf4, 0x63, 0x79, + 0x6b, 0x43, 0x69, 0x3c, 0x5d, 0x17, 0x05, 0x94, 0x85, 0x69, 0x79, 0x6b, 0xab, 0xa9, 0xac, 0x3d, + 0x13, 0x13, 0xd2, 0x1d, 0xc8, 0xf7, 0x96, 0x08, 0x01, 0x4c, 0xc9, 0xd5, 0x8d, 0x2d, 0xfa, 0x35, + 0x84, 0x0c, 0x4c, 0xae, 0x6f, 0x55, 0x96, 0xd7, 0x45, 0x41, 0xfa, 0x13, 0x01, 0x66, 0xca, 0xec, + 0xf3, 0x1e, 0xcc, 0x6b, 0xe4, 0x83, 0x3e, 0xef, 0x8e, 0xd7, 0xa2, 0x01, 0xea, 0x30, 0xaf, 0x8e, + 0x34, 0x1f, 0x63, 0x5e, 0x08, 0xd2, 0xe2, 0x70, 0x58, 0x42, 0x77, 0xf0, 0x3c, 0x37, 0x52, 0x8f, + 0x0d, 0x35, 0x40, 0x54, 0x3d, 0xdd, 0x2a, 0xbc, 0x24, 0xc3, 0x9d, 0x49, 0xfb, 0x9a, 0xc1, 0x1b, + 0x31, 0x6a, 0x2f, 0xf9, 0xfd, 0xd4, 0xe7, 0x5f, 0x2c, 0x4e, 0x48, 0x7f, 0x96, 0x82, 0x5c, 0x39, + 0xfc, 0x29, 0x13, 0x54, 0xef, 0xab, 0xec, 0x9d, 0xc8, 0x45, 0x2f, 0xc4, 0x51, 0x1a, 0xf1, 0x91, + 0xa8, 0x4c, 0xf0, 0xdd, 0x14, 0x56, 0xf7, 0xeb, 0x23, 0x1c, 0x63, 0xc2, 0x95, 0x0f, 0x18, 0xe7, + 0xff, 0x6d, 0xd2, 0x5f, 0x0d, 0x4b, 0x30, 0xc9, 0xc2, 0x5c, 0x85, 0x81, 0x1b, 0x38, 0xe8, 0x3c, + 0x48, 0xc0, 0x00, 0x49, 0x97, 0x59, 0x36, 0xb2, 0x7a, 0x36, 0x5f, 0xc9, 0xa3, 0x26, 0xb0, 0x39, + 0x2e, 0xfe, 0xf9, 0xd5, 0x2e, 0xbb, 0x6f, 0xf6, 0xff, 0xa1, 0xff, 0x2a, 0x79, 0x1f, 0xfa, 0x4d, + 0x28, 0x68, 0x56, 0xab, 0xc5, 0x6c, 0x48, 0x36, 0xff, 0x0e, 0xde, 0x40, 0x45, 0x8b, 0xc0, 0xbf, + 0xb8, 0x5b, 0xf2, 0xbf, 0xbc, 0x5b, 0x92, 0xf9, 0x97, 0x77, 0x43, 0xc1, 0x45, 0x79, 0x5f, 0x18, + 0x9b, 0xb6, 0xfb, 0xe2, 0x9c, 0xa6, 0x5f, 0x25, 0xce, 0x89, 0x45, 0x87, 0xf1, 0x9e, 0xf7, 0x4b, + 0x81, 0x7b, 0x99, 0xae, 0x5b, 0xd6, 0x61, 0xd7, 0xf7, 0xdd, 0x9b, 0x0f, 0xdf, 0x1e, 0x1b, 0x84, + 0x60, 0xd0, 0x10, 0xc2, 0xa8, 0x85, 0x3a, 0xf1, 0xd5, 0x16, 0xea, 0x1b, 0x30, 0xd3, 0xb1, 0xf1, + 0x1e, 0x76, 0xb5, 0x03, 0xc5, 0xec, 0xb6, 0x79, 0xfc, 0x64, 0xd6, 0xa3, 0x6d, 0x76, 0xdb, 0xe8, + 0x1e, 0x88, 0x7e, 0x16, 0x8e, 0xeb, 0xbd, 0xab, 0x0b, 0x3d, 0x3a, 0xdf, 0x05, 0x90, 0xfe, 0xbb, + 0x00, 0x73, 0x3d, 0x75, 0xe2, 0x63, 0x6a, 0x15, 0xb2, 0xba, 0x6f, 0x1a, 0x39, 0x45, 0xe1, 0x82, + 0xc1, 0x39, 0x61, 0x66, 0xa4, 0xc0, 0x15, 0xef, 0xb5, 0xf4, 0x5b, 0x23, 0x81, 0xd8, 0xc4, 0x05, + 0xc5, 0x5e, 0x0e, 0xe4, 0xac, 0x84, 0x5e, 0xe0, 0x0f, 0xb2, 0xe4, 0x58, 0x83, 0x4c, 0xfa, 0x9f, + 0x02, 0x88, 0xf4, 0x05, 0x8f, 0x31, 0xd6, 0x63, 0x99, 0x32, 0xbd, 0x28, 0xb8, 0xc4, 0xf8, 0x11, + 0x92, 0x3d, 0xdf, 0x47, 0x4a, 0xf6, 0x7d, 0x1f, 0x29, 0x6a, 0xfe, 0x4c, 0x7d, 0xc5, 0xf9, 0x53, + 0xfa, 0x42, 0x80, 0xbc, 0x5f, 0x6d, 0xf6, 0x61, 0xd4, 0x11, 0x37, 0x1f, 0xbf, 0xda, 0xc7, 0x3f, + 0xbd, 0x1b, 0x9a, 0xc6, 0xfa, 0x56, 0x6b, 0xf8, 0x86, 0x26, 0xf6, 0xd1, 0xca, 0xbf, 0xe5, 0x75, + 0x47, 0x52, 0xc4, 0x4a, 0x70, 0x35, 0xce, 0x2b, 0x44, 0xa0, 0x7e, 0x1d, 0x6e, 0xaa, 0x8f, 0x43, + 0x0a, 0xa4, 0x3d, 0x8a, 0x68, 0x69, 0xac, 0xf9, 0xdd, 0xd3, 0x12, 0xeb, 0x80, 0x3f, 0x0b, 0xb7, + 0x04, 0xbb, 0x5a, 0xe1, 0x11, 0x24, 0x8f, 0xd4, 0xd6, 0x28, 0x67, 0xcc, 0x9e, 0x96, 0x93, 0x49, + 0x6e, 0xf4, 0xb8, 0xe7, 0x46, 0xa1, 0xc4, 0xf0, 0x3d, 0xbf, 0x41, 0x95, 0xf6, 0xdc, 0x3c, 0xf4, + 0x6e, 0xef, 0x00, 0x1a, 0xf9, 0xfa, 0xf0, 0x48, 0x7a, 0x3f, 0xf5, 0xd3, 0x2f, 0x16, 0x05, 0xe9, + 0x43, 0x40, 0xc4, 0xd6, 0x71, 0x9f, 0x76, 0x2d, 0x3b, 0xb8, 0x9d, 0xa9, 0x3f, 0xda, 0x6d, 0x32, + 0x3a, 0xda, 0x4d, 0xba, 0x0c, 0x73, 0x3d, 0xdc, 0x6c, 0x06, 0x92, 0xde, 0x85, 0xd7, 0x9e, 0x58, + 0x8e, 0x63, 0x74, 0x1a, 0xdd, 0x5d, 0x36, 0xd4, 0xc9, 0x7a, 0xe5, 0xcf, 0xb9, 0xe9, 0x0e, 0xdd, + 0x50, 0x32, 0xd9, 0xdc, 0x94, 0x91, 0xfd, 0x67, 0xe9, 0x5f, 0x09, 0x70, 0x75, 0x90, 0x93, 0x69, + 0x39, 0x2a, 0x60, 0x7e, 0x5a, 0xb3, 0x82, 0xcb, 0x43, 0xcf, 0xef, 0xad, 0x5e, 0x76, 0x62, 0x7b, + 0xf3, 0x77, 0x2a, 0x6d, 0x95, 0xce, 0x49, 0xfc, 0x2e, 0x8b, 0x3c, 0x27, 0x6f, 0x30, 0x6a, 0x30, + 0x3d, 0xa5, 0xc6, 0x9b, 0x9e, 0xfe, 0x97, 0x00, 0xb3, 0x4d, 0x6c, 0xaa, 0xa6, 0x4b, 0xe6, 0xfd, + 0x6e, 0x9b, 0x5d, 0x32, 0x50, 0x80, 0xa4, 0xad, 0x74, 0x69, 0xd1, 0x05, 0x39, 0x61, 0xef, 0xa0, + 0x9b, 0x90, 0xa3, 0x6b, 0x4b, 0xc8, 0x56, 0x13, 0xee, 0xa6, 0x64, 0x1a, 0xbb, 0x21, 0x7b, 0x86, + 0xd8, 0x1b, 0x00, 0x34, 0x13, 0xc3, 0x40, 0x49, 0x9a, 0x23, 0x43, 0x28, 0x3e, 0x02, 0xa2, 0xb1, + 0x70, 0x81, 0x10, 0x16, 0xee, 0x92, 0xa3, 0x54, 0x5f, 0xca, 0x22, 0x64, 0x59, 0x36, 0x26, 0x66, + 0x92, 0xe6, 0x01, 0x4a, 0x62, 0x72, 0x1e, 0xc3, 0x25, 0xe7, 0x45, 0x4b, 0xe9, 0x58, 0xba, 0xa3, + 0x68, 0x9d, 0x2e, 0x8f, 0x23, 0x60, 0xdf, 0x5a, 0x16, 0xca, 0x97, 0xcf, 0x4e, 0x17, 0x67, 0x1b, + 0x4f, 0xd7, 0xb7, 0x2d, 0xdd, 0xa9, 0x6c, 0xef, 0xb0, 0x28, 0x02, 0x47, 0x9e, 0x75, 0x5e, 0xb4, + 0x28, 0xa9, 0xd3, 0xe5, 0x24, 0xe9, 0xb3, 0x04, 0x20, 0x7a, 0x59, 0x4d, 0x99, 0xde, 0xf6, 0xe2, + 0x35, 0xb7, 0x05, 0x0b, 0x5a, 0xa0, 0x0a, 0xc5, 0x31, 0x4c, 0x62, 0xdf, 0xab, 0x8e, 0xeb, 0x95, + 0x9b, 0x8f, 0x9b, 0xc8, 0x3b, 0x36, 0xfb, 0x35, 0xc9, 0xdb, 0xf1, 0x5a, 0x48, 0x62, 0x83, 0x08, + 0x5c, 0x57, 0x1d, 0xff, 0x85, 0xf7, 0x20, 0xe3, 0x52, 0x3e, 0xef, 0x2a, 0xa0, 0x54, 0x79, 0xe6, + 0xec, 0x74, 0x31, 0xcd, 0x84, 0xd5, 0x57, 0xe4, 0x34, 0x4b, 0xae, 0xeb, 0x68, 0x09, 0xb2, 0x86, + 0xe9, 0xb8, 0x2a, 0x29, 0x12, 0xdf, 0x2a, 0xc8, 0xb1, 0xb0, 0xf3, 0x3a, 0x27, 0xd7, 0x57, 0x64, + 0xf0, 0xb2, 0xd0, 0xe8, 0xce, 0xbc, 0xcf, 0xc0, 0xf6, 0xfe, 0x69, 0x28, 0xb2, 0x9c, 0xf3, 0xa8, + 0xec, 0xb2, 0xba, 0x06, 0xcc, 0xf5, 0x68, 0x82, 0x2f, 0xcc, 0x1f, 0xf6, 0x4e, 0x38, 0x61, 0xeb, + 0x94, 0xd2, 0x1d, 0x7a, 0x9c, 0xa9, 0x59, 0x3a, 0x1f, 0xa9, 0xbd, 0x13, 0x4f, 0x13, 0x0a, 0xab, + 0x96, 0x61, 0x12, 0xf4, 0xe8, 0x55, 0x75, 0x19, 0xf2, 0xbb, 0x86, 0xa9, 0xda, 0x27, 0x8a, 0x17, + 0x36, 0x21, 0x9c, 0x17, 0x36, 0x21, 0xe7, 0x18, 0x07, 0x7f, 0x94, 0x7e, 0x21, 0x80, 0x18, 0x88, + 0xe5, 0x05, 0xfd, 0x16, 0x80, 0xd6, 0xea, 0x3a, 0x2e, 0xb6, 0xbd, 0x09, 0x60, 0x86, 0x85, 0x67, + 0x56, 0x18, 0xb5, 0xbe, 0x22, 0x67, 0x78, 0x86, 0xba, 0x8e, 0x6e, 0xf6, 0xde, 0xbc, 0x34, 0x59, + 0x86, 0xb3, 0x81, 0xfb, 0x96, 0xc8, 0x8c, 0xe2, 0xb8, 0x96, 0xed, 0xeb, 0x99, 0xcf, 0x28, 0xde, + 0x9d, 0x74, 0xf4, 0xee, 0x15, 0x4c, 0x23, 0xb0, 0xf3, 0xc4, 0xbc, 0x3d, 0xc2, 0x7e, 0x95, 0x52, + 0xe7, 0x57, 0x89, 0x71, 0x78, 0x55, 0xfa, 0x03, 0x01, 0x0a, 0x15, 0x36, 0xd0, 0xfd, 0xc9, 0x63, + 0xc4, 0x62, 0xb9, 0x02, 0x69, 0xf7, 0xd8, 0x54, 0xda, 0xd8, 0xff, 0x8c, 0xda, 0x05, 0x2e, 0x8d, + 0x9d, 0x76, 0xd9, 0x23, 0xfd, 0x32, 0x6f, 0xd7, 0x0e, 0xbe, 0x1e, 0x4b, 0x8c, 0x90, 0x7d, 0x7a, + 0x6d, 0x57, 0xa9, 0x63, 0x5b, 0xae, 0xb5, 0xdb, 0xdd, 0x2b, 0xad, 0xf0, 0x0c, 0xcc, 0x08, 0xf9, + 0xfc, 0x3f, 0x2c, 0x0a, 0xb2, 0xcf, 0xc4, 0xec, 0xd4, 0xfb, 0x0d, 0x32, 0xa1, 0x0e, 0x58, 0x92, + 0x28, 0x0f, 0x10, 0xfa, 0x3e, 0xde, 0x04, 0xfd, 0x6a, 0x5e, 0x75, 0x79, 0x45, 0xd9, 0xd9, 0xac, + 0x6c, 0x6d, 0x6c, 0xd4, 0x9b, 0xcd, 0xea, 0x8a, 0x28, 0x20, 0x11, 0x66, 0x7a, 0xbe, 0xae, 0x97, + 0x98, 0x4f, 0x7d, 0xf6, 0x93, 0x85, 0x89, 0xfb, 0x6f, 0x41, 0xae, 0x67, 0xdb, 0x07, 0x15, 0x20, + 0xbb, 0x5e, 0x5d, 0x6e, 0x54, 0x6b, 0x5b, 0xeb, 0x2b, 0x14, 0xc8, 0x66, 0x61, 0x7a, 0xb3, 0xba, + 0x2c, 0x57, 0x1b, 0x4d, 0x51, 0xb8, 0xff, 0x17, 0x00, 0x82, 0xef, 0x7a, 0x92, 0x57, 0xaf, 0x55, + 0x3f, 0x56, 0x9e, 0x2d, 0xaf, 0xef, 0x54, 0x1b, 0xe2, 0x04, 0x42, 0x90, 0x2f, 0x2f, 0x37, 0x2b, + 0x35, 0x45, 0xae, 0x36, 0xb6, 0xb7, 0x36, 0x1b, 0x55, 0x51, 0xe0, 0xaf, 0x59, 0x81, 0x99, 0xf0, + 0xcd, 0x79, 0x68, 0x0e, 0x0a, 0x95, 0x5a, 0xb5, 0xb2, 0xa6, 0x3c, 0xab, 0x2f, 0x2b, 0x4f, 0x77, + 0xaa, 0x3b, 0x04, 0xf9, 0x92, 0x9a, 0x50, 0xe2, 0xe3, 0x9d, 0x75, 0x02, 0x9a, 0x0b, 0x90, 0x65, + 0xcf, 0xf4, 0xc3, 0x7d, 0x62, 0xe2, 0xfe, 0x06, 0x64, 0x43, 0x37, 0xfa, 0x93, 0xd7, 0x6d, 0xef, + 0x34, 0x6a, 0x4a, 0xb3, 0xbe, 0x51, 0x6d, 0x34, 0x97, 0x37, 0xb6, 0x99, 0x0c, 0x4a, 0x5b, 0x2e, + 0x6f, 0xc9, 0x4d, 0x51, 0xf0, 0x9f, 0x9b, 0x5b, 0x3b, 0x95, 0x9a, 0x57, 0x6b, 0x29, 0x95, 0x4e, + 0x8a, 0xc9, 0xfb, 0xc7, 0x70, 0x75, 0xc8, 0x25, 0x72, 0xa4, 0xd2, 0x3b, 0x26, 0xbd, 0xdd, 0x5c, + 0x9c, 0x40, 0x39, 0xc8, 0x90, 0x9e, 0x4a, 0xaf, 0x98, 0x10, 0x05, 0x94, 0x86, 0xd4, 0x81, 0xeb, + 0x76, 0xc4, 0x04, 0x9a, 0x82, 0x84, 0xf3, 0x48, 0x4c, 0x92, 0xff, 0xfb, 0x8e, 0x98, 0x22, 0xd0, + 0x5d, 0xfd, 0xb4, 0x6b, 0x63, 0x71, 0x92, 0xc0, 0xff, 0xae, 0x83, 0xed, 0x3d, 0xa3, 0x85, 0xc5, + 0x69, 0xc2, 0x62, 0x76, 0x5b, 0x2d, 0x31, 0x2d, 0xa5, 0xd2, 0x53, 0xe2, 0xd4, 0xfd, 0x1b, 0x10, + 0xba, 0xcb, 0x87, 0xa0, 0xff, 0x75, 0xd5, 0xc5, 0x8e, 0x2b, 0x4e, 0xa0, 0x69, 0x48, 0x2e, 0xb7, + 0x5a, 0xa2, 0xf0, 0xf0, 0xf3, 0x49, 0x48, 0x7b, 0xdf, 0xa5, 0x43, 0xeb, 0x30, 0x49, 0x91, 0x2e, + 0x5a, 0x1c, 0x8e, 0x81, 0xe9, 0xb0, 0x9f, 0xbf, 0x7e, 0x1e, 0x48, 0x96, 0x26, 0xd0, 0x5f, 0x84, + 0x6c, 0x08, 0x1b, 0xa0, 0xa1, 0xc7, 0x06, 0x3d, 0x78, 0x68, 0xfe, 0xf6, 0x79, 0xd9, 0x7c, 0xf9, + 0xcf, 0x21, 0xe3, 0x9b, 0x15, 0xe8, 0xe6, 0x28, 0xa3, 0xc3, 0x93, 0x3d, 0xda, 0x32, 0x21, 0xa3, + 0x54, 0x9a, 0x78, 0x5b, 0x40, 0x36, 0xa0, 0x41, 0x0b, 0x00, 0x45, 0xf9, 0x11, 0x0f, 0x35, 0x31, + 0xe6, 0xef, 0x8f, 0x95, 0x3b, 0x78, 0x27, 0x51, 0x56, 0x60, 0xc6, 0x44, 0x2b, 0x6b, 0xc0, 0x48, + 0x8a, 0x56, 0x56, 0x84, 0x35, 0x44, 0x1b, 0x23, 0xb4, 0x1e, 0x44, 0xca, 0x1f, 0x5c, 0x39, 0x23, + 0xe5, 0x47, 0x2c, 0x2b, 0xd2, 0x04, 0x7a, 0x0a, 0x29, 0x32, 0x87, 0xa3, 0x28, 0x80, 0xd1, 0xb7, + 0x66, 0xcc, 0xdf, 0x1c, 0x99, 0xc7, 0x13, 0x59, 0xbe, 0xf7, 0xd3, 0xff, 0xb8, 0x30, 0xf1, 0xd3, + 0xb3, 0x05, 0xe1, 0x17, 0x67, 0x0b, 0xc2, 0x1f, 0x9d, 0x2d, 0x08, 0x7f, 0x7c, 0xb6, 0x20, 0xfc, + 0xe8, 0x57, 0x0b, 0x13, 0xbf, 0xf8, 0xd5, 0xc2, 0xc4, 0x1f, 0xfd, 0x6a, 0x61, 0xe2, 0x93, 0x69, + 0xce, 0xbd, 0x3b, 0x45, 0x27, 0xb8, 0x47, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xda, 0x0f, 0x74, + 0xf0, 0x96, 0x88, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -13652,6 +13864,162 @@ func (m *QueryResolvedTimestampResponse) MarshalToSizedBuffer(dAtA []byte) (int, return len(dAtA) - i, nil } +func (m *ScanInterleavedIntentsRequest) 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 *ScanInterleavedIntentsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ScanInterleavedIntentsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.RequestHeader.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *ScanInterleavedIntentsResponse) 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 *ScanInterleavedIntentsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ScanInterleavedIntentsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Intents) > 0 { + for iNdEx := len(m.Intents) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Intents[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + { + size, err := m.ResponseHeader.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *BarrierRequest) 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 *BarrierRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BarrierRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.RequestHeader.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *BarrierResponse) 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 *BarrierResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BarrierResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Timestamp.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.ResponseHeader.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func (m *RequestUnion) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -14670,6 +15038,52 @@ func (m *RequestUnion_QueryResolvedTimestamp) MarshalToSizedBuffer(dAtA []byte) } return len(dAtA) - i, nil } +func (m *RequestUnion_ScanInterleavedIntents) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RequestUnion_ScanInterleavedIntents) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ScanInterleavedIntents != nil { + { + size, err := m.ScanInterleavedIntents.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *RequestUnion_Barrier) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RequestUnion_Barrier) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Barrier != nil { + { + size, err := m.Barrier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} func (m *ResponseUnion) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -15665,6 +16079,52 @@ func (m *ResponseUnion_QueryResolvedTimestamp) MarshalToSizedBuffer(dAtA []byte) } return len(dAtA) - i, nil } +func (m *ResponseUnion_ScanInterleavedIntents) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ResponseUnion_ScanInterleavedIntents) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ScanInterleavedIntents != nil { + { + size, err := m.ScanInterleavedIntents.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ResponseUnion_Barrier) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ResponseUnion_Barrier) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Barrier != nil { + { + size, err := m.Barrier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} func (m *Header) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -16821,12 +17281,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n274, err274 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err274 != nil { - return 0, err274 + n283, err283 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err283 != nil { + return 0, err283 } - i -= n274 - i = encodeVarintApi(dAtA, i, uint64(n274)) + i -= n283 + i = encodeVarintApi(dAtA, i, uint64(n283)) i-- dAtA[i] = 0x1a { @@ -18708,6 +19168,58 @@ func (m *QueryResolvedTimestampResponse) Size() (n int) { return n } +func (m *ScanInterleavedIntentsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.RequestHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + +func (m *ScanInterleavedIntentsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.ResponseHeader.Size() + n += 1 + l + sovApi(uint64(l)) + if len(m.Intents) > 0 { + for _, e := range m.Intents { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + return n +} + +func (m *BarrierRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.RequestHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + +func (m *BarrierResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.ResponseHeader.Size() + n += 1 + l + sovApi(uint64(l)) + l = m.Timestamp.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + func (m *RequestUnion) Size() (n int) { if m == nil { return 0 @@ -19248,6 +19760,30 @@ func (m *RequestUnion_QueryResolvedTimestamp) Size() (n int) { } return n } +func (m *RequestUnion_ScanInterleavedIntents) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ScanInterleavedIntents != nil { + l = m.ScanInterleavedIntents.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} +func (m *RequestUnion_Barrier) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Barrier != nil { + l = m.Barrier.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *ResponseUnion) Size() (n int) { if m == nil { return 0 @@ -19776,6 +20312,30 @@ func (m *ResponseUnion_QueryResolvedTimestamp) Size() (n int) { } return n } +func (m *ResponseUnion_ScanInterleavedIntents) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ScanInterleavedIntents != nil { + l = m.ScanInterleavedIntents.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} +func (m *ResponseUnion_Barrier) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Barrier != nil { + l = m.Barrier.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *Header) Size() (n int) { if m == nil { return 0 @@ -33010,33 +33570,232 @@ func (m *AddSSTableRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.MVCCStats == nil { - m.MVCCStats = &enginepb.MVCCStats{} - } - if err := m.MVCCStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.MVCCStats == nil { + m.MVCCStats = &enginepb.MVCCStats{} + } + if err := m.MVCCStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IngestAsWrites", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IngestAsWrites = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AddSSTableResponse) 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 ErrIntOverflowApi + } + 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: AddSSTableResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddSSTableResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RefreshRequest) 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 ErrIntOverflowApi + } + 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: RefreshRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RefreshRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RefreshFrom", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RefreshFrom.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IngestAsWrites", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.IngestAsWrites = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -33058,7 +33817,7 @@ func (m *AddSSTableRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *AddSSTableResponse) Unmarshal(dAtA []byte) error { +func (m *RefreshResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33081,10 +33840,10 @@ func (m *AddSSTableResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AddSSTableResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AddSSTableResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33141,7 +33900,7 @@ func (m *AddSSTableResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshRequest) Unmarshal(dAtA []byte) error { +func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33164,10 +33923,10 @@ func (m *RefreshRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshRangeRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33257,7 +34016,7 @@ func (m *RefreshRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshResponse) Unmarshal(dAtA []byte) error { +func (m *RefreshRangeResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33280,10 +34039,10 @@ func (m *RefreshResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshRangeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33340,7 +34099,7 @@ func (m *RefreshResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { +func (m *SubsumeRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33363,10 +34122,10 @@ func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshRangeRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SubsumeRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SubsumeRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33402,9 +34161,42 @@ func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LeftDesc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.LeftDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RefreshFrom", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RightDesc", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33431,7 +34223,7 @@ func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RefreshFrom.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RightDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -33456,7 +34248,7 @@ func (m *RefreshRangeRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshRangeResponse) Unmarshal(dAtA []byte) error { +func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33479,10 +34271,10 @@ func (m *RefreshRangeResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshRangeResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SubsumeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SubsumeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33518,59 +34310,94 @@ func (m *RefreshRangeResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipApi(dAtA[iNdEx:]) - if err != nil { - return err + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MVCCStats", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { return ErrInvalidLengthApi } - if (iNdEx + skippy) > l { + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { return io.ErrUnexpectedEOF } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SubsumeRequest) 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 ErrIntOverflowApi + if err := m.MVCCStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - if iNdEx >= l { + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LeaseAppliedIndex", wireType) + } + m.LeaseAppliedIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LeaseAppliedIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FreezeStart", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if err := m.FreezeStart.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SubsumeRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SubsumeRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33597,13 +34424,13 @@ func (m *SubsumeRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LeftDesc", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReadSummary", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33630,13 +34457,66 @@ func (m *SubsumeRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.LeftDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.ReadSummary == nil { + m.ReadSummary = &rspb.ReadSummary{} + } + if err := m.ReadSummary.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RangeStatsRequest) 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 ErrIntOverflowApi + } + 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: RangeStatsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RangeStatsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RightDesc", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33663,7 +34543,7 @@ func (m *SubsumeRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RightDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -33688,7 +34568,7 @@ func (m *SubsumeRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { +func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33711,10 +34591,10 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SubsumeResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RangeStatsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SubsumeResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RangeStatsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33750,7 +34630,7 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field MVCCStats", wireType) } @@ -33783,11 +34663,22 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedLastQueriesPerSecond", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.DeprecatedLastQueriesPerSecond = float64(math.Float64frombits(v)) case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LeaseAppliedIndex", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RangeInfo", wireType) } - m.LeaseAppliedIndex = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowApi @@ -33797,16 +34688,41 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LeaseAppliedIndex |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RangeInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field FreezeStart", wireType) + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecond", wireType) } - var msglen int + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.MaxQueriesPerSecond = float64(math.Float64frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecondSet", wireType) + } + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowApi @@ -33816,28 +34732,65 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthApi + m.MaxQueriesPerSecondSet = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthApi } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if err := m.FreezeStart.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MigrateRequest) 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 ErrIntOverflowApi } - iNdEx = postIndex - case 6: + 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: MigrateRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MigrateRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33864,13 +34817,13 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 7: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReadSummary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33897,10 +34850,7 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ReadSummary == nil { - m.ReadSummary = &rspb.ReadSummary{} - } - if err := m.ReadSummary.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -33925,7 +34875,7 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RangeStatsRequest) Unmarshal(dAtA []byte) error { +func (m *MigrateResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33948,15 +34898,15 @@ func (m *RangeStatsRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RangeStatsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: MigrateResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RangeStatsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MigrateResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33983,7 +34933,7 @@ func (m *RangeStatsRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -34008,7 +34958,7 @@ func (m *RangeStatsRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { +func (m *QueryResolvedTimestampRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34031,92 +34981,15 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RangeStatsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: QueryResolvedTimestampRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RangeStatsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryResolvedTimestampRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthApi - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MVCCStats", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthApi - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.MVCCStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedLastQueriesPerSecond", wireType) - } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.DeprecatedLastQueriesPerSecond = float64(math.Float64frombits(v)) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RangeInfo", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -34143,41 +35016,10 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RangeInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 5: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecond", wireType) - } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.MaxQueriesPerSecond = float64(math.Float64frombits(v)) - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecondSet", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.MaxQueriesPerSecondSet = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -34199,7 +35041,7 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *MigrateRequest) Unmarshal(dAtA []byte) error { +func (m *QueryResolvedTimestampResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34222,15 +35064,15 @@ func (m *MigrateRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: MigrateRequest: wiretype end group for non-group") + return fmt.Errorf("proto: QueryResolvedTimestampResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: MigrateRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryResolvedTimestampResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -34257,13 +35099,13 @@ func (m *MigrateRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResolvedTS", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -34290,7 +35132,7 @@ func (m *MigrateRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ResolvedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -34315,7 +35157,7 @@ func (m *MigrateRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *MigrateResponse) Unmarshal(dAtA []byte) error { +func (m *ScanInterleavedIntentsRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34338,10 +35180,93 @@ func (m *MigrateResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: MigrateResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ScanInterleavedIntentsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: MigrateResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScanInterleavedIntentsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ScanInterleavedIntentsResponse) 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 ErrIntOverflowApi + } + 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: ScanInterleavedIntentsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScanInterleavedIntentsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -34377,6 +35302,40 @@ func (m *MigrateResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Intents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Intents = append(m.Intents, Intent{}) + if err := m.Intents[len(m.Intents)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -34398,7 +35357,7 @@ func (m *MigrateResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryResolvedTimestampRequest) Unmarshal(dAtA []byte) error { +func (m *BarrierRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34421,10 +35380,10 @@ func (m *QueryResolvedTimestampRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryResolvedTimestampRequest: wiretype end group for non-group") + return fmt.Errorf("proto: BarrierRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryResolvedTimestampRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BarrierRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -34481,7 +35440,7 @@ func (m *QueryResolvedTimestampRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryResolvedTimestampResponse) Unmarshal(dAtA []byte) error { +func (m *BarrierResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34504,10 +35463,10 @@ func (m *QueryResolvedTimestampResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryResolvedTimestampResponse: wiretype end group for non-group") + return fmt.Errorf("proto: BarrierResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryResolvedTimestampResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BarrierResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -34545,7 +35504,7 @@ func (m *QueryResolvedTimestampResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResolvedTS", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -34572,7 +35531,7 @@ func (m *QueryResolvedTimestampResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.ResolvedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Timestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -36166,6 +37125,76 @@ func (m *RequestUnion) Unmarshal(dAtA []byte) error { } m.Value = &RequestUnion_QueryResolvedTimestamp{v} iNdEx = postIndex + case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ScanInterleavedIntents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ScanInterleavedIntentsRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_ScanInterleavedIntents{v} + iNdEx = postIndex + case 53: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Barrier", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &BarrierRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_Barrier{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -37721,6 +38750,76 @@ func (m *ResponseUnion) Unmarshal(dAtA []byte) error { } m.Value = &ResponseUnion_QueryResolvedTimestamp{v} iNdEx = postIndex + case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ScanInterleavedIntents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ScanInterleavedIntentsResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_ScanInterleavedIntents{v} + iNdEx = postIndex + case 53: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Barrier", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &BarrierResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_Barrier{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 041c0b32dfc6..1411ab62aa7c 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1795,6 +1795,39 @@ message QueryResolvedTimestampResponse { (gogoproto.nullable) = false, (gogoproto.customname) = "ResolvedTS"]; } +// ScanInterleavedIntentsRequest is the request for a ScanInterleavedIntents operation. +// This is a read-only operation that returns all interleaved (non-separated) +// intents found over the request range. +message ScanInterleavedIntentsRequest { + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + +// ScanInterleavedIntentsResponse is the response to a ScanInterleavedIntents operation. +message ScanInterleavedIntentsResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + + // The intents encountered in the part of the request span that was + // evaluated. A resume span is set in the response header if the entirety of + // the request span was not evaluated. + repeated Intent intents = 3 [(gogoproto.nullable) = false]; +} + +// BarrierRequest is the request for a Barrier operation. This goes through Raft +// and has the purpose of waiting until all conflicting in-flight operations on +// this range have completed, without blocking any new operations. +message BarrierRequest { + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + +// BarrierResponse is the response for a Barrier operation. +message BarrierResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + + // Timestamp at which this Barrier was evaluated. Can be used to guarantee + // future operations happen on the same or newer leaseholders. + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; +} + // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. // @@ -1846,6 +1879,8 @@ message RequestUnion { AdminVerifyProtectedTimestampRequest admin_verify_protected_timestamp = 49; MigrateRequest migrate = 50; QueryResolvedTimestampRequest query_resolved_timestamp = 51; + ScanInterleavedIntentsRequest scan_interleaved_intents = 52; + BarrierRequest barrier = 53; } reserved 8, 15, 23, 25, 27, 31, 34; } @@ -1897,6 +1932,8 @@ message ResponseUnion { AdminVerifyProtectedTimestampResponse admin_verify_protected_timestamp = 49; MigrateResponse migrate = 50; QueryResolvedTimestampResponse query_resolved_timestamp = 51; + ScanInterleavedIntentsResponse scan_interleaved_intents = 52; + BarrierResponse barrier = 53; } reserved 8, 15, 23, 25, 27, 28, 31, 34; } diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 1d091d5551f8..c4818ab1346b 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -168,6 +168,10 @@ func (ru RequestUnion) GetInner() Request { return t.Migrate case *RequestUnion_QueryResolvedTimestamp: return t.QueryResolvedTimestamp + case *RequestUnion_ScanInterleavedIntents: + return t.ScanInterleavedIntents + case *RequestUnion_Barrier: + return t.Barrier default: return nil } @@ -262,6 +266,10 @@ func (ru ResponseUnion) GetInner() Response { return t.Migrate case *ResponseUnion_QueryResolvedTimestamp: return t.QueryResolvedTimestamp + case *ResponseUnion_ScanInterleavedIntents: + return t.ScanInterleavedIntents + case *ResponseUnion_Barrier: + return t.Barrier default: return nil } @@ -431,6 +439,10 @@ func (ru *RequestUnion) MustSetInner(r Request) { union = &RequestUnion_Migrate{t} case *QueryResolvedTimestampRequest: union = &RequestUnion_QueryResolvedTimestamp{t} + case *ScanInterleavedIntentsRequest: + union = &RequestUnion_ScanInterleavedIntents{t} + case *BarrierRequest: + union = &RequestUnion_Barrier{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } @@ -528,13 +540,17 @@ func (ru *ResponseUnion) MustSetInner(r Response) { union = &ResponseUnion_Migrate{t} case *QueryResolvedTimestampResponse: union = &ResponseUnion_QueryResolvedTimestamp{t} + case *ScanInterleavedIntentsResponse: + union = &ResponseUnion_ScanInterleavedIntents{t} + case *BarrierResponse: + union = &ResponseUnion_Barrier{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } ru.Value = union } -type reqCounts [44]int32 +type reqCounts [46]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -630,6 +646,10 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[42]++ case *RequestUnion_QueryResolvedTimestamp: counts[43]++ + case *RequestUnion_ScanInterleavedIntents: + counts[44]++ + case *RequestUnion_Barrier: + counts[45]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -682,6 +702,8 @@ var requestNames = []string{ "AdmVerifyProtectedTimestamp", "Migrate", "QueryResolvedTimestamp", + "ScanInterleavedIntents", + "Barrier", } // Summary prints a short summary of the requests in a batch. @@ -889,6 +911,14 @@ type queryResolvedTimestampResponseAlloc struct { union ResponseUnion_QueryResolvedTimestamp resp QueryResolvedTimestampResponse } +type scanInterleavedIntentsResponseAlloc struct { + union ResponseUnion_ScanInterleavedIntents + resp ScanInterleavedIntentsResponse +} +type barrierResponseAlloc struct { + union ResponseUnion_Barrier + resp BarrierResponse +} // CreateReply creates replies for each of the contained requests, wrapped in a // BatchResponse. The response objects are batch allocated to minimize @@ -943,6 +973,8 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf41 []adminVerifyProtectedTimestampResponseAlloc var buf42 []migrateResponseAlloc var buf43 []queryResolvedTimestampResponseAlloc + var buf44 []scanInterleavedIntentsResponseAlloc + var buf45 []barrierResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -1254,6 +1286,20 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf43[0].union.QueryResolvedTimestamp = &buf43[0].resp br.Responses[i].Value = &buf43[0].union buf43 = buf43[1:] + case *RequestUnion_ScanInterleavedIntents: + if buf44 == nil { + buf44 = make([]scanInterleavedIntentsResponseAlloc, counts[44]) + } + buf44[0].union.ScanInterleavedIntents = &buf44[0].resp + br.Responses[i].Value = &buf44[0].union + buf44 = buf44[1:] + case *RequestUnion_Barrier: + if buf45 == nil { + buf45 = make([]barrierResponseAlloc, counts[45]) + } + buf45[0].union.Barrier = &buf45[0].resp + br.Responses[i].Value = &buf45[0].union + buf45 = buf45[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } @@ -1352,6 +1398,10 @@ func CreateRequest(method Method) Request { return &MigrateRequest{} case QueryResolvedTimestamp: return &QueryResolvedTimestampRequest{} + case ScanInterleavedIntents: + return &ScanInterleavedIntentsRequest{} + case Barrier: + return &BarrierRequest{} default: panic(fmt.Sprintf("unsupported method: %+v", method)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index aaad6cbbc271..3bf4e1b5954f 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -162,6 +162,13 @@ const ( // QueryResolvedTimestamp requests the resolved timestamp of the key span it // is issued over. QueryResolvedTimestamp + // ScanInterleavedIntents is a command to return interleaved intents + // encountered over a key range. + ScanInterleavedIntents + // Barrier is a command that ensures all conflicting in-flight operations on + // this range before this command have finished by the time it returns. It + // does not block new operations that started after this command's evaluation. + Barrier // NumMethods represents the total number of API methods. NumMethods ) diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index c4a2e66741d5..925689e4557f 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -53,12 +53,14 @@ func _() { _ = x[RangeStats-42] _ = x[AdminVerifyProtectedTimestamp-43] _ = x[QueryResolvedTimestamp-44] - _ = x[NumMethods-45] + _ = x[ScanInterleavedIntents-45] + _ = x[Barrier-46] + _ = x[NumMethods-47] } -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportAdminScatterAddSSTableMigrateRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampQueryResolvedTimestampNumMethods" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportAdminScatterAddSSTableMigrateRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampQueryResolvedTimestampScanInterleavedIntentsBarrierNumMethods" -var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 214, 225, 238, 256, 261, 272, 284, 297, 306, 321, 337, 344, 354, 360, 372, 382, 389, 403, 410, 422, 429, 439, 468, 490, 500} +var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 214, 225, 238, 256, 261, 272, 284, 297, 306, 321, 337, 344, 354, 360, 372, 382, 389, 403, 410, 422, 429, 439, 468, 490, 512, 519, 529} func (i Method) String() string { if i < 0 || i >= Method(len(_Method_index)-1) { diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 74ce58dcbf2a..2f21e9a06a33 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -780,19 +780,28 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // tenant. Regular tenants are disallowed from changing cluster // versions. var c migration.Cluster + var systemDeps migration.SystemDeps if codec.ForSystemTenant() { c = migrationcluster.New(migrationcluster.ClusterConfig{ NodeLiveness: nodeLiveness, Dialer: cfg.nodeDialer, DB: cfg.db, + Stopper: cfg.stopper, + DistSender: cfg.distSender, }) + systemDeps = migration.SystemDeps{ + Cluster: c, + DB: cfg.db, + DistSender: cfg.distSender, + Stopper: cfg.stopper, + } } else { c = migrationcluster.NewTenantCluster(cfg.db) } knobs, _ := cfg.TestingKnobs.MigrationManager.(*migrationmanager.TestingKnobs) migrationMgr := migrationmanager.NewManager( - c, cfg.circularInternalExecutor, jobRegistry, codec, cfg.Settings, knobs, + systemDeps, cfg.circularInternalExecutor, jobRegistry, codec, cfg.Settings, knobs, ) execCfg.MigrationJobDeps = migrationMgr execCfg.VersionUpgradeHook = migrationMgr.Migrate diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index 9bd6e8cb72be..e1c03f514ade 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -42,6 +42,9 @@ import ( // running with this flag set to true for some time, most ranges will only // have separated intents. Similarly, setting this to false will gradually // cause most ranges to only have interleaved intents. +// +// TODO(bilal): Remove this setting and replace it with a testing knob, as we +// no longer expect this setting to be toggled to false outside of tests. var SeparatedIntentsEnabled = settings.RegisterBoolSetting( "storage.transaction.separated_intents.enabled", "if enabled, intents will be written to a separate lock table, instead of being "+ diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 8fe18efe6949..4460d10761a8 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -306,6 +306,7 @@ var charts = []sectionDescription{ "distsender.rpc.leaseinfo.sent", "distsender.rpc.merge.sent", "distsender.rpc.migrate.sent", + "distsender.rpc.migratelocktable.sent", "distsender.rpc.pushtxn.sent", "distsender.rpc.put.sent", "distsender.rpc.queryintent.sent",