diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 2e534d59dee7..d26e72bb2eac 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-12 set the active cluster version in the format '.' +version version 21.2-14 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index b58e9981d400..ffa56d38fee7 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-12set the active cluster version in the format '.' +versionversion21.2-14set 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..191fcbb3c809 100644 --- a/pkg/ccl/importccl/bench_test.go +++ b/pkg/ccl/importccl/bench_test.go @@ -135,8 +135,9 @@ 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 */, 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 e3eedde9ae03..c9a66ff901dd 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -278,13 +278,15 @@ const ( // DrainingNamesMigration adds the migration which guarantees that no // descriptors have draining names. DrainingNamesMigration - // TraceIDDoesntImplyStructuredRecording changes the contract about the kind // of span that RPCs get on the server depending on the tracing context. TraceIDDoesntImplyStructuredRecording // AlterSystemTableStatisticsAddAvgSizeCol adds the column avgSize to the // table system.table_statistics that contains a new statistic. AlterSystemTableStatisticsAddAvgSizeCol + // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new + // WriteAtRequestTimestamp and DisallowConflicts parameters. + MVCCAddSSTable // ************************************************* // Step (1): Add new versions here. @@ -490,6 +492,10 @@ var versionsSingleton = keyedVersions{ Key: AlterSystemTableStatisticsAddAvgSizeCol, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 12}, }, + { + Key: MVCCAddSSTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 14}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 9714b56f136d..40398527ddf1 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -49,11 +49,12 @@ func _() { _ = x[DrainingNamesMigration-38] _ = x[TraceIDDoesntImplyStructuredRecording-39] _ = x[AlterSystemTableStatisticsAddAvgSizeCol-40] + _ = x[MVCCAddSSTable-41] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeCol" +const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTable" -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} +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, 924} 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..039cb8ec986d 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -764,9 +764,11 @@ func (b *Batch) adminRelocateRange( func (b *Batch) addSSTable( s, e interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, stats *enginepb.MVCCStats, ingestAsWrites bool, + writeAtRequestTimestamp bool, ) { begin, err := marshalKey(s) if err != nil { @@ -783,10 +785,12 @@ func (b *Batch) addSSTable( Key: begin, EndKey: end, }, - Data: data, - DisallowShadowing: disallowShadowing, - MVCCStats: stats, - IngestAsWrites: ingestAsWrites, + Data: data, + DisallowConflicts: disallowConflicts, + DisallowShadowing: disallowShadowing, + 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..72fc495296af 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -385,10 +385,12 @@ type SSTSender interface { ctx context.Context, begin, end interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, stats *enginepb.MVCCStats, ingestAsWrites bool, batchTs hlc.Timestamp, + writeAtBatchTs bool, ) error SplitAndScatter(ctx context.Context, key roachpb.Key, expirationTime hlc.Timestamp) error } @@ -457,7 +459,8 @@ 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, &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..67af75925018 100644 --- a/pkg/kv/bulk/sst_batcher_test.go +++ b/pkg/kv/bulk/sst_batcher_test.go @@ -260,10 +260,12 @@ func (m mockSender) AddSSTable( ctx context.Context, begin, end interface{}, data []byte, + disallowConflicts bool, disallowShadowing bool, _ *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..e81c449776df 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -664,17 +664,23 @@ func (db *DB) AdminRelocateRange( // AddSSTable links a file into the RocksDB log-structured merge-tree. Existing // data in the range is cleared. +// +// The disallowConflicts 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, 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, 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..1fa04c485d6a 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,873 @@ 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 allowShadowing = false + const allowConflicts = false + 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, 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, 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, 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, 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, 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, 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 +908,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, 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 +1157,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 +1180,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 +1329,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..686c4d28388b 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -4411,15 +4411,100 @@ 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 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 compliance, since + // it can write below or replace versions that were written by another writer + // (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. + 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). 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 +8039,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 +13732,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 +16711,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 +17755,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 +17825,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 +17959,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 +19715,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 +34512,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..f2a8b86633cc 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1571,15 +1571,105 @@ 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 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 compliance, since + // it can write below or replace versions that were written by another writer + // (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. + bool disallow_conflicts = 7; + + // DisallowShadowing implies DisallowConflicts, and additionally rejects + // writing above keys that have an existing/visible value (but will write + // above tombstones). 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 -}