From 6b8f350cf18a5667a6c8527a8aff9b83f4874308 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 2 Nov 2021 15:55:21 +0000 Subject: [PATCH 1/4] storage: tweak `ScanIntents()` This renames `ScanIntents` from `ScanSeparatedIntents`, and adds a context parameter. Callers have been updated to pass `storage.mvcc.max_intents_per_error` for the intent limit, as is done elsewhere, and `targetBytes` is passed as 0 for now (no limit) since other intent collectors currently don't use a byte limit. Release note: None --- pkg/kv/kvserver/batcheval/cmd_clear_range.go | 17 +------- .../batcheval/cmd_clear_range_test.go | 21 ++++++--- pkg/storage/engine.go | 43 ++++++++++--------- pkg/storage/engine_test.go | 10 +++-- 4 files changed, 47 insertions(+), 44 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index ee21e0465e15..d198cc13a7e3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -79,21 +79,8 @@ func ClearRange( // txns. Otherwise, txn recovery would fail to find these intents and // consider the txn incomplete, uncommitting it and its writes (even those // outside of the cleared range). - // - // We return 1000 at a time, or 1 MB. The intent resolver currently - // processes intents in batches of 100, so this gives it a few to chew on. - // - // NOTE: This only takes into account separated intents, which are currently - // not enabled by default. For interleaved intents we would have to do full - // range scans, which would be too expensive. We could mitigate this by - // relying on statistics to skip scans when no intents are known, but due - // to #60585 we are often likely to encounter intents. See discussion in: - // https://github.com/cockroachdb/cockroach/pull/61850 - var ( - maxIntents int64 = 1000 - intentBytes int64 = 1e6 - ) - intents, err := storage.ScanSeparatedIntents(readWriter, from, to, maxIntents, intentBytes) + maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) + intents, err := storage.ScanIntents(ctx, readWriter, from, to, maxIntents, 0) if err != nil { return result.Result{}, err } else if len(intents) > 0 { diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index 143439f084bd..105061c936ff 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "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" @@ -107,7 +108,12 @@ func TestCmdClearRangeBytesThreshold(t *testing.T) { h.RangeID = desc.RangeID cArgs := CommandArgs{Header: h} - cArgs.EvalCtx = (&MockEvalCtx{Desc: &desc, Clock: hlc.NewClock(hlc.UnixNano, time.Nanosecond), Stats: stats}).EvalContext() + cArgs.EvalCtx = (&MockEvalCtx{ + ClusterSettings: cluster.MakeTestingClusterSettings(), + Desc: &desc, + Clock: hlc.NewClock(hlc.UnixNano, time.Nanosecond), + Stats: stats, + }).EvalContext() cArgs.Args = &roachpb.ClearRangeRequest{ RequestHeader: roachpb.RequestHeader{ Key: startKey, @@ -172,10 +178,15 @@ func TestCmdClearRangeDeadline(t *testing.T) { } cArgs := CommandArgs{ - Header: roachpb.Header{RangeID: desc.RangeID}, - EvalCtx: (&MockEvalCtx{Desc: &desc, Clock: clock, Stats: stats}).EvalContext(), - Stats: &enginepb.MVCCStats{}, - Args: &args, + Header: roachpb.Header{RangeID: desc.RangeID}, + EvalCtx: (&MockEvalCtx{ + ClusterSettings: cluster.MakeTestingClusterSettings(), + Desc: &desc, + Clock: clock, + Stats: stats, + }).EvalContext(), + Stats: &enginepb.MVCCStats{}, + Args: &args, } batch := eng.NewBatch() diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index e48b36cb9299..1047923b4e98 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -972,16 +972,15 @@ func Scan(reader Reader, start, end roachpb.Key, max int64) ([]MVCCKeyValue, err return kvs, err } -// ScanSeparatedIntents scans intents using only the separated intents lock -// table. It does not take interleaved intents into account at all. -// -// TODO(erikgrinaker): When we are fully migrated to separated intents, this -// should be renamed ScanIntents. -func ScanSeparatedIntents( - reader Reader, start, end roachpb.Key, max int64, targetBytes int64, +// ScanIntents scans intents using only the separated intents lock table. It +// does not take interleaved intents into account at all. +func ScanIntents( + ctx context.Context, reader Reader, start, end roachpb.Key, maxIntents int64, targetBytes int64, ) ([]roachpb.Intent, error) { + intents := []roachpb.Intent{} + if bytes.Compare(start, end) >= 0 { - return []roachpb.Intent{}, nil + return intents, nil } ltStart, _ := keys.LockTableSingleKey(start, nil) @@ -989,14 +988,18 @@ func ScanSeparatedIntents( iter := reader.NewEngineIterator(IterOptions{LowerBound: ltStart, UpperBound: ltEnd}) defer iter.Close() - var ( - intents = []roachpb.Intent{} - intentBytes int64 - meta enginepb.MVCCMetadata - ) - valid, err := iter.SeekEngineKeyGE(EngineKey{Key: ltStart}) - for ; valid; valid, err = iter.NextEngineKey() { - if max != 0 && int64(len(intents)) >= max { + var meta enginepb.MVCCMetadata + var intentBytes int64 + var ok bool + var err error + for ok, err = iter.SeekEngineKeyGE(EngineKey{Key: ltStart}); ok; ok, err = iter.NextEngineKey() { + if err := ctx.Err(); err != nil { + return nil, err + } + if maxIntents != 0 && int64(len(intents)) >= maxIntents { + break + } + if targetBytes != 0 && intentBytes >= targetBytes { break } key, err := iter.EngineKey() @@ -1012,11 +1015,11 @@ func ScanSeparatedIntents( } intents = append(intents, roachpb.MakeIntent(meta.Txn, lockedKey)) intentBytes += int64(len(lockedKey)) + int64(len(iter.Value())) - if (max > 0 && int64(len(intents)) >= max) || (targetBytes > 0 && intentBytes >= targetBytes) { - break - } } - return intents, err + if err != nil { + return nil, err + } + return intents, nil } // WriteSyncNoop carries out a synchronous no-op write to the engine. diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index d87dd9156c5c..096ec7bc1e93 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1678,7 +1678,7 @@ func TestFS(t *testing.T) { } } -func TestScanSeparatedIntents(t *testing.T) { +func TestScanIntents(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1705,8 +1705,10 @@ func TestScanSeparatedIntents(t *testing.T) { "offset last": {keys[2], maxKey, 0, 0, keys[2:]}, "offset post": {roachpb.Key("x"), maxKey, 0, 0, []roachpb.Key{}}, "nil end": {keys[0], nil, 0, 0, []roachpb.Key{}}, - "limit keys": {keys[0], maxKey, 2, 0, keys[0:2]}, - "one byte": {keys[0], maxKey, 0, 1, keys[0:1]}, + "-1 max": {keys[0], maxKey, -1, 0, keys[:0]}, + "2 max": {keys[0], maxKey, 2, 0, keys[0:2]}, + "-1 byte": {keys[0], maxKey, 0, -1, keys[:0]}, + "1 byte": {keys[0], maxKey, 0, 1, keys[0:1]}, "80 bytes": {keys[0], maxKey, 0, 80, keys[0:2]}, "80 bytes or one": {keys[0], maxKey, 1, 80, keys[0:1]}, "1000 bytes": {keys[0], maxKey, 0, 1000, keys}, @@ -1727,7 +1729,7 @@ func TestScanSeparatedIntents(t *testing.T) { for name, tc := range testcases { tc := tc t.Run(name, func(t *testing.T) { - intents, err := ScanSeparatedIntents(eng, tc.from, tc.to, tc.max, tc.targetBytes) + intents, err := ScanIntents(ctx, eng, tc.from, tc.to, tc.max, tc.targetBytes) require.NoError(t, err) if enableSeparatedIntents { require.Len(t, intents, len(tc.expectIntents), "unexpected number of separated intents") From c6b72c2f379017fcc2cfe040dfdb9509457d6679 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 11 Nov 2021 11:35:48 +0000 Subject: [PATCH 2/4] roachpb: assert request flag combinations Request flags have implicit dependencies and incompatibilities (e.g. `isLocking` implies `isTxn`). However, these were never checked and developers were expected to satisfy them manually, which is error-prone. This patch adds `TestFlagCombinations` that checks these dependencies and incompatibilities, based on `flagDependencies` and `flagExclusions` maps which encodes them. It also adds a new `flag` type for flags, renames `skipLeaseCheck` to `skipsLeaseCheck`, and adds `isAlone` for `CheckConsistencyRequest`. Release note: None --- pkg/kv/kvserver/replica.go | 2 +- pkg/kv/kvserver/replica_raft.go | 2 +- pkg/roachpb/BUILD.bazel | 2 + pkg/roachpb/api.go | 151 ++++++++++++++++++-------------- pkg/roachpb/api_test.go | 43 +++++++++ pkg/roachpb/batch.go | 16 ++-- pkg/testutils/lint/lint_test.go | 2 + 7 files changed, 140 insertions(+), 78 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 6c19bcc7e9aa..e81a849322bd 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1386,7 +1386,7 @@ func (r *Replica) checkGCThresholdAndLeaseRLocked( // // If the request is an INCONSISTENT request (and thus a read), it similarly // doesn't check the lease. - leaseChecked := !ba.IsSingleSkipLeaseCheckRequest() && ba.ReadConsistency != roachpb.INCONSISTENT + leaseChecked := !ba.IsSingleSkipsLeaseCheckRequest() && ba.ReadConsistency != roachpb.INCONSISTENT if leaseChecked { // Now check the lease. var err error diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 84ee914d6b7f..584294e46616 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -174,7 +174,7 @@ func (r *Replica) evalAndPropose( // reference the previous lease. Note that TransferLease also skip lease // checks (for technical reasons, see `TransferLease.flags`) and uses the // same mechanism. - if ba.IsSingleSkipLeaseCheckRequest() { + if ba.IsSingleSkipsLeaseCheckRequest() { proposal.command.ProposerLeaseSequence = ba.GetPrevLeaseForLeaseRequest().Sequence } else if !st.Lease.OwnedBy(r.store.StoreID()) { // Perform a sanity check that the lease is owned by this replica. This must diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 0ef0fee92502..766d888fd01d 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -171,6 +171,8 @@ go_test( "@com_github_cockroachdb_apd_v2//:apd", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", + "@com_github_gogo_protobuf//proto", + "@com_github_golang_protobuf//proto:go_default_library", "@com_github_kr_pretty//:pretty", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 12765c3f5806..712a5a8f8a0f 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -71,25 +71,40 @@ func (rc ReadConsistencyType) SupportsBatch(ba BatchRequest) error { panic("unreachable") } +type flag int + const ( - isAdmin = 1 << iota // admin cmds don't go through raft, but run on lease holder - isRead // read-only cmds don't go through raft, but may run on lease holder - isWrite // write cmds go through raft and must be proposed on lease holder - isTxn // txn commands may be part of a transaction - isLocking // locking cmds acquire locks for their transaction (implies isTxn) - isIntentWrite // intent write cmds leave intents when they succeed (implies isWrite and isLocking) - isRange // range commands may span multiple keys - isReverse // reverse commands traverse ranges in descending direction - isAlone // requests which must be alone in a batch - isPrefix // requests which should be grouped with the next request in a batch - isUnsplittable // range command that must not be split during sending - skipLeaseCheck // commands which skip the check that the evaluating replica has a valid lease - updatesTSCache // commands which update the timestamp cache - updatesTSCacheOnErr // commands which make read data available on errors - needsRefresh // commands which require refreshes to avoid serializable retries - canBackpressure // commands which deserve backpressure when a Range grows too large + isAdmin flag = 1 << iota // admin cmds don't go through raft, but run on lease holder + isRead // read-only cmds don't go through raft, but may run on lease holder + isWrite // write cmds go through raft and must be proposed on lease holder + isTxn // txn commands may be part of a transaction + isLocking // locking cmds acquire locks for their transaction + isIntentWrite // intent write cmds leave intents when they succeed + isRange // range commands may span multiple keys + isReverse // reverse commands traverse ranges in descending direction + isAlone // requests which must be alone in a batch + isPrefix // requests which, in a batch, must not be split from the following request + isUnsplittable // range command that must not be split during sending + skipsLeaseCheck // commands which skip the check that the evaluating replica has a valid lease + updatesTSCache // commands which update the timestamp cache + updatesTSCacheOnErr // commands which make read data available on errors + needsRefresh // commands which require refreshes to avoid serializable retries + canBackpressure // commands which deserve backpressure when a Range grows too large ) +// flagDependencies specifies flag dependencies, asserted by TestFlagCombinations. +var flagDependencies = map[flag][]flag{ + isAdmin: {isAlone}, + isLocking: {isTxn}, + isIntentWrite: {isWrite, isLocking}, + skipsLeaseCheck: {isAlone}, +} + +// flagExclusions specifies flag incompatibilities, asserted by TestFlagCombinations. +var flagExclusions = map[flag][]flag{ + skipsLeaseCheck: {isIntentWrite}, +} + // IsReadOnly returns true iff the request is read-only. A request is // read-only if it does not go through raft, meaning that it cannot // change any replicated state. However, read-only requests may still @@ -187,7 +202,7 @@ type Request interface { Method() Method // ShallowCopy returns a shallow copy of the receiver. ShallowCopy() Request - flags() int + flags() flag } // SizedWriteRequest is an interface used to expose the number of bytes a @@ -1160,19 +1175,19 @@ func scanLockStrength(forUpdate bool) lock.Strength { return lock.None } -func flagForLockStrength(l lock.Strength) int { +func flagForLockStrength(l lock.Strength) flag { if l != lock.None { return isLocking } return 0 } -func (gr *GetRequest) flags() int { +func (gr *GetRequest) flags() flag { maybeLocking := flagForLockStrength(gr.KeyLocking) return isRead | isTxn | maybeLocking | updatesTSCache | needsRefresh } -func (*PutRequest) flags() int { +func (*PutRequest) flags() flag { return isWrite | isTxn | isLocking | isIntentWrite | canBackpressure } @@ -1181,7 +1196,7 @@ func (*PutRequest) flags() int { // ConditionalPuts do not require a refresh because on write-too-old errors, // they return an error immediately instead of continuing a serializable // transaction to be retried at end transaction. -func (*ConditionalPutRequest) flags() int { +func (*ConditionalPutRequest) flags() flag { return isRead | isWrite | isTxn | isLocking | isIntentWrite | updatesTSCache | updatesTSCacheOnErr | canBackpressure } @@ -1190,7 +1205,7 @@ func (*ConditionalPutRequest) flags() int { // InitPuts do not require a refresh because on write-too-old errors, they // return an error immediately instead of continuing a serializable transaction // to be retried at end transaction. -func (*InitPutRequest) flags() int { +func (*InitPutRequest) flags() flag { return isRead | isWrite | isTxn | isLocking | isIntentWrite | updatesTSCache | updatesTSCacheOnErr | canBackpressure } @@ -1199,15 +1214,15 @@ func (*InitPutRequest) flags() int { // require a refresh because on write-too-old errors, they return an // error immediately instead of continuing a serializable transaction // to be retried at end transaction. -func (*IncrementRequest) flags() int { +func (*IncrementRequest) flags() flag { return isRead | isWrite | isTxn | isLocking | isIntentWrite | canBackpressure } -func (*DeleteRequest) flags() int { +func (*DeleteRequest) flags() flag { return isWrite | isTxn | isLocking | isIntentWrite | canBackpressure } -func (drr *DeleteRangeRequest) flags() int { +func (drr *DeleteRangeRequest) flags() flag { // DeleteRangeRequest has different properties if the "inline" flag is set. // This flag indicates that the request is deleting inline MVCC values, // which cannot be deleted transactionally - inline DeleteRange will thus @@ -1236,18 +1251,18 @@ func (drr *DeleteRangeRequest) flags() int { // Note that ClearRange commands cannot be part of a transaction as // they clear all MVCC versions. -func (*ClearRangeRequest) flags() int { return isWrite | isRange | isAlone } +func (*ClearRangeRequest) flags() flag { return isWrite | isRange | isAlone } // Note that RevertRange commands cannot be part of a transaction as // they clear all MVCC versions above their target time. -func (*RevertRangeRequest) flags() int { return isWrite | isRange } +func (*RevertRangeRequest) flags() flag { return isWrite | isRange } -func (sr *ScanRequest) flags() int { +func (sr *ScanRequest) flags() flag { maybeLocking := flagForLockStrength(sr.KeyLocking) return isRead | isRange | isTxn | maybeLocking | updatesTSCache | needsRefresh } -func (rsr *ReverseScanRequest) flags() int { +func (rsr *ReverseScanRequest) flags() flag { maybeLocking := flagForLockStrength(rsr.KeyLocking) return isRead | isRange | isReverse | isTxn | maybeLocking | updatesTSCache | needsRefresh } @@ -1255,43 +1270,43 @@ func (rsr *ReverseScanRequest) flags() int { // EndTxn updates the timestamp cache to prevent replays. // Replays for the same transaction key and timestamp will have // Txn.WriteTooOld=true and must retry on EndTxn. -func (*EndTxnRequest) flags() int { return isWrite | isTxn | isAlone | updatesTSCache } -func (*AdminSplitRequest) flags() int { return isAdmin | isAlone } -func (*AdminUnsplitRequest) flags() int { return isAdmin | isAlone } -func (*AdminMergeRequest) flags() int { return isAdmin | isAlone } -func (*AdminTransferLeaseRequest) flags() int { return isAdmin | isAlone } -func (*AdminChangeReplicasRequest) flags() int { return isAdmin | isAlone } -func (*AdminRelocateRangeRequest) flags() int { return isAdmin | isAlone } -func (*GCRequest) flags() int { return isWrite | isRange } +func (*EndTxnRequest) flags() flag { return isWrite | isTxn | isAlone | updatesTSCache } +func (*AdminSplitRequest) flags() flag { return isAdmin | isAlone } +func (*AdminUnsplitRequest) flags() flag { return isAdmin | isAlone } +func (*AdminMergeRequest) flags() flag { return isAdmin | isAlone } +func (*AdminTransferLeaseRequest) flags() flag { return isAdmin | isAlone } +func (*AdminChangeReplicasRequest) flags() flag { return isAdmin | isAlone } +func (*AdminRelocateRangeRequest) flags() flag { return isAdmin | isAlone } +func (*GCRequest) flags() flag { return isWrite | isRange } // HeartbeatTxn updates the timestamp cache with transaction records, // to avoid checking for them on disk when considering 1PC evaluation. -func (*HeartbeatTxnRequest) flags() int { return isWrite | isTxn | updatesTSCache } +func (*HeartbeatTxnRequest) flags() flag { return isWrite | isTxn | updatesTSCache } // PushTxnRequest updates different marker keys in the timestamp cache when // pushing a transaction's timestamp and when aborting a transaction. -func (*PushTxnRequest) flags() int { +func (*PushTxnRequest) flags() flag { return isWrite | isAlone | updatesTSCache | updatesTSCache } -func (*RecoverTxnRequest) flags() int { return isWrite | isAlone | updatesTSCache } -func (*QueryTxnRequest) flags() int { return isRead | isAlone } +func (*RecoverTxnRequest) flags() flag { return isWrite | isAlone | updatesTSCache } +func (*QueryTxnRequest) flags() flag { return isRead | isAlone } // QueryIntent only updates the timestamp cache when attempting to prevent an // intent that is found missing from ever being written in the future. See // QueryIntentRequest_PREVENT. -func (*QueryIntentRequest) flags() int { +func (*QueryIntentRequest) flags() flag { return isRead | isPrefix | updatesTSCache | updatesTSCacheOnErr } -func (*ResolveIntentRequest) flags() int { return isWrite } -func (*ResolveIntentRangeRequest) flags() int { return isWrite | isRange } -func (*TruncateLogRequest) flags() int { return isWrite } -func (*MergeRequest) flags() int { return isWrite | canBackpressure } -func (*RequestLeaseRequest) flags() int { return isWrite | isAlone | skipLeaseCheck } +func (*ResolveIntentRequest) flags() flag { return isWrite } +func (*ResolveIntentRangeRequest) flags() flag { return isWrite | isRange } +func (*TruncateLogRequest) flags() flag { return isWrite } +func (*MergeRequest) flags() flag { return isWrite | canBackpressure } +func (*RequestLeaseRequest) flags() flag { return isWrite | isAlone | skipsLeaseCheck } // LeaseInfoRequest is usually executed in an INCONSISTENT batch, which has the -// effect of the `skipLeaseCheck` flag that lease write operations have. -func (*LeaseInfoRequest) flags() int { return isRead | isAlone } -func (*TransferLeaseRequest) flags() int { +// effect of the `skipsLeaseCheck` flag that lease write operations have. +func (*LeaseInfoRequest) flags() flag { return isRead | isAlone } +func (*TransferLeaseRequest) flags() flag { // TransferLeaseRequest requires the lease, which is checked in // `AdminTransferLease()` before the TransferLeaseRequest is created and sent // for evaluation and in the usual way at application time (i.e. @@ -1299,38 +1314,38 @@ func (*TransferLeaseRequest) flags() int { // command resulting from the evaluation of TransferLeaseRequest was // proposed). // - // But we're marking it with skipLeaseCheck because `redirectOnOrAcquireLease` + // But we're marking it with skipsLeaseCheck because `redirectOnOrAcquireLease` // can't be used before evaluation as, by the time that call would be made, // the store has registered that a transfer is in progress and // `redirectOnOrAcquireLease` would already tentatively redirect to the future // lease holder. - return isWrite | isAlone | skipLeaseCheck -} -func (*RecomputeStatsRequest) flags() int { return isWrite | isAlone } -func (*ComputeChecksumRequest) flags() int { return isWrite } -func (*CheckConsistencyRequest) flags() int { return isAdmin | isRange } -func (*ExportRequest) flags() int { return isRead | isRange | updatesTSCache } -func (*AdminScatterRequest) flags() int { return isAdmin | isRange | isAlone } -func (*AdminVerifyProtectedTimestampRequest) flags() int { return isAdmin | isRange | isAlone } -func (*AddSSTableRequest) flags() int { + return isWrite | isAlone | skipsLeaseCheck +} +func (*RecomputeStatsRequest) flags() flag { return isWrite | isAlone } +func (*ComputeChecksumRequest) flags() flag { return isWrite } +func (*CheckConsistencyRequest) flags() flag { return isAdmin | isRange | isAlone } +func (*ExportRequest) flags() flag { return isRead | isRange | updatesTSCache } +func (*AdminScatterRequest) flags() flag { return isAdmin | isRange | isAlone } +func (*AdminVerifyProtectedTimestampRequest) flags() flag { return isAdmin | isRange | isAlone } +func (*AddSSTableRequest) flags() flag { return isWrite | isRange | isAlone | isUnsplittable | canBackpressure } -func (*MigrateRequest) flags() int { return isWrite | isRange | isAlone } +func (*MigrateRequest) flags() flag { return isWrite | isRange | isAlone } // RefreshRequest and RefreshRangeRequest both determine which timestamp cache // they update based on their Write parameter. -func (r *RefreshRequest) flags() int { +func (r *RefreshRequest) flags() flag { return isRead | isTxn | updatesTSCache } -func (r *RefreshRangeRequest) flags() int { +func (r *RefreshRangeRequest) flags() flag { return isRead | isTxn | isRange | updatesTSCache } -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 } +func (*SubsumeRequest) flags() flag { return isRead | isAlone | updatesTSCache } +func (*RangeStatsRequest) flags() flag { return isRead } +func (*QueryResolvedTimestampRequest) flags() flag { return isRead | isRange } +func (*ScanInterleavedIntentsRequest) flags() flag { return isRead | isRange } +func (*BarrierRequest) flags() flag { 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_test.go b/pkg/roachpb/api_test.go index 9f4279925035..d4a6a59880bc 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -14,9 +14,12 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/redact" + gogoproto "github.com/gogo/protobuf/proto" + "github.com/golang/protobuf/proto" // nolint deprecated, but required for Protobuf v1 reflection "github.com/stretchr/testify/require" ) @@ -306,3 +309,43 @@ func TestTenantConsumptionAddSub(t *testing.T) { t.Errorf("expected\n%#v\ngot\n%#v", exp, c) } } + +// TestFlagCombinations tests that flag dependencies and exclusions as specified +// in flagDependencies and flagExclusions are satisfied by all requests. +func TestFlagCombinations(t *testing.T) { + // Any non-zero-valued request variants that conditionally affect flags. + reqVariants := []Request{ + &DeleteRangeRequest{Inline: true}, + &GetRequest{KeyLocking: lock.Exclusive}, + &ReverseScanRequest{KeyLocking: lock.Exclusive}, + &ScanRequest{KeyLocking: lock.Exclusive}, + } + + reqTypes := []Request{} + oneofFields := proto.MessageReflect(&RequestUnion{}).Descriptor().Oneofs().Get(0).Fields() + for i := 0; i < oneofFields.Len(); i++ { + msgName := string(oneofFields.Get(i).Message().FullName()) + msgType := gogoproto.MessageType(msgName).Elem() + require.NotNil(t, msgType, "unknown message type %s", msgName) + reqTypes = append(reqTypes, reflect.New(msgType).Interface().(Request)) + } + + for _, req := range append(reqTypes, reqVariants...) { + name := reflect.TypeOf(req).Elem().Name() + flags := req.flags() + for flag, deps := range flagDependencies { + if flags&flag != 0 { + for _, dep := range deps { + require.NotZero(t, flags&dep, "%s has flag %d but not dependant flag %d", name, flag, dep) + } + } + } + for flag, excls := range flagExclusions { + if flags&flag != 0 { + for _, excl := range excls { + require.Zero(t, flags&excl, "%s flag %d cannot be combined with flag %d", name, flag, excl) + } + } + } + } +} diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 5bcf5d3cef1d..b31ac9203e6e 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -185,10 +185,10 @@ func (ba *BatchRequest) IsSingleRequest() bool { return len(ba.Requests) == 1 } -// IsSingleSkipLeaseCheckRequest returns true iff the batch contains a single -// request, and that request has the skipLeaseCheck flag set. -func (ba *BatchRequest) IsSingleSkipLeaseCheckRequest() bool { - return ba.IsSingleRequest() && ba.hasFlag(skipLeaseCheck) +// IsSingleSkipsLeaseCheckRequest returns true iff the batch contains a single +// request, and that request has the skipsLeaseCheck flag set. +func (ba *BatchRequest) IsSingleSkipsLeaseCheckRequest() bool { + return ba.IsSingleRequest() && ba.hasFlag(skipsLeaseCheck) } func (ba *BatchRequest) isSingleRequestWithMethod(m Method) bool { @@ -339,7 +339,7 @@ func (ba *BatchRequest) GetPrevLeaseForLeaseRequest() Lease { // hasFlag returns true iff one of the requests within the batch contains the // specified flag. -func (ba *BatchRequest) hasFlag(flag int) bool { +func (ba *BatchRequest) hasFlag(flag flag) bool { for _, union := range ba.Requests { if (union.GetInner().flags() & flag) != 0 { return true @@ -350,7 +350,7 @@ func (ba *BatchRequest) hasFlag(flag int) bool { // hasFlagForAll returns true iff all of the requests within the batch contains // the specified flag. -func (ba *BatchRequest) hasFlagForAll(flag int) bool { +func (ba *BatchRequest) hasFlagForAll(flag flag) bool { if len(ba.Requests) == 0 { return false } @@ -534,7 +534,7 @@ func (ba *BatchRequest) Methods() []Method { // read that acquired a latch @ ts10 can't simply be bumped to ts 20 because // there might have been overlapping writes in the 10..20 window). func (ba BatchRequest) Split(canSplitET bool) [][]RequestUnion { - compatible := func(exFlags, newFlags int) bool { + compatible := func(exFlags, newFlags flag) bool { // isAlone requests are never compatible. if (exFlags&isAlone) != 0 || (newFlags&isAlone) != 0 { return false @@ -557,7 +557,7 @@ func (ba BatchRequest) Split(canSplitET bool) [][]RequestUnion { var parts [][]RequestUnion for len(ba.Requests) > 0 { part := ba.Requests - var gFlags, hFlags = -1, -1 + var gFlags, hFlags flag = -1, -1 for i, union := range ba.Requests { args := union.GetInner() flags := args.flags() diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 233cce18b885..3af6cd4d1b13 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1732,6 +1732,8 @@ func TestLint(t *testing.T) { stream.GrepNot(`pkg/.*.go:.* func .*\.Cause is unused`), // Using deprecated WireLength call. stream.GrepNot(`pkg/rpc/stats_handler.go:.*v.WireLength is deprecated: This field is never set.*`), + // roachpb/api.go needs v1 Protobuf reflection + stream.GrepNot(`pkg/roachpb/api_test.go:.*package github.com/golang/protobuf/proto is deprecated: Use the "google.golang.org/protobuf/proto" package instead.`), // rpc/codec.go imports the same proto package that grpc-go imports (as of crdb@dd87d1145 and grpc-go@7b167fd6). stream.GrepNot(`pkg/rpc/codec.go:.*package github.com/golang/protobuf/proto is deprecated: Use the "google.golang.org/protobuf/proto" package instead.`), // goschedstats contains partial copies of go runtime structures, with From e17b557ae254f0536e5bada69e60dd3ff4ff661b Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 2 Nov 2021 19:35:31 +0000 Subject: [PATCH 3/4] roachpb: add `appliesTSCache` request flag Previously, `isIntentWrite` determined whether a request checked the timestamp cache and possibly pushed its timestamp. However, some requests may want to check the timestamp cache without writing intents, notably an MVCC-compliant `AddSSTable` request. This patch introduces a new flag `appliesTSCache`, and uses it as a condition for applying the timestamp cache to the request. Release note: None --- pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go | 7 +++--- pkg/kv/kvserver/kvserverpb/proposer_kv.proto | 7 +++--- pkg/kv/kvserver/replica_application_result.go | 4 +--- .../replica_application_state_machine.go | 6 ++--- pkg/kv/kvserver/replica_proposal.go | 2 +- pkg/kv/kvserver/replica_proposal_buf.go | 2 +- pkg/kv/kvserver/replica_tscache.go | 2 +- pkg/kv/kvserver/replica_write.go | 2 +- pkg/roachpb/api.go | 24 ++++++++++++++----- pkg/roachpb/batch.go | 7 ++++++ 10 files changed, 41 insertions(+), 22 deletions(-) diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go index 7aa9b3ff1d27..1c58660d9f3c 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go @@ -287,9 +287,10 @@ type ReplicatedEvalResult struct { IsLeaseRequest bool `protobuf:"varint,6,opt,name=is_lease_request,json=isLeaseRequest,proto3" json:"is_lease_request,omitempty"` // The timestamp at which this command is writing. Used to verify the validity // of the command against the GC threshold and to update the followers' - // clocks. If the request that produced this command is not an IntentWrite - // one, then the request's write timestamp is meaningless; for such request's, - // this field is simply a clock reading from the proposer. + // clocks. If the request that produced this command is not a write that cares + // about the timestamp cache, then the request's write timestamp is + // meaningless; for such request's, this field is simply a clock reading from + // the proposer. WriteTimestamp hlc.Timestamp `protobuf:"bytes,8,opt,name=write_timestamp,json=writeTimestamp,proto3" json:"write_timestamp"` // The stats delta corresponding to the data in this WriteBatch. On // a split, contains only the contributions to the left-hand side. diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index 66979e208f89..91c8633d7008 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -127,9 +127,10 @@ message ReplicatedEvalResult { bool is_lease_request = 6; // The timestamp at which this command is writing. Used to verify the validity // of the command against the GC threshold and to update the followers' - // clocks. If the request that produced this command is not an IntentWrite - // one, then the request's write timestamp is meaningless; for such request's, - // this field is simply a clock reading from the proposer. + // clocks. If the request that produced this command is not a write that cares + // about the timestamp cache, then the request's write timestamp is + // meaningless; for such request's, this field is simply a clock reading from + // the proposer. util.hlc.Timestamp write_timestamp = 8 [(gogoproto.nullable) = false]; // The stats delta corresponding to the data in this WriteBatch. On // a split, contains only the contributions to the left-hand side. diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 5512c3116af0..b97199232051 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -212,9 +212,7 @@ func (r *Replica) tryReproposeWithNewLeaseIndex( defer tok.DoneIfNotMoved(ctx) // NB: p.Request.Timestamp reflects the action of ba.SetActiveTimestamp. - // The IsIntentWrite condition matches the similar logic for caring - // about the closed timestamp cache in applyTimestampCache(). - if p.Request.IsIntentWrite() && p.Request.WriteTimestamp().LessEq(minTS) { + if p.Request.AppliesTimestampCache() && p.Request.WriteTimestamp().LessEq(minTS) { // The tracker wants us to forward the request timestamp, but we can't // do that without re-evaluating, so give up. The error returned here // will go to back to DistSender, so send something it can digest. diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 262041a71916..5be965e141e7 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -955,8 +955,8 @@ func (b *replicaAppBatch) Close() { var raftClosedTimestampAssertionsEnabled = envutil.EnvOrDefaultBool("COCKROACH_RAFT_CLOSEDTS_ASSERTIONS_ENABLED", true) // Assert that the current command is not writing under the closed timestamp. -// This check only applies to IntentWrite commands, since others (for example, -// EndTxn) can operate below the closed timestamp. +// This check only applies to certain write commands, mainly IsIntentWrite, +// since others (for example, EndTxn) can operate below the closed timestamp. // // Note that we check that we're we're writing under b.state.RaftClosedTimestamp // (i.e. below the timestamp closed by previous commands), not below @@ -964,7 +964,7 @@ var raftClosedTimestampAssertionsEnabled = envutil.EnvOrDefaultBool("COCKROACH_R // timestamp carried by itself; in other words cmd.raftCmd.ClosedTimestamp is a // promise about future commands, not the command carrying it. func (b *replicaAppBatch) assertNoWriteBelowClosedTimestamp(cmd *replicatedCmd) error { - if !cmd.IsLocal() || !cmd.proposal.Request.IsIntentWrite() { + if !cmd.IsLocal() || !cmd.proposal.Request.AppliesTimestampCache() { return nil } if !raftClosedTimestampAssertionsEnabled { diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index bc1ecfcce31f..866747662253 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -841,7 +841,7 @@ func (r *Replica) evaluateProposal( // Set the proposal's replicated result, which contains metadata and // side-effects that are to be replicated to all replicas. res.Replicated.IsLeaseRequest = ba.IsLeaseRequest() - if ba.IsIntentWrite() { + if ba.AppliesTimestampCache() { res.Replicated.WriteTimestamp = ba.WriteTimestamp() } else { // For misc requests, use WriteTimestamp to propagate a clock signal. This diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index e30ed7cf2b52..8b8d107826ff 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -475,7 +475,7 @@ func (b *propBuf) FlushLockedWithRaftGroup( b.p.registerProposalLocked(p) // Exit the tracker. - if !reproposal && p.Request.IsIntentWrite() { + if !reproposal && p.Request.AppliesTimestampCache() { // Sanity check that the request is tracked by the evaluation tracker at // this point. It's supposed to be tracked until the // doneIfNotMovedLocked() call below. diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 920281098309..9bd13f4f60f9 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -300,7 +300,7 @@ func (r *Replica) applyTimestampCache( for _, union := range ba.Requests { args := union.GetInner() - if roachpb.IsIntentWrite(args) { + if roachpb.AppliesTimestampCache(args) { header := args.Header() // Forward the timestamp if there's been a more recent read (by someone else). diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 14b62aba6f44..6b1953bedd4c 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -115,7 +115,7 @@ func (r *Replica) executeWriteBatch( // starts being tracked after we apply the timestamp cache. var minTS hlc.Timestamp var tok TrackedRequestToken - if ba.IsIntentWrite() { + if ba.AppliesTimestampCache() { minTS, tok = r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) } defer tok.DoneIfNotMoved(ctx) diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 712a5a8f8a0f..8a0d7f601776 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -86,6 +86,7 @@ const ( isPrefix // requests which, in a batch, must not be split from the following request isUnsplittable // range command that must not be split during sending skipsLeaseCheck // commands which skip the check that the evaluating replica has a valid lease + appliesTSCache // commands which apply the timestamp cache and closed timestamp updatesTSCache // commands which update the timestamp cache updatesTSCacheOnErr // commands which make read data available on errors needsRefresh // commands which require refreshes to avoid serializable retries @@ -97,6 +98,7 @@ var flagDependencies = map[flag][]flag{ isAdmin: {isAlone}, isLocking: {isTxn}, isIntentWrite: {isWrite, isLocking}, + appliesTSCache: {isWrite}, skipsLeaseCheck: {isAlone}, } @@ -164,6 +166,13 @@ func IsRange(args Request) bool { return (args.flags() & isRange) != 0 } +// AppliesTimestampCache returns whether the command is a write that applies the +// timestamp cache (and closed timestamp), possibly pushing its write timestamp +// into the future to avoid re-writing history. +func AppliesTimestampCache(args Request) bool { + return (args.flags() & appliesTSCache) != 0 +} + // UpdatesTimestampCache returns whether the command must update // the timestamp cache in order to set a low water mark for the // timestamp at which mutations to overlapping key(s) can write @@ -1188,7 +1197,7 @@ func (gr *GetRequest) flags() flag { } func (*PutRequest) flags() flag { - return isWrite | isTxn | isLocking | isIntentWrite | canBackpressure + return isWrite | isTxn | isLocking | isIntentWrite | appliesTSCache | canBackpressure } // ConditionalPut effectively reads without writing if it hits a @@ -1197,7 +1206,8 @@ func (*PutRequest) flags() flag { // they return an error immediately instead of continuing a serializable // transaction to be retried at end transaction. func (*ConditionalPutRequest) flags() flag { - return isRead | isWrite | isTxn | isLocking | isIntentWrite | updatesTSCache | updatesTSCacheOnErr | canBackpressure + return isRead | isWrite | isTxn | isLocking | isIntentWrite | + appliesTSCache | updatesTSCache | updatesTSCacheOnErr | canBackpressure } // InitPut, like ConditionalPut, effectively reads without writing if it hits a @@ -1206,7 +1216,8 @@ func (*ConditionalPutRequest) flags() flag { // return an error immediately instead of continuing a serializable transaction // to be retried at end transaction. func (*InitPutRequest) flags() flag { - return isRead | isWrite | isTxn | isLocking | isIntentWrite | updatesTSCache | updatesTSCacheOnErr | canBackpressure + return isRead | isWrite | isTxn | isLocking | isIntentWrite | + appliesTSCache | updatesTSCache | updatesTSCacheOnErr | canBackpressure } // Increment reads the existing value, but always leaves an intent so @@ -1215,11 +1226,11 @@ func (*InitPutRequest) flags() flag { // error immediately instead of continuing a serializable transaction // to be retried at end transaction. func (*IncrementRequest) flags() flag { - return isRead | isWrite | isTxn | isLocking | isIntentWrite | canBackpressure + return isRead | isWrite | isTxn | isLocking | isIntentWrite | appliesTSCache | canBackpressure } func (*DeleteRequest) flags() flag { - return isWrite | isTxn | isLocking | isIntentWrite | canBackpressure + return isWrite | isTxn | isLocking | isIntentWrite | appliesTSCache | canBackpressure } func (drr *DeleteRangeRequest) flags() flag { @@ -1246,7 +1257,8 @@ func (drr *DeleteRangeRequest) flags() flag { // it. Note that, even if we didn't update the ts cache, deletes of keys // that exist would not be lost (since the DeleteRange leaves intents on // those keys), but deletes of "empty space" would. - return isRead | isWrite | isTxn | isLocking | isIntentWrite | isRange | updatesTSCache | needsRefresh | canBackpressure + return isRead | isWrite | isTxn | isLocking | isIntentWrite | isRange | + appliesTSCache | updatesTSCache | needsRefresh | canBackpressure } // Note that ClearRange commands cannot be part of a transaction as diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index b31ac9203e6e..4a4a5aa8073f 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -133,6 +133,13 @@ func (ba *BatchRequest) IsLeaseRequest() bool { return ok } +// AppliesTimestampCache returns whether the command is a write that applies the +// timestamp cache (and closed timestamp), possibly pushing its write timestamp +// into the future to avoid re-writing history. +func (ba *BatchRequest) AppliesTimestampCache() bool { + return ba.hasFlag(appliesTSCache) +} + // IsAdmin returns true iff the BatchRequest contains an admin request. func (ba *BatchRequest) IsAdmin() bool { return ba.hasFlag(isAdmin) From 54f574651bbf904057f826ac11521967991ae901 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 28 Oct 2021 08:23:59 +0000 Subject: [PATCH 4/4] kvserver: add MVCC-compliant `AddSSTable` variant `AddSSTable` does not comply with MVCC, the timestamp cache, nor the closed timestamp, since the SST MVCC timestamps are written exactly as given and thus can rewrite history. This patch adds three new parameters that can make `AddSSTable` fully MVCC-compliant, with a corresponding `MVCCAddSSTable` version gate: * `WriteAtRequestTimestamp`: rewrites the MVCC timestamps to the request timestamp, complying with the timestamp cache and closed timestamp. * `DisallowConflicts`: checks for any conflicting keys and intents at or above the SST's MVCC timestamps, complying with MVCC. * `DisallowShadowingBelow`: implies `DisallowConflicts`, and also errors if shadowing visible keys (but not tombstones). Unlike the existing `DisallowShadowing`, this allows shadowing existing keys above the given timestamp if the new key has the same value as the existing one, and also allows idempotent writes at or above the given timestamp. The existing `DisallowShadowing` parameter implies `DisallowConflicts`, and also errors on any existing visible keys below the SST key's timestamp (but not tombstones). It no longer allows replacing a tombstone with a value at the exact same timestamp. Additionally, even blind `AddSSTable` requests that do not check for conflicts now take out lock spans and scan for existing intents, returning a `WriteIntentError` to resolve them. This should be cheap in the common case, since the caller is expected to ensure there are no concurrent writes over the span, and so there should be no or few intents. The `WriteAtRequestTimestamp` SST rewrite implementation here is correct but slow. Optimizations will be explored later. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/importccl/bench_test.go | 6 +- pkg/clusterversion/cockroach_versions.go | 7 + pkg/clusterversion/key_string.go | 5 +- pkg/kv/batch.go | 14 +- pkg/kv/bulk/sst_batcher.go | 7 +- pkg/kv/bulk/sst_batcher_test.go | 3 + pkg/kv/db.go | 9 +- pkg/kv/kvserver/batcheval/BUILD.bazel | 1 - pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 200 +- .../batcheval/cmd_add_sstable_test.go | 2084 +++++++++-------- pkg/kv/kvserver/spanset/batch.go | 7 - pkg/roachpb/api.go | 8 +- pkg/roachpb/api.pb.go | 1440 +++++++----- pkg/roachpb/api.proto | 108 +- pkg/roachpb/api_test.go | 1 + pkg/storage/BUILD.bazel | 3 +- pkg/storage/engine.go | 8 - pkg/storage/engine_test.go | 78 - pkg/storage/error.go | 21 - pkg/storage/intent_interleaving_iter.go | 6 - pkg/storage/mvcc.go | 142 -- pkg/storage/pebble_iterator.go | 8 - pkg/storage/sst.go | 258 ++ pkg/storage/sst_iterator.go | 4 +- pkg/storage/sst_test.go | 171 ++ pkg/testutils/trace.go | 22 - 28 files changed, 2618 insertions(+), 2007 deletions(-) delete mode 100644 pkg/storage/error.go create mode 100644 pkg/storage/sst.go create mode 100644 pkg/storage/sst_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 312193ed32d4..7cecacee34d4 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -167,4 +167,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-14 set the active cluster version in the format '.' +version version 21.2-16 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 437560b1f477..daaec529e04e 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -172,6 +172,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-14set the active cluster version in the format '.' +versionversion21.2-16set the active cluster version in the format '.' diff --git a/pkg/ccl/importccl/bench_test.go b/pkg/ccl/importccl/bench_test.go index 4e241908afde..2a83682bb6ca 100644 --- a/pkg/ccl/importccl/bench_test.go +++ b/pkg/ccl/importccl/bench_test.go @@ -135,8 +135,10 @@ func benchmarkAddSSTable(b *testing.B, dir string, tables []tableSSTable) { b.StartTimer() for _, t := range tables { totalBytes += int64(len(t.sstData)) - require.NoError(b, kvDB.AddSSTable( - ctx, t.span.Key, t.span.EndKey, t.sstData, true /* disallowShadowing */, nil /* stats */, false /*ingestAsWrites */, hlc.Timestamp{}, + require.NoError(b, kvDB.AddSSTable(ctx, t.span.Key, t.span.EndKey, t.sstData, + false /* disallowConflicts */, true, /* disallowShadowing */ + hlc.Timestamp{} /* disallowShadowingBelow */, nil, /* stats */ + false /*ingestAsWrites */, hlc.Timestamp{}, false, /* writeAtBatchTS */ )) } b.StopTimer() diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index f9895d9d0e9d..46b005f45084 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -288,6 +288,9 @@ const ( // system.statement_diagnostics_requests table to support collecting stmt // bundles when the query latency exceeds the user provided threshold. AlterSystemStmtDiagReqs + // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new + // WriteAtRequestTimestamp and DisallowConflicts parameters. + MVCCAddSSTable // ************************************************* // Step (1): Add new versions here. @@ -497,6 +500,10 @@ var versionsSingleton = keyedVersions{ Key: AlterSystemStmtDiagReqs, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 14}, }, + { + Key: MVCCAddSSTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 16}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index ff62e7caae06..cd98d63da603 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -50,11 +50,12 @@ func _() { _ = x[TraceIDDoesntImplyStructuredRecording-39] _ = x[AlterSystemTableStatisticsAddAvgSizeCol-40] _ = x[AlterSystemStmtDiagReqs-41] + _ = x[MVCCAddSSTable-42] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqs" +const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTable" -var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 436, 465, 498, 521, 540, 559, 582, 598, 618, 639, 661, 680, 714, 728, 758, 763, 772, 794, 812, 834, 871, 910, 933} +var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 436, 465, 498, 521, 540, 559, 582, 598, 618, 639, 661, 680, 714, 728, 758, 763, 772, 794, 812, 834, 871, 910, 933, 947} 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 7cd5d17c48ce..06ac082da0df 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -764,9 +764,12 @@ func (b *Batch) adminRelocateRange( func (b *Batch) addSSTable( s, e interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, + disallowShadowingBelow hlc.Timestamp, stats *enginepb.MVCCStats, ingestAsWrites bool, + writeAtRequestTimestamp bool, ) { begin, err := marshalKey(s) if err != nil { @@ -783,10 +786,13 @@ func (b *Batch) addSSTable( Key: begin, EndKey: end, }, - Data: data, - DisallowShadowing: disallowShadowing, - MVCCStats: stats, - IngestAsWrites: ingestAsWrites, + Data: data, + DisallowConflicts: disallowConflicts, + DisallowShadowing: disallowShadowing, + DisallowShadowingBelow: disallowShadowingBelow, + MVCCStats: stats, + IngestAsWrites: ingestAsWrites, + WriteAtRequestTimestamp: writeAtRequestTimestamp, } b.appendReqs(req) b.initResult(1, 0, notRaw, nil) diff --git a/pkg/kv/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index 4a4ec0039df9..99e934a7563d 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -385,10 +385,13 @@ type SSTSender interface { ctx context.Context, begin, end interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, + disallowShadowingBelow hlc.Timestamp, stats *enginepb.MVCCStats, ingestAsWrites bool, batchTs hlc.Timestamp, + writeAtBatchTs bool, ) error SplitAndScatter(ctx context.Context, key roachpb.Key, expirationTime hlc.Timestamp) error } @@ -457,7 +460,9 @@ func AddSSTable( ingestAsWriteBatch = true } // This will fail if the range has split but we'll check for that below. - err = db.AddSSTable(ctx, item.start, item.end, item.sstBytes, item.disallowShadowing, &item.stats, ingestAsWriteBatch, batchTs) + err = db.AddSSTable(ctx, item.start, item.end, item.sstBytes, false, /* disallowConflicts */ + item.disallowShadowing, hlc.Timestamp{} /* disallowShadowingBelow */, &item.stats, + ingestAsWriteBatch, batchTs, false /* writeAtBatchTs */) if err == nil { log.VEventf(ctx, 3, "adding %s AddSSTable [%s,%s) took %v", sz(len(item.sstBytes)), item.start, item.end, timeutil.Since(before)) return nil diff --git a/pkg/kv/bulk/sst_batcher_test.go b/pkg/kv/bulk/sst_batcher_test.go index 7cd278733828..0d665e3ad594 100644 --- a/pkg/kv/bulk/sst_batcher_test.go +++ b/pkg/kv/bulk/sst_batcher_test.go @@ -260,10 +260,13 @@ func (m mockSender) AddSSTable( ctx context.Context, begin, end interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, + disallowShadowingBelow hlc.Timestamp, _ *enginepb.MVCCStats, ingestAsWrites bool, batchTS hlc.Timestamp, + writeAtBatchTS bool, ) error { return m(roachpb.Span{Key: begin.(roachpb.Key), EndKey: end.(roachpb.Key)}) } diff --git a/pkg/kv/db.go b/pkg/kv/db.go index c5f5bf148e4a..a0b1c9376a76 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -664,17 +664,24 @@ func (db *DB) AdminRelocateRange( // AddSSTable links a file into the RocksDB log-structured merge-tree. Existing // data in the range is cleared. +// +// The disallowConflicts, disallowShadowingBelow, and writeAtBatchTs parameters +// require the MVCCAddSSTable version gate, as they are new in 22.1. func (db *DB) AddSSTable( ctx context.Context, begin, end interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, + disallowShadowingBelow hlc.Timestamp, stats *enginepb.MVCCStats, ingestAsWrites bool, batchTs hlc.Timestamp, + writeAtBatchTs bool, ) error { b := &Batch{Header: roachpb.Header{Timestamp: batchTs}} - b.addSSTable(begin, end, data, disallowShadowing, stats, ingestAsWrites) + b.addSSTable(begin, end, data, disallowConflicts, disallowShadowing, disallowShadowingBelow, + stats, ingestAsWrites, writeAtBatchTs) return getOneErr(db.Run(ctx, b), b) } diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 6d53b78ca46b..5cc95e355464 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -146,7 +146,6 @@ go_test( "//pkg/util/uint128", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", - "@com_github_kr_pretty//:pretty", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", ], diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 78f7c8ff6edd..0b8875a6d21d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "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" @@ -30,123 +29,90 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.AddSSTable, declareKeysAddSSTable, EvalAddSSTable) -} - -func declareKeysAddSSTable( - rs ImmutableRangeState, - header roachpb.Header, - req roachpb.Request, - latchSpans, lockSpans *spanset.SpanSet, -) { - // AddSSTable violates MVCC and closed timestamp invariants, so the - // concurrency semantics deserve special attention. - // - // AddSSTable cannot be in a transaction, cannot write intents or tombstones, - // cannot be split across ranges, and is always alone in a batch. - // - // The KV pairs in the SST already have fixed MVCC timestamps, independent of - // the batch timestamp. Pushes by other txns or the closed timestamp do not - // affect the MVCC timestamps. They can be at any time (past or future), even - // below the closed timestamp, and by default they can replace existing - // versions or write below existing versions and intents. This violates MVCC, - // because history must be immutable, and the closed timestamp, because writes - // should never happen below it. - // - // DisallowShadowing=true will prevent writing to keys that already exist - // (with any timestamp), returning an error -- except if the last version is a - // tombstone with a timestamp below the written key or if the timestamp and - // value exactly match the incoming write (for idempotency). If an intent is - // found, WriteIntentError will be returned in order to resolve it and retry: - // if the intent was aborted or a tombstone the request may succeed, but if it - // was a committed value the request will fail. This still violates MVCC (it - // may write a key in the past whose absence has already been observed by a - // reader) and the closed timestamp (it may write a key below it). - // - // The request header's Key and EndKey are set to cover the first and last key - // in the SST. Below, we always declare write latches across this span for - // isolation from concurrent requests. If DisallowShadowing=true, we must also - // declare lock spans over this span for isolation from concurrent - // transactions, and return WriteIntentError for any encountered intents to - // resolve them. This is particularly relevant for IMPORT INTO, which imports - // into an offline table that may contain unresolved intents from previous - // transactions. - // // Taking out latches/locks across the entire SST span is very coarse, and we // could instead iterate over the SST and take out point latches/locks, but // the cost is likely not worth it since AddSSTable is often used with // unpopulated spans. - // - // AddSSTable callers must take extreme care to only write into key/time spans - // that have never been accessed by a past transaction, and will not be - // accessed by a concurrent transaction, or to make sure these accesses are - // safe. Below is a list of current operations that use AddSSTable and their - // characteristics: - // - // | Operation | DisallowShadowing | Timestamp | Isolation via | - // |------------------------|-------------------|--------------|-------------------| - // | Import | true | Now | Offline table | - // | CREATE TABLE AS SELECT | true | Read TS | Table descriptor | - // | Materialized views | true | Read TS | Table descriptor | - // | Index backfills | false | Now | Index descriptor | - // | Restore (backup) | true | Key TS | Table descriptor | - // | Streaming replication | false | Key TS | Offline tenant | - // - args := req.(*roachpb.AddSSTableRequest) - if args.DisallowShadowing { - DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans) - } else { - DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans) - } + RegisterReadWriteCommand(roachpb.AddSSTable, DefaultDeclareIsolatedKeys, EvalAddSSTable) } -// EvalAddSSTable evaluates an AddSSTable command. -// NB: These sstables do not contain intents/locks, so the code below only -// needs to deal with MVCCKeys. +// EvalAddSSTable evaluates an AddSSTable command. For details, see doc comment +// on AddSSTableRequest. func EvalAddSSTable( ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response, ) (result.Result, error) { args := cArgs.Args.(*roachpb.AddSSTableRequest) h := cArgs.Header ms := cArgs.Stats - mvccStartKey, mvccEndKey := storage.MVCCKey{Key: args.Key}, storage.MVCCKey{Key: args.EndKey} + start, end := storage.MVCCKey{Key: args.Key}, storage.MVCCKey{Key: args.EndKey} + sst := args.Data var span *tracing.Span - ctx, span = tracing.ChildSpan(ctx, fmt.Sprintf("AddSSTable [%s,%s)", args.Key, args.EndKey)) + var err error + ctx, span = tracing.ChildSpan(ctx, fmt.Sprintf("AddSSTable [%s,%s)", start.Key, end.Key)) defer span.Finish() - log.Eventf(ctx, "evaluating AddSSTable [%s,%s)", mvccStartKey.Key, mvccEndKey.Key) + log.Eventf(ctx, "evaluating AddSSTable [%s,%s)", start.Key, end.Key) - // IMPORT INTO should not proceed if any KVs from the SST shadow existing data - // entries - #38044. - var skippedKVStats enginepb.MVCCStats - var err error - if args.DisallowShadowing { - maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) - skippedKVStats, err = checkForKeyCollisions( - ctx, readWriter, mvccStartKey, mvccEndKey, args.Data, maxIntents) + // If requested, rewrite the SST's MVCC timestamps to the request timestamp. + // This ensures the writes comply with the timestamp cache and closed + // timestamp, i.e. by not writing to timestamps that have already been + // observed or closed. + if args.WriteAtRequestTimestamp { + sst, err = storage.UpdateSSTTimestamps(sst, h.Timestamp) + if err != nil { + return result.Result{}, errors.Wrap(err, "updating SST timestamps") + } + } + + var statsDelta enginepb.MVCCStats + maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) + checkConflicts := args.DisallowConflicts || args.DisallowShadowing || + !args.DisallowShadowingBelow.IsEmpty() + if checkConflicts { + // If requested, check for MVCC conflicts with existing keys. This enforces + // all MVCC invariants by returning WriteTooOldError for any existing + // values at or above the SST timestamp, returning WriteIntentError to + // resolve any encountered intents, and accurately updating MVCC stats. + // + // Additionally, if DisallowShadowing or DisallowShadowingBelow is set, it + // will not write above existing/visible values (but will write above + // tombstones). + statsDelta, err = storage.CheckSSTConflicts(ctx, sst, readWriter, start, end, + args.DisallowShadowing, args.DisallowShadowingBelow, maxIntents) if err != nil { return result.Result{}, errors.Wrap(err, "checking for key collisions") } + + } else { + // If not checking for MVCC conflicts, at least check for separated intents. + // The caller is expected to make sure there are no writers across the span, + // and thus no or few intents, so this is cheap in the common case. + intents, err := storage.ScanIntents(ctx, readWriter, start.Key, end.Key, maxIntents, 0) + if err != nil { + return result.Result{}, errors.Wrap(err, "scanning intents") + } else if len(intents) > 0 { + return result.Result{}, &roachpb.WriteIntentError{Intents: intents} + } } // Verify that the keys in the sstable are within the range specified by the // request header, and if the request did not include pre-computed stats, // compute the expected MVCC stats delta of ingesting the SST. - dataIter, err := storage.NewMemSSTIterator(args.Data, true) + sstIter, err := storage.NewMemSSTIterator(sst, true) if err != nil { return result.Result{}, err } - defer dataIter.Close() + defer sstIter.Close() // Check that the first key is in the expected range. - dataIter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) - ok, err := dataIter.Valid() + sstIter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) + ok, err := sstIter.Valid() if err != nil { return result.Result{}, err } else if ok { - if unsafeKey := dataIter.UnsafeKey(); unsafeKey.Less(mvccStartKey) { + if unsafeKey := sstIter.UnsafeKey(); unsafeKey.Less(start) { return result.Result{}, errors.Errorf("first key %s not in request range [%s,%s)", - unsafeKey.Key, mvccStartKey.Key, mvccEndKey.Key) + unsafeKey.Key, start.Key, end.Key) } } @@ -156,15 +122,13 @@ func EvalAddSSTable( stats = *args.MVCCStats } - // Stats are computed on-the-fly when shadowing of keys is disallowed. If we - // took the fast path and race is enabled, assert the stats were correctly - // computed. - verifyFastPath := args.DisallowShadowing && util.RaceEnabled + // Stats are computed on-the-fly when checking for conflicts. If we took the + // fast path and race is enabled, assert the stats were correctly computed. + verifyFastPath := checkConflicts && util.RaceEnabled if args.MVCCStats == nil || verifyFastPath { - log.VEventf(ctx, 2, "computing MVCCStats for SSTable [%s,%s)", mvccStartKey.Key, mvccEndKey.Key) + log.VEventf(ctx, 2, "computing MVCCStats for SSTable [%s,%s)", start.Key, end.Key) - computed, err := storage.ComputeStatsForRange( - dataIter, mvccStartKey.Key, mvccEndKey.Key, h.Timestamp.WallTime) + computed, err := storage.ComputeStatsForRange(sstIter, start.Key, end.Key, h.Timestamp.WallTime) if err != nil { return result.Result{}, errors.Wrap(err, "computing SSTable MVCC stats") } @@ -181,13 +145,13 @@ func EvalAddSSTable( stats = computed } - dataIter.SeekGE(mvccEndKey) - ok, err = dataIter.Valid() + sstIter.SeekGE(end) + ok, err = sstIter.Valid() if err != nil { return result.Result{}, err } else if ok { return result.Result{}, errors.Errorf("last key %s not in request range [%s,%s)", - dataIter.UnsafeKey(), mvccStartKey.Key, mvccEndKey.Key) + sstIter.UnsafeKey(), start.Key, end.Key) } // The above MVCCStats represents what is in this new SST. @@ -243,8 +207,8 @@ func EvalAddSSTable( // checking for the collision condition in C++ and subtract them from the // stats of the SST being ingested before adding them to the running // cumulative for this command. These stats can then be marked as accurate. - if args.DisallowShadowing { - stats.Subtract(skippedKVStats) + if checkConflicts { + stats.Add(statsDelta) stats.ContainsEstimates = 0 } else { stats.ContainsEstimates++ @@ -253,11 +217,11 @@ func EvalAddSSTable( ms.Add(stats) if args.IngestAsWrites { - span.RecordStructured(&types.StringValue{Value: fmt.Sprintf("ingesting SST (%d keys/%d bytes) via regular write batch", stats.KeyCount, len(args.Data))}) - log.VEventf(ctx, 2, "ingesting SST (%d keys/%d bytes) via regular write batch", stats.KeyCount, len(args.Data)) - dataIter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) + span.RecordStructured(&types.StringValue{Value: fmt.Sprintf("ingesting SST (%d keys/%d bytes) via regular write batch", stats.KeyCount, len(sst))}) + log.VEventf(ctx, 2, "ingesting SST (%d keys/%d bytes) via regular write batch", stats.KeyCount, len(sst)) + sstIter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) for { - ok, err := dataIter.Valid() + ok, err := sstIter.Valid() if err != nil { return result.Result{}, err } else if !ok { @@ -266,17 +230,17 @@ func EvalAddSSTable( // NB: This is *not* a general transformation of any arbitrary SST to a // WriteBatch: it assumes every key in the SST is a simple Set. This is // already assumed elsewhere in this RPC though, so that's OK here. - k := dataIter.UnsafeKey() + k := sstIter.UnsafeKey() if k.Timestamp.IsEmpty() { - if err := readWriter.PutUnversioned(k.Key, dataIter.UnsafeValue()); err != nil { + if err := readWriter.PutUnversioned(k.Key, sstIter.UnsafeValue()); err != nil { return result.Result{}, err } } else { - if err := readWriter.PutMVCC(dataIter.UnsafeKey(), dataIter.UnsafeValue()); err != nil { + if err := readWriter.PutMVCC(k, sstIter.UnsafeValue()); err != nil { return result.Result{}, err } } - dataIter.Next() + sstIter.Next() } return result.Result{}, nil } @@ -284,31 +248,9 @@ func EvalAddSSTable( return result.Result{ Replicated: kvserverpb.ReplicatedEvalResult{ AddSSTable: &kvserverpb.ReplicatedEvalResult_AddSSTable{ - Data: args.Data, - CRC32: util.CRC32(args.Data), + Data: sst, + CRC32: util.CRC32(sst), }, }, }, nil } - -func checkForKeyCollisions( - _ context.Context, - reader storage.Reader, - mvccStartKey storage.MVCCKey, - mvccEndKey storage.MVCCKey, - data []byte, - maxIntents int64, -) (enginepb.MVCCStats, error) { - // Create iterator over the existing data. - existingDataIter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: mvccEndKey.Key}) - defer existingDataIter.Close() - existingDataIter.SeekGE(mvccStartKey) - if ok, err := existingDataIter.Valid(); err != nil { - return enginepb.MVCCStats{}, errors.Wrap(err, "checking for key collisions") - } else if !ok { - // Target key range is empty, so it is safe to ingest. - return enginepb.MVCCStats{}, nil - } - - return existingDataIter.CheckForKeyCollisions(data, mvccStartKey.Key, mvccEndKey.Key, maxIntents) -} diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index d254f0186e01..45c3b6090205 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -11,11 +11,10 @@ package batcheval_test import ( - "bytes" "context" "os" + "path/filepath" "regexp" - "sort" "strings" "testing" @@ -34,262 +33,874 @@ import ( "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/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" - "github.com/kr/pretty" "github.com/stretchr/testify/require" ) -var engineImpls = []struct { - name string - create func(...storage.ConfigOption) storage.Engine -}{ - {"pebble", storage.NewDefaultInMemForTesting}, +type mvccKV struct { + key string + ts int64 // 0 for inline + value string // "" for nil (tombstone) } -func singleKVSSTable(key storage.MVCCKey, value []byte) ([]byte, error) { - sstFile := &storage.MemFile{} - sst := storage.MakeBackupSSTWriter(sstFile) - defer sst.Close() - if err := sst.Put(key, value); err != nil { - return nil, err +func (kv mvccKV) Key() roachpb.Key { return roachpb.Key(kv.key) } +func (kv mvccKV) TS() hlc.Timestamp { return hlc.Timestamp{WallTime: kv.ts} } +func (kv mvccKV) MVCCKey() storage.MVCCKey { return storage.MVCCKey{Key: kv.Key(), Timestamp: kv.TS()} } +func (kv mvccKV) ValueBytes() []byte { return kv.Value().RawBytes } + +func (kv mvccKV) Value() roachpb.Value { + value := roachpb.MakeValueFromString(kv.value) + if kv.value == "" { + value = roachpb.Value{} } - if err := sst.Finish(); err != nil { - return nil, err + value.InitChecksum(kv.Key()) + return value +} + +// TestEvalAddSSTable tests EvalAddSSTable directly, using only an on-disk +// Pebble engine. This allows precise manipulation of timestamps. +func TestEvalAddSSTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + const defaultReqTS = 10 // request sent with this timestamp by default + const intentTS = 100 // values with this timestamp are written as intents + + // These are run with IngestAsWrites both disabled and enabled. + testcases := map[string]struct { + data []mvccKV + sst []mvccKV + atReqTS int64 // WriteAtRequestTimestamp with given timestamp + noConflict bool // DisallowConflicts + noShadow bool // DisallowShadowing + noShadowBelow int64 // DisallowShadowingBelow + expect []mvccKV + expectErr interface{} // error type, substring, or true (any error) + expectStatsEst bool // expect MVCCStats.ContainsEstimates, don't check stats + }{ + // Blind writes. + "blind writes below existing": { + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, ""}}, + sst: []mvccKV{{"a", 3, "sst"}, {"b", 2, "sst"}}, + expect: []mvccKV{{"a", 5, "a5"}, {"a", 3, "sst"}, {"b", 7, ""}, {"b", 2, "sst"}}, + expectStatsEst: true, + }, + "blind replaces existing": { + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expect: []mvccKV{{"a", 2, "sst"}}, + expectStatsEst: true, + }, + "blind returns WriteIntentError on conflict": { + data: []mvccKV{{"b", intentTS, "b0"}}, + sst: []mvccKV{{"b", 1, "sst"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "blind returns WriteIntentError in span": { + data: []mvccKV{{"b", intentTS, "b0"}}, + sst: []mvccKV{{"a", 1, "sst"}, {"c", 1, "sst"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "blind writes tombstones": { // unfortunately, for performance + sst: []mvccKV{{"a", 1, ""}}, + expect: []mvccKV{{"a", 1, ""}}, + expectStatsEst: true, + }, + "blind writes SST inline values": { // unfortunately, for performance + sst: []mvccKV{{"a", 0, "inline"}}, + expect: []mvccKV{{"a", 0, "inline"}}, + expectStatsEst: true, + }, + "blind writes above existing inline values": { // unfortunately, for performance + data: []mvccKV{{"a", 0, "inline"}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expect: []mvccKV{{"a", 0, "inline"}, {"a", 2, "sst"}}, + expectStatsEst: true, + }, + + // WriteAtRequestTimestamp + "WriteAtRequestTimestamp sets timestamp": { + atReqTS: 10, + sst: []mvccKV{{"a", 1, "a1"}, {"b", 3, "b3"}}, + expect: []mvccKV{{"a", 10, "a1"}, {"b", 10, "b3"}}, + expectStatsEst: true, + }, + "WriteAtRequestTimestamp rejects tombstones": { + atReqTS: 10, + sst: []mvccKV{{"a", 1, ""}}, + expectErr: "SST values cannot be tombstones", + }, + "WriteAtRequestTimestamp rejects inline values": { + atReqTS: 10, + sst: []mvccKV{{"a", 0, "inline"}}, + expectErr: "inline values or intents are not supported", + }, + "WriteAtRequestTimestamp writes below and replaces": { + atReqTS: 5, + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, "b7"}}, + sst: []mvccKV{{"a", 1, "sst"}, {"b", 1, "sst"}}, + expect: []mvccKV{{"a", 5, "sst"}, {"b", 7, "b7"}, {"b", 5, "sst"}}, + expectStatsEst: true, + }, + "WriteAtRequestTimestamp returns WriteIntentError for intents": { + atReqTS: 10, + data: []mvccKV{{"a", intentTS, "intent"}}, + sst: []mvccKV{{"a", 1, "a@1"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "WriteAtRequestTimestamp errors with DisallowConflicts below existing": { + atReqTS: 5, + noConflict: true, + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, "b7"}}, + sst: []mvccKV{{"a", 10, "sst"}, {"b", 10, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "WriteAtRequestTimestamp succeeds with DisallowConflicts above existing": { + atReqTS: 8, + noConflict: true, + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, "b7"}}, + sst: []mvccKV{{"a", 1, "sst"}, {"b", 1, "sst"}}, + expect: []mvccKV{{"a", 8, "sst"}, {"a", 5, "a5"}, {"b", 8, "sst"}, {"b", 7, "b7"}}, + }, + "WriteAtRequestTimestamp errors with DisallowShadowing below existing": { + atReqTS: 5, + noShadow: true, + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, "b7"}}, + sst: []mvccKV{{"a", 10, "sst"}, {"b", 10, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "WriteAtRequestTimestamp errors with DisallowShadowing above existing": { + atReqTS: 8, + noShadow: true, + data: []mvccKV{{"a", 5, "a5"}, {"b", 7, "b7"}}, + sst: []mvccKV{{"a", 1, "sst"}, {"b", 1, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "WriteAtRequestTimestamp succeeds with DisallowShadowing above tombstones": { + atReqTS: 8, + noShadow: true, + data: []mvccKV{{"a", 5, ""}, {"b", 7, ""}}, + sst: []mvccKV{{"a", 1, "sst"}, {"b", 1, "sst"}}, + expect: []mvccKV{{"a", 8, "sst"}, {"a", 5, ""}, {"b", 8, "sst"}, {"b", 7, ""}}, + }, + "WriteAtRequestTimestamp succeeds with DisallowShadowing and idempotent writes": { + atReqTS: 5, + noShadow: true, + data: []mvccKV{{"a", 5, "a5"}, {"b", 5, "b5"}}, + sst: []mvccKV{{"a", 1, "a5"}, {"b", 1, "b5"}}, + expect: []mvccKV{{"a", 5, "a5"}, {"b", 5, "b5"}}, + }, + "WriteAtRequestTimestamp errors with DisallowShadowingBelow equal value above existing below limit": { + atReqTS: 7, + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 10, "a3"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "WriteAtRequestTimestamp errors with DisallowShadowingBelow errors above existing above limit": { + atReqTS: 7, + noShadowBelow: 5, + data: []mvccKV{{"a", 6, "a6"}}, + sst: []mvccKV{{"a", 10, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "WriteAtRequestTimestamp allows DisallowShadowingBelow equal value above existing above limit": { + atReqTS: 7, + noShadowBelow: 5, + data: []mvccKV{{"a", 6, "a6"}}, + sst: []mvccKV{{"a", 10, "a6"}}, + expect: []mvccKV{{"a", 7, "a6"}, {"a", 6, "a6"}}, + }, + + // DisallowConflicts + "DisallowConflicts allows above and beside": { + noConflict: true, + data: []mvccKV{{"a", 3, "a3"}, {"b", 1, ""}}, + sst: []mvccKV{{"a", 4, "sst"}, {"b", 3, "sst"}, {"c", 1, "sst"}}, + expect: []mvccKV{ + {"a", 4, "sst"}, {"a", 3, "a3"}, {"b", 3, "sst"}, {"b", 1, ""}, {"c", 1, "sst"}, + }, + }, + "DisallowConflicts returns WriteTooOldError below existing": { + noConflict: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowConflicts returns WriteTooOldError at existing": { + noConflict: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowConflicts returns WriteTooOldError at existing tombstone": { + noConflict: true, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowConflicts returns WriteIntentError below intent": { + noConflict: true, + data: []mvccKV{{"a", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "DisallowConflicts ignores intents in span": { // inconsistent with blind writes + noConflict: true, + data: []mvccKV{{"b", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}, {"c", 3, "sst"}}, + expect: []mvccKV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + }, + "DisallowConflicts is not idempotent": { + noConflict: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "a3"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowConflicts allows new SST tombstones": { // unfortunately, for performance + noConflict: true, + sst: []mvccKV{{"a", 3, ""}}, + expect: []mvccKV{{"a", 3, ""}}, + }, + "DisallowConflicts rejects SST tombstones when shadowing": { + noConflict: true, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 3, ""}}, + expectErr: "SST values cannot be tombstones", + }, + "DisallowConflicts allows new SST inline values": { // unfortunately, for performance + noConflict: true, + sst: []mvccKV{{"a", 0, "inline"}}, + expect: []mvccKV{{"a", 0, "inline"}}, + }, + "DisallowConflicts rejects SST inline values when shadowing": { + noConflict: true, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 0, ""}}, + expectErr: "SST keys must have timestamps", + }, + "DisallowConflicts rejects existing inline values when shadowing": { + noConflict: true, + data: []mvccKV{{"a", 0, "a0"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: "inline values are unsupported", + }, + + // DisallowShadowing + "DisallowShadowing errors above existing": { + noShadow: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 4, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowing errors below existing": { + noShadow: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowing errors at existing": { + noShadow: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowing returns WriteTooOldError at existing tombstone": { + noShadow: true, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowShadowing returns WriteTooOldError below existing tombstone": { + noShadow: true, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowShadowing allows above existing tombstone": { + noShadow: true, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 4, "sst"}}, + expect: []mvccKV{{"a", 4, "sst"}, {"a", 3, ""}}, + }, + "DisallowShadowing returns WriteIntentError below intent": { + noShadow: true, + data: []mvccKV{{"a", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "DisallowShadowing ignores intents in span": { // inconsistent with blind writes + noShadow: true, + data: []mvccKV{{"b", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}, {"c", 3, "sst"}}, + expect: []mvccKV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + }, + "DisallowShadowing is idempotent": { + noShadow: true, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "a3"}}, + expect: []mvccKV{{"a", 3, "a3"}}, + }, + "DisallowShadowing allows new SST tombstones": { // unfortunately, for performance + noShadow: true, + sst: []mvccKV{{"a", 3, ""}}, + expect: []mvccKV{{"a", 3, ""}}, + }, + "DisallowShadowing rejects SST tombstones when shadowing": { + noShadow: true, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 3, ""}}, + expectErr: "SST values cannot be tombstones", + }, + "DisallowShadowing allows new SST inline values": { // unfortunately, for performance + noShadow: true, + sst: []mvccKV{{"a", 0, "inline"}}, + expect: []mvccKV{{"a", 0, "inline"}}, + }, + "DisallowShadowing rejects SST inline values when shadowing": { + noShadow: true, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 0, "inline"}}, + expectErr: "SST keys must have timestamps", + }, + "DisallowShadowing rejects existing inline values when shadowing": { + noShadow: true, + data: []mvccKV{{"a", 0, "a0"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: "inline values are unsupported", + }, + "DisallowShadowing collision SST start, existing start, above": { + noShadow: true, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowing collision SST start, existing middle, below": { + noShadow: true, + data: []mvccKV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"c", 3, "c3"}}, + sst: []mvccKV{{"b", 1, "sst"}}, + expectErr: `ingested key collides with an existing one: "b"`, + }, + "DisallowShadowing collision SST end, existing end, above": { + noShadow: true, + data: []mvccKV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"d", 3, "d3"}}, + sst: []mvccKV{{"c", 3, "sst"}, {"d", 4, "sst"}}, + expectErr: `ingested key collides with an existing one: "d"`, + }, + "DisallowShadowing collision after write above tombstone": { + noShadow: true, + data: []mvccKV{{"a", 2, ""}, {"a", 1, "a1"}, {"b", 2, "b2"}}, + sst: []mvccKV{{"a", 3, "sst"}, {"b", 1, "sst"}}, + expectErr: `ingested key collides with an existing one: "b"`, + }, + + // DisallowShadowingBelow + "DisallowShadowingBelow cannot be used with DisallowShadowing": { + noShadow: true, + noShadowBelow: 5, + sst: []mvccKV{{"a", 1, "sst"}}, + expectErr: `cannot set both DisallowShadowing and DisallowShadowingBelow`, + }, + "DisallowShadowingBelow errors above existing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 4, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow errors below existing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow errors at existing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow returns WriteTooOldError at existing tombstone": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowShadowingBelow returns WriteTooOldError below existing tombstone": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 2, "sst"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowShadowingBelow allows above existing tombstone": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, ""}}, + sst: []mvccKV{{"a", 4, "sst"}}, + expect: []mvccKV{{"a", 4, "sst"}, {"a", 3, ""}}, + }, + "DisallowShadowingBelow returns WriteIntentError below intent": { + noShadowBelow: 5, + data: []mvccKV{{"a", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: &roachpb.WriteIntentError{}, + }, + "DisallowShadowingBelow ignores intents in span": { // inconsistent with blind writes + noShadowBelow: 5, + data: []mvccKV{{"b", intentTS, "intent"}}, + sst: []mvccKV{{"a", 3, "sst"}, {"c", 3, "sst"}}, + expect: []mvccKV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + }, + "DisallowShadowingBelow is not generally idempotent": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 3, "a3"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow allows new SST tombstones": { // unfortunately, for performance + noShadowBelow: 5, + sst: []mvccKV{{"a", 3, ""}}, + expect: []mvccKV{{"a", 3, ""}}, + }, + "DisallowShadowingBelow rejects SST tombstones when shadowing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 3, ""}}, + expectErr: "SST values cannot be tombstones", + }, + "DisallowShadowingBelow allows new SST inline values": { // unfortunately, for performance + noShadowBelow: 5, + sst: []mvccKV{{"a", 0, "inline"}}, + expect: []mvccKV{{"a", 0, "inline"}}, + }, + "DisallowShadowingBelow rejects SST inline values when shadowing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 0, "inline"}}, + expectErr: "SST keys must have timestamps", + }, + "DisallowShadowingBelow rejects existing inline values when shadowing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 0, "a0"}}, + sst: []mvccKV{{"a", 3, "sst"}}, + expectErr: "inline values are unsupported", + }, + "DisallowShadowingBelow collision SST start, existing start, above": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, "a2"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow collision SST start, existing middle, below": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"c", 3, "c3"}}, + sst: []mvccKV{{"b", 1, "sst"}}, + expectErr: `ingested key collides with an existing one: "b"`, + }, + "DisallowShadowingBelow collision SST end, existing end, above": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"d", 3, "d3"}}, + sst: []mvccKV{{"c", 3, "sst"}, {"d", 4, "sst"}}, + expectErr: `ingested key collides with an existing one: "d"`, + }, + "DisallowShadowingBelow collision after write above tombstone": { + noShadowBelow: 5, + data: []mvccKV{{"a", 2, ""}, {"a", 1, "a1"}, {"b", 2, "b2"}}, + sst: []mvccKV{{"a", 3, "sst"}, {"b", 1, "sst"}}, + expectErr: `ingested key collides with an existing one: "b"`, + }, + "DisallowShadowingBelow at limit writes": { + noShadowBelow: 5, + sst: []mvccKV{{"a", 5, "sst"}}, + expect: []mvccKV{{"a", 5, "sst"}}, + }, + "DisallowShadowingBelow at limit errors above existing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 5, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow at limit errors above existing with same value": { + noShadowBelow: 5, + data: []mvccKV{{"a", 3, "a3"}}, + sst: []mvccKV{{"a", 5, "a3"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow at limit errors on replacing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 5, "a3"}}, + sst: []mvccKV{{"a", 5, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow at limit is idempotent": { + noShadowBelow: 5, + data: []mvccKV{{"a", 5, "a3"}}, + sst: []mvccKV{{"a", 5, "a3"}}, + expect: []mvccKV{{"a", 5, "a3"}}, + }, + "DisallowShadowingBelow above limit writes": { + noShadowBelow: 5, + sst: []mvccKV{{"a", 7, "sst"}}, + expect: []mvccKV{{"a", 7, "sst"}}, + }, + "DisallowShadowingBelow above limit errors on existing below limit": { + noShadowBelow: 5, + data: []mvccKV{{"a", 4, "a4"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit errors on existing below limit with same value": { + noShadowBelow: 5, + data: []mvccKV{{"a", 4, "a4"}}, + sst: []mvccKV{{"a", 7, "a3"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit errors on existing at limit": { + noShadowBelow: 5, + data: []mvccKV{{"a", 5, "a5"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit allows equal value at limit": { + noShadowBelow: 5, + data: []mvccKV{{"a", 5, "a5"}}, + sst: []mvccKV{{"a", 7, "a5"}}, + expect: []mvccKV{{"a", 7, "a5"}, {"a", 5, "a5"}}, + }, + "DisallowShadowingBelow above limit errors on existing above limit": { + noShadowBelow: 5, + data: []mvccKV{{"a", 6, "a6"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit allows equal value above limit": { + noShadowBelow: 5, + data: []mvccKV{{"a", 6, "a6"}}, + sst: []mvccKV{{"a", 7, "a6"}}, + expect: []mvccKV{{"a", 7, "a6"}, {"a", 6, "a6"}}, + }, + "DisallowShadowingBelow above limit errors on replacing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 7, "a7"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit is idempotent": { + noShadowBelow: 5, + data: []mvccKV{{"a", 7, "a7"}}, + sst: []mvccKV{{"a", 7, "a7"}}, + expect: []mvccKV{{"a", 7, "a7"}}, + }, + "DisallowShadowingBelow above limit errors below existing": { + noShadowBelow: 5, + data: []mvccKV{{"a", 8, "a8"}}, + sst: []mvccKV{{"a", 7, "sst"}}, + expectErr: `ingested key collides with an existing one: "a"`, + }, + "DisallowShadowingBelow above limit errors below existing with same value": { + noShadowBelow: 5, + data: []mvccKV{{"a", 8, "a8"}}, + sst: []mvccKV{{"a", 7, "a8"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, + "DisallowShadowingBelow above limit errors below tombstone": { + noShadowBelow: 5, + data: []mvccKV{{"a", 8, ""}}, + sst: []mvccKV{{"a", 7, "a8"}}, + expectErr: &roachpb.WriteTooOldError{}, + }, } - return sstFile.Data(), nil + testutils.RunTrueAndFalse(t, "IngestAsWrites", func(t *testing.T, ingestAsWrites bool) { + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + st := cluster.MakeTestingClusterSettings() + ctx := context.Background() + + dir := t.TempDir() + engine, err := storage.Open(ctx, storage.Filesystem(filepath.Join(dir, "db"))) + require.NoError(t, err) + defer engine.Close() + + // Write initial data. + intentTxn := roachpb.MakeTransaction("intentTxn", nil, 0, hlc.Timestamp{WallTime: intentTS}, 0) + b := engine.NewBatch() + for i := len(tc.data) - 1; i >= 0; i-- { // reverse, older timestamps first + kv := tc.data[i] + var txn *roachpb.Transaction + if kv.ts == intentTS { + txn = &intentTxn + } + require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.TS(), kv.Value(), txn)) + } + require.NoError(t, b.Commit(false)) + stats := engineStats(t, engine) + + // Build and add SST. + sst, start, end := makeSST(t, tc.sst) + reqTS := hlc.Timestamp{WallTime: defaultReqTS} + if tc.atReqTS != 0 { + reqTS.WallTime = tc.atReqTS + } + resp := &roachpb.AddSSTableResponse{} + result, err := batcheval.EvalAddSSTable(ctx, engine, batcheval.CommandArgs{ + EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext(), + Stats: stats, + Header: roachpb.Header{ + Timestamp: reqTS, + }, + Args: &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + MVCCStats: sstStats(t, sst), + DisallowConflicts: tc.noConflict, + DisallowShadowing: tc.noShadow, + DisallowShadowingBelow: hlc.Timestamp{WallTime: tc.noShadowBelow}, + WriteAtRequestTimestamp: tc.atReqTS != 0, + IngestAsWrites: ingestAsWrites, + }, + }, resp) + + if tc.expectErr != nil { + require.Error(t, err) + if b, ok := tc.expectErr.(bool); ok && b { + // any error is fine + } else if expectMsg, ok := tc.expectErr.(string); ok { + require.Contains(t, err.Error(), expectMsg) + } else if expectErr, ok := tc.expectErr.(error); ok { + require.True(t, errors.HasType(err, expectErr), "expected %T, got %v", expectErr, err) + } else { + require.Fail(t, "invalid expectErr", "expectErr=%v", tc.expectErr) + } + return + } + require.NoError(t, err) + + if ingestAsWrites { + require.Nil(t, result.Replicated.AddSSTable) + } else { + require.NotNil(t, result.Replicated.AddSSTable) + sstPath := filepath.Join(dir, "sst") + require.NoError(t, engine.WriteFile(sstPath, result.Replicated.AddSSTable.Data)) + require.NoError(t, engine.IngestExternalFiles(ctx, []string{sstPath})) + } + + // Scan resulting data from engine. + iter := storage.NewMVCCIncrementalIterator(engine, storage.MVCCIncrementalIterOptions{ + EndKey: keys.MaxKey, + StartTime: hlc.MinTimestamp, + EndTime: hlc.MaxTimestamp, + IntentPolicy: storage.MVCCIncrementalIterIntentPolicyEmit, + InlinePolicy: storage.MVCCIncrementalIterInlinePolicyEmit, + }) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: keys.SystemPrefix}) + scan := []mvccKV{} + for { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + key := string(iter.Key().Key) + ts := iter.Key().Timestamp.WallTime + var value []byte + if iter.Key().IsValue() { + if len(iter.Value()) > 0 { + value, err = roachpb.Value{RawBytes: iter.Value()}.GetBytes() + require.NoError(t, err) + } + } else { + var meta enginepb.MVCCMetadata + require.NoError(t, protoutil.Unmarshal(iter.UnsafeValue(), &meta)) + if meta.RawBytes == nil { + // Skip intent metadata records (value emitted separately). + iter.Next() + continue + } + value, err = roachpb.Value{RawBytes: meta.RawBytes}.GetBytes() + require.NoError(t, err) + } + scan = append(scan, mvccKV{key: key, ts: ts, value: string(value)}) + iter.Next() + } + require.Equal(t, tc.expect, scan) + + // Check that stats were updated correctly. + if tc.expectStatsEst { + require.True(t, stats.ContainsEstimates > 0, "expected stats to be estimated") + } else { + require.False(t, stats.ContainsEstimates > 0, "found estimated stats") + stats.LastUpdateNanos = 0 // avoid spurious diffs + require.Equal(t, stats, engineStats(t, engine)) + } + }) + } + }) } +// TestDBAddSSTable tests application of an SST to a database, both in-memory +// and on disk. func TestDBAddSSTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + t.Run("store=in-memory", func(t *testing.T) { - si, _, db := serverutils.StartServer(t, base.TestServerArgs{Insecure: true}) - s := si.(*server.TestServer) ctx := context.Background() + s, _, db := serverutils.StartServer(t, base.TestServerArgs{Insecure: true}) defer s.Stopper().Stop(ctx) - - tr := s.Tracer() + tr := s.TracerI().(*tracing.Tracer) runTestDBAddSSTable(ctx, t, db, tr, nil) }) - t.Run("store=on-disk", func(t *testing.T) { - dir, dirCleanupFn := testutils.TempDir(t) - defer dirCleanupFn() + t.Run("store=on-disk", func(t *testing.T) { + ctx := context.Background() storeSpec := base.DefaultTestStoreSpec storeSpec.InMemory = false - storeSpec.Path = dir - si, _, db := serverutils.StartServer(t, base.TestServerArgs{ + storeSpec.Path = t.TempDir() + s, _, db := serverutils.StartServer(t, base.TestServerArgs{ Insecure: true, StoreSpecs: []base.StoreSpec{storeSpec}, }) - s := si.(*server.TestServer) - ctx := context.Background() defer s.Stopper().Stop(ctx) - store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) - if err != nil { - t.Fatal(err) - } - tr := s.TracerI().(*tracing.Tracer) + store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) + require.NoError(t, err) runTestDBAddSSTable(ctx, t, db, tr, store) }) } -const ingestAsWrites, ingestAsSST = true, false - -var nilStats *enginepb.MVCCStats - // if store != nil, assume it is on-disk and check ingestion semantics. func runTestDBAddSSTable( ctx context.Context, t *testing.T, db *kv.DB, tr *tracing.Tracer, store *kvserver.Store, ) { tr.TestingRecordAsyncSpans() // we assert on async span traces in this test + const ingestAsWrites, ingestAsSST = true, false + const writeAtSST = false + const allowConflicts = false + const allowShadowing = false + var allowShadowingBelow hlc.Timestamp + var nilStats *enginepb.MVCCStats + var noTS hlc.Timestamp + { - key := storage.MVCCKey{Key: []byte("bb"), Timestamp: hlc.Timestamp{WallTime: 2}} - data, err := singleKVSSTable(key, roachpb.MakeValueFromString("1").RawBytes) - if err != nil { - t.Fatalf("%+v", err) - } + sst, start, end := makeSST(t, []mvccKV{{"bb", 2, "1"}}) // Key is before the range in the request span. - if err := db.AddSSTable( - ctx, "d", "e", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); !testutils.IsError(err, "not in request range") { - t.Fatalf("expected request range error got: %+v", err) - } + err := db.AddSSTable( + ctx, "d", "e", sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST) + require.Error(t, err) + require.Contains(t, err.Error(), "not in request range") + // Key is after the range in the request span. - if err := db.AddSSTable( - ctx, "a", "b", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); !testutils.IsError(err, "not in request range") { - t.Fatalf("expected request range error got: %+v", err) - } + err = db.AddSSTable( + ctx, "a", "b", sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST) + require.Error(t, err) + require.Contains(t, err.Error(), "not in request range") // Do an initial ingest. ingestCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test-recording") defer getRecAndFinish() - if err := db.AddSSTable( - ingestCtx, "b", "c", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); err != nil { - t.Fatalf("%+v", err) - } - formatted := getRecAndFinish().String() - if err := testutils.MatchEach(formatted, - "evaluating AddSSTable", - "sideloadable proposal detected", - "ingested SSTable at index", - ); err != nil { - t.Fatal(err) - } + require.NoError(t, db.AddSSTable( + ingestCtx, start, end, sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST)) + trace := getRecAndFinish().String() + require.Contains(t, trace, "evaluating AddSSTable") + require.Contains(t, trace, "sideloadable proposal detected") + require.Contains(t, trace, "ingested SSTable at index") if store != nil { // Look for the ingested path and verify it still exists. re := regexp.MustCompile(`ingested SSTable at index \d+, term \d+: (\S+)`) - match := re.FindStringSubmatch(formatted) - if len(match) != 2 { - t.Fatalf("failed to extract ingested path from message %q,\n got: %v", formatted, match) - } + match := re.FindStringSubmatch(trace) + require.Len(t, match, 2, "failed to extract ingested path from message %q,\n got: %v", trace, match) + // The on-disk paths have `.ingested` appended unlike in-memory. - suffix := ".ingested" - if _, err := os.Stat(strings.TrimSuffix(match[1], suffix)); err != nil { - t.Fatalf("%q file missing after ingest: %+v", match[1], err) - } - } - if r, err := db.Get(ctx, "bb"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("1"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) + _, err = os.Stat(strings.TrimSuffix(match[1], ".ingested")) + require.NoError(t, err, "%q file missing after ingest: %+v", match[1], err) } + r, err := db.Get(ctx, "bb") + require.NoError(t, err) + require.Equal(t, []byte("1"), r.ValueBytes()) } // Check that ingesting a key with an earlier mvcc timestamp doesn't affect // the value returned by Get. { - key := storage.MVCCKey{Key: []byte("bb"), Timestamp: hlc.Timestamp{WallTime: 1}} - data, err := singleKVSSTable(key, roachpb.MakeValueFromString("2").RawBytes) - if err != nil { - t.Fatalf("%+v", err) - } - - if err := db.AddSSTable( - ctx, "b", "c", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); err != nil { - t.Fatalf("%+v", err) - } - if r, err := db.Get(ctx, "bb"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("1"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) - } + sst, start, end := makeSST(t, []mvccKV{{"bb", 1, "2"}}) + require.NoError(t, db.AddSSTable( + ctx, start, end, sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST)) + r, err := db.Get(ctx, "bb") + require.NoError(t, err) + require.Equal(t, []byte("1"), r.ValueBytes()) if store != nil { - metrics := store.Metrics() - if expected, got := int64(2), metrics.AddSSTableApplications.Count(); expected != got { - t.Fatalf("expected %d sst ingestions, got %d", expected, got) - } + require.EqualValues(t, 2, store.Metrics().AddSSTableApplications.Count()) } } // Key range in request span is not empty. First time through a different // key is present. Second time through checks the idempotency. { - key := storage.MVCCKey{Key: []byte("bc"), Timestamp: hlc.Timestamp{WallTime: 1}} - data, err := singleKVSSTable(key, roachpb.MakeValueFromString("3").RawBytes) - if err != nil { - t.Fatalf("%+v", err) - } + sst, start, end := makeSST(t, []mvccKV{{"bc", 1, "3"}}) - var metrics *kvserver.StoreMetrics var before int64 if store != nil { - metrics = store.Metrics() - before = metrics.AddSSTableApplicationCopies.Count() + before = store.Metrics().AddSSTableApplicationCopies.Count() } for i := 0; i < 2; i++ { ingestCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test-recording") defer getRecAndFinish() - if err := db.AddSSTable( - ingestCtx, "b", "c", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); err != nil { - t.Fatalf("%+v", err) - } - if err := testutils.MatchEach(getRecAndFinish().String(), - "evaluating AddSSTable", - "sideloadable proposal detected", - "ingested SSTable at index", - ); err != nil { - t.Fatal(err) - } - - if r, err := db.Get(ctx, "bb"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("1"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) - } - if r, err := db.Get(ctx, "bc"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("3"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) - } + require.NoError(t, db.AddSSTable( + ingestCtx, start, end, sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST)) + trace := getRecAndFinish().String() + require.Contains(t, trace, "evaluating AddSSTable") + require.Contains(t, trace, "sideloadable proposal detected") + require.Contains(t, trace, "ingested SSTable at index") + + r, err := db.Get(ctx, "bb") + require.NoError(t, err) + require.Equal(t, []byte("1"), r.ValueBytes()) + + r, err = db.Get(ctx, "bc") + require.NoError(t, err) + require.Equal(t, []byte("3"), r.ValueBytes()) } if store != nil { - if expected, got := int64(4), metrics.AddSSTableApplications.Count(); expected != got { - t.Fatalf("expected %d sst ingestions, got %d", expected, got) - } + require.EqualValues(t, 4, store.Metrics().AddSSTableApplications.Count()) // The second time though we had to make a copy of the SST since rocks saw // existing data (from the first time), and rejected the no-modification // attempt. - if after := metrics.AddSSTableApplicationCopies.Count(); before != after { - t.Fatalf("expected sst copies not to increase, %d before %d after", before, after) - } + require.Equal(t, before, store.Metrics().AddSSTableApplicationCopies.Count()) } } // ... and doing the same thing but via write-batch works the same. { - key := storage.MVCCKey{Key: []byte("bd"), Timestamp: hlc.Timestamp{WallTime: 1}} - data, err := singleKVSSTable(key, roachpb.MakeValueFromString("3").RawBytes) - if err != nil { - t.Fatalf("%+v", err) - } + sst, start, end := makeSST(t, []mvccKV{{"bd", 1, "3"}}) - var metrics *kvserver.StoreMetrics var before int64 if store != nil { - metrics = store.Metrics() - before = metrics.AddSSTableApplications.Count() + before = store.Metrics().AddSSTableApplications.Count() } for i := 0; i < 2; i++ { ingestCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test-recording") defer getRecAndFinish() - if err := db.AddSSTable( - ingestCtx, "b", "c", data, false /* disallowShadowing */, nilStats, ingestAsWrites, hlc.Timestamp{}, - ); err != nil { - t.Fatalf("%+v", err) - } - if err := testutils.MatchEach(getRecAndFinish().String(), - "evaluating AddSSTable", - "via regular write batch", - ); err != nil { - t.Fatal(err) - } - - if r, err := db.Get(ctx, "bb"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("1"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) - } - if r, err := db.Get(ctx, "bd"); err != nil { - t.Fatalf("%+v", err) - } else if expected := []byte("3"); !bytes.Equal(expected, r.ValueBytes()) { - t.Errorf("expected %q, got %q", expected, r.ValueBytes()) - } + require.NoError(t, db.AddSSTable( + ingestCtx, start, end, sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsWrites, noTS, writeAtSST)) + trace := getRecAndFinish().String() + require.Contains(t, trace, "evaluating AddSSTable") + require.Contains(t, trace, "via regular write batch") + + r, err := db.Get(ctx, "bb") + require.NoError(t, err) + require.Equal(t, []byte("1"), r.ValueBytes()) + + r, err = db.Get(ctx, "bd") + require.NoError(t, err) + require.Equal(t, []byte("3"), r.ValueBytes()) } if store != nil { - if expected, got := before, metrics.AddSSTableApplications.Count(); expected != got { - t.Fatalf("expected %d sst ingestions, got %d", expected, got) - } + require.Equal(t, before, store.Metrics().AddSSTableApplications.Count()) } } @@ -298,790 +909,241 @@ func runTestDBAddSSTable( key := storage.MVCCKey{Key: []byte("bb"), Timestamp: hlc.Timestamp{WallTime: 1}} value := roachpb.MakeValueFromString("1") value.InitChecksum([]byte("foo")) - data, err := singleKVSSTable(key, value.RawBytes) - if err != nil { - t.Fatalf("%+v", err) - } - - if err := db.AddSSTable( - ctx, "b", "c", data, false /* disallowShadowing */, nilStats, ingestAsSST, hlc.Timestamp{}, - ); !testutils.IsError(err, "invalid checksum") { - t.Fatalf("expected 'invalid checksum' error got: %+v", err) - } - } -} -type strKv struct { - k string - ts int64 - v string -} + sstFile := &storage.MemFile{} + w := storage.MakeBackupSSTWriter(sstFile) + defer w.Close() + require.NoError(t, w.Put(key, value.RawBytes)) + require.NoError(t, w.Finish()) -func mvccKVsFromStrs(in []strKv) []storage.MVCCKeyValue { - kvs := make([]storage.MVCCKeyValue, len(in)) - for i := range kvs { - kvs[i].Key.Key = []byte(in[i].k) - kvs[i].Key.Timestamp.WallTime = in[i].ts - if in[i].v != "" { - kvs[i].Value = roachpb.MakeValueFromBytes([]byte(in[i].v)).RawBytes - } else { - kvs[i].Value = nil - } + err := db.AddSSTable( + ctx, "b", "c", sstFile.Data(), allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS, writeAtSST) + require.Error(t, err) + require.Contains(t, err.Error(), "invalid checksum") } - sort.Slice(kvs, func(i, j int) bool { return kvs[i].Key.Less(kvs[j].Key) }) - return kvs } +// TestAddSSTableMVCCStats tests that statistics are computed accurately. func TestAddSSTableMVCCStats(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range engineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - for _, kv := range mvccKVsFromStrs([]strKv{ - {"A", 1, "A"}, - {"a", 1, "a"}, - {"a", 6, ""}, - {"b", 5, "bb"}, - {"c", 6, "ccccccccccccccccccccccccccccccccccccccccccccc"}, // key 4b, 50b, live 64b - {"d", 1, "d"}, - {"d", 2, ""}, - {"e", 1, "e"}, - {"z", 2, "zzzzzz"}, - }) { - if err := e.PutMVCC(kv.Key, kv.Value); err != nil { - t.Fatalf("%+v", err) - } - } - - sstKVs := mvccKVsFromStrs([]strKv{ - {"a", 2, "aa"}, // mvcc-shadowed within SST. - {"a", 4, "aaaaaa"}, // mvcc-shadowed by existing delete. - {"c", 6, "ccc"}, // same TS as existing, LSM-shadows existing. - {"d", 4, "dddd"}, // mvcc-shadow existing deleted d. - {"e", 4, "eeee"}, // mvcc-shadow existing 1b. - {"j", 2, "jj"}, // no colission – via MVCC or LSM – with existing. - }) - var delta enginepb.MVCCStats - // the sst will think it added 4 keys here, but a, c, and e shadow or are shadowed. - delta.LiveCount = -3 - delta.LiveBytes = -109 - // the sst will think it added 5 keys, but only j is new so 4 are over-counted. - delta.KeyCount = -4 - delta.KeyBytes = -20 - // the sst will think it added 6 values, but since one was a perfect (key+ts) - // collision, it *replaced* the existing value and is over-counted. - delta.ValCount = -1 - delta.ValBytes = -50 - - // Add in a random metadata key. - ts := hlc.Timestamp{WallTime: 7} - txn := roachpb.MakeTransaction( - "test", - nil, // baseKey - roachpb.NormalUserPriority, - ts, - base.DefaultMaxClockOffset.Nanoseconds(), - ) - if err := storage.MVCCPut( - ctx, e, nil, []byte("i"), ts, - roachpb.MakeValueFromBytes([]byte("it")), - &txn, - ); err != nil { - if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { - t.Fatalf("%+v", err) - } - } + st := cluster.MakeTestingClusterSettings() + evalCtx := (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext() - // After EvalAddSSTable, cArgs.Stats contains a diff to the existing - // stats. Make sure recomputing from scratch gets the same answer as - // applying the diff to the stats - beforeStats := func() enginepb.MVCCStats { - iter := e.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) - defer iter.Close() - beforeStats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, roachpb.KeyMax, 10) - if err != nil { - t.Fatalf("%+v", err) - } - return beforeStats - }() - - mkSST := func(kvs []storage.MVCCKeyValue) []byte { - sstFile := &storage.MemFile{} - sst := storage.MakeBackupSSTWriter(sstFile) - defer sst.Close() - for _, kv := range kvs { - if err := sst.Put(kv.Key, kv.Value); err != nil { - t.Fatalf("%+v", err) - } - } - if err := sst.Finish(); err != nil { - t.Fatalf("%+v", err) - } - return sstFile.Data() - } - - sstBytes := mkSST(sstKVs) - - cArgs := batcheval.CommandArgs{ - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: keys.MinKey, EndKey: keys.MaxKey}, - Data: sstBytes, - }, - Stats: &enginepb.MVCCStats{}, - } - if _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil); err != nil { - t.Fatalf("%+v", err) - } - - evaledStats := beforeStats - evaledStats.Add(*cArgs.Stats) - - if err := e.WriteFile("sst", sstBytes); err != nil { - t.Fatalf("%+v", err) - } - if err := e.IngestExternalFiles(ctx, []string{"sst"}); err != nil { - t.Fatalf("%+v", err) - } - - afterStats := func() enginepb.MVCCStats { - iter := e.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) - defer iter.Close() - afterStats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, roachpb.KeyMax, 10) - if err != nil { - t.Fatalf("%+v", err) - } - return afterStats - }() - evaledStats.Add(delta) - evaledStats.ContainsEstimates = 0 - if !afterStats.Equal(evaledStats) { - t.Errorf("mvcc stats mismatch: diff(expected, actual): %s", pretty.Diff(afterStats, evaledStats)) - } - - cArgsWithStats := batcheval.CommandArgs{ - Header: roachpb.Header{Timestamp: hlc.Timestamp{WallTime: 7}}, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: keys.MinKey, EndKey: keys.MaxKey}, - Data: mkSST([]storage.MVCCKeyValue{{ - Key: storage.MVCCKey{Key: roachpb.Key("zzzzzzz"), Timestamp: ts}, - Value: roachpb.MakeValueFromBytes([]byte("zzz")).RawBytes, - }}), - MVCCStats: &enginepb.MVCCStats{KeyCount: 10}, - }, - Stats: &enginepb.MVCCStats{}, - } - if _, err := batcheval.EvalAddSSTable(ctx, e, cArgsWithStats, nil); err != nil { - t.Fatalf("%+v", err) - } - expected := enginepb.MVCCStats{ContainsEstimates: 1, KeyCount: 10} - if got := *cArgsWithStats.Stats; got != expected { - t.Fatalf("expected %v got %v", expected, got) - } - }) + dir := t.TempDir() + engine, err := storage.Open(ctx, storage.Filesystem(filepath.Join(dir, "db"))) + require.NoError(t, err) + defer engine.Close() + + for _, kv := range []mvccKV{ + {"A", 1, "A"}, + {"a", 1, "a"}, + {"a", 6, ""}, + {"b", 5, "bb"}, + {"c", 6, "ccccccccccccccccccccccccccccccccccccccccccccc"}, // key 4b, 50b, live 64b + {"d", 1, "d"}, + {"d", 2, ""}, + {"e", 1, "e"}, + {"z", 2, "zzzzzz"}, + } { + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + } + + sst, start, end := makeSST(t, []mvccKV{ + {"a", 4, "aaaaaa"}, // mvcc-shadowed by existing delete. + {"a", 2, "aa"}, // mvcc-shadowed within SST. + {"c", 6, "ccc"}, // same TS as existing, LSM-shadows existing. + {"d", 4, "dddd"}, // mvcc-shadow existing deleted d. + {"e", 4, "eeee"}, // mvcc-shadow existing 1b. + {"j", 2, "jj"}, // no colission – via MVCC or LSM – with existing. + }) + statsDelta := enginepb.MVCCStats{ + // the sst will think it added 4 keys here, but a, c, and e shadow or are shadowed. + LiveCount: -3, + LiveBytes: -109, + // the sst will think it added 5 keys, but only j is new so 4 are over-counted. + KeyCount: -4, + KeyBytes: -20, + // the sst will think it added 6 values, but since one was a perfect (key+ts) + // collision, it *replaced* the existing value and is over-counted. + ValCount: -1, + ValBytes: -50, + } + + // After EvalAddSSTable, cArgs.Stats contains a diff to the existing + // stats. Make sure recomputing from scratch gets the same answer as + // applying the diff to the stats + statsBefore := engineStats(t, engine) + ts := hlc.Timestamp{WallTime: 7} + cArgs := batcheval.CommandArgs{ + EvalCtx: evalCtx, + Header: roachpb.Header{ + Timestamp: ts, + }, + Args: &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + }, + Stats: &enginepb.MVCCStats{}, } + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, nil) + require.NoError(t, err) + + sstPath := filepath.Join(dir, "sst") + require.NoError(t, engine.WriteFile(sstPath, sst)) + require.NoError(t, engine.IngestExternalFiles(ctx, []string{sstPath})) + + statsEvaled := statsBefore + statsEvaled.Add(*cArgs.Stats) + statsEvaled.Add(statsDelta) + statsEvaled.ContainsEstimates = 0 + statsEvaled.LastUpdateNanos = 0 + require.Equal(t, engineStats(t, engine), statsEvaled) + + // Check stats for a single KV. + sst, start, end = makeSST(t, []mvccKV{{"zzzzzzz", ts.WallTime, "zzz"}}) + cArgsWithStats := batcheval.CommandArgs{ + EvalCtx: evalCtx, + Header: roachpb.Header{Timestamp: ts}, + Args: &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + MVCCStats: &enginepb.MVCCStats{KeyCount: 10}, + }, + Stats: &enginepb.MVCCStats{}, + } + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgsWithStats, nil) + require.NoError(t, err) + require.Equal(t, enginepb.MVCCStats{ContainsEstimates: 1, KeyCount: 10}, *cArgsWithStats.Stats) } -func TestAddSSTableDisallowShadowing(t *testing.T) { +// TestAddSSTableMVCCStatsDisallowShadowing tests that stats are computed +// accurately when DisallowShadowing is set. +func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range engineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - st := cluster.MakeTestingClusterSettings() - evalCtx := (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext() - - for _, kv := range mvccKVsFromStrs([]strKv{ - {"a", 2, "aa"}, - {"b", 1, "bb"}, - {"b", 6, ""}, - {"g", 5, "gg"}, - {"r", 1, "rr"}, - {"y", 1, "yy"}, - {"y", 2, ""}, - {"y", 5, "yyy"}, - {"z", 2, "zz"}, - }) { - if err := e.PutMVCC(kv.Key, kv.Value); err != nil { - t.Fatalf("%+v", err) - } - } - - getSSTBytes := func(sstKVs []storage.MVCCKeyValue) []byte { - sstFile := &storage.MemFile{} - sst := storage.MakeBackupSSTWriter(sstFile) - defer sst.Close() - for _, kv := range sstKVs { - if err := sst.Put(kv.Key, kv.Value); err != nil { - t.Fatalf("%+v", err) - } - } - if err := sst.Finish(); err != nil { - t.Fatalf("%+v", err) - } - return sstFile.Data() - } - - getStats := func(startKey, endKey roachpb.Key, data []byte) enginepb.MVCCStats { - dataIter, err := storage.NewMemSSTIterator(data, true) - if err != nil { - return enginepb.MVCCStats{} - } - defer dataIter.Close() - - stats, err := storage.ComputeStatsForRange(dataIter, startKey, endKey, 0) - if err != nil { - t.Fatalf("%+v", err) - } - return stats - } - - // Test key collision when ingesting a key in the start of existing data, and - // SST. The colliding key is also equal to the header start key. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"a", 7, "aa"}, // colliding key has a higher timestamp than existing version. - }) - - sstBytes := getSSTBytes(sstKVs) - stats := getStats(roachpb.Key("a"), roachpb.Key("b"), sstBytes) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, - Data: sstBytes, - DisallowShadowing: true, - MVCCStats: &stats, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"a\"") { - t.Fatalf("%+v", err) - } - } - - // Test key collision when ingesting a key in the middle of existing data, and - // start of the SST. The key is equal to the header start key. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"g", 4, "ggg"}, // colliding key has a lower timestamp than existing version. - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("g"), EndKey: roachpb.Key("h")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"g\"") { - t.Fatalf("%+v", err) - } - } - - // Test key collision when ingesting a key at the end of the existing data and - // SST. The colliding key is not equal to header start key. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "f"}, - {"h", 4, "h"}, - {"s", 1, "s"}, - {"z", 3, "z"}, // colliding key has a higher timestamp than existing version. - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - }, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"z\"") { - t.Fatalf("%+v", err) - } - } - - // Test for no key collision where the key range being ingested into is - // non-empty. The header start and end keys are not existing keys. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"c", 2, "bb"}, - {"h", 6, "hh"}, - }) - - sstBytes := getSSTBytes(sstKVs) - stats := getStats(roachpb.Key("c"), roachpb.Key("i"), sstBytes) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("c"), EndKey: roachpb.Key("i")}, - Data: sstBytes, - DisallowShadowing: true, - MVCCStats: &stats, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if err != nil { - t.Fatalf("%+v", err) - } - } - - // Test that a collision is not reported when ingesting a key for which we - // find a tombstone from an MVCC delete, and the sst key has a ts >= tombstone - // ts. Also test that iteration continues from the next key in the existing - // data after skipping over all the versions of the deleted key. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"b", 7, "bb"}, // colliding key has a higher timestamp than its deleted version. - {"b", 1, "bbb"}, // older version of deleted key (should be skipped over). - {"f", 3, "ff"}, - {"y", 3, "yyyy"}, // colliding key. - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("b"), EndKey: roachpb.Key("z")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { - t.Fatalf("%+v", err) - } - } - - // Test that a collision is reported when ingesting a key for which we find a - // tombstone from an MVCC delete, but the sst key has a ts < tombstone ts. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"b", 4, "bb"}, // colliding key has a lower timestamp than its deleted version. - {"f", 3, "ff"}, - {"y", 3, "yyyy"}, - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("b"), EndKey: roachpb.Key("z")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"b\"") { - t.Fatalf("%+v", err) - } - } - - // Test key collision when ingesting a key which has been deleted, and readded - // in the middle of the existing data. The colliding key is in the middle of - // the SST, and is the earlier of the two possible collisions. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"y", 4, "yyy"}, // colliding key has a lower timestamp than the readded version. - {"z", 3, "zzz"}, - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { - t.Fatalf("%+v", err) - } - } - - // Test key collision when ingesting keys which have write intents in the - // existing data. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"q", 4, "qq"}, // has a write intent in the existing data - {"t", 3, "ttt"}, // has a write intent in the existing data - }) - - // Add in two write intents. - ts := hlc.Timestamp{WallTime: 7} - txn := roachpb.MakeTransaction( - "test", - nil, // baseKey - roachpb.NormalUserPriority, - ts, - base.DefaultMaxClockOffset.Nanoseconds(), - ) - if err := storage.MVCCPut( - ctx, e, nil, []byte("q"), ts, - roachpb.MakeValueFromBytes([]byte("q")), - &txn, - ); err != nil { - t.Fatalf("%+v", err) - } - if err := storage.MVCCPut( - ctx, e, nil, []byte("t"), ts, - roachpb.MakeValueFromBytes([]byte("tt")), - &txn, - ); err != nil { - t.Fatalf("%+v", err) - } - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("u")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "conflicting intents on \"q\", \"t\"") { - t.Fatalf("%+v", err) - } - } - - // Test key collision when ingesting a key which has an inline value in the - // existing data. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"i", 4, "ii"}, // has an inline value in existing data. - {"j", 3, "jj"}, - }) - - // Add in an inline value. - ts := hlc.Timestamp{} - if err := storage.MVCCPut( - ctx, e, nil, []byte("i"), ts, - roachpb.MakeValueFromBytes([]byte("i")), - nil, - ); err != nil { - t.Fatalf("%+v", err) - } - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("k")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "inline values are unsupported when checking for key collisions") { - t.Fatalf("%+v", err) - } - } - - // Test ingesting a key with the same timestamp and value. This should not - // trigger a collision error. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"e", 4, "ee"}, - {"f", 2, "ff"}, - {"y", 5, "yyy"}, // key has the same timestamp and value as the one present in the existing data. - }) - - sstBytes := getSSTBytes(sstKVs) - stats := getStats(roachpb.Key("e"), roachpb.Key("zz"), sstBytes) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("e"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - MVCCStats: &stats, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if err != nil { - t.Fatalf("%+v", err) - } - } - - // Test ingesting a key with different timestamp but same value. This should - // trigger a collision error. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"y", 6, "yyy"}, // key has a higher timestamp but same value as the one present in the existing data. - {"z", 3, "zzz"}, - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { - t.Fatalf("%+v", err) - } - } - - // Test ingesting a key with the same timestamp but different value. This should - // trigger a collision error. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"y", 5, "yyyy"}, // key has the same timestamp but different value as the one present in the existing data. - {"z", 3, "zzz"}, - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("f"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } - - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { - t.Fatalf("%+v", err) - } - } - - // Test that a collision after a key with the same timestamp and value causes - // a collision error. - { - sstKVs := mvccKVsFromStrs([]strKv{ - {"f", 2, "ff"}, - {"y", 5, "yyy"}, // key has the same timestamp and value as the one present in the existing data - not a collision. - {"z", 3, "zzz"}, // shadow key - }) - - sstBytes := getSSTBytes(sstKVs) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("e"), EndKey: roachpb.Key("zz")}, - Data: sstBytes, - DisallowShadowing: true, - }, - Stats: &enginepb.MVCCStats{}, - } + st := cluster.MakeTestingClusterSettings() + evalCtx := (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext() + + engine := storage.NewDefaultInMemForTesting() + defer engine.Close() + + for _, kv := range []mvccKV{ + {"a", 2, "aa"}, + {"b", 1, "bb"}, + {"b", 6, ""}, + {"g", 5, "gg"}, + {"r", 1, "rr"}, + {"y", 1, "yy"}, + {"y", 2, ""}, + {"y", 5, "yyy"}, + {"z", 2, "zz"}, + } { + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + } - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"z\"") { - t.Fatalf("%+v", err) - } - } - - // This test ensures accuracy of MVCCStats in the situation that successive - // SSTs being ingested via AddSSTable have "perfectly shadowing" keys (same ts - // and value). Such KVs are not considered as collisions and so while they are - // skipped during ingestion, their stats would previously be double counted. - // To mitigate this problem we now return the stats of such skipped KVs while - // evaluating the AddSSTable command, and accumulate accurate stats in the - // CommandArgs Stats field by using: - // cArgs.Stats + ingested_stats - skipped_stats. - { - // Successfully evaluate the first SST as there are no key collisions. - sstKVs := mvccKVsFromStrs([]strKv{ - {"c", 2, "bb"}, - {"h", 6, "hh"}, - }) + // This test ensures accuracy of MVCCStats in the situation that successive + // SSTs being ingested via AddSSTable have "perfectly shadowing" keys (same ts + // and value). Such KVs are not considered as collisions and so while they are + // skipped during ingestion, their stats would previously be double counted. + // To mitigate this problem we now return the stats of such skipped KVs while + // evaluating the AddSSTable command, and accumulate accurate stats in the + // CommandArgs Stats field by using: + // cArgs.Stats + ingested_stats - skipped_stats. + // Successfully evaluate the first SST as there are no key collisions. + kvs := []mvccKV{ + {"c", 2, "bb"}, + {"h", 6, "hh"}, + } + sst, start, end := makeSST(t, kvs) + + // Accumulate stats across SST ingestion. + commandStats := enginepb.MVCCStats{} + + cArgs := batcheval.CommandArgs{ + EvalCtx: evalCtx, + Header: roachpb.Header{ + Timestamp: hlc.Timestamp{WallTime: 7}, + }, + Args: &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + DisallowShadowing: true, + MVCCStats: sstStats(t, sst), + }, + Stats: &commandStats, + } + _, err := batcheval.EvalAddSSTable(ctx, engine, cArgs, nil) + require.NoError(t, err) + firstSSTStats := commandStats - sstBytes := getSSTBytes(sstKVs) - stats := getStats(roachpb.Key("c"), roachpb.Key("i"), sstBytes) + // Insert KV entries so that we can correctly identify keys to skip when + // ingesting the perfectly shadowing KVs (same ts and same value) in the + // second SST. + for _, kv := range kvs { + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + } - // Accumulate stats across SST ingestion. - commandStats := enginepb.MVCCStats{} + // Evaluate the second SST. Both the KVs are perfectly shadowing and should + // not contribute to the stats. + sst, start, end = makeSST(t, []mvccKV{ + {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. + {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. + }) - cArgs := batcheval.CommandArgs{ - EvalCtx: evalCtx, - Header: roachpb.Header{ - Timestamp: hlc.Timestamp{WallTime: 7}, - }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("c"), EndKey: roachpb.Key("i")}, - Data: sstBytes, - DisallowShadowing: true, - MVCCStats: &stats, - }, - Stats: &commandStats, - } - _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if err != nil { - t.Fatalf("%+v", err) - } - firstSSTStats := commandStats - - // Insert KV entries so that we can correctly identify keys to skip when - // ingesting the perfectly shadowing KVs (same ts and same value) in the - // second SST. - for _, kv := range sstKVs { - if err := e.PutMVCC(kv.Key, kv.Value); err != nil { - t.Fatalf("%+v", err) - } - } + cArgs.Args = &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + DisallowShadowing: true, + MVCCStats: sstStats(t, sst), + } + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, nil) + require.NoError(t, err) - // Evaluate the second SST. Both the KVs are perfectly shadowing and should - // not contribute to the stats. - secondSSTKVs := mvccKVsFromStrs([]strKv{ - {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. - {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. - }) - secondSSTBytes := getSSTBytes(secondSSTKVs) - secondStats := getStats(roachpb.Key("c"), roachpb.Key("i"), secondSSTBytes) - - cArgs.Args = &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("c"), EndKey: roachpb.Key("i")}, - Data: secondSSTBytes, - DisallowShadowing: true, - MVCCStats: &secondStats, - } - _, err = batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if err != nil { - t.Fatalf("%+v", err) - } + // Check that there has been no double counting of stats. All keys in second SST are shadowing. + require.Equal(t, firstSSTStats, *cArgs.Stats) - // Check that there has been no double counting of stats. - if !firstSSTStats.Equal(*cArgs.Stats) { - t.Errorf("mvcc stats should not have changed as all keys in second SST are shadowing: %s", - pretty.Diff(firstSSTStats, *cArgs.Stats)) - } + // Evaluate the third SST. Two of the three KVs are perfectly shadowing, but + // there is one valid KV which should contribute to the stats. + sst, start, end = makeSST(t, []mvccKV{ + {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. + {"e", 2, "ee"}, + {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. + }) - // Evaluate the third SST. Two of the three KVs are perfectly shadowing, but - // there is one valid KV which should contribute to the stats. - thirdSSTKVs := mvccKVsFromStrs([]strKv{ - {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. - {"e", 2, "ee"}, - {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. - }) - thirdSSTBytes := getSSTBytes(thirdSSTKVs) - thirdStats := getStats(roachpb.Key("c"), roachpb.Key("i"), thirdSSTBytes) - - cArgs.Args = &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("c"), EndKey: roachpb.Key("i")}, - Data: thirdSSTBytes, - DisallowShadowing: true, - MVCCStats: &thirdStats, - } - _, err = batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if err != nil { - t.Fatalf("%+v", err) - } + cArgs.Args = &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + DisallowShadowing: true, + MVCCStats: sstStats(t, sst), + } + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, nil) + require.NoError(t, err) - // This is the stats contribution of the KV {"e", 2, "ee"}. This should be - // the only addition to the cumulative stats, as the other two KVs are - // perfect shadows of existing data. - var delta enginepb.MVCCStats - delta.LiveCount = 1 - delta.LiveBytes = 21 - delta.KeyCount = 1 - delta.KeyBytes = 14 - delta.ValCount = 1 - delta.ValBytes = 7 - - // Check that there has been no double counting of stats. - firstSSTStats.Add(delta) - if !firstSSTStats.Equal(*cArgs.Stats) { - t.Errorf("mvcc stats are not accurate: %s", - pretty.Diff(firstSSTStats, *cArgs.Stats)) - } - } - }) + // This is the stats contribution of the KV {"e", 2, "ee"}. This should be + // the only addition to the cumulative stats, as the other two KVs are + // perfect shadows of existing data. + delta := enginepb.MVCCStats{ + LiveCount: 1, + LiveBytes: 21, + KeyCount: 1, + KeyBytes: 14, + ValCount: 1, + ValBytes: 7, } + + // Check that there has been no double counting of stats. + firstSSTStats.Add(delta) + require.Equal(t, firstSSTStats, *cArgs.Stats) } -func TestAddSSTableDisallowShadowingIntentResolution(t *testing.T) { +// TestAddSSTableIntentResolution tests that AddSSTable resolves +// intents of conflicting transactions. +func TestAddSSTableIntentResolution(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1096,19 +1158,18 @@ func TestAddSSTableDisallowShadowingIntentResolution(t *testing.T) { // Generate an SSTable that covers keys a, b, and c, and submit it with high // priority. This is going to abort the transaction above, encounter its // intent, and resolve it. - sst := makeSST(t, s.Clock().Now(), map[string]string{ - "a": "1", - "b": "2", - "c": "3", + sst, start, end := makeSST(t, []mvccKV{ + {"a", 1, "1"}, + {"b", 1, "2"}, + {"c", 1, "3"}, }) - stats := sstStats(t, sst) - - ba := roachpb.BatchRequest{} - ba.Header.UserPriority = roachpb.MaxUserPriority + ba := roachpb.BatchRequest{ + Header: roachpb.Header{UserPriority: roachpb.MaxUserPriority}, + } ba.Add(&roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a"), EndKey: roachpb.Key("d")}, + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: stats, + MVCCStats: sstStats(t, sst), DisallowShadowing: true, }) _, pErr := db.NonTransactionalSender().Send(ctx, ba) @@ -1120,31 +1181,144 @@ func TestAddSSTableDisallowShadowingIntentResolution(t *testing.T) { require.Contains(t, err.Error(), "TransactionRetryWithProtoRefreshError: TransactionAbortedError") } -func makeSST(t *testing.T, ts hlc.Timestamp, kvs map[string]string) []byte { +// TestAddSSTableWriteAtRequestTimestampRespectsTSCache checks that AddSSTable +// with WriteAtRequestTimestamp respects the timestamp cache. +func TestAddSSTableWriteAtRequestTimestampRespectsTSCache(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{}, + }) + defer s.Stopper().Stop(ctx) + + // Write key. + txn := db.NewTxn(ctx, "txn") + require.NoError(t, txn.Put(ctx, "key", "txn")) + require.NoError(t, txn.Commit(ctx)) + txnTS := txn.CommitTimestamp() + + // Add an SST writing below the previous write. + sst, start, end := makeSST(t, []mvccKV{{"key", 1, "sst"}}) + sstReq := &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + MVCCStats: sstStats(t, sst), + WriteAtRequestTimestamp: true, + } + ba := roachpb.BatchRequest{ + Header: roachpb.Header{Timestamp: txnTS.Prev()}, + } + ba.Add(sstReq) + _, pErr := db.NonTransactionalSender().Send(ctx, ba) + require.Nil(t, pErr) + + // Reading gets the value from the txn, because the tscache allowed writing + // below the committed value. + kv, err := db.Get(ctx, "key") + require.NoError(t, err) + require.Equal(t, "txn", string(kv.ValueBytes())) + + // Adding the SST again and reading results in the new value, because the + // tscache pushed the SST forward. + ba = roachpb.BatchRequest{ + Header: roachpb.Header{Timestamp: txnTS.Prev()}, + } + ba.Add(sstReq) + _, pErr = db.NonTransactionalSender().Send(ctx, ba) + require.Nil(t, pErr) + + kv, err = db.Get(ctx, "key") + require.NoError(t, err) + require.Equal(t, "sst", string(kv.ValueBytes())) +} + +// TestAddSSTableWriteAtRequestTimestampRespectsClosedTS checks that AddSSTable +// with WriteAtRequestTimestamp respects the closed timestamp. +func TestAddSSTableWriteAtRequestTimestampRespectsClosedTS(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + si, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{}, + }) + defer si.Stopper().Stop(ctx) + s := si.(*server.TestServer) + + // Issue a write to trigger a closed timestamp. + require.NoError(t, db.Put(ctx, "someKey", "someValue")) + + // Get the closed timestamp for the range owning "key". + rd, err := s.LookupRange(roachpb.Key("key")) + require.NoError(t, err) + r, store, err := s.GetStores().(*kvserver.Stores).GetReplicaForRangeID(ctx, rd.RangeID) + require.NoError(t, err) + closedTS := r.GetClosedTimestamp(ctx) + require.NotZero(t, closedTS) + + // Add an SST writing below the closed timestamp. It should get pushed above it. + reqTS := closedTS.Prev() + sst, start, end := makeSST(t, []mvccKV{{"key", 1, "sst"}}) + sstReq := &roachpb.AddSSTableRequest{ + RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Data: sst, + MVCCStats: sstStats(t, sst), + WriteAtRequestTimestamp: true, + } + ba := roachpb.BatchRequest{ + Header: roachpb.Header{Timestamp: reqTS}, + } + ba.Add(sstReq) + result, pErr := db.NonTransactionalSender().Send(ctx, ba) + require.Nil(t, pErr) + writeTS := result.Timestamp + require.True(t, reqTS.Less(writeTS), "timestamp did not get pushed") + require.True(t, closedTS.LessEq(writeTS), "timestamp %s below closed timestamp %s", result.Timestamp, closedTS) + + // Check that the value was in fact written at the write timestamp. + kvs, err := storage.Scan(store.Engine(), roachpb.Key("key"), roachpb.Key("key").Next(), 0) + require.NoError(t, err) + require.Len(t, kvs, 1) + require.Equal(t, storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}, kvs[0].Key) + v, err := roachpb.Value{RawBytes: kvs[0].Value}.GetBytes() + require.NoError(t, err) + require.Equal(t, "sst", string(v)) +} + +// makeSST builds a binary in-memory SST from the given data. +func makeSST(t *testing.T, kvs []mvccKV) ([]byte, roachpb.Key, roachpb.Key) { t.Helper() sstFile := &storage.MemFile{} writer := storage.MakeBackupSSTWriter(sstFile) defer writer.Close() - keys := make([]string, 0, len(kvs)) - for key := range kvs { - keys = append(keys, key) - } - sort.Strings(keys) - - for _, k := range keys { - key := storage.MVCCKey{Key: roachpb.Key(k), Timestamp: ts} - value := roachpb.Value{} - value.SetString(kvs[k]) - value.InitChecksum(key.Key) - require.NoError(t, writer.Put(key, value.RawBytes)) + start, end := keys.MaxKey, keys.MinKey + for _, kv := range kvs { + if kv.key < string(start) { + start = roachpb.Key(kv.key) + } + if kv.key > string(end) { + end = roachpb.Key(kv.key) + } + if kv.ts == 0 { + meta := &enginepb.MVCCMetadata{RawBytes: kv.ValueBytes()} + metaBytes, err := protoutil.Marshal(meta) + require.NoError(t, err) + require.NoError(t, writer.PutUnversioned(kv.Key(), metaBytes)) + } else { + require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + } } require.NoError(t, writer.Finish()) writer.Close() - return sstFile.Data() + + return sstFile.Data(), start, end.Next() } +// sstStats computes the MVCC stats for the given binary SST. func sstStats(t *testing.T, sst []byte) *enginepb.MVCCStats { t.Helper() @@ -1156,3 +1330,19 @@ func sstStats(t *testing.T, sst []byte) *enginepb.MVCCStats { require.NoError(t, err) return &stats } + +// engineStats computes the MVCC stats for the given engine. +func engineStats(t *testing.T, engine storage.Engine) *enginepb.MVCCStats { + t.Helper() + + iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + LowerBound: keys.LocalMax, + UpperBound: keys.MaxKey, + }) + defer iter.Close() + // We don't care about nowNanos, because the SST can't contain intents or + // tombstones and all existing intents will be resolved. + stats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, keys.MaxKey, 0) + require.NoError(t, err) + return &stats +} diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 58ed3bf61a09..7b02d74c0dd8 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -213,13 +213,6 @@ func (i *MVCCIterator) FindSplitKey( return i.i.FindSplitKey(start, end, minSplitKey, targetSize) } -// CheckForKeyCollisions is part of the storage.MVCCIterator interface. -func (i *MVCCIterator) CheckForKeyCollisions( - sstData []byte, start, end roachpb.Key, maxIntents int64, -) (enginepb.MVCCStats, error) { - return i.i.CheckForKeyCollisions(sstData, start, end, maxIntents) -} - // SetUpperBound is part of the storage.MVCCIterator interface. func (i *MVCCIterator) SetUpperBound(key roachpb.Key) { i.i.SetUpperBound(key) diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 8a0d7f601776..d05192676493 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1339,8 +1339,12 @@ func (*CheckConsistencyRequest) flags() flag { return isAdmin | isR func (*ExportRequest) flags() flag { return isRead | isRange | updatesTSCache } func (*AdminScatterRequest) flags() flag { return isAdmin | isRange | isAlone } func (*AdminVerifyProtectedTimestampRequest) flags() flag { return isAdmin | isRange | isAlone } -func (*AddSSTableRequest) flags() flag { - return isWrite | isRange | isAlone | isUnsplittable | canBackpressure +func (r *AddSSTableRequest) flags() flag { + flags := isWrite | isRange | isAlone | isUnsplittable | canBackpressure + if r.WriteAtRequestTimestamp { + flags |= appliesTSCache + } + return flags } func (*MigrateRequest) flags() flag { return isWrite | isRange | isAlone } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 868d46d0a667..6bd06c5b12fb 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -4411,15 +4411,108 @@ func (m *AdminVerifyProtectedTimestampResponse_FailedRange) XXX_DiscardUnknown() var xxx_messageInfo_AdminVerifyProtectedTimestampResponse_FailedRange proto.InternalMessageInfo -// AddSSTableRequest is arguments to the AddSSTable() method, to link a file -// into the RocksDB log-structured merge-tree. +// AddSSTableRequest contains arguments to the AddSSTable method, which links an +// SST file into the Pebble log-structured merge-tree. The SST should only +// contain committed versioned values with non-zero MVCC timestamps (no intents +// or inline values) and no tombstones, but this is only fully enforced when +// WriteAtRequestTimestamp is enabled, for performance. It cannot be used in a +// transaction, cannot be split across ranges, and must be alone in a batch. +// +// By default, AddSSTable will blindly write the SST contents into Pebble, with +// fixed MVCC timestamps unaffected by pushes. This can violate many CRDB +// guarantees, including ACID, serializability and single-key linearizability: +// it mutates MVCC history (by replacing existing versions or writing below +// their timestamp) and does not respect the timestamp cache (by writing at +// timestamps that have already been read) nor the closed timestamp (by writing +// at immutable timestamps). +// +// The following parameters can be used to make AddSSTable enforce these +// guarantees, at a performance cost: +// +// * WriteAtRequestTimestamp: ensures compliance with the timestamp cache and +// closed timestamp, by rewriting SST timestamps to the request timestamp. +// +// * DisallowConflicts, DisallowShadowing, or DisallowShadowingBelow: ensures +// compliance with MVCC, by checking for conflicting keys in existing data +// instead of writing blindly. +// +// If the above parameters are not enabled, the caller must make sure these +// guarantees are upheld via other mechanisms. These options are orthogonal, +// providing different guarantees, and neither is sufficient by itself to +// enforce ACID guarantees -- they must both be enabled. See comments on these +// parameters for more details. +// +// AddSSTable always synchronizes with ongoing transactions, by taking out a +// lock span, scanning for separated intents, and resolving them. This is done +// even in the case of blind writes, since the caller is expected to make sure +// there are no ongoing writes to the ingested key span, so there should be few +// or no intents in the common case. +// +// If writing blindly (without e.g. DisallowConflicts), the range's MVCC stats +// may be incorrect as the SST stats are not adjusted for existing keys, so they +// will be marked with ContainsEstimates. The caller should recompute statistics +// after ingestion. type AddSSTableRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - // If set, indicates that AddSSTable will not allow ingestion of keys which - // shadow already existing key entries. This disallows any key slice overlap - // regardless of the timestamps. + // WriteAtRequestTimestamp updates all MVCC timestamps in the SST to the + // request timestamp, even if the request gets pushed. This ensures the writes + // comply with the timestamp cache and closed timestamp. + // + // Callers should always set this, except in very special circumstances when + // the timestamp cache and closed timestamp can safely be ignored (e.g. + // streaming replication into an offline tenant). + // + // Note that this alone is not sufficient to guarantee MVCC correctness, since + // it can write below or replace existing committed versions (the tscache is + // only bumped when the values are subsequently read). Use DisallowConflicts + // or DisallowShadowing in addition to guarantee MVCC correctness and + // serializability. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + WriteAtRequestTimestamp bool `protobuf:"varint,6,opt,name=write_at_request_timestamp,json=writeAtRequestTimestamp,proto3" json:"write_at_request_timestamp,omitempty"` + // DisallowConflicts will check for MVCC conflicts with existing keys, i.e. + // scan for existing keys with a timestamp at or above the SST key and + // return WriteTooOldError (possibly retrying). It also ensures MVCC + // statistics are accurately updated. + // + // Note that this alone is not sufficient to guarantee serializability or + // single-key linearizability, since it can write to a timestamp that another + // reader has already observed, changing the value at that timestamp and above + // it. Use WriteAtRequestTimestamp in addition to guarantee serializability. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + // + // TODO(erikgrinaker): It might be possible to avoid this parameter if we + // could pick an MVCC timestamp that's guaranteed to not collide with + // existing keys, see: https://github.com/cockroachdb/cockroach/issues/73047. + // However, this would always lead to inaccurate MVCC stats. + DisallowConflicts bool `protobuf:"varint,7,opt,name=disallow_conflicts,json=disallowConflicts,proto3" json:"disallow_conflicts,omitempty"` + // DisallowShadowing implies DisallowConflicts, and additionally rejects + // writing above keys that have an existing/visible value (but will write + // above tombstones). + // + // TODO(erikgrinaker): Consider removing this in 22.1 if all callers have + // been migrated to DisallowShadowingBelow. DisallowShadowing bool `protobuf:"varint,3,opt,name=disallow_shadowing,json=disallowShadowing,proto3" json:"disallow_shadowing,omitempty"` + // DisallowShadowingBelow implies DisallowConflicts, and additionally rejects + // writing above keys that have an existing/visible value (but will write + // above tombstones). Unlike DisallowShadowing, it allows shadowing keys + // that have a timestamp at or above the given timestamp as long as the + // value is identical to the existing value, and also allows idempotent writes + // (same key/timestamp/value) at or above the given timestamp. + // + // This is a specialized method for the IMPORT INTO use-case, where we do not + // want to shadow existing keys (which could cause them to be GCed before the + // import finalizes, preventing a rollback), but we need to allow shadowing + // keys that were previously written by the import itself in the case of a + // resumption or retry. The equal value requirement is to avoid unique + // constraint violations. + // + // This parameter cannot be used together with DisallowShadowing. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + DisallowShadowingBelow hlc.Timestamp `protobuf:"bytes,8,opt,name=disallow_shadowing_below,json=disallowShadowingBelow,proto3" json:"disallow_shadowing_below"` // MVCCStats, if set, is the MVCCStats for the contents of this SSTable and is // used as-is during evaluation of the AddSSTable command to update the range // MVCCStats, instead of computing the stats for the SSTable by iterating it. @@ -7954,597 +8047,601 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 9437 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x24, 0x4b, 0x57, 0xfd, 0xa3, 0x56, 0xcf, 0x48, 0xdd, - 0xd5, 0xd3, 0xbf, 0x9e, 0xa1, 0xa6, 0xbb, 0x77, 0x33, 0xb3, 0x33, 0xb3, 0xb3, 0x16, 0x29, 0x76, - 0x93, 0xfa, 0xef, 0x22, 0xd5, 0x8d, 0x19, 0xaf, 0x53, 0x2e, 0x55, 0x5d, 0x49, 0xb5, 0x22, 0xab, - 0xd8, 0x55, 0x45, 0xb5, 0x38, 0x40, 0x1e, 0x92, 0x38, 0x9b, 0xcd, 0x4b, 0xb0, 0x01, 0x02, 0x78, - 0x17, 0x0e, 0x8c, 0x75, 0x6c, 0x24, 0x0f, 0x09, 0x90, 0x20, 0x09, 0x12, 0x38, 0x48, 0xbc, 0x8f, - 0x59, 0x04, 0x4e, 0xb2, 0xfb, 0x66, 0x04, 0x88, 0xe2, 0x68, 0xf3, 0xe0, 0x45, 0x10, 0x24, 0x48, - 0x02, 0x18, 0x18, 0x20, 0x3f, 0xb8, 0x3f, 0xf5, 0x43, 0xb2, 0x48, 0x51, 0x3d, 0x35, 0xf1, 0x00, - 0x7e, 0x91, 0x58, 0xe7, 0xde, 0x73, 0xea, 0xde, 0x73, 0xff, 0xce, 0x77, 0xef, 0x39, 0xb7, 0x60, - 0xd6, 0xb6, 0x54, 0xed, 0xb0, 0xbd, 0xb7, 0xac, 0xb6, 0x8d, 0x62, 0xdb, 0xb6, 0x5c, 0x0b, 0xcd, - 0x6a, 0x96, 0x76, 0x44, 0xc9, 0x45, 0x9e, 0xb8, 0x70, 0x19, 0xdb, 0xb6, 0x65, 0x3b, 0xed, 0xbd, - 0x65, 0xf6, 0x83, 0xe5, 0x5c, 0x78, 0x70, 0x74, 0xbc, 0x7c, 0x74, 0xec, 0x60, 0xfb, 0x18, 0xdb, - 0xcb, 0x9a, 0x65, 0x6a, 0x1d, 0xdb, 0xc6, 0xa6, 0xd6, 0x5d, 0x6e, 0x5a, 0xda, 0x11, 0xfd, 0x63, - 0x98, 0x07, 0x51, 0x79, 0x6d, 0xac, 0xea, 0x4e, 0xa7, 0xd5, 0x52, 0xed, 0xee, 0x32, 0x15, 0xcb, - 0x1f, 0x78, 0x5e, 0xe4, 0x15, 0x4a, 0x57, 0x5d, 0x95, 0xd3, 0x2e, 0x79, 0xb4, 0x9e, 0x12, 0x5c, - 0xf1, 0xa8, 0x2d, 0xec, 0xaa, 0xa1, 0xdc, 0xd7, 0x3c, 0xba, 0xd3, 0x56, 0x4d, 0x45, 0xb3, 0xcc, - 0x7d, 0xc3, 0x2b, 0xc8, 0x75, 0xc7, 0xb5, 0x6c, 0xf5, 0x00, 0x2f, 0x63, 0xf3, 0xc0, 0x30, 0x31, - 0xe1, 0x3d, 0xd6, 0x34, 0x9e, 0xf8, 0x46, 0x64, 0xe2, 0x63, 0x9e, 0x3a, 0xdf, 0x71, 0x8d, 0xe6, - 0xf2, 0x61, 0x53, 0x5b, 0x76, 0x8d, 0x16, 0x76, 0x5c, 0xb5, 0xd5, 0xf6, 0x6a, 0x47, 0x53, 0x5c, - 0x5b, 0xd5, 0x0c, 0xf3, 0xc0, 0xfb, 0xdf, 0xde, 0x5b, 0xb6, 0xb1, 0x66, 0xd9, 0x3a, 0xd6, 0x15, - 0x52, 0x0e, 0xaf, 0x26, 0x07, 0xd6, 0x81, 0x45, 0x7f, 0x2e, 0x93, 0x5f, 0x9c, 0xba, 0x78, 0x60, - 0x59, 0x07, 0x4d, 0xbc, 0x4c, 0x9f, 0xf6, 0x3a, 0xfb, 0xcb, 0x7a, 0xc7, 0x56, 0x5d, 0xc3, 0xe2, - 0x5c, 0xd2, 0x3f, 0x15, 0x20, 0x27, 0xe3, 0x97, 0x1d, 0xec, 0xb8, 0x55, 0xac, 0xea, 0xd8, 0x46, - 0xd7, 0x20, 0x79, 0x84, 0xbb, 0xf3, 0xc9, 0x1b, 0xc2, 0xbd, 0x99, 0xd2, 0xf4, 0xe7, 0xa7, 0x4b, - 0xc9, 0x75, 0xdc, 0x95, 0x09, 0x0d, 0xdd, 0x80, 0x69, 0x6c, 0xea, 0x0a, 0x49, 0x4e, 0xf5, 0x26, - 0x4f, 0x61, 0x53, 0x5f, 0xc7, 0x5d, 0xf4, 0x6d, 0x48, 0x3b, 0x44, 0x9a, 0xa9, 0xe1, 0xf9, 0xc9, - 0x1b, 0xc2, 0xbd, 0xc9, 0xd2, 0x2f, 0x7f, 0x7e, 0xba, 0xf4, 0xd1, 0x81, 0xe1, 0x1e, 0x76, 0xf6, - 0x8a, 0x9a, 0xd5, 0x5a, 0xf6, 0x7b, 0x81, 0xbe, 0x17, 0xfc, 0x5e, 0x6e, 0x1f, 0x1d, 0x2c, 0xf7, - 0xeb, 0xa8, 0xd8, 0x38, 0x31, 0xeb, 0xf8, 0xa5, 0xec, 0x4b, 0x5c, 0x4b, 0xa5, 0x05, 0x31, 0xb1, - 0x96, 0x4a, 0x27, 0xc4, 0xa4, 0xf4, 0xf7, 0x13, 0x90, 0x97, 0xb1, 0xd3, 0xb6, 0x4c, 0x07, 0xf3, - 0x92, 0xbf, 0x0b, 0x49, 0xf7, 0xc4, 0xa4, 0x25, 0xcf, 0x3e, 0x5a, 0x2c, 0x0e, 0xf4, 0xb7, 0x62, - 0xc3, 0x56, 0x4d, 0x47, 0xd5, 0x48, 0xf5, 0x65, 0x92, 0x15, 0xbd, 0x0f, 0x59, 0x1b, 0x3b, 0x9d, - 0x16, 0xa6, 0x8a, 0xa4, 0x95, 0xca, 0x3e, 0xba, 0x1a, 0xc1, 0x59, 0x6f, 0xab, 0xa6, 0x0c, 0x2c, - 0x2f, 0xf9, 0x8d, 0x56, 0x21, 0xc7, 0x39, 0x6d, 0xac, 0x3a, 0x96, 0x39, 0x3f, 0x7d, 0x43, 0xb8, - 0x97, 0x7f, 0xb4, 0x14, 0xc1, 0x2b, 0xd3, 0x7c, 0x32, 0xcd, 0x26, 0xcf, 0xd8, 0xa1, 0x27, 0xf4, - 0x00, 0x66, 0xb9, 0x14, 0x13, 0x9f, 0xb8, 0xca, 0x5e, 0xd7, 0xc5, 0xce, 0x7c, 0xe6, 0x86, 0x70, - 0x2f, 0x29, 0x17, 0x58, 0xc2, 0x16, 0x3e, 0x71, 0x4b, 0x84, 0x8c, 0xae, 0x41, 0xda, 0xec, 0xb4, - 0x88, 0xf2, 0x1d, 0xaa, 0xda, 0xa4, 0x3c, 0x6d, 0x76, 0x5a, 0xeb, 0xb8, 0xeb, 0xa0, 0xeb, 0x90, - 0x21, 0x49, 0x8c, 0x3d, 0x4d, 0xd3, 0x48, 0x5e, 0xca, 0xb7, 0x96, 0x4a, 0x4f, 0x89, 0xd3, 0xd2, - 0xef, 0x08, 0x00, 0x4f, 0xb1, 0xcb, 0x9b, 0x1a, 0x95, 0x60, 0xea, 0x90, 0x2a, 0x6d, 0x5e, 0xa0, - 0x75, 0xbe, 0x11, 0x59, 0xee, 0x50, 0xb7, 0x28, 0xa5, 0x7f, 0x72, 0xba, 0x34, 0xf1, 0xd3, 0xd3, - 0x25, 0x41, 0xe6, 0x9c, 0xe8, 0x19, 0x64, 0x8f, 0x70, 0x57, 0xe1, 0xe3, 0x71, 0x3e, 0x41, 0x15, - 0xf0, 0x6e, 0x48, 0xd0, 0xd1, 0x71, 0xd1, 0x1b, 0x9a, 0xc5, 0xd0, 0x30, 0x2e, 0x12, 0x8e, 0x62, - 0xdd, 0xb5, 0xb1, 0x79, 0xe0, 0x1e, 0xca, 0x70, 0x84, 0xbb, 0x1b, 0x4c, 0x86, 0xf4, 0x63, 0x01, - 0xb2, 0xb4, 0x94, 0xac, 0x5d, 0x51, 0xb9, 0xaf, 0x98, 0x37, 0xa3, 0xd5, 0x1b, 0xea, 0x04, 0x11, - 0xe5, 0x2c, 0xc2, 0xe4, 0xb1, 0xda, 0xec, 0x60, 0x5a, 0xc2, 0xec, 0xa3, 0xf9, 0x08, 0x19, 0xcf, - 0x49, 0xba, 0xcc, 0xb2, 0xa1, 0x0f, 0x61, 0xc6, 0x30, 0x5d, 0x6c, 0xba, 0x0a, 0x63, 0x4b, 0x9e, - 0xc3, 0x96, 0x65, 0xb9, 0xe9, 0x83, 0xf4, 0x4f, 0x04, 0x80, 0x9d, 0x4e, 0xac, 0x7a, 0xfe, 0xda, - 0x98, 0xe5, 0x2f, 0xa5, 0x08, 0xab, 0x57, 0x8b, 0x2b, 0x30, 0x65, 0x98, 0x4d, 0xc3, 0x64, 0xe5, - 0x4f, 0xcb, 0xfc, 0x09, 0x5d, 0x82, 0xc9, 0xbd, 0xa6, 0x61, 0xea, 0xb4, 0xb3, 0xa7, 0x65, 0xf6, - 0x20, 0xc9, 0x90, 0xa5, 0xa5, 0x8e, 0x51, 0xef, 0xd2, 0x69, 0x02, 0x2e, 0x97, 0x2d, 0x53, 0x37, - 0xc8, 0x78, 0x53, 0x9b, 0x5f, 0x09, 0xad, 0xac, 0xc1, 0x25, 0x1d, 0xb7, 0x6d, 0xac, 0xa9, 0x2e, - 0xd6, 0x15, 0x7c, 0xd2, 0x1e, 0xb3, 0x8d, 0x51, 0xc0, 0x55, 0x39, 0x69, 0x53, 0x1a, 0x19, 0x75, - 0x44, 0x00, 0x1b, 0x75, 0x53, 0x64, 0x3e, 0x94, 0xd3, 0xf8, 0xa4, 0xcd, 0x46, 0x6b, 0xa4, 0x9a, - 0xd1, 0xd7, 0xe0, 0xaa, 0xda, 0x6c, 0x5a, 0xaf, 0x14, 0x63, 0x5f, 0xd1, 0x2d, 0xec, 0x28, 0xa6, - 0xe5, 0x2a, 0xf8, 0xc4, 0x70, 0x5c, 0x3a, 0xa4, 0xd3, 0xf2, 0x1c, 0x4d, 0xae, 0xed, 0xaf, 0x5a, - 0xd8, 0xd9, 0xb2, 0xdc, 0x0a, 0x49, 0x0a, 0x35, 0xe5, 0x74, 0xb8, 0x29, 0xa5, 0x5f, 0x85, 0x2b, - 0xfd, 0xfa, 0x8d, 0xb3, 0xfd, 0xfe, 0x40, 0x80, 0x7c, 0xcd, 0x34, 0xdc, 0xaf, 0x44, 0xc3, 0xf9, - 0xfa, 0x4c, 0x86, 0xf5, 0xf9, 0x00, 0xc4, 0x7d, 0xd5, 0x68, 0x6e, 0x9b, 0x0d, 0xab, 0xb5, 0xe7, - 0xb8, 0x96, 0x89, 0x1d, 0xae, 0xf0, 0x01, 0xba, 0xf4, 0x1c, 0x0a, 0x7e, 0x6d, 0xe2, 0x54, 0x93, - 0x0b, 0x62, 0xcd, 0xd4, 0x6c, 0xdc, 0xc2, 0x66, 0xac, 0x7a, 0x7a, 0x03, 0x32, 0x86, 0x27, 0x97, - 0xea, 0x2a, 0x29, 0x07, 0x04, 0xa9, 0x03, 0xb3, 0xa1, 0xb7, 0xc6, 0x39, 0x5d, 0x92, 0xc5, 0x04, - 0xbf, 0x52, 0x82, 0x36, 0x22, 0x8b, 0x09, 0x7e, 0xc5, 0xa6, 0xb7, 0x3a, 0xe4, 0x56, 0x71, 0x13, - 0xbb, 0x38, 0xc6, 0x9a, 0x4a, 0xbb, 0x90, 0xf7, 0x84, 0xc6, 0xd9, 0x30, 0xbf, 0x21, 0x00, 0xe2, - 0x72, 0x55, 0xf3, 0x20, 0xce, 0x12, 0xa3, 0x25, 0x62, 0x37, 0xb8, 0x1d, 0xdb, 0x64, 0xcb, 0x31, - 0xeb, 0x93, 0xc0, 0x48, 0x74, 0x45, 0x0e, 0x86, 0x6c, 0x2a, 0x3c, 0x64, 0xb9, 0xed, 0xf2, 0x0a, - 0xe6, 0x7a, 0x0a, 0x16, 0x6f, 0xf3, 0xa5, 0x68, 0x99, 0x12, 0x37, 0x92, 0x61, 0x03, 0x8d, 0x12, - 0xa5, 0x1f, 0x08, 0x30, 0x5b, 0x6e, 0x62, 0xd5, 0x8e, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x8e, 0x55, - 0x9d, 0x56, 0x99, 0x0d, 0xec, 0x37, 0x43, 0x52, 0x88, 0x19, 0x5b, 0x3c, 0x6c, 0x6a, 0xc5, 0x86, - 0x67, 0xe0, 0xf2, 0xd1, 0xed, 0x33, 0x49, 0x9f, 0x00, 0x0a, 0x97, 0x2c, 0xce, 0x8e, 0xf0, 0xbb, - 0x09, 0x40, 0x32, 0x3e, 0xc6, 0xb6, 0x1b, 0x7b, 0xb5, 0x57, 0x21, 0xeb, 0xaa, 0xf6, 0x01, 0x76, - 0x15, 0x62, 0xba, 0x5f, 0xa4, 0xe6, 0xc0, 0xf8, 0x08, 0x19, 0x35, 0xe0, 0x2e, 0x36, 0xd5, 0xbd, - 0x26, 0xa6, 0x52, 0x94, 0x3d, 0xab, 0x63, 0xea, 0x8a, 0xe1, 0x62, 0x5b, 0x75, 0x2d, 0x5b, 0xb1, - 0xda, 0xae, 0xd1, 0x32, 0x3e, 0xa3, 0x56, 0x3b, 0xef, 0x6a, 0xb7, 0x58, 0x76, 0xc2, 0x5c, 0x22, - 0x99, 0x6b, 0x3c, 0xef, 0x76, 0x28, 0x2b, 0x2a, 0xc2, 0x9c, 0x71, 0x60, 0x5a, 0x36, 0x56, 0x0e, - 0x34, 0xc5, 0x3d, 0xb4, 0xb1, 0x73, 0x68, 0x35, 0xbd, 0x05, 0x69, 0x96, 0x25, 0x3d, 0xd5, 0x1a, - 0x5e, 0x82, 0xf4, 0x29, 0xcc, 0xf5, 0x68, 0x29, 0xce, 0x26, 0xf8, 0xef, 0x02, 0x64, 0xeb, 0x9a, - 0x6a, 0xc6, 0xa9, 0xfb, 0x8f, 0x21, 0xeb, 0x68, 0xaa, 0xa9, 0xec, 0x5b, 0x76, 0x4b, 0x75, 0x69, - 0xbd, 0xf2, 0x3d, 0xba, 0xf7, 0x8d, 0x77, 0x4d, 0x35, 0x9f, 0xd0, 0x4c, 0x32, 0x38, 0xfe, 0xef, - 0x7e, 0xfb, 0x75, 0xf2, 0x8b, 0xdb, 0xaf, 0x6c, 0x78, 0xaf, 0xa5, 0xd2, 0x49, 0x31, 0x25, 0xfd, - 0x89, 0x00, 0x33, 0xac, 0xca, 0x71, 0x0e, 0xef, 0xaf, 0x43, 0xca, 0xb6, 0x5e, 0xb1, 0xe1, 0x9d, - 0x7d, 0x74, 0x3d, 0x42, 0xc4, 0x3a, 0xee, 0x86, 0xd7, 0x4f, 0x9a, 0x1d, 0x95, 0x80, 0x5b, 0xa9, - 0x0a, 0xe5, 0x4e, 0x8e, 0xcb, 0x0d, 0x8c, 0x4b, 0x26, 0x32, 0xee, 0x42, 0x61, 0x4f, 0x75, 0xb5, - 0x43, 0xc5, 0xe6, 0x85, 0x24, 0x6b, 0x6d, 0xf2, 0xde, 0x8c, 0x9c, 0xa7, 0x64, 0xaf, 0xe8, 0x0e, - 0xa9, 0x39, 0x1b, 0x6f, 0x0e, 0xfe, 0x33, 0xd6, 0xe6, 0xff, 0x47, 0xe0, 0x63, 0xc8, 0xab, 0xf9, - 0x9f, 0xb5, 0xa6, 0xff, 0x61, 0x02, 0xae, 0x96, 0x0f, 0xb1, 0x76, 0x54, 0xb6, 0x4c, 0xc7, 0x70, - 0x5c, 0xa2, 0xbb, 0x38, 0xdb, 0xff, 0x3a, 0x64, 0x5e, 0x19, 0xee, 0xa1, 0xa2, 0x1b, 0xfb, 0xfb, - 0x74, 0xb6, 0x4d, 0xcb, 0x69, 0x42, 0x58, 0x35, 0xf6, 0xf7, 0xd1, 0x63, 0x48, 0xb5, 0x2c, 0x9d, - 0x19, 0xf3, 0xd1, 0x50, 0x9c, 0x16, 0xcd, 0xe9, 0xb4, 0x36, 0x2d, 0x1d, 0xcb, 0x34, 0x33, 0x5a, - 0x04, 0xd0, 0x08, 0xb5, 0x6d, 0x19, 0xa6, 0xcb, 0x27, 0xc7, 0x10, 0x05, 0x55, 0x21, 0xe3, 0x62, - 0xbb, 0x65, 0x98, 0xaa, 0x8b, 0xe7, 0x27, 0xa9, 0xf2, 0xde, 0x8a, 0x2c, 0x78, 0xbb, 0x69, 0x68, - 0xea, 0x2a, 0x76, 0x34, 0xdb, 0x68, 0xbb, 0x96, 0xcd, 0xb5, 0x18, 0x30, 0x4b, 0x7f, 0x3d, 0x05, - 0xf3, 0x83, 0xba, 0x89, 0xb3, 0x87, 0xec, 0xc0, 0x94, 0x8d, 0x9d, 0x4e, 0xd3, 0xe5, 0x7d, 0xe4, - 0xd1, 0x30, 0x15, 0x44, 0x94, 0x80, 0x6e, 0x53, 0x34, 0x5d, 0x5e, 0x6c, 0x2e, 0x67, 0xe1, 0x5f, - 0x08, 0x30, 0xc5, 0x12, 0xd0, 0x43, 0x48, 0xdb, 0x64, 0x61, 0x50, 0x0c, 0x9d, 0x96, 0x31, 0x59, - 0xba, 0x72, 0x76, 0xba, 0x34, 0x4d, 0x17, 0x8b, 0xda, 0xea, 0xe7, 0xc1, 0x4f, 0x79, 0x9a, 0xe6, - 0xab, 0xe9, 0xa4, 0xb5, 0x1c, 0x57, 0xb5, 0x5d, 0xba, 0x63, 0x94, 0x60, 0x08, 0x89, 0x12, 0xd6, - 0x71, 0x17, 0xad, 0xc1, 0x94, 0xe3, 0xaa, 0x6e, 0xc7, 0xe1, 0xed, 0x75, 0xa1, 0xc2, 0xd6, 0x29, - 0xa7, 0xcc, 0x25, 0x10, 0x73, 0x4b, 0xc7, 0xae, 0x6a, 0x34, 0x69, 0x03, 0x66, 0x64, 0xfe, 0x24, - 0xfd, 0xa6, 0x00, 0x53, 0x2c, 0x2b, 0xba, 0x0a, 0x73, 0xf2, 0xca, 0xd6, 0xd3, 0x8a, 0x52, 0xdb, - 0x5a, 0xad, 0x34, 0x2a, 0xf2, 0x66, 0x6d, 0x6b, 0xa5, 0x51, 0x11, 0x27, 0xd0, 0x15, 0x40, 0x5e, - 0x42, 0x79, 0x7b, 0xab, 0x5e, 0xab, 0x37, 0x2a, 0x5b, 0x0d, 0x51, 0x40, 0x97, 0x40, 0x64, 0xf4, - 0x10, 0x35, 0x81, 0xde, 0x82, 0x1b, 0xfd, 0x54, 0xa5, 0xde, 0x58, 0x69, 0xd4, 0x95, 0x4a, 0xbd, - 0x51, 0xdb, 0x5c, 0x69, 0x54, 0x56, 0xc5, 0xe4, 0x88, 0x5c, 0xe4, 0x25, 0xb2, 0x5c, 0x29, 0x37, - 0xc4, 0x94, 0xe4, 0xc2, 0x65, 0x19, 0x6b, 0x56, 0xab, 0xdd, 0x71, 0x31, 0x29, 0xa5, 0x13, 0xe7, - 0x48, 0xb9, 0x0a, 0xd3, 0xba, 0xdd, 0x55, 0xec, 0x8e, 0xc9, 0xc7, 0xc9, 0x94, 0x6e, 0x77, 0xe5, - 0x8e, 0x29, 0xfd, 0x43, 0x01, 0xae, 0xf4, 0xbf, 0x36, 0xce, 0x4e, 0xf8, 0x0c, 0xb2, 0xaa, 0xae, - 0x63, 0x5d, 0xd1, 0x71, 0xd3, 0x55, 0xb9, 0x49, 0xf4, 0x20, 0x24, 0x89, 0xef, 0xf3, 0x15, 0xfd, - 0x7d, 0xbe, 0xcd, 0xe7, 0xe5, 0x32, 0x2d, 0xc8, 0x2a, 0xe1, 0xf0, 0xa6, 0x1f, 0x2a, 0x84, 0x52, - 0xa4, 0x1f, 0xa6, 0x20, 0x57, 0x31, 0xf5, 0xc6, 0x49, 0xac, 0x6b, 0xc9, 0x15, 0x98, 0xd2, 0xac, - 0x56, 0xcb, 0x70, 0x3d, 0x05, 0xb1, 0x27, 0xf4, 0x8d, 0x90, 0x29, 0x9b, 0x1c, 0xc3, 0xa0, 0x0b, - 0x8c, 0x58, 0xf4, 0x6b, 0x70, 0x95, 0xcc, 0x9a, 0xb6, 0xa9, 0x36, 0x15, 0x26, 0x4d, 0x71, 0x6d, - 0xe3, 0xe0, 0x00, 0xdb, 0x7c, 0x6f, 0xf1, 0x5e, 0x44, 0x39, 0x6b, 0x9c, 0xa3, 0x4c, 0x19, 0x1a, - 0x2c, 0xbf, 0x7c, 0xd9, 0x88, 0x22, 0xa3, 0x8f, 0x00, 0xc8, 0x52, 0x44, 0xf7, 0x2b, 0x1d, 0x3e, - 0x1f, 0x0d, 0xdb, 0xb0, 0xf4, 0xa6, 0x20, 0xc2, 0x40, 0x9e, 0x1d, 0xf4, 0x0c, 0x44, 0xc3, 0x54, - 0xf6, 0x9b, 0xc6, 0xc1, 0xa1, 0xab, 0xbc, 0xb2, 0x0d, 0x17, 0x3b, 0xf3, 0xb3, 0x54, 0x46, 0x54, - 0x53, 0xd7, 0xf9, 0xbe, 0xab, 0xfe, 0x82, 0xe4, 0xe4, 0xd2, 0xf2, 0x86, 0xf9, 0x84, 0xf2, 0x53, - 0xa2, 0x83, 0x96, 0x09, 0x14, 0x7a, 0xd9, 0x31, 0x6c, 0xac, 0x3c, 0x6c, 0x6b, 0x74, 0x1f, 0x24, - 0x5d, 0xca, 0x9f, 0x9d, 0x2e, 0x81, 0xcc, 0xc8, 0x0f, 0x77, 0xca, 0x04, 0x1a, 0xb1, 0xdf, 0x6d, - 0x8d, 0xa8, 0xbd, 0x6d, 0x19, 0x8e, 0x65, 0xd2, 0x8d, 0xce, 0xb4, 0xcc, 0x9f, 0xd0, 0x7d, 0x10, - 0xdd, 0x13, 0x53, 0x39, 0xc4, 0xaa, 0xed, 0xee, 0x61, 0xd5, 0x25, 0xeb, 0x33, 0xd0, 0x1c, 0x05, - 0xf7, 0xc4, 0xac, 0x86, 0xc8, 0x6b, 0xa9, 0xf4, 0xb4, 0x98, 0x5e, 0x4b, 0xa5, 0xd3, 0x62, 0x46, - 0xfa, 0x0f, 0x02, 0xe4, 0xbd, 0xbe, 0x11, 0x67, 0x37, 0xbe, 0x07, 0xa2, 0x65, 0x62, 0xa5, 0x7d, - 0xa8, 0x3a, 0x98, 0xb7, 0x25, 0x5f, 0x1d, 0xf2, 0x96, 0x89, 0x77, 0x08, 0x99, 0xb5, 0x0c, 0xda, - 0x81, 0x59, 0xc7, 0x55, 0x0f, 0x0c, 0xf3, 0x40, 0xf1, 0xf7, 0xef, 0xa9, 0x65, 0x31, 0x26, 0x12, - 0x10, 0x39, 0xb7, 0x4f, 0xef, 0x31, 0x29, 0x7e, 0x26, 0xc0, 0xec, 0x8a, 0xde, 0x32, 0xcc, 0x7a, - 0xbb, 0x69, 0xc4, 0xba, 0xc1, 0xf0, 0x16, 0x64, 0x1c, 0x22, 0x33, 0x98, 0x9d, 0x03, 0xb8, 0x98, - 0xa6, 0x29, 0x64, 0x9a, 0xde, 0x80, 0x02, 0x3e, 0x69, 0x1b, 0xec, 0xd0, 0x80, 0xa1, 0x9c, 0xd4, - 0xf8, 0x75, 0xcb, 0x07, 0xbc, 0x24, 0x89, 0xd7, 0xe9, 0x13, 0x40, 0xe1, 0x2a, 0xc5, 0x09, 0x34, - 0x3e, 0x81, 0x39, 0x2a, 0x7a, 0xd7, 0x74, 0x62, 0xd6, 0x97, 0xf4, 0x2b, 0x70, 0xa9, 0x57, 0x74, - 0x9c, 0xe5, 0x7e, 0xc1, 0x5b, 0x79, 0x13, 0xdb, 0xb1, 0x22, 0x54, 0x5f, 0xd7, 0x5c, 0x70, 0x9c, - 0x65, 0xfe, 0x75, 0x01, 0xae, 0x51, 0xd9, 0xf4, 0x5c, 0x65, 0x1f, 0xdb, 0x1b, 0x58, 0x75, 0x62, - 0x85, 0xd7, 0xb7, 0x60, 0x8a, 0xc1, 0x64, 0xda, 0x3f, 0x27, 0x4b, 0x59, 0x62, 0x66, 0xd4, 0x5d, - 0xcb, 0x26, 0x66, 0x06, 0x4f, 0x92, 0x54, 0x58, 0x88, 0x2a, 0x45, 0x9c, 0x35, 0xfd, 0x2d, 0x01, - 0x66, 0xb9, 0x85, 0x47, 0xba, 0x72, 0xf9, 0x90, 0x18, 0x38, 0xa8, 0x02, 0x59, 0x8d, 0xfe, 0x52, - 0xdc, 0x6e, 0x1b, 0x53, 0xf9, 0xf9, 0x51, 0xc6, 0x21, 0x63, 0x6b, 0x74, 0xdb, 0x98, 0x58, 0x98, - 0xde, 0x6f, 0xa2, 0xa8, 0x50, 0x25, 0x47, 0x9a, 0x97, 0x74, 0x1c, 0xd1, 0xbc, 0x9e, 0x9d, 0xc6, - 0x75, 0xf0, 0x8f, 0x93, 0x5c, 0x09, 0xec, 0x1d, 0x3c, 0x7b, 0xac, 0x06, 0xc5, 0xa7, 0x70, 0x25, - 0xb4, 0x75, 0x1e, 0xae, 0x78, 0xe2, 0x02, 0x15, 0x0f, 0x6d, 0xbf, 0x07, 0x54, 0xf4, 0x09, 0x84, - 0x36, 0xd8, 0x15, 0x56, 0x27, 0x0f, 0xaa, 0x5c, 0x44, 0x1d, 0xb3, 0x81, 0x14, 0x46, 0x77, 0x50, - 0x19, 0xd2, 0xf8, 0xa4, 0xad, 0xe8, 0xd8, 0xd1, 0xf8, 0xc4, 0x25, 0x45, 0x09, 0x24, 0x45, 0x19, - 0x30, 0xde, 0xa7, 0xf1, 0x49, 0x9b, 0x10, 0xd1, 0x2e, 0x59, 0x37, 0xbd, 0x75, 0x9d, 0x16, 0xdb, - 0x39, 0x1f, 0x0b, 0x04, 0x3d, 0x85, 0x8b, 0x2b, 0xf8, 0x4b, 0x3a, 0x13, 0x21, 0xfd, 0x48, 0x80, - 0xeb, 0x91, 0xad, 0x16, 0xe7, 0x42, 0xf6, 0x11, 0xa4, 0x68, 0xe5, 0x13, 0x17, 0xac, 0x3c, 0xe5, - 0x92, 0xbe, 0x97, 0xe0, 0x63, 0x5c, 0xc6, 0x4d, 0x8b, 0x28, 0x36, 0xf6, 0x2d, 0xb4, 0x6d, 0xc8, - 0x1d, 0x5b, 0x2e, 0xb6, 0xfd, 0x66, 0x4f, 0x5c, 0xb8, 0xd9, 0x67, 0xa8, 0x00, 0xaf, 0xc5, 0x9f, - 0xc3, 0xac, 0x69, 0x99, 0x4a, 0xaf, 0xd0, 0x8b, 0xf7, 0xa5, 0x82, 0x69, 0x99, 0xcf, 0x43, 0x72, - 0xfd, 0x79, 0xa6, 0x4f, 0x13, 0x71, 0xce, 0x33, 0xdf, 0x17, 0x60, 0xce, 0xb7, 0x74, 0x62, 0x36, - 0x77, 0xbf, 0x0e, 0x49, 0xd3, 0x7a, 0x75, 0x91, 0x2d, 0x4a, 0x92, 0x9f, 0xac, 0x7a, 0xbd, 0x25, - 0x8a, 0xb3, 0xbe, 0xff, 0x32, 0x01, 0x99, 0xa7, 0xe5, 0x38, 0x6b, 0xf9, 0x11, 0xdf, 0xfe, 0x66, - 0xed, 0x1d, 0xd5, 0xdb, 0xfd, 0xf7, 0x15, 0x9f, 0x96, 0xd7, 0x71, 0xd7, 0xeb, 0xed, 0x84, 0x0b, - 0xad, 0x40, 0xa6, 0x77, 0xa3, 0x74, 0x4c, 0x4d, 0x05, 0x5c, 0x0b, 0x18, 0x26, 0xa9, 0x5c, 0xcf, - 0x8f, 0x42, 0x88, 0xf0, 0xa3, 0x20, 0xaf, 0xf1, 0x2d, 0xc5, 0xc4, 0x45, 0x5e, 0x13, 0x32, 0x11, - 0x27, 0xc5, 0x29, 0xe9, 0x19, 0x00, 0xa9, 0x4e, 0x9c, 0x4d, 0xf2, 0xdd, 0x24, 0xe4, 0x77, 0x3a, - 0xce, 0x61, 0xcc, 0xbd, 0xaf, 0x0c, 0xd0, 0xee, 0x38, 0x87, 0x64, 0x44, 0x9e, 0x98, 0xbc, 0xce, - 0xe7, 0xb8, 0x68, 0x78, 0x95, 0x66, 0x7c, 0x8d, 0x13, 0x13, 0x55, 0xb9, 0x10, 0xac, 0x04, 0x7e, - 0x1e, 0xb7, 0x46, 0x21, 0xcb, 0xc6, 0x89, 0xb9, 0x89, 0x7d, 0x48, 0xc9, 0x24, 0x61, 0x22, 0xe9, - 0x23, 0x98, 0x26, 0x0f, 0x8a, 0x6b, 0x5d, 0xa4, 0x99, 0xa7, 0x08, 0x4f, 0xc3, 0x42, 0x1f, 0x42, - 0x86, 0x71, 0x93, 0xd5, 0x6f, 0x8a, 0xae, 0x7e, 0x51, 0x75, 0xe1, 0x6a, 0xa4, 0xeb, 0x5e, 0x9a, - 0xb2, 0x92, 0xb5, 0xee, 0x12, 0x4c, 0xee, 0x5b, 0xb6, 0xe6, 0x1d, 0xe6, 0xb2, 0x07, 0xd6, 0x9e, - 0x0c, 0xd2, 0xac, 0xa5, 0xd2, 0x19, 0x11, 0xa4, 0xdf, 0x14, 0xa0, 0xe0, 0x37, 0x44, 0x9c, 0x0b, - 0x42, 0xb9, 0x47, 0x8b, 0x17, 0x6f, 0x0a, 0xa2, 0x40, 0xe9, 0x5f, 0x51, 0x8b, 0x48, 0xb3, 0x8e, - 0x69, 0xcb, 0xc4, 0xd9, 0x53, 0x3e, 0x64, 0x5e, 0x3c, 0x89, 0x8b, 0xb6, 0x2e, 0x75, 0xe8, 0x79, - 0x08, 0x97, 0x8c, 0x16, 0x99, 0xcf, 0x0d, 0xb7, 0xd9, 0xe5, 0xb0, 0xcd, 0xc5, 0xde, 0xa9, 0xf1, - 0x5c, 0x90, 0x56, 0xf6, 0x92, 0xa4, 0xdf, 0xa5, 0xbb, 0xd5, 0x41, 0x4d, 0xe2, 0x54, 0x75, 0x0d, - 0x72, 0x36, 0x13, 0x4d, 0xcc, 0x9a, 0x0b, 0x6a, 0x7b, 0xc6, 0x67, 0x25, 0x0a, 0xff, 0xed, 0x04, - 0x14, 0x9e, 0x75, 0xb0, 0xdd, 0xfd, 0x2a, 0xa9, 0xfb, 0x0e, 0x14, 0x5e, 0xa9, 0x86, 0xab, 0xec, - 0x5b, 0xb6, 0xd2, 0x69, 0xeb, 0xaa, 0xeb, 0x79, 0x9b, 0xe4, 0x08, 0xf9, 0x89, 0x65, 0xef, 0x52, - 0x22, 0xc2, 0x80, 0x8e, 0x4c, 0xeb, 0x95, 0xa9, 0x10, 0x32, 0x05, 0xca, 0x27, 0x26, 0xdf, 0x42, - 0x2e, 0xbd, 0xf7, 0xef, 0x4f, 0x97, 0x1e, 0x8f, 0xe5, 0x20, 0x46, 0x9d, 0xe1, 0x3a, 0x1d, 0x43, - 0x2f, 0xee, 0xee, 0xd6, 0x56, 0x65, 0x91, 0x8a, 0x7c, 0xc1, 0x24, 0x36, 0x4e, 0x4c, 0x47, 0xfa, - 0x3b, 0x09, 0x10, 0x03, 0x1d, 0xc5, 0xd9, 0x90, 0x15, 0xc8, 0xbe, 0xec, 0x60, 0xdb, 0x78, 0x8d, - 0x66, 0x04, 0xce, 0x48, 0xa6, 0x9d, 0x07, 0x30, 0xeb, 0x9e, 0x98, 0x0a, 0x73, 0xdf, 0x63, 0x8e, - 0x1f, 0x9e, 0xc3, 0x42, 0xc1, 0x25, 0x65, 0x26, 0x74, 0xea, 0xf4, 0xe1, 0xa0, 0x4f, 0x61, 0xa6, - 0x47, 0x5b, 0xc9, 0x2f, 0xa6, 0xad, 0xec, 0xab, 0x90, 0xa2, 0x7e, 0x2c, 0x00, 0xa2, 0x8a, 0xaa, - 0xb1, 0x3d, 0xfe, 0xaf, 0x4a, 0x7f, 0xba, 0x07, 0x22, 0xf5, 0xc3, 0x54, 0x8c, 0x7d, 0xa5, 0x65, - 0x38, 0x8e, 0x61, 0x1e, 0xf0, 0x0e, 0x95, 0xa7, 0xf4, 0xda, 0xfe, 0x26, 0xa3, 0x4a, 0x7f, 0x01, - 0xe6, 0x7a, 0x2a, 0x10, 0x67, 0x63, 0xdf, 0x84, 0x99, 0x7d, 0x76, 0x04, 0x4b, 0x85, 0xf3, 0xed, - 0xc1, 0x2c, 0xa5, 0xb1, 0xf7, 0x49, 0xff, 0x25, 0x01, 0x97, 0x64, 0xec, 0x58, 0xcd, 0x63, 0x1c, - 0xbf, 0x0a, 0xab, 0xc0, 0xcf, 0x5e, 0x94, 0xd7, 0xd2, 0x64, 0x86, 0x31, 0xb3, 0x65, 0xae, 0x77, - 0x8f, 0xfd, 0xad, 0xd1, 0x3d, 0x76, 0x70, 0x57, 0x9d, 0xef, 0xd4, 0xa5, 0x7a, 0x76, 0xea, 0x2c, - 0x28, 0xb0, 0xd3, 0x63, 0x5d, 0x71, 0xf0, 0x4b, 0xb3, 0xd3, 0xf2, 0xc0, 0x50, 0x71, 0x54, 0x21, - 0x6b, 0x8c, 0xa5, 0x8e, 0x5f, 0x6e, 0x75, 0x5a, 0xd4, 0x76, 0x2e, 0x5d, 0x21, 0xe5, 0x3d, 0x3b, - 0x5d, 0xca, 0xf7, 0xa4, 0x39, 0x72, 0xde, 0xf0, 0x9f, 0x89, 0x74, 0xe9, 0xdb, 0x70, 0xb9, 0x4f, - 0xd9, 0x71, 0x5a, 0x3c, 0xff, 0x3c, 0x09, 0xd7, 0x7a, 0xc5, 0xc7, 0x0d, 0x71, 0xbe, 0xea, 0x0d, - 0x5a, 0x85, 0x5c, 0xcb, 0x30, 0x5f, 0x6f, 0xf7, 0x72, 0xa6, 0x65, 0x98, 0x3e, 0x2d, 0xaa, 0x6b, - 0x4c, 0x7d, 0xa9, 0x5d, 0x43, 0x85, 0x85, 0xa8, 0xb6, 0x8b, 0xb3, 0x7f, 0x7c, 0x4f, 0x80, 0x99, - 0xb8, 0xb7, 0xe5, 0x5e, 0xcf, 0x0b, 0x4e, 0x6a, 0x40, 0xee, 0x4b, 0xd8, 0xc7, 0xfb, 0x6d, 0x01, - 0x50, 0xc3, 0xee, 0x98, 0x04, 0xd4, 0x6e, 0x58, 0x07, 0x71, 0x56, 0xf3, 0x12, 0x4c, 0x1a, 0xa6, - 0x8e, 0x4f, 0x68, 0x35, 0x53, 0x32, 0x7b, 0xe8, 0x39, 0x4a, 0x4c, 0x8e, 0x75, 0x94, 0x28, 0x7d, - 0x0a, 0x73, 0x3d, 0x45, 0x8c, 0xb3, 0xfe, 0xbf, 0x48, 0xc0, 0x1c, 0xaf, 0x48, 0xec, 0x3b, 0x98, - 0x5f, 0x83, 0xc9, 0x26, 0x91, 0x39, 0xa2, 0x9d, 0xe9, 0x3b, 0xbd, 0x76, 0xa6, 0x99, 0xd1, 0x37, - 0x01, 0xda, 0x36, 0x3e, 0x56, 0x18, 0x6b, 0x72, 0x2c, 0xd6, 0x0c, 0xe1, 0xa0, 0x04, 0xf4, 0x03, - 0x01, 0x0a, 0x64, 0x40, 0xb7, 0x6d, 0xab, 0x6d, 0x39, 0xc4, 0x66, 0x71, 0xc6, 0x83, 0x39, 0xcf, - 0xce, 0x4e, 0x97, 0x72, 0x9b, 0x86, 0xb9, 0xc3, 0x19, 0x1b, 0xf5, 0xb1, 0xbd, 0xf7, 0xbd, 0x18, - 0x86, 0x62, 0xb9, 0x69, 0x69, 0x47, 0xc1, 0xe1, 0x18, 0x99, 0x59, 0x7c, 0x71, 0x8e, 0xf4, 0xef, - 0x04, 0xb8, 0xf4, 0xa5, 0x6d, 0x17, 0xff, 0x69, 0x28, 0x5b, 0x7a, 0x0e, 0x22, 0xfd, 0x51, 0x33, - 0xf7, 0xad, 0x38, 0x37, 0xee, 0xff, 0xb7, 0x00, 0xb3, 0x21, 0xc1, 0x71, 0x1a, 0x38, 0xaf, 0xab, - 0xa7, 0x1c, 0x73, 0x87, 0x71, 0xc7, 0x53, 0x95, 0x3c, 0xc3, 0xb3, 0xb3, 0x4e, 0x59, 0x84, 0x19, - 0x4c, 0x66, 0x31, 0xba, 0xc5, 0xbb, 0xc7, 0x22, 0x48, 0xfa, 0x76, 0xf4, 0xb3, 0x7e, 0x86, 0x52, - 0x57, 0xfa, 0x15, 0x62, 0x61, 0x85, 0x07, 0x65, 0x9c, 0x43, 0xfe, 0x9f, 0x25, 0xe0, 0x4a, 0x99, - 0x1d, 0x81, 0x7b, 0x3e, 0x21, 0x71, 0x76, 0xc4, 0x79, 0x98, 0x3e, 0xc6, 0xb6, 0x63, 0x58, 0x6c, - 0xb5, 0xcf, 0xc9, 0xde, 0x23, 0x5a, 0x80, 0xb4, 0x63, 0xaa, 0x6d, 0xe7, 0xd0, 0xf2, 0x8e, 0x13, - 0xfd, 0x67, 0xdf, 0x7f, 0x65, 0xf2, 0xf5, 0xfd, 0x57, 0xa6, 0x46, 0xfb, 0xaf, 0x4c, 0x7f, 0x01, - 0xff, 0x15, 0x7e, 0x76, 0xf7, 0xaf, 0x05, 0xb8, 0x3a, 0xa0, 0xb9, 0x38, 0x3b, 0xe7, 0x77, 0x20, - 0xab, 0x71, 0xc1, 0x64, 0x7d, 0x60, 0x07, 0x93, 0x35, 0x92, 0xed, 0x35, 0xa1, 0xcf, 0xd9, 0xe9, - 0x12, 0x78, 0x45, 0xad, 0xad, 0x72, 0xe5, 0x90, 0xdf, 0xba, 0xf4, 0x5f, 0x01, 0x0a, 0x95, 0x13, - 0xb6, 0x29, 0x5f, 0x67, 0x56, 0x09, 0x7a, 0x02, 0xe9, 0xb6, 0x6d, 0x1d, 0x1b, 0x5e, 0x35, 0xf2, - 0x3d, 0xce, 0x0b, 0x5e, 0x35, 0xfa, 0xb8, 0x76, 0x38, 0x87, 0xec, 0xf3, 0xa2, 0x06, 0x64, 0x36, - 0x2c, 0x4d, 0x6d, 0x3e, 0x31, 0x9a, 0xde, 0x40, 0x7b, 0xf7, 0x7c, 0x41, 0x45, 0x9f, 0x67, 0x47, - 0x75, 0x0f, 0xbd, 0x46, 0xf0, 0x89, 0xa8, 0x06, 0xe9, 0xaa, 0xeb, 0xb6, 0x49, 0x22, 0x1f, 0x7f, - 0x77, 0xc7, 0x10, 0x4a, 0x58, 0x3c, 0x8f, 0x5b, 0x8f, 0x1d, 0x35, 0x60, 0xf6, 0x29, 0x0d, 0x0e, - 0x2b, 0x37, 0xad, 0x8e, 0x5e, 0xa6, 0xc1, 0x6c, 0x7c, 0x99, 0xb8, 0x33, 0x86, 0xcc, 0xa7, 0xe5, - 0xba, 0x3c, 0x28, 0x00, 0xad, 0x40, 0xba, 0xfe, 0x98, 0x0b, 0x63, 0x66, 0xe4, 0xed, 0x31, 0x84, - 0xd5, 0x1f, 0xcb, 0x3e, 0x1b, 0x5a, 0x83, 0xec, 0xca, 0x67, 0x1d, 0x1b, 0x73, 0x29, 0x53, 0x43, - 0x3d, 0x27, 0xfa, 0xa5, 0x50, 0x2e, 0x39, 0xcc, 0x8c, 0xbe, 0x0d, 0x05, 0xa2, 0xb7, 0x86, 0xba, - 0xd7, 0xf4, 0xe4, 0xa5, 0xa9, 0xbc, 0xb7, 0xc7, 0x90, 0xe7, 0x73, 0x7a, 0x47, 0x02, 0x7d, 0xa2, - 0x16, 0x64, 0xc8, 0xf5, 0xb4, 0x17, 0x42, 0x90, 0x6a, 0x93, 0xa6, 0x11, 0xa8, 0x1b, 0x12, 0xfd, - 0x8d, 0xde, 0x81, 0x69, 0xd3, 0xd2, 0xb1, 0xd7, 0x99, 0x73, 0xa5, 0x4b, 0x67, 0xa7, 0x4b, 0x53, - 0x5b, 0x96, 0xce, 0x6c, 0x1d, 0xfe, 0x4b, 0x9e, 0x22, 0x99, 0x6a, 0xfa, 0xc2, 0x0d, 0x48, 0x91, - 0x26, 0x22, 0x73, 0xc8, 0x9e, 0xea, 0xe0, 0x5d, 0xdb, 0xe0, 0xd2, 0xbc, 0xc7, 0x85, 0x7f, 0x90, - 0x80, 0x44, 0xfd, 0x31, 0xb1, 0xe6, 0xf7, 0x3a, 0xda, 0x11, 0x76, 0x79, 0x3a, 0x7f, 0xa2, 0x56, - 0xbe, 0x8d, 0xf7, 0x0d, 0x66, 0x74, 0x65, 0x64, 0xfe, 0x84, 0xde, 0x04, 0x50, 0x35, 0x0d, 0x3b, - 0x8e, 0xe2, 0xc5, 0xf7, 0x65, 0xe4, 0x0c, 0xa3, 0xac, 0xe3, 0x2e, 0x61, 0x73, 0xb0, 0x66, 0x63, - 0xd7, 0xf3, 0xa1, 0x62, 0x4f, 0x84, 0xcd, 0xc5, 0xad, 0xb6, 0xe2, 0x5a, 0x47, 0xd8, 0xa4, 0x4d, - 0x9a, 0x21, 0xb3, 0x42, 0xab, 0xdd, 0x20, 0x04, 0x32, 0xa1, 0x61, 0x53, 0x0f, 0x66, 0x9f, 0x8c, - 0xec, 0x3f, 0x13, 0x91, 0x36, 0x3e, 0x30, 0x78, 0x74, 0x5c, 0x46, 0xe6, 0x4f, 0x44, 0x4b, 0x6a, - 0xc7, 0x3d, 0xa4, 0x2d, 0x91, 0x91, 0xe9, 0x6f, 0x74, 0x07, 0x0a, 0xcc, 0xed, 0x52, 0xc1, 0xa6, - 0xa6, 0xd0, 0x79, 0x30, 0x43, 0x93, 0x73, 0x8c, 0x5c, 0x31, 0x35, 0x32, 0xeb, 0xa1, 0xc7, 0xc0, - 0x09, 0xca, 0x51, 0xcb, 0x21, 0x3a, 0x05, 0x92, 0xab, 0x54, 0x38, 0x3b, 0x5d, 0xca, 0xd6, 0x69, - 0xc2, 0xfa, 0x66, 0x9d, 0xac, 0x25, 0x2c, 0xd7, 0x7a, 0xcb, 0xa9, 0xe9, 0x0b, 0x7f, 0x53, 0x80, - 0xe4, 0xd3, 0x72, 0xfd, 0xc2, 0x2a, 0xf3, 0x0a, 0x9a, 0x0c, 0x15, 0xf4, 0x2e, 0x14, 0xf6, 0x8c, - 0x66, 0xd3, 0x30, 0x0f, 0x88, 0x7d, 0xf5, 0x1d, 0xac, 0x79, 0x0a, 0xcb, 0x73, 0xf2, 0x0e, 0xa3, - 0xa2, 0x1b, 0x90, 0xd5, 0x6c, 0xac, 0x63, 0xd3, 0x35, 0xd4, 0xa6, 0xc3, 0x35, 0x17, 0x26, 0x2d, - 0xfc, 0x45, 0x01, 0x26, 0x69, 0x67, 0x45, 0x6f, 0x40, 0x46, 0xb3, 0x4c, 0x57, 0x35, 0x4c, 0x3e, - 0xeb, 0x64, 0xe4, 0x80, 0x30, 0xb4, 0x78, 0x37, 0x61, 0x46, 0xd5, 0x34, 0xab, 0x63, 0xba, 0x8a, - 0xa9, 0xb6, 0x30, 0x2f, 0x66, 0x96, 0xd3, 0xb6, 0xd4, 0x16, 0x46, 0x4b, 0xe0, 0x3d, 0xfa, 0x61, - 0x9b, 0x19, 0x19, 0x38, 0x69, 0x1d, 0x77, 0x17, 0x30, 0x64, 0xfc, 0x5e, 0x4d, 0xea, 0xdb, 0x71, - 0xfc, 0x12, 0xd0, 0xdf, 0xe8, 0x5d, 0xb8, 0xf4, 0xb2, 0xa3, 0x36, 0x8d, 0x7d, 0xba, 0xf9, 0x45, - 0xbd, 0xd4, 0xe9, 0xcb, 0x58, 0x51, 0x90, 0x9f, 0x46, 0x25, 0xd0, 0x77, 0x7a, 0x83, 0x20, 0x19, - 0x0c, 0x02, 0xe6, 0xb2, 0x23, 0x75, 0x61, 0x56, 0xc6, 0xae, 0xdd, 0x6d, 0xb0, 0x48, 0xd6, 0xca, - 0x31, 0x36, 0x5d, 0x52, 0x77, 0xab, 0x8d, 0x99, 0x93, 0x88, 0x57, 0x77, 0x9f, 0x80, 0x6e, 0x43, - 0x5e, 0x75, 0x49, 0x77, 0x73, 0x15, 0xb3, 0xd3, 0xda, 0xc3, 0x36, 0x73, 0x05, 0x90, 0x73, 0x9c, - 0xba, 0x45, 0x89, 0x3c, 0x22, 0xc3, 0xee, 0x2a, 0x74, 0x9f, 0x88, 0xbf, 0x1a, 0x28, 0xa9, 0x42, - 0x28, 0xd2, 0x7d, 0xb8, 0x4c, 0xea, 0x59, 0x31, 0x35, 0xbb, 0xdb, 0x26, 0x92, 0xb7, 0xe9, 0x5f, - 0x07, 0x89, 0xa1, 0x73, 0x1a, 0x7a, 0x3c, 0x23, 0xfd, 0x64, 0x1a, 0x72, 0x95, 0x93, 0xb6, 0x65, - 0xc7, 0xba, 0xab, 0x53, 0x82, 0x69, 0x0e, 0x7c, 0x47, 0x1c, 0xc5, 0xf6, 0xcd, 0x40, 0xde, 0x39, - 0x34, 0x67, 0x44, 0xcf, 0xfd, 0xa8, 0xd3, 0x23, 0xdc, 0x25, 0x56, 0xfd, 0xcc, 0x38, 0x56, 0xfd, - 0x1c, 0xc7, 0xd2, 0x59, 0x16, 0x89, 0xba, 0x8e, 0xbb, 0x8d, 0xba, 0xcc, 0x03, 0x5f, 0xc9, 0x83, - 0x83, 0xb6, 0x20, 0xdb, 0x3a, 0xd6, 0x34, 0x65, 0xdf, 0x68, 0xba, 0xdc, 0x57, 0x2d, 0xda, 0xad, - 0x7a, 0xf3, 0x79, 0xb9, 0xfc, 0x84, 0x66, 0x62, 0x3e, 0x5e, 0xc1, 0xb3, 0x0c, 0x44, 0x02, 0xfb, - 0x8d, 0x4a, 0x00, 0xcc, 0xf1, 0x93, 0xfa, 0x0b, 0x25, 0x2f, 0x70, 0xc2, 0x45, 0xd9, 0x68, 0x50, - 0x84, 0x04, 0x39, 0xe6, 0x9e, 0xd4, 0x32, 0x58, 0xc8, 0x71, 0x8e, 0x6d, 0xc3, 0x51, 0xe2, 0xa6, - 0x41, 0xc3, 0x8d, 0xdf, 0x06, 0x1e, 0x74, 0xa3, 0x38, 0x5e, 0x08, 0x5d, 0x29, 0x77, 0x76, 0xba, - 0x94, 0x91, 0x29, 0xb5, 0x5e, 0x6f, 0xc8, 0x19, 0x96, 0xa1, 0xee, 0xb8, 0x17, 0x09, 0xb3, 0x98, - 0x1e, 0x3f, 0xcc, 0xe2, 0xaf, 0x0a, 0x70, 0x85, 0xb7, 0x8f, 0xb2, 0x47, 0x5d, 0xcb, 0xd5, 0xa6, - 0xe1, 0x76, 0x95, 0xa3, 0xe3, 0xf9, 0x34, 0x35, 0xb7, 0xbe, 0x11, 0xd9, 0xce, 0xa1, 0xee, 0x55, - 0xf4, 0x5a, 0xbb, 0xbb, 0xc1, 0x99, 0xd7, 0x8f, 0x2b, 0xa6, 0x6b, 0x77, 0x4b, 0x57, 0xcf, 0x4e, - 0x97, 0xe6, 0x06, 0x53, 0x9f, 0xcb, 0x73, 0xce, 0x20, 0x0b, 0xaa, 0x02, 0x60, 0xbf, 0x7b, 0xd3, - 0xd9, 0x33, 0x7a, 0xd9, 0x8c, 0x1c, 0x07, 0x72, 0x88, 0x17, 0xdd, 0x03, 0x91, 0x87, 0xb5, 0xec, - 0x1b, 0x4d, 0xac, 0x38, 0xc6, 0x67, 0x98, 0xce, 0xb3, 0x49, 0x39, 0xcf, 0xe8, 0x44, 0x44, 0xdd, - 0xf8, 0x0c, 0xa3, 0x87, 0x70, 0x39, 0x68, 0x01, 0x65, 0x0f, 0x37, 0xad, 0x57, 0x2c, 0x7b, 0x96, - 0x66, 0x47, 0xbe, 0xf6, 0x4b, 0x24, 0x89, 0xb0, 0x2c, 0x7c, 0x07, 0xe6, 0x87, 0x55, 0x38, 0x3c, - 0x18, 0x33, 0xec, 0xac, 0xf4, 0xfd, 0xde, 0x8d, 0x92, 0x31, 0x06, 0x0d, 0xdf, 0x2c, 0xf9, 0x20, - 0xf1, 0xbe, 0x20, 0xfd, 0xbd, 0x04, 0xe4, 0x4a, 0x9d, 0xe6, 0xd1, 0x76, 0xbb, 0xce, 0xae, 0x02, - 0x40, 0xd7, 0x21, 0xa3, 0xab, 0xae, 0xca, 0x0a, 0x29, 0xb0, 0xf0, 0x36, 0x42, 0xa0, 0xb5, 0xb9, - 0x0b, 0x85, 0x90, 0x1f, 0x0a, 0xf7, 0xb6, 0xa7, 0xd5, 0x0e, 0xc8, 0xd4, 0x21, 0xfe, 0x7d, 0x98, - 0x0f, 0x65, 0xa4, 0xbb, 0x1a, 0x0a, 0x36, 0x5d, 0xdb, 0xc0, 0x6c, 0x67, 0x2e, 0x29, 0x87, 0x9c, - 0x65, 0x6a, 0x24, 0xb9, 0xc2, 0x52, 0x51, 0x03, 0x66, 0x48, 0xc6, 0xae, 0x42, 0x67, 0x60, 0x6f, - 0xe7, 0xf4, 0x61, 0x44, 0xb5, 0x7a, 0xca, 0x5d, 0xa4, 0xfa, 0x29, 0x53, 0x1e, 0xfa, 0x53, 0xce, - 0xe2, 0x80, 0xb2, 0xf0, 0x31, 0x88, 0xfd, 0x19, 0xc2, 0xba, 0x4c, 0x31, 0x5d, 0x5e, 0x0a, 0xeb, - 0x32, 0x19, 0xd2, 0xd3, 0x5a, 0x2a, 0x9d, 0x12, 0x27, 0xa5, 0xdf, 0x4a, 0x41, 0xde, 0xeb, 0x99, - 0x71, 0x9a, 0xf4, 0x25, 0x98, 0x24, 0xfd, 0xc8, 0x73, 0xed, 0xb8, 0x33, 0x62, 0x40, 0x70, 0xff, - 0x6e, 0xd2, 0xbf, 0x3c, 0xf4, 0x49, 0x59, 0xe3, 0x98, 0x52, 0x16, 0x7e, 0x3f, 0x01, 0x29, 0x6a, - 0x45, 0x3f, 0x84, 0x14, 0x0d, 0xf8, 0x17, 0x46, 0x06, 0xfc, 0x7b, 0xae, 0x01, 0x24, 0x2b, 0xda, - 0x24, 0x83, 0x4b, 0xf7, 0xe6, 0xdd, 0xcc, 0x38, 0xef, 0x17, 0xf9, 0xbc, 0x9b, 0xae, 0xd0, 0xcb, - 0x11, 0x1a, 0x75, 0x6a, 0x1a, 0xb1, 0x19, 0xd7, 0x5b, 0x23, 0x13, 0x21, 0x43, 0xb1, 0x44, 0x5d, - 0x95, 0x2c, 0xdb, 0xc5, 0x3a, 0x37, 0x7a, 0x6f, 0x9c, 0xd7, 0x2d, 0x3c, 0xa3, 0xde, 0xe3, 0x43, - 0xd7, 0x20, 0x49, 0xa6, 0xc2, 0x69, 0xe6, 0x75, 0x70, 0x76, 0xba, 0x94, 0x24, 0x93, 0x20, 0xa1, - 0xa1, 0x65, 0xc8, 0xf6, 0x4e, 0x4e, 0xc4, 0x6e, 0xa2, 0xb3, 0x78, 0x68, 0x62, 0x81, 0xa6, 0x3f, - 0x22, 0x19, 0xe0, 0x63, 0x5d, 0x83, 0xfb, 0x1b, 0xfc, 0xba, 0xc0, 0xdd, 0x2b, 0xeb, 0x1a, 0x59, - 0x7e, 0xed, 0x38, 0xd7, 0xc7, 0xfb, 0x20, 0xda, 0xaa, 0xa9, 0x5b, 0x2d, 0xe3, 0x33, 0xcc, 0x36, - 0x18, 0x1c, 0x7e, 0xf2, 0x52, 0xf0, 0xe9, 0x74, 0x27, 0xc0, 0x91, 0xfe, 0x58, 0xe0, 0xae, 0x98, - 0x7e, 0x31, 0xe2, 0x3d, 0x1f, 0xcf, 0xf2, 0xdd, 0x49, 0x73, 0xdf, 0xf2, 0x3c, 0x49, 0xde, 0x18, - 0xe6, 0x37, 0x55, 0x33, 0xf7, 0x2d, 0xef, 0xa4, 0xcf, 0xf6, 0x08, 0xce, 0xc2, 0x2f, 0xc3, 0x24, - 0x4d, 0x7e, 0x8d, 0xae, 0xe6, 0xbb, 0xff, 0x26, 0xc4, 0xa4, 0xf4, 0x47, 0x09, 0x78, 0x8b, 0x56, - 0xf5, 0x39, 0xb6, 0x8d, 0xfd, 0xee, 0x8e, 0x6d, 0xb9, 0x58, 0x73, 0xb1, 0x1e, 0x6c, 0xb0, 0xc5, - 0xd8, 0x04, 0x3a, 0x64, 0xf8, 0xd1, 0xa4, 0xa1, 0xf3, 0x1b, 0x3e, 0x9e, 0x7e, 0x31, 0xe0, 0x9d, - 0x66, 0x47, 0x9a, 0xb5, 0x55, 0x39, 0xcd, 0x24, 0xd7, 0x74, 0xb4, 0x02, 0x99, 0xb6, 0x57, 0x8d, - 0x0b, 0x79, 0xbf, 0xf8, 0x5c, 0x68, 0x1d, 0x0a, 0xbc, 0xa0, 0x6a, 0xd3, 0x38, 0xc6, 0x8a, 0xea, - 0x5e, 0x64, 0x46, 0xc8, 0x31, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x4a, 0x7f, 0x23, 0x05, 0xb7, 0xcf, - 0x51, 0x71, 0x9c, 0xdd, 0x6b, 0x01, 0xd2, 0xc7, 0xe4, 0x45, 0x06, 0xaf, 0x7d, 0x5a, 0xf6, 0x9f, - 0xd1, 0x5e, 0xcf, 0xb2, 0xb2, 0xaf, 0x1a, 0x4d, 0xb2, 0x0c, 0x31, 0x7f, 0xc3, 0xe1, 0x1e, 0x4d, - 0xd1, 0xfe, 0x7b, 0xa1, 0x05, 0xe8, 0x09, 0x15, 0x44, 0xb3, 0x39, 0xe8, 0x7b, 0x02, 0x2c, 0xb0, - 0x17, 0x32, 0xa7, 0xb7, 0xbe, 0xd7, 0xa4, 0xe8, 0x6b, 0x56, 0x23, 0x5e, 0x33, 0x96, 0x8e, 0x8a, - 0xa1, 0x77, 0xf1, 0x82, 0xcc, 0x87, 0xdf, 0x16, 0x2e, 0xca, 0xc2, 0x6f, 0x08, 0x90, 0x0d, 0x11, - 0xd0, 0x9d, 0x81, 0x10, 0xa3, 0xec, 0x59, 0x54, 0x5c, 0xd1, 0xed, 0x81, 0xb8, 0xa2, 0x52, 0xfa, - 0xf3, 0xd3, 0xa5, 0x94, 0xcc, 0x5c, 0xd7, 0xbd, 0x08, 0xa3, 0x9b, 0xc1, 0x75, 0x35, 0xc9, 0xbe, - 0x4c, 0xde, 0x7d, 0x35, 0x14, 0xa1, 0xaa, 0xde, 0x89, 0x18, 0x45, 0xa8, 0xe4, 0x49, 0xfa, 0x61, - 0x02, 0x66, 0x57, 0x74, 0xbd, 0x5e, 0xa7, 0xa8, 0x26, 0xce, 0x31, 0x86, 0x20, 0x45, 0xcc, 0x0d, - 0x1e, 0x0e, 0x45, 0x7f, 0xa3, 0x77, 0x00, 0xe9, 0x86, 0xc3, 0x6e, 0x86, 0x70, 0x0e, 0x55, 0xdd, - 0x7a, 0x15, 0x1c, 0x7c, 0xcf, 0x7a, 0x29, 0x75, 0x2f, 0x01, 0xd5, 0x81, 0xda, 0xda, 0x8a, 0xe3, - 0xaa, 0xfe, 0xc6, 0xfe, 0xed, 0xb1, 0x02, 0x6c, 0x98, 0x71, 0xec, 0x3f, 0xca, 0x19, 0x22, 0x87, - 0xfe, 0x24, 0x26, 0x9f, 0x41, 0x1a, 0xc5, 0x55, 0x54, 0xc7, 0x0b, 0x0d, 0x61, 0x77, 0x52, 0xe4, - 0x19, 0x7d, 0xc5, 0x61, 0x11, 0x1f, 0xcc, 0xa3, 0x3c, 0x50, 0x4d, 0x9c, 0xdb, 0xb2, 0x7f, 0x5b, - 0x80, 0xbc, 0x8c, 0xf7, 0x6d, 0xec, 0x1c, 0xc6, 0xa9, 0xf3, 0x27, 0x30, 0x63, 0x33, 0xa9, 0xca, - 0xbe, 0x6d, 0xb5, 0x2e, 0x32, 0x57, 0x64, 0x39, 0xe3, 0x13, 0xdb, 0x6a, 0xf1, 0x29, 0xf9, 0x39, - 0x14, 0xfc, 0x32, 0xc6, 0x59, 0xf9, 0xbf, 0x4b, 0x83, 0x47, 0x99, 0xe0, 0xb8, 0x4f, 0xa0, 0xe3, - 0xd5, 0x00, 0xdd, 0x9a, 0x0f, 0x17, 0x34, 0x4e, 0x35, 0xfc, 0x67, 0x01, 0xf2, 0xf5, 0xce, 0x1e, - 0xbb, 0x25, 0x29, 0x3e, 0x0d, 0x54, 0x20, 0xd3, 0xc4, 0xfb, 0xae, 0xf2, 0x5a, 0xbe, 0xd0, 0x69, - 0xc2, 0x4a, 0x3d, 0xc1, 0x9f, 0x02, 0xd8, 0x34, 0x7a, 0x8a, 0xca, 0x49, 0x5e, 0x50, 0x4e, 0x86, - 0xf2, 0x12, 0x32, 0x59, 0x75, 0x0a, 0x7e, 0x35, 0xe3, 0x5c, 0x5f, 0x5e, 0xf4, 0xcc, 0x0e, 0xc9, - 0x8b, 0xcc, 0x0e, 0xb3, 0xdc, 0x60, 0x8d, 0x9e, 0x21, 0x8a, 0x30, 0x47, 0xcd, 0x32, 0x45, 0x6d, - 0xb7, 0x9b, 0x86, 0x07, 0x7b, 0xe8, 0xfc, 0x93, 0x92, 0x67, 0x69, 0xd2, 0x0a, 0x4b, 0xa1, 0x80, - 0x07, 0x7d, 0x57, 0x80, 0x99, 0x7d, 0x1b, 0xe3, 0xcf, 0xb0, 0x42, 0xa7, 0xe4, 0xf1, 0xbc, 0x0a, - 0x56, 0x49, 0x19, 0xbe, 0xf0, 0xa9, 0x63, 0x96, 0xbd, 0xb8, 0x4e, 0xde, 0x8b, 0xb6, 0x40, 0xd4, - 0x9a, 0xec, 0x1c, 0xd4, 0xf7, 0x70, 0x98, 0x1a, 0x7f, 0x00, 0x14, 0x18, 0x73, 0xe0, 0xe4, 0xf0, - 0x8c, 0x0c, 0x26, 0x55, 0x57, 0xf8, 0xed, 0x72, 0xd4, 0xd8, 0xee, 0xf5, 0x70, 0x08, 0x47, 0x91, - 0x87, 0x2e, 0xa5, 0x2b, 0xca, 0x58, 0xd5, 0xb9, 0xe5, 0x4e, 0xc6, 0x95, 0xff, 0xc0, 0xc7, 0xd5, - 0x0b, 0x98, 0xa5, 0xfd, 0x26, 0xee, 0x60, 0x50, 0xe9, 0x1f, 0x25, 0x01, 0x85, 0x25, 0x7f, 0x79, - 0xfd, 0x2d, 0x11, 0x5f, 0x7f, 0x5b, 0x03, 0x29, 0x64, 0x0c, 0x35, 0x55, 0xc7, 0x55, 0x98, 0x2b, - 0x9d, 0xa3, 0xb4, 0xb1, 0xad, 0x38, 0x58, 0xb3, 0xf8, 0x7d, 0x40, 0x82, 0xbc, 0x18, 0xe4, 0xdc, - 0x50, 0x1d, 0xf7, 0x19, 0xcb, 0xb7, 0x83, 0xed, 0x3a, 0xcd, 0x85, 0x1e, 0xc3, 0x95, 0x96, 0x7a, - 0x12, 0xc5, 0x3f, 0x49, 0xf9, 0xe7, 0x5a, 0xea, 0xc9, 0x00, 0xd3, 0x07, 0xb0, 0x10, 0xcd, 0xa4, - 0x38, 0xd8, 0x3b, 0x6a, 0xbb, 0x12, 0xc1, 0x58, 0xc7, 0x2e, 0x5a, 0x01, 0x08, 0x40, 0x04, 0x5f, - 0xa3, 0xc7, 0xc1, 0x10, 0x19, 0x1f, 0x43, 0x48, 0xdf, 0x17, 0x20, 0xbf, 0x69, 0x1c, 0xd8, 0x6a, - 0xac, 0xb7, 0xed, 0xa0, 0x0f, 0x7a, 0xcf, 0x26, 0xb3, 0x8f, 0x16, 0xa2, 0x7c, 0x4f, 0x58, 0x0e, - 0x6f, 0xff, 0x91, 0x33, 0x90, 0xa5, 0xcf, 0x2f, 0x51, 0x9c, 0x73, 0xbe, 0x06, 0x6f, 0x52, 0x6f, - 0x3e, 0xee, 0xca, 0xf3, 0xa5, 0xa0, 0x1b, 0xe9, 0xf7, 0x04, 0x58, 0x1c, 0xf6, 0x96, 0x38, 0x07, - 0x84, 0x4c, 0x2f, 0x15, 0xa4, 0x6f, 0x50, 0xfc, 0x11, 0x71, 0xce, 0x4c, 0x83, 0xf8, 0x48, 0x00, - 0xbf, 0x6c, 0x75, 0x7a, 0xdd, 0x20, 0xfb, 0xed, 0x10, 0x05, 0xd5, 0x35, 0xd5, 0xa4, 0xa1, 0xc2, - 0x4d, 0xac, 0x1e, 0x63, 0xee, 0x88, 0x18, 0xeb, 0x2c, 0xf1, 0x23, 0x01, 0x16, 0x87, 0xbd, 0x25, - 0x4e, 0x05, 0x7d, 0x03, 0xa6, 0x99, 0x47, 0x9b, 0x07, 0x6a, 0xae, 0x0d, 0x89, 0x8a, 0x36, 0xbd, - 0x58, 0x1c, 0x2f, 0xbf, 0xd4, 0x80, 0x7c, 0x49, 0xb5, 0x6d, 0x23, 0xd6, 0xad, 0x07, 0xe9, 0x87, - 0x02, 0x14, 0x7c, 0xb1, 0x71, 0xd6, 0xf4, 0x8b, 0x07, 0x7a, 0x48, 0xff, 0xed, 0x3a, 0xcc, 0xf0, - 0xf2, 0xef, 0x9a, 0x86, 0x65, 0xa2, 0x87, 0x90, 0x3c, 0xe0, 0x07, 0x58, 0xd9, 0xc8, 0x3d, 0xfa, - 0xe0, 0x9a, 0xc7, 0xea, 0x84, 0x4c, 0xf2, 0x12, 0x96, 0x76, 0xc7, 0x8d, 0x28, 0x40, 0x10, 0xa9, - 0x10, 0x66, 0x69, 0x77, 0x5c, 0x54, 0x87, 0x82, 0x16, 0xdc, 0x2d, 0xa7, 0x10, 0xf6, 0xe4, 0xd0, - 0x0d, 0xe5, 0xc8, 0x5b, 0xfe, 0xaa, 0x13, 0x72, 0x5e, 0xeb, 0x49, 0x40, 0xe5, 0xf0, 0x95, 0x66, - 0xa9, 0x01, 0x37, 0xc8, 0xa0, 0xe9, 0x7b, 0xaf, 0x53, 0xab, 0x4e, 0x84, 0x6e, 0x3e, 0x43, 0x1f, - 0xc0, 0x94, 0x4e, 0x2f, 0xcf, 0xe2, 0xf6, 0x44, 0x54, 0x83, 0xf7, 0xdc, 0x51, 0x56, 0x9d, 0x90, - 0x39, 0x07, 0x5a, 0x83, 0x19, 0xf6, 0x8b, 0xa1, 0x5d, 0x6e, 0x05, 0xdc, 0x1e, 0x2e, 0x21, 0x64, - 0x87, 0x57, 0x27, 0xe4, 0xac, 0x1e, 0x50, 0xd1, 0x53, 0xc8, 0x6a, 0x4d, 0xac, 0xda, 0x5c, 0xd4, - 0x9d, 0xa1, 0xb1, 0x9b, 0x03, 0x17, 0x6e, 0x55, 0x27, 0x64, 0xd0, 0x7c, 0x22, 0x29, 0x94, 0x4d, - 0xef, 0x5d, 0xe2, 0x92, 0xde, 0x1d, 0x5a, 0xa8, 0xc1, 0x4b, 0xac, 0xaa, 0xd4, 0x3e, 0xf7, 0xa9, - 0xe8, 0x6b, 0x90, 0x72, 0x34, 0xd5, 0xe4, 0x26, 0xc9, 0xe2, 0x90, 0x8b, 0x71, 0x02, 0x66, 0x9a, - 0x1b, 0x7d, 0xc8, 0x80, 0xb2, 0x7b, 0xe2, 0x9d, 0x1a, 0x44, 0xe9, 0xb4, 0xe7, 0x02, 0x06, 0xa2, - 0x53, 0x4c, 0x09, 0x44, 0x0f, 0xaa, 0xde, 0x32, 0x4c, 0x85, 0x1e, 0xcc, 0xd0, 0x63, 0x82, 0x68, - 0x3d, 0x0c, 0x44, 0xb1, 0x57, 0xe9, 0x2d, 0x0f, 0x1e, 0x11, 0x6d, 0x42, 0x8e, 0x09, 0xea, 0xb0, - 0x00, 0xeb, 0xf9, 0xe5, 0xa1, 0xbe, 0x08, 0x11, 0x21, 0xde, 0xd5, 0x09, 0x79, 0x46, 0x0d, 0x91, - 0x83, 0x72, 0xb5, 0xb0, 0x7d, 0xc0, 0xce, 0x23, 0x46, 0x94, 0x2b, 0xec, 0xe0, 0xe9, 0x97, 0x8b, - 0x12, 0xd1, 0xaf, 0xc1, 0x25, 0x26, 0xc8, 0xe5, 0x7e, 0x6b, 0xdc, 0xfd, 0xe9, 0xcd, 0xa1, 0x7e, - 0x04, 0x43, 0x83, 0xa2, 0xab, 0x13, 0x32, 0x52, 0x07, 0x12, 0x91, 0x06, 0x97, 0xd9, 0x1b, 0x78, - 0x54, 0xad, 0xcd, 0x03, 0x41, 0xe7, 0x6f, 0xd1, 0x57, 0xbc, 0x33, 0xec, 0x15, 0x91, 0xc1, 0xbe, - 0xd5, 0x09, 0x79, 0x4e, 0x1d, 0x4c, 0x0d, 0xaa, 0x61, 0xf3, 0xf8, 0x45, 0xde, 0xdd, 0xde, 0x19, - 0x5d, 0x8d, 0xa8, 0xb8, 0x4f, 0xbf, 0x1a, 0x3d, 0x89, 0xa4, 0x01, 0xfd, 0xdb, 0x1b, 0x68, 0x67, - 0x9a, 0x19, 0xda, 0x80, 0x11, 0x41, 0x8e, 0xa4, 0x01, 0x0f, 0x43, 0x64, 0x54, 0x84, 0xc4, 0x81, - 0x46, 0x4f, 0xfd, 0xa2, 0x4d, 0x27, 0x3f, 0x90, 0xaf, 0x3a, 0x21, 0x27, 0x0e, 0x34, 0xf4, 0x31, - 0xa4, 0x59, 0x54, 0xd6, 0x89, 0x39, 0x9f, 0x1f, 0x3a, 0x67, 0xf7, 0xc6, 0xb6, 0x55, 0x27, 0x64, - 0x1a, 0x08, 0xc6, 0x3b, 0x32, 0x8f, 0xb8, 0xa1, 0x22, 0x8a, 0x23, 0x82, 0xb1, 0xfb, 0xe2, 0x9e, - 0x48, 0x87, 0xb1, 0x7d, 0x22, 0xda, 0x81, 0x3c, 0x5f, 0xba, 0xbd, 0x08, 0x02, 0x71, 0xa8, 0xa7, - 0x4e, 0x54, 0x10, 0x41, 0x95, 0x6e, 0x51, 0x86, 0xe8, 0xa4, 0xed, 0x7a, 0x25, 0xf2, 0xb6, 0x9b, - 0x1d, 0xda, 0x76, 0x43, 0x1d, 0xda, 0x49, 0xdb, 0xd9, 0x03, 0x89, 0xe8, 0x3d, 0x98, 0x64, 0xe3, - 0x04, 0x51, 0x91, 0x51, 0xce, 0x67, 0x7d, 0x43, 0x84, 0xe5, 0x27, 0xb3, 0x97, 0xcb, 0x1d, 0x73, - 0x95, 0xa6, 0x75, 0x30, 0x3f, 0x37, 0x74, 0xf6, 0x1a, 0x74, 0x31, 0x26, 0xb3, 0x97, 0x1b, 0x50, - 0x49, 0x07, 0xb2, 0x59, 0x0a, 0x1f, 0x62, 0x97, 0x86, 0x76, 0xa0, 0x08, 0x7f, 0xdd, 0x2a, 0x0d, - 0x99, 0x0a, 0xc8, 0xfe, 0xc4, 0xea, 0x60, 0x85, 0x4e, 0x8a, 0x97, 0x47, 0x4f, 0xac, 0x3d, 0xb7, - 0x95, 0xf9, 0x13, 0x2b, 0xa3, 0xa2, 0xe7, 0x20, 0xf2, 0x2b, 0x73, 0x14, 0xcf, 0x7f, 0x6c, 0xfe, - 0x0a, 0x95, 0x77, 0x3f, 0x72, 0x41, 0x8c, 0x72, 0x2d, 0xac, 0x12, 0x2c, 0xd9, 0x9b, 0x82, 0x3e, - 0x81, 0x59, 0x2a, 0x4f, 0xd1, 0x82, 0x5b, 0x8e, 0xe6, 0xe7, 0x07, 0xee, 0xcc, 0x19, 0x7e, 0x21, - 0x92, 0x27, 0x59, 0xd4, 0xfa, 0x92, 0xc8, 0x78, 0x30, 0x4c, 0xc3, 0xa5, 0x6b, 0xf7, 0xc2, 0xd0, - 0xf1, 0xd0, 0x7b, 0xc3, 0x6b, 0x95, 0xda, 0x5a, 0x94, 0x42, 0xba, 0x71, 0xdf, 0x8c, 0xf7, 0xc6, - 0xd0, 0x6e, 0x3c, 0x64, 0xb2, 0xcb, 0xb9, 0x3d, 0xf3, 0xdc, 0x2a, 0x00, 0xdb, 0x41, 0xa0, 0xa0, - 0x68, 0x71, 0xa8, 0x01, 0xd0, 0xef, 0x4f, 0x4b, 0x0c, 0x80, 0xa6, 0x47, 0x23, 0x06, 0x00, 0x3b, - 0xee, 0x9a, 0xbf, 0x31, 0x7c, 0xb1, 0x0a, 0x9f, 0xaf, 0xd3, 0xc5, 0x8a, 0x12, 0x88, 0x41, 0x46, - 0xe0, 0x5c, 0x97, 0x8e, 0xf0, 0x9b, 0x43, 0x77, 0x6f, 0xfa, 0x02, 0xed, 0xaa, 0x13, 0x72, 0xfa, - 0x25, 0x27, 0x91, 0x5e, 0xc5, 0x44, 0xf0, 0xb1, 0xfd, 0x60, 0x68, 0xaf, 0x1a, 0x8c, 0xb0, 0x22, - 0xbd, 0xea, 0x65, 0x40, 0x0d, 0x96, 0x3c, 0x87, 0x1d, 0x64, 0xcd, 0xbf, 0x35, 0x7a, 0xc9, 0xeb, - 0x3d, 0x76, 0xf3, 0x97, 0x3c, 0x4e, 0x66, 0x4b, 0x9e, 0xae, 0x38, 0x0e, 0x75, 0xd3, 0x99, 0xbf, - 0x3d, 0x62, 0xc9, 0xeb, 0xdb, 0xda, 0x66, 0x4b, 0x9e, 0x5e, 0x67, 0x9c, 0xc4, 0xfa, 0xb3, 0xbd, - 0x2b, 0xa2, 0x38, 0xb0, 0xbf, 0x3b, 0xd4, 0xfa, 0x8b, 0xbc, 0xc3, 0x8a, 0x58, 0x7f, 0x76, 0x4f, - 0x02, 0xfa, 0x26, 0x4c, 0xf3, 0xad, 0xc4, 0xf9, 0x7b, 0x23, 0x4c, 0xea, 0xf0, 0xee, 0x2f, 0xe9, - 0x8e, 0x9c, 0x87, 0x4d, 0x0e, 0x6c, 0x0b, 0x93, 0x4d, 0x7e, 0xf7, 0x47, 0x4c, 0x0e, 0x03, 0xbb, - 0xa8, 0x6c, 0x72, 0x08, 0xc8, 0xa4, 0x34, 0x0e, 0xdb, 0x7e, 0x9b, 0xff, 0xa5, 0xa1, 0xa5, 0xe9, - 0xdd, 0x87, 0x24, 0xa5, 0xe1, 0x3c, 0x74, 0xb1, 0xa0, 0x6b, 0x35, 0xd3, 0xce, 0xdb, 0xc3, 0x17, - 0x8b, 0xfe, 0x0d, 0x9d, 0xaa, 0x77, 0x50, 0xc8, 0xb4, 0xf2, 0x97, 0x04, 0xb8, 0xc1, 0xfa, 0x00, - 0x3d, 0x26, 0xe9, 0x2a, 0xfe, 0x29, 0x57, 0x68, 0xb7, 0xea, 0x21, 0x15, 0xff, 0xde, 0xc5, 0x0f, - 0x65, 0xbc, 0x37, 0xbe, 0xa9, 0x8e, 0xca, 0x47, 0x94, 0xd1, 0x62, 0xb8, 0x7e, 0xfe, 0xd1, 0x50, - 0x65, 0xf4, 0xee, 0x45, 0x10, 0x65, 0x70, 0x1e, 0xd4, 0x84, 0x79, 0x36, 0x24, 0x02, 0xdc, 0xeb, - 0x17, 0xfd, 0xf1, 0x50, 0xcf, 0xd7, 0x91, 0x88, 0xbf, 0x3a, 0x21, 0x5f, 0x79, 0x19, 0x99, 0x81, - 0xbc, 0x8d, 0xde, 0x01, 0x69, 0x04, 0x30, 0x55, 0xf1, 0xf0, 0xe4, 0xd7, 0x86, 0xbe, 0x6d, 0x24, - 0x7c, 0x26, 0x6f, 0x73, 0x22, 0x33, 0x10, 0xd5, 0xec, 0x31, 0x68, 0x38, 0xff, 0xf5, 0xa1, 0xaa, - 0xe9, 0xc5, 0xa4, 0x44, 0x35, 0x9c, 0xa7, 0x34, 0xcd, 0x5d, 0x2e, 0xfc, 0xd0, 0xee, 0x82, 0x28, - 0xae, 0xa5, 0xd2, 0x57, 0xc5, 0xf9, 0xb5, 0x54, 0xfa, 0x9a, 0xb8, 0xb0, 0x96, 0x4a, 0x5f, 0x17, - 0xdf, 0x58, 0x4b, 0xa5, 0x97, 0xc4, 0x1b, 0x6b, 0xa9, 0xb4, 0x24, 0xde, 0x92, 0x7e, 0x76, 0x1d, - 0x72, 0x1e, 0xb2, 0x64, 0x90, 0xef, 0x51, 0x18, 0xf2, 0x2d, 0x0e, 0x83, 0x7c, 0x1c, 0x8b, 0x72, - 0xcc, 0xf7, 0x28, 0x8c, 0xf9, 0x16, 0x87, 0x61, 0xbe, 0x80, 0x87, 0x80, 0xbe, 0xc6, 0x30, 0xd0, - 0x77, 0x7f, 0x0c, 0xd0, 0xe7, 0x8b, 0xea, 0x47, 0x7d, 0xab, 0x83, 0xa8, 0xef, 0xad, 0xd1, 0xa8, - 0xcf, 0x17, 0x15, 0x82, 0x7d, 0x1f, 0xf6, 0xc1, 0xbe, 0x9b, 0x23, 0x60, 0x9f, 0xcf, 0xef, 0xe1, - 0xbe, 0xf5, 0x48, 0xdc, 0x77, 0xe7, 0x3c, 0xdc, 0xe7, 0xcb, 0xe9, 0x01, 0x7e, 0xd5, 0x28, 0xe0, - 0x77, 0xfb, 0x1c, 0xe0, 0xe7, 0x8b, 0x0a, 0x23, 0xbf, 0xf5, 0x48, 0xe4, 0x77, 0xe7, 0x3c, 0xe4, - 0x17, 0x14, 0x2b, 0x0c, 0xfd, 0xbe, 0xde, 0x03, 0xfd, 0x96, 0x86, 0x42, 0x3f, 0x9f, 0x9b, 0x61, - 0xbf, 0x8f, 0xfa, 0xb1, 0xdf, 0xcd, 0x11, 0xd8, 0x2f, 0x50, 0x2c, 0x07, 0x7f, 0xd5, 0x28, 0xf0, - 0x77, 0xfb, 0x1c, 0xf0, 0x17, 0xe8, 0x22, 0x84, 0xfe, 0xb6, 0xa2, 0xd1, 0xdf, 0xdd, 0x73, 0xd1, - 0x9f, 0x2f, 0xad, 0x17, 0xfe, 0x55, 0xa3, 0xe0, 0xdf, 0xed, 0x73, 0xe0, 0x5f, 0x5f, 0xc9, 0x18, - 0xfe, 0x53, 0x47, 0xe2, 0xbf, 0x77, 0xc6, 0xc4, 0x7f, 0xbe, 0xe8, 0x28, 0x00, 0xa8, 0x8f, 0x06, - 0x80, 0xc5, 0x71, 0x01, 0xa0, 0xff, 0x92, 0x48, 0x04, 0xa8, 0x8e, 0x44, 0x80, 0xef, 0x8c, 0x89, - 0x00, 0xfb, 0x2a, 0xd2, 0x0b, 0x01, 0xb7, 0xa2, 0x21, 0xe0, 0xdd, 0x73, 0x21, 0x60, 0xd0, 0x8a, - 0x3d, 0x18, 0x70, 0x39, 0x84, 0x01, 0xdf, 0x1c, 0x82, 0x01, 0x7d, 0x56, 0x02, 0x02, 0xbf, 0x35, - 0x00, 0x02, 0xa5, 0x51, 0x20, 0xd0, 0xe7, 0xf5, 0x51, 0x60, 0x35, 0x0a, 0x05, 0xde, 0x3e, 0x07, - 0x05, 0x06, 0xfd, 0x26, 0x04, 0x03, 0x9f, 0x0d, 0x81, 0x81, 0xf7, 0xce, 0x87, 0x81, 0xbe, 0xbc, - 0x3e, 0x1c, 0xa8, 0x8e, 0xc4, 0x81, 0xef, 0x8c, 0x89, 0x03, 0x83, 0x16, 0x8c, 0x00, 0x82, 0xef, - 0xf7, 0x02, 0xc1, 0x1b, 0xc3, 0x81, 0xa0, 0x2f, 0x86, 0x23, 0xc1, 0xf5, 0x48, 0x24, 0x78, 0xe7, - 0x3c, 0x24, 0x18, 0xcc, 0x66, 0x61, 0x28, 0xb8, 0x15, 0x0d, 0x05, 0xef, 0x9e, 0x0b, 0x05, 0x83, - 0x8e, 0xd4, 0x83, 0x05, 0xd7, 0x23, 0xb1, 0xe0, 0x9d, 0xf3, 0xb0, 0x60, 0xdf, 0x54, 0xcb, 0xc1, - 0xe0, 0x8b, 0xa1, 0x60, 0xf0, 0xc1, 0x38, 0x60, 0xd0, 0x17, 0x3a, 0x80, 0x06, 0x3f, 0x1d, 0x8e, - 0x06, 0x7f, 0xe9, 0x02, 0xd7, 0xe3, 0x46, 0xc2, 0xc1, 0x6f, 0x0d, 0xc0, 0x41, 0x69, 0x14, 0x1c, - 0x0c, 0x46, 0x86, 0x87, 0x07, 0x2b, 0x11, 0xe8, 0xed, 0xad, 0xd1, 0xe8, 0x2d, 0x58, 0xc8, 0x03, - 0xf8, 0xf6, 0x61, 0x1f, 0x7c, 0xbb, 0x79, 0xae, 0x37, 0x68, 0x08, 0xbf, 0x95, 0x06, 0xf1, 0xdb, - 0xad, 0x91, 0xf8, 0xcd, 0x97, 0x10, 0x00, 0xb8, 0xf5, 0x48, 0x00, 0x77, 0xe7, 0x3c, 0x00, 0x17, - 0x74, 0x85, 0x30, 0x82, 0xdb, 0x8a, 0x46, 0x70, 0x77, 0xcf, 0x45, 0x70, 0x7d, 0xcb, 0x96, 0x07, - 0xe1, 0xaa, 0x51, 0x10, 0xee, 0xf6, 0x39, 0x10, 0x2e, 0xbc, 0x6c, 0xf9, 0x18, 0xae, 0x31, 0x0c, - 0xc3, 0xdd, 0x1f, 0x03, 0xc3, 0x05, 0xc6, 0x5c, 0x1f, 0x88, 0xfb, 0xb8, 0x1f, 0xc4, 0x49, 0xa3, - 0x40, 0x5c, 0xd0, 0x89, 0x3c, 0x14, 0xb7, 0x15, 0x8d, 0xe2, 0xee, 0x9e, 0x8b, 0xe2, 0xc2, 0xe3, - 0x3a, 0x04, 0xe3, 0x3e, 0xee, 0x87, 0x71, 0xd2, 0x28, 0x18, 0x17, 0x94, 0xc7, 0xc3, 0x71, 0xd5, - 0x28, 0x1c, 0x77, 0xfb, 0x1c, 0x1c, 0x17, 0x9a, 0xee, 0x03, 0x20, 0xf7, 0x97, 0xc7, 0x07, 0x72, - 0xef, 0xbf, 0xae, 0x77, 0xdd, 0xf9, 0x48, 0xee, 0xe3, 0x7e, 0x24, 0x27, 0x8d, 0x42, 0x72, 0x81, - 0x3e, 0x3c, 0x28, 0xd7, 0x3a, 0x17, 0xca, 0x3d, 0xbc, 0x00, 0x94, 0xf3, 0xe5, 0x0f, 0xc3, 0x72, - 0xad, 0x73, 0xb1, 0xdc, 0xc3, 0x0b, 0x60, 0xb9, 0xe0, 0x75, 0x43, 0xc0, 0xdc, 0xc7, 0xfd, 0x60, - 0x4e, 0x1a, 0x05, 0xe6, 0x02, 0xed, 0x5c, 0x18, 0xcd, 0xbd, 0x21, 0xbe, 0xd9, 0x83, 0xe9, 0xfe, - 0x0a, 0xc0, 0x54, 0x35, 0xe2, 0x4c, 0x50, 0x78, 0x9d, 0x33, 0x41, 0xf4, 0x4d, 0xb8, 0xee, 0x3f, - 0x50, 0x7f, 0x2e, 0x85, 0x87, 0xc4, 0x69, 0x4d, 0x4b, 0x3b, 0xa2, 0x4b, 0x65, 0x5a, 0x9e, 0xf7, - 0xb3, 0x3c, 0xb1, 0xad, 0x16, 0x0b, 0x8d, 0xa3, 0x5e, 0x32, 0x68, 0x95, 0x8c, 0x61, 0x6a, 0x13, - 0x9e, 0x7f, 0xe3, 0xe8, 0xe0, 0x9d, 0x98, 0x9c, 0xf5, 0x35, 0x2e, 0x71, 0x40, 0x5f, 0x87, 0x5c, - 0xc7, 0xc1, 0xb6, 0xd2, 0xb6, 0x0d, 0xcb, 0x36, 0x5c, 0x16, 0x8e, 0x26, 0x94, 0xc4, 0xcf, 0x4f, - 0x97, 0x66, 0x76, 0x1d, 0x6c, 0xef, 0x70, 0xba, 0x3c, 0xd3, 0x09, 0x3d, 0x79, 0xdf, 0xf5, 0x9b, - 0x1c, 0xff, 0xbb, 0x7e, 0xcf, 0x40, 0xa4, 0x1e, 0x3a, 0xe1, 0x65, 0x94, 0xdd, 0xd3, 0x16, 0xbd, - 0xe2, 0xab, 0x7a, 0x68, 0xa5, 0xa4, 0xf7, 0xb5, 0x15, 0xec, 0x5e, 0x22, 0x7a, 0x0a, 0x79, 0xdb, - 0xea, 0xd0, 0xeb, 0x98, 0xda, 0x56, 0xd3, 0xd0, 0xba, 0xd4, 0xbe, 0xc9, 0x47, 0x9f, 0x37, 0xb3, - 0x8c, 0x3b, 0x34, 0x9f, 0x9c, 0xb3, 0xc3, 0x8f, 0xa8, 0x0e, 0xf4, 0x2a, 0x26, 0x4f, 0x0a, 0x1a, - 0xb8, 0xfc, 0x7e, 0xe4, 0x27, 0x28, 0x5e, 0xa8, 0x86, 0xcb, 0xe5, 0xc2, 0x2b, 0xff, 0x37, 0x7a, - 0x02, 0x33, 0xf4, 0x5a, 0x70, 0xd2, 0xe6, 0x56, 0xc7, 0xe5, 0xe6, 0xcd, 0xb5, 0x22, 0xfb, 0xfa, - 0x63, 0xd1, 0xfb, 0xfa, 0x63, 0x71, 0x95, 0x7f, 0xfd, 0x91, 0x9d, 0x55, 0xff, 0xe0, 0x3f, 0x2e, - 0x09, 0x72, 0xd6, 0xf3, 0x9c, 0xb2, 0x3a, 0x2e, 0x7a, 0x08, 0x97, 0x5b, 0xea, 0x09, 0xbd, 0x5d, - 0x5c, 0xf1, 0xec, 0x2f, 0x7a, 0x9f, 0x22, 0xfb, 0xaa, 0x20, 0x6a, 0xa9, 0x27, 0xf4, 0x53, 0x88, - 0x2c, 0x89, 0x7e, 0xea, 0xe8, 0x26, 0xcc, 0xf0, 0x58, 0x21, 0xf6, 0x25, 0xb4, 0x02, 0xcd, 0xc9, - 0x3f, 0x8b, 0xc3, 0x3e, 0x86, 0xf6, 0x1e, 0xcc, 0x87, 0xb3, 0x28, 0xcc, 0xd3, 0x15, 0xb7, 0xda, - 0x6e, 0x77, 0xfe, 0x2a, 0xed, 0xac, 0x97, 0x43, 0xd9, 0x57, 0x48, 0x6a, 0x85, 0x24, 0x12, 0x46, - 0x1e, 0x5d, 0x64, 0x99, 0x6c, 0xbd, 0x60, 0x51, 0x5b, 0xaa, 0xcd, 0xcc, 0xa2, 0xb4, 0xcc, 0xa3, - 0x8f, 0xb6, 0x4d, 0xe6, 0x71, 0xcc, 0x13, 0xd1, 0x6d, 0xc8, 0xeb, 0x86, 0xe3, 0x1a, 0xa6, 0xe6, - 0xf2, 0xab, 0xd2, 0xd9, 0x65, 0xe3, 0x39, 0x8f, 0xca, 0xee, 0x43, 0x6f, 0xc0, 0xac, 0xd6, 0x34, - 0x7c, 0x3b, 0x9a, 0x59, 0x36, 0xb3, 0x43, 0x67, 0x86, 0x32, 0xcd, 0xdb, 0xef, 0xb2, 0x53, 0xd0, - 0x7a, 0xc9, 0xa8, 0x0c, 0x85, 0x03, 0xd5, 0xc5, 0xaf, 0xd4, 0xae, 0xe2, 0x05, 0xfe, 0x66, 0xe9, - 0x65, 0x07, 0xd7, 0xcf, 0x4e, 0x97, 0x72, 0x4f, 0x59, 0xd2, 0x40, 0xfc, 0x6f, 0xee, 0x20, 0x94, - 0xa0, 0xa3, 0xbb, 0x50, 0x50, 0x9d, 0xae, 0xa9, 0xd1, 0x3e, 0x8c, 0x4d, 0xa7, 0xe3, 0xf0, 0x00, - 0xb8, 0x3c, 0x25, 0x97, 0x3d, 0x2a, 0xfa, 0x10, 0x16, 0xf8, 0x17, 0x51, 0x5e, 0xa9, 0xb6, 0xae, - 0xd0, 0x7e, 0x1f, 0x4c, 0x30, 0x22, 0xe5, 0xb9, 0xca, 0xbe, 0x80, 0x42, 0x32, 0x90, 0xce, 0x1e, - 0xcc, 0xbf, 0xcf, 0x61, 0x96, 0x2a, 0x14, 0xeb, 0x64, 0x01, 0x6c, 0x62, 0x13, 0x3b, 0xce, 0x88, - 0x73, 0x8d, 0x12, 0xcb, 0x5b, 0xf7, 0xb2, 0xb2, 0x29, 0x4d, 0x16, 0xf7, 0xfa, 0xe8, 0xfe, 0x0d, - 0xed, 0x20, 0x66, 0xd7, 0x52, 0xe9, 0x19, 0x31, 0xb7, 0x96, 0x4a, 0xe7, 0xc5, 0x82, 0xf4, 0x7f, - 0x05, 0xb8, 0x12, 0x2d, 0x04, 0xd5, 0x61, 0xae, 0xe7, 0x12, 0x22, 0xd6, 0xd2, 0x17, 0x99, 0x21, - 0x67, 0xc3, 0x57, 0x11, 0xd1, 0xb7, 0x10, 0xe5, 0x44, 0x08, 0x55, 0x1c, 0xd7, 0x36, 0x34, 0xef, - 0x62, 0xaf, 0xab, 0x03, 0x6c, 0x75, 0x9a, 0x4c, 0x4b, 0xa4, 0x9e, 0x0c, 0x94, 0x28, 0x79, 0x91, - 0x12, 0xa9, 0x27, 0xbd, 0xa2, 0xa5, 0xef, 0x26, 0xa1, 0x40, 0xd6, 0x7a, 0xc7, 0x31, 0x2c, 0xb3, - 0xea, 0x87, 0x04, 0xf8, 0x53, 0xa2, 0x40, 0xa3, 0x5b, 0xfd, 0x67, 0xb4, 0x44, 0xa3, 0x88, 0x09, - 0xa6, 0xf2, 0xbf, 0x30, 0x95, 0x94, 0x81, 0x91, 0x68, 0x9c, 0xe4, 0x0a, 0x4c, 0x39, 0x56, 0xc7, - 0xd6, 0xbc, 0xef, 0x9e, 0xdc, 0x1f, 0x62, 0x5c, 0x84, 0x5e, 0x58, 0xac, 0x53, 0x06, 0x99, 0x33, - 0xa2, 0x4f, 0xa1, 0xc0, 0x7e, 0xd1, 0xe8, 0x45, 0x1a, 0x3d, 0xc8, 0x42, 0x40, 0x1f, 0x8e, 0x2d, - 0x6b, 0x83, 0x33, 0xca, 0x79, 0xa7, 0xe7, 0x19, 0x7d, 0x0c, 0x6f, 0x98, 0x96, 0xd2, 0xc2, 0x2d, - 0x8b, 0x19, 0x15, 0x64, 0x5e, 0xd3, 0x15, 0xd5, 0x55, 0x78, 0xa1, 0x99, 0x8f, 0xf9, 0xbc, 0x69, - 0x6d, 0xd2, 0x2c, 0x32, 0xcf, 0xb1, 0xe2, 0x32, 0xb9, 0x52, 0x11, 0xa6, 0xd8, 0x2f, 0x94, 0x81, - 0xc9, 0xed, 0x46, 0xb5, 0x22, 0x8b, 0x13, 0x68, 0x06, 0xd2, 0x4f, 0xe4, 0xed, 0x4d, 0xa5, 0xfe, - 0x6c, 0x43, 0x14, 0x50, 0x16, 0xa6, 0xe5, 0xed, 0xed, 0x86, 0xb2, 0xfe, 0x5c, 0x4c, 0x48, 0x77, - 0x21, 0xdf, 0x5b, 0x22, 0x04, 0x30, 0x25, 0x57, 0x36, 0xb7, 0xe9, 0xc7, 0x3e, 0x32, 0x30, 0xb9, - 0xb1, 0x5d, 0x5e, 0xd9, 0x10, 0x05, 0xe9, 0x17, 0x02, 0xcc, 0x94, 0xd8, 0xd7, 0x6b, 0x98, 0x27, - 0xd1, 0x87, 0x7d, 0x1e, 0x3f, 0xd7, 0xa2, 0x37, 0x2d, 0x86, 0x79, 0xfa, 0xa4, 0xf9, 0x7c, 0xe9, - 0x45, 0xb9, 0x2d, 0x0d, 0x87, 0xaa, 0x74, 0x57, 0xd7, 0x73, 0x2d, 0xf6, 0xd8, 0x50, 0x1d, 0x44, - 0xd5, 0xd3, 0xad, 0xc2, 0x4b, 0x32, 0xdc, 0xc1, 0xb8, 0xaf, 0x19, 0xbc, 0xb9, 0x48, 0xed, 0x25, - 0x7f, 0x90, 0xfa, 0xc1, 0x8f, 0x96, 0x26, 0xa4, 0x3f, 0x49, 0x41, 0xae, 0x14, 0xfe, 0x52, 0x0f, - 0xaa, 0xf5, 0x55, 0xf6, 0x6e, 0xa4, 0x21, 0x14, 0xe2, 0x28, 0x8e, 0xf8, 0x06, 0x5a, 0x26, 0xf8, - 0x2c, 0x10, 0xab, 0xfb, 0x8d, 0x11, 0xce, 0x52, 0xe1, 0xca, 0x07, 0x8c, 0x0b, 0xff, 0x36, 0xe9, - 0x5b, 0x48, 0x45, 0x98, 0x64, 0x51, 0xdc, 0xc2, 0xc0, 0x05, 0x33, 0x74, 0x6d, 0x24, 0x00, 0x91, - 0xa4, 0xcb, 0x2c, 0x1b, 0xb1, 0xa8, 0x1a, 0xaf, 0xe5, 0x65, 0x15, 0xcc, 0x83, 0x17, 0xff, 0x74, - 0x70, 0x87, 0x5d, 0xa7, 0xfc, 0xff, 0xd1, 0xa7, 0x99, 0xbc, 0x0f, 0xfd, 0x2a, 0x14, 0x34, 0xab, - 0xd9, 0x64, 0xb8, 0x82, 0xad, 0x6c, 0x83, 0x17, 0xac, 0xd1, 0x22, 0xf0, 0xaf, 0x45, 0x17, 0xfd, - 0xaf, 0x46, 0x17, 0x65, 0xfe, 0xd5, 0xe8, 0x50, 0xc0, 0x59, 0xde, 0x17, 0xc6, 0x16, 0xc4, 0xbe, - 0xd8, 0xb7, 0xe9, 0xd7, 0x89, 0x7d, 0x63, 0x11, 0x83, 0xbc, 0xe7, 0xfd, 0x4c, 0xe0, 0x9e, 0xc7, - 0x1b, 0x96, 0x75, 0xd4, 0xf1, 0xfd, 0x39, 0x17, 0xc2, 0x97, 0x23, 0x07, 0x61, 0x39, 0x34, 0x4a, - 0x35, 0xca, 0x78, 0x4b, 0x7c, 0x31, 0xe3, 0xed, 0x26, 0xcc, 0xb4, 0x6d, 0xbc, 0x8f, 0x5d, 0xed, - 0x50, 0x31, 0x3b, 0x2d, 0x1e, 0xa2, 0x9b, 0xf5, 0x68, 0x5b, 0x9d, 0x16, 0xba, 0x0f, 0xa2, 0x9f, - 0x85, 0xef, 0xf5, 0x78, 0x37, 0x73, 0x7a, 0x74, 0xbe, 0x33, 0x24, 0xfd, 0x0f, 0x01, 0xe6, 0x7a, - 0xea, 0xc4, 0xc7, 0xd4, 0x1a, 0x64, 0x75, 0xdf, 0x5c, 0x76, 0xe6, 0x85, 0x0b, 0x06, 0x6c, 0x85, - 0x99, 0x91, 0x02, 0x57, 0xbc, 0xd7, 0xd2, 0x4f, 0xe9, 0x04, 0x62, 0x13, 0x17, 0x14, 0x7b, 0x39, - 0x90, 0xb3, 0x1a, 0x7a, 0x81, 0x3f, 0xc8, 0x92, 0x63, 0x0d, 0x32, 0xe9, 0x7f, 0x09, 0x20, 0xd2, - 0x17, 0x3c, 0xc1, 0x58, 0x8f, 0x65, 0xca, 0xf4, 0x22, 0x23, 0x13, 0xe3, 0x07, 0xe1, 0xf6, 0x7c, - 0xfe, 0x2b, 0xd9, 0xf7, 0xf9, 0xaf, 0xa8, 0xf9, 0x33, 0xf5, 0x05, 0xe7, 0x4f, 0xe9, 0x47, 0x02, - 0xe4, 0xfd, 0x6a, 0xb3, 0xef, 0xfe, 0x8e, 0xb8, 0xd8, 0xfb, 0xf5, 0xbe, 0x6d, 0xeb, 0x5d, 0x40, - 0x36, 0xd6, 0xa7, 0x88, 0xc3, 0x17, 0x90, 0xb1, 0x6f, 0xb2, 0xfe, 0x2d, 0xaf, 0x3b, 0x92, 0x22, - 0x96, 0x83, 0x9b, 0x9f, 0x5e, 0x23, 0xc8, 0xf9, 0xcb, 0x70, 0x5d, 0x7e, 0x12, 0x52, 0x20, 0xed, - 0x51, 0x44, 0x4b, 0x63, 0xcd, 0xef, 0x9e, 0x96, 0x58, 0x07, 0xfc, 0x83, 0x70, 0x4b, 0xb0, 0x9b, - 0x43, 0x1e, 0x43, 0xf2, 0x58, 0x6d, 0x8e, 0x72, 0xd0, 0xed, 0x69, 0x39, 0x99, 0xe4, 0x46, 0x4f, - 0x7a, 0x2e, 0xcc, 0x4a, 0x0c, 0xdf, 0x07, 0x1e, 0x54, 0x69, 0xcf, 0xc5, 0x5a, 0xef, 0xf5, 0x0e, - 0xa0, 0x91, 0xaf, 0x0f, 0x8f, 0xa4, 0x0f, 0x52, 0x3f, 0xf9, 0xd1, 0x92, 0x20, 0x7d, 0x04, 0x88, - 0xd8, 0x3a, 0xee, 0xb3, 0x8e, 0x65, 0x07, 0x97, 0x8f, 0xf5, 0x47, 0x40, 0x4e, 0x46, 0x47, 0x40, - 0x4a, 0x97, 0x61, 0xae, 0x87, 0x9b, 0xcd, 0x40, 0xd2, 0x7b, 0x70, 0xed, 0xa9, 0xe5, 0x38, 0x46, - 0xbb, 0xde, 0xd9, 0x63, 0x43, 0x9d, 0xac, 0x57, 0xfe, 0x9c, 0x9b, 0x6e, 0xd3, 0x4d, 0x46, 0x93, - 0xcd, 0x4d, 0x19, 0xd9, 0x7f, 0x96, 0x7e, 0x5f, 0x80, 0xab, 0x83, 0x9c, 0x4c, 0xcb, 0x51, 0x77, - 0x32, 0x4c, 0x6b, 0x56, 0x70, 0x37, 0xee, 0xf9, 0xbd, 0xd5, 0xcb, 0x4e, 0x50, 0x0d, 0x7f, 0xa7, - 0xd2, 0x52, 0xe9, 0x9c, 0xc4, 0xaf, 0x6a, 0xc9, 0x73, 0xf2, 0x26, 0xa3, 0x06, 0xd3, 0x53, 0x6a, - 0xbc, 0xe9, 0xe9, 0xc7, 0x09, 0x98, 0x6d, 0x60, 0x53, 0x35, 0x5d, 0x32, 0xef, 0x77, 0x5a, 0xec, - 0x1e, 0x8b, 0x02, 0x24, 0x6d, 0xa5, 0x43, 0x8b, 0x2e, 0xc8, 0x09, 0x7b, 0x17, 0xdd, 0x82, 0x1c, - 0x5d, 0x5b, 0x42, 0xb6, 0x9a, 0x70, 0x2f, 0x25, 0xd3, 0x78, 0x1e, 0xd9, 0x33, 0xc4, 0xde, 0x04, - 0xa0, 0x99, 0x18, 0x9e, 0x4d, 0xd2, 0x1c, 0x19, 0x42, 0x61, 0x68, 0xf6, 0x36, 0xe4, 0x69, 0x7c, - 0x64, 0x20, 0x84, 0x85, 0x40, 0xe5, 0x28, 0xd5, 0x97, 0xb2, 0x04, 0x59, 0x96, 0x8d, 0x89, 0x99, - 0xa4, 0x79, 0x80, 0x92, 0x98, 0x9c, 0x27, 0x70, 0xc9, 0x79, 0xd9, 0x54, 0xda, 0x96, 0xee, 0x28, - 0x5a, 0xbb, 0xc3, 0x63, 0x4b, 0xd8, 0xa7, 0xc4, 0x85, 0xd2, 0xe5, 0xb3, 0xd3, 0xa5, 0xd9, 0xfa, - 0xb3, 0x8d, 0x1d, 0x4b, 0x77, 0xca, 0x3b, 0xbb, 0x2c, 0xb2, 0xc4, 0x91, 0x67, 0x9d, 0x97, 0x4d, - 0x4a, 0x6a, 0x77, 0x38, 0x09, 0x55, 0x60, 0xae, 0x7d, 0xf0, 0xca, 0xb0, 0xb1, 0x82, 0x0f, 0x6c, - 0xec, 0x38, 0xfc, 0x85, 0xd3, 0xe4, 0x85, 0x4c, 0xcc, 0xce, 0xd3, 0x17, 0x86, 0x8d, 0x2b, 0x34, - 0x95, 0xbe, 0x5b, 0x9e, 0x65, 0x1c, 0x21, 0x92, 0xf4, 0x8b, 0x24, 0x20, 0x7a, 0xa5, 0x53, 0x89, - 0xde, 0x89, 0xe4, 0xf5, 0x1a, 0x0b, 0x16, 0xb5, 0x40, 0xa3, 0x8a, 0x63, 0x98, 0x04, 0x26, 0xa8, - 0x8e, 0xeb, 0x55, 0x9f, 0x0f, 0xbf, 0xc8, 0x9b, 0x68, 0xfb, 0x1b, 0x84, 0x77, 0x87, 0xeb, 0x21, - 0x89, 0x75, 0x22, 0x70, 0x43, 0x75, 0xfc, 0x17, 0xde, 0x87, 0x8c, 0x4b, 0xf9, 0xbc, 0x0b, 0xb3, - 0x52, 0xa5, 0x99, 0xb3, 0xd3, 0xa5, 0x34, 0x13, 0x56, 0x5b, 0x95, 0xd3, 0x2c, 0xb9, 0xa6, 0xa3, - 0x65, 0xc8, 0x1a, 0xa6, 0xe3, 0xaa, 0xa4, 0x48, 0x7c, 0x17, 0x2a, 0xc7, 0x6e, 0x34, 0xa8, 0x71, - 0x72, 0x6d, 0x55, 0x06, 0x2f, 0x4b, 0x4d, 0x47, 0x35, 0xb8, 0x6c, 0xe2, 0x13, 0x57, 0xa1, 0x41, - 0xe3, 0x61, 0xd6, 0x34, 0x65, 0xbd, 0x72, 0x76, 0xba, 0x84, 0xb6, 0xf0, 0x89, 0xbb, 0x61, 0x1c, - 0xe3, 0x90, 0x08, 0x64, 0xf6, 0xd3, 0x74, 0xd2, 0x0b, 0x7c, 0x01, 0xec, 0x84, 0x8a, 0x06, 0xcc, - 0xcb, 0x39, 0x8f, 0xca, 0xce, 0x9d, 0xae, 0xc2, 0xb4, 0x83, 0x5f, 0x52, 0xa3, 0x63, 0x9a, 0x1a, - 0x1d, 0x53, 0x0e, 0xbd, 0xc9, 0x16, 0x49, 0xfe, 0x01, 0x17, 0xd6, 0x15, 0xd2, 0x49, 0x59, 0x20, - 0x52, 0xd6, 0x27, 0xca, 0xbb, 0xe8, 0x05, 0xf0, 0x7d, 0x11, 0x7f, 0x2f, 0xa6, 0x8d, 0x6d, 0xc3, - 0xf2, 0x6e, 0x87, 0x18, 0x6b, 0x7b, 0x67, 0x8e, 0x49, 0xe0, 0xda, 0xdd, 0xa1, 0xfc, 0xd2, 0x1f, - 0x0b, 0x30, 0xd7, 0xd3, 0xd6, 0xdc, 0x82, 0xf9, 0xa8, 0x77, 0x66, 0x0e, 0x9b, 0xf1, 0x94, 0xee, - 0x50, 0x5f, 0x00, 0xcd, 0xd2, 0xf9, 0x94, 0xd6, 0x33, 0x43, 0xa3, 0x45, 0xc8, 0x1e, 0xd8, 0xaa, - 0xe9, 0x55, 0x28, 0x41, 0x2b, 0x94, 0xe1, 0x24, 0x79, 0x17, 0x6d, 0x81, 0x48, 0x80, 0xf5, 0x51, - 0x13, 0x2b, 0x3a, 0x2f, 0x28, 0x9f, 0x3c, 0xc7, 0xaa, 0x49, 0x81, 0x33, 0x7b, 0x49, 0x64, 0x30, - 0xef, 0xab, 0xcd, 0xe6, 0x9e, 0xaa, 0x1d, 0x29, 0x74, 0xc7, 0x9b, 0x6e, 0x27, 0xca, 0x33, 0x1e, - 0x51, 0x56, 0x5d, 0x2c, 0x35, 0xa0, 0xb0, 0x66, 0x19, 0xe6, 0x96, 0xa5, 0xfb, 0x51, 0x54, 0x2b, - 0x90, 0xdf, 0x33, 0x4c, 0xd5, 0xee, 0x2a, 0x5e, 0x20, 0x94, 0x70, 0x5e, 0x20, 0x94, 0x9c, 0x63, - 0x1c, 0xfc, 0x51, 0xfa, 0xa9, 0x00, 0x62, 0x20, 0x96, 0x6b, 0xef, 0x6d, 0x00, 0xad, 0xd9, 0x71, - 0x5c, 0x6c, 0x7b, 0xd3, 0xf7, 0x0c, 0x0b, 0xb8, 0x2e, 0x33, 0x6a, 0x6d, 0x55, 0xce, 0xf0, 0x0c, - 0x35, 0x1d, 0xdd, 0xea, 0xbd, 0x16, 0x6e, 0xb2, 0x04, 0x67, 0x03, 0x97, 0xc1, 0x91, 0xf5, 0xc0, - 0x71, 0x2d, 0xdb, 0xef, 0xde, 0x7c, 0x3d, 0xf0, 0x2e, 0xcc, 0xa4, 0x17, 0x43, 0x61, 0x7a, 0xa7, - 0x42, 0x9e, 0x80, 0x93, 0x63, 0xec, 0x57, 0x29, 0x75, 0x7e, 0x95, 0x18, 0x87, 0x57, 0xa5, 0xdf, - 0x13, 0xa0, 0x50, 0x66, 0xd3, 0xb4, 0x3f, 0xf5, 0x8f, 0x30, 0x75, 0x56, 0x21, 0xed, 0x9e, 0x98, - 0x4a, 0x0b, 0xfb, 0xdf, 0x78, 0xbc, 0xc0, 0x8d, 0xd6, 0xd3, 0x2e, 0x7b, 0xa4, 0x9f, 0x0d, 0x7f, - 0x8d, 0xae, 0xe0, 0x33, 0x71, 0x94, 0xf1, 0x6f, 0x04, 0xc8, 0xd4, 0x35, 0xd5, 0xf4, 0x03, 0x55, - 0xcd, 0x4e, 0x8b, 0x9d, 0x29, 0xec, 0xab, 0x1a, 0x56, 0x1c, 0x8c, 0x8f, 0x1c, 0x7e, 0x11, 0xce, - 0xac, 0xd9, 0x69, 0xd5, 0xbc, 0x94, 0x3a, 0x49, 0x40, 0x6f, 0x03, 0xf2, 0xf3, 0x9b, 0x6a, 0x93, - 0x67, 0x67, 0x2b, 0x83, 0xe8, 0x65, 0x37, 0xd5, 0x26, 0xcb, 0x3d, 0x28, 0xdd, 0xc5, 0x6d, 0x6f, - 0x99, 0xe8, 0x95, 0x4e, 0x12, 0x06, 0xa5, 0xd3, 0xec, 0xa9, 0x41, 0xe9, 0x84, 0xce, 0xea, 0xf3, - 0xa0, 0x4e, 0x96, 0xf7, 0x01, 0x5c, 0x83, 0xf2, 0x00, 0xa1, 0x8f, 0x91, 0x4e, 0xd0, 0x4f, 0x94, - 0x56, 0x56, 0x56, 0x95, 0xdd, 0xad, 0xf2, 0xf6, 0xe6, 0x66, 0xad, 0xd1, 0xa8, 0xac, 0x8a, 0x02, - 0x12, 0x61, 0xa6, 0xe7, 0x53, 0xa6, 0x89, 0x85, 0xd4, 0xf7, 0x7e, 0x67, 0x71, 0xe2, 0xc1, 0x3b, - 0x90, 0xeb, 0xd9, 0x98, 0x46, 0x05, 0xc8, 0x6e, 0x54, 0x56, 0xea, 0x95, 0xea, 0xf6, 0xc6, 0x2a, - 0xdd, 0x56, 0xc9, 0xc2, 0xf4, 0x56, 0x65, 0x45, 0xae, 0xd4, 0x1b, 0xa2, 0xf0, 0xe0, 0xaf, 0x09, - 0x30, 0xc3, 0x2e, 0x0c, 0x93, 0xe9, 0x9d, 0x08, 0x08, 0x41, 0x5e, 0xae, 0xd4, 0x77, 0x37, 0x2b, - 0xca, 0xee, 0xd6, 0xfa, 0xd6, 0xf6, 0x8b, 0x2d, 0xaf, 0x04, 0x94, 0xb6, 0x5e, 0xf9, 0x44, 0xd9, - 0xa8, 0x6d, 0xd6, 0x1a, 0xa2, 0x80, 0x2e, 0xc3, 0x2c, 0xa7, 0x96, 0x3e, 0x69, 0x54, 0x38, 0x39, - 0x41, 0x3f, 0xc1, 0xca, 0xc8, 0xb5, 0x2d, 0xfa, 0x49, 0x54, 0x96, 0x90, 0x44, 0xd7, 0xe0, 0x32, - 0x4f, 0x60, 0x5f, 0x4d, 0x2d, 0x6d, 0xef, 0x6e, 0xad, 0xae, 0xc8, 0x9f, 0x88, 0x29, 0x5e, 0xf4, - 0x3f, 0x07, 0x10, 0x7c, 0xd0, 0x99, 0xa8, 0x81, 0xbc, 0xed, 0xf9, 0xca, 0xc6, 0x6e, 0xa5, 0x2e, - 0x4e, 0x90, 0x82, 0x95, 0x56, 0x1a, 0xe5, 0xaa, 0x22, 0x57, 0xea, 0x3b, 0xdb, 0x5b, 0xf5, 0x8a, - 0x28, 0x70, 0xbe, 0x55, 0x98, 0x09, 0x5f, 0x99, 0x8a, 0xe6, 0xa0, 0x50, 0xae, 0x56, 0xca, 0xeb, - 0xca, 0xf3, 0xda, 0x8a, 0xf2, 0x6c, 0xb7, 0xb2, 0x5b, 0x11, 0x27, 0xa8, 0x56, 0x29, 0xf1, 0xc9, - 0xee, 0xc6, 0x86, 0x28, 0x10, 0xb5, 0xb0, 0x67, 0xfa, 0xc5, 0x56, 0x31, 0xf1, 0x60, 0x13, 0xb2, - 0xa1, 0x4f, 0xb9, 0x90, 0xd7, 0xed, 0xec, 0xd6, 0xab, 0x4a, 0xa3, 0xb6, 0x59, 0xa9, 0x37, 0x56, - 0x36, 0x77, 0x98, 0x0c, 0x4a, 0x5b, 0x29, 0x6d, 0xcb, 0x44, 0x03, 0xde, 0x73, 0x63, 0x7b, 0xb7, - 0x5c, 0xf5, 0x5a, 0x40, 0x4a, 0xa5, 0x93, 0x62, 0xf2, 0xc1, 0x09, 0x5c, 0x1d, 0x72, 0x7b, 0x28, - 0x69, 0x80, 0x5d, 0x93, 0x7e, 0xd6, 0x42, 0x9c, 0x40, 0x39, 0xc8, 0x90, 0x59, 0x80, 0x5e, 0xc8, - 0x23, 0x0a, 0x28, 0x0d, 0xa9, 0x43, 0xd7, 0x6d, 0x8b, 0x09, 0x34, 0x05, 0x09, 0xe7, 0xb1, 0x98, - 0x24, 0xff, 0x0f, 0x1c, 0x31, 0x85, 0x32, 0x30, 0xa9, 0x7e, 0xd6, 0xb1, 0xb1, 0x38, 0x89, 0x66, - 0x20, 0xdd, 0x71, 0xb0, 0xbd, 0x6f, 0x34, 0xb1, 0x38, 0x4d, 0x58, 0xcc, 0x4e, 0xb3, 0x29, 0xa6, - 0xa5, 0x54, 0x7a, 0x4a, 0x9c, 0x7a, 0x70, 0x13, 0x42, 0x17, 0xb6, 0x21, 0x80, 0xa9, 0x0d, 0xd5, - 0xc5, 0x8e, 0x2b, 0x4e, 0xa0, 0x69, 0x48, 0xae, 0x34, 0x9b, 0xa2, 0xf0, 0xe8, 0x7f, 0x4e, 0x41, - 0xda, 0xeb, 0x91, 0x68, 0x03, 0x26, 0xe9, 0x1e, 0x10, 0x5a, 0x1a, 0xbe, 0x3b, 0x44, 0xa7, 0xd4, - 0x85, 0x1b, 0xe7, 0x6d, 0x1f, 0x49, 0x13, 0xe8, 0xcf, 0x43, 0x36, 0x84, 0x9a, 0xd1, 0xd0, 0x43, - 0xd6, 0x9e, 0x9d, 0x82, 0x85, 0x3b, 0xe7, 0x65, 0xf3, 0xe5, 0xbf, 0x80, 0x8c, 0x6f, 0x70, 0xa3, - 0x5b, 0xa3, 0xcc, 0x71, 0x4f, 0xf6, 0x68, 0x9b, 0x9d, 0xcc, 0x80, 0xd2, 0xc4, 0xbb, 0x02, 0xb2, - 0x01, 0x0d, 0xda, 0xc6, 0x28, 0x2a, 0xea, 0x62, 0xa8, 0xf1, 0xbd, 0xf0, 0x60, 0xac, 0xdc, 0xc1, - 0x3b, 0x89, 0xb2, 0x02, 0x03, 0x3f, 0x5a, 0x59, 0x03, 0xf0, 0x21, 0x5a, 0x59, 0x11, 0x38, 0x81, - 0x36, 0x46, 0xc8, 0x00, 0x88, 0x94, 0x3f, 0x68, 0x0c, 0x46, 0xca, 0x8f, 0xb0, 0x23, 0xa4, 0x09, - 0xf4, 0x0c, 0x52, 0x64, 0x7d, 0x44, 0x51, 0xd0, 0xbb, 0x6f, 0x3d, 0x5e, 0xb8, 0x35, 0x32, 0x8f, - 0x2f, 0xf2, 0x00, 0xf2, 0x4f, 0x31, 0x3d, 0xb8, 0x61, 0xb7, 0xaf, 0x3a, 0xe8, 0x5e, 0xb4, 0x2b, - 0x6c, 0x28, 0x8b, 0xf7, 0x8a, 0xfb, 0x63, 0xe4, 0xf4, 0x5f, 0xd4, 0x86, 0x59, 0xf6, 0xdd, 0x9a, - 0xf0, 0xbb, 0xa2, 0xfc, 0x6f, 0x06, 0x72, 0x79, 0xaf, 0x7b, 0x7b, 0xbc, 0xcc, 0xde, 0x1b, 0x4b, - 0xf7, 0x7f, 0xf2, 0x9f, 0x16, 0x27, 0x7e, 0x72, 0xb6, 0x28, 0xfc, 0xf4, 0x6c, 0x51, 0xf8, 0xc3, - 0xb3, 0x45, 0xe1, 0x8f, 0xce, 0x16, 0x85, 0xef, 0xff, 0x7c, 0x71, 0xe2, 0xa7, 0x3f, 0x5f, 0x9c, - 0xf8, 0xc3, 0x9f, 0x2f, 0x4e, 0x7c, 0x3a, 0xcd, 0xc5, 0xec, 0x4d, 0xd1, 0x75, 0xf1, 0xf1, 0xff, - 0x0b, 0x00, 0x00, 0xff, 0xff, 0x72, 0x01, 0x4c, 0x4f, 0x62, 0x8f, 0x00, 0x00, + // 9497 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, 0xdd, 0x6a, 0xb5, 0xda, 0x96, 0xba, + 0xab, 0xdd, 0xbf, 0x6b, 0x4b, 0xee, 0x6e, 0x3b, 0xf6, 0xda, 0x1e, 0xcf, 0x8a, 0x14, 0xbb, 0x49, + 0xfd, 0x77, 0x91, 0xea, 0x86, 0xbd, 0x33, 0xa9, 0x2d, 0x55, 0x5d, 0x49, 0x35, 0x22, 0xab, 0xd8, + 0x55, 0x45, 0xb5, 0x68, 0x20, 0x0f, 0x49, 0x36, 0x93, 0xc9, 0x4b, 0x30, 0x01, 0x02, 0xec, 0x0c, + 0x36, 0x58, 0x78, 0xb3, 0x8b, 0xe4, 0x21, 0x01, 0x12, 0x24, 0x41, 0x82, 0x0d, 0x92, 0x9d, 0xc7, + 0x18, 0xc1, 0x26, 0x99, 0x79, 0x5b, 0x04, 0x88, 0xb2, 0xd1, 0xec, 0xc3, 0x0e, 0x82, 0x20, 0x41, + 0x12, 0x60, 0x01, 0x03, 0xf9, 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, + 0xb4, 0x6d, 0xa9, 0xda, 0x61, 0x6b, 0x6f, 0x59, 0x6d, 0x19, 0x4b, 0x2d, 0xdb, 0x72, 0x2d, 0x34, + 0xad, 0x59, 0xda, 0x11, 0x25, 0x2f, 0xf1, 0xc4, 0xf9, 0xcb, 0xd8, 0xb6, 0x2d, 0xdb, 0x69, 0xed, + 0x2d, 0xb3, 0x1f, 0x2c, 0xe7, 0xfc, 0xfd, 0xa3, 0xe3, 0xe5, 0xa3, 0x63, 0x07, 0xdb, 0xc7, 0xd8, + 0x5e, 0xd6, 0x2c, 0x53, 0x6b, 0xdb, 0x36, 0x36, 0xb5, 0xce, 0x72, 0xc3, 0xd2, 0x8e, 0xe8, 0x1f, + 0xc3, 0x3c, 0x88, 0xca, 0x6b, 0x63, 0x55, 0x77, 0xda, 0xcd, 0xa6, 0x6a, 0x77, 0x96, 0xa9, 0x58, + 0xfe, 0xc0, 0xf3, 0x22, 0xaf, 0x50, 0xba, 0xea, 0xaa, 0x9c, 0x76, 0xc9, 0xa3, 0x75, 0x95, 0x60, + 0xd6, 0xa3, 0x36, 0xb1, 0xab, 0x86, 0x72, 0x5f, 0xf5, 0xe8, 0x4e, 0x4b, 0x35, 0x15, 0xcd, 0x32, + 0xf7, 0x0d, 0xaf, 0x20, 0xd7, 0x1c, 0xd7, 0xb2, 0xd5, 0x03, 0xbc, 0x8c, 0xcd, 0x03, 0xc3, 0xc4, + 0x84, 0xf7, 0x58, 0xd3, 0x78, 0xe2, 0x6b, 0x91, 0x89, 0x8f, 0x78, 0xea, 0x5c, 0xdb, 0x35, 0x1a, + 0xcb, 0x87, 0x0d, 0x6d, 0xd9, 0x35, 0x9a, 0xd8, 0x71, 0xd5, 0x66, 0xcb, 0xab, 0x1d, 0x4d, 0x71, + 0x6d, 0x55, 0x33, 0xcc, 0x03, 0xef, 0x7f, 0x6b, 0x6f, 0xd9, 0xc6, 0x9a, 0x65, 0xeb, 0x58, 0x57, + 0x48, 0x39, 0xbc, 0x9a, 0x1c, 0x58, 0x07, 0x16, 0xfd, 0xb9, 0x4c, 0x7e, 0x71, 0xea, 0xc2, 0x81, + 0x65, 0x1d, 0x34, 0xf0, 0x32, 0x7d, 0xda, 0x6b, 0xef, 0x2f, 0xeb, 0x6d, 0x5b, 0x75, 0x0d, 0x8b, + 0x73, 0x49, 0xff, 0x54, 0x80, 0x9c, 0x8c, 0x5f, 0xb4, 0xb1, 0xe3, 0x56, 0xb0, 0xaa, 0x63, 0x1b, + 0x5d, 0x85, 0xe4, 0x11, 0xee, 0xcc, 0x25, 0xaf, 0x0b, 0x77, 0xa7, 0x8a, 0x93, 0x5f, 0x9e, 0x2e, + 0x26, 0xd7, 0x71, 0x47, 0x26, 0x34, 0x74, 0x1d, 0x26, 0xb1, 0xa9, 0x2b, 0x24, 0x39, 0xd5, 0x9d, + 0x3c, 0x81, 0x4d, 0x7d, 0x1d, 0x77, 0xd0, 0x77, 0x20, 0xed, 0x10, 0x69, 0xa6, 0x86, 0xe7, 0xc6, + 0xaf, 0x0b, 0x77, 0xc7, 0x8b, 0xbf, 0xf2, 0xe5, 0xe9, 0xe2, 0x47, 0x07, 0x86, 0x7b, 0xd8, 0xde, + 0x5b, 0xd2, 0xac, 0xe6, 0xb2, 0xdf, 0x0b, 0xf4, 0xbd, 0xe0, 0xf7, 0x72, 0xeb, 0xe8, 0x60, 0xb9, + 0x57, 0x47, 0x4b, 0xf5, 0x13, 0xb3, 0x86, 0x5f, 0xc8, 0xbe, 0xc4, 0xb5, 0x54, 0x5a, 0x10, 0x13, + 0x6b, 0xa9, 0x74, 0x42, 0x4c, 0x4a, 0x7f, 0x3f, 0x01, 0x79, 0x19, 0x3b, 0x2d, 0xcb, 0x74, 0x30, + 0x2f, 0xf9, 0xdb, 0x90, 0x74, 0x4f, 0x4c, 0x5a, 0xf2, 0xec, 0xc3, 0x85, 0xa5, 0xbe, 0xfe, 0xb6, + 0x54, 0xb7, 0x55, 0xd3, 0x51, 0x35, 0x52, 0x7d, 0x99, 0x64, 0x45, 0xef, 0x43, 0xd6, 0xc6, 0x4e, + 0xbb, 0x89, 0xa9, 0x22, 0x69, 0xa5, 0xb2, 0x0f, 0xaf, 0x44, 0x70, 0xd6, 0x5a, 0xaa, 0x29, 0x03, + 0xcb, 0x4b, 0x7e, 0xa3, 0x55, 0xc8, 0x71, 0x4e, 0x1b, 0xab, 0x8e, 0x65, 0xce, 0x4d, 0x5e, 0x17, + 0xee, 0xe6, 0x1f, 0x2e, 0x46, 0xf0, 0xca, 0x34, 0x9f, 0x4c, 0xb3, 0xc9, 0x53, 0x76, 0xe8, 0x09, + 0xdd, 0x87, 0x69, 0x2e, 0xc5, 0xc4, 0x27, 0xae, 0xb2, 0xd7, 0x71, 0xb1, 0x33, 0x97, 0xb9, 0x2e, + 0xdc, 0x4d, 0xca, 0x05, 0x96, 0xb0, 0x85, 0x4f, 0xdc, 0x22, 0x21, 0xa3, 0xab, 0x90, 0x36, 0xdb, + 0x4d, 0xa2, 0x7c, 0x87, 0xaa, 0x36, 0x29, 0x4f, 0x9a, 0xed, 0xe6, 0x3a, 0xee, 0x38, 0xe8, 0x1a, + 0x64, 0x48, 0x12, 0x63, 0x4f, 0xd3, 0x34, 0x92, 0x97, 0xf2, 0xad, 0xa5, 0xd2, 0x13, 0xe2, 0xa4, + 0xf4, 0x3b, 0x02, 0xc0, 0x13, 0xec, 0xf2, 0xa6, 0x46, 0x45, 0x98, 0x38, 0xa4, 0x4a, 0x9b, 0x13, + 0x68, 0x9d, 0xaf, 0x47, 0x96, 0x3b, 0xd4, 0x2d, 0x8a, 0xe9, 0x2f, 0x4e, 0x17, 0xc7, 0x7e, 0x7a, + 0xba, 0x28, 0xc8, 0x9c, 0x13, 0x3d, 0x85, 0xec, 0x11, 0xee, 0x28, 0x7c, 0x3c, 0xce, 0x25, 0xa8, + 0x02, 0xde, 0x0e, 0x09, 0x3a, 0x3a, 0x5e, 0xf2, 0x86, 0xe6, 0x52, 0x68, 0x18, 0x2f, 0x11, 0x8e, + 0xa5, 0x9a, 0x6b, 0x63, 0xf3, 0xc0, 0x3d, 0x94, 0xe1, 0x08, 0x77, 0x36, 0x98, 0x0c, 0xe9, 0x27, + 0x02, 0x64, 0x69, 0x29, 0x59, 0xbb, 0xa2, 0x52, 0x4f, 0x31, 0x6f, 0x44, 0xab, 0x37, 0xd4, 0x09, + 0x22, 0xca, 0xb9, 0x04, 0xe3, 0xc7, 0x6a, 0xa3, 0x8d, 0x69, 0x09, 0xb3, 0x0f, 0xe7, 0x22, 0x64, + 0x3c, 0x23, 0xe9, 0x32, 0xcb, 0x86, 0x3e, 0x84, 0x29, 0xc3, 0x74, 0xb1, 0xe9, 0x2a, 0x8c, 0x2d, + 0x79, 0x0e, 0x5b, 0x96, 0xe5, 0xa6, 0x0f, 0xd2, 0x3f, 0x11, 0x00, 0x76, 0xda, 0xb1, 0xea, 0xf9, + 0x9d, 0x11, 0xcb, 0x5f, 0x4c, 0x11, 0x56, 0xaf, 0x16, 0xb3, 0x30, 0x61, 0x98, 0x0d, 0xc3, 0x64, + 0xe5, 0x4f, 0xcb, 0xfc, 0x09, 0x5d, 0x82, 0xf1, 0xbd, 0x86, 0x61, 0xea, 0xb4, 0xb3, 0xa7, 0x65, + 0xf6, 0x20, 0xc9, 0x90, 0xa5, 0xa5, 0x8e, 0x51, 0xef, 0xd2, 0x69, 0x02, 0x2e, 0x97, 0x2c, 0x53, + 0x37, 0xc8, 0x78, 0x53, 0x1b, 0xdf, 0x08, 0xad, 0xac, 0xc1, 0x25, 0x1d, 0xb7, 0x6c, 0xac, 0xa9, + 0x2e, 0xd6, 0x15, 0x7c, 0xd2, 0x1a, 0xb1, 0x8d, 0x51, 0xc0, 0x55, 0x3e, 0x69, 0x51, 0x1a, 0x19, + 0x75, 0x44, 0x00, 0x1b, 0x75, 0x13, 0x64, 0x3e, 0x94, 0xd3, 0xf8, 0xa4, 0xc5, 0x46, 0x6b, 0xa4, + 0x9a, 0xd1, 0x3b, 0x70, 0x45, 0x6d, 0x34, 0xac, 0x97, 0x8a, 0xb1, 0xaf, 0xe8, 0x16, 0x76, 0x14, + 0xd3, 0x72, 0x15, 0x7c, 0x62, 0x38, 0x2e, 0x1d, 0xd2, 0x69, 0x79, 0x86, 0x26, 0x57, 0xf7, 0x57, + 0x2d, 0xec, 0x6c, 0x59, 0x6e, 0x99, 0x24, 0x85, 0x9a, 0x72, 0x32, 0xdc, 0x94, 0xd2, 0x77, 0x61, + 0xb6, 0x57, 0xbf, 0x71, 0xb6, 0xdf, 0x1f, 0x08, 0x90, 0xaf, 0x9a, 0x86, 0xfb, 0x8d, 0x68, 0x38, + 0x5f, 0x9f, 0xc9, 0xb0, 0x3e, 0xef, 0x83, 0xb8, 0xaf, 0x1a, 0x8d, 0x6d, 0xb3, 0x6e, 0x35, 0xf7, + 0x1c, 0xd7, 0x32, 0xb1, 0xc3, 0x15, 0xde, 0x47, 0x97, 0x9e, 0x41, 0xc1, 0xaf, 0x4d, 0x9c, 0x6a, + 0x72, 0x41, 0xac, 0x9a, 0x9a, 0x8d, 0x9b, 0xd8, 0x8c, 0x55, 0x4f, 0xaf, 0x41, 0xc6, 0xf0, 0xe4, + 0x52, 0x5d, 0x25, 0xe5, 0x80, 0x20, 0xb5, 0x61, 0x3a, 0xf4, 0xd6, 0x38, 0xa7, 0x4b, 0xb2, 0x98, + 0xe0, 0x97, 0x4a, 0xd0, 0x46, 0x64, 0x31, 0xc1, 0x2f, 0xd9, 0xf4, 0x56, 0x83, 0xdc, 0x2a, 0x6e, + 0x60, 0x17, 0xc7, 0x58, 0x53, 0x69, 0x17, 0xf2, 0x9e, 0xd0, 0x38, 0x1b, 0xe6, 0x37, 0x04, 0x40, + 0x5c, 0xae, 0x6a, 0x1e, 0xc4, 0x59, 0x62, 0xb4, 0x48, 0xec, 0x06, 0xb7, 0x6d, 0x9b, 0x6c, 0x39, + 0x66, 0x7d, 0x12, 0x18, 0x89, 0xae, 0xc8, 0xc1, 0x90, 0x4d, 0x85, 0x87, 0x2c, 0xb7, 0x5d, 0x5e, + 0xc2, 0x4c, 0x57, 0xc1, 0xe2, 0x6d, 0xbe, 0x14, 0x2d, 0x53, 0xe2, 0x7a, 0x32, 0x6c, 0xa0, 0x51, + 0xa2, 0xf4, 0x23, 0x01, 0xa6, 0x4b, 0x0d, 0xac, 0xda, 0xb1, 0x6b, 0xe4, 0xdb, 0x90, 0xd6, 0xb1, + 0xaa, 0xd3, 0x2a, 0xb3, 0x81, 0xfd, 0x7a, 0x48, 0x0a, 0x31, 0x63, 0x97, 0x0e, 0x1b, 0xda, 0x52, + 0xdd, 0x33, 0x70, 0xf9, 0xe8, 0xf6, 0x99, 0xa4, 0x4f, 0x00, 0x85, 0x4b, 0x16, 0x67, 0x47, 0xf8, + 0xdd, 0x04, 0x20, 0x19, 0x1f, 0x63, 0xdb, 0x8d, 0xbd, 0xda, 0xab, 0x90, 0x75, 0x55, 0xfb, 0x00, + 0xbb, 0x0a, 0x31, 0xdd, 0x2f, 0x52, 0x73, 0x60, 0x7c, 0x84, 0x8c, 0xea, 0x70, 0x07, 0x9b, 0xea, + 0x5e, 0x03, 0x53, 0x29, 0xca, 0x9e, 0xd5, 0x36, 0x75, 0xc5, 0x70, 0xb1, 0xad, 0xba, 0x96, 0xad, + 0x58, 0x2d, 0xd7, 0x68, 0x1a, 0x9f, 0x51, 0xab, 0x9d, 0x77, 0xb5, 0x9b, 0x2c, 0x3b, 0x61, 0x2e, + 0x92, 0xcc, 0x55, 0x9e, 0x77, 0x3b, 0x94, 0x15, 0x2d, 0xc1, 0x8c, 0x71, 0x60, 0x5a, 0x36, 0x56, + 0x0e, 0x34, 0xc5, 0x3d, 0xb4, 0xb1, 0x73, 0x68, 0x35, 0xbc, 0x05, 0x69, 0x9a, 0x25, 0x3d, 0xd1, + 0xea, 0x5e, 0x82, 0xf4, 0x29, 0xcc, 0x74, 0x69, 0x29, 0xce, 0x26, 0xf8, 0x6f, 0x02, 0x64, 0x6b, + 0x9a, 0x6a, 0xc6, 0xa9, 0xfb, 0x8f, 0x21, 0xeb, 0x68, 0xaa, 0xa9, 0xec, 0x5b, 0x76, 0x53, 0x75, + 0x69, 0xbd, 0xf2, 0x5d, 0xba, 0xf7, 0x8d, 0x77, 0x4d, 0x35, 0x1f, 0xd3, 0x4c, 0x32, 0x38, 0xfe, + 0xef, 0x5e, 0xfb, 0x75, 0xfc, 0xab, 0xdb, 0xaf, 0x6c, 0x78, 0xaf, 0xa5, 0xd2, 0x49, 0x31, 0x25, + 0xfd, 0xa9, 0x00, 0x53, 0xac, 0xca, 0x71, 0x0e, 0xef, 0x77, 0x21, 0x65, 0x5b, 0x2f, 0xd9, 0xf0, + 0xce, 0x3e, 0xbc, 0x16, 0x21, 0x62, 0x1d, 0x77, 0xc2, 0xeb, 0x27, 0xcd, 0x8e, 0x8a, 0xc0, 0xad, + 0x54, 0x85, 0x72, 0x27, 0x47, 0xe5, 0x06, 0xc6, 0x25, 0x13, 0x19, 0x77, 0xa0, 0xb0, 0xa7, 0xba, + 0xda, 0xa1, 0x62, 0xf3, 0x42, 0x92, 0xb5, 0x36, 0x79, 0x77, 0x4a, 0xce, 0x53, 0xb2, 0x57, 0x74, + 0x87, 0xd4, 0x9c, 0x8d, 0x37, 0x07, 0xff, 0x19, 0x6b, 0xf3, 0xff, 0x2d, 0xf0, 0x31, 0xe4, 0xd5, + 0xfc, 0xcf, 0x5a, 0xd3, 0xff, 0x38, 0x01, 0x57, 0x4a, 0x87, 0x58, 0x3b, 0x2a, 0x59, 0xa6, 0x63, + 0x38, 0x2e, 0xd1, 0x5d, 0x9c, 0xed, 0x7f, 0x0d, 0x32, 0x2f, 0x0d, 0xf7, 0x50, 0xd1, 0x8d, 0xfd, + 0x7d, 0x3a, 0xdb, 0xa6, 0xe5, 0x34, 0x21, 0xac, 0x1a, 0xfb, 0xfb, 0xe8, 0x11, 0xa4, 0x9a, 0x96, + 0xce, 0x8c, 0xf9, 0x68, 0x28, 0x4e, 0x8b, 0xe6, 0xb4, 0x9b, 0x9b, 0x96, 0x8e, 0x65, 0x9a, 0x19, + 0x2d, 0x00, 0x68, 0x84, 0xda, 0xb2, 0x0c, 0xd3, 0xe5, 0x93, 0x63, 0x88, 0x82, 0x2a, 0x90, 0x71, + 0xb1, 0xdd, 0x34, 0x4c, 0xd5, 0xc5, 0x73, 0xe3, 0x54, 0x79, 0x6f, 0x44, 0x16, 0xbc, 0xd5, 0x30, + 0x34, 0x75, 0x15, 0x3b, 0x9a, 0x6d, 0xb4, 0x5c, 0xcb, 0xe6, 0x5a, 0x0c, 0x98, 0xa5, 0xbf, 0x9e, + 0x82, 0xb9, 0x7e, 0xdd, 0xc4, 0xd9, 0x43, 0x76, 0x60, 0xc2, 0xc6, 0x4e, 0xbb, 0xe1, 0xf2, 0x3e, + 0xf2, 0x70, 0x90, 0x0a, 0x22, 0x4a, 0x40, 0xb7, 0x29, 0x1a, 0x2e, 0x2f, 0x36, 0x97, 0x33, 0xff, + 0x2f, 0x04, 0x98, 0x60, 0x09, 0xe8, 0x01, 0xa4, 0x6d, 0xb2, 0x30, 0x28, 0x86, 0x4e, 0xcb, 0x98, + 0x2c, 0xce, 0x9e, 0x9d, 0x2e, 0x4e, 0xd2, 0xc5, 0xa2, 0xba, 0xfa, 0x65, 0xf0, 0x53, 0x9e, 0xa4, + 0xf9, 0xaa, 0x3a, 0x69, 0x2d, 0xc7, 0x55, 0x6d, 0x97, 0xee, 0x18, 0x25, 0x18, 0x42, 0xa2, 0x84, + 0x75, 0xdc, 0x41, 0x6b, 0x30, 0xe1, 0xb8, 0xaa, 0xdb, 0x76, 0x78, 0x7b, 0x5d, 0xa8, 0xb0, 0x35, + 0xca, 0x29, 0x73, 0x09, 0xc4, 0xdc, 0xd2, 0xb1, 0xab, 0x1a, 0x0d, 0xda, 0x80, 0x19, 0x99, 0x3f, + 0x49, 0xbf, 0x29, 0xc0, 0x04, 0xcb, 0x8a, 0xae, 0xc0, 0x8c, 0xbc, 0xb2, 0xf5, 0xa4, 0xac, 0x54, + 0xb7, 0x56, 0xcb, 0xf5, 0xb2, 0xbc, 0x59, 0xdd, 0x5a, 0xa9, 0x97, 0xc5, 0x31, 0x34, 0x0b, 0xc8, + 0x4b, 0x28, 0x6d, 0x6f, 0xd5, 0xaa, 0xb5, 0x7a, 0x79, 0xab, 0x2e, 0x0a, 0xe8, 0x12, 0x88, 0x8c, + 0x1e, 0xa2, 0x26, 0xd0, 0x1b, 0x70, 0xbd, 0x97, 0xaa, 0xd4, 0xea, 0x2b, 0xf5, 0x9a, 0x52, 0xae, + 0xd5, 0xab, 0x9b, 0x2b, 0xf5, 0xf2, 0xaa, 0x98, 0x1c, 0x92, 0x8b, 0xbc, 0x44, 0x96, 0xcb, 0xa5, + 0xba, 0x98, 0x92, 0x5c, 0xb8, 0x2c, 0x63, 0xcd, 0x6a, 0xb6, 0xda, 0x2e, 0x26, 0xa5, 0x74, 0xe2, + 0x1c, 0x29, 0x57, 0x60, 0x52, 0xb7, 0x3b, 0x8a, 0xdd, 0x36, 0xf9, 0x38, 0x99, 0xd0, 0xed, 0x8e, + 0xdc, 0x36, 0xa5, 0x7f, 0x28, 0xc0, 0x6c, 0xef, 0x6b, 0xe3, 0xec, 0x84, 0x4f, 0x21, 0xab, 0xea, + 0x3a, 0xd6, 0x15, 0x1d, 0x37, 0x5c, 0x95, 0x9b, 0x44, 0xf7, 0x43, 0x92, 0xf8, 0x3e, 0xdf, 0x92, + 0xbf, 0xcf, 0xb7, 0xf9, 0xac, 0x54, 0xa2, 0x05, 0x59, 0x25, 0x1c, 0xde, 0xf4, 0x43, 0x85, 0x50, + 0x8a, 0xf4, 0xe3, 0x14, 0xe4, 0xca, 0xa6, 0x5e, 0x3f, 0x89, 0x75, 0x2d, 0x99, 0x85, 0x09, 0xcd, + 0x6a, 0x36, 0x0d, 0xd7, 0x53, 0x10, 0x7b, 0x42, 0xbf, 0x1c, 0x32, 0x65, 0x93, 0x23, 0x18, 0x74, + 0x81, 0x11, 0x8b, 0x7e, 0x0d, 0xae, 0x90, 0x59, 0xd3, 0x36, 0xd5, 0x86, 0xc2, 0xa4, 0x29, 0xae, + 0x6d, 0x1c, 0x1c, 0x60, 0x9b, 0xef, 0x2d, 0xde, 0x8d, 0x28, 0x67, 0x95, 0x73, 0x94, 0x28, 0x43, + 0x9d, 0xe5, 0x97, 0x2f, 0x1b, 0x51, 0x64, 0xf4, 0x11, 0x00, 0x59, 0x8a, 0xe8, 0x7e, 0xa5, 0xc3, + 0xe7, 0xa3, 0x41, 0x1b, 0x96, 0xde, 0x14, 0x44, 0x18, 0xc8, 0xb3, 0x83, 0x9e, 0x82, 0x68, 0x98, + 0xca, 0x7e, 0xc3, 0x38, 0x38, 0x74, 0x95, 0x97, 0xb6, 0xe1, 0x62, 0x67, 0x6e, 0x9a, 0xca, 0x88, + 0x6a, 0xea, 0x1a, 0xdf, 0x77, 0xd5, 0x9f, 0x93, 0x9c, 0x5c, 0x5a, 0xde, 0x30, 0x1f, 0x53, 0x7e, + 0x4a, 0x74, 0xd0, 0x32, 0x81, 0x42, 0x2f, 0xda, 0x86, 0x8d, 0x95, 0x07, 0x2d, 0x8d, 0xee, 0x83, + 0xa4, 0x8b, 0xf9, 0xb3, 0xd3, 0x45, 0x90, 0x19, 0xf9, 0xc1, 0x4e, 0x89, 0x40, 0x23, 0xf6, 0xbb, + 0xa5, 0x11, 0xb5, 0xb7, 0x2c, 0xc3, 0xb1, 0x4c, 0xba, 0xd1, 0x99, 0x96, 0xf9, 0x13, 0xba, 0x07, + 0xa2, 0x7b, 0x62, 0x2a, 0x87, 0x58, 0xb5, 0xdd, 0x3d, 0xac, 0xba, 0x64, 0x7d, 0x06, 0x9a, 0xa3, + 0xe0, 0x9e, 0x98, 0x95, 0x10, 0x79, 0x2d, 0x95, 0x9e, 0x14, 0xd3, 0x6b, 0xa9, 0x74, 0x5a, 0xcc, + 0x48, 0xff, 0x41, 0x80, 0xbc, 0xd7, 0x37, 0xe2, 0xec, 0xc6, 0x77, 0x41, 0xb4, 0x4c, 0xac, 0xb4, + 0x0e, 0x55, 0x07, 0xf3, 0xb6, 0xe4, 0xab, 0x43, 0xde, 0x32, 0xf1, 0x0e, 0x21, 0xb3, 0x96, 0x41, + 0x3b, 0x30, 0xed, 0xb8, 0xea, 0x81, 0x61, 0x1e, 0x28, 0xfe, 0xfe, 0x3d, 0xb5, 0x2c, 0x46, 0x44, + 0x02, 0x22, 0xe7, 0xf6, 0xe9, 0x5d, 0x26, 0xc5, 0xcf, 0x04, 0x98, 0x5e, 0xd1, 0x9b, 0x86, 0x59, + 0x6b, 0x35, 0x8c, 0x58, 0x37, 0x18, 0xde, 0x80, 0x8c, 0x43, 0x64, 0x06, 0xb3, 0x73, 0x00, 0x17, + 0xd3, 0x34, 0x85, 0x4c, 0xd3, 0x1b, 0x50, 0xc0, 0x27, 0x2d, 0x83, 0x1d, 0x1a, 0x30, 0x94, 0x93, + 0x1a, 0xbd, 0x6e, 0xf9, 0x80, 0x97, 0x24, 0xf1, 0x3a, 0x7d, 0x02, 0x28, 0x5c, 0xa5, 0x38, 0x81, + 0xc6, 0x27, 0x30, 0x43, 0x45, 0xef, 0x9a, 0x4e, 0xcc, 0xfa, 0x92, 0x7e, 0x15, 0x2e, 0x75, 0x8b, + 0x8e, 0xb3, 0xdc, 0xcf, 0x79, 0x2b, 0x6f, 0x62, 0x3b, 0x56, 0x84, 0xea, 0xeb, 0x9a, 0x0b, 0x8e, + 0xb3, 0xcc, 0xbf, 0x2e, 0xc0, 0x55, 0x2a, 0x9b, 0x9e, 0xab, 0xec, 0x63, 0x7b, 0x03, 0xab, 0x4e, + 0xac, 0xf0, 0xfa, 0x26, 0x4c, 0x30, 0x98, 0x4c, 0xfb, 0xe7, 0x78, 0x31, 0x4b, 0xcc, 0x8c, 0x9a, + 0x6b, 0xd9, 0xc4, 0xcc, 0xe0, 0x49, 0x92, 0x0a, 0xf3, 0x51, 0xa5, 0x88, 0xb3, 0xa6, 0xbf, 0x25, + 0xc0, 0x34, 0xb7, 0xf0, 0x48, 0x57, 0x2e, 0x1d, 0x12, 0x03, 0x07, 0x95, 0x21, 0xab, 0xd1, 0x5f, + 0x8a, 0xdb, 0x69, 0x61, 0x2a, 0x3f, 0x3f, 0xcc, 0x38, 0x64, 0x6c, 0xf5, 0x4e, 0x0b, 0x13, 0x0b, + 0xd3, 0xfb, 0x4d, 0x14, 0x15, 0xaa, 0xe4, 0x50, 0xf3, 0x92, 0x8e, 0x23, 0x9a, 0xd7, 0xb3, 0xd3, + 0xb8, 0x0e, 0xfe, 0x71, 0x92, 0x2b, 0x81, 0xbd, 0x83, 0x67, 0x8f, 0xd5, 0xa0, 0xf8, 0x14, 0x66, + 0x43, 0x5b, 0xe7, 0xe1, 0x8a, 0x27, 0x2e, 0x50, 0xf1, 0xd0, 0xf6, 0x7b, 0x40, 0x45, 0x9f, 0x40, + 0x68, 0x83, 0x5d, 0x61, 0x75, 0xf2, 0xa0, 0xca, 0x45, 0xd4, 0x31, 0x1d, 0x48, 0x61, 0x74, 0x07, + 0x95, 0x20, 0x8d, 0x4f, 0x5a, 0x8a, 0x8e, 0x1d, 0x8d, 0x4f, 0x5c, 0x52, 0x94, 0x40, 0x52, 0x94, + 0x3e, 0xe3, 0x7d, 0x12, 0x9f, 0xb4, 0x08, 0x11, 0xed, 0x92, 0x75, 0xd3, 0x5b, 0xd7, 0x69, 0xb1, + 0x9d, 0xf3, 0xb1, 0x40, 0xd0, 0x53, 0xb8, 0xb8, 0x82, 0xbf, 0xa4, 0x33, 0x11, 0xd2, 0xe7, 0x02, + 0x5c, 0x8b, 0x6c, 0xb5, 0x38, 0x17, 0xb2, 0x8f, 0x20, 0x45, 0x2b, 0x9f, 0xb8, 0x60, 0xe5, 0x29, + 0x97, 0xf4, 0x83, 0x04, 0x1f, 0xe3, 0x32, 0x6e, 0x58, 0x44, 0xb1, 0xb1, 0x6f, 0xa1, 0x6d, 0x43, + 0xee, 0xd8, 0x72, 0xb1, 0xed, 0x37, 0x7b, 0xe2, 0xc2, 0xcd, 0x3e, 0x45, 0x05, 0x78, 0x2d, 0xfe, + 0x0c, 0xa6, 0x4d, 0xcb, 0x54, 0xba, 0x85, 0x5e, 0xbc, 0x2f, 0x15, 0x4c, 0xcb, 0x7c, 0x16, 0x92, + 0xeb, 0xcf, 0x33, 0x3d, 0x9a, 0x88, 0x73, 0x9e, 0xf9, 0xa1, 0x00, 0x33, 0xbe, 0xa5, 0x13, 0xb3, + 0xb9, 0xfb, 0x2e, 0x24, 0x4d, 0xeb, 0xe5, 0x45, 0xb6, 0x28, 0x49, 0x7e, 0xb2, 0xea, 0x75, 0x97, + 0x28, 0xce, 0xfa, 0xfe, 0xcb, 0x04, 0x64, 0x9e, 0x94, 0xe2, 0xac, 0xe5, 0x47, 0x7c, 0xfb, 0x9b, + 0xb5, 0x77, 0x54, 0x6f, 0xf7, 0xdf, 0xb7, 0xf4, 0xa4, 0xb4, 0x8e, 0x3b, 0x5e, 0x6f, 0x27, 0x5c, + 0x68, 0x05, 0x32, 0xdd, 0x1b, 0xa5, 0x23, 0x6a, 0x2a, 0xe0, 0x9a, 0xc7, 0x30, 0x4e, 0xe5, 0x7a, + 0x7e, 0x14, 0x42, 0x84, 0x1f, 0x05, 0x79, 0x8d, 0x6f, 0x29, 0x26, 0x2e, 0xf2, 0x9a, 0x90, 0x89, + 0x38, 0x2e, 0x4e, 0x48, 0x4f, 0x01, 0x48, 0x75, 0xe2, 0x6c, 0x92, 0xef, 0x27, 0x21, 0xbf, 0xd3, + 0x76, 0x0e, 0x63, 0xee, 0x7d, 0x25, 0x80, 0x56, 0xdb, 0x39, 0x24, 0x23, 0xf2, 0xc4, 0xe4, 0x75, + 0x3e, 0xc7, 0x45, 0xc3, 0xab, 0x34, 0xe3, 0xab, 0x9f, 0x98, 0xa8, 0xc2, 0x85, 0x60, 0x25, 0xf0, + 0xf3, 0xb8, 0x39, 0x0c, 0x59, 0xd6, 0x4f, 0xcc, 0x4d, 0xec, 0x43, 0x4a, 0x26, 0x09, 0x13, 0x49, + 0x1f, 0xc1, 0x24, 0x79, 0x50, 0x5c, 0xeb, 0x22, 0xcd, 0x3c, 0x41, 0x78, 0xea, 0x16, 0xfa, 0x10, + 0x32, 0x8c, 0x9b, 0xac, 0x7e, 0x13, 0x74, 0xf5, 0x8b, 0xaa, 0x0b, 0x57, 0x23, 0x5d, 0xf7, 0xd2, + 0x94, 0x95, 0xac, 0x75, 0x97, 0x60, 0x7c, 0xdf, 0xb2, 0x35, 0xef, 0x30, 0x97, 0x3d, 0xb0, 0xf6, + 0x64, 0x90, 0x66, 0x2d, 0x95, 0xce, 0x88, 0x20, 0xfd, 0xa6, 0x00, 0x05, 0xbf, 0x21, 0xe2, 0x5c, + 0x10, 0x4a, 0x5d, 0x5a, 0xbc, 0x78, 0x53, 0x10, 0x05, 0x4a, 0xff, 0x8a, 0x5a, 0x44, 0x9a, 0x75, + 0x4c, 0x5b, 0x26, 0xce, 0x9e, 0xf2, 0x21, 0xf3, 0xe2, 0x49, 0x5c, 0xb4, 0x75, 0xa9, 0x43, 0xcf, + 0x03, 0xb8, 0x64, 0x34, 0xc9, 0x7c, 0x6e, 0xb8, 0x8d, 0x0e, 0x87, 0x6d, 0x2e, 0xf6, 0x4e, 0x8d, + 0x67, 0x82, 0xb4, 0x92, 0x97, 0x24, 0xfd, 0x2e, 0xdd, 0xad, 0x0e, 0x6a, 0x12, 0xa7, 0xaa, 0xab, + 0x90, 0xb3, 0x99, 0x68, 0x62, 0xd6, 0x5c, 0x50, 0xdb, 0x53, 0x3e, 0x2b, 0x51, 0xf8, 0x6f, 0x27, + 0xa0, 0xf0, 0xb4, 0x8d, 0xed, 0xce, 0x37, 0x49, 0xdd, 0xb7, 0xa1, 0xf0, 0x52, 0x35, 0x5c, 0x65, + 0xdf, 0xb2, 0x95, 0x76, 0x4b, 0x57, 0x5d, 0xcf, 0xdb, 0x24, 0x47, 0xc8, 0x8f, 0x2d, 0x7b, 0x97, + 0x12, 0x11, 0x06, 0x74, 0x64, 0x5a, 0x2f, 0x4d, 0x85, 0x90, 0x29, 0x50, 0x3e, 0x31, 0xf9, 0x16, + 0x72, 0xf1, 0xbd, 0x7f, 0x7f, 0xba, 0xf8, 0x68, 0x24, 0x07, 0x31, 0xea, 0x0c, 0xd7, 0x6e, 0x1b, + 0xfa, 0xd2, 0xee, 0x6e, 0x75, 0x55, 0x16, 0xa9, 0xc8, 0xe7, 0x4c, 0x62, 0xfd, 0xc4, 0x74, 0xa4, + 0xbf, 0x93, 0x00, 0x31, 0xd0, 0x51, 0x9c, 0x0d, 0x59, 0x86, 0xec, 0x8b, 0x36, 0xb6, 0x8d, 0x57, + 0x68, 0x46, 0xe0, 0x8c, 0x64, 0xda, 0xb9, 0x0f, 0xd3, 0xee, 0x89, 0xa9, 0x30, 0xf7, 0x3d, 0xe6, + 0xf8, 0xe1, 0x39, 0x2c, 0x14, 0x5c, 0x52, 0x66, 0x42, 0xa7, 0x4e, 0x1f, 0x0e, 0xfa, 0x14, 0xa6, + 0xba, 0xb4, 0x95, 0xfc, 0x6a, 0xda, 0xca, 0xbe, 0x0c, 0x29, 0xea, 0x27, 0x02, 0x20, 0xaa, 0xa8, + 0x2a, 0xdb, 0xe3, 0xff, 0xa6, 0xf4, 0xa7, 0xbb, 0x20, 0x52, 0x3f, 0x4c, 0xc5, 0xd8, 0x57, 0x9a, + 0x86, 0xe3, 0x18, 0xe6, 0x01, 0xef, 0x50, 0x79, 0x4a, 0xaf, 0xee, 0x6f, 0x32, 0xaa, 0xf4, 0x17, + 0x60, 0xa6, 0xab, 0x02, 0x71, 0x36, 0xf6, 0x0d, 0x98, 0xda, 0x67, 0x47, 0xb0, 0x54, 0x38, 0xdf, + 0x1e, 0xcc, 0x52, 0x1a, 0x7b, 0x9f, 0xf4, 0x9f, 0x13, 0x70, 0x49, 0xc6, 0x8e, 0xd5, 0x38, 0xc6, + 0xf1, 0xab, 0xb0, 0x02, 0xfc, 0xec, 0x45, 0x79, 0x25, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xba, + 0xf7, 0xd8, 0xdf, 0x18, 0xde, 0x63, 0xfb, 0x77, 0xd5, 0xf9, 0x4e, 0x5d, 0xaa, 0x6b, 0xa7, 0xce, + 0x82, 0x02, 0x3b, 0x3d, 0xd6, 0x15, 0x07, 0xbf, 0x30, 0xdb, 0x4d, 0x0f, 0x0c, 0x2d, 0x0d, 0x2b, + 0x64, 0x95, 0xb1, 0xd4, 0xf0, 0x8b, 0xad, 0x76, 0x93, 0xda, 0xce, 0xc5, 0x59, 0x52, 0xde, 0xb3, + 0xd3, 0xc5, 0x7c, 0x57, 0x9a, 0x23, 0xe7, 0x0d, 0xff, 0x99, 0x48, 0x97, 0xbe, 0x03, 0x97, 0x7b, + 0x94, 0x1d, 0xa7, 0xc5, 0xf3, 0xcf, 0x93, 0x70, 0xb5, 0x5b, 0x7c, 0xdc, 0x10, 0xe7, 0x9b, 0xde, + 0xa0, 0x15, 0xc8, 0x35, 0x0d, 0xf3, 0xd5, 0x76, 0x2f, 0xa7, 0x9a, 0x86, 0xe9, 0xd3, 0xa2, 0xba, + 0xc6, 0xc4, 0xd7, 0xda, 0x35, 0x54, 0x98, 0x8f, 0x6a, 0xbb, 0x38, 0xfb, 0xc7, 0x0f, 0x04, 0x98, + 0x8a, 0x7b, 0x5b, 0xee, 0xd5, 0xbc, 0xe0, 0xa4, 0x3a, 0xe4, 0xbe, 0x86, 0x7d, 0xbc, 0xdf, 0x16, + 0x00, 0xd5, 0xed, 0xb6, 0x49, 0x40, 0xed, 0x86, 0x75, 0x10, 0x67, 0x35, 0x2f, 0xc1, 0xb8, 0x61, + 0xea, 0xf8, 0x84, 0x56, 0x33, 0x25, 0xb3, 0x87, 0xae, 0xa3, 0xc4, 0xe4, 0x48, 0x47, 0x89, 0xd2, + 0xa7, 0x30, 0xd3, 0x55, 0xc4, 0x38, 0xeb, 0xff, 0x8b, 0x04, 0xcc, 0xf0, 0x8a, 0xc4, 0xbe, 0x83, + 0xf9, 0x0e, 0x8c, 0x37, 0x88, 0xcc, 0x21, 0xed, 0x4c, 0xdf, 0xe9, 0xb5, 0x33, 0xcd, 0x8c, 0xbe, + 0x05, 0xd0, 0xb2, 0xf1, 0xb1, 0xc2, 0x58, 0x93, 0x23, 0xb1, 0x66, 0x08, 0x07, 0x25, 0xa0, 0x1f, + 0x09, 0x50, 0x20, 0x03, 0xba, 0x65, 0x5b, 0x2d, 0xcb, 0x21, 0x36, 0x8b, 0x33, 0x1a, 0xcc, 0x79, + 0x7a, 0x76, 0xba, 0x98, 0xdb, 0x34, 0xcc, 0x1d, 0xce, 0x58, 0xaf, 0x8d, 0xec, 0xbd, 0xef, 0xc5, + 0x30, 0x2c, 0x95, 0x1a, 0x96, 0x76, 0x14, 0x1c, 0x8e, 0x91, 0x99, 0xc5, 0x17, 0xe7, 0x48, 0xff, + 0x4e, 0x80, 0x4b, 0x5f, 0xdb, 0x76, 0xf1, 0xff, 0x0f, 0x65, 0x4b, 0xcf, 0x40, 0xa4, 0x3f, 0xaa, + 0xe6, 0xbe, 0x15, 0xe7, 0xc6, 0xfd, 0xff, 0x12, 0x60, 0x3a, 0x24, 0x38, 0x4e, 0x03, 0xe7, 0x55, + 0xf5, 0x94, 0x63, 0xee, 0x30, 0xee, 0x68, 0xaa, 0x92, 0xa7, 0x78, 0x76, 0xd6, 0x29, 0x97, 0x60, + 0x0a, 0x93, 0x59, 0x8c, 0x6e, 0xf1, 0xee, 0xb1, 0x08, 0x92, 0x9e, 0x1d, 0xfd, 0xac, 0x9f, 0xa1, + 0xd8, 0x91, 0x7e, 0x95, 0x58, 0x58, 0xe1, 0x41, 0x19, 0xe7, 0x90, 0xff, 0x67, 0x09, 0x98, 0x2d, + 0xb1, 0x23, 0x70, 0xcf, 0x27, 0x24, 0xce, 0x8e, 0x38, 0x07, 0x93, 0xc7, 0xd8, 0x76, 0x0c, 0x8b, + 0xad, 0xf6, 0x39, 0xd9, 0x7b, 0x44, 0xf3, 0x90, 0x76, 0x4c, 0xb5, 0xe5, 0x1c, 0x5a, 0xde, 0x71, + 0xa2, 0xff, 0xec, 0xfb, 0xaf, 0x8c, 0xbf, 0xba, 0xff, 0xca, 0xc4, 0x70, 0xff, 0x95, 0xc9, 0xaf, + 0xe0, 0xbf, 0xc2, 0xcf, 0xee, 0xfe, 0xb5, 0x00, 0x57, 0xfa, 0x34, 0x17, 0x67, 0xe7, 0xfc, 0x1e, + 0x64, 0x35, 0x2e, 0x98, 0xac, 0x0f, 0xec, 0x60, 0xb2, 0x4a, 0xb2, 0xbd, 0x22, 0xf4, 0x39, 0x3b, + 0x5d, 0x04, 0xaf, 0xa8, 0xd5, 0x55, 0xae, 0x1c, 0xf2, 0x5b, 0x97, 0xfe, 0x0b, 0x40, 0xa1, 0x7c, + 0xc2, 0x36, 0xe5, 0x6b, 0xcc, 0x2a, 0x41, 0x8f, 0x21, 0xdd, 0xb2, 0xad, 0x63, 0xc3, 0xab, 0x46, + 0xbe, 0xcb, 0x79, 0xc1, 0xab, 0x46, 0x0f, 0xd7, 0x0e, 0xe7, 0x90, 0x7d, 0x5e, 0x54, 0x87, 0xcc, + 0x86, 0xa5, 0xa9, 0x8d, 0xc7, 0x46, 0xc3, 0x1b, 0x68, 0x6f, 0x9f, 0x2f, 0x68, 0xc9, 0xe7, 0xd9, + 0x51, 0xdd, 0x43, 0xaf, 0x11, 0x7c, 0x22, 0xaa, 0x42, 0xba, 0xe2, 0xba, 0x2d, 0x92, 0xc8, 0xc7, + 0xdf, 0x9d, 0x11, 0x84, 0x12, 0x16, 0xcf, 0xe3, 0xd6, 0x63, 0x47, 0x75, 0x98, 0x7e, 0x42, 0x83, + 0xc3, 0x4a, 0x0d, 0xab, 0xad, 0x97, 0x68, 0x30, 0x1b, 0x5f, 0x26, 0x6e, 0x8f, 0x20, 0xf3, 0x49, + 0xa9, 0x26, 0xf7, 0x0b, 0x40, 0x2b, 0x90, 0xae, 0x3d, 0xe2, 0xc2, 0x98, 0x19, 0x79, 0x6b, 0x04, + 0x61, 0xb5, 0x47, 0xb2, 0xcf, 0x86, 0xd6, 0x20, 0xbb, 0xf2, 0x59, 0xdb, 0xc6, 0x5c, 0xca, 0xc4, + 0x40, 0xcf, 0x89, 0x5e, 0x29, 0x94, 0x4b, 0x0e, 0x33, 0xa3, 0xef, 0x40, 0x81, 0xe8, 0xad, 0xae, + 0xee, 0x35, 0x3c, 0x79, 0x69, 0x2a, 0xef, 0xcd, 0x11, 0xe4, 0xf9, 0x9c, 0xde, 0x91, 0x40, 0x8f, + 0xa8, 0x79, 0x19, 0x72, 0x5d, 0xed, 0x85, 0x10, 0xa4, 0x5a, 0xa4, 0x69, 0x04, 0xea, 0x86, 0x44, + 0x7f, 0xa3, 0xb7, 0x60, 0xd2, 0xb4, 0x74, 0xec, 0x75, 0xe6, 0x5c, 0xf1, 0xd2, 0xd9, 0xe9, 0xe2, + 0xc4, 0x96, 0xa5, 0x33, 0x5b, 0x87, 0xff, 0x92, 0x27, 0x48, 0xa6, 0xaa, 0x3e, 0x7f, 0x1d, 0x52, + 0xa4, 0x89, 0xc8, 0x1c, 0xb2, 0xa7, 0x3a, 0x78, 0xd7, 0x36, 0xb8, 0x34, 0xef, 0x71, 0xfe, 0x1f, + 0x24, 0x20, 0x51, 0x7b, 0x44, 0xac, 0xf9, 0xbd, 0xb6, 0x76, 0x84, 0x5d, 0x9e, 0xce, 0x9f, 0xa8, + 0x95, 0x6f, 0xe3, 0x7d, 0x83, 0x19, 0x5d, 0x19, 0x99, 0x3f, 0xa1, 0xd7, 0x01, 0x54, 0x4d, 0xc3, + 0x8e, 0xa3, 0x78, 0xf1, 0x7d, 0x19, 0x39, 0xc3, 0x28, 0xeb, 0xb8, 0x43, 0xd8, 0x1c, 0xac, 0xd9, + 0xd8, 0xf5, 0x7c, 0xa8, 0xd8, 0x13, 0x61, 0x73, 0x71, 0xb3, 0xa5, 0xb8, 0xd6, 0x11, 0x36, 0x69, + 0x93, 0x66, 0xc8, 0xac, 0xd0, 0x6c, 0xd5, 0x09, 0x81, 0x4c, 0x68, 0xd8, 0xd4, 0x83, 0xd9, 0x27, + 0x23, 0xfb, 0xcf, 0x44, 0xa4, 0x8d, 0x0f, 0x0c, 0x1e, 0x1d, 0x97, 0x91, 0xf9, 0x13, 0xd1, 0x92, + 0xda, 0x76, 0x0f, 0x69, 0x4b, 0x64, 0x64, 0xfa, 0x1b, 0xdd, 0x86, 0x02, 0x73, 0xbb, 0x54, 0xb0, + 0xa9, 0x29, 0x74, 0x1e, 0xcc, 0xd0, 0xe4, 0x1c, 0x23, 0x97, 0x4d, 0x8d, 0xcc, 0x7a, 0xe8, 0x11, + 0x70, 0x82, 0x72, 0xd4, 0x74, 0x88, 0x4e, 0x81, 0xe4, 0x2a, 0x16, 0xce, 0x4e, 0x17, 0xb3, 0x35, + 0x9a, 0xb0, 0xbe, 0x59, 0x23, 0x6b, 0x09, 0xcb, 0xb5, 0xde, 0x74, 0xaa, 0xfa, 0xfc, 0xdf, 0x14, + 0x20, 0xf9, 0xa4, 0x54, 0xbb, 0xb0, 0xca, 0xbc, 0x82, 0x26, 0x43, 0x05, 0xbd, 0x03, 0x85, 0x3d, + 0xa3, 0xd1, 0x30, 0xcc, 0x03, 0x62, 0x5f, 0x7d, 0x0f, 0x6b, 0x9e, 0xc2, 0xf2, 0x9c, 0xbc, 0xc3, + 0xa8, 0xe8, 0x3a, 0x64, 0x35, 0x1b, 0xeb, 0xd8, 0x74, 0x0d, 0xb5, 0xe1, 0x70, 0xcd, 0x85, 0x49, + 0xf3, 0x7f, 0x51, 0x80, 0x71, 0xda, 0x59, 0xd1, 0x6b, 0x90, 0xd1, 0x2c, 0xd3, 0x55, 0x0d, 0x93, + 0xcf, 0x3a, 0x19, 0x39, 0x20, 0x0c, 0x2c, 0xde, 0x0d, 0x98, 0x52, 0x35, 0xcd, 0x6a, 0x9b, 0xae, + 0x62, 0xaa, 0x4d, 0xcc, 0x8b, 0x99, 0xe5, 0xb4, 0x2d, 0xb5, 0x89, 0xd1, 0x22, 0x78, 0x8f, 0x7e, + 0xd8, 0x66, 0x46, 0x06, 0x4e, 0x5a, 0xc7, 0x9d, 0x79, 0x0c, 0x19, 0xbf, 0x57, 0x93, 0xfa, 0xb6, + 0x1d, 0xbf, 0x04, 0xf4, 0x37, 0x7a, 0x1b, 0x2e, 0xbd, 0x68, 0xab, 0x0d, 0x63, 0x9f, 0x6e, 0x7e, + 0x51, 0x2f, 0x75, 0xfa, 0x32, 0x56, 0x14, 0xe4, 0xa7, 0x51, 0x09, 0xf4, 0x9d, 0xde, 0x20, 0x48, + 0x06, 0x83, 0x80, 0xb9, 0xec, 0x48, 0x1d, 0x98, 0x96, 0xb1, 0x6b, 0x77, 0xea, 0x2c, 0x92, 0xb5, + 0x7c, 0x8c, 0x4d, 0x97, 0xd4, 0xdd, 0x6a, 0x61, 0xe6, 0x24, 0xe2, 0xd5, 0xdd, 0x27, 0xa0, 0x5b, + 0x90, 0x57, 0x5d, 0xd2, 0xdd, 0x5c, 0xc5, 0x6c, 0x37, 0xf7, 0xb0, 0xcd, 0x5c, 0x01, 0xe4, 0x1c, + 0xa7, 0x6e, 0x51, 0x22, 0x8f, 0xc8, 0xb0, 0x3b, 0x0a, 0xdd, 0x27, 0xe2, 0xaf, 0x06, 0x4a, 0x2a, + 0x13, 0x8a, 0x74, 0x0f, 0x2e, 0x93, 0x7a, 0x96, 0x4d, 0xcd, 0xee, 0xb4, 0x88, 0xe4, 0x6d, 0xfa, + 0xd7, 0x41, 0x62, 0xe8, 0x9c, 0x86, 0x1e, 0xcf, 0x48, 0x5f, 0x4c, 0x42, 0xae, 0x7c, 0xd2, 0xb2, + 0xec, 0x58, 0x77, 0x75, 0x8a, 0x30, 0xc9, 0x81, 0xef, 0x90, 0xa3, 0xd8, 0x9e, 0x19, 0xc8, 0x3b, + 0x87, 0xe6, 0x8c, 0xe8, 0x99, 0x1f, 0x75, 0x7a, 0x84, 0x3b, 0xc4, 0xaa, 0x9f, 0x1a, 0xc5, 0xaa, + 0x9f, 0xe1, 0x58, 0x3a, 0xcb, 0x22, 0x51, 0xd7, 0x71, 0xa7, 0x5e, 0x93, 0x79, 0xe0, 0x2b, 0x79, + 0x70, 0xd0, 0x16, 0x64, 0x9b, 0xc7, 0x9a, 0xa6, 0xec, 0x1b, 0x0d, 0x97, 0xfb, 0xaa, 0x45, 0xbb, + 0x55, 0x6f, 0x3e, 0x2b, 0x95, 0x1e, 0xd3, 0x4c, 0xcc, 0xc7, 0x2b, 0x78, 0x96, 0x81, 0x48, 0x60, + 0xbf, 0x51, 0x11, 0x80, 0x39, 0x7e, 0x52, 0x7f, 0xa1, 0xe4, 0x05, 0x4e, 0xb8, 0x28, 0x1b, 0x0d, + 0x8a, 0x90, 0x20, 0xc7, 0xdc, 0x93, 0x9a, 0x06, 0x0b, 0x39, 0xce, 0xb1, 0x6d, 0x38, 0x4a, 0xdc, + 0x34, 0x68, 0xb8, 0xf1, 0x9b, 0xc0, 0x83, 0x6e, 0x14, 0xc7, 0x0b, 0xa1, 0x2b, 0xe6, 0xce, 0x4e, + 0x17, 0x33, 0x32, 0xa5, 0xd6, 0x6a, 0x75, 0x39, 0xc3, 0x32, 0xd4, 0x1c, 0xf7, 0x22, 0x61, 0x16, + 0x93, 0xa3, 0x87, 0x59, 0xfc, 0x55, 0x01, 0x66, 0x79, 0xfb, 0x28, 0x7b, 0xd4, 0xb5, 0x5c, 0x6d, + 0x18, 0x6e, 0x47, 0x39, 0x3a, 0x9e, 0x4b, 0x53, 0x73, 0xeb, 0x97, 0x23, 0xdb, 0x39, 0xd4, 0xbd, + 0x96, 0xbc, 0xd6, 0xee, 0x6c, 0x70, 0xe6, 0xf5, 0xe3, 0xb2, 0xe9, 0xda, 0x9d, 0xe2, 0x95, 0xb3, + 0xd3, 0xc5, 0x99, 0xfe, 0xd4, 0x67, 0xf2, 0x8c, 0xd3, 0xcf, 0x82, 0x2a, 0x00, 0xd8, 0xef, 0xde, + 0x74, 0xf6, 0x8c, 0x5e, 0x36, 0x23, 0xc7, 0x81, 0x1c, 0xe2, 0x45, 0x77, 0x41, 0xe4, 0x61, 0x2d, + 0xfb, 0x46, 0x03, 0x2b, 0x8e, 0xf1, 0x19, 0xa6, 0xf3, 0x6c, 0x52, 0xce, 0x33, 0x3a, 0x11, 0x51, + 0x33, 0x3e, 0xc3, 0xe8, 0x01, 0x5c, 0x0e, 0x5a, 0x40, 0xd9, 0xc3, 0x0d, 0xeb, 0x25, 0xcb, 0x9e, + 0xa5, 0xd9, 0x91, 0xaf, 0xfd, 0x22, 0x49, 0x22, 0x2c, 0xf3, 0xdf, 0x83, 0xb9, 0x41, 0x15, 0x0e, + 0x0f, 0xc6, 0x0c, 0x3b, 0x2b, 0x7d, 0xbf, 0x7b, 0xa3, 0x64, 0x84, 0x41, 0xc3, 0x37, 0x4b, 0x3e, + 0x48, 0xbc, 0x2f, 0x48, 0x7f, 0x2f, 0x01, 0xb9, 0x62, 0xbb, 0x71, 0xb4, 0xdd, 0xaa, 0xb1, 0xab, + 0x00, 0xd0, 0x35, 0xc8, 0xe8, 0xaa, 0xab, 0xb2, 0x42, 0x0a, 0x2c, 0xbc, 0x8d, 0x10, 0x68, 0x6d, + 0xee, 0x40, 0x21, 0xe4, 0x87, 0xc2, 0xbd, 0xed, 0x69, 0xb5, 0x03, 0x32, 0x75, 0x88, 0x7f, 0x1f, + 0xe6, 0x42, 0x19, 0xe9, 0xae, 0x86, 0x82, 0x4d, 0xd7, 0x36, 0x30, 0xdb, 0x99, 0x4b, 0xca, 0x21, + 0x67, 0x99, 0x2a, 0x49, 0x2e, 0xb3, 0x54, 0x54, 0x87, 0x29, 0x92, 0xb1, 0xa3, 0xd0, 0x19, 0xd8, + 0xdb, 0x39, 0x7d, 0x10, 0x51, 0xad, 0xae, 0x72, 0x2f, 0x51, 0xfd, 0x94, 0x28, 0x0f, 0xfd, 0x29, + 0x67, 0x71, 0x40, 0x99, 0xff, 0x18, 0xc4, 0xde, 0x0c, 0x61, 0x5d, 0xa6, 0x98, 0x2e, 0x2f, 0x85, + 0x75, 0x99, 0x0c, 0xe9, 0x69, 0x2d, 0x95, 0x4e, 0x89, 0xe3, 0xd2, 0x6f, 0xa5, 0x20, 0xef, 0xf5, + 0xcc, 0x38, 0x4d, 0xfa, 0x22, 0x8c, 0x93, 0x7e, 0xe4, 0xb9, 0x76, 0xdc, 0x1e, 0x32, 0x20, 0xb8, + 0x7f, 0x37, 0xe9, 0x5f, 0x1e, 0xfa, 0xa4, 0xac, 0x71, 0x4c, 0x29, 0xf3, 0xbf, 0x9f, 0x80, 0x14, + 0xb5, 0xa2, 0x1f, 0x40, 0x8a, 0x06, 0xfc, 0x0b, 0x43, 0x03, 0xfe, 0x3d, 0xd7, 0x00, 0x92, 0x15, + 0x6d, 0x92, 0xc1, 0xa5, 0x7b, 0xf3, 0x6e, 0x66, 0x94, 0xf7, 0x8b, 0x7c, 0xde, 0x4d, 0x97, 0xe9, + 0xe5, 0x08, 0xf5, 0x1a, 0x35, 0x8d, 0xd8, 0x8c, 0xeb, 0xad, 0x91, 0x89, 0x90, 0xa1, 0x58, 0xa4, + 0xae, 0x4a, 0x96, 0xed, 0x62, 0x9d, 0x1b, 0xbd, 0xd7, 0xcf, 0xeb, 0x16, 0x9e, 0x51, 0xef, 0xf1, + 0xa1, 0xab, 0x90, 0x24, 0x53, 0xe1, 0x24, 0xf3, 0x3a, 0x38, 0x3b, 0x5d, 0x4c, 0x92, 0x49, 0x90, + 0xd0, 0xd0, 0x32, 0x64, 0xbb, 0x27, 0x27, 0x62, 0x37, 0xd1, 0x59, 0x3c, 0x34, 0xb1, 0x40, 0xc3, + 0x1f, 0x91, 0x0c, 0xf0, 0xb1, 0xae, 0xc1, 0xfd, 0x0d, 0x7e, 0x5d, 0xe0, 0xee, 0x95, 0x35, 0x8d, + 0x2c, 0xbf, 0x76, 0x9c, 0xeb, 0xe3, 0x3d, 0x10, 0x6d, 0xd5, 0xd4, 0xad, 0xa6, 0xf1, 0x19, 0x66, + 0x1b, 0x0c, 0x0e, 0x3f, 0x79, 0x29, 0xf8, 0x74, 0xba, 0x13, 0xe0, 0x48, 0x7f, 0x22, 0x70, 0x57, + 0x4c, 0xbf, 0x18, 0xf1, 0x9e, 0x8f, 0x67, 0xf9, 0xee, 0xa4, 0xb9, 0x6f, 0x79, 0x9e, 0x24, 0xaf, + 0x0d, 0xf2, 0x9b, 0xaa, 0x9a, 0xfb, 0x96, 0x77, 0xd2, 0x67, 0x7b, 0x04, 0x67, 0xfe, 0x57, 0x60, + 0x9c, 0x26, 0xbf, 0x42, 0x57, 0xf3, 0xdd, 0x7f, 0x13, 0x62, 0x52, 0xfa, 0xa3, 0x04, 0xbc, 0x41, + 0xab, 0xfa, 0x0c, 0xdb, 0xc6, 0x7e, 0x67, 0xc7, 0xb6, 0x5c, 0xac, 0xb9, 0x58, 0x0f, 0x36, 0xd8, + 0x62, 0x6c, 0x02, 0x1d, 0x32, 0xfc, 0x68, 0xd2, 0xd0, 0xf9, 0x0d, 0x1f, 0x4f, 0xbe, 0x1a, 0xf0, + 0x4e, 0xb3, 0x23, 0xcd, 0xea, 0xaa, 0x9c, 0x66, 0x92, 0xab, 0x3a, 0x5a, 0x81, 0x4c, 0xcb, 0xab, + 0xc6, 0x85, 0xbc, 0x5f, 0x7c, 0x2e, 0xb4, 0x0e, 0x05, 0x5e, 0x50, 0xb5, 0x61, 0x1c, 0x63, 0x45, + 0x75, 0x2f, 0x32, 0x23, 0xe4, 0x18, 0xef, 0x0a, 0x61, 0x5d, 0x71, 0xa5, 0xbf, 0x91, 0x82, 0x5b, + 0xe7, 0xa8, 0x38, 0xce, 0xee, 0x35, 0x0f, 0xe9, 0x63, 0xf2, 0x22, 0x83, 0xd7, 0x3e, 0x2d, 0xfb, + 0xcf, 0x68, 0xaf, 0x6b, 0x59, 0xd9, 0x57, 0x8d, 0x06, 0x59, 0x86, 0x98, 0xbf, 0xe1, 0x60, 0x8f, + 0xa6, 0x68, 0xff, 0xbd, 0xd0, 0x02, 0xf4, 0x98, 0x0a, 0xa2, 0xd9, 0x1c, 0xf4, 0x03, 0x01, 0xe6, + 0xd9, 0x0b, 0x99, 0xd3, 0x5b, 0xcf, 0x6b, 0x52, 0xf4, 0x35, 0xab, 0x11, 0xaf, 0x19, 0x49, 0x47, + 0x4b, 0xa1, 0x77, 0xf1, 0x82, 0xcc, 0x85, 0xdf, 0x16, 0x2e, 0xca, 0xfc, 0x6f, 0x08, 0x90, 0x0d, + 0x11, 0xd0, 0xed, 0xbe, 0x10, 0xa3, 0xec, 0x59, 0x54, 0x5c, 0xd1, 0xad, 0xbe, 0xb8, 0xa2, 0x62, + 0xfa, 0xcb, 0xd3, 0xc5, 0x94, 0xcc, 0x5c, 0xd7, 0xbd, 0x08, 0xa3, 0x1b, 0xc1, 0x75, 0x35, 0xc9, + 0x9e, 0x4c, 0xde, 0x7d, 0x35, 0x14, 0xa1, 0xaa, 0xde, 0x89, 0x18, 0x45, 0xa8, 0xe4, 0x49, 0xfa, + 0xe3, 0x24, 0x4c, 0xaf, 0xe8, 0x7a, 0xad, 0x46, 0x51, 0x4d, 0x9c, 0x63, 0x0c, 0x41, 0x8a, 0x98, + 0x1b, 0x3c, 0x1c, 0x8a, 0xfe, 0x46, 0x1f, 0xc2, 0x3c, 0x0d, 0xc6, 0x50, 0x54, 0x57, 0xb1, 0x19, + 0x7f, 0xe8, 0x30, 0x8e, 0xed, 0xe9, 0x5d, 0xa1, 0x39, 0x56, 0x3c, 0x9b, 0x31, 0x38, 0x72, 0x7b, + 0x0b, 0x90, 0x6e, 0x38, 0xec, 0x5a, 0x09, 0xcd, 0x32, 0xf7, 0x1b, 0x86, 0xe6, 0x3a, 0xdc, 0x80, + 0x9d, 0xf6, 0x52, 0x4a, 0x5e, 0x42, 0x57, 0x76, 0xe7, 0x50, 0xd5, 0xad, 0x97, 0xc1, 0x21, 0xbb, + 0x9f, 0xbd, 0xe6, 0x25, 0xa0, 0xef, 0xc2, 0x5c, 0x7f, 0x76, 0x66, 0xe7, 0xf1, 0x8d, 0x94, 0x91, + 0x86, 0xdc, 0x6c, 0x9f, 0x64, 0x6a, 0x0f, 0xa2, 0x1a, 0x50, 0xd8, 0xa0, 0x38, 0xae, 0xea, 0x9f, + 0x51, 0xdc, 0x1a, 0x29, 0x56, 0x88, 0xd9, 0xf9, 0xfe, 0xa3, 0x9c, 0x21, 0x72, 0xe8, 0x4f, 0x62, + 0xbd, 0x1a, 0xa4, 0x7f, 0xb9, 0x8a, 0xea, 0x78, 0x51, 0x2e, 0xec, 0x7a, 0x8d, 0x3c, 0xa3, 0xaf, + 0x38, 0x2c, 0x78, 0x85, 0x39, 0xc7, 0x07, 0xad, 0x1c, 0xe7, 0x0e, 0xf3, 0xdf, 0x16, 0x20, 0x2f, + 0xe3, 0x7d, 0x1b, 0x3b, 0x87, 0x71, 0x76, 0x9f, 0xc7, 0x30, 0x65, 0x33, 0xa9, 0xca, 0xbe, 0x6d, + 0x35, 0x2f, 0x32, 0xed, 0x65, 0x39, 0xe3, 0x63, 0xdb, 0x6a, 0xf2, 0xd5, 0xe5, 0x19, 0x14, 0xfc, + 0x32, 0xc6, 0x59, 0xf9, 0xbf, 0x4b, 0xe3, 0x60, 0x99, 0xe0, 0xb8, 0x0f, 0xd3, 0xe3, 0xd5, 0x00, + 0x3d, 0x65, 0x08, 0x17, 0x34, 0x4e, 0x35, 0xfc, 0xb1, 0x00, 0xf9, 0x5a, 0x7b, 0x8f, 0x5d, 0xf8, + 0x14, 0x9f, 0x06, 0xca, 0x90, 0x69, 0xe0, 0x7d, 0x57, 0x79, 0x25, 0xb7, 0xee, 0x34, 0x61, 0xa5, + 0x4e, 0xed, 0x4f, 0x00, 0x6c, 0x1a, 0x08, 0x46, 0xe5, 0x24, 0x2f, 0x28, 0x27, 0x43, 0x79, 0x09, + 0x99, 0x2c, 0xa0, 0x05, 0xbf, 0x9a, 0x71, 0x2e, 0x95, 0xcf, 0xbb, 0x66, 0x87, 0xe4, 0x45, 0x66, + 0x87, 0x69, 0x6e, 0x7b, 0x47, 0xcf, 0x10, 0x4b, 0x30, 0x43, 0x2d, 0x4c, 0x45, 0x6d, 0xb5, 0x1a, + 0x86, 0x87, 0xe0, 0xe8, 0xfc, 0x93, 0x92, 0xa7, 0x69, 0xd2, 0x0a, 0x4b, 0xa1, 0xd8, 0x0d, 0x7d, + 0x5f, 0x80, 0xa9, 0x7d, 0x1b, 0xe3, 0xcf, 0xb0, 0x42, 0x57, 0x97, 0xd1, 0x1c, 0x24, 0x56, 0x49, + 0x19, 0xbe, 0xf2, 0x01, 0x6a, 0x96, 0xbd, 0xb8, 0x46, 0xde, 0x8b, 0xb6, 0x40, 0xd4, 0x1a, 0xec, + 0x48, 0xb7, 0x6b, 0x7d, 0x18, 0x71, 0x00, 0x14, 0x18, 0x73, 0xb0, 0x78, 0x3c, 0x25, 0x83, 0x49, + 0xd5, 0x15, 0x7e, 0x51, 0x1e, 0x5d, 0x36, 0xba, 0x9d, 0x35, 0xc2, 0x01, 0xf1, 0xa1, 0xfb, 0xf5, + 0x96, 0x64, 0xac, 0xea, 0x1c, 0x84, 0x90, 0x71, 0xe5, 0x3f, 0xf0, 0x71, 0xf5, 0x1c, 0xa6, 0x69, + 0xbf, 0x89, 0x3b, 0xae, 0x55, 0xfa, 0x47, 0x49, 0x40, 0x61, 0xc9, 0x5f, 0x5f, 0x7f, 0x4b, 0xc4, + 0xd7, 0xdf, 0xd6, 0x40, 0x0a, 0xd9, 0x75, 0x0d, 0xd5, 0x71, 0x15, 0xe6, 0x15, 0xe8, 0x28, 0x2d, + 0x6c, 0x2b, 0x0e, 0xd6, 0x2c, 0x7e, 0xb5, 0x91, 0x20, 0x2f, 0x04, 0x39, 0x37, 0x54, 0xc7, 0x7d, + 0xca, 0xf2, 0xed, 0x60, 0xbb, 0x46, 0x73, 0xa1, 0x47, 0x30, 0xdb, 0x54, 0x4f, 0xa2, 0xf8, 0xc7, + 0x29, 0xff, 0x4c, 0x53, 0x3d, 0xe9, 0x63, 0xfa, 0x00, 0xe6, 0xa3, 0x99, 0x14, 0x07, 0x7b, 0xa7, + 0x86, 0xb3, 0x11, 0x8c, 0x35, 0xec, 0xa2, 0x15, 0x80, 0x00, 0x0f, 0xf1, 0x35, 0x7a, 0x14, 0x38, + 0x94, 0xf1, 0xe1, 0x90, 0xf4, 0x43, 0x01, 0xf2, 0x9b, 0xc6, 0x81, 0xad, 0xc6, 0x7a, 0x71, 0x10, + 0xfa, 0xa0, 0xfb, 0x98, 0x35, 0xfb, 0x70, 0x3e, 0xca, 0x8d, 0x86, 0xe5, 0xf0, 0xb6, 0x52, 0x39, + 0x03, 0x59, 0xfa, 0xfc, 0x12, 0xc5, 0x39, 0xe7, 0x6b, 0xf0, 0x3a, 0x75, 0x4c, 0xe4, 0x5e, 0x49, + 0x5f, 0x0b, 0x50, 0x93, 0x7e, 0x4f, 0x80, 0x85, 0x41, 0x6f, 0x89, 0x73, 0x40, 0xc8, 0xf4, 0x7e, + 0x44, 0xfa, 0x06, 0xc5, 0x1f, 0x11, 0xe7, 0xcc, 0x34, 0x88, 0x8f, 0x04, 0xf0, 0xcb, 0x56, 0xa3, + 0x37, 0x27, 0xb2, 0xdf, 0x0e, 0x51, 0x50, 0x4d, 0x53, 0x4d, 0x1a, 0xf5, 0xdc, 0xc0, 0xea, 0x31, + 0xe6, 0x3e, 0x95, 0xb1, 0xce, 0x12, 0x9f, 0x0b, 0xb0, 0x30, 0xe8, 0x2d, 0x71, 0x2a, 0xe8, 0x97, + 0x61, 0x92, 0x39, 0xe7, 0x79, 0xf8, 0xec, 0xea, 0x80, 0x00, 0x6f, 0xd3, 0x0b, 0x2b, 0xf2, 0xf2, + 0x4b, 0x75, 0xc8, 0x17, 0x55, 0xdb, 0x36, 0x62, 0xdd, 0x45, 0x91, 0x7e, 0x2c, 0x40, 0xc1, 0x17, + 0x1b, 0x67, 0x4d, 0xbf, 0x7a, 0xcc, 0x8a, 0xf4, 0x5f, 0xaf, 0xc1, 0x14, 0x2f, 0xff, 0xae, 0x69, + 0x58, 0x26, 0x7a, 0x00, 0xc9, 0x03, 0x7e, 0x16, 0x97, 0x8d, 0x3c, 0x6e, 0x08, 0x6e, 0xac, 0xac, + 0x8c, 0xc9, 0x24, 0x2f, 0x61, 0x69, 0xb5, 0xdd, 0x88, 0x02, 0x04, 0x41, 0x17, 0x61, 0x96, 0x56, + 0xdb, 0x45, 0x35, 0x28, 0x68, 0xc1, 0x35, 0x79, 0x0a, 0x61, 0x4f, 0x0e, 0xdc, 0x1b, 0x8f, 0xbc, + 0xb0, 0xb0, 0x32, 0x26, 0xe7, 0xb5, 0xae, 0x04, 0x54, 0x0a, 0xdf, 0xce, 0x96, 0xea, 0xf3, 0xe8, + 0x0c, 0x9a, 0xbe, 0xfb, 0x66, 0xb8, 0xca, 0x58, 0xe8, 0x12, 0x37, 0xf4, 0x01, 0x4c, 0xe8, 0xf4, + 0x1e, 0x30, 0x6e, 0x4f, 0x44, 0x35, 0x78, 0xd7, 0x75, 0x6b, 0x95, 0x31, 0x99, 0x73, 0xa0, 0x35, + 0x98, 0x62, 0xbf, 0x18, 0x70, 0xe7, 0x56, 0xc0, 0xad, 0xc1, 0x12, 0x42, 0x76, 0x78, 0x65, 0x4c, + 0xce, 0xea, 0x01, 0x15, 0x3d, 0x81, 0xac, 0xd6, 0xc0, 0xaa, 0xcd, 0x45, 0xdd, 0x1e, 0x18, 0x86, + 0xda, 0x77, 0x77, 0x58, 0x65, 0x4c, 0x06, 0xcd, 0x27, 0x92, 0x42, 0xd9, 0xf4, 0x0a, 0x29, 0x2e, + 0xe9, 0xed, 0x81, 0x85, 0xea, 0xbf, 0x8f, 0xab, 0x42, 0xed, 0x73, 0x9f, 0x8a, 0xde, 0x81, 0x94, + 0xa3, 0xa9, 0x26, 0x37, 0x49, 0x16, 0x06, 0xdc, 0xf1, 0x13, 0x30, 0xd3, 0xdc, 0xe8, 0x43, 0x86, + 0xf9, 0xdd, 0x13, 0xef, 0x00, 0x24, 0x4a, 0xa7, 0x5d, 0x77, 0x49, 0x10, 0x9d, 0x62, 0x4a, 0x20, + 0x7a, 0x50, 0xf5, 0xa6, 0x61, 0x2a, 0xf4, 0x8c, 0x89, 0x9e, 0x78, 0x44, 0xeb, 0xa1, 0x2f, 0x20, + 0xbf, 0x42, 0x2f, 0xac, 0xf0, 0x88, 0x68, 0x13, 0x72, 0x4c, 0x50, 0x9b, 0xc5, 0x8a, 0xcf, 0x2d, + 0x0f, 0x74, 0xab, 0x88, 0x88, 0x56, 0xaf, 0x8c, 0xc9, 0x53, 0x6a, 0x88, 0x1c, 0x94, 0xab, 0x89, + 0xed, 0x03, 0x76, 0xb4, 0x32, 0xa4, 0x5c, 0x61, 0x5f, 0x55, 0xbf, 0x5c, 0x94, 0x88, 0x7e, 0x0d, + 0x2e, 0x31, 0x41, 0x2e, 0x77, 0xc1, 0xe3, 0x9e, 0x5c, 0xaf, 0x0f, 0x74, 0x89, 0x18, 0x18, 0xdf, + 0x5d, 0x19, 0x93, 0x91, 0xda, 0x97, 0x88, 0x34, 0xb8, 0xcc, 0xde, 0xc0, 0x03, 0x84, 0x6d, 0x1e, + 0xd3, 0x3a, 0x77, 0x93, 0xbe, 0xe2, 0xad, 0x41, 0xaf, 0x88, 0x8c, 0x5b, 0xae, 0x8c, 0xc9, 0x33, + 0x6a, 0x7f, 0x6a, 0x50, 0x0d, 0x9b, 0x87, 0x62, 0xf2, 0xee, 0xf6, 0xd6, 0xf0, 0x6a, 0x44, 0x85, + 0xb0, 0xfa, 0xd5, 0xe8, 0x4a, 0x24, 0x0d, 0xe8, 0x5f, 0x44, 0x41, 0x3b, 0xd3, 0xd4, 0xc0, 0x06, + 0x8c, 0x88, 0xd7, 0x24, 0x0d, 0x78, 0x18, 0x22, 0xa3, 0x25, 0x48, 0x1c, 0x68, 0xf4, 0x00, 0x33, + 0xda, 0x74, 0xf2, 0x63, 0x12, 0x2b, 0x63, 0x72, 0xe2, 0x40, 0x43, 0x1f, 0x43, 0x9a, 0x05, 0x98, + 0x9d, 0x98, 0x73, 0xf9, 0x81, 0x73, 0x76, 0x77, 0x98, 0x5e, 0x65, 0x4c, 0xa6, 0x31, 0x6d, 0xbc, + 0x23, 0xf3, 0xe0, 0x21, 0x2a, 0x62, 0x69, 0x48, 0x5c, 0x79, 0x4f, 0x08, 0x17, 0xe9, 0x30, 0xb6, + 0x4f, 0x44, 0x3b, 0x90, 0xe7, 0x4b, 0xb7, 0x17, 0x0c, 0x21, 0x0e, 0x74, 0x3a, 0x8a, 0x8a, 0x87, + 0xa8, 0xd0, 0xdd, 0xd6, 0x10, 0x9d, 0xb4, 0x5d, 0xb7, 0x44, 0xde, 0x76, 0xd3, 0x03, 0xdb, 0x6e, + 0xa0, 0x6f, 0x3e, 0x69, 0x3b, 0xbb, 0x2f, 0x11, 0xbd, 0x07, 0xe3, 0x6c, 0x9c, 0x20, 0x2a, 0x32, + 0xca, 0x8f, 0xae, 0x67, 0x88, 0xb0, 0xfc, 0x64, 0xf6, 0x72, 0xb9, 0x8f, 0xb1, 0xd2, 0xb0, 0x0e, + 0xe6, 0x66, 0x06, 0xce, 0x5e, 0xfd, 0xde, 0xd2, 0x64, 0xf6, 0x72, 0x03, 0x2a, 0xe9, 0x40, 0xde, + 0x4e, 0x1e, 0x1b, 0x62, 0x97, 0x06, 0x76, 0xa0, 0x08, 0xd7, 0xe3, 0x0a, 0x8d, 0xfe, 0x0a, 0xc8, + 0xfe, 0xc4, 0xea, 0x60, 0x85, 0x4e, 0x8a, 0x97, 0x87, 0x4f, 0xac, 0x5d, 0x17, 0xaf, 0xf9, 0x13, + 0x2b, 0xa3, 0xa2, 0x67, 0x20, 0xf2, 0xdb, 0x7f, 0x14, 0xcf, 0x15, 0x6e, 0x6e, 0x96, 0xca, 0xbb, + 0x17, 0xb9, 0x20, 0x46, 0x79, 0x49, 0x56, 0x08, 0x96, 0xec, 0x4e, 0x41, 0x9f, 0xc0, 0x34, 0x95, + 0xa7, 0x68, 0xc1, 0x85, 0x4d, 0x73, 0x73, 0x7d, 0xd7, 0xff, 0x0c, 0xbe, 0xdb, 0xc9, 0x93, 0x2c, + 0x6a, 0x3d, 0x49, 0x64, 0x3c, 0x18, 0xa6, 0xe1, 0xd2, 0xb5, 0x7b, 0x7e, 0xe0, 0x78, 0xe8, 0xbe, + 0xac, 0xb6, 0x42, 0x6d, 0x2d, 0x4a, 0x21, 0xdd, 0xb8, 0x67, 0xc6, 0x7b, 0x6d, 0x60, 0x37, 0x1e, + 0x30, 0xd9, 0xe5, 0xdc, 0xae, 0x79, 0x6e, 0x15, 0x80, 0xed, 0x20, 0x50, 0x50, 0xb4, 0x30, 0xd0, + 0x00, 0xe8, 0x75, 0x0d, 0x26, 0x06, 0x40, 0xc3, 0xa3, 0x11, 0x03, 0x80, 0x9d, 0xdc, 0xcd, 0x5d, + 0x1f, 0xbc, 0x58, 0x85, 0x5d, 0x05, 0xe8, 0x62, 0x45, 0x09, 0xc4, 0x20, 0x23, 0x70, 0xae, 0x43, + 0x47, 0xf8, 0x8d, 0x81, 0xbb, 0x37, 0x3d, 0x31, 0x83, 0x95, 0x31, 0x39, 0xfd, 0x82, 0x93, 0x48, + 0xaf, 0x62, 0x22, 0xf8, 0xd8, 0xbe, 0x3f, 0xb0, 0x57, 0xf5, 0x07, 0x8b, 0x91, 0x5e, 0xf5, 0x22, + 0xa0, 0x06, 0x4b, 0x9e, 0xc3, 0xce, 0xe4, 0xe6, 0xde, 0x18, 0xbe, 0xe4, 0x75, 0x9f, 0x20, 0xfa, + 0x4b, 0x1e, 0x27, 0xb3, 0x25, 0x4f, 0x57, 0x1c, 0x87, 0x7a, 0x1c, 0xcd, 0xdd, 0x1a, 0xb2, 0xe4, + 0xf5, 0xec, 0xd2, 0xb3, 0x25, 0x4f, 0xaf, 0x31, 0x4e, 0x62, 0xfd, 0xd9, 0xde, 0x6d, 0x57, 0x1c, + 0xd8, 0xdf, 0x19, 0x68, 0xfd, 0x45, 0x5e, 0xc7, 0x45, 0xac, 0x3f, 0xbb, 0x2b, 0x01, 0x7d, 0x0b, + 0x26, 0xf9, 0x56, 0xe2, 0xdc, 0xdd, 0x21, 0x26, 0x75, 0x78, 0xf7, 0x97, 0x74, 0x47, 0xce, 0xc3, + 0x26, 0x07, 0xb6, 0x85, 0xc9, 0x26, 0xbf, 0x7b, 0x43, 0x26, 0x87, 0xbe, 0x5d, 0x54, 0x36, 0x39, + 0x04, 0x64, 0x52, 0x1a, 0x87, 0x6d, 0xbf, 0xcd, 0xfd, 0xd2, 0xc0, 0xd2, 0x74, 0xef, 0x43, 0x92, + 0xd2, 0x70, 0x1e, 0xba, 0x58, 0xd0, 0xb5, 0x9a, 0x69, 0xe7, 0xcd, 0xc1, 0x8b, 0x45, 0xef, 0x86, + 0x4e, 0xc5, 0x3b, 0xf3, 0x64, 0x5a, 0xf9, 0x4b, 0x02, 0x5c, 0x67, 0x7d, 0x80, 0x9e, 0xf8, 0x74, + 0x14, 0xff, 0xc0, 0x2e, 0xb4, 0x5b, 0xf5, 0x80, 0x8a, 0x7f, 0xef, 0xe2, 0xe7, 0x4b, 0xde, 0x1b, + 0x5f, 0x57, 0x87, 0xe5, 0x23, 0xca, 0x68, 0x32, 0x5c, 0x3f, 0xf7, 0x70, 0xa0, 0x32, 0xba, 0xf7, + 0x22, 0x88, 0x32, 0x38, 0x0f, 0x6a, 0xc0, 0x1c, 0x1b, 0x12, 0x01, 0xee, 0xf5, 0x8b, 0xfe, 0x68, + 0xa0, 0x13, 0xef, 0x50, 0xc4, 0x5f, 0x19, 0x93, 0x67, 0x5f, 0x44, 0x66, 0x20, 0x6f, 0xa3, 0xd7, + 0x59, 0x1a, 0x01, 0x4c, 0x55, 0x3c, 0x3c, 0xf9, 0xce, 0xc0, 0xb7, 0x0d, 0x85, 0xcf, 0xe4, 0x6d, + 0x4e, 0x64, 0x06, 0xa2, 0x9a, 0x3d, 0x06, 0x0d, 0xe7, 0xde, 0x1d, 0xa8, 0x9a, 0x6e, 0x4c, 0x4a, + 0x54, 0xc3, 0x79, 0x8a, 0x93, 0xdc, 0x7b, 0xc4, 0x8f, 0x52, 0x2f, 0x88, 0xe2, 0x5a, 0x2a, 0x7d, + 0x45, 0x9c, 0x5b, 0x4b, 0xa5, 0xaf, 0x8a, 0xf3, 0x6b, 0xa9, 0xf4, 0x35, 0xf1, 0xb5, 0xb5, 0x54, + 0x7a, 0x51, 0xbc, 0xbe, 0x96, 0x4a, 0x4b, 0xe2, 0x4d, 0xe9, 0x67, 0xd7, 0x20, 0xe7, 0x21, 0x4b, + 0x06, 0xf9, 0x1e, 0x86, 0x21, 0xdf, 0xc2, 0x20, 0xc8, 0xc7, 0xb1, 0x28, 0xc7, 0x7c, 0x0f, 0xc3, + 0x98, 0x6f, 0x61, 0x10, 0xe6, 0x0b, 0x78, 0x08, 0xe8, 0xab, 0x0f, 0x02, 0x7d, 0xf7, 0x46, 0x00, + 0x7d, 0xbe, 0xa8, 0x5e, 0xd4, 0xb7, 0xda, 0x8f, 0xfa, 0xde, 0x18, 0x8e, 0xfa, 0x7c, 0x51, 0x21, + 0xd8, 0xf7, 0x61, 0x0f, 0xec, 0xbb, 0x31, 0x04, 0xf6, 0xf9, 0xfc, 0x1e, 0xee, 0x5b, 0x8f, 0xc4, + 0x7d, 0xb7, 0xcf, 0xc3, 0x7d, 0xbe, 0x9c, 0x2e, 0xe0, 0x57, 0x89, 0x02, 0x7e, 0xb7, 0xce, 0x01, + 0x7e, 0xbe, 0xa8, 0x30, 0xf2, 0x5b, 0x8f, 0x44, 0x7e, 0xb7, 0xcf, 0x43, 0x7e, 0x41, 0xb1, 0xc2, + 0xd0, 0xef, 0xdd, 0x2e, 0xe8, 0xb7, 0x38, 0x10, 0xfa, 0xf9, 0xdc, 0x0c, 0xfb, 0x7d, 0xd4, 0x8b, + 0xfd, 0x6e, 0x0c, 0xc1, 0x7e, 0x81, 0x62, 0x39, 0xf8, 0xab, 0x44, 0x81, 0xbf, 0x5b, 0xe7, 0x80, + 0xbf, 0x40, 0x17, 0x21, 0xf4, 0xb7, 0x15, 0x8d, 0xfe, 0xee, 0x9c, 0x8b, 0xfe, 0x7c, 0x69, 0xdd, + 0xf0, 0xaf, 0x12, 0x05, 0xff, 0x6e, 0x9d, 0x03, 0xff, 0x7a, 0x4a, 0xc6, 0xf0, 0x9f, 0x3a, 0x14, + 0xff, 0xbd, 0x35, 0x22, 0xfe, 0xf3, 0x45, 0x47, 0x01, 0x40, 0x7d, 0x38, 0x00, 0x5c, 0x1a, 0x15, + 0x00, 0xfa, 0x2f, 0x89, 0x44, 0x80, 0xea, 0x50, 0x04, 0xf8, 0xd6, 0x88, 0x08, 0xb0, 0xa7, 0x22, + 0xdd, 0x10, 0x70, 0x2b, 0x1a, 0x02, 0xde, 0x39, 0x17, 0x02, 0x06, 0xad, 0xd8, 0x85, 0x01, 0x97, + 0x43, 0x18, 0xf0, 0xf5, 0x01, 0x18, 0xd0, 0x67, 0x25, 0x20, 0xf0, 0xdb, 0x7d, 0x20, 0x50, 0x1a, + 0x06, 0x02, 0x7d, 0x5e, 0x1f, 0x05, 0x56, 0xa2, 0x50, 0xe0, 0xad, 0x73, 0x50, 0x60, 0xd0, 0x6f, + 0x42, 0x30, 0xf0, 0xe9, 0x00, 0x18, 0x78, 0xf7, 0x7c, 0x18, 0xe8, 0xcb, 0xeb, 0xc1, 0x81, 0xea, + 0x50, 0x1c, 0xf8, 0xd6, 0x88, 0x38, 0x30, 0x68, 0xc1, 0x08, 0x20, 0xf8, 0x7e, 0x37, 0x10, 0xbc, + 0x3e, 0x18, 0x08, 0xfa, 0x62, 0x38, 0x12, 0x5c, 0x8f, 0x44, 0x82, 0xb7, 0xcf, 0x43, 0x82, 0xc1, + 0x6c, 0x16, 0x86, 0x82, 0x5b, 0xd1, 0x50, 0xf0, 0xce, 0xb9, 0x50, 0x30, 0xe8, 0x48, 0x5d, 0x58, + 0x70, 0x3d, 0x12, 0x0b, 0xde, 0x3e, 0x0f, 0x0b, 0xf6, 0x4c, 0xb5, 0x1c, 0x0c, 0x3e, 0x1f, 0x08, + 0x06, 0xef, 0x8f, 0x02, 0x06, 0x7d, 0xa1, 0x7d, 0x68, 0xf0, 0xd3, 0xc1, 0x68, 0xf0, 0x97, 0x2e, + 0x70, 0xd3, 0x6f, 0x24, 0x1c, 0xfc, 0x76, 0x1f, 0x1c, 0x94, 0x86, 0xc1, 0xc1, 0x60, 0x64, 0x78, + 0x78, 0xb0, 0x1c, 0x81, 0xde, 0xde, 0x18, 0x8e, 0xde, 0x82, 0x85, 0x3c, 0x80, 0x6f, 0x1f, 0xf6, + 0xc0, 0xb7, 0x1b, 0xe7, 0x3a, 0xb6, 0x86, 0xf0, 0x5b, 0xb1, 0x1f, 0xbf, 0xdd, 0x1c, 0x8a, 0xdf, + 0x7c, 0x09, 0x01, 0x80, 0x5b, 0x8f, 0x04, 0x70, 0xb7, 0xcf, 0x03, 0x70, 0x41, 0x57, 0x08, 0x23, + 0xb8, 0xad, 0x68, 0x04, 0x77, 0xe7, 0x5c, 0x04, 0xd7, 0xb3, 0x6c, 0x79, 0x10, 0xae, 0x12, 0x05, + 0xe1, 0x6e, 0x9d, 0x03, 0xe1, 0xc2, 0xcb, 0x96, 0x8f, 0xe1, 0xea, 0x83, 0x30, 0xdc, 0xbd, 0x11, + 0x30, 0x5c, 0x60, 0xcc, 0xf5, 0x80, 0xb8, 0x8f, 0x7b, 0x41, 0x9c, 0x34, 0x0c, 0xc4, 0x05, 0x9d, + 0xc8, 0x43, 0x71, 0x5b, 0xd1, 0x28, 0xee, 0xce, 0xb9, 0x28, 0x2e, 0x3c, 0xae, 0x43, 0x30, 0xee, + 0xe3, 0x5e, 0x18, 0x27, 0x0d, 0x83, 0x71, 0x41, 0x79, 0x3c, 0x1c, 0x57, 0x89, 0xc2, 0x71, 0xb7, + 0xce, 0xc1, 0x71, 0xa1, 0xe9, 0x3e, 0x00, 0x72, 0x7f, 0x79, 0x74, 0x20, 0xf7, 0xfe, 0xab, 0x3a, + 0x0a, 0x9e, 0x8f, 0xe4, 0x3e, 0xee, 0x45, 0x72, 0xd2, 0x30, 0x24, 0x17, 0xe8, 0xc3, 0x83, 0x72, + 0xcd, 0x73, 0xa1, 0xdc, 0x83, 0x0b, 0x40, 0x39, 0x5f, 0xfe, 0x20, 0x2c, 0xd7, 0x3c, 0x17, 0xcb, + 0x3d, 0xb8, 0x00, 0x96, 0x0b, 0x5e, 0x37, 0x00, 0xcc, 0x7d, 0xdc, 0x0b, 0xe6, 0xa4, 0x61, 0x60, + 0x2e, 0xd0, 0xce, 0x85, 0xd1, 0xdc, 0x6b, 0xe2, 0xeb, 0x5d, 0x98, 0xee, 0xaf, 0x00, 0x4c, 0x54, + 0x22, 0xce, 0x04, 0x85, 0x57, 0x39, 0x13, 0x44, 0xdf, 0x82, 0x6b, 0xfe, 0x03, 0xf5, 0xe7, 0x52, + 0x78, 0x74, 0x9f, 0xd6, 0xb0, 0xb4, 0x23, 0xba, 0x54, 0xa6, 0xe5, 0x39, 0x3f, 0xcb, 0x63, 0xdb, + 0x6a, 0xb2, 0x28, 0x3f, 0xea, 0x25, 0x83, 0x56, 0xc9, 0x18, 0xa6, 0x36, 0xe1, 0xf9, 0x97, 0xa7, + 0xf6, 0x5f, 0xef, 0xc9, 0x59, 0x5f, 0xe1, 0x3e, 0x0a, 0xf4, 0x2e, 0xe4, 0xda, 0x0e, 0xb6, 0x95, + 0x96, 0x6d, 0x58, 0xb6, 0xe1, 0xb2, 0xc8, 0x3a, 0xa1, 0x28, 0x7e, 0x79, 0xba, 0x38, 0xb5, 0xeb, + 0x60, 0x7b, 0x87, 0xd3, 0xe5, 0xa9, 0x76, 0xe8, 0xc9, 0xfb, 0x44, 0xe1, 0xf8, 0xe8, 0x9f, 0x28, + 0x7c, 0x0a, 0x22, 0xf5, 0xd0, 0x09, 0x2f, 0xa3, 0xec, 0xca, 0xb9, 0xe8, 0x15, 0x5f, 0xd5, 0x43, + 0x2b, 0x25, 0xbd, 0x7a, 0xae, 0x60, 0x77, 0x13, 0xd1, 0x13, 0xc8, 0xdb, 0x56, 0x9b, 0xde, 0x2c, + 0xd5, 0xb2, 0x1a, 0x86, 0xd6, 0xa1, 0xf6, 0x4d, 0x3e, 0xfa, 0xbc, 0x99, 0x65, 0xdc, 0xa1, 0xf9, + 0xe4, 0x9c, 0x1d, 0x7e, 0x44, 0x35, 0xa0, 0xb7, 0x4a, 0x79, 0x52, 0x50, 0xdf, 0x3d, 0xfe, 0x43, + 0xbf, 0xa6, 0xf1, 0x5c, 0x35, 0x5c, 0x2e, 0x17, 0x5e, 0xfa, 0xbf, 0xd1, 0x63, 0x98, 0xa2, 0x37, + 0x9c, 0x93, 0x36, 0xb7, 0xda, 0x2e, 0x37, 0x6f, 0xae, 0x2e, 0xb1, 0x0f, 0x59, 0x2e, 0x79, 0x1f, + 0xb2, 0x5c, 0x5a, 0xe5, 0x1f, 0xb2, 0x64, 0x67, 0xd5, 0x3f, 0xfa, 0x8f, 0x8b, 0x82, 0x9c, 0xf5, + 0x3c, 0xa7, 0xac, 0xb6, 0x8b, 0x1e, 0xc0, 0xe5, 0xa6, 0x7a, 0x42, 0x2f, 0x4a, 0xf7, 0x9d, 0x6a, + 0xe9, 0xd5, 0x90, 0xec, 0x03, 0x89, 0xa8, 0xa9, 0x9e, 0xd0, 0xaf, 0x3a, 0xb2, 0x24, 0xfa, 0xd5, + 0xa6, 0x1b, 0x30, 0xc5, 0xc3, 0x9e, 0xd8, 0x47, 0xdd, 0x0a, 0x34, 0x27, 0xff, 0xc2, 0x0f, 0xfb, + 0xae, 0xdb, 0x7b, 0x30, 0x17, 0xce, 0xa2, 0x30, 0xcf, 0x58, 0xdc, 0x6c, 0xb9, 0x9d, 0xb9, 0x2b, + 0xb4, 0xb3, 0x5e, 0x0e, 0x65, 0x5f, 0x21, 0xa9, 0x65, 0x92, 0x48, 0x18, 0x79, 0xa0, 0x94, 0x65, + 0xb2, 0xf5, 0x82, 0x05, 0xa0, 0xa9, 0x36, 0x33, 0x8b, 0xd2, 0x32, 0x0f, 0xa4, 0xda, 0x36, 0x99, + 0xf3, 0x34, 0x4f, 0x44, 0xb7, 0x20, 0xaf, 0x1b, 0x8e, 0x6b, 0x98, 0x9a, 0xcb, 0x6f, 0x7d, 0x67, + 0xf7, 0xa6, 0xe7, 0x3c, 0x2a, 0xbb, 0xda, 0xbd, 0x0e, 0xd3, 0x5a, 0xc3, 0xf0, 0xed, 0x68, 0x66, + 0xd9, 0x4c, 0x0f, 0x9c, 0x19, 0x4a, 0x34, 0x6f, 0xaf, 0xcb, 0x4e, 0x41, 0xeb, 0x26, 0xa3, 0x12, + 0x14, 0x0e, 0x54, 0x17, 0xbf, 0x54, 0x3b, 0x8a, 0x17, 0xc3, 0x9c, 0xa5, 0xf7, 0x36, 0x5c, 0x3b, + 0x3b, 0x5d, 0xcc, 0x3d, 0x61, 0x49, 0x7d, 0xa1, 0xcc, 0xb9, 0x83, 0x50, 0x82, 0x8e, 0xee, 0x40, + 0x41, 0x75, 0x3a, 0xa6, 0x46, 0xfb, 0x30, 0x36, 0x9d, 0xb6, 0xc3, 0x63, 0xf9, 0xf2, 0x94, 0x5c, + 0xf2, 0xa8, 0xe8, 0x43, 0x98, 0xe7, 0x1f, 0x77, 0x79, 0xa9, 0xda, 0xba, 0x42, 0xfb, 0x7d, 0x30, + 0xc1, 0x88, 0xcc, 0x0f, 0x9a, 0x7d, 0xcc, 0x85, 0x64, 0x20, 0x9d, 0x3d, 0x98, 0x7f, 0x9f, 0xc1, + 0x34, 0x55, 0x28, 0xd6, 0xc9, 0x02, 0xd8, 0xc0, 0x26, 0x76, 0x9c, 0x21, 0xe7, 0x1a, 0x45, 0x96, + 0xb7, 0xe6, 0x65, 0x65, 0x53, 0x9a, 0x2c, 0xee, 0xf5, 0xd0, 0xfd, 0xcb, 0xe6, 0x41, 0xcc, 0xae, + 0xa5, 0xd2, 0x53, 0x62, 0x6e, 0x2d, 0x95, 0xce, 0x8b, 0x05, 0xe9, 0xff, 0x08, 0x30, 0x1b, 0x2d, + 0x04, 0xd5, 0x60, 0xa6, 0xeb, 0x3e, 0x25, 0xd6, 0xd2, 0x17, 0x99, 0x21, 0xa7, 0xc3, 0xb7, 0x2a, + 0xd1, 0xb7, 0x10, 0xe5, 0x44, 0x08, 0x55, 0x1c, 0xd7, 0x36, 0x34, 0xef, 0x8e, 0xb2, 0x2b, 0x7d, + 0x6c, 0x35, 0x9a, 0x4c, 0x4b, 0xa4, 0x9e, 0xf4, 0x95, 0x28, 0x79, 0x91, 0x12, 0xa9, 0x27, 0xdd, + 0xa2, 0xa5, 0xef, 0x27, 0xa1, 0x40, 0xd6, 0x7a, 0xc7, 0x31, 0x2c, 0xb3, 0xe2, 0x47, 0x37, 0xf8, + 0x53, 0xa2, 0x40, 0x03, 0x75, 0xfd, 0x67, 0xb4, 0x48, 0x03, 0xa2, 0x09, 0xa6, 0xf2, 0x3f, 0x96, + 0x95, 0x94, 0x81, 0x91, 0x68, 0xc8, 0xe7, 0x0a, 0x4c, 0x38, 0x56, 0xdb, 0xd6, 0xbc, 0x4f, 0xb8, + 0xdc, 0x1b, 0x60, 0x5c, 0x84, 0x5e, 0xb8, 0x54, 0xa3, 0x0c, 0x32, 0x67, 0x44, 0x9f, 0x42, 0x81, + 0xfd, 0xa2, 0x81, 0x98, 0x34, 0x10, 0x92, 0x45, 0xb3, 0x3e, 0x18, 0x59, 0xd6, 0x06, 0x67, 0x94, + 0xf3, 0x4e, 0xd7, 0x33, 0xfa, 0x18, 0x5e, 0x33, 0x2d, 0xa5, 0x89, 0x9b, 0x16, 0x33, 0x2a, 0xc8, + 0xbc, 0xa6, 0x2b, 0xaa, 0xab, 0xf0, 0x42, 0x33, 0x1f, 0xf3, 0x39, 0xd3, 0xda, 0xa4, 0x59, 0x64, + 0x9e, 0x63, 0xc5, 0x65, 0x72, 0xa5, 0x25, 0x98, 0x60, 0xbf, 0x50, 0x06, 0xc6, 0xb7, 0xeb, 0x95, + 0xb2, 0x2c, 0x8e, 0xa1, 0x29, 0x48, 0x3f, 0x96, 0xb7, 0x37, 0x95, 0xda, 0xd3, 0x0d, 0x51, 0x40, + 0x59, 0x98, 0x94, 0xb7, 0xb7, 0xeb, 0xca, 0xfa, 0x33, 0x31, 0x21, 0xdd, 0x81, 0x7c, 0x77, 0x89, + 0x10, 0xc0, 0x84, 0x5c, 0xde, 0xdc, 0xa6, 0xdf, 0x2d, 0xc9, 0xc0, 0xf8, 0xc6, 0x76, 0x69, 0x65, + 0x43, 0x14, 0xa4, 0x5f, 0x08, 0x30, 0x55, 0x64, 0x1f, 0xe2, 0x61, 0x9e, 0x44, 0x1f, 0xf6, 0x78, + 0xfc, 0x5c, 0x8d, 0xde, 0xb4, 0x18, 0xe4, 0xe9, 0x93, 0xe6, 0xf3, 0xa5, 0x17, 0xb0, 0xb7, 0x38, + 0x18, 0xaa, 0xd2, 0x5d, 0x5d, 0xcf, 0xb5, 0xd8, 0x63, 0x43, 0x35, 0x10, 0x55, 0x4f, 0xb7, 0x0a, + 0x2f, 0xc9, 0x60, 0x07, 0xe3, 0x9e, 0x66, 0xf0, 0xe6, 0x22, 0xb5, 0x9b, 0xfc, 0x41, 0xea, 0x47, + 0x9f, 0x2f, 0x8e, 0x49, 0x7f, 0x9a, 0x82, 0x5c, 0x31, 0xfc, 0xd1, 0x21, 0x54, 0xed, 0xa9, 0xec, + 0x9d, 0x48, 0x43, 0x28, 0xc4, 0xb1, 0x34, 0xe4, 0x73, 0x6e, 0x99, 0xe0, 0x0b, 0x47, 0xac, 0xee, + 0xd7, 0x87, 0x38, 0x4b, 0x85, 0x2b, 0x1f, 0x30, 0xce, 0xff, 0xdb, 0xa4, 0x6f, 0x21, 0x2d, 0xc1, + 0x38, 0x0b, 0x48, 0x17, 0xfa, 0xee, 0xca, 0xa1, 0x6b, 0x23, 0x01, 0x88, 0x24, 0x5d, 0x66, 0xd9, + 0x88, 0x45, 0x55, 0x7f, 0x25, 0x2f, 0xab, 0x60, 0x1e, 0xbc, 0xf8, 0x57, 0x90, 0xdb, 0xec, 0x66, + 0xe8, 0xff, 0x87, 0x3e, 0xcd, 0xe4, 0x7d, 0xe8, 0xbb, 0x50, 0xd0, 0xac, 0x46, 0x83, 0xe1, 0x0a, + 0xb6, 0xb2, 0xf5, 0xdf, 0x15, 0x47, 0x8b, 0xc0, 0x3f, 0x7c, 0xbd, 0xe4, 0x7f, 0x00, 0x7b, 0x49, + 0xe6, 0x1f, 0xc0, 0x0e, 0xc5, 0xce, 0xe5, 0x7d, 0x61, 0x6c, 0x41, 0xec, 0x09, 0xe3, 0x9b, 0x7c, + 0x95, 0x30, 0x3e, 0x16, 0xfc, 0xc8, 0x7b, 0xde, 0xcf, 0x04, 0xee, 0x79, 0xbc, 0x61, 0x59, 0x47, + 0x6d, 0xdf, 0x9f, 0x73, 0x3e, 0x7c, 0xcf, 0x73, 0x10, 0x61, 0x44, 0x03, 0x6e, 0xa3, 0x8c, 0xb7, + 0xc4, 0x57, 0x33, 0xde, 0x6e, 0xc0, 0x54, 0xcb, 0xc6, 0xfb, 0xd8, 0xd5, 0x0e, 0x15, 0xb3, 0xdd, + 0xe4, 0xd1, 0xc6, 0x59, 0x8f, 0xb6, 0xd5, 0x6e, 0xa2, 0x7b, 0x20, 0xfa, 0x59, 0xf8, 0x5e, 0x8f, + 0x77, 0xc9, 0xa8, 0x47, 0xe7, 0x3b, 0x43, 0xd2, 0x7f, 0x17, 0x60, 0xa6, 0xab, 0x4e, 0x7c, 0x4c, + 0xad, 0x41, 0x56, 0xf7, 0xcd, 0x65, 0x67, 0x4e, 0xb8, 0x60, 0xec, 0x59, 0x98, 0x19, 0x29, 0x30, + 0xeb, 0xbd, 0x96, 0x7e, 0x15, 0x28, 0x10, 0x9b, 0xb8, 0xa0, 0xd8, 0xcb, 0x81, 0x9c, 0xd5, 0xd0, + 0x0b, 0xfc, 0x41, 0x96, 0x1c, 0x69, 0x90, 0x49, 0xff, 0x53, 0x00, 0x91, 0xbe, 0xe0, 0x31, 0xc6, + 0x7a, 0x2c, 0x53, 0xa6, 0x17, 0xe4, 0x99, 0x18, 0x3d, 0x9e, 0xb8, 0xeb, 0x4b, 0x66, 0xc9, 0x9e, + 0x2f, 0x99, 0x45, 0xcd, 0x9f, 0xa9, 0xaf, 0x38, 0x7f, 0x4a, 0x9f, 0x0b, 0x90, 0xf7, 0xab, 0xcd, + 0x3e, 0x61, 0x3c, 0xe4, 0x8e, 0xf2, 0x57, 0xfb, 0x4c, 0xaf, 0x77, 0x97, 0xda, 0x48, 0x5f, 0x55, + 0x0e, 0xdf, 0xa5, 0xc6, 0x3e, 0x2f, 0xfb, 0xb7, 0xbc, 0xee, 0x48, 0x8a, 0x58, 0x0a, 0x2e, 0xb1, + 0x7a, 0x85, 0x78, 0xed, 0xaf, 0xc3, 0x75, 0xf9, 0x71, 0x48, 0x81, 0xb4, 0x47, 0x11, 0x2d, 0x8d, + 0x34, 0xbf, 0x7b, 0x5a, 0x62, 0x1d, 0xf0, 0x0f, 0xc2, 0x2d, 0xc1, 0x2e, 0x41, 0x79, 0x04, 0xc9, + 0x63, 0xb5, 0x31, 0xcc, 0x41, 0xb7, 0xab, 0xe5, 0x64, 0x92, 0x1b, 0x3d, 0xee, 0xba, 0xfb, 0x2b, + 0x31, 0x78, 0x1f, 0xb8, 0x5f, 0xa5, 0x5d, 0x77, 0x84, 0xbd, 0xd7, 0x3d, 0x80, 0x86, 0xbe, 0x3e, + 0x3c, 0x92, 0x3e, 0x48, 0x7d, 0xf1, 0xf9, 0xa2, 0x20, 0x7d, 0x04, 0x88, 0xd8, 0x3a, 0xee, 0xd3, + 0xb6, 0x65, 0x07, 0xf7, 0xa8, 0xf5, 0x06, 0x73, 0x8e, 0x47, 0x07, 0x73, 0x4a, 0x97, 0x61, 0xa6, + 0x8b, 0x9b, 0xcd, 0x40, 0xd2, 0x7b, 0x70, 0xf5, 0x89, 0xe5, 0x38, 0x46, 0xab, 0xd6, 0xde, 0x63, + 0x43, 0x9d, 0xac, 0x57, 0xfe, 0x9c, 0x9b, 0x6e, 0xd1, 0x4d, 0x46, 0x93, 0xcd, 0x4d, 0x19, 0xd9, + 0x7f, 0x96, 0x7e, 0x5f, 0x80, 0x2b, 0xfd, 0x9c, 0x4c, 0xcb, 0x51, 0xd7, 0x4b, 0x4c, 0x6a, 0x56, + 0x70, 0xcd, 0xef, 0xf9, 0xbd, 0xd5, 0xcb, 0x4e, 0x50, 0x0d, 0x7f, 0xa7, 0xd2, 0x54, 0xe9, 0x9c, + 0xc4, 0x6f, 0x9d, 0xc9, 0x73, 0xf2, 0x26, 0xa3, 0x06, 0xd3, 0x53, 0x6a, 0xb4, 0xe9, 0xe9, 0x27, + 0x09, 0x98, 0xae, 0x63, 0x53, 0x35, 0x5d, 0x32, 0xef, 0xb7, 0x9b, 0xec, 0x4a, 0x8e, 0x02, 0x24, + 0x6d, 0xa5, 0x4d, 0x8b, 0x2e, 0xc8, 0x09, 0x7b, 0x17, 0xdd, 0x84, 0x1c, 0x5d, 0x5b, 0x42, 0xb6, + 0x9a, 0x70, 0x37, 0x25, 0xd3, 0x78, 0x1e, 0xd9, 0x33, 0xc4, 0x5e, 0x07, 0xa0, 0x99, 0x18, 0x9e, + 0x4d, 0xd2, 0x1c, 0x19, 0x42, 0x61, 0x68, 0xf6, 0x16, 0xe4, 0x59, 0xe0, 0xa9, 0x2f, 0x84, 0x85, + 0x40, 0xe5, 0x28, 0xd5, 0x97, 0xb2, 0x08, 0x59, 0x96, 0x8d, 0x89, 0x19, 0xa7, 0x79, 0x80, 0x92, + 0x98, 0x9c, 0xc7, 0x70, 0xc9, 0x79, 0xd1, 0x50, 0x5a, 0x96, 0xee, 0x28, 0x5a, 0xab, 0xcd, 0x63, + 0x4b, 0xd8, 0x57, 0xd1, 0x85, 0xe2, 0xe5, 0xb3, 0xd3, 0xc5, 0xe9, 0xda, 0xd3, 0x8d, 0x1d, 0x4b, + 0x77, 0x4a, 0x3b, 0xbb, 0x2c, 0xb2, 0xc4, 0x91, 0xa7, 0x9d, 0x17, 0x0d, 0x4a, 0x6a, 0xb5, 0x39, + 0x09, 0x95, 0x61, 0xa6, 0x75, 0xf0, 0xd2, 0xb0, 0xb1, 0x82, 0x0f, 0x6c, 0xec, 0x38, 0xfc, 0x85, + 0x93, 0xe4, 0x85, 0x4c, 0xcc, 0xce, 0x93, 0xe7, 0x86, 0x8d, 0xcb, 0x34, 0x95, 0xbe, 0x5b, 0x9e, + 0x66, 0x1c, 0x21, 0x92, 0xf4, 0x8b, 0x24, 0x20, 0x7a, 0x3b, 0x55, 0x91, 0x5e, 0xef, 0xe4, 0xf5, + 0x1a, 0x0b, 0x16, 0xb4, 0x40, 0xa3, 0x8a, 0x63, 0x98, 0x04, 0x26, 0xa8, 0x8e, 0x1f, 0x74, 0xcb, + 0x87, 0x5f, 0xe4, 0xa5, 0xba, 0xbd, 0x0d, 0xc2, 0xbb, 0xc3, 0xb5, 0x90, 0xc4, 0x1a, 0x11, 0xb8, + 0xa1, 0x3a, 0xfe, 0x0b, 0xef, 0x41, 0xc6, 0xa5, 0x7c, 0xde, 0xdd, 0x5f, 0xa9, 0xe2, 0xd4, 0xd9, + 0xe9, 0x62, 0x9a, 0x09, 0xab, 0xae, 0xca, 0x69, 0x96, 0x5c, 0xd5, 0xd1, 0x32, 0x64, 0x0d, 0xd3, + 0x71, 0x55, 0x52, 0x24, 0xbe, 0x0b, 0x95, 0x63, 0x97, 0x33, 0x54, 0x39, 0xb9, 0xba, 0x2a, 0x83, + 0x97, 0xa5, 0xaa, 0xa3, 0x2a, 0x5c, 0x36, 0xf1, 0x89, 0xab, 0xd0, 0xf8, 0xf7, 0x30, 0x6b, 0x9a, + 0xb2, 0xce, 0x9e, 0x9d, 0x2e, 0xa2, 0x2d, 0x7c, 0xe2, 0x6e, 0x18, 0xc7, 0x38, 0x24, 0x02, 0x99, + 0xbd, 0x34, 0x9d, 0xf4, 0x02, 0x5f, 0x00, 0x3b, 0xa1, 0xa2, 0xb1, 0xff, 0x72, 0xce, 0xa3, 0xb2, + 0x73, 0xa7, 0x2b, 0x30, 0xe9, 0xe0, 0x17, 0xd4, 0xe8, 0x98, 0xa4, 0x46, 0xc7, 0x84, 0x43, 0x2f, + 0xe5, 0x45, 0x92, 0x7f, 0xc0, 0x85, 0x75, 0x85, 0x74, 0x52, 0x16, 0x88, 0x94, 0xf5, 0x89, 0xf2, + 0x2e, 0x7a, 0x0e, 0x7c, 0x5f, 0xc4, 0xdf, 0x8b, 0x69, 0x61, 0xdb, 0xb0, 0xbc, 0x8b, 0x2e, 0x46, + 0xda, 0xde, 0x99, 0x61, 0x12, 0xb8, 0x76, 0x77, 0x28, 0xbf, 0xf4, 0x27, 0x02, 0xcc, 0x74, 0xb5, + 0x35, 0xb7, 0x60, 0x3e, 0xea, 0x9e, 0x99, 0xc3, 0x66, 0x3c, 0xa5, 0x3b, 0xd4, 0x17, 0x40, 0xb3, + 0x74, 0x3e, 0xa5, 0x75, 0xcd, 0xd0, 0x68, 0x01, 0xb2, 0x07, 0xb6, 0x6a, 0x7a, 0x15, 0x4a, 0xd0, + 0x0a, 0x65, 0x38, 0x49, 0xde, 0x45, 0x5b, 0x20, 0x12, 0x60, 0x7d, 0xd4, 0xc0, 0x8a, 0xce, 0x0b, + 0xca, 0x27, 0xcf, 0x91, 0x6a, 0x52, 0xe0, 0xcc, 0x5e, 0x12, 0x19, 0xcc, 0xfb, 0x6a, 0xa3, 0xb1, + 0xa7, 0x6a, 0x47, 0x0a, 0xdd, 0xf1, 0xa6, 0xdb, 0x89, 0xf2, 0x94, 0x47, 0x94, 0x55, 0x17, 0x4b, + 0x75, 0x28, 0xac, 0x59, 0x86, 0xb9, 0x65, 0xe9, 0x7e, 0x14, 0xd5, 0x0a, 0xe4, 0xf7, 0x0c, 0x53, + 0xb5, 0x3b, 0x8a, 0x17, 0x08, 0x25, 0x9c, 0x17, 0x08, 0x25, 0xe7, 0x18, 0x07, 0x7f, 0x94, 0x7e, + 0x2a, 0x80, 0x18, 0x88, 0xe5, 0xda, 0x7b, 0x13, 0x40, 0x6b, 0xb4, 0x1d, 0x17, 0xdb, 0xde, 0xf4, + 0x3d, 0xc5, 0x02, 0xae, 0x4b, 0x8c, 0x5a, 0x5d, 0x95, 0x33, 0x3c, 0x43, 0x55, 0x47, 0x37, 0xbb, + 0x6f, 0xb8, 0x1b, 0x2f, 0xc2, 0x59, 0xdf, 0xbd, 0x76, 0x64, 0x3d, 0x70, 0x5c, 0xcb, 0xf6, 0xbb, + 0x37, 0x5f, 0x0f, 0xbc, 0xbb, 0x3f, 0xe9, 0x1d, 0x57, 0x98, 0x5e, 0x0f, 0x91, 0x27, 0xe0, 0xe4, + 0x18, 0xfb, 0x55, 0x4a, 0x9d, 0x5f, 0x25, 0xc6, 0xe1, 0x55, 0xe9, 0xf7, 0x04, 0x28, 0x94, 0xd8, + 0x34, 0xed, 0x4f, 0xfd, 0x43, 0x4c, 0x9d, 0x55, 0x48, 0xbb, 0x27, 0xa6, 0xd2, 0xc4, 0xfe, 0xe7, + 0x2a, 0x2f, 0x70, 0x39, 0xf7, 0xa4, 0xcb, 0x1e, 0xe9, 0x17, 0xd0, 0x5f, 0xa1, 0x2b, 0xf8, 0x4c, + 0x1c, 0x65, 0xfc, 0x1b, 0x01, 0x32, 0x35, 0x4d, 0x35, 0xfd, 0x40, 0x55, 0xb3, 0xdd, 0x64, 0x67, + 0x0a, 0xfb, 0xaa, 0x86, 0x15, 0x07, 0xe3, 0x23, 0x87, 0xdf, 0xe9, 0x33, 0x6d, 0xb6, 0x9b, 0x55, + 0x2f, 0xa5, 0x46, 0x12, 0xd0, 0x9b, 0x80, 0xfc, 0xfc, 0xa6, 0xda, 0xe0, 0xd9, 0xd9, 0xca, 0x20, + 0x7a, 0xd9, 0x4d, 0xb5, 0xc1, 0x72, 0xf7, 0x4b, 0x77, 0x71, 0xcb, 0x5b, 0x26, 0xba, 0xa5, 0x93, + 0x84, 0x7e, 0xe9, 0x34, 0x7b, 0xaa, 0x5f, 0x3a, 0xa1, 0xb3, 0xfa, 0xdc, 0xaf, 0x91, 0xe5, 0xbd, + 0x0f, 0xd7, 0xa0, 0x3c, 0x40, 0xe8, 0xbb, 0xaa, 0x63, 0xf4, 0x6b, 0xab, 0xe5, 0x95, 0x55, 0x65, + 0x77, 0xab, 0xb4, 0xbd, 0xb9, 0x59, 0xad, 0xd7, 0xcb, 0xab, 0xa2, 0x80, 0x44, 0x98, 0xea, 0xfa, + 0x2a, 0x6b, 0x62, 0x3e, 0xf5, 0x83, 0xdf, 0x59, 0x18, 0xbb, 0xff, 0x16, 0xe4, 0xba, 0x36, 0xa6, + 0x51, 0x01, 0xb2, 0x1b, 0xe5, 0x95, 0x5a, 0xb9, 0xb2, 0xbd, 0xb1, 0x4a, 0xb7, 0x55, 0xb2, 0x30, + 0xb9, 0x55, 0x5e, 0x91, 0xcb, 0xb5, 0xba, 0x28, 0xdc, 0xff, 0x6b, 0x02, 0x4c, 0xb1, 0xbb, 0xcf, + 0x64, 0x7a, 0xbd, 0x03, 0x42, 0x90, 0x97, 0xcb, 0xb5, 0xdd, 0xcd, 0xb2, 0xb2, 0xbb, 0xb5, 0xbe, + 0xb5, 0xfd, 0x7c, 0xcb, 0x2b, 0x01, 0xa5, 0xad, 0x97, 0x3f, 0x51, 0x36, 0xaa, 0x9b, 0xd5, 0xba, + 0x28, 0xa0, 0xcb, 0x30, 0xcd, 0xa9, 0xc5, 0x4f, 0xea, 0x65, 0x4e, 0x4e, 0xd0, 0xaf, 0xc9, 0x32, + 0x72, 0x75, 0x8b, 0x7e, 0xdd, 0x95, 0x25, 0x24, 0xd1, 0x55, 0xb8, 0xcc, 0x13, 0xd8, 0x07, 0x60, + 0x8b, 0xdb, 0xbb, 0x5b, 0xab, 0x2b, 0xf2, 0x27, 0x62, 0x8a, 0x17, 0xfd, 0xcf, 0x01, 0x04, 0xdf, + 0xa6, 0x26, 0x6a, 0x20, 0x6f, 0x7b, 0xb6, 0xb2, 0xb1, 0x5b, 0xae, 0x89, 0x63, 0xa4, 0x60, 0xc5, + 0x95, 0x7a, 0xa9, 0xa2, 0xc8, 0xe5, 0xda, 0xce, 0xf6, 0x56, 0xad, 0x2c, 0x0a, 0x9c, 0x6f, 0x15, + 0xa6, 0xc2, 0xb7, 0xbf, 0xa2, 0x19, 0x28, 0x94, 0x2a, 0xe5, 0xd2, 0xba, 0xf2, 0xac, 0xba, 0xa2, + 0x3c, 0xdd, 0x2d, 0xef, 0x96, 0xc5, 0x31, 0xaa, 0x55, 0x4a, 0x7c, 0xbc, 0xbb, 0xb1, 0x21, 0x0a, + 0x44, 0x2d, 0xec, 0x99, 0x7e, 0x7c, 0x56, 0x4c, 0xdc, 0xdf, 0x84, 0x6c, 0xe8, 0xab, 0x34, 0xe4, + 0x75, 0x3b, 0xbb, 0xb5, 0x8a, 0x52, 0xaf, 0x6e, 0x96, 0x6b, 0xf5, 0x95, 0xcd, 0x1d, 0x26, 0x83, + 0xd2, 0x56, 0x8a, 0xdb, 0x32, 0xd1, 0x80, 0xf7, 0x5c, 0xdf, 0xde, 0x2d, 0x55, 0xbc, 0x16, 0x90, + 0x52, 0xe9, 0xa4, 0x98, 0xbc, 0x7f, 0x02, 0x57, 0x06, 0x5c, 0x84, 0x4a, 0x1a, 0x60, 0xd7, 0xa4, + 0x5f, 0xe8, 0x10, 0xc7, 0x50, 0x0e, 0x32, 0x64, 0x16, 0xa0, 0x77, 0x0b, 0x89, 0x02, 0x4a, 0x43, + 0xea, 0xd0, 0x75, 0x5b, 0x62, 0x02, 0x4d, 0x40, 0xc2, 0x79, 0x24, 0x26, 0xc9, 0xff, 0x03, 0x47, + 0x4c, 0xa1, 0x0c, 0x8c, 0xab, 0x9f, 0xb5, 0x6d, 0x2c, 0x8e, 0xa3, 0x29, 0x48, 0xb7, 0x1d, 0x6c, + 0xef, 0x1b, 0x0d, 0x2c, 0x4e, 0x12, 0x16, 0xb3, 0xdd, 0x68, 0x88, 0x69, 0x29, 0x95, 0x9e, 0x10, + 0x27, 0xee, 0xdf, 0x80, 0xd0, 0xdd, 0x73, 0x08, 0x60, 0x62, 0x43, 0x75, 0xb1, 0xe3, 0x8a, 0x63, + 0x68, 0x12, 0x92, 0x2b, 0x8d, 0x86, 0x28, 0x3c, 0xfc, 0x1f, 0x13, 0x90, 0xf6, 0x7a, 0x24, 0xda, + 0x80, 0x71, 0xba, 0x07, 0x84, 0x16, 0x07, 0xef, 0x0e, 0xd1, 0x29, 0x75, 0xfe, 0xfa, 0x79, 0xdb, + 0x47, 0xd2, 0x18, 0xfa, 0xf3, 0x90, 0x0d, 0xa1, 0x66, 0x34, 0xf0, 0x90, 0xb5, 0x6b, 0xa7, 0x60, + 0xfe, 0xf6, 0x79, 0xd9, 0x7c, 0xf9, 0xcf, 0x21, 0xe3, 0x1b, 0xdc, 0xe8, 0xe6, 0x30, 0x73, 0xdc, + 0x93, 0x3d, 0xdc, 0x66, 0x27, 0x33, 0xa0, 0x34, 0xf6, 0xb6, 0x80, 0x6c, 0x40, 0xfd, 0xb6, 0x31, + 0x8a, 0x8a, 0xba, 0x18, 0x68, 0x7c, 0xcf, 0xdf, 0x1f, 0x29, 0x77, 0xf0, 0x4e, 0xa2, 0xac, 0xc0, + 0xc0, 0x8f, 0x56, 0x56, 0x1f, 0x7c, 0x88, 0x56, 0x56, 0x04, 0x4e, 0xa0, 0x8d, 0x11, 0x32, 0x00, + 0x22, 0xe5, 0xf7, 0x1b, 0x83, 0x91, 0xf2, 0x23, 0xec, 0x08, 0x69, 0x0c, 0x3d, 0x85, 0x14, 0x59, + 0x1f, 0x51, 0x14, 0xf4, 0xee, 0x59, 0x8f, 0xe7, 0x6f, 0x0e, 0xcd, 0xe3, 0x8b, 0x3c, 0x80, 0xfc, + 0x13, 0x4c, 0x0f, 0x6e, 0xd8, 0x45, 0xb2, 0x0e, 0xba, 0x1b, 0xed, 0x0a, 0x1b, 0xca, 0xe2, 0xbd, + 0xe2, 0xde, 0x08, 0x39, 0xfd, 0x17, 0xb5, 0x60, 0x9a, 0x7d, 0x82, 0x27, 0xfc, 0xae, 0x28, 0xff, + 0x9b, 0xbe, 0x5c, 0xde, 0xeb, 0xde, 0x1c, 0x2d, 0xb3, 0xf7, 0xc6, 0xe2, 0xbd, 0x2f, 0xfe, 0xd3, + 0xc2, 0xd8, 0x17, 0x67, 0x0b, 0xc2, 0x4f, 0xcf, 0x16, 0x84, 0x3f, 0x3c, 0x5b, 0x10, 0xfe, 0xe8, + 0x6c, 0x41, 0xf8, 0xe1, 0xcf, 0x17, 0xc6, 0x7e, 0xfa, 0xf3, 0x85, 0xb1, 0x3f, 0xfc, 0xf9, 0xc2, + 0xd8, 0xa7, 0x93, 0x5c, 0xcc, 0xde, 0x04, 0x5d, 0x17, 0x1f, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, + 0xff, 0x48, 0x01, 0x33, 0x37, 0x2d, 0x90, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -13643,6 +13740,36 @@ func (m *AddSSTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.DisallowShadowingBelow.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + if m.DisallowConflicts { + i-- + if m.DisallowConflicts { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x38 + } + if m.WriteAtRequestTimestamp { + i-- + if m.WriteAtRequestTimestamp { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + } if m.IngestAsWrites { i-- if m.IngestAsWrites { @@ -16592,12 +16719,12 @@ func (m *Header) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0xb2 } - n257, err257 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.LockTimeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout):]) - if err257 != nil { - return 0, err257 + n258, err258 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.LockTimeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout):]) + if err258 != nil { + return 0, err258 } - i -= n257 - i = encodeVarintApi(dAtA, i, uint64(n257)) + i -= n258 + i = encodeVarintApi(dAtA, i, uint64(n258)) i-- dAtA[i] = 0x1 i-- @@ -17636,12 +17763,12 @@ func (m *TokenBucketRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x38 } - n285, err285 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TargetRequestPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TargetRequestPeriod):]) - if err285 != nil { - return 0, err285 + n286, err286 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TargetRequestPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TargetRequestPeriod):]) + if err286 != nil { + return 0, err286 } - i -= n285 - i = encodeVarintApi(dAtA, i, uint64(n285)) + i -= n286 + i = encodeVarintApi(dAtA, i, uint64(n286)) i-- dAtA[i] = 0x32 if m.RequestedRU != 0 { @@ -17706,12 +17833,12 @@ func (m *TokenBucketResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x21 } - n287, err287 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TrickleDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TrickleDuration):]) - if err287 != nil { - return 0, err287 + n288, err288 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TrickleDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TrickleDuration):]) + if err288 != nil { + return 0, err288 } - i -= n287 - i = encodeVarintApi(dAtA, i, uint64(n287)) + i -= n288 + i = encodeVarintApi(dAtA, i, uint64(n288)) i-- dAtA[i] = 0x1a if m.GrantedRU != 0 { @@ -17840,12 +17967,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n291, err291 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err291 != nil { - return 0, err291 + n292, err292 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err292 != nil { + return 0, err292 } - i -= n291 - i = encodeVarintApi(dAtA, i, uint64(n291)) + i -= n292 + i = encodeVarintApi(dAtA, i, uint64(n292)) i-- dAtA[i] = 0x1a { @@ -19596,6 +19723,14 @@ func (m *AddSSTableRequest) Size() (n int) { if m.IngestAsWrites { n += 2 } + if m.WriteAtRequestTimestamp { + n += 2 + } + if m.DisallowConflicts { + n += 2 + } + l = m.DisallowShadowingBelow.Size() + n += 1 + l + sovApi(uint64(l)) return n } @@ -34385,6 +34520,79 @@ func (m *AddSSTableRequest) Unmarshal(dAtA []byte) error { } } m.IngestAsWrites = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WriteAtRequestTimestamp", 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.WriteAtRequestTimestamp = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DisallowConflicts", 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.DisallowConflicts = bool(v != 0) + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DisallowShadowingBelow", 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.DisallowShadowingBelow.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 99bab09bdc80..b397bb118730 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1571,15 +1571,113 @@ message AdminVerifyProtectedTimestampResponse { repeated FailedRange verification_failed_ranges = 4 [(gogoproto.nullable) = false]; } -// AddSSTableRequest is arguments to the AddSSTable() method, to link a file -// into the RocksDB log-structured merge-tree. +// AddSSTableRequest contains arguments to the AddSSTable method, which links an +// SST file into the Pebble log-structured merge-tree. The SST should only +// contain committed versioned values with non-zero MVCC timestamps (no intents +// or inline values) and no tombstones, but this is only fully enforced when +// WriteAtRequestTimestamp is enabled, for performance. It cannot be used in a +// transaction, cannot be split across ranges, and must be alone in a batch. +// +// By default, AddSSTable will blindly write the SST contents into Pebble, with +// fixed MVCC timestamps unaffected by pushes. This can violate many CRDB +// guarantees, including ACID, serializability and single-key linearizability: +// it mutates MVCC history (by replacing existing versions or writing below +// their timestamp) and does not respect the timestamp cache (by writing at +// timestamps that have already been read) nor the closed timestamp (by writing +// at immutable timestamps). +// +// The following parameters can be used to make AddSSTable enforce these +// guarantees, at a performance cost: +// +// * WriteAtRequestTimestamp: ensures compliance with the timestamp cache and +// closed timestamp, by rewriting SST timestamps to the request timestamp. +// +// * DisallowConflicts, DisallowShadowing, or DisallowShadowingBelow: ensures +// compliance with MVCC, by checking for conflicting keys in existing data +// instead of writing blindly. +// +// If the above parameters are not enabled, the caller must make sure these +// guarantees are upheld via other mechanisms. These options are orthogonal, +// providing different guarantees, and neither is sufficient by itself to +// enforce ACID guarantees -- they must both be enabled. See comments on these +// parameters for more details. +// +// AddSSTable always synchronizes with ongoing transactions, by taking out a +// lock span, scanning for separated intents, and resolving them. This is done +// even in the case of blind writes, since the caller is expected to make sure +// there are no ongoing writes to the ingested key span, so there should be few +// or no intents in the common case. +// +// If writing blindly (without e.g. DisallowConflicts), the range's MVCC stats +// may be incorrect as the SST stats are not adjusted for existing keys, so they +// will be marked with ContainsEstimates. The caller should recompute statistics +// after ingestion. message AddSSTableRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; bytes data = 2; - // If set, indicates that AddSSTable will not allow ingestion of keys which - // shadow already existing key entries. This disallows any key slice overlap - // regardless of the timestamps. + + // WriteAtRequestTimestamp updates all MVCC timestamps in the SST to the + // request timestamp, even if the request gets pushed. This ensures the writes + // comply with the timestamp cache and closed timestamp. + // + // Callers should always set this, except in very special circumstances when + // the timestamp cache and closed timestamp can safely be ignored (e.g. + // streaming replication into an offline tenant). + // + // Note that this alone is not sufficient to guarantee MVCC correctness, since + // it can write below or replace existing committed versions (the tscache is + // only bumped when the values are subsequently read). Use DisallowConflicts + // or DisallowShadowing in addition to guarantee MVCC correctness and + // serializability. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + bool write_at_request_timestamp = 6; + + // DisallowConflicts will check for MVCC conflicts with existing keys, i.e. + // scan for existing keys with a timestamp at or above the SST key and + // return WriteTooOldError (possibly retrying). It also ensures MVCC + // statistics are accurately updated. + // + // Note that this alone is not sufficient to guarantee serializability or + // single-key linearizability, since it can write to a timestamp that another + // reader has already observed, changing the value at that timestamp and above + // it. Use WriteAtRequestTimestamp in addition to guarantee serializability. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + // + // TODO(erikgrinaker): It might be possible to avoid this parameter if we + // could pick an MVCC timestamp that's guaranteed to not collide with + // existing keys, see: https://github.com/cockroachdb/cockroach/issues/73047. + // However, this would always lead to inaccurate MVCC stats. + bool disallow_conflicts = 7; + + // DisallowShadowing implies DisallowConflicts, and additionally rejects + // writing above keys that have an existing/visible value (but will write + // above tombstones). + // + // TODO(erikgrinaker): Consider removing this in 22.1 if all callers have + // been migrated to DisallowShadowingBelow. bool disallow_shadowing = 3; + + // DisallowShadowingBelow implies DisallowConflicts, and additionally rejects + // writing above keys that have an existing/visible value (but will write + // above tombstones). Unlike DisallowShadowing, it allows shadowing keys + // that have a timestamp at or above the given timestamp as long as the + // value is identical to the existing value, and also allows idempotent writes + // (same key/timestamp/value) at or above the given timestamp. + // + // This is a specialized method for the IMPORT INTO use-case, where we do not + // want to shadow existing keys (which could cause them to be GCed before the + // import finalizes, preventing a rollback), but we need to allow shadowing + // keys that were previously written by the import itself in the case of a + // resumption or retry. The equal value requirement is to avoid unique + // constraint violations. + // + // This parameter cannot be used together with DisallowShadowing. + // + // Added in 22.1, so check the MVCCAddSSTable version gate before using. + util.hlc.Timestamp disallow_shadowing_below = 8 [(gogoproto.nullable) = false]; + // MVCCStats, if set, is the MVCCStats for the contents of this SSTable and is // used as-is during evaluation of the AddSSTable command to update the range // MVCCStats, instead of computing the stats for the SSTable by iterating it. diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index d4a6a59880bc..2a445bc5dc74 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -315,6 +315,7 @@ func TestTenantConsumptionAddSub(t *testing.T) { func TestFlagCombinations(t *testing.T) { // Any non-zero-valued request variants that conditionally affect flags. reqVariants := []Request{ + &AddSSTableRequest{WriteAtRequestTimestamp: true}, &DeleteRangeRequest{Inline: true}, &GetRequest{KeyLocking: lock.Exclusive}, &ReverseScanRequest{KeyLocking: lock.Exclusive}, diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 1e0093847212..2abcd8b3a8a6 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -12,7 +12,6 @@ go_library( "doc.go", "engine.go", "engine_key.go", - "error.go", "in_mem.go", "intent_interleaving_iter.go", "intent_reader_writer.go", @@ -32,6 +31,7 @@ go_library( "row_counter.go", "slice.go", "slice_go1.9.go", + "sst.go", "sst_iterator.go", "sst_writer.go", "store_properties.go", @@ -111,6 +111,7 @@ go_test( "pebble_test.go", "resource_limiter_test.go", "sst_iterator_test.go", + "sst_test.go", "sst_writer_test.go", "temp_engine_test.go", ], diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 1047923b4e98..e42b0e743016 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -176,14 +176,6 @@ type MVCCIterator interface { // package-level MVCCFindSplitKey instead. For correct operation, the caller // must set the upper bound on the iterator before calling this method. FindSplitKey(start, end, minSplitKey roachpb.Key, targetSize int64) (MVCCKey, error) - // CheckForKeyCollisions checks whether any keys collide between the iterator - // and the encoded SST data specified, within the provided key range. - // maxIntents specifies the number of intents to collect and return in a - // WriteIntentError (0 disables batching, pass math.MaxInt64 to collect all). - // Returns stats on skipped KVs, or an error if a collision is found. - CheckForKeyCollisions( - sstData []byte, start, end roachpb.Key, maxIntents int64, - ) (enginepb.MVCCStats, error) // SetUpperBound installs a new upper bound for this iterator. The caller // can modify the parameter after this function returns. This must not be a // nil key. When Reader.ConsistentIterators is true, prefer creating a new diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 096ec7bc1e93..c30b2e8bb729 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1013,84 +1013,6 @@ func TestEngineDeleteIterRange(t *testing.T) { }) } -func TestMVCCIteratorCheckForKeyCollisionsMaxIntents(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - keys := []string{"aa", "bb", "cc", "dd"} - intents := []string{"a", "b", "c"} - - testcases := []struct { - maxIntents int64 - expectIntents []string - }{ - {maxIntents: -1, expectIntents: []string{"a"}}, - {maxIntents: 0, expectIntents: []string{"a"}}, - {maxIntents: 1, expectIntents: []string{"a"}}, - {maxIntents: 2, expectIntents: []string{"a", "b"}}, - {maxIntents: 3, expectIntents: []string{"a", "b", "c"}}, - {maxIntents: 4, expectIntents: []string{"a", "b", "c"}}, - } - - // Create SST with keys equal to intents at txn2TS. - sstFile := &MemFile{} - sstWriter := MakeBackupSSTWriter(sstFile) - defer sstWriter.Close() - for _, k := range intents { - key := MVCCKey{Key: roachpb.Key(k), Timestamp: txn2TS} - value := roachpb.Value{} - value.SetString("sst") - value.InitChecksum(key.Key) - require.NoError(t, sstWriter.Put(key, value.RawBytes)) - } - require.NoError(t, sstWriter.Finish()) - sstWriter.Close() - - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() - - // Write some committed keys and intents at txn1TS. - batch := engine.NewBatch() - for _, key := range keys { - require.NoError(t, batch.PutMVCC( - MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) - } - for _, key := range intents { - require.NoError(t, MVCCPut( - ctx, batch, nil, roachpb.Key(key), txn1TS, roachpb.MakeValueFromString("intent"), txn1)) - } - require.NoError(t, batch.Commit(true)) - batch.Close() - require.NoError(t, engine.Flush()) - - for _, tc := range testcases { - t.Run(fmt.Sprintf("maxIntents=%d", tc.maxIntents), func(t *testing.T) { - // Provoke and check WriteIntentErrors. - iter := engine.NewMVCCIterator( - MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.Key("z")}) - defer iter.Close() - iter.SeekGE(MVCCKey{Key: roachpb.Key("a")}) - - _, err := iter.CheckForKeyCollisions( - sstFile.Bytes(), roachpb.Key("a"), roachpb.Key("z"), tc.maxIntents) - require.Error(t, err) - writeIntentErr := &roachpb.WriteIntentError{} - require.ErrorAs(t, err, &writeIntentErr) - - actual := []string{} - for _, i := range writeIntentErr.Intents { - actual = append(actual, string(i.Key)) - } - require.Equal(t, tc.expectIntents, actual) - }) - } - }) - } -} - func TestSnapshot(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/storage/error.go b/pkg/storage/error.go deleted file mode 100644 index bbf0a2640ccc..000000000000 --- a/pkg/storage/error.go +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2017 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 storage - -// A Error wraps an error returned from a RocksDB operation. -type Error struct { - msg string -} - -// Error implements the error interface. -func (err *Error) Error() string { - return err.msg -} diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index fd7d5f4098f6..72a0ab226834 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -941,12 +941,6 @@ func (i *intentInterleavingIter) FindSplitKey( return findSplitKeyUsingIterator(i, start, end, minSplitKey, targetSize) } -func (i *intentInterleavingIter) CheckForKeyCollisions( - sstData []byte, start, end roachpb.Key, maxIntents int64, -) (enginepb.MVCCStats, error) { - return checkForKeyCollisionsGo(i, sstData, start, end, maxIntents) -} - func (i *intentInterleavingIter) SetUpperBound(key roachpb.Key) { i.iter.SetUpperBound(key) // Preceding call to SetUpperBound has confirmed that key != nil. diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 50ff4bb509c4..7f7b45e487ce 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4178,145 +4178,3 @@ func ComputeStatsForRange( ms.LastUpdateNanos = nowNanos return ms, nil } - -// checkForKeyCollisionsGo iterates through both existingIter and an SST -// iterator on the provided data in lockstep and errors out at the first key -// collision, where a collision refers to any two MVCC keys with the -// same user key, and with a different timestamp or value. -// -// An exception is made when the latest version of the colliding key is a -// tombstone from an MVCC delete in the existing data. If the timestamp of the -// SST key is greater than or equal to the timestamp of the tombstone, then it -// is not considered a collision and we continue iteration from the next key in -// the existing data. -func checkForKeyCollisionsGo( - existingIter MVCCIterator, sstData []byte, start, end roachpb.Key, maxIntents int64, -) (enginepb.MVCCStats, error) { - var skippedKVStats enginepb.MVCCStats - var intents []roachpb.Intent - sstIter, err := NewMemSSTIterator(sstData, false) - if err != nil { - return enginepb.MVCCStats{}, err - } - - defer sstIter.Close() - sstIter.SeekGE(MakeMVCCMetadataKey(start)) - if ok, err := sstIter.Valid(); err != nil || !ok { - return enginepb.MVCCStats{}, errors.Wrap(err, "checking for key collisions") - } - - ok, extErr := existingIter.Valid() - ok2, sstErr := sstIter.Valid() - for extErr == nil && sstErr == nil && ok && ok2 { - existingKey := existingIter.UnsafeKey() - existingValue := existingIter.UnsafeValue() - sstKey := sstIter.UnsafeKey() - sstValue := sstIter.UnsafeValue() - - if !existingKey.IsValue() { - var mvccMeta enginepb.MVCCMetadata - err := existingIter.ValueProto(&mvccMeta) - if err != nil { - return enginepb.MVCCStats{}, err - } - - // Check for an inline value, as these are only used in non-user data. - // This method is currently used by AddSSTable when performing an IMPORT - // INTO. We do not expect to encounter any inline values, and thus we - // report an error. - if len(mvccMeta.RawBytes) > 0 { - return enginepb.MVCCStats{}, errors.Errorf("inline values are unsupported when checking for key collisions") - } else if mvccMeta.Txn != nil { - // Check for a write intent. We keep looking for additional intents to - // return a large batch for intent resolution. The caller will likely - // resolve the returned intents and retry the call, which would be - // quadratic, so this significantly reduces the overall number of scans. - intents = append(intents, roachpb.MakeIntent(mvccMeta.Txn, existingIter.Key().Key)) - if int64(len(intents)) >= maxIntents { - return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} - } - existingIter.NextKey() - ok, extErr = existingIter.Valid() - continue - } else { - return enginepb.MVCCStats{}, errors.Errorf("intent without transaction") - } - } - - bytesCompare := bytes.Compare(existingKey.Key, sstKey.Key) - if bytesCompare == 0 { - // If the colliding key is a tombstone in the existing data, and the - // timestamp of the sst key is greater than or equal to the timestamp of - // the tombstone, then this is not considered a collision. We move the - // iterator over the existing data to the next potentially colliding key - // (skipping all versions of the deleted key), and resume iteration. - // - // If the ts of the sst key is less than that of the tombstone it is - // changing existing data, and we treat this as a collision. - if len(existingValue) == 0 && existingKey.Timestamp.LessEq(sstKey.Timestamp) { - existingIter.NextKey() - ok, extErr = existingIter.Valid() - ok2, sstErr = sstIter.Valid() - - continue - } - - // If the ingested KV has an identical timestamp and value as the existing - // data then we do not consider it to be a collision. We move the iterator - // over the existing data to the next potentially colliding key (skipping - // all versions of the current key), and resume iteration. - if sstKey.Timestamp.Equal(existingKey.Timestamp) && bytes.Equal(existingValue, sstValue) { - // Even though we skip over the KVs described above, their stats have - // already been accounted for resulting in a problem of double-counting. - // To solve this we send back the stats of these skipped KVs so that we - // can subtract them later. This enables us to construct accurate - // MVCCStats and prevents expensive recomputation in the future. - metaKeySize := int64(len(sstKey.Key) + 1) - metaValSize := int64(0) - totalBytes := metaKeySize + metaValSize - - // Update the skipped stats to account fot the skipped meta key. - skippedKVStats.LiveBytes += totalBytes - skippedKVStats.LiveCount++ - skippedKVStats.KeyBytes += metaKeySize - skippedKVStats.ValBytes += metaValSize - skippedKVStats.KeyCount++ - - // Update the stats to account for the skipped versioned key/value. - totalBytes = int64(len(sstValue)) + MVCCVersionTimestampSize - skippedKVStats.LiveBytes += totalBytes - skippedKVStats.KeyBytes += MVCCVersionTimestampSize - skippedKVStats.ValBytes += int64(len(sstValue)) - skippedKVStats.ValCount++ - - existingIter.NextKey() - ok, extErr = existingIter.Valid() - ok2, sstErr = sstIter.Valid() - - continue - } - - err := &Error{msg: existingIter.Key().Key.String()} - return enginepb.MVCCStats{}, errors.Wrap(err, "ingested key collides with an existing one") - } else if bytesCompare < 0 { - existingIter.SeekGE(MVCCKey{Key: sstKey.Key}) - } else { - sstIter.SeekGE(MVCCKey{Key: existingKey.Key}) - } - - ok, extErr = existingIter.Valid() - ok2, sstErr = sstIter.Valid() - } - - if extErr != nil { - return enginepb.MVCCStats{}, extErr - } - if sstErr != nil { - return enginepb.MVCCStats{}, sstErr - } - if len(intents) > 0 { - return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} - } - - return skippedKVStats, nil -} diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 4d140155ed38..860f73185155 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -775,14 +775,6 @@ func (p *pebbleIterator) SupportsPrev() bool { return true } -// CheckForKeyCollisions indicates if the provided SST data collides with this -// iterator in the specified range. -func (p *pebbleIterator) CheckForKeyCollisions( - sstData []byte, start, end roachpb.Key, maxIntents int64, -) (enginepb.MVCCStats, error) { - return checkForKeyCollisionsGo(p, sstData, start, end, maxIntents) -} - // GetRawIter is part of the EngineIterator interface. func (p *pebbleIterator) GetRawIter() *pebble.Iterator { return p.iter diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go new file mode 100644 index 000000000000..0bca9d93b480 --- /dev/null +++ b/pkg/storage/sst.go @@ -0,0 +1,258 @@ +// 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 storage + +import ( + "bytes" + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +// CheckSSTConflicts iterates over an SST and a Reader in lockstep and errors +// out if it finds any conflicts. This includes intents and existing keys with a +// timestamp at or above the SST key timestamp. +// +// If disallowShadowing is true, it also errors for any existing live key at the +// SST key timestamp, and ignores entries that exactly match an existing entry +// (key/value/timestamp), for backwards compatibility. +// +// If disallowShadowingBelow is non-empty, it also errors for any existing live +// key at the SST key timestamp, but allows shadowing an existing key if its +// timestamp is above the given timestamp and the values are equal. See comment +// on AddSSTableRequest.DisallowShadowingBelow for details. +// +// The given SST and reader cannot contain intents or inline values (i.e. zero +// timestamps), nor tombstones (i.e. empty values), but this is only checked for +// keys that exist in both sides, for performance. +// +// The returned MVCC statistics is a delta between the SST-only statistics and +// their effect when applied, which when added to the SST statistics will adjust +// them for existing keys and values. +func CheckSSTConflicts( + ctx context.Context, + sst []byte, + reader Reader, + start, end MVCCKey, + disallowShadowing bool, + disallowShadowingBelow hlc.Timestamp, + maxIntents int64, +) (enginepb.MVCCStats, error) { + var statsDiff enginepb.MVCCStats + var intents []roachpb.Intent + + if disallowShadowing && !disallowShadowingBelow.IsEmpty() { + return enginepb.MVCCStats{}, errors.New( + "cannot set both DisallowShadowing and DisallowShadowingBelow") + } + + extIter := reader.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: end.Key}) + defer extIter.Close() + extIter.SeekGE(start) + + sstIter, err := NewMemSSTIterator(sst, false) + if err != nil { + return enginepb.MVCCStats{}, err + } + defer sstIter.Close() + sstIter.SeekGE(start) + + extOK, extErr := extIter.Valid() + sstOK, sstErr := sstIter.Valid() + for extErr == nil && sstErr == nil && extOK && sstOK { + if err := ctx.Err(); err != nil { + return enginepb.MVCCStats{}, err + } + + extKey, extValue := extIter.UnsafeKey(), extIter.UnsafeValue() + sstKey, sstValue := sstIter.UnsafeKey(), sstIter.UnsafeValue() + + // Keep seeking the iterators until both keys are equal. + if cmp := bytes.Compare(extKey.Key, sstKey.Key); cmp < 0 { + extIter.SeekGE(MVCCKey{Key: sstKey.Key}) + extOK, extErr = extIter.Valid() + continue + } else if cmp > 0 { + sstIter.SeekGE(MVCCKey{Key: extKey.Key}) + sstOK, sstErr = sstIter.Valid() + continue + } + + // Make sure both keys are proper committed MVCC keys. Note that this is + // only checked when the key exists both in the SST and existing data, it is + // not an exhaustive check of the SST. + if !sstKey.IsValue() { + return enginepb.MVCCStats{}, errors.New("SST keys must have timestamps") + } + if len(sstValue) == 0 { + return enginepb.MVCCStats{}, errors.New("SST values cannot be tombstones") + } + if !extKey.IsValue() { + var mvccMeta enginepb.MVCCMetadata + if err = extIter.ValueProto(&mvccMeta); err != nil { + return enginepb.MVCCStats{}, err + } + if len(mvccMeta.RawBytes) > 0 { + return enginepb.MVCCStats{}, errors.New("inline values are unsupported") + } else if mvccMeta.Txn == nil { + return enginepb.MVCCStats{}, errors.New("found intent without transaction") + } else { + // If we encounter a write intent, keep looking for additional intents + // in order to return a large batch for intent resolution. The caller + // will likely resolve the returned intents and retry the call, which + // would be quadratic, so this significantly reduces the overall number + // of scans. + intents = append(intents, roachpb.MakeIntent(mvccMeta.Txn, extIter.Key().Key)) + if int64(len(intents)) >= maxIntents { + return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} + } + extIter.NextKey() + extOK, extErr = extIter.Valid() + continue + } + } + + // Allow certain idempotent writes where key/timestamp/value all match: + // + // * disallowShadowing: any matching key. + // * disallowShadowingBelow: any matching key at or above the given timestamp. + allowIdempotent := disallowShadowing || + (!disallowShadowingBelow.IsEmpty() && disallowShadowingBelow.LessEq(extKey.Timestamp)) + if allowIdempotent && sstKey.Timestamp.Equal(extKey.Timestamp) && + bytes.Equal(extValue, sstValue) { + // This SST entry will effectively be a noop, but its stats have already + // been accounted for resulting in double-counting. To address this we + // send back a stats diff for these existing KVs so that we can subtract + // them later. This enables us to construct accurate MVCCStats and + // prevents expensive recomputation in the future. + metaKeySize := int64(len(sstKey.Key) + 1) + metaValSize := int64(0) + totalBytes := metaKeySize + metaValSize + + // Update the skipped stats to account for the skipped meta key. + statsDiff.LiveBytes -= totalBytes + statsDiff.LiveCount-- + statsDiff.KeyBytes -= metaKeySize + statsDiff.ValBytes -= metaValSize + statsDiff.KeyCount-- + + // Update the stats to account for the skipped versioned key/value. + totalBytes = int64(len(sstValue)) + MVCCVersionTimestampSize + statsDiff.LiveBytes -= totalBytes + statsDiff.KeyBytes -= MVCCVersionTimestampSize + statsDiff.ValBytes -= int64(len(sstValue)) + statsDiff.ValCount-- + + extIter.NextKey() + extOK, extErr = extIter.Valid() + continue + } + + // If requested, check that we're not shadowing a live key. Note that + // we check this before we check the timestamp, and avoid returning + // a WriteTooOldError -- that error implies that the client should + // retry at a higher timestamp, but we already know that such a retry + // would fail (because it will shadow an existing key). + if len(extValue) > 0 && (disallowShadowing || !disallowShadowingBelow.IsEmpty()) { + allowShadow := !disallowShadowingBelow.IsEmpty() && + disallowShadowingBelow.LessEq(extKey.Timestamp) && bytes.Equal(extValue, sstValue) + if !allowShadow { + return enginepb.MVCCStats{}, errors.Errorf( + "ingested key collides with an existing one: %s", sstKey.Key) + } + } + + // If the existing key has a timestamp at or above the SST key, return a + // WriteTooOldError. Normally this could cause a transactional request to be + // automatically retried after a read refresh, which we would only want to + // do if AddSSTable had WriteAtRequestTimestamp set, but AddSSTable cannot + // be used in transactions so we don't need to check. + if sstKey.Timestamp.LessEq(extKey.Timestamp) { + return enginepb.MVCCStats{}, roachpb.NewWriteTooOldError( + sstKey.Timestamp, extKey.Timestamp.Next()) + } + + // If we are shadowing an existing key, we must update the stats accordingly + // to take into account the existing KV pair. + statsDiff.KeyCount-- + statsDiff.KeyBytes -= int64(len(extKey.Key) + 1) + if len(extValue) > 0 { + statsDiff.LiveCount-- + statsDiff.LiveBytes -= int64(len(extKey.Key) + 1) + statsDiff.LiveBytes -= int64(len(extValue)) + MVCCVersionTimestampSize + } + + extIter.NextKey() + extOK, extErr = extIter.Valid() + } + + if extErr != nil { + return enginepb.MVCCStats{}, extErr + } + if sstErr != nil { + return enginepb.MVCCStats{}, sstErr + } + if len(intents) > 0 { + return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} + } + + return statsDiff, nil +} + +// UpdateSSTTimestamps replaces all MVCC timestamp in the provided SST with the +// given timestamp. All keys must have a non-zero timestamp, otherwise an error +// is returned to protect against accidental inclusion of intents or inline +// values. Tombstones are also rejected opportunistically, since we're iterating +// over the entire SST anyway. +// +// TODO(erikgrinaker): This is a naïve implementation that will need significant +// optimization. For example, the SST blocks can be rewritten in parallel, and +// the Bloom filters and value checksums (which do not depend on the timestamp) +// can be copied across without recomputation. +func UpdateSSTTimestamps(sst []byte, ts hlc.Timestamp) ([]byte, error) { + sstOut := &MemFile{} + writer := MakeIngestionSSTWriter(sstOut) + defer writer.Close() + + iter, err := NewMemSSTIterator(sst, false) + if err != nil { + return nil, err + } + defer iter.Close() + + for iter.SeekGE(MVCCKey{Key: keys.MinKey}); ; iter.Next() { + if ok, err := iter.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + if iter.UnsafeKey().Timestamp.IsEmpty() { + return nil, errors.New("inline values or intents are not supported") + } + if len(iter.UnsafeValue()) == 0 { + return nil, errors.New("SST values cannot be tombstones") + } + err = writer.PutMVCC(MVCCKey{Key: iter.UnsafeKey().Key, Timestamp: ts}, iter.UnsafeValue()) + if err != nil { + return nil, err + } + } + + if err = writer.Finish(); err != nil { + return nil, err + } + + return sstOut.Bytes(), nil +} diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index c7c45243c6d9..449a0426896f 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -94,7 +94,7 @@ func (r *sstIterator) SeekGE(key MVCCKey) { r.iterValid = false r.err = r.iter.Error() } - if r.iterValid && r.err == nil && r.verify { + if r.iterValid && r.err == nil && r.verify && r.mvccKey.IsValue() { r.err = roachpb.Value{RawBytes: r.value}.Verify(r.mvccKey.Key) } } @@ -117,7 +117,7 @@ func (r *sstIterator) Next() { r.iterValid = false r.err = r.iter.Error() } - if r.iterValid && r.err == nil && r.verify { + if r.iterValid && r.err == nil && r.verify && r.mvccKey.IsValue() { r.err = roachpb.Value{RawBytes: r.value}.Verify(r.mvccKey.Key) } } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go new file mode 100644 index 000000000000..f29225c52284 --- /dev/null +++ b/pkg/storage/sst_test.go @@ -0,0 +1,171 @@ +// 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 storage + +import ( + "context" + "encoding/binary" + "fmt" + "math/rand" + "os" + "runtime/pprof" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestCheckSSTConflictsMaxIntents(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + keys := []string{"aa", "bb", "cc", "dd"} + intents := []string{"a", "b", "c"} + start, end := "a", "z" + + testcases := []struct { + maxIntents int64 + expectIntents []string + }{ + {maxIntents: -1, expectIntents: []string{"a"}}, + {maxIntents: 0, expectIntents: []string{"a"}}, + {maxIntents: 1, expectIntents: []string{"a"}}, + {maxIntents: 2, expectIntents: []string{"a", "b"}}, + {maxIntents: 3, expectIntents: []string{"a", "b", "c"}}, + {maxIntents: 4, expectIntents: []string{"a", "b", "c"}}, + } + + // Create SST with keys equal to intents at txn2TS. + sstFile := &MemFile{} + sstWriter := MakeBackupSSTWriter(sstFile) + defer sstWriter.Close() + for _, k := range intents { + key := MVCCKey{Key: roachpb.Key(k), Timestamp: txn2TS} + value := roachpb.Value{} + value.SetString("sst") + value.InitChecksum(key.Key) + require.NoError(t, sstWriter.Put(key, value.RawBytes)) + } + require.NoError(t, sstWriter.Finish()) + sstWriter.Close() + + for _, engineImpl := range mvccEngineImpls { + t.Run(engineImpl.name, func(t *testing.T) { + ctx := context.Background() + engine := engineImpl.create() + defer engine.Close() + + // Write some committed keys and intents at txn1TS. + batch := engine.NewBatch() + for _, key := range keys { + require.NoError(t, batch.PutMVCC(MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) + } + for _, key := range intents { + require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, roachpb.MakeValueFromString("intent"), txn1)) + } + require.NoError(t, batch.Commit(true)) + batch.Close() + require.NoError(t, engine.Flush()) + + for _, tc := range testcases { + t.Run(fmt.Sprintf("maxIntents=%d", tc.maxIntents), func(t *testing.T) { + // Provoke and check WriteIntentErrors. + startKey, endKey := MVCCKey{Key: roachpb.Key(start)}, MVCCKey{Key: roachpb.Key(end)} + _, err := CheckSSTConflicts(ctx, sstFile.Bytes(), engine, startKey, endKey, + false /*disallowShadowing*/, hlc.Timestamp{} /*disallowShadowingBelow*/, tc.maxIntents) + require.Error(t, err) + writeIntentErr := &roachpb.WriteIntentError{} + require.ErrorAs(t, err, &writeIntentErr) + + actual := []string{} + for _, i := range writeIntentErr.Intents { + actual = append(actual, string(i.Key)) + } + require.Equal(t, tc.expectIntents, actual) + }) + } + }) + } +} + +func BenchmarkUpdateSSTTimestamps(b *testing.B) { + const ( + modeZero = iota + 1 // all zeroes + modeCounter // uint64 counter in first 8 bytes + modeRandom // random values + + sstSize = 0 + keyCount = 500000 + valueSize = 8 + valueMode = modeRandom + profile = false // cpuprofile.pprof + ) + + if sstSize > 0 && keyCount > 0 { + b.Fatal("Can't set both sstSize and keyCount") + } + + b.StopTimer() + + r := rand.New(rand.NewSource(7)) + + sstFile := &MemFile{} + writer := MakeIngestionSSTWriter(sstFile) + defer writer.Close() + + key := make([]byte, 8) + value := make([]byte, valueSize) + ts := hlc.Timestamp{WallTime: 1} + var i uint64 + for i = 0; (keyCount > 0 && i < keyCount) || (sstSize > 0 && sstFile.Len() < sstSize); i++ { + binary.BigEndian.PutUint64(key, i) + + switch valueMode { + case modeZero: + case modeCounter: + binary.BigEndian.PutUint64(value, i) + ts.WallTime++ + case modeRandom: + r.Read(value) + ts.WallTime = r.Int63() + ts.Logical = r.Int31() + default: + b.Fatalf("unknown value mode %d", valueMode) + } + + var v roachpb.Value + v.SetBytes(value) + v.InitChecksum(key) + + require.NoError(b, writer.PutMVCC(MVCCKey{Key: key, Timestamp: ts}, v.RawBytes)) + } + writer.Close() + b.Logf("%vMB %v keys", sstFile.Len()/1e6, i) + + if profile { + f, err := os.Create("cpuprofile.pprof") + require.NoError(b, err) + defer f.Close() + + require.NoError(b, pprof.StartCPUProfile(f)) + defer pprof.StopCPUProfile() + } + + b.StartTimer() + for i := 0; i < b.N; i++ { + ts := hlc.Timestamp{WallTime: 1634899098417970999, Logical: 9} + _, err := UpdateSSTTimestamps(sstFile.Bytes(), ts) + require.NoError(b, err) + } +} diff --git a/pkg/testutils/trace.go b/pkg/testutils/trace.go index 83d605baa393..90c2f78b0abc 100644 --- a/pkg/testutils/trace.go +++ b/pkg/testutils/trace.go @@ -50,25 +50,3 @@ func MatchInOrder(s string, res ...string) error { } return nil } - -// MatchEach matches interprets the given slice of strings as a slice of -// regular expressions and checks that they individually match against the given string. -// For example, if s=abcdefg and res=bc,ab,fg no error is returned, whereas -// res=abc,cdg would return a descriptive error about failing to match cde. -func MatchEach(s string, res ...string) error { - for i := range res { - reStr := "(?ms)" + res[i] - re, err := regexp.Compile(reStr) - if err != nil { - return errors.Errorf("regexp %d (%q) does not compile: %s", i, reStr, err) - } - if re.FindStringIndex(s) == nil { - // Not found. - return errors.Errorf( - "unable to find regexp %d (%q) in string:\n\n%s", - i, reStr, s, - ) - } - } - return nil -}