From 768d3b6281a3d1ae492b78f1c782acf3f9a53573 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 28 Oct 2021 08:23:59 +0000 Subject: [PATCH] kvserver: add MVCC-compliant `AddSSTable` variant Previously, `AddSSTable` did not comply with MVCC or closed timestamp invariants: MVCC timestamps in the SST were client-generated and used as-is, which could mutate MVCC history and write below the closed timestamp. This patch adds a `WriteAtRequestTimestamp` parameter and corresponding `MVCCAddSSTable` version gate, which will rewrite the SST MVCC timestamps to be at the request's write timestamp and error out if they conflict with existing MVCC writes. The implementation here is correct but unoptimized, simply iterating over the existing SST and writing a new one with updated timestamps. This has significant room for improvement, which will be explored later. An additional `Blind` parameter is added to write the SST blindly, without checking for write conflicts. This is believed to uphold serializability, but will yield incorrect MVCC statistics. Release note: None --- pkg/clusterversion/cockroach_versions.go | 8 + pkg/clusterversion/key_string.go | 5 +- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 77 +- .../batcheval/cmd_add_sstable_test.go | 20 +- pkg/kv/kvserver/spanset/batch.go | 7 - pkg/roachpb/api.pb.go | 1076 +++++++++-------- pkg/roachpb/api.proto | 27 + pkg/storage/engine.go | 8 - pkg/storage/engine_test.go | 78 -- pkg/storage/error.go | 21 - pkg/storage/intent_interleaving_iter.go | 6 - pkg/storage/mvcc.go | 142 --- pkg/storage/pebble_iterator.go | 8 - pkg/storage/sst.go | 224 ++++ pkg/storage/sst_test.go | 168 +++ 15 files changed, 1059 insertions(+), 816 deletions(-) delete mode 100644 pkg/storage/error.go create mode 100644 pkg/storage/sst.go create mode 100644 pkg/storage/sst_test.go diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 543f40892287..7aad8cb40432 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -277,6 +277,10 @@ const ( // limits when splitting requests. TargetBytesAvoidExcess + // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new + // WriteAtRequestTimestamp parameter. + MVCCAddSSTable + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -473,6 +477,10 @@ var versionsSingleton = keyedVersions{ Key: TargetBytesAvoidExcess, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 4}, }, + { + Key: MVCCAddSSTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 6}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 1d2e73449f2f..9fc1738978cb 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -47,11 +47,12 @@ func _() { _ = x[V21_2-36] _ = x[Start22_1-37] _ = x[TargetBytesAvoidExcess-38] + _ = x[MVCCAddSSTable-39] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcess" +const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessMVCCAddSSTable" -var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 446, 471, 488, 517, 550, 573, 592, 611, 634, 650, 670, 691, 713, 732, 766, 780, 810, 815, 824, 846} +var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 446, 471, 488, 517, 550, 573, 592, 611, 634, 650, 670, 691, 713, 732, 766, 780, 810, 815, 824, 846, 860} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 78f7c8ff6edd..37a85b2cd5c7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -93,7 +93,7 @@ func declareKeysAddSSTable( // | Streaming replication | false | Key TS | Offline tenant | // args := req.(*roachpb.AddSSTableRequest) - if args.DisallowShadowing { + if (args.WriteAtRequestTimestamp && !args.Blind) || args.DisallowShadowing { DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans) } else { DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans) @@ -110,20 +110,36 @@ func EvalAddSSTable( h := cArgs.Header ms := cArgs.Stats mvccStartKey, mvccEndKey := storage.MVCCKey{Key: args.Key}, storage.MVCCKey{Key: args.EndKey} + sst := args.Data + + if args.Blind { + if args.DisallowShadowing { + return result.Result{}, errors.New("Blind cannot be combined with DisallowShadowing") + } + if args.IngestAsWrites { + return result.Result{}, errors.New("Blind cannot be combined with IngestAsWrites") + } + } var span *tracing.Span + var err error ctx, span = tracing.ChildSpan(ctx, fmt.Sprintf("AddSSTable [%s,%s)", args.Key, args.EndKey)) defer span.Finish() log.Eventf(ctx, "evaluating AddSSTable [%s,%s)", mvccStartKey.Key, mvccEndKey.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 { + if args.WriteAtRequestTimestamp { + sst, err = storage.UpdateSSTTimestamps(sst, h.WriteTimestamp()) + if err != nil { + return result.Result{}, errors.Wrap(err, "updating SST timestamps") + } + } + + var statsDiff enginepb.MVCCStats + checkConflicts := (args.WriteAtRequestTimestamp && !args.Blind) || args.DisallowShadowing + if checkConflicts { maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) - skippedKVStats, err = checkForKeyCollisions( - ctx, readWriter, mvccStartKey, mvccEndKey, args.Data, maxIntents) + statsDiff, err = storage.CheckSSTConflicts( + ctx, sst, readWriter, mvccStartKey, mvccEndKey, args.DisallowShadowing, maxIntents) if err != nil { return result.Result{}, errors.Wrap(err, "checking for key collisions") } @@ -132,7 +148,7 @@ func EvalAddSSTable( // 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) + dataIter, err := storage.NewMemSSTIterator(sst, true) if err != nil { return result.Result{}, err } @@ -156,10 +172,9 @@ 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) @@ -243,8 +258,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(statsDiff) stats.ContainsEstimates = 0 } else { stats.ContainsEstimates++ @@ -253,8 +268,8 @@ 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)) + 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)) dataIter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) for { ok, err := dataIter.Valid() @@ -272,7 +287,7 @@ func EvalAddSSTable( return result.Result{}, err } } else { - if err := readWriter.PutMVCC(dataIter.UnsafeKey(), dataIter.UnsafeValue()); err != nil { + if err := readWriter.PutMVCC(k, dataIter.UnsafeValue()); err != nil { return result.Result{}, err } } @@ -284,31 +299,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 fd694eaeb4d4..daea5f8de047 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -571,7 +571,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"a\"") { + if !testutils.IsError(err, "SST key \"a\"/0.000000007,0 shadows existing key \"a\"/0.000000002,0") { t.Fatalf("%+v", err) } } @@ -598,7 +598,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"g\"") { + if !testutils.IsError(err, "WriteTooOldError: write at timestamp 0.000000004,0 too old; wrote at 0.000000005,1") { t.Fatalf("%+v", err) } } @@ -627,7 +627,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"z\"") { + if !testutils.IsError(err, "SST key \"z\"/0.000000003,0 shadows existing key \"z\"/0.000000002,0") { t.Fatalf("%+v", err) } } @@ -689,7 +689,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { + if !testutils.IsError(err, "WriteTooOldError: write at timestamp 0.000000003,0 too old; wrote at 0.000000005,1") { t.Fatalf("%+v", err) } } @@ -718,7 +718,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"b\"") { + if !testutils.IsError(err, "WriteTooOldError: write at timestamp 0.000000004,0 too old; wrote at 0.000000006,1") { t.Fatalf("%+v", err) } } @@ -748,7 +748,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { + if !testutils.IsError(err, "WriteTooOldError: write at timestamp 0.000000004,0 too old; wrote at 0.000000005,1") { t.Fatalf("%+v", err) } } @@ -840,7 +840,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "inline values are unsupported when checking for key collisions") { + if !testutils.IsError(err, "inline values are unsupported") { t.Fatalf("%+v", err) } } @@ -900,7 +900,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { + if !testutils.IsError(err, "SST key \"y\"/0.000000006,0 shadows existing key \"y\"/0.000000005,0") { t.Fatalf("%+v", err) } } @@ -929,7 +929,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"y\"") { + if !testutils.IsError(err, "WriteTooOldError: write at timestamp 0.000000005,0 too old; wrote at 0.000000005,1") { t.Fatalf("%+v", err) } } @@ -958,7 +958,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) { } _, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil) - if !testutils.IsError(err, "ingested key collides with an existing one: \"z\"") { + if !testutils.IsError(err, "SST key \"z\"/0.000000003,0 shadows existing key \"z\"/0.000000002,0") { t.Fatalf("%+v", err) } } 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.pb.go b/pkg/roachpb/api.pb.go index 3708eac365d6..c9625da3dedf 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -4428,6 +4428,31 @@ type AddSSTableRequest struct { // the usual write pipeline (on-disk raft log, WAL, etc). // TODO(dt): https://github.com/cockroachdb/cockroach/issues/34579#issuecomment-544627193 IngestAsWrites bool `protobuf:"varint,5,opt,name=ingest_as_writes,json=ingestAsWrites,proto3" json:"ingest_as_writes,omitempty"` + // WriteAtRequestTimestamp will update all MVCC timestamps in the SST to be at + // the request timestamp, even if the request gets pushed. The writes will + // comply with MVCC, in that they cannot write below the timestamp of an + // existing key or below an intent, and they respect the closed timestamp. All + // SST entries must have a non-zero MVCC timestamp, and cannot be tombstones. + // + // Use of this parameter requires the MVCCAddSSTable version gate. + WriteAtRequestTimestamp bool `protobuf:"varint,6,opt,name=write_at_request_timestamp,json=writeAtRequestTimestamp,proto3" json:"write_at_request_timestamp,omitempty"` + // Blind will write the SST blindly when WriteAtRequestTimestamp is set, + // without checking for conflicting keys and intents, and without + // synchronizing with transactions by taking out locks. This is still + // serializable because: + // + // * AddSSTable cannot be in a transaction nor write intents. + // * AddSSTable cannot be split across ranges, and is always alone in a batch. + // * Write latches provide isolation from concurrent requests. + // * The timestamp cache ensures noone has read above the write timestamp. + // * The write timestamp is globally unique. (FIXME: verify this) + // + // It also complies with MVCC, because the timestamp cache ensures noone + // has observed the absence of the value, and with the closed timestamp. + // + // However, it will not update the MVCC statistics accurately, overcounting + // any keys that already exist and setting ContainsEstimates. + Blind bool `protobuf:"varint,7,opt,name=blind,proto3" json:"blind,omitempty"` } func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } @@ -7926,214 +7951,214 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 9393 bytes of a gzipped FileDescriptorProto + // 9424 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x57, 0x96, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x24, 0x4b, 0x57, 0xfd, 0xa3, 0x56, 0xdb, 0x52, 0x77, 0xb5, 0xfb, 0x77, 0x6d, 0xca, 0xdd, 0x6d, 0xc7, 0x1e, 0xdb, 0xe3, 0x59, 0x91, 0x62, 0x37, 0xa9, 0xbf, 0x56, 0x17, 0xa9, 0x6e, 0xd8, 0x3b, 0x9b, 0xda, 0x52, 0xd5, 0x95, 0x54, 0x23, 0xb2, 0x8a, 0x5d, 0x55, 0x54, 0x8b, 0x06, 0xf2, 0x90, 0x64, 0x31, 0x99, 0xa7, 0x60, 0x02, 0x04, 0xd8, 0x19, 0x6c, 0xb0, 0xf0, 0x66, 0x17, 0xc9, 0x43, 0x02, 0x24, 0x48, 0x82, 0x04, 0x1b, 0x24, 0x3b, 0x8f, - 0x31, 0x82, 0x49, 0x32, 0xf3, 0xb6, 0x08, 0x10, 0x65, 0xa3, 0xc9, 0xc3, 0x0e, 0x82, 0x20, 0x41, - 0x12, 0x60, 0x01, 0x03, 0xf9, 0xc1, 0xfd, 0xa9, 0x1f, 0x92, 0x45, 0x8a, 0x6a, 0x97, 0xb3, 0x06, - 0xf6, 0x45, 0x62, 0x9d, 0x7b, 0xcf, 0xa9, 0x7b, 0xcf, 0xfd, 0x3b, 0xdf, 0xbd, 0xe7, 0xdc, 0x82, - 0x59, 0xdb, 0x52, 0xb5, 0x83, 0xf6, 0xee, 0xb2, 0xda, 0x36, 0x8a, 0x6d, 0xdb, 0x72, 0x2d, 0x34, - 0xab, 0x59, 0xda, 0x21, 0x25, 0x17, 0x79, 0xe2, 0xc2, 0x45, 0x6c, 0xdb, 0x96, 0xed, 0xb4, 0x77, - 0x97, 0xd9, 0x0f, 0x96, 0x73, 0xe1, 0xde, 0xe1, 0xd1, 0xf2, 0xe1, 0x91, 0x83, 0xed, 0x23, 0x6c, - 0x2f, 0x6b, 0x96, 0xa9, 0x75, 0x6c, 0x1b, 0x9b, 0x5a, 0x77, 0xb9, 0x69, 0x69, 0x87, 0xf4, 0x8f, - 0x61, 0xee, 0x47, 0xe5, 0xb5, 0xb1, 0xaa, 0x3b, 0x9d, 0x56, 0x4b, 0xb5, 0xbb, 0xcb, 0x54, 0x2c, - 0x7f, 0xe0, 0x79, 0x91, 0x57, 0x28, 0x5d, 0x75, 0x55, 0x4e, 0xbb, 0xe0, 0xd1, 0x7a, 0x4a, 0x70, - 0xc9, 0xa3, 0xb6, 0xb0, 0xab, 0x86, 0x72, 0x5f, 0xf1, 0xe8, 0x4e, 0x5b, 0x35, 0x15, 0xcd, 0x32, - 0xf7, 0x0c, 0xaf, 0x20, 0x57, 0x1d, 0xd7, 0xb2, 0xd5, 0x7d, 0xbc, 0x8c, 0xcd, 0x7d, 0xc3, 0xc4, - 0x84, 0xf7, 0x48, 0xd3, 0x78, 0xe2, 0x6b, 0x91, 0x89, 0x0f, 0x79, 0xea, 0x7c, 0xc7, 0x35, 0x9a, - 0xcb, 0x07, 0x4d, 0x6d, 0xd9, 0x35, 0x5a, 0xd8, 0x71, 0xd5, 0x56, 0xdb, 0xab, 0x1d, 0x4d, 0x71, - 0x6d, 0x55, 0x33, 0xcc, 0x7d, 0xef, 0x7f, 0x7b, 0x77, 0xd9, 0xc6, 0x9a, 0x65, 0xeb, 0x58, 0x57, - 0x48, 0x39, 0xbc, 0x9a, 0xec, 0x5b, 0xfb, 0x16, 0xfd, 0xb9, 0x4c, 0x7e, 0x71, 0xea, 0xe2, 0xbe, - 0x65, 0xed, 0x37, 0xf1, 0x32, 0x7d, 0xda, 0xed, 0xec, 0x2d, 0xeb, 0x1d, 0x5b, 0x75, 0x0d, 0x8b, - 0x73, 0x49, 0xff, 0x54, 0x80, 0x9c, 0x8c, 0x5f, 0x74, 0xb0, 0xe3, 0x56, 0xb1, 0xaa, 0x63, 0x1b, - 0x5d, 0x81, 0xe4, 0x21, 0xee, 0xce, 0x27, 0xaf, 0x09, 0x77, 0x66, 0x4a, 0xd3, 0x5f, 0x9e, 0x2c, - 0x25, 0xd7, 0x71, 0x57, 0x26, 0x34, 0x74, 0x0d, 0xa6, 0xb1, 0xa9, 0x2b, 0x24, 0x39, 0xd5, 0x9b, - 0x3c, 0x85, 0x4d, 0x7d, 0x1d, 0x77, 0xd1, 0x77, 0x21, 0xed, 0x10, 0x69, 0xa6, 0x86, 0xe7, 0x27, - 0xaf, 0x09, 0x77, 0x26, 0x4b, 0xbf, 0xfa, 0xe5, 0xc9, 0xd2, 0x47, 0xfb, 0x86, 0x7b, 0xd0, 0xd9, - 0x2d, 0x6a, 0x56, 0x6b, 0xd9, 0xef, 0x05, 0xfa, 0x6e, 0xf0, 0x7b, 0xb9, 0x7d, 0xb8, 0xbf, 0xdc, - 0xaf, 0xa3, 0x62, 0xe3, 0xd8, 0xac, 0xe3, 0x17, 0xb2, 0x2f, 0x71, 0x2d, 0x95, 0x16, 0xc4, 0xc4, - 0x5a, 0x2a, 0x9d, 0x10, 0x93, 0xd2, 0xdf, 0x4f, 0x40, 0x5e, 0xc6, 0x4e, 0xdb, 0x32, 0x1d, 0xcc, - 0x4b, 0xfe, 0x36, 0x24, 0xdd, 0x63, 0x93, 0x96, 0x3c, 0xfb, 0x60, 0xb1, 0x38, 0xd0, 0xdf, 0x8a, - 0x0d, 0x5b, 0x35, 0x1d, 0x55, 0x23, 0xd5, 0x97, 0x49, 0x56, 0xf4, 0x3e, 0x64, 0x6d, 0xec, 0x74, - 0x5a, 0x98, 0x2a, 0x92, 0x56, 0x2a, 0xfb, 0xe0, 0x72, 0x04, 0x67, 0xbd, 0xad, 0x9a, 0x32, 0xb0, - 0xbc, 0xe4, 0x37, 0x5a, 0x85, 0x1c, 0xe7, 0xb4, 0xb1, 0xea, 0x58, 0xe6, 0xfc, 0xf4, 0x35, 0xe1, - 0x4e, 0xfe, 0xc1, 0x52, 0x04, 0xaf, 0x4c, 0xf3, 0xc9, 0x34, 0x9b, 0x3c, 0x63, 0x87, 0x9e, 0xd0, - 0x3d, 0x98, 0xe5, 0x52, 0x4c, 0x7c, 0xec, 0x2a, 0xbb, 0x5d, 0x17, 0x3b, 0xf3, 0x99, 0x6b, 0xc2, - 0x9d, 0xa4, 0x5c, 0x60, 0x09, 0x5b, 0xf8, 0xd8, 0x2d, 0x11, 0x32, 0xba, 0x02, 0x69, 0xb3, 0xd3, - 0x22, 0xca, 0x77, 0xa8, 0x6a, 0x93, 0xf2, 0xb4, 0xd9, 0x69, 0xad, 0xe3, 0xae, 0x83, 0xae, 0x42, - 0x86, 0x24, 0x31, 0xf6, 0x34, 0x4d, 0x23, 0x79, 0x29, 0xdf, 0x5a, 0x2a, 0x3d, 0x25, 0x4e, 0x4b, - 0xbf, 0x27, 0x00, 0x3c, 0xc6, 0x2e, 0x6f, 0x6a, 0x54, 0x82, 0xa9, 0x03, 0xaa, 0xb4, 0x79, 0x81, - 0xd6, 0xf9, 0x5a, 0x64, 0xb9, 0x43, 0xdd, 0xa2, 0x94, 0xfe, 0xe2, 0x64, 0x69, 0xe2, 0x67, 0x27, - 0x4b, 0x82, 0xcc, 0x39, 0xd1, 0x53, 0xc8, 0x1e, 0xe2, 0xae, 0xc2, 0xc7, 0xe3, 0x7c, 0x82, 0x2a, - 0xe0, 0xed, 0x90, 0xa0, 0xc3, 0xa3, 0xa2, 0x37, 0x34, 0x8b, 0xa1, 0x61, 0x5c, 0x24, 0x1c, 0xc5, - 0xba, 0x6b, 0x63, 0x73, 0xdf, 0x3d, 0x90, 0xe1, 0x10, 0x77, 0x37, 0x98, 0x0c, 0xe9, 0x27, 0x02, - 0x64, 0x69, 0x29, 0x59, 0xbb, 0xa2, 0x72, 0x5f, 0x31, 0xaf, 0x47, 0xab, 0x37, 0xd4, 0x09, 0x22, - 0xca, 0x59, 0x84, 0xc9, 0x23, 0xb5, 0xd9, 0xc1, 0xb4, 0x84, 0xd9, 0x07, 0xf3, 0x11, 0x32, 0x9e, - 0x91, 0x74, 0x99, 0x65, 0x43, 0x1f, 0xc2, 0x8c, 0x61, 0xba, 0xd8, 0x74, 0x15, 0xc6, 0x96, 0x3c, - 0x83, 0x2d, 0xcb, 0x72, 0xd3, 0x07, 0xe9, 0x9f, 0x08, 0x00, 0xdb, 0x9d, 0x58, 0xf5, 0xfc, 0xce, - 0x98, 0xe5, 0x2f, 0xa5, 0x08, 0xab, 0x57, 0x8b, 0x4b, 0x30, 0x65, 0x98, 0x4d, 0xc3, 0x64, 0xe5, - 0x4f, 0xcb, 0xfc, 0x09, 0x5d, 0x80, 0xc9, 0xdd, 0xa6, 0x61, 0xea, 0xb4, 0xb3, 0xa7, 0x65, 0xf6, - 0x20, 0xc9, 0x90, 0xa5, 0xa5, 0x8e, 0x51, 0xef, 0xd2, 0x49, 0x02, 0x2e, 0x96, 0x2d, 0x53, 0x37, - 0xc8, 0x78, 0x53, 0x9b, 0xdf, 0x08, 0xad, 0xac, 0xc1, 0x05, 0x1d, 0xb7, 0x6d, 0xac, 0xa9, 0x2e, - 0xd6, 0x15, 0x7c, 0xdc, 0x1e, 0xb3, 0x8d, 0x51, 0xc0, 0x55, 0x39, 0x6e, 0x53, 0x1a, 0x19, 0x75, - 0x44, 0x00, 0x1b, 0x75, 0x53, 0x64, 0x3e, 0x94, 0xd3, 0xf8, 0xb8, 0xcd, 0x46, 0x6b, 0xa4, 0x9a, - 0xd1, 0x3b, 0x70, 0x59, 0x6d, 0x36, 0xad, 0x97, 0x8a, 0xb1, 0xa7, 0xe8, 0x16, 0x76, 0x14, 0xd3, - 0x72, 0x15, 0x7c, 0x6c, 0x38, 0x2e, 0x1d, 0xd2, 0x69, 0x79, 0x8e, 0x26, 0xd7, 0xf6, 0x56, 0x2d, - 0xec, 0x6c, 0x59, 0x6e, 0x85, 0x24, 0x85, 0x9a, 0x72, 0x3a, 0xdc, 0x94, 0xd2, 0xaf, 0xc3, 0xa5, - 0x7e, 0xfd, 0xc6, 0xd9, 0x7e, 0x3f, 0x15, 0x20, 0x5f, 0x33, 0x0d, 0xf7, 0x1b, 0xd1, 0x70, 0xbe, - 0x3e, 0x93, 0x61, 0x7d, 0xde, 0x03, 0x71, 0x4f, 0x35, 0x9a, 0x4f, 0xcc, 0x86, 0xd5, 0xda, 0x75, - 0x5c, 0xcb, 0xc4, 0x0e, 0x57, 0xf8, 0x00, 0x5d, 0x7a, 0x06, 0x05, 0xbf, 0x36, 0x71, 0xaa, 0xc9, - 0x05, 0xb1, 0x66, 0x6a, 0x36, 0x6e, 0x61, 0x33, 0x56, 0x3d, 0xbd, 0x06, 0x19, 0xc3, 0x93, 0x4b, - 0x75, 0x95, 0x94, 0x03, 0x82, 0xd4, 0x81, 0xd9, 0xd0, 0x5b, 0xe3, 0x9c, 0x2e, 0xc9, 0x62, 0x82, - 0x5f, 0x2a, 0x41, 0x1b, 0x91, 0xc5, 0x04, 0xbf, 0x64, 0xd3, 0x5b, 0x1d, 0x72, 0xab, 0xb8, 0x89, - 0x5d, 0x1c, 0x63, 0x4d, 0xa5, 0x1d, 0xc8, 0x7b, 0x42, 0xe3, 0x6c, 0x98, 0xdf, 0x12, 0x00, 0x71, - 0xb9, 0xaa, 0xb9, 0x1f, 0x67, 0x89, 0xd1, 0x12, 0xb1, 0x1b, 0xdc, 0x8e, 0x6d, 0xb2, 0xe5, 0x98, - 0xf5, 0x49, 0x60, 0x24, 0xba, 0x22, 0x07, 0x43, 0x36, 0x15, 0x1e, 0xb2, 0xdc, 0x76, 0x79, 0x09, - 0x73, 0x3d, 0x05, 0x8b, 0xb7, 0xf9, 0x52, 0xb4, 0x4c, 0x89, 0x6b, 0xc9, 0xb0, 0x81, 0x46, 0x89, - 0xd2, 0x8f, 0x04, 0x98, 0x2d, 0x37, 0xb1, 0x6a, 0xc7, 0xae, 0x91, 0xef, 0x40, 0x5a, 0xc7, 0xaa, - 0x4e, 0xab, 0xcc, 0x06, 0xf6, 0xeb, 0x21, 0x29, 0xc4, 0x8c, 0x2d, 0x1e, 0x34, 0xb5, 0x62, 0xc3, - 0x33, 0x70, 0xf9, 0xe8, 0xf6, 0x99, 0xa4, 0x4f, 0x00, 0x85, 0x4b, 0x16, 0x67, 0x47, 0xf8, 0xfd, - 0x04, 0x20, 0x19, 0x1f, 0x61, 0xdb, 0x8d, 0xbd, 0xda, 0xab, 0x90, 0x75, 0x55, 0x7b, 0x1f, 0xbb, - 0x0a, 0x31, 0xdd, 0xcf, 0x53, 0x73, 0x60, 0x7c, 0x84, 0x8c, 0x1a, 0x70, 0x1b, 0x9b, 0xea, 0x6e, - 0x13, 0x53, 0x29, 0xca, 0xae, 0xd5, 0x31, 0x75, 0xc5, 0x70, 0xb1, 0xad, 0xba, 0x96, 0xad, 0x58, - 0x6d, 0xd7, 0x68, 0x19, 0x9f, 0x51, 0xab, 0x9d, 0x77, 0xb5, 0x1b, 0x2c, 0x3b, 0x61, 0x2e, 0x91, - 0xcc, 0x35, 0x9e, 0xf7, 0x49, 0x28, 0x2b, 0x2a, 0xc2, 0x9c, 0xb1, 0x6f, 0x5a, 0x36, 0x56, 0xf6, - 0x35, 0xc5, 0x3d, 0xb0, 0xb1, 0x73, 0x60, 0x35, 0xbd, 0x05, 0x69, 0x96, 0x25, 0x3d, 0xd6, 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, 0x59, 0x76, 0x4b, 0x75, 0x69, - 0xbd, 0xf2, 0x3d, 0xba, 0xf7, 0x8d, 0x77, 0x4d, 0x35, 0x1f, 0xd1, 0x4c, 0x32, 0x38, 0xfe, 0xef, - 0x7e, 0xfb, 0x75, 0xf2, 0xab, 0xdb, 0xaf, 0x6c, 0x78, 0xaf, 0xa5, 0xd2, 0x49, 0x31, 0x25, 0xfd, - 0xa9, 0x00, 0x33, 0xac, 0xca, 0x71, 0x0e, 0xef, 0x77, 0x21, 0x65, 0x5b, 0x2f, 0xd9, 0xf0, 0xce, - 0x3e, 0xb8, 0x1a, 0x21, 0x62, 0x1d, 0x77, 0xc3, 0xeb, 0x27, 0xcd, 0x8e, 0x4a, 0xc0, 0xad, 0x54, - 0x85, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x13, 0x19, 0xb7, 0xa1, 0xb0, 0xab, 0xba, 0xda, - 0x81, 0x62, 0xf3, 0x42, 0x92, 0xb5, 0x36, 0x79, 0x67, 0x46, 0xce, 0x53, 0xb2, 0x57, 0x74, 0x87, - 0xd4, 0x9c, 0x8d, 0x37, 0x07, 0xff, 0x39, 0x6b, 0xf3, 0xff, 0x23, 0xf0, 0x31, 0xe4, 0xd5, 0xfc, - 0xcf, 0x5b, 0xd3, 0xff, 0x38, 0x01, 0x97, 0xcb, 0x07, 0x58, 0x3b, 0x2c, 0x5b, 0xa6, 0x63, 0x38, - 0x2e, 0xd1, 0x5d, 0x9c, 0xed, 0x7f, 0x15, 0x32, 0x2f, 0x0d, 0xf7, 0x40, 0xd1, 0x8d, 0xbd, 0x3d, - 0x3a, 0xdb, 0xa6, 0xe5, 0x34, 0x21, 0xac, 0x1a, 0x7b, 0x7b, 0xe8, 0x21, 0xa4, 0x5a, 0x96, 0xce, - 0x8c, 0xf9, 0x68, 0x28, 0x4e, 0x8b, 0xe6, 0x74, 0x5a, 0x9b, 0x96, 0x8e, 0x65, 0x9a, 0x19, 0x2d, - 0x02, 0x68, 0x84, 0xda, 0xb6, 0x0c, 0xd3, 0xe5, 0x93, 0x63, 0x88, 0x82, 0xaa, 0x90, 0x71, 0xb1, - 0xdd, 0x32, 0x4c, 0xd5, 0xc5, 0xf3, 0x93, 0x54, 0x79, 0x6f, 0x44, 0x16, 0xbc, 0xdd, 0x34, 0x34, - 0x75, 0x15, 0x3b, 0x9a, 0x6d, 0xb4, 0x5d, 0xcb, 0xe6, 0x5a, 0x0c, 0x98, 0xa5, 0xbf, 0x9e, 0x82, - 0xf9, 0x41, 0xdd, 0xc4, 0xd9, 0x43, 0xb6, 0x61, 0xca, 0xc6, 0x4e, 0xa7, 0xe9, 0xf2, 0x3e, 0xf2, - 0x60, 0x98, 0x0a, 0x22, 0x4a, 0x40, 0xb7, 0x29, 0x9a, 0x2e, 0x2f, 0x36, 0x97, 0xb3, 0xf0, 0x2f, - 0x04, 0x98, 0x62, 0x09, 0xe8, 0x3e, 0xa4, 0x6d, 0xb2, 0x30, 0x28, 0x86, 0x4e, 0xcb, 0x98, 0x2c, - 0x5d, 0x3a, 0x3d, 0x59, 0x9a, 0xa6, 0x8b, 0x45, 0x6d, 0xf5, 0xcb, 0xe0, 0xa7, 0x3c, 0x4d, 0xf3, - 0xd5, 0x74, 0xd2, 0x5a, 0x8e, 0xab, 0xda, 0x2e, 0xdd, 0x31, 0x4a, 0x30, 0x84, 0x44, 0x09, 0xeb, - 0xb8, 0x8b, 0xd6, 0x60, 0xca, 0x71, 0x55, 0xb7, 0xe3, 0xf0, 0xf6, 0x3a, 0x57, 0x61, 0xeb, 0x94, - 0x53, 0xe6, 0x12, 0x88, 0xb9, 0xa5, 0x63, 0x57, 0x35, 0x9a, 0xb4, 0x01, 0x33, 0x32, 0x7f, 0x92, - 0x7e, 0x5b, 0x80, 0x29, 0x96, 0x15, 0x5d, 0x86, 0x39, 0x79, 0x65, 0xeb, 0x71, 0x45, 0xa9, 0x6d, - 0xad, 0x56, 0x1a, 0x15, 0x79, 0xb3, 0xb6, 0xb5, 0xd2, 0xa8, 0x88, 0x13, 0xe8, 0x12, 0x20, 0x2f, - 0xa1, 0xfc, 0x64, 0xab, 0x5e, 0xab, 0x37, 0x2a, 0x5b, 0x0d, 0x51, 0x40, 0x17, 0x40, 0x64, 0xf4, - 0x10, 0x35, 0x81, 0xde, 0x80, 0x6b, 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, 0x45, 0x19, 0x6b, 0x56, 0xab, 0xdd, 0x71, 0x31, 0x29, 0xa5, 0x13, 0xe7, - 0x48, 0xb9, 0x0c, 0xd3, 0xba, 0xdd, 0x55, 0xec, 0x8e, 0xc9, 0xc7, 0xc9, 0x94, 0x6e, 0x77, 0xe5, - 0x8e, 0x29, 0xfd, 0x43, 0x01, 0x2e, 0xf5, 0xbf, 0x36, 0xce, 0x4e, 0xf8, 0x14, 0xb2, 0xaa, 0xae, - 0x63, 0x5d, 0xd1, 0x71, 0xd3, 0x55, 0xb9, 0x49, 0x74, 0x2f, 0x24, 0x89, 0xef, 0xf3, 0x15, 0xfd, - 0x7d, 0xbe, 0xcd, 0x67, 0xe5, 0x32, 0x2d, 0xc8, 0x2a, 0xe1, 0xf0, 0xa6, 0x1f, 0x2a, 0x84, 0x52, - 0xa4, 0x1f, 0xa7, 0x20, 0x57, 0x31, 0xf5, 0xc6, 0x71, 0xac, 0x6b, 0xc9, 0x25, 0x98, 0xd2, 0xac, - 0x56, 0xcb, 0x70, 0x3d, 0x05, 0xb1, 0x27, 0xf4, 0xad, 0x90, 0x29, 0x9b, 0x1c, 0xc3, 0xa0, 0x0b, - 0x8c, 0x58, 0xf4, 0x1b, 0x70, 0x99, 0xcc, 0x9a, 0xb6, 0xa9, 0x36, 0x15, 0x26, 0x4d, 0x71, 0x6d, - 0x63, 0x7f, 0x1f, 0xdb, 0x7c, 0x6f, 0xf1, 0x4e, 0x44, 0x39, 0x6b, 0x9c, 0xa3, 0x4c, 0x19, 0x1a, - 0x2c, 0xbf, 0x7c, 0xd1, 0x88, 0x22, 0xa3, 0x8f, 0x00, 0xc8, 0x52, 0x44, 0xf7, 0x2b, 0x1d, 0x3e, - 0x1f, 0x0d, 0xdb, 0xb0, 0xf4, 0xa6, 0x20, 0xc2, 0x40, 0x9e, 0x1d, 0xf4, 0x14, 0x44, 0xc3, 0x54, - 0xf6, 0x9a, 0xc6, 0xfe, 0x81, 0xab, 0xbc, 0xb4, 0x0d, 0x17, 0x3b, 0xf3, 0xb3, 0x54, 0x46, 0x54, - 0x53, 0xd7, 0xf9, 0xbe, 0xab, 0xfe, 0x9c, 0xe4, 0xe4, 0xd2, 0xf2, 0x86, 0xf9, 0x88, 0xf2, 0x53, - 0xa2, 0x83, 0x96, 0x09, 0x14, 0x7a, 0xd1, 0x31, 0x6c, 0xac, 0xdc, 0x6f, 0x6b, 0x74, 0x1f, 0x24, - 0x5d, 0xca, 0x9f, 0x9e, 0x2c, 0x81, 0xcc, 0xc8, 0xf7, 0xb7, 0xcb, 0x04, 0x1a, 0xb1, 0xdf, 0x6d, - 0x8d, 0xa8, 0xbd, 0x6d, 0x19, 0x8e, 0x65, 0xd2, 0x8d, 0xce, 0xb4, 0xcc, 0x9f, 0xd0, 0x5d, 0x10, - 0xdd, 0x63, 0x53, 0x39, 0xc0, 0xaa, 0xed, 0xee, 0x62, 0xd5, 0x25, 0xeb, 0x33, 0xd0, 0x1c, 0x05, - 0xf7, 0xd8, 0xac, 0x86, 0xc8, 0x6b, 0xa9, 0xf4, 0xb4, 0x98, 0x5e, 0x4b, 0xa5, 0xd3, 0x62, 0x46, - 0xfa, 0x0f, 0x02, 0xe4, 0xbd, 0xbe, 0x11, 0x67, 0x37, 0xbe, 0x03, 0xa2, 0x65, 0x62, 0xa5, 0x7d, - 0xa0, 0x3a, 0x98, 0xb7, 0x25, 0x5f, 0x1d, 0xf2, 0x96, 0x89, 0xb7, 0x09, 0x99, 0xb5, 0x0c, 0xda, - 0x86, 0x59, 0xc7, 0x55, 0xf7, 0x0d, 0x73, 0x5f, 0xf1, 0xf7, 0xef, 0xa9, 0x65, 0x31, 0x26, 0x12, - 0x10, 0x39, 0xb7, 0x4f, 0xef, 0x31, 0x29, 0x7e, 0x2e, 0xc0, 0xec, 0x8a, 0xde, 0x32, 0xcc, 0x7a, - 0xbb, 0x69, 0xc4, 0xba, 0xc1, 0xf0, 0x06, 0x64, 0x1c, 0x22, 0x33, 0x98, 0x9d, 0x03, 0xb8, 0x98, - 0xa6, 0x29, 0x64, 0x9a, 0xde, 0x80, 0x02, 0x3e, 0x6e, 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, 0xc7, 0x74, 0x62, 0xd6, 0x97, 0xf4, 0x6b, 0x70, 0xa1, 0x57, 0x74, - 0x9c, 0xe5, 0x7e, 0xce, 0x5b, 0x79, 0x13, 0xdb, 0xb1, 0x22, 0x54, 0x5f, 0xd7, 0x5c, 0x70, 0x9c, - 0x65, 0xfe, 0x4d, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0x5c, 0x65, 0x0f, 0xdb, 0x1b, 0x58, 0x75, 0x62, - 0x85, 0xd7, 0x37, 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, 0x1d, 0x01, - 0x66, 0xb9, 0x85, 0x47, 0xba, 0x72, 0xf9, 0x80, 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, 0x29, - 0xb4, 0x75, 0x1e, 0xae, 0x78, 0xe2, 0x1c, 0x15, 0x0f, 0x6d, 0xbf, 0x07, 0x54, 0xf4, 0x09, 0x84, - 0x36, 0xd8, 0x15, 0x56, 0x27, 0x0f, 0xaa, 0x9c, 0x47, 0x1d, 0xb3, 0x81, 0x14, 0x46, 0x77, 0x50, - 0x19, 0xd2, 0xf8, 0xb8, 0xad, 0xe8, 0xd8, 0xd1, 0xf8, 0xc4, 0x25, 0x45, 0x09, 0x24, 0x45, 0x19, - 0x30, 0xde, 0xa7, 0xf1, 0x71, 0x9b, 0x10, 0xd1, 0x0e, 0x59, 0x37, 0xbd, 0x75, 0x9d, 0x16, 0xdb, - 0x39, 0x1b, 0x0b, 0x04, 0x3d, 0x85, 0x8b, 0x2b, 0xf8, 0x4b, 0x3a, 0x13, 0x21, 0x7d, 0x2e, 0xc0, - 0xd5, 0xc8, 0x56, 0x8b, 0x73, 0x21, 0xfb, 0x08, 0x52, 0xb4, 0xf2, 0x89, 0x73, 0x56, 0x9e, 0x72, - 0x49, 0x3f, 0x48, 0xf0, 0x31, 0x2e, 0xe3, 0xa6, 0x45, 0x14, 0x1b, 0xfb, 0x16, 0xda, 0x13, 0xc8, - 0x1d, 0x59, 0x2e, 0xb6, 0xfd, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, 0x67, 0xa8, 0x00, 0xaf, 0xc5, 0x9f, - 0xc1, 0xac, 0x69, 0x99, 0x4a, 0xaf, 0xd0, 0xf3, 0xf7, 0xa5, 0x82, 0x69, 0x99, 0xcf, 0x42, 0x72, - 0xfd, 0x79, 0xa6, 0x4f, 0x13, 0x71, 0xce, 0x33, 0x3f, 0x14, 0x60, 0xce, 0xb7, 0x74, 0x62, 0x36, - 0x77, 0xdf, 0x85, 0xa4, 0x69, 0xbd, 0x3c, 0xcf, 0x16, 0x25, 0xc9, 0x4f, 0x56, 0xbd, 0xde, 0x12, - 0xc5, 0x59, 0xdf, 0x7f, 0x99, 0x80, 0xcc, 0xe3, 0x72, 0x9c, 0xb5, 0xfc, 0x88, 0x6f, 0x7f, 0xb3, - 0xf6, 0x8e, 0xea, 0xed, 0xfe, 0xfb, 0x8a, 0x8f, 0xcb, 0xeb, 0xb8, 0xeb, 0xf5, 0x76, 0xc2, 0x85, - 0x56, 0x20, 0xd3, 0xbb, 0x51, 0x3a, 0xa6, 0xa6, 0x02, 0xae, 0x05, 0x0c, 0x93, 0x54, 0xae, 0xe7, - 0x47, 0x21, 0x44, 0xf8, 0x51, 0x90, 0xd7, 0xf8, 0x96, 0x62, 0xe2, 0x3c, 0xaf, 0x09, 0x99, 0x88, - 0x93, 0xe2, 0x94, 0xf4, 0x14, 0x80, 0x54, 0x27, 0xce, 0x26, 0xf9, 0x7e, 0x12, 0xf2, 0xdb, 0x1d, - 0xe7, 0x20, 0xe6, 0xde, 0x57, 0x06, 0x68, 0x77, 0x9c, 0x03, 0x32, 0x22, 0x8f, 0x4d, 0x5e, 0xe7, - 0x33, 0x5c, 0x34, 0xbc, 0x4a, 0x33, 0xbe, 0xc6, 0xb1, 0x89, 0xaa, 0x5c, 0x08, 0x56, 0x02, 0x3f, - 0x8f, 0x1b, 0xa3, 0x90, 0x65, 0xe3, 0xd8, 0xdc, 0xc4, 0x3e, 0xa4, 0x64, 0x92, 0x30, 0x91, 0xf4, - 0x11, 0x4c, 0x93, 0x07, 0xc5, 0xb5, 0xce, 0xd3, 0xcc, 0x53, 0x84, 0xa7, 0x61, 0xa1, 0x0f, 0x21, - 0xc3, 0xb8, 0xc9, 0xea, 0x37, 0x45, 0x57, 0xbf, 0xa8, 0xba, 0x70, 0x35, 0xd2, 0x75, 0x2f, 0x4d, - 0x59, 0xc9, 0x5a, 0x77, 0x01, 0x26, 0xf7, 0x2c, 0x5b, 0xf3, 0x0e, 0x73, 0xd9, 0x03, 0x6b, 0x4f, - 0x06, 0x69, 0xd6, 0x52, 0xe9, 0x8c, 0x08, 0xd2, 0x6f, 0x0b, 0x50, 0xf0, 0x1b, 0x22, 0xce, 0x05, - 0xa1, 0xdc, 0xa3, 0xc5, 0xf3, 0x37, 0x05, 0x51, 0xa0, 0xf4, 0xaf, 0xa8, 0x45, 0xa4, 0x59, 0x47, - 0xb4, 0x65, 0xe2, 0xec, 0x29, 0x1f, 0x32, 0x2f, 0x9e, 0xc4, 0x79, 0x5b, 0x97, 0x3a, 0xf4, 0xdc, - 0x87, 0x0b, 0x46, 0x8b, 0xcc, 0xe7, 0x86, 0xdb, 0xec, 0x72, 0xd8, 0xe6, 0x62, 0xef, 0xd4, 0x78, - 0x2e, 0x48, 0x2b, 0x7b, 0x49, 0xd2, 0xef, 0xd3, 0xdd, 0xea, 0xa0, 0x26, 0x71, 0xaa, 0xba, 0x06, - 0x39, 0x9b, 0x89, 0x26, 0x66, 0xcd, 0x39, 0xb5, 0x3d, 0xe3, 0xb3, 0x12, 0x85, 0xff, 0x6e, 0x02, - 0x0a, 0x4f, 0x3b, 0xd8, 0xee, 0x7e, 0x93, 0xd4, 0x7d, 0x0b, 0x0a, 0x2f, 0x55, 0xc3, 0x55, 0xf6, - 0x2c, 0x5b, 0xe9, 0xb4, 0x75, 0xd5, 0xf5, 0xbc, 0x4d, 0x72, 0x84, 0xfc, 0xc8, 0xb2, 0x77, 0x28, - 0x11, 0x61, 0x40, 0x87, 0xa6, 0xf5, 0xd2, 0x54, 0x08, 0x99, 0x02, 0xe5, 0x63, 0x93, 0x6f, 0x21, - 0x97, 0xde, 0xfb, 0xf7, 0x27, 0x4b, 0x0f, 0xc7, 0x72, 0x10, 0xa3, 0xce, 0x70, 0x9d, 0x8e, 0xa1, - 0x17, 0x77, 0x76, 0x6a, 0xab, 0xb2, 0x48, 0x45, 0x3e, 0x67, 0x12, 0x1b, 0xc7, 0xa6, 0x23, 0xfd, - 0x9d, 0x04, 0x88, 0x81, 0x8e, 0xe2, 0x6c, 0xc8, 0x0a, 0x64, 0x5f, 0x74, 0xb0, 0x6d, 0xbc, 0x42, - 0x33, 0x02, 0x67, 0x24, 0xd3, 0xce, 0x3d, 0x98, 0x75, 0x8f, 0x4d, 0x85, 0xb9, 0xef, 0x31, 0xc7, - 0x0f, 0xcf, 0x61, 0xa1, 0xe0, 0x92, 0x32, 0x13, 0x3a, 0x75, 0xfa, 0x70, 0xd0, 0xa7, 0x30, 0xd3, - 0xa3, 0xad, 0xe4, 0x57, 0xd3, 0x56, 0xf6, 0x65, 0x48, 0x51, 0x3f, 0x11, 0x00, 0x51, 0x45, 0xd5, - 0xd8, 0x1e, 0xff, 0x37, 0xa5, 0x3f, 0xdd, 0x01, 0x91, 0xfa, 0x61, 0x2a, 0xc6, 0x9e, 0xd2, 0x32, - 0x1c, 0xc7, 0x30, 0xf7, 0x79, 0x87, 0xca, 0x53, 0x7a, 0x6d, 0x6f, 0x93, 0x51, 0xa5, 0xbf, 0x04, - 0x73, 0x3d, 0x15, 0x88, 0xb3, 0xb1, 0xaf, 0xc3, 0xcc, 0x1e, 0x3b, 0x82, 0xa5, 0xc2, 0xf9, 0xf6, - 0x60, 0x96, 0xd2, 0xd8, 0xfb, 0xa4, 0xff, 0x92, 0x80, 0x0b, 0x32, 0x76, 0xac, 0xe6, 0x11, 0x8e, - 0x5f, 0x85, 0x55, 0xe0, 0x67, 0x2f, 0xca, 0x2b, 0x69, 0x32, 0xc3, 0x98, 0xd9, 0x32, 0xd7, 0xbb, - 0xc7, 0xfe, 0xc6, 0xe8, 0x1e, 0x3b, 0xb8, 0xab, 0xce, 0x77, 0xea, 0x52, 0x3d, 0x3b, 0x75, 0x16, - 0x14, 0xd8, 0xe9, 0xb1, 0xae, 0x38, 0xf8, 0x85, 0xd9, 0x69, 0x79, 0x60, 0xa8, 0x38, 0xaa, 0x90, - 0x35, 0xc6, 0x52, 0xc7, 0x2f, 0xb6, 0x3a, 0x2d, 0x6a, 0x3b, 0x97, 0x2e, 0x91, 0xf2, 0x9e, 0x9e, - 0x2c, 0xe5, 0x7b, 0xd2, 0x1c, 0x39, 0x6f, 0xf8, 0xcf, 0x44, 0xba, 0xf4, 0x5d, 0xb8, 0xd8, 0xa7, - 0xec, 0x38, 0x2d, 0x9e, 0x7f, 0x9e, 0x84, 0x2b, 0xbd, 0xe2, 0xe3, 0x86, 0x38, 0xdf, 0xf4, 0x06, - 0xad, 0x42, 0xae, 0x65, 0x98, 0xaf, 0xb6, 0x7b, 0x39, 0xd3, 0x32, 0x4c, 0x9f, 0x16, 0xd5, 0x35, - 0xa6, 0xbe, 0xd6, 0xae, 0xa1, 0xc2, 0x42, 0x54, 0xdb, 0xc5, 0xd9, 0x3f, 0x7e, 0x20, 0xc0, 0x4c, - 0xdc, 0xdb, 0x72, 0xaf, 0xe6, 0x05, 0x27, 0x35, 0x20, 0xf7, 0x35, 0xec, 0xe3, 0xfd, 0xae, 0x00, - 0xa8, 0x61, 0x77, 0x4c, 0x02, 0x6a, 0x37, 0xac, 0xfd, 0x38, 0xab, 0x79, 0x01, 0x26, 0x0d, 0x53, - 0xc7, 0xc7, 0xb4, 0x9a, 0x29, 0x99, 0x3d, 0xf4, 0x1c, 0x25, 0x26, 0xc7, 0x3a, 0x4a, 0x94, 0x3e, - 0x85, 0xb9, 0x9e, 0x22, 0xc6, 0x59, 0xff, 0x5f, 0x26, 0x60, 0x8e, 0x57, 0x24, 0xf6, 0x1d, 0xcc, - 0x77, 0x60, 0xb2, 0x49, 0x64, 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0x6d, - 0x80, 0xb6, 0x8d, 0x8f, 0x14, 0xc6, 0x9a, 0x1c, 0x8b, 0x35, 0x43, 0x38, 0x28, 0x01, 0xfd, 0x48, - 0x80, 0x02, 0x19, 0xd0, 0x6d, 0xdb, 0x6a, 0x5b, 0x0e, 0xb1, 0x59, 0x9c, 0xf1, 0x60, 0xce, 0xd3, - 0xd3, 0x93, 0xa5, 0xdc, 0xa6, 0x61, 0x6e, 0x73, 0xc6, 0x46, 0x7d, 0x6c, 0xef, 0x7d, 0x2f, 0x86, - 0xa1, 0x58, 0x6e, 0x5a, 0xda, 0x61, 0x70, 0x38, 0x46, 0x66, 0x16, 0x5f, 0x9c, 0x23, 0xfd, 0x3b, - 0x01, 0x2e, 0x7c, 0x6d, 0xdb, 0xc5, 0x7f, 0x16, 0xca, 0x96, 0x9e, 0x81, 0x48, 0x7f, 0xd4, 0xcc, - 0x3d, 0x2b, 0xce, 0x8d, 0xfb, 0xff, 0x2d, 0xc0, 0x6c, 0x48, 0x70, 0x9c, 0x06, 0xce, 0xab, 0xea, + 0x31, 0x82, 0x49, 0x32, 0xf3, 0xb6, 0x08, 0x10, 0x65, 0xa3, 0xc9, 0xc3, 0x0e, 0x92, 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, + 0xac, 0x6d, 0xa9, 0xda, 0x41, 0x7b, 0x77, 0x59, 0x6d, 0x1b, 0xc5, 0xb6, 0x6d, 0xb9, 0x16, 0x9a, + 0xd5, 0x2c, 0xed, 0x90, 0x92, 0x8b, 0x3c, 0x71, 0xe1, 0x22, 0xb6, 0x6d, 0xcb, 0x76, 0xda, 0xbb, + 0xcb, 0xec, 0x07, 0xcb, 0xb9, 0x70, 0xef, 0xf0, 0x68, 0xf9, 0xf0, 0xc8, 0xc1, 0xf6, 0x11, 0xb6, + 0x97, 0x35, 0xcb, 0xd4, 0x3a, 0xb6, 0x8d, 0x4d, 0xad, 0xbb, 0xdc, 0xb4, 0xb4, 0x43, 0xfa, 0xc7, + 0x30, 0xf7, 0xa3, 0xf2, 0xda, 0x58, 0xd5, 0x9d, 0x4e, 0xab, 0xa5, 0xda, 0xdd, 0x65, 0x2a, 0x96, + 0x3f, 0xf0, 0xbc, 0xc8, 0x2b, 0x94, 0xae, 0xba, 0x2a, 0xa7, 0x5d, 0xf0, 0x68, 0x3d, 0x25, 0xb8, + 0xe4, 0x51, 0x5b, 0xd8, 0x55, 0x43, 0xb9, 0xaf, 0x78, 0x74, 0xa7, 0xad, 0x9a, 0x8a, 0x66, 0x99, + 0x7b, 0x86, 0x57, 0x90, 0xab, 0x8e, 0x6b, 0xd9, 0xea, 0x3e, 0x5e, 0xc6, 0xe6, 0xbe, 0x61, 0x62, + 0xc2, 0x7b, 0xa4, 0x69, 0x3c, 0xf1, 0xb5, 0xc8, 0xc4, 0x87, 0x3c, 0x75, 0xbe, 0xe3, 0x1a, 0xcd, + 0xe5, 0x83, 0xa6, 0xb6, 0xec, 0x1a, 0x2d, 0xec, 0xb8, 0x6a, 0xab, 0xed, 0xd5, 0x8e, 0xa6, 0xb8, + 0xb6, 0xaa, 0x19, 0xe6, 0xbe, 0xf7, 0xbf, 0xbd, 0xbb, 0x6c, 0x63, 0xcd, 0xb2, 0x75, 0xac, 0x2b, + 0xa4, 0x1c, 0x5e, 0x4d, 0xf6, 0xad, 0x7d, 0x8b, 0xfe, 0x5c, 0x26, 0xbf, 0x38, 0x75, 0x71, 0xdf, + 0xb2, 0xf6, 0x9b, 0x78, 0x99, 0x3e, 0xed, 0x76, 0xf6, 0x96, 0xf5, 0x8e, 0xad, 0xba, 0x86, 0xc5, + 0xb9, 0xa4, 0x7f, 0x2c, 0x40, 0x4e, 0xc6, 0x2f, 0x3a, 0xd8, 0x71, 0xab, 0x58, 0xd5, 0xb1, 0x8d, + 0xae, 0x40, 0xf2, 0x10, 0x77, 0xe7, 0x93, 0xd7, 0x84, 0x3b, 0x33, 0xa5, 0xe9, 0x2f, 0x4f, 0x96, + 0x92, 0xeb, 0xb8, 0x2b, 0x13, 0x1a, 0xba, 0x06, 0xd3, 0xd8, 0xd4, 0x15, 0x92, 0x9c, 0xea, 0x4d, + 0x9e, 0xc2, 0xa6, 0xbe, 0x8e, 0xbb, 0xe8, 0xbb, 0x90, 0x76, 0x88, 0x34, 0x53, 0xc3, 0xf3, 0x93, + 0xd7, 0x84, 0x3b, 0x93, 0xa5, 0x5f, 0xfd, 0xf2, 0x64, 0xe9, 0xa3, 0x7d, 0xc3, 0x3d, 0xe8, 0xec, + 0x16, 0x35, 0xab, 0xb5, 0xec, 0xf7, 0x02, 0x7d, 0x37, 0xf8, 0xbd, 0xdc, 0x3e, 0xdc, 0x5f, 0xee, + 0xd7, 0x51, 0xb1, 0x71, 0x6c, 0xd6, 0xf1, 0x0b, 0xd9, 0x97, 0xb8, 0x96, 0x4a, 0x0b, 0x62, 0x62, + 0x2d, 0x95, 0x4e, 0x88, 0x49, 0xe9, 0xef, 0x26, 0x20, 0x2f, 0x63, 0xa7, 0x6d, 0x99, 0x0e, 0xe6, + 0x25, 0x7f, 0x1b, 0x92, 0xee, 0xb1, 0x49, 0x4b, 0x9e, 0x7d, 0xb0, 0x58, 0x1c, 0xe8, 0x6f, 0xc5, + 0x86, 0xad, 0x9a, 0x8e, 0xaa, 0x91, 0xea, 0xcb, 0x24, 0x2b, 0x7a, 0x1f, 0xb2, 0x36, 0x76, 0x3a, + 0x2d, 0x4c, 0x15, 0x49, 0x2b, 0x95, 0x7d, 0x70, 0x39, 0x82, 0xb3, 0xde, 0x56, 0x4d, 0x19, 0x58, + 0x5e, 0xf2, 0x1b, 0xad, 0x42, 0x8e, 0x73, 0xda, 0x58, 0x75, 0x2c, 0x73, 0x7e, 0xfa, 0x9a, 0x70, + 0x27, 0xff, 0x60, 0x29, 0x82, 0x57, 0xa6, 0xf9, 0x64, 0x9a, 0x4d, 0x9e, 0xb1, 0x43, 0x4f, 0xe8, + 0x1e, 0xcc, 0x72, 0x29, 0x26, 0x3e, 0x76, 0x95, 0xdd, 0xae, 0x8b, 0x9d, 0xf9, 0xcc, 0x35, 0xe1, + 0x4e, 0x52, 0x2e, 0xb0, 0x84, 0x2d, 0x7c, 0xec, 0x96, 0x08, 0x19, 0x5d, 0x81, 0xb4, 0xd9, 0x69, + 0x11, 0xe5, 0x3b, 0x54, 0xb5, 0x49, 0x79, 0xda, 0xec, 0xb4, 0xd6, 0x71, 0xd7, 0x41, 0x57, 0x21, + 0x43, 0x92, 0x18, 0x7b, 0x9a, 0xa6, 0x91, 0xbc, 0x94, 0x6f, 0x2d, 0x95, 0x9e, 0x12, 0xa7, 0xa5, + 0xdf, 0x13, 0x00, 0x1e, 0x63, 0x97, 0x37, 0x35, 0x2a, 0xc1, 0xd4, 0x01, 0x55, 0xda, 0xbc, 0x40, + 0xeb, 0x7c, 0x2d, 0xb2, 0xdc, 0xa1, 0x6e, 0x51, 0x4a, 0x7f, 0x71, 0xb2, 0x34, 0xf1, 0xb3, 0x93, + 0x25, 0x41, 0xe6, 0x9c, 0xe8, 0x29, 0x64, 0x0f, 0x71, 0x57, 0xe1, 0xe3, 0x71, 0x3e, 0x41, 0x15, + 0xf0, 0x76, 0x48, 0xd0, 0xe1, 0x51, 0xd1, 0x1b, 0x9a, 0xc5, 0xd0, 0x30, 0x2e, 0x12, 0x8e, 0x62, + 0xdd, 0xb5, 0xb1, 0xb9, 0xef, 0x1e, 0xc8, 0x70, 0x88, 0xbb, 0x1b, 0x4c, 0x86, 0xf4, 0x13, 0x01, + 0xb2, 0xb4, 0x94, 0xac, 0x5d, 0x51, 0xb9, 0xaf, 0x98, 0xd7, 0xa3, 0xd5, 0x1b, 0xea, 0x04, 0x11, + 0xe5, 0x2c, 0xc2, 0xe4, 0x91, 0xda, 0xec, 0x60, 0x5a, 0xc2, 0xec, 0x83, 0xf9, 0x08, 0x19, 0xcf, + 0x48, 0xba, 0xcc, 0xb2, 0xa1, 0x0f, 0x61, 0xc6, 0x30, 0x5d, 0x6c, 0xba, 0x0a, 0x63, 0x4b, 0x9e, + 0xc1, 0x96, 0x65, 0xb9, 0xe9, 0x83, 0xf4, 0x8f, 0x04, 0x80, 0xed, 0x4e, 0xac, 0x7a, 0x7e, 0x67, + 0xcc, 0xf2, 0x97, 0x52, 0x84, 0xd5, 0xab, 0xc5, 0x25, 0x98, 0x32, 0xcc, 0xa6, 0x61, 0xb2, 0xf2, + 0xa7, 0x65, 0xfe, 0x84, 0x2e, 0xc0, 0xe4, 0x6e, 0xd3, 0x30, 0x75, 0xda, 0xd9, 0xd3, 0x32, 0x7b, + 0x90, 0x64, 0xc8, 0xd2, 0x52, 0xc7, 0xa8, 0x77, 0xe9, 0x24, 0x01, 0x17, 0xcb, 0x96, 0xa9, 0x1b, + 0x64, 0xbc, 0xa9, 0xcd, 0x6f, 0x84, 0x56, 0xd6, 0xe0, 0x82, 0x8e, 0xdb, 0x36, 0xd6, 0x54, 0x17, + 0xeb, 0x0a, 0x3e, 0x6e, 0x8f, 0xd9, 0xc6, 0x28, 0xe0, 0xaa, 0x1c, 0xb7, 0x29, 0x8d, 0x8c, 0x3a, + 0x22, 0x80, 0x8d, 0xba, 0x29, 0x32, 0x1f, 0xca, 0x69, 0x7c, 0xdc, 0x66, 0xa3, 0x35, 0x52, 0xcd, + 0xe8, 0x1d, 0xb8, 0xac, 0x36, 0x9b, 0xd6, 0x4b, 0xc5, 0xd8, 0x53, 0x74, 0x0b, 0x3b, 0x8a, 0x69, + 0xb9, 0x0a, 0x3e, 0x36, 0x1c, 0x97, 0x0e, 0xe9, 0xb4, 0x3c, 0x47, 0x93, 0x6b, 0x7b, 0xab, 0x16, + 0x76, 0xb6, 0x2c, 0xb7, 0x42, 0x92, 0x42, 0x4d, 0x39, 0x1d, 0x6e, 0x4a, 0xe9, 0xd7, 0xe1, 0x52, + 0xbf, 0x7e, 0xe3, 0x6c, 0xbf, 0x9f, 0x0a, 0x90, 0xaf, 0x99, 0x86, 0xfb, 0x8d, 0x68, 0x38, 0x5f, + 0x9f, 0xc9, 0xb0, 0x3e, 0xef, 0x81, 0xb8, 0xa7, 0x1a, 0xcd, 0x27, 0x66, 0xc3, 0x6a, 0xed, 0x3a, + 0xae, 0x65, 0x62, 0x87, 0x2b, 0x7c, 0x80, 0x2e, 0x3d, 0x83, 0x82, 0x5f, 0x9b, 0x38, 0xd5, 0xe4, + 0x82, 0x58, 0x33, 0x35, 0x1b, 0xb7, 0xb0, 0x19, 0xab, 0x9e, 0x5e, 0x83, 0x8c, 0xe1, 0xc9, 0xa5, + 0xba, 0x4a, 0xca, 0x01, 0x41, 0xea, 0xc0, 0x6c, 0xe8, 0xad, 0x71, 0x4e, 0x97, 0x64, 0x31, 0xc1, + 0x2f, 0x95, 0xa0, 0x8d, 0xc8, 0x62, 0x82, 0x5f, 0xb2, 0xe9, 0xad, 0x0e, 0xb9, 0x55, 0xdc, 0xc4, + 0x2e, 0x8e, 0xb1, 0xa6, 0xd2, 0x0e, 0xe4, 0x3d, 0xa1, 0x71, 0x36, 0xcc, 0x6f, 0x09, 0x80, 0xb8, + 0x5c, 0xd5, 0xdc, 0x8f, 0xb3, 0xc4, 0x68, 0x89, 0xd8, 0x0d, 0x6e, 0xc7, 0x36, 0xd9, 0x72, 0xcc, + 0xfa, 0x24, 0x30, 0x12, 0x5d, 0x91, 0x83, 0x21, 0x9b, 0x0a, 0x0f, 0x59, 0x6e, 0xbb, 0xbc, 0x84, + 0xb9, 0x9e, 0x82, 0xc5, 0xdb, 0x7c, 0x29, 0x5a, 0xa6, 0xc4, 0xb5, 0x64, 0xd8, 0x40, 0xa3, 0x44, + 0xe9, 0x47, 0x02, 0xcc, 0x96, 0x9b, 0x58, 0xb5, 0x63, 0xd7, 0xc8, 0x77, 0x20, 0xad, 0x63, 0x55, + 0xa7, 0x55, 0x66, 0x03, 0xfb, 0xf5, 0x90, 0x14, 0x62, 0xc6, 0x16, 0x0f, 0x9a, 0x5a, 0xb1, 0xe1, + 0x19, 0xb8, 0x7c, 0x74, 0xfb, 0x4c, 0xd2, 0x27, 0x80, 0xc2, 0x25, 0x8b, 0xb3, 0x23, 0xfc, 0x7e, + 0x02, 0x90, 0x8c, 0x8f, 0xb0, 0xed, 0xc6, 0x5e, 0xed, 0x55, 0xc8, 0xba, 0xaa, 0xbd, 0x8f, 0x5d, + 0x85, 0x98, 0xee, 0xe7, 0xa9, 0x39, 0x30, 0x3e, 0x42, 0x46, 0x0d, 0xb8, 0x8d, 0x4d, 0x75, 0xb7, + 0x89, 0xa9, 0x14, 0x65, 0xd7, 0xea, 0x98, 0xba, 0x62, 0xb8, 0xd8, 0x56, 0x5d, 0xcb, 0x56, 0xac, + 0xb6, 0x6b, 0xb4, 0x8c, 0xcf, 0xa8, 0xd5, 0xce, 0xbb, 0xda, 0x0d, 0x96, 0x9d, 0x30, 0x97, 0x48, + 0xe6, 0x1a, 0xcf, 0xfb, 0x24, 0x94, 0x15, 0x15, 0x61, 0xce, 0xd8, 0x37, 0x2d, 0x1b, 0x2b, 0xfb, + 0x9a, 0xe2, 0x1e, 0xd8, 0xd8, 0x39, 0xb0, 0x9a, 0xde, 0x82, 0x34, 0xcb, 0x92, 0x1e, 0x6b, 0x0d, + 0x2f, 0x41, 0xfa, 0x14, 0xe6, 0x7a, 0xb4, 0x14, 0x67, 0x13, 0xfc, 0x37, 0x01, 0xb2, 0x75, 0x4d, + 0x35, 0xe3, 0xd4, 0xfd, 0xc7, 0x90, 0x75, 0x34, 0xd5, 0x54, 0xf6, 0x2c, 0xbb, 0xa5, 0xba, 0xb4, + 0x5e, 0xf9, 0x1e, 0xdd, 0xfb, 0xc6, 0xbb, 0xa6, 0x9a, 0x8f, 0x68, 0x26, 0x19, 0x1c, 0xff, 0x77, + 0xbf, 0xfd, 0x3a, 0xf9, 0xd5, 0xed, 0x57, 0x36, 0xbc, 0xd7, 0x52, 0xe9, 0xa4, 0x98, 0x92, 0xfe, + 0x54, 0x80, 0x19, 0x56, 0xe5, 0x38, 0x87, 0xf7, 0xbb, 0x90, 0xb2, 0xad, 0x97, 0x6c, 0x78, 0x67, + 0x1f, 0x5c, 0x8d, 0x10, 0xb1, 0x8e, 0xbb, 0xe1, 0xf5, 0x93, 0x66, 0x47, 0x25, 0xe0, 0x56, 0xaa, + 0x42, 0xb9, 0x93, 0xe3, 0x72, 0x03, 0xe3, 0x92, 0x89, 0x8c, 0xdb, 0x50, 0xd8, 0x55, 0x5d, 0xed, + 0x40, 0xb1, 0x79, 0x21, 0xc9, 0x5a, 0x9b, 0xbc, 0x33, 0x23, 0xe7, 0x29, 0xd9, 0x2b, 0xba, 0x43, + 0x6a, 0xce, 0xc6, 0x9b, 0x83, 0xff, 0x8c, 0xb5, 0xf9, 0xff, 0x16, 0xf8, 0x18, 0xf2, 0x6a, 0xfe, + 0x67, 0xad, 0xe9, 0x7f, 0x9c, 0x80, 0xcb, 0xe5, 0x03, 0xac, 0x1d, 0x96, 0x2d, 0xd3, 0x31, 0x1c, + 0x97, 0xe8, 0x2e, 0xce, 0xf6, 0xbf, 0x0a, 0x99, 0x97, 0x86, 0x7b, 0xa0, 0xe8, 0xc6, 0xde, 0x1e, + 0x9d, 0x6d, 0xd3, 0x72, 0x9a, 0x10, 0x56, 0x8d, 0xbd, 0x3d, 0xf4, 0x10, 0x52, 0x2d, 0x4b, 0x67, + 0xc6, 0x7c, 0x34, 0x14, 0xa7, 0x45, 0x73, 0x3a, 0xad, 0x4d, 0x4b, 0xc7, 0x32, 0xcd, 0x8c, 0x16, + 0x01, 0x34, 0x42, 0x6d, 0x5b, 0x86, 0xe9, 0xf2, 0xc9, 0x31, 0x44, 0x41, 0x55, 0xc8, 0xb8, 0xd8, + 0x6e, 0x19, 0xa6, 0xea, 0xe2, 0xf9, 0x49, 0xaa, 0xbc, 0x37, 0x22, 0x0b, 0xde, 0x6e, 0x1a, 0x9a, + 0xba, 0x8a, 0x1d, 0xcd, 0x36, 0xda, 0xae, 0x65, 0x73, 0x2d, 0x06, 0xcc, 0xd2, 0x5f, 0x4d, 0xc1, + 0xfc, 0xa0, 0x6e, 0xe2, 0xec, 0x21, 0xdb, 0x30, 0x65, 0x63, 0xa7, 0xd3, 0x74, 0x79, 0x1f, 0x79, + 0x30, 0x4c, 0x05, 0x11, 0x25, 0xa0, 0xdb, 0x14, 0x4d, 0x97, 0x17, 0x9b, 0xcb, 0x59, 0xf8, 0x67, + 0x02, 0x4c, 0xb1, 0x04, 0x74, 0x1f, 0xd2, 0x36, 0x59, 0x18, 0x14, 0x43, 0xa7, 0x65, 0x4c, 0x96, + 0x2e, 0x9d, 0x9e, 0x2c, 0x4d, 0xd3, 0xc5, 0xa2, 0xb6, 0xfa, 0x65, 0xf0, 0x53, 0x9e, 0xa6, 0xf9, + 0x6a, 0x3a, 0x69, 0x2d, 0xc7, 0x55, 0x6d, 0x97, 0xee, 0x18, 0x25, 0x18, 0x42, 0xa2, 0x84, 0x75, + 0xdc, 0x45, 0x6b, 0x30, 0xe5, 0xb8, 0xaa, 0xdb, 0x71, 0x78, 0x7b, 0x9d, 0xab, 0xb0, 0x75, 0xca, + 0x29, 0x73, 0x09, 0xc4, 0xdc, 0xd2, 0xb1, 0xab, 0x1a, 0x4d, 0xda, 0x80, 0x19, 0x99, 0x3f, 0x49, + 0xbf, 0x2d, 0xc0, 0x14, 0xcb, 0x8a, 0x2e, 0xc3, 0x9c, 0xbc, 0xb2, 0xf5, 0xb8, 0xa2, 0xd4, 0xb6, + 0x56, 0x2b, 0x8d, 0x8a, 0xbc, 0x59, 0xdb, 0x5a, 0x69, 0x54, 0xc4, 0x09, 0x74, 0x09, 0x90, 0x97, + 0x50, 0x7e, 0xb2, 0x55, 0xaf, 0xd5, 0x1b, 0x95, 0xad, 0x86, 0x28, 0xa0, 0x0b, 0x20, 0x32, 0x7a, + 0x88, 0x9a, 0x40, 0x6f, 0xc0, 0xb5, 0x7e, 0xaa, 0x52, 0x6f, 0xac, 0x34, 0xea, 0x4a, 0xa5, 0xde, + 0xa8, 0x6d, 0xae, 0x34, 0x2a, 0xab, 0x62, 0x72, 0x44, 0x2e, 0xf2, 0x12, 0x59, 0xae, 0x94, 0x1b, + 0x62, 0x4a, 0x72, 0xe1, 0xa2, 0x8c, 0x35, 0xab, 0xd5, 0xee, 0xb8, 0x98, 0x94, 0xd2, 0x89, 0x73, + 0xa4, 0x5c, 0x86, 0x69, 0xdd, 0xee, 0x2a, 0x76, 0xc7, 0xe4, 0xe3, 0x64, 0x4a, 0xb7, 0xbb, 0x72, + 0xc7, 0x94, 0xfe, 0xbe, 0x00, 0x97, 0xfa, 0x5f, 0x1b, 0x67, 0x27, 0x7c, 0x0a, 0x59, 0x55, 0xd7, + 0xb1, 0xae, 0xe8, 0xb8, 0xe9, 0xaa, 0xdc, 0x24, 0xba, 0x17, 0x92, 0xc4, 0xf7, 0xf9, 0x8a, 0xfe, + 0x3e, 0xdf, 0xe6, 0xb3, 0x72, 0x99, 0x16, 0x64, 0x95, 0x70, 0x78, 0xd3, 0x0f, 0x15, 0x42, 0x29, + 0xd2, 0x8f, 0x53, 0x90, 0xab, 0x98, 0x7a, 0xe3, 0x38, 0xd6, 0xb5, 0xe4, 0x12, 0x4c, 0x69, 0x56, + 0xab, 0x65, 0xb8, 0x9e, 0x82, 0xd8, 0x13, 0xfa, 0x56, 0xc8, 0x94, 0x4d, 0x8e, 0x61, 0xd0, 0x05, + 0x46, 0x2c, 0xfa, 0x0d, 0xb8, 0x4c, 0x66, 0x4d, 0xdb, 0x54, 0x9b, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, + 0xb1, 0xbf, 0x8f, 0x6d, 0xbe, 0xb7, 0x78, 0x27, 0xa2, 0x9c, 0x35, 0xce, 0x51, 0xa6, 0x0c, 0x0d, + 0x96, 0x5f, 0xbe, 0x68, 0x44, 0x91, 0xd1, 0x47, 0x00, 0x64, 0x29, 0xa2, 0xfb, 0x95, 0x0e, 0x9f, + 0x8f, 0x86, 0x6d, 0x58, 0x7a, 0x53, 0x10, 0x61, 0x20, 0xcf, 0x0e, 0x7a, 0x0a, 0xa2, 0x61, 0x2a, + 0x7b, 0x4d, 0x63, 0xff, 0xc0, 0x55, 0x5e, 0xda, 0x86, 0x8b, 0x9d, 0xf9, 0x59, 0x2a, 0x23, 0xaa, + 0xa9, 0xeb, 0x7c, 0xdf, 0x55, 0x7f, 0x4e, 0x72, 0x72, 0x69, 0x79, 0xc3, 0x7c, 0x44, 0xf9, 0x29, + 0xd1, 0x41, 0xcb, 0x04, 0x0a, 0xbd, 0xe8, 0x18, 0x36, 0x56, 0xee, 0xb7, 0x35, 0xba, 0x0f, 0x92, + 0x2e, 0xe5, 0x4f, 0x4f, 0x96, 0x40, 0x66, 0xe4, 0xfb, 0xdb, 0x65, 0x02, 0x8d, 0xd8, 0xef, 0xb6, + 0x46, 0xd4, 0xde, 0xb6, 0x0c, 0xc7, 0x32, 0xe9, 0x46, 0x67, 0x5a, 0xe6, 0x4f, 0xe8, 0x2e, 0x88, + 0xee, 0xb1, 0xa9, 0x1c, 0x60, 0xd5, 0x76, 0x77, 0xb1, 0xea, 0x92, 0xf5, 0x19, 0x68, 0x8e, 0x82, + 0x7b, 0x6c, 0x56, 0x43, 0xe4, 0xb5, 0x54, 0x7a, 0x5a, 0x4c, 0xaf, 0xa5, 0xd2, 0x69, 0x31, 0x23, + 0xfd, 0x3b, 0x01, 0xf2, 0x5e, 0xdf, 0x88, 0xb3, 0x1b, 0xdf, 0x01, 0xd1, 0x32, 0xb1, 0xd2, 0x3e, + 0x50, 0x1d, 0xcc, 0xdb, 0x92, 0xaf, 0x0e, 0x79, 0xcb, 0xc4, 0xdb, 0x84, 0xcc, 0x5a, 0x06, 0x6d, + 0xc3, 0xac, 0xe3, 0xaa, 0xfb, 0x86, 0xb9, 0xaf, 0xf8, 0xfb, 0xf7, 0xd4, 0xb2, 0x18, 0x13, 0x09, + 0x88, 0x9c, 0xdb, 0xa7, 0xf7, 0x98, 0x14, 0x3f, 0x17, 0x60, 0x76, 0x45, 0x6f, 0x19, 0x66, 0xbd, + 0xdd, 0x34, 0x62, 0xdd, 0x60, 0x78, 0x03, 0x32, 0x0e, 0x91, 0x19, 0xcc, 0xce, 0x01, 0x5c, 0x4c, + 0xd3, 0x14, 0x32, 0x4d, 0x6f, 0x40, 0x01, 0x1f, 0xb7, 0x0d, 0x76, 0x68, 0xc0, 0x50, 0x4e, 0x6a, + 0xfc, 0xba, 0xe5, 0x03, 0x5e, 0x92, 0xc4, 0xeb, 0xf4, 0x09, 0xa0, 0x70, 0x95, 0xe2, 0x04, 0x1a, + 0x9f, 0xc0, 0x1c, 0x15, 0xbd, 0x63, 0x3a, 0x31, 0xeb, 0x4b, 0xfa, 0x35, 0xb8, 0xd0, 0x2b, 0x3a, + 0xce, 0x72, 0x3f, 0xe7, 0xad, 0xbc, 0x89, 0xed, 0x58, 0x11, 0xaa, 0xaf, 0x6b, 0x2e, 0x38, 0xce, + 0x32, 0xff, 0xa6, 0x00, 0x57, 0xa8, 0x6c, 0x7a, 0xae, 0xb2, 0x87, 0xed, 0x0d, 0xac, 0x3a, 0xb1, + 0xc2, 0xeb, 0x1b, 0x30, 0xc5, 0x60, 0x32, 0xed, 0x9f, 0x93, 0xa5, 0x2c, 0x31, 0x33, 0xea, 0xae, + 0x65, 0x13, 0x33, 0x83, 0x27, 0x49, 0x2a, 0x2c, 0x44, 0x95, 0x22, 0xce, 0x9a, 0xfe, 0x8e, 0x00, + 0xb3, 0xdc, 0xc2, 0x23, 0x5d, 0xb9, 0x7c, 0x40, 0x0c, 0x1c, 0x54, 0x81, 0xac, 0x46, 0x7f, 0x29, + 0x6e, 0xb7, 0x8d, 0xa9, 0xfc, 0xfc, 0x28, 0xe3, 0x90, 0xb1, 0x35, 0xba, 0x6d, 0x4c, 0x2c, 0x4c, + 0xef, 0x37, 0x51, 0x54, 0xa8, 0x92, 0x23, 0xcd, 0x4b, 0x3a, 0x8e, 0x68, 0x5e, 0xcf, 0x4e, 0xe3, + 0x3a, 0xf8, 0x87, 0x49, 0xae, 0x04, 0xf6, 0x0e, 0x9e, 0x3d, 0x56, 0x83, 0xe2, 0x53, 0xb8, 0x14, + 0xda, 0x3a, 0x0f, 0x57, 0x3c, 0x71, 0x8e, 0x8a, 0x87, 0xb6, 0xdf, 0x03, 0x2a, 0xfa, 0x04, 0x42, + 0x1b, 0xec, 0x0a, 0xab, 0x93, 0x07, 0x55, 0xce, 0xa3, 0x8e, 0xd9, 0x40, 0x0a, 0xa3, 0x3b, 0xa8, + 0x0c, 0x69, 0x7c, 0xdc, 0x56, 0x74, 0xec, 0x68, 0x7c, 0xe2, 0x92, 0xa2, 0x04, 0x92, 0xa2, 0x0c, + 0x18, 0xef, 0xd3, 0xf8, 0xb8, 0x4d, 0x88, 0x68, 0x87, 0xac, 0x9b, 0xde, 0xba, 0x4e, 0x8b, 0xed, + 0x9c, 0x8d, 0x05, 0x82, 0x9e, 0xc2, 0xc5, 0x15, 0xfc, 0x25, 0x9d, 0x89, 0x90, 0x3e, 0x17, 0xe0, + 0x6a, 0x64, 0xab, 0xc5, 0xb9, 0x90, 0x7d, 0x04, 0x29, 0x5a, 0xf9, 0xc4, 0x39, 0x2b, 0x4f, 0xb9, + 0xa4, 0x1f, 0x24, 0xf8, 0x18, 0x97, 0x71, 0xd3, 0x22, 0x8a, 0x8d, 0x7d, 0x0b, 0xed, 0x09, 0xe4, + 0x8e, 0x2c, 0x17, 0xdb, 0x7e, 0xb3, 0x27, 0xce, 0xdd, 0xec, 0x33, 0x54, 0x80, 0xd7, 0xe2, 0xcf, + 0x60, 0xd6, 0xb4, 0x4c, 0xa5, 0x57, 0xe8, 0xf9, 0xfb, 0x52, 0xc1, 0xb4, 0xcc, 0x67, 0x21, 0xb9, + 0xfe, 0x3c, 0xd3, 0xa7, 0x89, 0x38, 0xe7, 0x99, 0x1f, 0x0a, 0x30, 0xe7, 0x5b, 0x3a, 0x31, 0x9b, + 0xbb, 0xef, 0x42, 0xd2, 0xb4, 0x5e, 0x9e, 0x67, 0x8b, 0x92, 0xe4, 0x27, 0xab, 0x5e, 0x6f, 0x89, + 0xe2, 0xac, 0xef, 0x3f, 0x4f, 0x40, 0xe6, 0x71, 0x39, 0xce, 0x5a, 0x7e, 0xc4, 0xb7, 0xbf, 0x59, + 0x7b, 0x47, 0xf5, 0x76, 0xff, 0x7d, 0xc5, 0xc7, 0xe5, 0x75, 0xdc, 0xf5, 0x7a, 0x3b, 0xe1, 0x42, + 0x2b, 0x90, 0xe9, 0xdd, 0x28, 0x1d, 0x53, 0x53, 0x01, 0xd7, 0x02, 0x86, 0x49, 0x2a, 0xd7, 0xf3, + 0xa3, 0x10, 0x22, 0xfc, 0x28, 0xc8, 0x6b, 0x7c, 0x4b, 0x31, 0x71, 0x9e, 0xd7, 0x84, 0x4c, 0xc4, + 0x49, 0x71, 0x4a, 0x7a, 0x0a, 0x40, 0xaa, 0x13, 0x67, 0x93, 0x7c, 0x3f, 0x09, 0xf9, 0xed, 0x8e, + 0x73, 0x10, 0x73, 0xef, 0x2b, 0x03, 0xb4, 0x3b, 0xce, 0x01, 0x19, 0x91, 0xc7, 0x26, 0xaf, 0xf3, + 0x19, 0x2e, 0x1a, 0x5e, 0xa5, 0x19, 0x5f, 0xe3, 0xd8, 0x44, 0x55, 0x2e, 0x04, 0x2b, 0x81, 0x9f, + 0xc7, 0x8d, 0x51, 0xc8, 0xb2, 0x71, 0x6c, 0x6e, 0x62, 0x1f, 0x52, 0x32, 0x49, 0x98, 0x48, 0xfa, + 0x08, 0xa6, 0xc9, 0x83, 0xe2, 0x5a, 0xe7, 0x69, 0xe6, 0x29, 0xc2, 0xd3, 0xb0, 0xd0, 0x87, 0x90, + 0x61, 0xdc, 0x64, 0xf5, 0x9b, 0xa2, 0xab, 0x5f, 0x54, 0x5d, 0xb8, 0x1a, 0xe9, 0xba, 0x97, 0xa6, + 0xac, 0x64, 0xad, 0xbb, 0x00, 0x93, 0x7b, 0x96, 0xad, 0x79, 0x87, 0xb9, 0xec, 0x81, 0xb5, 0x27, + 0x83, 0x34, 0x6b, 0xa9, 0x74, 0x46, 0x04, 0xe9, 0xb7, 0x05, 0x28, 0xf8, 0x0d, 0x11, 0xe7, 0x82, + 0x50, 0xee, 0xd1, 0xe2, 0xf9, 0x9b, 0x82, 0x28, 0x50, 0xfa, 0x17, 0xd4, 0x22, 0xd2, 0xac, 0x23, + 0xda, 0x32, 0x71, 0xf6, 0x94, 0x0f, 0x99, 0x17, 0x4f, 0xe2, 0xbc, 0xad, 0x4b, 0x1d, 0x7a, 0xee, + 0xc3, 0x05, 0xa3, 0x45, 0xe6, 0x73, 0xc3, 0x6d, 0x76, 0x39, 0x6c, 0x73, 0xb1, 0x77, 0x6a, 0x3c, + 0x17, 0xa4, 0x95, 0xbd, 0x24, 0xe9, 0xf7, 0xe9, 0x6e, 0x75, 0x50, 0x93, 0x38, 0x55, 0x5d, 0x83, + 0x9c, 0xcd, 0x44, 0x13, 0xb3, 0xe6, 0x9c, 0xda, 0x9e, 0xf1, 0x59, 0x89, 0xc2, 0x7f, 0x37, 0x01, + 0x85, 0xa7, 0x1d, 0x6c, 0x77, 0xbf, 0x49, 0xea, 0xbe, 0x05, 0x85, 0x97, 0xaa, 0xe1, 0x2a, 0x7b, + 0x96, 0xad, 0x74, 0xda, 0xba, 0xea, 0x7a, 0xde, 0x26, 0x39, 0x42, 0x7e, 0x64, 0xd9, 0x3b, 0x94, + 0x88, 0x30, 0xa0, 0x43, 0xd3, 0x7a, 0x69, 0x2a, 0x84, 0x4c, 0x81, 0xf2, 0xb1, 0xc9, 0xb7, 0x90, + 0x4b, 0xef, 0xfd, 0xdb, 0x93, 0xa5, 0x87, 0x63, 0x39, 0x88, 0x51, 0x67, 0xb8, 0x4e, 0xc7, 0xd0, + 0x8b, 0x3b, 0x3b, 0xb5, 0x55, 0x59, 0xa4, 0x22, 0x9f, 0x33, 0x89, 0x8d, 0x63, 0xd3, 0x91, 0xfe, + 0x56, 0x02, 0xc4, 0x40, 0x47, 0x71, 0x36, 0x64, 0x05, 0xb2, 0x2f, 0x3a, 0xd8, 0x36, 0x5e, 0xa1, + 0x19, 0x81, 0x33, 0x92, 0x69, 0xe7, 0x1e, 0xcc, 0xba, 0xc7, 0xa6, 0xc2, 0xdc, 0xf7, 0x98, 0xe3, + 0x87, 0xe7, 0xb0, 0x50, 0x70, 0x49, 0x99, 0x09, 0x9d, 0x3a, 0x7d, 0x38, 0xe8, 0x53, 0x98, 0xe9, + 0xd1, 0x56, 0xf2, 0xab, 0x69, 0x2b, 0xfb, 0x32, 0xa4, 0xa8, 0x9f, 0x08, 0x80, 0xa8, 0xa2, 0x6a, + 0x6c, 0x8f, 0xff, 0x9b, 0xd2, 0x9f, 0xee, 0x80, 0x48, 0xfd, 0x30, 0x15, 0x63, 0x4f, 0x69, 0x19, + 0x8e, 0x63, 0x98, 0xfb, 0xbc, 0x43, 0xe5, 0x29, 0xbd, 0xb6, 0xb7, 0xc9, 0xa8, 0xd2, 0x5f, 0x80, + 0xb9, 0x9e, 0x0a, 0xc4, 0xd9, 0xd8, 0xd7, 0x61, 0x66, 0x8f, 0x1d, 0xc1, 0x52, 0xe1, 0x7c, 0x7b, + 0x30, 0x4b, 0x69, 0xec, 0x7d, 0xd2, 0x7f, 0x4a, 0xc0, 0x05, 0x19, 0x3b, 0x56, 0xf3, 0x08, 0xc7, + 0xaf, 0xc2, 0x2a, 0xf0, 0xb3, 0x17, 0xe5, 0x95, 0x34, 0x99, 0x61, 0xcc, 0x6c, 0x99, 0xeb, 0xdd, + 0x63, 0x7f, 0x63, 0x74, 0x8f, 0x1d, 0xdc, 0x55, 0xe7, 0x3b, 0x75, 0xa9, 0x9e, 0x9d, 0x3a, 0x0b, + 0x0a, 0xec, 0xf4, 0x58, 0x57, 0x1c, 0xfc, 0xc2, 0xec, 0xb4, 0x3c, 0x30, 0x54, 0x1c, 0x55, 0xc8, + 0x1a, 0x63, 0xa9, 0xe3, 0x17, 0x5b, 0x9d, 0x16, 0xb5, 0x9d, 0x4b, 0x97, 0x48, 0x79, 0x4f, 0x4f, + 0x96, 0xf2, 0x3d, 0x69, 0x8e, 0x9c, 0x37, 0xfc, 0x67, 0x22, 0x5d, 0xfa, 0x2e, 0x5c, 0xec, 0x53, + 0x76, 0x9c, 0x16, 0xcf, 0x3f, 0x4d, 0xc2, 0x95, 0x5e, 0xf1, 0x71, 0x43, 0x9c, 0x6f, 0x7a, 0x83, + 0x56, 0x21, 0xd7, 0x32, 0xcc, 0x57, 0xdb, 0xbd, 0x9c, 0x69, 0x19, 0xa6, 0x4f, 0x8b, 0xea, 0x1a, + 0x53, 0x5f, 0x6b, 0xd7, 0x50, 0x61, 0x21, 0xaa, 0xed, 0xe2, 0xec, 0x1f, 0x3f, 0x10, 0x60, 0x26, + 0xee, 0x6d, 0xb9, 0x57, 0xf3, 0x82, 0x93, 0x1a, 0x90, 0xfb, 0x1a, 0xf6, 0xf1, 0x7e, 0x57, 0x00, + 0xd4, 0xb0, 0x3b, 0x26, 0x01, 0xb5, 0x1b, 0xd6, 0x7e, 0x9c, 0xd5, 0xbc, 0x00, 0x93, 0x86, 0xa9, + 0xe3, 0x63, 0x5a, 0xcd, 0x94, 0xcc, 0x1e, 0x7a, 0x8e, 0x12, 0x93, 0x63, 0x1d, 0x25, 0x4a, 0x9f, + 0xc2, 0x5c, 0x4f, 0x11, 0xe3, 0xac, 0xff, 0x2f, 0x13, 0x30, 0xc7, 0x2b, 0x12, 0xfb, 0x0e, 0xe6, + 0x3b, 0x30, 0xd9, 0x24, 0x32, 0x47, 0xb4, 0x33, 0x7d, 0xa7, 0xd7, 0xce, 0x34, 0x33, 0xfa, 0x36, + 0x40, 0xdb, 0xc6, 0x47, 0x0a, 0x63, 0x4d, 0x8e, 0xc5, 0x9a, 0x21, 0x1c, 0x94, 0x80, 0x7e, 0x24, + 0x40, 0x81, 0x0c, 0xe8, 0xb6, 0x6d, 0xb5, 0x2d, 0x87, 0xd8, 0x2c, 0xce, 0x78, 0x30, 0xe7, 0xe9, + 0xe9, 0xc9, 0x52, 0x6e, 0xd3, 0x30, 0xb7, 0x39, 0x63, 0xa3, 0x3e, 0xb6, 0xf7, 0xbe, 0x17, 0xc3, + 0x50, 0x2c, 0x37, 0x2d, 0xed, 0x30, 0x38, 0x1c, 0x23, 0x33, 0x8b, 0x2f, 0xce, 0x91, 0xfe, 0x8d, + 0x00, 0x17, 0xbe, 0xb6, 0xed, 0xe2, 0xff, 0x1f, 0xca, 0x96, 0x9e, 0x81, 0x48, 0x7f, 0xd4, 0xcc, + 0x3d, 0x2b, 0xce, 0x8d, 0xfb, 0xff, 0x25, 0xc0, 0x6c, 0x48, 0x70, 0x9c, 0x06, 0xce, 0xab, 0xea, 0x29, 0xc7, 0xdc, 0x61, 0xdc, 0xf1, 0x54, 0x25, 0xcf, 0xf0, 0xec, 0xac, 0x53, 0x16, 0x61, 0x06, 0x93, 0x59, 0x8c, 0x6e, 0xf1, 0xee, 0xb2, 0x08, 0x92, 0xbe, 0x1d, 0xfd, 0xac, 0x9f, 0xa1, 0xd4, - 0x95, 0x7e, 0x8d, 0x58, 0x58, 0xe1, 0x41, 0x19, 0xe7, 0x90, 0xff, 0x67, 0x09, 0xb8, 0x54, 0x66, + 0x95, 0x7e, 0x8d, 0x58, 0x58, 0xe1, 0x41, 0x19, 0xe7, 0x90, 0xff, 0x27, 0x09, 0xb8, 0x54, 0x66, 0x47, 0xe0, 0x9e, 0x4f, 0x48, 0x9c, 0x1d, 0x71, 0x1e, 0xa6, 0x8f, 0xb0, 0xed, 0x18, 0x16, 0x5b, 0xed, 0x73, 0xb2, 0xf7, 0x88, 0x16, 0x20, 0xed, 0x98, 0x6a, 0xdb, 0x39, 0xb0, 0xbc, 0xe3, 0x44, 0xff, 0xd9, 0xf7, 0x5f, 0x99, 0x7c, 0x75, 0xff, 0x95, 0xa9, 0xd1, 0xfe, 0x2b, 0xd3, 0x5f, 0xc1, - 0x7f, 0x85, 0x9f, 0xdd, 0xfd, 0x6b, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0xce, 0xce, 0xf9, 0x3d, 0xc8, + 0x7f, 0x85, 0x9f, 0xdd, 0xfd, 0x4b, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0xce, 0xce, 0xf9, 0x3d, 0xc8, 0x6a, 0x5c, 0x30, 0x59, 0x1f, 0xd8, 0xc1, 0x64, 0x8d, 0x64, 0x7b, 0x45, 0xe8, 0x73, 0x7a, 0xb2, - 0x04, 0x5e, 0x51, 0x6b, 0xab, 0x5c, 0x39, 0xe4, 0xb7, 0x2e, 0xfd, 0x57, 0x80, 0x42, 0xe5, 0x98, + 0x04, 0x5e, 0x51, 0x6b, 0xab, 0x5c, 0x39, 0xe4, 0xb7, 0x2e, 0xfd, 0x17, 0x80, 0x42, 0xe5, 0x98, 0x6d, 0xca, 0xd7, 0x99, 0x55, 0x82, 0x1e, 0x41, 0xba, 0x6d, 0x5b, 0x47, 0x86, 0x57, 0x8d, 0x7c, 0x8f, 0xf3, 0x82, 0x57, 0x8d, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0xfb, 0xbc, 0xa8, 0x01, 0x99, 0x0d, 0x4b, 0x53, 0x9b, 0x8f, 0x8c, 0xa6, 0x37, 0xd0, 0xde, 0x3e, 0x5b, 0x50, 0xd1, 0xe7, 0xd9, 0x56, @@ -8147,7 +8172,7 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0x05, 0x19, 0x72, 0x3d, 0xed, 0x85, 0x10, 0xa4, 0xda, 0xa4, 0x69, 0x04, 0xea, 0x86, 0x44, 0x7f, 0xa3, 0xb7, 0x60, 0xda, 0xb4, 0x74, 0xec, 0x75, 0xe6, 0x5c, 0xe9, 0xc2, 0xe9, 0xc9, 0xd2, 0xd4, 0x96, 0xa5, 0x33, 0x5b, 0x87, 0xff, 0x92, 0xa7, 0x48, 0xa6, 0x9a, 0xbe, 0x70, 0x0d, 0x52, 0xa4, - 0x89, 0xc8, 0x1c, 0xb2, 0xab, 0x3a, 0x78, 0xc7, 0x36, 0xb8, 0x34, 0xef, 0x71, 0xe1, 0x1f, 0x24, + 0x89, 0xc8, 0x1c, 0xb2, 0xab, 0x3a, 0x78, 0xc7, 0x36, 0xb8, 0x34, 0xef, 0x71, 0xe1, 0xef, 0x25, 0x20, 0x51, 0x7f, 0x48, 0xac, 0xf9, 0xdd, 0x8e, 0x76, 0x88, 0x5d, 0x9e, 0xce, 0x9f, 0xa8, 0x95, 0x6f, 0xe3, 0x3d, 0x83, 0x19, 0x5d, 0x19, 0x99, 0x3f, 0xa1, 0xd7, 0x01, 0x54, 0x4d, 0xc3, 0x8e, 0xa3, 0x78, 0xf1, 0x7d, 0x19, 0x39, 0xc3, 0x28, 0xeb, 0xb8, 0x4b, 0xd8, 0x1c, 0xac, 0xd9, 0xd8, @@ -8157,11 +8182,11 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0x71, 0x0f, 0x68, 0x4b, 0x64, 0x64, 0xfa, 0x1b, 0xdd, 0x82, 0x02, 0x73, 0xbb, 0x54, 0xb0, 0xa9, 0x29, 0x74, 0x1e, 0xcc, 0xd0, 0xe4, 0x1c, 0x23, 0x57, 0x4c, 0x8d, 0xcc, 0x7a, 0xe8, 0x21, 0x70, 0x82, 0x72, 0xd8, 0x72, 0x88, 0x4e, 0x81, 0xe4, 0x2a, 0x15, 0x4e, 0x4f, 0x96, 0xb2, 0x75, 0x9a, - 0xb0, 0xbe, 0x59, 0x27, 0x6b, 0x09, 0xcb, 0xb5, 0xde, 0x72, 0x6a, 0xfa, 0xc2, 0xdf, 0x14, 0x20, + 0xb0, 0xbe, 0x59, 0x27, 0x6b, 0x09, 0xcb, 0xb5, 0xde, 0x72, 0x6a, 0xfa, 0xc2, 0x5f, 0x17, 0x20, 0xf9, 0xb8, 0x5c, 0x3f, 0xb7, 0xca, 0xbc, 0x82, 0x26, 0x43, 0x05, 0xbd, 0x0d, 0x85, 0x5d, 0xa3, 0xd9, 0x34, 0xcc, 0x7d, 0x62, 0x5f, 0x7d, 0x0f, 0x6b, 0x9e, 0xc2, 0xf2, 0x9c, 0xbc, 0xcd, 0xa8, 0xe8, 0x1a, 0x64, 0x35, 0x1b, 0xeb, 0xd8, 0x74, 0x0d, 0xb5, 0xe9, 0x70, 0xcd, 0x85, 0x49, 0x0b, - 0x7f, 0x59, 0x80, 0x49, 0xda, 0x59, 0xd1, 0x6b, 0x90, 0xd1, 0x2c, 0xd3, 0x55, 0x0d, 0x93, 0xcf, + 0x7f, 0x51, 0x80, 0x49, 0xda, 0x59, 0xd1, 0x6b, 0x90, 0xd1, 0x2c, 0xd3, 0x55, 0x0d, 0x93, 0xcf, 0x3a, 0x19, 0x39, 0x20, 0x0c, 0x2d, 0xde, 0x75, 0x98, 0x51, 0x35, 0xcd, 0xea, 0x98, 0xae, 0x62, 0xaa, 0x2d, 0xcc, 0x8b, 0x99, 0xe5, 0xb4, 0x2d, 0xb5, 0x85, 0xd1, 0x12, 0x78, 0x8f, 0x7e, 0xd8, 0x66, 0x46, 0x06, 0x4e, 0x5a, 0xc7, 0xdd, 0x05, 0x0c, 0x19, 0xbf, 0x57, 0x93, 0xfa, 0x76, 0x1c, @@ -8182,7 +8207,7 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0x21, 0x41, 0x8e, 0xb9, 0x27, 0xb5, 0x0c, 0x16, 0x72, 0x9c, 0x63, 0xdb, 0x70, 0x94, 0xb8, 0x69, 0xd0, 0x70, 0xe3, 0x37, 0x81, 0x07, 0xdd, 0x28, 0x8e, 0x17, 0x42, 0x57, 0xca, 0x9d, 0x9e, 0x2c, 0x65, 0x64, 0x4a, 0xad, 0xd7, 0x1b, 0x72, 0x86, 0x65, 0xa8, 0x3b, 0xee, 0x79, 0xc2, 0x2c, 0xa6, - 0xc7, 0x0f, 0xb3, 0xf8, 0x6b, 0x02, 0x5c, 0xe2, 0xed, 0xa3, 0xec, 0x52, 0xd7, 0x72, 0xb5, 0x69, + 0xc7, 0x0f, 0xb3, 0xf8, 0x2b, 0x02, 0x5c, 0xe2, 0xed, 0xa3, 0xec, 0x52, 0xd7, 0x72, 0xb5, 0x69, 0xb8, 0x5d, 0xe5, 0xf0, 0x68, 0x3e, 0x4d, 0xcd, 0xad, 0x6f, 0x45, 0xb6, 0x73, 0xa8, 0x7b, 0x15, 0xbd, 0xd6, 0xee, 0x6e, 0x70, 0xe6, 0xf5, 0xa3, 0x8a, 0xe9, 0xda, 0xdd, 0xd2, 0xe5, 0xd3, 0x93, 0xa5, 0xb9, 0xc1, 0xd4, 0x67, 0xf2, 0x9c, 0x33, 0xc8, 0x82, 0xaa, 0x00, 0xd8, 0xef, 0xde, 0x74, @@ -8191,7 +8216,7 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0x3e, 0xc3, 0xe8, 0x3e, 0x5c, 0x0c, 0x5a, 0x40, 0xd9, 0xc5, 0x4d, 0xeb, 0x25, 0xcb, 0x9e, 0xa5, 0xd9, 0x91, 0xaf, 0xfd, 0x12, 0x49, 0x22, 0x2c, 0x0b, 0xdf, 0x83, 0xf9, 0x61, 0x15, 0x0e, 0x0f, 0xc6, 0x0c, 0x3b, 0x2b, 0x7d, 0xbf, 0x77, 0xa3, 0x64, 0x8c, 0x41, 0xc3, 0x37, 0x4b, 0x3e, 0x48, - 0xbc, 0x2f, 0x48, 0x7f, 0x2f, 0x01, 0xb9, 0x52, 0xa7, 0x79, 0xf8, 0xa4, 0x5d, 0x67, 0x57, 0x01, + 0xbc, 0x2f, 0x48, 0x7f, 0x27, 0x01, 0xb9, 0x52, 0xa7, 0x79, 0xf8, 0xa4, 0x5d, 0x67, 0x57, 0x01, 0xa0, 0xab, 0x90, 0xd1, 0x55, 0x57, 0x65, 0x85, 0x14, 0x58, 0x78, 0x1b, 0x21, 0xd0, 0xda, 0xdc, 0x86, 0x42, 0xc8, 0x0f, 0x85, 0x7b, 0xdb, 0xd3, 0x6a, 0x07, 0x64, 0xea, 0x10, 0xff, 0x3e, 0xcc, 0x87, 0x32, 0xd2, 0x5d, 0x0d, 0x05, 0x9b, 0xae, 0x6d, 0x60, 0xb6, 0x33, 0x97, 0x94, 0x43, 0xce, @@ -8217,7 +8242,7 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0x6c, 0x02, 0x1d, 0x32, 0xfc, 0x68, 0xd2, 0xd0, 0xf9, 0x0d, 0x1f, 0x8f, 0xbf, 0x1a, 0xf0, 0x4e, 0xb3, 0x23, 0xcd, 0xda, 0xaa, 0x9c, 0x66, 0x92, 0x6b, 0x3a, 0x5a, 0x81, 0x4c, 0xdb, 0xab, 0xc6, 0xb9, 0xbc, 0x5f, 0x7c, 0x2e, 0xb4, 0x0e, 0x05, 0x5e, 0x50, 0xb5, 0x69, 0x1c, 0x61, 0x45, 0x75, - 0xcf, 0x33, 0x23, 0xe4, 0x18, 0xef, 0x0a, 0x61, 0x5d, 0x71, 0xa5, 0xbf, 0x91, 0x82, 0x9b, 0x67, + 0xcf, 0x33, 0x23, 0xe4, 0x18, 0xef, 0x0a, 0x61, 0x5d, 0x71, 0xa5, 0xbf, 0x96, 0x82, 0x9b, 0x67, 0xa8, 0x38, 0xce, 0xee, 0xb5, 0x00, 0xe9, 0x23, 0xf2, 0x22, 0x83, 0xd7, 0x3e, 0x2d, 0xfb, 0xcf, 0x68, 0xb7, 0x67, 0x59, 0xd9, 0x53, 0x8d, 0x26, 0x59, 0x86, 0x98, 0xbf, 0xe1, 0x70, 0x8f, 0xa6, 0x68, 0xff, 0xbd, 0xd0, 0x02, 0xf4, 0x88, 0x0a, 0xa2, 0xd9, 0x1c, 0xf4, 0x03, 0x01, 0x16, 0xd8, @@ -8225,296 +8250,297 @@ var fileDescriptor_e08772acc330f58b = []byte{ 0xd0, 0xbb, 0x78, 0x41, 0xe6, 0xc3, 0x6f, 0x0b, 0x17, 0x65, 0xe1, 0xb7, 0x04, 0xc8, 0x86, 0x08, 0xe8, 0xd6, 0x40, 0x88, 0x51, 0xf6, 0x34, 0x2a, 0xae, 0xe8, 0xe6, 0x40, 0x5c, 0x51, 0x29, 0xfd, 0xe5, 0xc9, 0x52, 0x4a, 0x66, 0xae, 0xeb, 0x5e, 0x84, 0xd1, 0xf5, 0xe0, 0xba, 0x9a, 0x64, 0x5f, - 0x26, 0xef, 0xbe, 0x1a, 0x8a, 0x50, 0x55, 0xef, 0x44, 0x8c, 0x22, 0x54, 0xf2, 0x24, 0xfd, 0x38, - 0x01, 0xb3, 0x2b, 0xba, 0x5e, 0xaf, 0x53, 0x54, 0x13, 0xe7, 0x18, 0x43, 0x90, 0x22, 0xe6, 0x06, - 0x0f, 0x87, 0xa2, 0xbf, 0xd1, 0x5b, 0x80, 0x74, 0xc3, 0x61, 0x37, 0x43, 0x38, 0x07, 0xaa, 0x6e, - 0xbd, 0x0c, 0x0e, 0xbe, 0x67, 0xbd, 0x94, 0xba, 0x97, 0x80, 0xea, 0x40, 0x6d, 0x6d, 0xc5, 0x71, - 0x55, 0x7f, 0x63, 0xff, 0xe6, 0x58, 0x01, 0x36, 0xcc, 0x38, 0xf6, 0x1f, 0xe5, 0x0c, 0x91, 0x43, - 0x7f, 0x12, 0x93, 0xcf, 0x20, 0x8d, 0xe2, 0x2a, 0xaa, 0xe3, 0x85, 0x86, 0xb0, 0x3b, 0x29, 0xf2, - 0x8c, 0xbe, 0xe2, 0xb0, 0x88, 0x0f, 0xe6, 0x51, 0x1e, 0xa8, 0x26, 0xce, 0x6d, 0xd9, 0xbf, 0x2d, - 0x40, 0x5e, 0xc6, 0x7b, 0x36, 0x76, 0x0e, 0xe2, 0xd4, 0xf9, 0x23, 0x98, 0xb1, 0x99, 0x54, 0x65, - 0xcf, 0xb6, 0x5a, 0xe7, 0x99, 0x2b, 0xb2, 0x9c, 0xf1, 0x91, 0x6d, 0xb5, 0xf8, 0x94, 0xfc, 0x0c, - 0x0a, 0x7e, 0x19, 0xe3, 0xac, 0xfc, 0xdf, 0xa5, 0xc1, 0xa3, 0x4c, 0x70, 0xdc, 0x27, 0xd0, 0xf1, - 0x6a, 0x80, 0x6e, 0xcd, 0x87, 0x0b, 0x1a, 0xa7, 0x1a, 0xfe, 0xb3, 0x00, 0xf9, 0x7a, 0x67, 0x97, - 0xdd, 0x92, 0x14, 0x9f, 0x06, 0x2a, 0x90, 0x69, 0xe2, 0x3d, 0x57, 0x79, 0x25, 0x5f, 0xe8, 0x34, - 0x61, 0xa5, 0x9e, 0xe0, 0x8f, 0x01, 0x6c, 0x1a, 0x3d, 0x45, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x43, - 0x79, 0x09, 0x99, 0xac, 0x3a, 0x05, 0xbf, 0x9a, 0x71, 0xae, 0x2f, 0xcf, 0x7b, 0x66, 0x87, 0xe4, - 0x79, 0x66, 0x87, 0x59, 0x6e, 0xb0, 0x46, 0xcf, 0x10, 0x45, 0x98, 0xa3, 0x66, 0x99, 0xa2, 0xb6, - 0xdb, 0x4d, 0xc3, 0x83, 0x3d, 0x74, 0xfe, 0x49, 0xc9, 0xb3, 0x34, 0x69, 0x85, 0xa5, 0x50, 0xc0, - 0x83, 0xbe, 0x2f, 0xc0, 0xcc, 0x9e, 0x8d, 0xf1, 0x67, 0x58, 0xa1, 0x53, 0xf2, 0x78, 0x5e, 0x05, - 0xab, 0xa4, 0x0c, 0x5f, 0xf9, 0xd4, 0x31, 0xcb, 0x5e, 0x5c, 0x27, 0xef, 0x45, 0x5b, 0x20, 0x6a, - 0x4d, 0x76, 0x0e, 0xea, 0x7b, 0x38, 0x4c, 0x8d, 0x3f, 0x00, 0x0a, 0x8c, 0x39, 0x70, 0x72, 0x78, - 0x4a, 0x06, 0x93, 0xaa, 0x2b, 0xfc, 0x76, 0x39, 0x6a, 0x6c, 0xf7, 0x7a, 0x38, 0x84, 0xa3, 0xc8, - 0x43, 0x97, 0xd2, 0x15, 0x65, 0xac, 0xea, 0xdc, 0x72, 0x27, 0xe3, 0xca, 0x7f, 0xe0, 0xe3, 0xea, - 0x39, 0xcc, 0xd2, 0x7e, 0x13, 0x77, 0x30, 0xa8, 0xf4, 0x8f, 0x92, 0x80, 0xc2, 0x92, 0xbf, 0xbe, - 0xfe, 0x96, 0x88, 0xaf, 0xbf, 0xad, 0x81, 0x14, 0x32, 0x86, 0x9a, 0xaa, 0xe3, 0x2a, 0xcc, 0x95, - 0xce, 0x51, 0xda, 0xd8, 0x56, 0x1c, 0xac, 0x59, 0xfc, 0x3e, 0x20, 0x41, 0x5e, 0x0c, 0x72, 0x6e, - 0xa8, 0x8e, 0xfb, 0x94, 0xe5, 0xdb, 0xc6, 0x76, 0x9d, 0xe6, 0x42, 0x0f, 0xe1, 0x52, 0x4b, 0x3d, - 0x8e, 0xe2, 0x9f, 0xa4, 0xfc, 0x73, 0x2d, 0xf5, 0x78, 0x80, 0xe9, 0x03, 0x58, 0x88, 0x66, 0x52, - 0x1c, 0xec, 0x1d, 0xb5, 0x5d, 0x8a, 0x60, 0xac, 0x63, 0x17, 0xad, 0x00, 0x04, 0x20, 0x82, 0xaf, - 0xd1, 0xe3, 0x60, 0x88, 0x8c, 0x8f, 0x21, 0xa4, 0x1f, 0x0a, 0x90, 0xdf, 0x34, 0xf6, 0x6d, 0x35, - 0xd6, 0xdb, 0x76, 0xd0, 0x07, 0xbd, 0x67, 0x93, 0xd9, 0x07, 0x0b, 0x51, 0xbe, 0x27, 0x2c, 0x87, - 0xb7, 0xff, 0xc8, 0x19, 0xc8, 0xd2, 0xe7, 0x97, 0x28, 0xce, 0x39, 0x5f, 0x83, 0xd7, 0xa9, 0x37, - 0x1f, 0x77, 0xe5, 0xf9, 0x5a, 0xd0, 0x8d, 0xf4, 0x07, 0x02, 0x2c, 0x0e, 0x7b, 0x4b, 0x9c, 0x03, - 0x42, 0xa6, 0x97, 0x0a, 0xd2, 0x37, 0x28, 0xfe, 0x88, 0x38, 0x63, 0xa6, 0x41, 0x7c, 0x24, 0x80, - 0x5f, 0xb6, 0x3a, 0xbd, 0x6e, 0x90, 0xfd, 0x76, 0x88, 0x82, 0xea, 0x9a, 0x6a, 0xd2, 0x50, 0xe1, - 0x26, 0x56, 0x8f, 0x30, 0x77, 0x44, 0x8c, 0x75, 0x96, 0xf8, 0x5c, 0x80, 0xc5, 0x61, 0x6f, 0x89, - 0x53, 0x41, 0xdf, 0x82, 0x69, 0xe6, 0xd1, 0xe6, 0x81, 0x9a, 0x2b, 0x43, 0xa2, 0xa2, 0x4d, 0x2f, - 0x16, 0xc7, 0xcb, 0x2f, 0x35, 0x20, 0x5f, 0x52, 0x6d, 0xdb, 0x88, 0x75, 0xeb, 0x41, 0xfa, 0xb1, - 0x00, 0x05, 0x5f, 0x6c, 0x9c, 0x35, 0xfd, 0xea, 0x81, 0x1e, 0xd2, 0x7f, 0xbb, 0x0a, 0x33, 0xbc, - 0xfc, 0x3b, 0xa6, 0x61, 0x99, 0xe8, 0x3e, 0x24, 0xf7, 0xf9, 0x01, 0x56, 0x36, 0x72, 0x8f, 0x3e, - 0xb8, 0xe6, 0xb1, 0x3a, 0x21, 0x93, 0xbc, 0x84, 0xa5, 0xdd, 0x71, 0x23, 0x0a, 0x10, 0x44, 0x2a, - 0x84, 0x59, 0xda, 0x1d, 0x17, 0xd5, 0xa1, 0xa0, 0x05, 0x77, 0xcb, 0x29, 0x84, 0x3d, 0x39, 0x74, - 0x43, 0x39, 0xf2, 0x96, 0xbf, 0xea, 0x84, 0x9c, 0xd7, 0x7a, 0x12, 0x50, 0x39, 0x7c, 0xa5, 0x59, - 0x6a, 0xc0, 0x0d, 0x32, 0x68, 0xfa, 0xde, 0xeb, 0xd4, 0xaa, 0x13, 0xa1, 0x9b, 0xcf, 0xd0, 0x07, - 0x30, 0xa5, 0xd3, 0xcb, 0xb3, 0xb8, 0x3d, 0x11, 0xd5, 0xe0, 0x3d, 0x77, 0x94, 0x55, 0x27, 0x64, - 0xce, 0x81, 0xd6, 0x60, 0x86, 0xfd, 0x62, 0x68, 0x97, 0x5b, 0x01, 0x37, 0x87, 0x4b, 0x08, 0xd9, - 0xe1, 0xd5, 0x09, 0x39, 0xab, 0x07, 0x54, 0xf4, 0x18, 0xb2, 0x5a, 0x13, 0xab, 0x36, 0x17, 0x75, - 0x6b, 0x68, 0xec, 0xe6, 0xc0, 0x85, 0x5b, 0xd5, 0x09, 0x19, 0x34, 0x9f, 0x48, 0x0a, 0x65, 0xd3, - 0x7b, 0x97, 0xb8, 0xa4, 0xb7, 0x87, 0x16, 0x6a, 0xf0, 0x12, 0xab, 0x2a, 0xb5, 0xcf, 0x7d, 0x2a, - 0x7a, 0x07, 0x52, 0x8e, 0xa6, 0x9a, 0xdc, 0x24, 0x59, 0x1c, 0x72, 0x31, 0x4e, 0xc0, 0x4c, 0x73, - 0xa3, 0x0f, 0x19, 0x50, 0x76, 0x8f, 0xbd, 0x53, 0x83, 0x28, 0x9d, 0xf6, 0x5c, 0xc0, 0x40, 0x74, - 0x8a, 0x29, 0x81, 0xe8, 0x41, 0xd5, 0x5b, 0x86, 0xa9, 0xd0, 0x83, 0x19, 0x7a, 0x4c, 0x10, 0xad, - 0x87, 0x81, 0x28, 0xf6, 0x2a, 0xbd, 0xe5, 0xc1, 0x23, 0xa2, 0x4d, 0xc8, 0x31, 0x41, 0x1d, 0x16, - 0x60, 0x3d, 0xbf, 0x3c, 0xd4, 0x17, 0x21, 0x22, 0xc4, 0xbb, 0x3a, 0x21, 0xcf, 0xa8, 0x21, 0x72, - 0x50, 0xae, 0x16, 0xb6, 0xf7, 0xd9, 0x79, 0xc4, 0x88, 0x72, 0x85, 0x1d, 0x3c, 0xfd, 0x72, 0x51, - 0x22, 0xfa, 0x0d, 0xb8, 0xc0, 0x04, 0xb9, 0xdc, 0x6f, 0x8d, 0xbb, 0x3f, 0xbd, 0x3e, 0xd4, 0x8f, - 0x60, 0x68, 0x50, 0x74, 0x75, 0x42, 0x46, 0xea, 0x40, 0x22, 0xd2, 0xe0, 0x22, 0x7b, 0x03, 0x8f, - 0xaa, 0xb5, 0x79, 0x20, 0xe8, 0xfc, 0x0d, 0xfa, 0x8a, 0xb7, 0x86, 0xbd, 0x22, 0x32, 0xd8, 0xb7, - 0x3a, 0x21, 0xcf, 0xa9, 0x83, 0xa9, 0x41, 0x35, 0x6c, 0x1e, 0xbf, 0xc8, 0xbb, 0xdb, 0x5b, 0xa3, - 0xab, 0x11, 0x15, 0xf7, 0xe9, 0x57, 0xa3, 0x27, 0x91, 0x34, 0xa0, 0x7f, 0x7b, 0x03, 0xed, 0x4c, - 0x33, 0x43, 0x1b, 0x30, 0x22, 0xc8, 0x91, 0x34, 0xe0, 0x41, 0x88, 0x8c, 0x8a, 0x90, 0xd8, 0xd7, - 0xe8, 0xa9, 0x5f, 0xb4, 0xe9, 0xe4, 0x07, 0xf2, 0x55, 0x27, 0xe4, 0xc4, 0xbe, 0x86, 0x3e, 0x86, - 0x34, 0x8b, 0xca, 0x3a, 0x36, 0xe7, 0xf3, 0x43, 0xe7, 0xec, 0xde, 0xd8, 0xb6, 0xea, 0x84, 0x4c, - 0x03, 0xc1, 0x78, 0x47, 0xe6, 0x11, 0x37, 0x54, 0x44, 0x71, 0x44, 0x30, 0x76, 0x5f, 0xdc, 0x13, - 0xe9, 0x30, 0xb6, 0x4f, 0x44, 0xdb, 0x90, 0xe7, 0x4b, 0xb7, 0x17, 0x41, 0x20, 0x0e, 0xf5, 0xd4, - 0x89, 0x0a, 0x22, 0xa8, 0xd2, 0x2d, 0xca, 0x10, 0x9d, 0xb4, 0x5d, 0xaf, 0x44, 0xde, 0x76, 0xb3, - 0x43, 0xdb, 0x6e, 0xa8, 0x43, 0x3b, 0x69, 0x3b, 0x7b, 0x20, 0x11, 0xbd, 0x07, 0x93, 0x6c, 0x9c, - 0x20, 0x2a, 0x32, 0xca, 0xf9, 0xac, 0x6f, 0x88, 0xb0, 0xfc, 0x64, 0xf6, 0x72, 0xb9, 0x63, 0xae, - 0xd2, 0xb4, 0xf6, 0xe7, 0xe7, 0x86, 0xce, 0x5e, 0x83, 0x2e, 0xc6, 0x64, 0xf6, 0x72, 0x03, 0x2a, - 0xe9, 0x40, 0x36, 0x4b, 0xe1, 0x43, 0xec, 0xc2, 0xd0, 0x0e, 0x14, 0xe1, 0xaf, 0x5b, 0xa5, 0x21, - 0x53, 0x01, 0xd9, 0x9f, 0x58, 0x1d, 0xac, 0xd0, 0x49, 0xf1, 0xe2, 0xe8, 0x89, 0xb5, 0xe7, 0xb6, - 0x32, 0x7f, 0x62, 0x65, 0x54, 0xf4, 0x0c, 0x44, 0x7e, 0x65, 0x8e, 0xe2, 0xf9, 0x8f, 0xcd, 0x5f, - 0xa2, 0xf2, 0xee, 0x46, 0x2e, 0x88, 0x51, 0xae, 0x85, 0x55, 0x82, 0x25, 0x7b, 0x53, 0xd0, 0x27, - 0x30, 0x4b, 0xe5, 0x29, 0x5a, 0x70, 0xcb, 0xd1, 0xfc, 0xfc, 0xc0, 0x9d, 0x39, 0xc3, 0x2f, 0x44, - 0xf2, 0x24, 0x8b, 0x5a, 0x5f, 0x12, 0x19, 0x0f, 0x86, 0x69, 0xb8, 0x74, 0xed, 0x5e, 0x18, 0x3a, - 0x1e, 0x7a, 0x6f, 0x78, 0xad, 0x52, 0x5b, 0x8b, 0x52, 0x48, 0x37, 0xee, 0x9b, 0xf1, 0x5e, 0x1b, - 0xda, 0x8d, 0x87, 0x4c, 0x76, 0x39, 0xb7, 0x67, 0x9e, 0x5b, 0x05, 0x60, 0x3b, 0x08, 0x14, 0x14, - 0x2d, 0x0e, 0x35, 0x00, 0xfa, 0xfd, 0x69, 0x89, 0x01, 0xd0, 0xf4, 0x68, 0xc4, 0x00, 0x60, 0xc7, - 0x5d, 0xf3, 0xd7, 0x86, 0x2f, 0x56, 0xe1, 0xf3, 0x75, 0xba, 0x58, 0x51, 0x02, 0x31, 0xc8, 0x08, - 0x9c, 0xeb, 0xd2, 0x11, 0x7e, 0x7d, 0xe8, 0xee, 0x4d, 0x5f, 0xa0, 0x5d, 0x75, 0x42, 0x4e, 0xbf, - 0xe0, 0x24, 0xd2, 0xab, 0x98, 0x08, 0x3e, 0xb6, 0xef, 0x0d, 0xed, 0x55, 0x83, 0x11, 0x56, 0xa4, - 0x57, 0xbd, 0x08, 0xa8, 0xc1, 0x92, 0xe7, 0xb0, 0x83, 0xac, 0xf9, 0x37, 0x46, 0x2f, 0x79, 0xbd, - 0xc7, 0x6e, 0xfe, 0x92, 0xc7, 0xc9, 0x6c, 0xc9, 0xd3, 0x15, 0xc7, 0xa1, 0x6e, 0x3a, 0xf3, 0x37, - 0x47, 0x2c, 0x79, 0x7d, 0x5b, 0xdb, 0x6c, 0xc9, 0xd3, 0xeb, 0x8c, 0x93, 0x58, 0x7f, 0xb6, 0x77, - 0x45, 0x14, 0x07, 0xf6, 0xb7, 0x87, 0x5a, 0x7f, 0x91, 0x77, 0x58, 0x11, 0xeb, 0xcf, 0xee, 0x49, - 0x40, 0xdf, 0x86, 0x69, 0xbe, 0x95, 0x38, 0x7f, 0x67, 0x84, 0x49, 0x1d, 0xde, 0xfd, 0x25, 0xdd, - 0x91, 0xf3, 0xb0, 0xc9, 0x81, 0x6d, 0x61, 0xb2, 0xc9, 0xef, 0xee, 0x88, 0xc9, 0x61, 0x60, 0x17, - 0x95, 0x4d, 0x0e, 0x01, 0x99, 0x94, 0xc6, 0x61, 0xdb, 0x6f, 0xf3, 0xbf, 0x32, 0xb4, 0x34, 0xbd, - 0xfb, 0x90, 0xa4, 0x34, 0x9c, 0x87, 0x2e, 0x16, 0x74, 0xad, 0x66, 0xda, 0x79, 0x73, 0xf8, 0x62, - 0xd1, 0xbf, 0xa1, 0x53, 0xf5, 0x0e, 0x0a, 0x99, 0x56, 0xfe, 0x8a, 0x00, 0xd7, 0x58, 0x1f, 0xa0, - 0xc7, 0x24, 0x5d, 0xc5, 0x3f, 0xe5, 0x0a, 0xed, 0x56, 0xdd, 0xa7, 0xe2, 0xdf, 0x3b, 0xff, 0xa1, - 0x8c, 0xf7, 0xc6, 0xd7, 0xd5, 0x51, 0xf9, 0x88, 0x32, 0x5a, 0x0c, 0xd7, 0xcf, 0x3f, 0x18, 0xaa, - 0x8c, 0xde, 0xbd, 0x08, 0xa2, 0x0c, 0xce, 0x83, 0x9a, 0x30, 0xcf, 0x86, 0x44, 0x80, 0x7b, 0xfd, - 0xa2, 0x3f, 0x1c, 0xea, 0xf9, 0x3a, 0x12, 0xf1, 0x57, 0x27, 0xe4, 0x4b, 0x2f, 0x22, 0x33, 0x90, - 0xb7, 0xd1, 0x3b, 0x20, 0x8d, 0x00, 0xa6, 0x2a, 0x1e, 0x9e, 0x7c, 0x67, 0xe8, 0xdb, 0x46, 0xc2, - 0x67, 0xf2, 0x36, 0x27, 0x32, 0x03, 0x51, 0xcd, 0x2e, 0x83, 0x86, 0xf3, 0xef, 0x0e, 0x55, 0x4d, - 0x2f, 0x26, 0x25, 0xaa, 0xe1, 0x3c, 0xa5, 0x69, 0xee, 0x72, 0xe1, 0x87, 0x76, 0x17, 0x44, 0x71, - 0x2d, 0x95, 0xbe, 0x2c, 0xce, 0xaf, 0xa5, 0xd2, 0x57, 0xc4, 0x85, 0xb5, 0x54, 0xfa, 0xaa, 0xf8, - 0xda, 0x5a, 0x2a, 0xbd, 0x24, 0x5e, 0x5b, 0x4b, 0xa5, 0x25, 0xf1, 0x86, 0xf4, 0xf3, 0xab, 0x90, - 0xf3, 0x90, 0x25, 0x83, 0x7c, 0x0f, 0xc2, 0x90, 0x6f, 0x71, 0x18, 0xe4, 0xe3, 0x58, 0x94, 0x63, - 0xbe, 0x07, 0x61, 0xcc, 0xb7, 0x38, 0x0c, 0xf3, 0x05, 0x3c, 0x04, 0xf4, 0x35, 0x86, 0x81, 0xbe, - 0xbb, 0x63, 0x80, 0x3e, 0x5f, 0x54, 0x3f, 0xea, 0x5b, 0x1d, 0x44, 0x7d, 0x6f, 0x8c, 0x46, 0x7d, - 0xbe, 0xa8, 0x10, 0xec, 0xfb, 0xb0, 0x0f, 0xf6, 0x5d, 0x1f, 0x01, 0xfb, 0x7c, 0x7e, 0x0f, 0xf7, - 0xad, 0x47, 0xe2, 0xbe, 0x5b, 0x67, 0xe1, 0x3e, 0x5f, 0x4e, 0x0f, 0xf0, 0xab, 0x46, 0x01, 0xbf, - 0x9b, 0x67, 0x00, 0x3f, 0x5f, 0x54, 0x18, 0xf9, 0xad, 0x47, 0x22, 0xbf, 0x5b, 0x67, 0x21, 0xbf, - 0xa0, 0x58, 0x61, 0xe8, 0xf7, 0x6e, 0x0f, 0xf4, 0x5b, 0x1a, 0x0a, 0xfd, 0x7c, 0x6e, 0x86, 0xfd, - 0x3e, 0xea, 0xc7, 0x7e, 0xd7, 0x47, 0x60, 0xbf, 0x40, 0xb1, 0x1c, 0xfc, 0x55, 0xa3, 0xc0, 0xdf, - 0xcd, 0x33, 0xc0, 0x5f, 0xa0, 0x8b, 0x10, 0xfa, 0xdb, 0x8a, 0x46, 0x7f, 0xb7, 0xcf, 0x44, 0x7f, - 0xbe, 0xb4, 0x5e, 0xf8, 0x57, 0x8d, 0x82, 0x7f, 0x37, 0xcf, 0x80, 0x7f, 0x7d, 0x25, 0x63, 0xf8, - 0x4f, 0x1d, 0x89, 0xff, 0xde, 0x1a, 0x13, 0xff, 0xf9, 0xa2, 0xa3, 0x00, 0xa0, 0x3e, 0x1a, 0x00, - 0x16, 0xc7, 0x05, 0x80, 0xfe, 0x4b, 0x22, 0x11, 0xa0, 0x3a, 0x12, 0x01, 0xbe, 0x35, 0x26, 0x02, - 0xec, 0xab, 0x48, 0x2f, 0x04, 0xdc, 0x8a, 0x86, 0x80, 0xb7, 0xcf, 0x84, 0x80, 0x41, 0x2b, 0xf6, - 0x60, 0xc0, 0xe5, 0x10, 0x06, 0x7c, 0x7d, 0x08, 0x06, 0xf4, 0x59, 0x09, 0x08, 0xfc, 0xce, 0x00, - 0x08, 0x94, 0x46, 0x81, 0x40, 0x9f, 0xd7, 0x47, 0x81, 0xd5, 0x28, 0x14, 0x78, 0xf3, 0x0c, 0x14, - 0x18, 0xf4, 0x9b, 0x10, 0x0c, 0x7c, 0x3a, 0x04, 0x06, 0xde, 0x39, 0x1b, 0x06, 0xfa, 0xf2, 0xfa, - 0x70, 0xa0, 0x3a, 0x12, 0x07, 0xbe, 0x35, 0x26, 0x0e, 0x0c, 0x5a, 0x30, 0x02, 0x08, 0xbe, 0xdf, - 0x0b, 0x04, 0xaf, 0x0d, 0x07, 0x82, 0xbe, 0x18, 0x8e, 0x04, 0xd7, 0x23, 0x91, 0xe0, 0xad, 0xb3, - 0x90, 0x60, 0x30, 0x9b, 0x85, 0xa1, 0xe0, 0x56, 0x34, 0x14, 0xbc, 0x7d, 0x26, 0x14, 0x0c, 0x3a, - 0x52, 0x0f, 0x16, 0x5c, 0x8f, 0xc4, 0x82, 0xb7, 0xce, 0xc2, 0x82, 0x7d, 0x53, 0x2d, 0x07, 0x83, - 0xcf, 0x87, 0x82, 0xc1, 0x7b, 0xe3, 0x80, 0x41, 0x5f, 0xe8, 0x00, 0x1a, 0xfc, 0x74, 0x38, 0x1a, - 0xfc, 0x95, 0x73, 0x5c, 0x8f, 0x1b, 0x09, 0x07, 0xbf, 0x33, 0x00, 0x07, 0xa5, 0x51, 0x70, 0x30, - 0x18, 0x19, 0x1e, 0x1e, 0xac, 0x44, 0xa0, 0xb7, 0x37, 0x46, 0xa3, 0xb7, 0x60, 0x21, 0x0f, 0xe0, - 0xdb, 0x87, 0x7d, 0xf0, 0xed, 0xfa, 0x99, 0xde, 0xa0, 0x21, 0xfc, 0x56, 0x1a, 0xc4, 0x6f, 0x37, - 0x46, 0xe2, 0x37, 0x5f, 0x42, 0x00, 0xe0, 0xd6, 0x23, 0x01, 0xdc, 0xad, 0xb3, 0x00, 0x5c, 0xd0, - 0x15, 0xc2, 0x08, 0x6e, 0x2b, 0x1a, 0xc1, 0xdd, 0x3e, 0x13, 0xc1, 0xf5, 0x2d, 0x5b, 0x1e, 0x84, - 0xab, 0x46, 0x41, 0xb8, 0x9b, 0x67, 0x40, 0xb8, 0xf0, 0xb2, 0xe5, 0x63, 0xb8, 0xc6, 0x30, 0x0c, - 0x77, 0x77, 0x0c, 0x0c, 0x17, 0x18, 0x73, 0x7d, 0x20, 0xee, 0xe3, 0x7e, 0x10, 0x27, 0x8d, 0x02, - 0x71, 0x41, 0x27, 0xf2, 0x50, 0xdc, 0x56, 0x34, 0x8a, 0xbb, 0x7d, 0x26, 0x8a, 0x0b, 0x8f, 0xeb, - 0x10, 0x8c, 0xfb, 0xb8, 0x1f, 0xc6, 0x49, 0xa3, 0x60, 0x5c, 0x50, 0x1e, 0x0f, 0xc7, 0x55, 0xa3, - 0x70, 0xdc, 0xcd, 0x33, 0x70, 0x5c, 0x68, 0xba, 0x0f, 0x80, 0xdc, 0x5f, 0x1d, 0x1f, 0xc8, 0xbd, - 0xff, 0xaa, 0xde, 0x75, 0x67, 0x23, 0xb9, 0x8f, 0xfb, 0x91, 0x9c, 0x34, 0x0a, 0xc9, 0x05, 0xfa, - 0xf0, 0xa0, 0x5c, 0xeb, 0x4c, 0x28, 0x77, 0xff, 0x1c, 0x50, 0xce, 0x97, 0x3f, 0x0c, 0xcb, 0xb5, - 0xce, 0xc4, 0x72, 0xf7, 0xcf, 0x81, 0xe5, 0x82, 0xd7, 0x0d, 0x01, 0x73, 0x1f, 0xf7, 0x83, 0x39, - 0x69, 0x14, 0x98, 0x0b, 0xb4, 0x73, 0x6e, 0x34, 0xf7, 0x9a, 0xf8, 0x7a, 0x0f, 0xa6, 0xfb, 0x69, - 0x06, 0xa6, 0xaa, 0x11, 0x67, 0x82, 0xc2, 0xab, 0x9c, 0x09, 0xa2, 0x6f, 0xc3, 0x55, 0xff, 0x81, - 0xfa, 0x73, 0x29, 0x3c, 0x24, 0x4e, 0x6b, 0x5a, 0xda, 0x21, 0x5d, 0x2a, 0xd3, 0xf2, 0xbc, 0x9f, - 0xe5, 0x91, 0x6d, 0xb5, 0x58, 0x68, 0x1c, 0xf5, 0x92, 0x41, 0xab, 0x64, 0x0c, 0x53, 0x9b, 0xf0, - 0xec, 0x1b, 0x47, 0x07, 0xef, 0xc4, 0xe4, 0xac, 0xaf, 0x70, 0x89, 0x03, 0x7a, 0x17, 0x72, 0x1d, - 0x07, 0xdb, 0x4a, 0xdb, 0x36, 0x2c, 0xdb, 0x70, 0x59, 0x38, 0x9a, 0x50, 0x12, 0xbf, 0x3c, 0x59, - 0x9a, 0xd9, 0x71, 0xb0, 0xbd, 0xcd, 0xe9, 0xf2, 0x4c, 0x27, 0xf4, 0xe4, 0x7d, 0xd7, 0x6f, 0x72, - 0xfc, 0xef, 0xfa, 0x3d, 0x05, 0x91, 0x7a, 0xe8, 0x84, 0x97, 0x51, 0x76, 0x4f, 0x5b, 0xf4, 0x8a, - 0xaf, 0xea, 0xa1, 0x95, 0x92, 0xde, 0xd7, 0x56, 0xb0, 0x7b, 0x89, 0xe8, 0x31, 0xe4, 0x6d, 0xab, - 0x43, 0xaf, 0x63, 0x6a, 0x5b, 0x4d, 0x43, 0xeb, 0x52, 0xfb, 0x26, 0x1f, 0x7d, 0xde, 0xcc, 0x32, - 0x6e, 0xd3, 0x7c, 0x72, 0xce, 0x0e, 0x3f, 0xa2, 0x3a, 0xd0, 0xab, 0x98, 0x3c, 0x29, 0x68, 0xe0, - 0xf2, 0xfb, 0x91, 0x9f, 0xa0, 0x78, 0xae, 0x1a, 0x2e, 0x97, 0x0b, 0x2f, 0xfd, 0xdf, 0xe8, 0x11, - 0xcc, 0xd0, 0x6b, 0xc1, 0x49, 0x9b, 0x5b, 0x1d, 0x97, 0x9b, 0x37, 0x57, 0x8a, 0xec, 0xeb, 0x8f, - 0x45, 0xef, 0xeb, 0x8f, 0xc5, 0x55, 0xfe, 0xf5, 0x47, 0x76, 0x56, 0xfd, 0xa3, 0xff, 0xb8, 0x24, - 0xc8, 0x59, 0xcf, 0x73, 0xca, 0xea, 0xb8, 0xe8, 0x3e, 0x5c, 0x6c, 0xa9, 0xc7, 0xf4, 0x76, 0x71, - 0xc5, 0xb3, 0xbf, 0xe8, 0x7d, 0x8a, 0xec, 0xab, 0x82, 0xa8, 0xa5, 0x1e, 0xd3, 0x4f, 0x21, 0xb2, - 0x24, 0xfa, 0xa9, 0xa3, 0xeb, 0x30, 0xc3, 0x63, 0x85, 0xd8, 0x97, 0xd0, 0x0a, 0x34, 0x27, 0xff, - 0x2c, 0x0e, 0xfb, 0x18, 0xda, 0x7b, 0x30, 0x1f, 0xce, 0xa2, 0x30, 0x4f, 0x57, 0xdc, 0x6a, 0xbb, - 0xdd, 0xf9, 0xcb, 0xb4, 0xb3, 0x5e, 0x0c, 0x65, 0x5f, 0x21, 0xa9, 0x15, 0x92, 0x88, 0x6e, 0x42, - 0x5e, 0x37, 0x1c, 0xd7, 0x30, 0x35, 0x97, 0xdf, 0x78, 0xce, 0xee, 0x0c, 0xcf, 0x79, 0x54, 0x76, - 0xad, 0x79, 0x03, 0x66, 0xb5, 0xa6, 0xe1, 0x9b, 0xc3, 0xcc, 0x40, 0x99, 0x1d, 0x3a, 0xc0, 0xcb, - 0x34, 0x6f, 0xbf, 0xe7, 0x4d, 0x41, 0xeb, 0x25, 0xa3, 0x32, 0x14, 0xf6, 0x55, 0x17, 0xbf, 0x54, - 0xbb, 0x8a, 0x17, 0xbf, 0x9b, 0xa5, 0x77, 0x16, 0x5c, 0x3d, 0x3d, 0x59, 0xca, 0x3d, 0x66, 0x49, - 0x03, 0x61, 0xbc, 0xb9, 0xfd, 0x50, 0x82, 0x8e, 0x6e, 0x43, 0x41, 0x75, 0xba, 0xa6, 0x46, 0xbb, - 0x22, 0x36, 0x9d, 0x8e, 0xc3, 0xe3, 0xd8, 0xf2, 0x94, 0x5c, 0xf6, 0xa8, 0xe8, 0x43, 0x58, 0xe0, - 0x1f, 0x36, 0x79, 0xa9, 0xda, 0xba, 0x42, 0xbb, 0x6f, 0x30, 0x4f, 0x88, 0x94, 0xe7, 0x32, 0xfb, - 0x90, 0x09, 0xc9, 0x40, 0xfa, 0x6c, 0x30, 0x8d, 0x3e, 0x83, 0x59, 0x1a, 0xcd, 0x86, 0x75, 0xb2, - 0x8e, 0x35, 0xb1, 0x89, 0x1d, 0x67, 0xc4, 0xf1, 0x44, 0x89, 0xe5, 0xad, 0x7b, 0x59, 0xd9, 0xcc, - 0x24, 0x8b, 0xbb, 0x7d, 0x74, 0xff, 0xa2, 0x75, 0x10, 0xb3, 0x6b, 0xa9, 0xf4, 0x8c, 0x98, 0x5b, - 0x4b, 0xa5, 0xf3, 0x62, 0x41, 0xfa, 0xbf, 0x02, 0x5c, 0x8a, 0x16, 0x82, 0xea, 0x30, 0xd7, 0x73, - 0x97, 0x10, 0x0b, 0xb3, 0x3b, 0xcf, 0x44, 0x37, 0x1b, 0xbe, 0x51, 0x88, 0xbe, 0x85, 0x28, 0x27, - 0x42, 0xa8, 0xe2, 0xb8, 0xb6, 0xa1, 0x79, 0xf7, 0x73, 0x5d, 0x1e, 0x60, 0xab, 0xd3, 0x64, 0x5a, - 0x22, 0xf5, 0x78, 0xa0, 0x44, 0xc9, 0xf3, 0x94, 0x48, 0x3d, 0xee, 0x15, 0x2d, 0x7d, 0x3f, 0x09, - 0x05, 0xb2, 0x64, 0x3b, 0x8e, 0x61, 0x99, 0x55, 0xdf, 0xb3, 0xdf, 0x9f, 0xd9, 0x04, 0x1a, 0xa4, - 0xea, 0x3f, 0xa3, 0x25, 0x1a, 0x0c, 0x4c, 0xa0, 0x91, 0xff, 0xa1, 0xa8, 0xa4, 0x0c, 0x8c, 0x44, - 0xc3, 0x1d, 0x57, 0x60, 0xca, 0xb1, 0x3a, 0xb6, 0xe6, 0x7d, 0xbe, 0xe4, 0xee, 0x10, 0x1b, 0x21, - 0xf4, 0xc2, 0x62, 0x9d, 0x32, 0xc8, 0x9c, 0x11, 0x7d, 0x0a, 0x05, 0xf6, 0x8b, 0x06, 0x21, 0xd2, - 0x20, 0x40, 0x16, 0xc9, 0x79, 0x7f, 0x6c, 0x59, 0x1b, 0x9c, 0x51, 0xce, 0x3b, 0x3d, 0xcf, 0xe8, - 0x63, 0x78, 0xcd, 0xb4, 0x94, 0x16, 0x6e, 0x59, 0xcc, 0x36, 0x20, 0xd3, 0x93, 0xae, 0xa8, 0xae, - 0xc2, 0x0b, 0xcd, 0x5c, 0xc5, 0xe7, 0x4d, 0x6b, 0x93, 0x66, 0x91, 0x79, 0x8e, 0x15, 0x97, 0xc9, - 0x95, 0x8a, 0x30, 0xc5, 0x7e, 0xa1, 0x0c, 0x4c, 0x3e, 0x69, 0x54, 0x2b, 0xb2, 0x38, 0x81, 0x66, - 0x20, 0xfd, 0x48, 0x7e, 0xb2, 0xa9, 0xd4, 0x9f, 0x6e, 0x88, 0x02, 0xca, 0xc2, 0xb4, 0xfc, 0xe4, - 0x49, 0x43, 0x59, 0x7f, 0x26, 0x26, 0xa4, 0xdb, 0x90, 0xef, 0x2d, 0x11, 0x02, 0x98, 0x92, 0x2b, - 0x9b, 0x4f, 0xe8, 0x37, 0x3b, 0x32, 0x30, 0xb9, 0xf1, 0xa4, 0xbc, 0xb2, 0x21, 0x0a, 0xd2, 0x2f, - 0x05, 0x98, 0x29, 0xb1, 0x8f, 0xd0, 0x30, 0x87, 0xa0, 0x0f, 0xfb, 0x1c, 0x77, 0xae, 0x44, 0xef, - 0x3d, 0x0c, 0x73, 0xd8, 0x49, 0xf3, 0x69, 0xcf, 0x0b, 0x56, 0x5b, 0x1a, 0x8e, 0x38, 0xe9, 0xe6, - 0xac, 0xe7, 0x21, 0xec, 0xb1, 0xa1, 0x3a, 0x88, 0xaa, 0xa7, 0x5b, 0x85, 0x97, 0x64, 0xb8, 0x9f, - 0x70, 0x5f, 0x33, 0x78, 0x73, 0x91, 0xda, 0x4b, 0xfe, 0x20, 0xf5, 0xa3, 0xcf, 0x97, 0x26, 0xa4, - 0x3f, 0x4d, 0x41, 0xae, 0x14, 0xfe, 0xe0, 0x0e, 0xaa, 0xf5, 0x55, 0xf6, 0x76, 0xa4, 0x3d, 0x13, - 0xe2, 0x28, 0x8e, 0xf8, 0x94, 0x59, 0x26, 0xf8, 0xba, 0x0f, 0xab, 0xfb, 0xb5, 0x11, 0x3e, 0x4f, - 0xe1, 0xca, 0x07, 0x8c, 0x0b, 0xff, 0x36, 0xe9, 0x1b, 0x3a, 0x45, 0x98, 0x64, 0xc1, 0xd8, 0xc2, - 0xc0, 0x3d, 0x31, 0x74, 0x89, 0x23, 0x38, 0x8f, 0xa4, 0xcb, 0x2c, 0x1b, 0x31, 0x8c, 0x1a, 0xaf, - 0xe4, 0x2c, 0x15, 0xcc, 0x83, 0xe7, 0xff, 0x02, 0x70, 0x87, 0xdd, 0x8a, 0xfc, 0xff, 0xd1, 0x35, - 0x99, 0xbc, 0x0f, 0xfd, 0x3a, 0x14, 0x34, 0xab, 0xd9, 0x64, 0xf0, 0x80, 0xad, 0x6c, 0x83, 0xf7, - 0xa4, 0xd1, 0x22, 0xf0, 0x8f, 0x3e, 0x17, 0xfd, 0x8f, 0x3f, 0x17, 0x65, 0xfe, 0xf1, 0xe7, 0x50, - 0xdc, 0x58, 0xde, 0x17, 0xc6, 0x16, 0xc4, 0xbe, 0x10, 0xb6, 0xe9, 0x57, 0x09, 0x61, 0x63, 0x81, - 0x7f, 0xbc, 0xe7, 0xfd, 0x5c, 0xe0, 0x0e, 0xc4, 0x1b, 0x96, 0x75, 0xd8, 0xf1, 0xdd, 0x32, 0x17, - 0xc2, 0x77, 0x1c, 0x07, 0xd1, 0x35, 0x34, 0xd8, 0x34, 0xca, 0x06, 0x4b, 0x7c, 0x35, 0x1b, 0xec, - 0x3a, 0xcc, 0xb4, 0x6d, 0xbc, 0x87, 0x5d, 0xed, 0x40, 0x31, 0x3b, 0x2d, 0x1e, 0x69, 0x9b, 0xf5, - 0x68, 0x5b, 0x9d, 0x16, 0xba, 0x0b, 0xa2, 0x9f, 0x85, 0x6f, 0xd9, 0x78, 0x17, 0x6c, 0x7a, 0x74, - 0xbe, 0xc1, 0x23, 0xfd, 0x0f, 0x01, 0xe6, 0x7a, 0xea, 0xc4, 0xc7, 0xd4, 0x1a, 0x64, 0x75, 0xdf, - 0xea, 0x75, 0xe6, 0x85, 0x73, 0xc6, 0x5d, 0x85, 0x99, 0x91, 0x02, 0x97, 0xbc, 0xd7, 0xd2, 0x2f, - 0xe2, 0x04, 0x62, 0x13, 0xe7, 0x14, 0x7b, 0x31, 0x90, 0xb3, 0x1a, 0x7a, 0x81, 0x3f, 0xc8, 0x92, - 0x63, 0x0d, 0x32, 0xe9, 0x7f, 0x09, 0x20, 0xd2, 0x17, 0x3c, 0xc2, 0x58, 0x8f, 0x65, 0xca, 0xf4, - 0x02, 0x1c, 0x13, 0xe3, 0xc7, 0xd2, 0xf6, 0x7c, 0xc5, 0x2b, 0xd9, 0xf7, 0x15, 0xaf, 0xa8, 0xf9, - 0x33, 0xf5, 0x15, 0xe7, 0x4f, 0xe9, 0x73, 0x01, 0xf2, 0x7e, 0xb5, 0xd9, 0xe7, 0x7b, 0x47, 0xdc, - 0xcf, 0xfd, 0x6a, 0x9f, 0xa8, 0xf5, 0xee, 0x11, 0x1b, 0xeb, 0x8b, 0xc2, 0xe1, 0x7b, 0xc4, 0xd8, - 0xa7, 0x55, 0xff, 0x96, 0xd7, 0x1d, 0x49, 0x11, 0xcb, 0xc1, 0x05, 0x4e, 0xaf, 0x10, 0xab, 0xfc, - 0x75, 0x78, 0x20, 0x3f, 0x0a, 0x29, 0x90, 0xf6, 0x28, 0xa2, 0xa5, 0xb1, 0xe6, 0x77, 0x4f, 0x4b, - 0xac, 0x03, 0xfe, 0x34, 0xdc, 0x12, 0xec, 0x02, 0x90, 0x87, 0x90, 0x3c, 0x52, 0x9b, 0xa3, 0xfc, - 0x6c, 0x7b, 0x5a, 0x4e, 0x26, 0xb9, 0xd1, 0xa3, 0x9e, 0x7b, 0xaf, 0x12, 0xc3, 0xb7, 0x73, 0x07, - 0x55, 0xda, 0x73, 0x3f, 0xd6, 0x7b, 0xbd, 0x03, 0x68, 0xe4, 0xeb, 0xc3, 0x23, 0xe9, 0x83, 0xd4, - 0x17, 0x9f, 0x2f, 0x09, 0xd2, 0x47, 0x80, 0x88, 0xad, 0xe3, 0x3e, 0xed, 0x58, 0x76, 0x70, 0x87, - 0x58, 0x7f, 0x20, 0xe3, 0x64, 0x74, 0x20, 0xa3, 0x74, 0x11, 0xe6, 0x7a, 0xb8, 0xd9, 0x0c, 0x24, - 0xbd, 0x07, 0x57, 0x1e, 0x5b, 0x8e, 0x63, 0xb4, 0xeb, 0x9d, 0x5d, 0x36, 0xd4, 0xc9, 0x7a, 0xe5, - 0xcf, 0xb9, 0xe9, 0x36, 0xdd, 0x2b, 0x34, 0xd9, 0xdc, 0x94, 0x91, 0xfd, 0x67, 0xe9, 0x0f, 0x05, - 0xb8, 0x3c, 0xc8, 0xc9, 0xb4, 0x1c, 0x75, 0xb5, 0xc2, 0xb4, 0x66, 0x05, 0x57, 0xdc, 0x9e, 0xdd, - 0x5b, 0xbd, 0xec, 0x04, 0xd5, 0xf0, 0x77, 0x2a, 0x2d, 0x95, 0xce, 0x49, 0xfc, 0xc6, 0x95, 0x3c, - 0x27, 0x6f, 0x32, 0x6a, 0x30, 0x3d, 0xa5, 0xc6, 0x9b, 0x9e, 0x7e, 0x92, 0x80, 0xd9, 0x06, 0x36, - 0x55, 0xd3, 0x25, 0xf3, 0x7e, 0xa7, 0xc5, 0xae, 0xa3, 0x28, 0x40, 0xd2, 0x56, 0x3a, 0xb4, 0xe8, - 0x82, 0x9c, 0xb0, 0x77, 0xd0, 0x0d, 0xc8, 0xd1, 0xb5, 0x25, 0x64, 0xab, 0x09, 0x77, 0x52, 0x32, - 0x0d, 0xcb, 0x91, 0x3d, 0x43, 0xec, 0x75, 0x00, 0x9a, 0x89, 0xc1, 0xd2, 0x24, 0xcd, 0x91, 0x21, - 0x14, 0x06, 0x4a, 0x6f, 0x42, 0x9e, 0x86, 0x39, 0x06, 0x42, 0x58, 0x24, 0x53, 0x8e, 0x52, 0x7d, - 0x29, 0x4b, 0x90, 0x65, 0xd9, 0x98, 0x98, 0x49, 0x9a, 0x07, 0x28, 0x89, 0xc9, 0x79, 0x04, 0x17, - 0x9c, 0x17, 0x4d, 0xa5, 0x6d, 0xe9, 0x8e, 0xa2, 0xb5, 0x3b, 0x3c, 0x44, 0x84, 0x7d, 0x11, 0x5c, - 0x28, 0x5d, 0x3c, 0x3d, 0x59, 0x9a, 0xad, 0x3f, 0xdd, 0xd8, 0xb6, 0x74, 0xa7, 0xbc, 0xbd, 0xc3, - 0x02, 0x44, 0x1c, 0x79, 0xd6, 0x79, 0xd1, 0xa4, 0xa4, 0x76, 0x87, 0x93, 0x50, 0x05, 0xe6, 0xda, - 0xfb, 0x2f, 0x0d, 0x1b, 0x2b, 0x78, 0xdf, 0xc6, 0x8e, 0xc3, 0x5f, 0x38, 0x4d, 0x5e, 0xc8, 0xc4, - 0x6c, 0x3f, 0x7e, 0x6e, 0xd8, 0xb8, 0x42, 0x53, 0xe9, 0xbb, 0xe5, 0x59, 0xc6, 0x11, 0x22, 0x49, - 0xbf, 0x4c, 0x02, 0xa2, 0x37, 0x33, 0x95, 0xe8, 0xd5, 0x46, 0x5e, 0xaf, 0xb1, 0x60, 0x51, 0x0b, - 0x34, 0xaa, 0x38, 0x86, 0x49, 0x60, 0x82, 0xea, 0xb8, 0x5e, 0xf5, 0xf9, 0xf0, 0x8b, 0xbc, 0x50, - 0xb6, 0xbf, 0x41, 0x78, 0x77, 0xb8, 0x1a, 0x92, 0x58, 0x27, 0x02, 0x37, 0x54, 0xc7, 0x7f, 0xe1, - 0x5d, 0xc8, 0xb8, 0x94, 0xcf, 0xbb, 0xf7, 0x2a, 0x55, 0x9a, 0x39, 0x3d, 0x59, 0x4a, 0x33, 0x61, - 0xb5, 0x55, 0x39, 0xcd, 0x92, 0x6b, 0x3a, 0x5a, 0x86, 0xac, 0x61, 0x3a, 0xae, 0x4a, 0x8a, 0xc4, - 0x37, 0x93, 0x72, 0xec, 0x62, 0x82, 0x1a, 0x27, 0xd7, 0x56, 0x65, 0xf0, 0xb2, 0xd4, 0x74, 0x54, - 0x83, 0x8b, 0x26, 0x3e, 0x76, 0x15, 0x1a, 0xfb, 0x1d, 0x66, 0x4d, 0x53, 0xd6, 0x4b, 0xa7, 0x27, - 0x4b, 0x68, 0x0b, 0x1f, 0xbb, 0x1b, 0xc6, 0x11, 0x0e, 0x89, 0x40, 0x66, 0x3f, 0x4d, 0x27, 0xbd, - 0xc0, 0x17, 0xc0, 0x0e, 0x9a, 0x68, 0xdc, 0xbb, 0x9c, 0xf3, 0xa8, 0xec, 0xf8, 0xe8, 0x32, 0x4c, - 0x3b, 0xf8, 0x05, 0x35, 0x3a, 0xa6, 0xa9, 0xd1, 0x31, 0xe5, 0xd0, 0x0b, 0x69, 0x91, 0xe4, 0x9f, - 0x53, 0x61, 0x5d, 0x21, 0x9d, 0x94, 0xc5, 0x13, 0x65, 0x7d, 0xa2, 0xbc, 0x83, 0x9e, 0x03, 0xdf, - 0xde, 0xf0, 0xb7, 0x54, 0xda, 0xd8, 0x36, 0x2c, 0xef, 0x92, 0x87, 0xb1, 0x76, 0x69, 0xe6, 0x98, - 0x04, 0xae, 0xdd, 0x6d, 0xca, 0x2f, 0xfd, 0x89, 0x00, 0x73, 0x3d, 0x6d, 0xcd, 0x2d, 0x98, 0x8f, - 0x7a, 0x67, 0xe6, 0xb0, 0x19, 0x4f, 0xe9, 0x0e, 0x3d, 0xd2, 0xd7, 0x2c, 0x9d, 0x4f, 0x69, 0x3d, - 0x33, 0x34, 0x5a, 0x84, 0xec, 0xbe, 0xad, 0x9a, 0x5e, 0x85, 0x12, 0xb4, 0x42, 0x19, 0x4e, 0x92, - 0x77, 0xd0, 0x16, 0x88, 0x04, 0x58, 0x1f, 0x36, 0xb1, 0xa2, 0xf3, 0x82, 0xf2, 0xc9, 0x73, 0xac, - 0x9a, 0x14, 0x38, 0xb3, 0x97, 0x44, 0x06, 0xf3, 0x9e, 0xda, 0x6c, 0xee, 0xaa, 0xda, 0xa1, 0x42, - 0x37, 0xae, 0xe9, 0xae, 0xa0, 0x3c, 0xe3, 0x11, 0x65, 0xd5, 0xc5, 0x52, 0x03, 0x0a, 0x6b, 0x96, - 0x61, 0x6e, 0x59, 0xba, 0x1f, 0x0c, 0xb5, 0x02, 0xf9, 0x5d, 0xc3, 0x54, 0xed, 0xae, 0xe2, 0xc5, - 0x33, 0x09, 0x67, 0xc5, 0x33, 0xc9, 0x39, 0xc6, 0xc1, 0x1f, 0xa5, 0x9f, 0x09, 0x20, 0x06, 0x62, - 0xb9, 0xf6, 0xde, 0x04, 0xd0, 0x9a, 0x1d, 0xc7, 0xc5, 0xb6, 0x37, 0x7d, 0xcf, 0xb0, 0xb8, 0xe9, - 0x32, 0xa3, 0xd6, 0x56, 0xe5, 0x0c, 0xcf, 0x50, 0xd3, 0xd1, 0x8d, 0xde, 0xdb, 0xdd, 0x26, 0x4b, - 0x70, 0x3a, 0x70, 0xa7, 0x1b, 0x59, 0x0f, 0x1c, 0xd7, 0xb2, 0xfd, 0xee, 0xcd, 0xd7, 0x03, 0xef, - 0xde, 0x4b, 0x7a, 0xbf, 0x13, 0xa6, 0x57, 0x23, 0xe4, 0x09, 0x38, 0x39, 0xc2, 0x7e, 0x95, 0x52, - 0x67, 0x57, 0x89, 0x71, 0x78, 0x55, 0xfa, 0x03, 0x01, 0x0a, 0x65, 0x36, 0x4d, 0xfb, 0x53, 0xff, - 0x08, 0x53, 0x67, 0x15, 0xd2, 0xee, 0xb1, 0xa9, 0xb4, 0xb0, 0xff, 0xa9, 0xc6, 0x73, 0x5c, 0x4c, - 0x3d, 0xed, 0xb2, 0x47, 0xfa, 0xf5, 0xef, 0x57, 0xe8, 0x0a, 0x3e, 0x13, 0x47, 0x19, 0xff, 0x46, - 0x80, 0x4c, 0x5d, 0x53, 0x4d, 0x3f, 0xde, 0xd4, 0xec, 0xb4, 0xd8, 0xd1, 0xc0, 0x9e, 0xaa, 0x61, - 0xc5, 0xc1, 0xf8, 0xd0, 0xe1, 0xf7, 0xd9, 0xcc, 0x9a, 0x9d, 0x56, 0xcd, 0x4b, 0xa9, 0x93, 0x04, - 0xf4, 0x26, 0x20, 0x3f, 0xbf, 0xa9, 0x36, 0x79, 0x76, 0xb6, 0x32, 0x88, 0x5e, 0x76, 0x53, 0x6d, - 0xb2, 0xdc, 0x83, 0xd2, 0x5d, 0xdc, 0xf6, 0x96, 0x89, 0x5e, 0xe9, 0x24, 0x61, 0x50, 0x3a, 0xcd, - 0x9e, 0x1a, 0x94, 0x4e, 0xe8, 0xac, 0x3e, 0xf7, 0xea, 0x64, 0x79, 0x1f, 0xc0, 0x35, 0x28, 0x0f, - 0x10, 0xfa, 0xa6, 0xe8, 0x04, 0xfd, 0xd2, 0x68, 0x65, 0x65, 0x55, 0xd9, 0xd9, 0x2a, 0x3f, 0xd9, - 0xdc, 0xac, 0x35, 0x1a, 0x95, 0x55, 0x51, 0x40, 0x22, 0xcc, 0xf4, 0x7c, 0x91, 0x34, 0xb1, 0x90, - 0xfa, 0xc1, 0xef, 0x2d, 0x4e, 0xdc, 0x7b, 0x0b, 0x72, 0x3d, 0xfb, 0xcb, 0xa8, 0x00, 0xd9, 0x8d, - 0xca, 0x4a, 0xbd, 0x52, 0x7d, 0xb2, 0xb1, 0x4a, 0xb7, 0x55, 0xb2, 0x30, 0xbd, 0x55, 0x59, 0x91, - 0x2b, 0xf5, 0x86, 0x28, 0xdc, 0x33, 0x61, 0x86, 0x5d, 0xfb, 0x25, 0xd3, 0x9b, 0x0d, 0x10, 0x82, - 0xbc, 0x5c, 0xa9, 0xef, 0x6c, 0x56, 0x94, 0x9d, 0xad, 0xf5, 0xad, 0x27, 0xcf, 0xb7, 0xbc, 0x02, - 0x50, 0xda, 0x7a, 0xe5, 0x13, 0x65, 0xa3, 0xb6, 0x59, 0x6b, 0x88, 0x02, 0xba, 0x08, 0xb3, 0x9c, - 0x5a, 0xfa, 0xa4, 0x51, 0xe1, 0xe4, 0x04, 0xfd, 0x90, 0x2a, 0x23, 0xd7, 0xb6, 0xe8, 0x87, 0x4d, - 0x59, 0x42, 0x92, 0x17, 0xef, 0x2f, 0x00, 0x04, 0xdf, 0x5e, 0x26, 0x55, 0x25, 0x22, 0x9f, 0xad, - 0x6c, 0xec, 0x54, 0xea, 0xe2, 0x04, 0x79, 0x7b, 0x69, 0xa5, 0x51, 0xae, 0x2a, 0x72, 0xa5, 0xbe, - 0xfd, 0x64, 0xab, 0x5e, 0x11, 0x05, 0xce, 0xb7, 0x0a, 0x33, 0xe1, 0xdb, 0x4d, 0xd1, 0x1c, 0x14, - 0xca, 0xd5, 0x4a, 0x79, 0x5d, 0x79, 0x56, 0x5b, 0x51, 0x9e, 0xee, 0x54, 0x76, 0x2a, 0xe2, 0x04, - 0xd5, 0x1c, 0x25, 0x3e, 0xda, 0xd9, 0xd8, 0x10, 0x05, 0x52, 0x75, 0xf6, 0x4c, 0x3f, 0xae, 0x2a, - 0x26, 0xee, 0x6d, 0x42, 0x36, 0xf4, 0xd5, 0x15, 0xf2, 0xba, 0xed, 0x9d, 0x7a, 0x55, 0x69, 0xd4, - 0x36, 0x2b, 0xf5, 0xc6, 0xca, 0xe6, 0x36, 0x93, 0x41, 0x69, 0x2b, 0xa5, 0x27, 0x32, 0xa9, 0xa6, - 0xf7, 0xdc, 0x78, 0xb2, 0x53, 0xae, 0x7a, 0x5a, 0x96, 0x52, 0xe9, 0xa4, 0x98, 0xbc, 0x77, 0x0c, - 0x97, 0x87, 0x5c, 0xf4, 0x49, 0x94, 0xbc, 0x63, 0xd2, 0x2f, 0x50, 0x88, 0x13, 0x28, 0x07, 0x19, - 0x32, 0xd2, 0xe9, 0xdd, 0x39, 0xa2, 0x80, 0xd2, 0x90, 0x3a, 0x70, 0xdd, 0xb6, 0x98, 0x40, 0x53, - 0x90, 0x70, 0x1e, 0x8a, 0x49, 0xf2, 0x7f, 0xdf, 0x11, 0x53, 0x28, 0x03, 0x93, 0xea, 0x67, 0x1d, - 0x1b, 0x8b, 0x93, 0x68, 0x06, 0xd2, 0x1d, 0x07, 0xdb, 0x7b, 0x46, 0x13, 0x8b, 0xd3, 0x84, 0xc5, - 0xec, 0x34, 0x9b, 0x62, 0x5a, 0x4a, 0xa5, 0xa7, 0xc4, 0xa9, 0x7b, 0xd7, 0x21, 0x74, 0xb7, 0x1a, - 0x02, 0x98, 0xda, 0x50, 0x5d, 0xec, 0xb8, 0xe2, 0x04, 0x9a, 0x86, 0xe4, 0x4a, 0xb3, 0x29, 0x0a, - 0x0f, 0xfe, 0xe7, 0x14, 0xa4, 0xbd, 0x5e, 0x87, 0x36, 0x60, 0x92, 0xee, 0xf3, 0xa0, 0xa5, 0xe1, - 0x3b, 0x40, 0x74, 0xda, 0x5c, 0xb8, 0x76, 0xd6, 0x16, 0x91, 0x34, 0x81, 0xfe, 0x22, 0x64, 0x43, - 0xc8, 0x18, 0x0d, 0x3d, 0x0f, 0xed, 0xd9, 0x0d, 0x58, 0xb8, 0x75, 0x56, 0x36, 0x5f, 0xfe, 0x73, - 0xc8, 0xf8, 0x46, 0x35, 0xba, 0x31, 0xca, 0xe4, 0xf6, 0x64, 0x8f, 0xb6, 0xcb, 0xc9, 0x2c, 0x27, - 0x4d, 0xbc, 0x2d, 0x20, 0x1b, 0xd0, 0xa0, 0xfd, 0x8b, 0xa2, 0x02, 0x24, 0x86, 0x1a, 0xd8, 0x0b, - 0xf7, 0xc6, 0xca, 0x1d, 0xbc, 0x93, 0x28, 0x2b, 0x30, 0xe2, 0xa3, 0x95, 0x35, 0x00, 0x11, 0xa2, - 0x95, 0x15, 0x81, 0x05, 0x68, 0x63, 0x84, 0x16, 0xf9, 0x48, 0xf9, 0x83, 0x06, 0x5f, 0xa4, 0xfc, - 0x08, 0x5b, 0x41, 0x9a, 0x40, 0x4f, 0x21, 0x45, 0xd6, 0x40, 0x14, 0x05, 0xaf, 0xfb, 0xd6, 0xdc, - 0x85, 0x1b, 0x23, 0xf3, 0xf8, 0x22, 0xf7, 0x21, 0xff, 0x18, 0xd3, 0xc3, 0x19, 0x76, 0x51, 0xaa, - 0x83, 0xee, 0x44, 0x7b, 0xad, 0x86, 0xb2, 0x78, 0xaf, 0xb8, 0x3b, 0x46, 0x4e, 0xff, 0x45, 0x6d, - 0x98, 0x65, 0x9f, 0x98, 0x09, 0xbf, 0x2b, 0xca, 0x55, 0x66, 0x20, 0x97, 0xf7, 0xba, 0x37, 0xc7, - 0xcb, 0xec, 0xbd, 0xb1, 0x74, 0xf7, 0x8b, 0xff, 0xb4, 0x38, 0xf1, 0xc5, 0xe9, 0xa2, 0xf0, 0xb3, - 0xd3, 0x45, 0xe1, 0x8f, 0x4e, 0x17, 0x85, 0x3f, 0x3e, 0x5d, 0x14, 0x7e, 0xf8, 0x8b, 0xc5, 0x89, - 0x9f, 0xfd, 0x62, 0x71, 0xe2, 0x8f, 0x7e, 0xb1, 0x38, 0xf1, 0xe9, 0x34, 0x17, 0xb3, 0x3b, 0x45, - 0xd7, 0xbe, 0x87, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x15, 0x13, 0x47, 0x41, 0x0d, 0x8f, 0x00, - 0x00, + 0x26, 0xef, 0xbe, 0x1a, 0x8a, 0x50, 0x55, 0xef, 0x44, 0x8c, 0x22, 0x54, 0xf2, 0x24, 0xfd, 0xe7, + 0x04, 0xcc, 0xae, 0xe8, 0x7a, 0xbd, 0x4e, 0x51, 0x4d, 0x9c, 0x63, 0x0c, 0x41, 0x8a, 0x98, 0x1b, + 0x3c, 0x1c, 0x8a, 0xfe, 0x46, 0x6f, 0x01, 0xd2, 0x0d, 0x87, 0xdd, 0x0c, 0xe1, 0x1c, 0xa8, 0xba, + 0xf5, 0x32, 0x38, 0xf8, 0x9e, 0xf5, 0x52, 0xea, 0x5e, 0x02, 0xaa, 0x03, 0xb5, 0xb5, 0x15, 0xc7, + 0x55, 0xfd, 0x8d, 0xfd, 0x9b, 0x63, 0x05, 0xd8, 0x30, 0xe3, 0xd8, 0x7f, 0x94, 0x33, 0x44, 0x0e, + 0xfd, 0x49, 0x4c, 0x3e, 0x83, 0x34, 0x8a, 0xab, 0xa8, 0x8e, 0x17, 0x1a, 0xc2, 0xee, 0xa4, 0xc8, + 0x33, 0xfa, 0x8a, 0xc3, 0x23, 0x3e, 0x3e, 0x84, 0x05, 0x9a, 0xae, 0xa8, 0xae, 0x62, 0xb3, 0xda, + 0x86, 0x8e, 0x0e, 0xd9, 0x0e, 0xe4, 0x65, 0x9a, 0x63, 0xc5, 0xb3, 0x70, 0x83, 0x03, 0x42, 0xff, + 0x1e, 0x87, 0xe9, 0xf0, 0xf5, 0x23, 0xd4, 0x49, 0x3d, 0xd0, 0x76, 0x9c, 0x3b, 0xbd, 0x7f, 0x53, + 0x80, 0xbc, 0x8c, 0xf7, 0x6c, 0xec, 0x1c, 0xc4, 0xd9, 0x8c, 0x8f, 0x60, 0xc6, 0x66, 0x52, 0x95, + 0x3d, 0xdb, 0x6a, 0x9d, 0x67, 0xfa, 0xc9, 0x72, 0xc6, 0x47, 0xb6, 0xd5, 0xe2, 0xb3, 0xfc, 0x33, + 0x28, 0xf8, 0x65, 0x8c, 0xb3, 0xf2, 0x7f, 0x9b, 0xc6, 0xa3, 0x32, 0xc1, 0x71, 0x1f, 0x6a, 0xc7, + 0xab, 0x01, 0xba, 0xdb, 0x1f, 0x2e, 0x68, 0x9c, 0x6a, 0xf8, 0x8f, 0x02, 0xe4, 0xeb, 0x9d, 0x5d, + 0x76, 0xf1, 0x52, 0x7c, 0x1a, 0xa8, 0x40, 0xa6, 0x89, 0xf7, 0x5c, 0xe5, 0x95, 0xdc, 0xab, 0xd3, + 0x84, 0x95, 0x3a, 0x97, 0x3f, 0x06, 0xb0, 0x69, 0x40, 0x16, 0x95, 0x93, 0x3c, 0xa7, 0x9c, 0x0c, + 0xe5, 0x25, 0x64, 0xb2, 0x90, 0x15, 0xfc, 0x6a, 0xc6, 0xb9, 0x64, 0x3d, 0xef, 0x99, 0x70, 0x92, + 0xe7, 0x99, 0x70, 0x66, 0xb9, 0x0d, 0x1c, 0x3d, 0xe9, 0x14, 0x61, 0x8e, 0x5a, 0x7a, 0x8a, 0xda, + 0x6e, 0x37, 0x0d, 0x0f, 0x49, 0xd1, 0x29, 0x2d, 0x25, 0xcf, 0xd2, 0xa4, 0x15, 0x96, 0x42, 0x31, + 0x14, 0xfa, 0xbe, 0x00, 0x33, 0x7b, 0x36, 0xc6, 0x9f, 0x61, 0x85, 0xce, 0xf2, 0xe3, 0x39, 0x2a, + 0xac, 0x92, 0x32, 0x7c, 0xe5, 0x83, 0xcc, 0x2c, 0x7b, 0x71, 0x9d, 0xbc, 0x17, 0x6d, 0x81, 0xa8, + 0x35, 0xd9, 0xd1, 0x6a, 0xcf, 0xcc, 0x37, 0xe6, 0x00, 0x28, 0x30, 0xe6, 0x60, 0x5a, 0x7c, 0x4a, + 0x06, 0x93, 0xaa, 0x2b, 0xfc, 0xc2, 0x3a, 0x3a, 0x3b, 0xf6, 0x3a, 0x4d, 0x84, 0x03, 0xd3, 0x43, + 0xf7, 0xdc, 0x15, 0x65, 0xac, 0xea, 0x1c, 0x0c, 0x90, 0x71, 0xe5, 0x3f, 0xf0, 0x71, 0xf5, 0x1c, + 0x66, 0x69, 0xbf, 0x89, 0x3b, 0xbe, 0x54, 0xfa, 0x07, 0x49, 0x40, 0x61, 0xc9, 0x5f, 0x5f, 0x7f, + 0x4b, 0xc4, 0xd7, 0xdf, 0xd6, 0x40, 0x0a, 0xd9, 0x57, 0x4d, 0xd5, 0x71, 0x15, 0xe6, 0x9d, 0xe7, + 0x28, 0x6d, 0x6c, 0x2b, 0x0e, 0xd6, 0x2c, 0x7e, 0xc5, 0x90, 0x20, 0x2f, 0x06, 0x39, 0x37, 0x54, + 0xc7, 0x7d, 0xca, 0xf2, 0x6d, 0x63, 0xbb, 0x4e, 0x73, 0xa1, 0x87, 0x70, 0xa9, 0xa5, 0x1e, 0x47, + 0xf1, 0x4f, 0x52, 0xfe, 0xb9, 0x96, 0x7a, 0x3c, 0xc0, 0xf4, 0x01, 0x2c, 0x44, 0x33, 0x29, 0x0e, + 0xf6, 0x4e, 0xef, 0x2e, 0x45, 0x30, 0xd6, 0xb1, 0x8b, 0x56, 0x00, 0x02, 0x5c, 0xc2, 0x97, 0xfd, + 0x71, 0x60, 0x49, 0xc6, 0x87, 0x25, 0xd2, 0x0f, 0x05, 0xc8, 0x6f, 0x1a, 0xfb, 0xb6, 0x1a, 0xeb, + 0x05, 0x3e, 0xe8, 0x83, 0xde, 0xe3, 0xce, 0xec, 0x83, 0x85, 0x28, 0x77, 0x16, 0x96, 0xc3, 0xdb, + 0xd2, 0xe4, 0x0c, 0x64, 0xe9, 0xf3, 0x4b, 0x14, 0xe7, 0x9c, 0xaf, 0xc1, 0xeb, 0xd4, 0x41, 0x90, + 0x7b, 0x07, 0x7d, 0x2d, 0x80, 0x49, 0xfa, 0x03, 0x01, 0x16, 0x87, 0xbd, 0x25, 0xce, 0x01, 0x21, + 0xd3, 0x7b, 0x0a, 0xe9, 0x1b, 0x14, 0x7f, 0x44, 0x9c, 0x31, 0xd3, 0x20, 0x3e, 0x12, 0xc0, 0x2f, + 0x5b, 0x9d, 0xde, 0x60, 0xc8, 0x7e, 0x3b, 0x44, 0x41, 0x75, 0x4d, 0x35, 0x69, 0xf4, 0x71, 0x13, + 0xab, 0x47, 0x98, 0xfb, 0x36, 0xc6, 0x3a, 0x4b, 0x7c, 0x2e, 0xc0, 0xe2, 0xb0, 0xb7, 0xc4, 0xa9, + 0xa0, 0x6f, 0xc1, 0x34, 0x73, 0x92, 0xf3, 0x70, 0xd2, 0x95, 0x21, 0x81, 0xd6, 0xa6, 0x17, 0xde, + 0xe3, 0xe5, 0x97, 0x1a, 0x90, 0x2f, 0xa9, 0xb6, 0x6d, 0xc4, 0xba, 0x9b, 0x21, 0xfd, 0x58, 0x80, + 0x82, 0x2f, 0x36, 0xce, 0x9a, 0x7e, 0xf5, 0xd8, 0x11, 0xe9, 0xbf, 0x5e, 0x85, 0x19, 0x5e, 0xfe, + 0x1d, 0xd3, 0xb0, 0x4c, 0x74, 0x1f, 0x92, 0xfb, 0xfc, 0x4c, 0x2c, 0x1b, 0xb9, 0xed, 0x1f, 0xdc, + 0x1c, 0x59, 0x9d, 0x90, 0x49, 0x5e, 0xc2, 0xd2, 0xee, 0xb8, 0x11, 0x05, 0x08, 0x82, 0x1f, 0xc2, + 0x2c, 0xed, 0x8e, 0x8b, 0xea, 0x50, 0xd0, 0x82, 0xeb, 0xea, 0x14, 0xc2, 0x9e, 0x1c, 0xba, 0x47, + 0x1d, 0x79, 0x71, 0x60, 0x75, 0x42, 0xce, 0x6b, 0x3d, 0x09, 0xa8, 0x1c, 0xbe, 0x25, 0x2d, 0x35, + 0xe0, 0x59, 0x19, 0x34, 0x7d, 0xef, 0x0d, 0x6d, 0xd5, 0x89, 0xd0, 0x65, 0x6a, 0xe8, 0x03, 0x98, + 0xd2, 0xe9, 0x7d, 0x5c, 0xdc, 0x9e, 0x88, 0x6a, 0xf0, 0x9e, 0x6b, 0xcf, 0xaa, 0x13, 0x32, 0xe7, + 0x40, 0x6b, 0x30, 0xc3, 0x7e, 0x31, 0x00, 0xcd, 0xad, 0x80, 0x9b, 0xc3, 0x25, 0x84, 0xec, 0xf0, + 0xea, 0x84, 0x9c, 0xd5, 0x03, 0x2a, 0x7a, 0x0c, 0x59, 0xad, 0x89, 0x55, 0x9b, 0x8b, 0xba, 0x35, + 0x34, 0x1c, 0x74, 0xe0, 0x0e, 0xaf, 0xea, 0x84, 0x0c, 0x9a, 0x4f, 0x24, 0x85, 0xb2, 0xe9, 0x55, + 0x4e, 0x5c, 0xd2, 0xdb, 0x43, 0x0b, 0x35, 0x78, 0x2f, 0x56, 0x95, 0xda, 0xe7, 0x3e, 0x15, 0xbd, + 0x03, 0x29, 0x47, 0x53, 0x4d, 0x6e, 0x92, 0x2c, 0x0e, 0xb9, 0x6b, 0x27, 0x60, 0xa6, 0xb9, 0xd1, + 0x87, 0x0c, 0x7b, 0xbb, 0xc7, 0xde, 0x41, 0x44, 0x94, 0x4e, 0x7b, 0xee, 0x74, 0x20, 0x3a, 0xc5, + 0x94, 0x40, 0xf4, 0xa0, 0xea, 0x2d, 0xc3, 0x54, 0xe8, 0x59, 0x0f, 0x3d, 0x79, 0x88, 0xd6, 0xc3, + 0x40, 0x60, 0x7c, 0x95, 0x5e, 0x1c, 0xe1, 0x11, 0xd1, 0x26, 0xe4, 0x98, 0xa0, 0x0e, 0x8b, 0xd9, + 0x9e, 0x5f, 0x1e, 0xea, 0xde, 0x10, 0x11, 0x35, 0x5e, 0x9d, 0x90, 0x67, 0xd4, 0x10, 0x39, 0x28, + 0x57, 0x0b, 0xdb, 0xfb, 0xec, 0x88, 0x63, 0x44, 0xb9, 0xc2, 0x3e, 0xa3, 0x7e, 0xb9, 0x28, 0x11, + 0xfd, 0x06, 0x5c, 0x60, 0x82, 0x5c, 0xee, 0x0a, 0xc7, 0x3d, 0xaa, 0x5e, 0x1f, 0xea, 0x9a, 0x30, + 0x34, 0xce, 0xba, 0x3a, 0x21, 0x23, 0x75, 0x20, 0x11, 0x69, 0x70, 0x91, 0xbd, 0x81, 0x07, 0xea, + 0xda, 0x3c, 0xb6, 0x74, 0xfe, 0x06, 0x7d, 0xc5, 0x5b, 0xc3, 0x5e, 0x11, 0x19, 0x3f, 0x5c, 0x9d, + 0x90, 0xe7, 0xd4, 0xc1, 0xd4, 0xa0, 0x1a, 0x36, 0x0f, 0x89, 0xe4, 0xdd, 0xed, 0xad, 0xd1, 0xd5, + 0x88, 0x0a, 0x25, 0xf5, 0xab, 0xd1, 0x93, 0x48, 0x1a, 0xd0, 0xbf, 0x10, 0x82, 0x76, 0xa6, 0x99, + 0xa1, 0x0d, 0x18, 0x11, 0x37, 0x49, 0x1a, 0xf0, 0x20, 0x44, 0x46, 0x45, 0x48, 0xec, 0x6b, 0xf4, + 0x20, 0x31, 0xda, 0x74, 0xf2, 0x63, 0x03, 0xab, 0x13, 0x72, 0x62, 0x5f, 0x43, 0x1f, 0x43, 0x9a, + 0x05, 0x7a, 0x1d, 0x9b, 0xf3, 0xf9, 0xa1, 0x73, 0x76, 0x6f, 0xb8, 0x5c, 0x75, 0x42, 0xa6, 0xb1, + 0x65, 0xbc, 0x23, 0xf3, 0x20, 0x1e, 0x2a, 0xa2, 0x38, 0x22, 0xbe, 0xbb, 0x2f, 0x94, 0x8a, 0x74, + 0x18, 0xdb, 0x27, 0xa2, 0x6d, 0xc8, 0xf3, 0xa5, 0xdb, 0x0b, 0x4a, 0x10, 0x87, 0x3a, 0xff, 0x44, + 0xc5, 0x25, 0x54, 0xe9, 0xae, 0x67, 0x88, 0x4e, 0xda, 0xae, 0x57, 0x22, 0x6f, 0xbb, 0xd9, 0xa1, + 0x6d, 0x37, 0xd4, 0x47, 0x9e, 0xb4, 0x9d, 0x3d, 0x90, 0x88, 0xde, 0x83, 0x49, 0x36, 0x4e, 0x10, + 0x15, 0x19, 0xe5, 0xcf, 0xd6, 0x37, 0x44, 0x58, 0x7e, 0x32, 0x7b, 0xb9, 0xdc, 0xd7, 0x57, 0x69, + 0x5a, 0xfb, 0xf3, 0x73, 0x43, 0x67, 0xaf, 0x41, 0xaf, 0x65, 0x32, 0x7b, 0xb9, 0x01, 0x95, 0x74, + 0x20, 0x6f, 0x8f, 0x8a, 0x0d, 0xb1, 0x0b, 0x43, 0x3b, 0x50, 0x84, 0x0b, 0x70, 0x95, 0x46, 0x61, + 0x05, 0x64, 0x7f, 0x62, 0x75, 0xb0, 0x42, 0x27, 0xc5, 0x8b, 0xa3, 0x27, 0xd6, 0x9e, 0x0b, 0xd0, + 0xfc, 0x89, 0x95, 0x51, 0xd1, 0x33, 0x10, 0xf9, 0x2d, 0x3c, 0x8a, 0xe7, 0x92, 0x36, 0x7f, 0x89, + 0xca, 0xbb, 0x1b, 0xb9, 0x20, 0x46, 0x79, 0x2b, 0x56, 0x09, 0x96, 0xec, 0x4d, 0x41, 0x9f, 0xc0, + 0x2c, 0x95, 0xa7, 0x68, 0xc1, 0xc5, 0x49, 0xf3, 0xf3, 0x03, 0xd7, 0xf0, 0x0c, 0xbf, 0x63, 0xc9, + 0x93, 0x2c, 0x6a, 0x7d, 0x49, 0x64, 0x3c, 0x18, 0xa6, 0xe1, 0xd2, 0xb5, 0x7b, 0x61, 0xe8, 0x78, + 0xe8, 0xbd, 0x34, 0xb6, 0x4a, 0x6d, 0x2d, 0x4a, 0x21, 0xdd, 0xb8, 0x6f, 0xc6, 0x7b, 0x6d, 0x68, + 0x37, 0x1e, 0x32, 0xd9, 0xe5, 0xdc, 0x9e, 0x79, 0x6e, 0x15, 0x80, 0xed, 0x20, 0x50, 0x50, 0xb4, + 0x38, 0xd4, 0x00, 0xe8, 0x77, 0xd1, 0x25, 0x06, 0x40, 0xd3, 0xa3, 0x11, 0x03, 0x80, 0x9d, 0xa0, + 0xcd, 0x5f, 0x1b, 0xbe, 0x58, 0x85, 0x8f, 0xec, 0xe9, 0x62, 0x45, 0x09, 0xc4, 0x20, 0x23, 0x70, + 0xae, 0x4b, 0x47, 0xf8, 0xf5, 0xa1, 0xbb, 0x37, 0x7d, 0xb1, 0x7b, 0xd5, 0x09, 0x39, 0xfd, 0x82, + 0x93, 0x48, 0xaf, 0x62, 0x22, 0xf8, 0xd8, 0xbe, 0x37, 0xb4, 0x57, 0x0d, 0x06, 0x6d, 0x91, 0x5e, + 0xf5, 0x22, 0xa0, 0x06, 0x4b, 0x9e, 0xc3, 0xce, 0xc6, 0xe6, 0xdf, 0x18, 0xbd, 0xe4, 0xf5, 0x9e, + 0xe4, 0xf9, 0x4b, 0x1e, 0x27, 0xb3, 0x25, 0x4f, 0x57, 0x1c, 0x87, 0x7a, 0xfe, 0xcc, 0xdf, 0x1c, + 0xb1, 0xe4, 0xf5, 0xed, 0x96, 0xb3, 0x25, 0x4f, 0xaf, 0x33, 0x4e, 0x62, 0xfd, 0xd9, 0xde, 0xad, + 0x53, 0x1c, 0xd8, 0xdf, 0x1e, 0x6a, 0xfd, 0x45, 0x5e, 0x8b, 0x45, 0xac, 0x3f, 0xbb, 0x27, 0x01, + 0x7d, 0x1b, 0xa6, 0xf9, 0x56, 0xe2, 0xfc, 0x9d, 0x11, 0x26, 0x75, 0x78, 0xf7, 0x97, 0x74, 0x47, + 0xce, 0xc3, 0x26, 0x07, 0xb6, 0x85, 0xc9, 0x26, 0xbf, 0xbb, 0x23, 0x26, 0x87, 0x81, 0x5d, 0x54, + 0x36, 0x39, 0x04, 0x64, 0x52, 0x1a, 0x87, 0x6d, 0xbf, 0xcd, 0xff, 0xca, 0xd0, 0xd2, 0xf4, 0xee, + 0x43, 0x92, 0xd2, 0x70, 0x1e, 0xba, 0x58, 0xd0, 0xb5, 0x9a, 0x69, 0xe7, 0xcd, 0xe1, 0x8b, 0x45, + 0xff, 0x86, 0x4e, 0xd5, 0x3b, 0x7b, 0x64, 0x5a, 0xf9, 0x4b, 0x02, 0x5c, 0x63, 0x7d, 0x80, 0x9e, + 0xbc, 0x74, 0x15, 0xff, 0xe0, 0x2c, 0xb4, 0x5b, 0x75, 0x9f, 0x8a, 0x7f, 0xef, 0xfc, 0xe7, 0x3c, + 0xde, 0x1b, 0x5f, 0x57, 0x47, 0xe5, 0x23, 0xca, 0x68, 0x31, 0x5c, 0x3f, 0xff, 0x60, 0xa8, 0x32, + 0x7a, 0xf7, 0x22, 0x88, 0x32, 0x38, 0x0f, 0x6a, 0xc2, 0x3c, 0x1b, 0x12, 0x01, 0xee, 0xf5, 0x8b, + 0xfe, 0x70, 0xa8, 0x33, 0xed, 0x48, 0xc4, 0x5f, 0x9d, 0x90, 0x2f, 0xbd, 0x88, 0xcc, 0x40, 0xde, + 0x46, 0xaf, 0x95, 0x34, 0x02, 0x98, 0xaa, 0x78, 0x78, 0xf2, 0x9d, 0xa1, 0x6f, 0x1b, 0x09, 0x9f, + 0xc9, 0xdb, 0x9c, 0xc8, 0x0c, 0x44, 0x35, 0xbb, 0x0c, 0x1a, 0xce, 0xbf, 0x3b, 0x54, 0x35, 0xbd, + 0x98, 0x94, 0xa8, 0x86, 0xf3, 0x94, 0xa6, 0xb9, 0x17, 0x87, 0x1f, 0x2d, 0x5e, 0x10, 0xc5, 0xb5, + 0x54, 0xfa, 0xb2, 0x38, 0xbf, 0x96, 0x4a, 0x5f, 0x11, 0x17, 0xd6, 0x52, 0xe9, 0xab, 0xe2, 0x6b, + 0x6b, 0xa9, 0xf4, 0x92, 0x78, 0x6d, 0x2d, 0x95, 0x96, 0xc4, 0x1b, 0xd2, 0xcf, 0xaf, 0x42, 0xce, + 0x43, 0x96, 0x0c, 0xf2, 0x3d, 0x08, 0x43, 0xbe, 0xc5, 0x61, 0x90, 0x8f, 0x63, 0x51, 0x8e, 0xf9, + 0x1e, 0x84, 0x31, 0xdf, 0xe2, 0x30, 0xcc, 0x17, 0xf0, 0x10, 0xd0, 0xd7, 0x18, 0x06, 0xfa, 0xee, + 0x8e, 0x01, 0xfa, 0x7c, 0x51, 0xfd, 0xa8, 0x6f, 0x75, 0x10, 0xf5, 0xbd, 0x31, 0x1a, 0xf5, 0xf9, + 0xa2, 0x42, 0xb0, 0xef, 0xc3, 0x3e, 0xd8, 0x77, 0x7d, 0x04, 0xec, 0xf3, 0xf9, 0x3d, 0xdc, 0xb7, + 0x1e, 0x89, 0xfb, 0x6e, 0x9d, 0x85, 0xfb, 0x7c, 0x39, 0x3d, 0xc0, 0xaf, 0x1a, 0x05, 0xfc, 0x6e, + 0x9e, 0x01, 0xfc, 0x7c, 0x51, 0x61, 0xe4, 0xb7, 0x1e, 0x89, 0xfc, 0x6e, 0x9d, 0x85, 0xfc, 0x82, + 0x62, 0x85, 0xa1, 0xdf, 0xbb, 0x3d, 0xd0, 0x6f, 0x69, 0x28, 0xf4, 0xf3, 0xb9, 0x19, 0xf6, 0xfb, + 0xa8, 0x1f, 0xfb, 0x5d, 0x1f, 0x81, 0xfd, 0x02, 0xc5, 0x72, 0xf0, 0x57, 0x8d, 0x02, 0x7f, 0x37, + 0xcf, 0x00, 0x7f, 0x81, 0x2e, 0x42, 0xe8, 0x6f, 0x2b, 0x1a, 0xfd, 0xdd, 0x3e, 0x13, 0xfd, 0xf9, + 0xd2, 0x7a, 0xe1, 0x5f, 0x35, 0x0a, 0xfe, 0xdd, 0x3c, 0x03, 0xfe, 0xf5, 0x95, 0x8c, 0xe1, 0x3f, + 0x75, 0x24, 0xfe, 0x7b, 0x6b, 0x4c, 0xfc, 0xe7, 0x8b, 0x8e, 0x02, 0x80, 0xfa, 0x68, 0x00, 0x58, + 0x1c, 0x17, 0x00, 0xfa, 0x2f, 0x89, 0x44, 0x80, 0xea, 0x48, 0x04, 0xf8, 0xd6, 0x98, 0x08, 0xb0, + 0xaf, 0x22, 0xbd, 0x10, 0x70, 0x2b, 0x1a, 0x02, 0xde, 0x3e, 0x13, 0x02, 0x06, 0xad, 0xd8, 0x83, + 0x01, 0x97, 0x43, 0x18, 0xf0, 0xf5, 0x21, 0x18, 0xd0, 0x67, 0x25, 0x20, 0xf0, 0x3b, 0x03, 0x20, + 0x50, 0x1a, 0x05, 0x02, 0x7d, 0x5e, 0x1f, 0x05, 0x56, 0xa3, 0x50, 0xe0, 0xcd, 0x33, 0x50, 0x60, + 0xd0, 0x6f, 0x42, 0x30, 0xf0, 0xe9, 0x10, 0x18, 0x78, 0xe7, 0x6c, 0x18, 0xe8, 0xcb, 0xeb, 0xc3, + 0x81, 0xea, 0x48, 0x1c, 0xf8, 0xd6, 0x98, 0x38, 0x30, 0x68, 0xc1, 0x08, 0x20, 0xf8, 0x7e, 0x2f, + 0x10, 0xbc, 0x36, 0x1c, 0x08, 0xfa, 0x62, 0x38, 0x12, 0x5c, 0x8f, 0x44, 0x82, 0xb7, 0xce, 0x42, + 0x82, 0xc1, 0x6c, 0x16, 0x86, 0x82, 0x5b, 0xd1, 0x50, 0xf0, 0xf6, 0x99, 0x50, 0x30, 0xe8, 0x48, + 0x3d, 0x58, 0x70, 0x3d, 0x12, 0x0b, 0xde, 0x3a, 0x0b, 0x0b, 0xf6, 0x4d, 0xb5, 0x1c, 0x0c, 0x3e, + 0x1f, 0x0a, 0x06, 0xef, 0x8d, 0x03, 0x06, 0x7d, 0xa1, 0x03, 0x68, 0xf0, 0xd3, 0xe1, 0x68, 0xf0, + 0x57, 0xce, 0x71, 0xe3, 0x6e, 0x24, 0x1c, 0xfc, 0xce, 0x00, 0x1c, 0x94, 0x46, 0xc1, 0xc1, 0x60, + 0x64, 0x78, 0x78, 0xb0, 0x12, 0x81, 0xde, 0xde, 0x18, 0x8d, 0xde, 0x82, 0x85, 0x3c, 0x80, 0x6f, + 0x1f, 0xf6, 0xc1, 0xb7, 0xeb, 0x67, 0x3a, 0x98, 0x86, 0xf0, 0x5b, 0x69, 0x10, 0xbf, 0xdd, 0x18, + 0x89, 0xdf, 0x7c, 0x09, 0x01, 0x80, 0x5b, 0x8f, 0x04, 0x70, 0xb7, 0xce, 0x02, 0x70, 0x41, 0x57, + 0x08, 0x23, 0xb8, 0xad, 0x68, 0x04, 0x77, 0xfb, 0x4c, 0x04, 0xd7, 0xb7, 0x6c, 0x79, 0x10, 0xae, + 0x1a, 0x05, 0xe1, 0x6e, 0x9e, 0x01, 0xe1, 0xc2, 0xcb, 0x96, 0x8f, 0xe1, 0x1a, 0xc3, 0x30, 0xdc, + 0xdd, 0x31, 0x30, 0x5c, 0x60, 0xcc, 0xf5, 0x81, 0xb8, 0x8f, 0xfb, 0x41, 0x9c, 0x34, 0x0a, 0xc4, + 0x05, 0x9d, 0xc8, 0x43, 0x71, 0x5b, 0xd1, 0x28, 0xee, 0xf6, 0x99, 0x28, 0x2e, 0x3c, 0xae, 0x43, + 0x30, 0xee, 0xe3, 0x7e, 0x18, 0x27, 0x8d, 0x82, 0x71, 0x41, 0x79, 0x3c, 0x1c, 0x57, 0x8d, 0xc2, + 0x71, 0x37, 0xcf, 0xc0, 0x71, 0xa1, 0xe9, 0x3e, 0x00, 0x72, 0x7f, 0x79, 0x7c, 0x20, 0xf7, 0xfe, + 0xab, 0x3a, 0xec, 0x9d, 0x8d, 0xe4, 0x3e, 0xee, 0x47, 0x72, 0xd2, 0x28, 0x24, 0x17, 0xe8, 0xc3, + 0x83, 0x72, 0xad, 0x33, 0xa1, 0xdc, 0xfd, 0x73, 0x40, 0x39, 0x5f, 0xfe, 0x30, 0x2c, 0xd7, 0x3a, + 0x13, 0xcb, 0xdd, 0x3f, 0x07, 0x96, 0x0b, 0x5e, 0x37, 0x04, 0xcc, 0x7d, 0xdc, 0x0f, 0xe6, 0xa4, + 0x51, 0x60, 0x2e, 0xd0, 0xce, 0xb9, 0xd1, 0xdc, 0x6b, 0xe2, 0xeb, 0x3d, 0x98, 0xee, 0xa7, 0x19, + 0x98, 0xaa, 0x46, 0x9c, 0x09, 0x0a, 0xaf, 0x72, 0x26, 0x88, 0xbe, 0x0d, 0x57, 0xfd, 0x07, 0xea, + 0xcf, 0xa5, 0xf0, 0x28, 0x3b, 0xad, 0x69, 0x69, 0x87, 0x74, 0xa9, 0x4c, 0xcb, 0xf3, 0x7e, 0x96, + 0x47, 0xb6, 0xd5, 0x62, 0xd1, 0x76, 0xd4, 0x4b, 0x06, 0xad, 0x92, 0x31, 0x4c, 0x6d, 0xc2, 0xb3, + 0x2f, 0x31, 0x1d, 0xbc, 0x66, 0x93, 0xb3, 0xbe, 0xc2, 0xbd, 0x10, 0xe8, 0x5d, 0xc8, 0x75, 0x1c, + 0x6c, 0x2b, 0x6d, 0xdb, 0xb0, 0x6c, 0xc3, 0x65, 0x11, 0x6e, 0x42, 0x49, 0xfc, 0xf2, 0x64, 0x69, + 0x66, 0xc7, 0xc1, 0xf6, 0x36, 0xa7, 0xcb, 0x33, 0x9d, 0xd0, 0x93, 0xf7, 0xa9, 0xc0, 0xc9, 0xf1, + 0x3f, 0x15, 0xf8, 0x14, 0x44, 0xea, 0xa1, 0x13, 0x5e, 0x46, 0xd9, 0xd5, 0x6f, 0xd1, 0x2b, 0xbe, + 0xaa, 0x87, 0x56, 0x4a, 0x7a, 0x05, 0x5c, 0xc1, 0xee, 0x25, 0xa2, 0xc7, 0x90, 0xb7, 0xad, 0x0e, + 0xbd, 0xe1, 0xa9, 0x6d, 0x35, 0x0d, 0xad, 0x4b, 0xed, 0x9b, 0x7c, 0xf4, 0x79, 0x33, 0xcb, 0xb8, + 0x4d, 0xf3, 0xc9, 0x39, 0x3b, 0xfc, 0x88, 0xea, 0x40, 0x6f, 0x77, 0xf2, 0xa4, 0xa0, 0x81, 0xfb, + 0xf4, 0x47, 0x7e, 0xd5, 0xe2, 0xb9, 0x6a, 0xb8, 0x5c, 0x2e, 0xbc, 0xf4, 0x7f, 0xa3, 0x47, 0x30, + 0x43, 0x6f, 0x1a, 0x27, 0x6d, 0x6e, 0x75, 0x5c, 0x6e, 0xde, 0x5c, 0x29, 0xb2, 0x0f, 0x4a, 0x16, + 0xbd, 0x0f, 0x4a, 0x16, 0x57, 0xf9, 0x07, 0x25, 0xd9, 0x59, 0xf5, 0x8f, 0xfe, 0xfd, 0x92, 0x20, + 0x67, 0x3d, 0xcf, 0x29, 0xab, 0xe3, 0xa2, 0xfb, 0x70, 0xb1, 0xa5, 0x1e, 0xd3, 0x0b, 0xcb, 0x7d, + 0x77, 0x51, 0x7a, 0x45, 0x23, 0xfb, 0x50, 0x21, 0x6a, 0xa9, 0xc7, 0xf4, 0xeb, 0x8a, 0x2c, 0x89, + 0x7e, 0x3d, 0xe9, 0x3a, 0xcc, 0xf0, 0xf0, 0x23, 0xf6, 0x71, 0xb5, 0x02, 0xcd, 0xc9, 0xbf, 0xb4, + 0xc3, 0xbe, 0xaf, 0xf6, 0x1e, 0xcc, 0x87, 0xb3, 0x28, 0xcc, 0x79, 0x16, 0xb7, 0xda, 0x6e, 0x77, + 0xfe, 0x32, 0xed, 0xac, 0x17, 0x43, 0xd9, 0x57, 0x48, 0x6a, 0x85, 0x24, 0xa2, 0x9b, 0x90, 0xd7, + 0x0d, 0xc7, 0x35, 0x4c, 0xcd, 0xe5, 0x97, 0xa8, 0xb3, 0x6b, 0xc8, 0x73, 0x1e, 0x95, 0xdd, 0x94, + 0xde, 0x80, 0x59, 0xad, 0x69, 0xf8, 0xe6, 0x30, 0x33, 0x50, 0x66, 0x87, 0x0e, 0xf0, 0x32, 0xcd, + 0xdb, 0xef, 0x79, 0x53, 0xd0, 0x7a, 0xc9, 0xa8, 0x0c, 0x85, 0x7d, 0xd5, 0xc5, 0x2f, 0xd5, 0xae, + 0xe2, 0x85, 0x04, 0x67, 0xe9, 0x35, 0x08, 0x57, 0x4f, 0x4f, 0x96, 0x72, 0x8f, 0x59, 0xd2, 0x40, + 0x64, 0x70, 0x6e, 0x3f, 0x94, 0xa0, 0xa3, 0xdb, 0x50, 0x50, 0x9d, 0xae, 0xa9, 0xd1, 0xae, 0x88, + 0x4d, 0xa7, 0xe3, 0xf0, 0xd0, 0xb8, 0x3c, 0x25, 0x97, 0x3d, 0x2a, 0xfa, 0x10, 0x16, 0xf8, 0xb7, + 0x52, 0x5e, 0xaa, 0xb6, 0xae, 0xd0, 0xee, 0x1b, 0xcc, 0x13, 0x22, 0x73, 0xd4, 0x65, 0xdf, 0x46, + 0x21, 0x19, 0x48, 0x9f, 0x0d, 0xa6, 0xd1, 0x67, 0x30, 0x4b, 0x03, 0xe4, 0xb0, 0x4e, 0xd6, 0xb1, + 0x26, 0x36, 0xb1, 0xe3, 0x8c, 0x38, 0x9e, 0x28, 0xb1, 0xbc, 0x75, 0x2f, 0x2b, 0x9b, 0x99, 0x64, + 0x71, 0xb7, 0x8f, 0xee, 0xdf, 0xdd, 0x0e, 0x62, 0x76, 0x2d, 0x95, 0x9e, 0x11, 0x73, 0x6b, 0xa9, + 0x74, 0x5e, 0x2c, 0x48, 0xff, 0x47, 0x80, 0x4b, 0xd1, 0x42, 0x50, 0x1d, 0xe6, 0x7a, 0xae, 0x27, + 0x62, 0x91, 0x7b, 0xe7, 0x99, 0xe8, 0x66, 0xc3, 0x97, 0x14, 0xd1, 0xb7, 0x10, 0xe5, 0x44, 0x08, + 0x55, 0x1c, 0xd7, 0x36, 0x34, 0xef, 0xca, 0xaf, 0xcb, 0x03, 0x6c, 0x75, 0x9a, 0x4c, 0x4b, 0xa4, + 0x1e, 0x0f, 0x94, 0x28, 0x79, 0x9e, 0x12, 0xa9, 0xc7, 0xbd, 0xa2, 0xa5, 0xef, 0x27, 0xa1, 0x40, + 0x96, 0x6c, 0xc7, 0x31, 0x2c, 0xb3, 0xea, 0x07, 0x0b, 0xf8, 0x33, 0x9b, 0x40, 0xe3, 0x5e, 0xfd, + 0x67, 0xb4, 0x44, 0xe3, 0x8b, 0x09, 0x34, 0xf2, 0xbf, 0x3d, 0x95, 0x94, 0x81, 0x91, 0x68, 0x04, + 0xe5, 0x0a, 0x4c, 0x39, 0x56, 0xc7, 0xd6, 0xbc, 0x2f, 0xa2, 0xdc, 0x1d, 0x62, 0x23, 0x84, 0x5e, + 0x58, 0xac, 0x53, 0x06, 0x99, 0x33, 0xa2, 0x4f, 0xa1, 0xc0, 0x7e, 0xd1, 0xb8, 0x46, 0x1a, 0x57, + 0xc8, 0x82, 0x43, 0xef, 0x8f, 0x2d, 0x6b, 0x83, 0x33, 0xca, 0x79, 0xa7, 0xe7, 0x19, 0x7d, 0x0c, + 0xaf, 0x99, 0x96, 0xd2, 0xc2, 0x2d, 0x8b, 0xd9, 0x06, 0x64, 0x7a, 0xd2, 0x15, 0xd5, 0x55, 0x78, + 0xa1, 0x99, 0xf7, 0xf9, 0xbc, 0x69, 0x6d, 0xd2, 0x2c, 0x32, 0xcf, 0xb1, 0xe2, 0x32, 0xb9, 0x52, + 0x11, 0xa6, 0xd8, 0x2f, 0x94, 0x81, 0xc9, 0x27, 0x8d, 0x6a, 0x45, 0x16, 0x27, 0xd0, 0x0c, 0xa4, + 0x1f, 0xc9, 0x4f, 0x36, 0x95, 0xfa, 0xd3, 0x0d, 0x51, 0x40, 0x59, 0x98, 0x96, 0x9f, 0x3c, 0x69, + 0x28, 0xeb, 0xcf, 0xc4, 0x84, 0x74, 0x1b, 0xf2, 0xbd, 0x25, 0x42, 0x00, 0x53, 0x72, 0x65, 0xf3, + 0x09, 0xfd, 0x0c, 0x48, 0x06, 0x26, 0x37, 0x9e, 0x94, 0x57, 0x36, 0x44, 0x41, 0xfa, 0xa5, 0x00, + 0x33, 0x25, 0xf6, 0x5d, 0x1b, 0xe6, 0x10, 0xf4, 0x61, 0x9f, 0xe3, 0xce, 0x95, 0xe8, 0xbd, 0x87, + 0x61, 0x0e, 0x3b, 0x69, 0x3e, 0xed, 0x79, 0xf1, 0x6f, 0x4b, 0xc3, 0x11, 0x27, 0xdd, 0x9c, 0xf5, + 0x3c, 0x84, 0x3d, 0x36, 0x54, 0x07, 0x51, 0xf5, 0x74, 0xab, 0xf0, 0x92, 0x0c, 0xf7, 0x13, 0xee, + 0x6b, 0x06, 0x6f, 0x2e, 0x52, 0x7b, 0xc9, 0x1f, 0xa4, 0x7e, 0xf4, 0xf9, 0xd2, 0x84, 0xf4, 0xa7, + 0x29, 0xc8, 0x95, 0xc2, 0xdf, 0xf0, 0x41, 0xb5, 0xbe, 0xca, 0xde, 0x8e, 0xb4, 0x67, 0x42, 0x1c, + 0xc5, 0x11, 0x5f, 0x47, 0xcb, 0x04, 0x1f, 0x0c, 0x62, 0x75, 0xbf, 0x36, 0xc2, 0xe7, 0x29, 0x5c, + 0xf9, 0x80, 0x71, 0xe1, 0x5f, 0x27, 0x7d, 0x43, 0xa7, 0x08, 0x93, 0x2c, 0xbe, 0x5b, 0x18, 0xb8, + 0x7a, 0x86, 0x2e, 0x71, 0x04, 0xe7, 0x91, 0x74, 0x99, 0x65, 0x23, 0x86, 0x51, 0xe3, 0x95, 0x9c, + 0xa5, 0x82, 0x79, 0xf0, 0xfc, 0x1f, 0x15, 0xee, 0xb0, 0x8b, 0x96, 0xff, 0x1f, 0xba, 0x26, 0x93, + 0xf7, 0xa1, 0x5f, 0x87, 0x82, 0x66, 0x35, 0x9b, 0x0c, 0x1e, 0xb0, 0x95, 0x6d, 0xf0, 0xea, 0x35, + 0x5a, 0x04, 0xfe, 0x1d, 0xe9, 0xa2, 0xff, 0x3d, 0xe9, 0xa2, 0xcc, 0xbf, 0x27, 0x1d, 0x0a, 0x45, + 0xcb, 0xfb, 0xc2, 0xd8, 0x82, 0xd8, 0x17, 0x15, 0x37, 0xfd, 0x2a, 0x51, 0x71, 0x2c, 0x96, 0x90, + 0xf7, 0xbc, 0x9f, 0x0b, 0xdc, 0x81, 0x78, 0xc3, 0xb2, 0x0e, 0x3b, 0xbe, 0x5b, 0xe6, 0x42, 0xf8, + 0xda, 0xe4, 0x20, 0x60, 0x87, 0xc6, 0xaf, 0x46, 0xd9, 0x60, 0x89, 0xaf, 0x66, 0x83, 0x5d, 0x87, + 0x99, 0xb6, 0x8d, 0xf7, 0xb0, 0xab, 0x1d, 0x28, 0x66, 0xa7, 0xc5, 0x83, 0x77, 0xb3, 0x1e, 0x6d, + 0xab, 0xd3, 0x42, 0x77, 0x41, 0xf4, 0xb3, 0xf0, 0x2d, 0x1b, 0xef, 0xce, 0x4e, 0x8f, 0xce, 0x37, + 0x78, 0xa4, 0xff, 0x2e, 0xc0, 0x5c, 0x4f, 0x9d, 0xf8, 0x98, 0x5a, 0x83, 0xac, 0xee, 0x5b, 0xbd, + 0xce, 0xbc, 0x70, 0xce, 0x50, 0xae, 0x30, 0x33, 0x52, 0xe0, 0x92, 0xf7, 0x5a, 0xfa, 0x91, 0x9d, + 0x40, 0x6c, 0xe2, 0x9c, 0x62, 0x2f, 0x06, 0x72, 0x56, 0x43, 0x2f, 0xf0, 0x07, 0x59, 0x72, 0xac, + 0x41, 0x26, 0xfd, 0x4f, 0x01, 0x44, 0xfa, 0x82, 0x47, 0x18, 0xeb, 0xb1, 0x4c, 0x99, 0x5e, 0xcc, + 0x64, 0x62, 0xfc, 0xf0, 0xdc, 0x9e, 0x0f, 0x83, 0x25, 0xfb, 0x3e, 0x0c, 0x16, 0x35, 0x7f, 0xa6, + 0xbe, 0xe2, 0xfc, 0x29, 0x7d, 0x2e, 0x40, 0xde, 0xaf, 0x36, 0xfb, 0x22, 0xf0, 0x88, 0x2b, 0xbf, + 0x5f, 0xed, 0xab, 0xb7, 0xde, 0xd5, 0x64, 0x63, 0x7d, 0xa4, 0x38, 0x7c, 0x35, 0x19, 0xfb, 0x5a, + 0xeb, 0xdf, 0xf0, 0xba, 0x23, 0x29, 0x62, 0x39, 0xb8, 0x13, 0xea, 0x15, 0xc2, 0x9f, 0xbf, 0x0e, + 0x0f, 0xe4, 0x47, 0x21, 0x05, 0xd2, 0x1e, 0x45, 0xb4, 0x34, 0xd6, 0xfc, 0xee, 0x69, 0x89, 0x75, + 0xc0, 0x9f, 0x86, 0x5b, 0x82, 0xdd, 0x29, 0xf2, 0x10, 0x92, 0x47, 0x6a, 0x73, 0x94, 0x9f, 0x6d, + 0x4f, 0xcb, 0xc9, 0x24, 0x37, 0x7a, 0xd4, 0x73, 0x95, 0x56, 0x62, 0xf8, 0x76, 0xee, 0xa0, 0x4a, + 0x7b, 0xae, 0xdc, 0x7a, 0xaf, 0x77, 0x00, 0x8d, 0x7c, 0x7d, 0x78, 0x24, 0x7d, 0x90, 0xfa, 0xe2, + 0xf3, 0x25, 0x41, 0xfa, 0x08, 0x10, 0xb1, 0x75, 0xdc, 0xa7, 0x1d, 0xcb, 0x0e, 0xae, 0x25, 0xeb, + 0x8f, 0x8d, 0x9c, 0x8c, 0x8e, 0x8d, 0x94, 0x2e, 0xc2, 0x5c, 0x0f, 0x37, 0x9b, 0x81, 0xa4, 0xf7, + 0xe0, 0xca, 0x63, 0xcb, 0x71, 0x8c, 0x76, 0xbd, 0xb3, 0xcb, 0x86, 0x3a, 0x59, 0xaf, 0xfc, 0x39, + 0x37, 0xdd, 0xa6, 0x7b, 0x85, 0x26, 0x9b, 0x9b, 0x32, 0xb2, 0xff, 0x2c, 0xfd, 0xa1, 0x00, 0x97, + 0x07, 0x39, 0x99, 0x96, 0xa3, 0x6e, 0x6b, 0x98, 0xd6, 0xac, 0xe0, 0xd6, 0xdc, 0xb3, 0x7b, 0xab, + 0x97, 0x9d, 0xa0, 0x1a, 0xfe, 0x4e, 0xa5, 0xa5, 0xd2, 0x39, 0x89, 0x5f, 0xe2, 0x92, 0xe7, 0xe4, + 0x4d, 0x46, 0x0d, 0xa6, 0xa7, 0xd4, 0x78, 0xd3, 0xd3, 0x4f, 0x12, 0x30, 0xdb, 0xc0, 0xa6, 0x6a, + 0xba, 0x64, 0xde, 0xef, 0xb4, 0xd8, 0x0d, 0x17, 0x05, 0x48, 0xda, 0x4a, 0x87, 0x16, 0x5d, 0x90, + 0x13, 0xf6, 0x0e, 0xba, 0x01, 0x39, 0xba, 0xb6, 0x84, 0x6c, 0x35, 0xe1, 0x4e, 0x4a, 0xa6, 0x61, + 0x39, 0xb2, 0x67, 0x88, 0xbd, 0x0e, 0x40, 0x33, 0x31, 0x58, 0x9a, 0xa4, 0x39, 0x32, 0x84, 0xc2, + 0x40, 0xe9, 0x4d, 0xc8, 0xb3, 0xc8, 0x48, 0x5f, 0x08, 0x8b, 0x64, 0xca, 0x51, 0xaa, 0x2f, 0x65, + 0x09, 0xb2, 0x2c, 0x1b, 0x13, 0x33, 0x49, 0xf3, 0x00, 0x25, 0x31, 0x39, 0x8f, 0xe0, 0x82, 0xf3, + 0xa2, 0xa9, 0xb4, 0x2d, 0xdd, 0x51, 0xb4, 0x76, 0x87, 0x87, 0x88, 0xb0, 0x8f, 0x8c, 0x0b, 0xa5, + 0x8b, 0xa7, 0x27, 0x4b, 0xb3, 0xf5, 0xa7, 0x1b, 0xdb, 0x96, 0xee, 0x94, 0xb7, 0x77, 0x58, 0x80, + 0x88, 0x23, 0xcf, 0x3a, 0x2f, 0x9a, 0x94, 0xd4, 0xee, 0x70, 0x12, 0xaa, 0xc0, 0x5c, 0x7b, 0xff, + 0xa5, 0x61, 0x63, 0x05, 0xef, 0xdb, 0xd8, 0x71, 0xf8, 0x0b, 0xa7, 0xc9, 0x0b, 0x99, 0x98, 0xed, + 0xc7, 0xcf, 0x0d, 0x1b, 0x57, 0x68, 0x2a, 0x7d, 0xb7, 0x3c, 0xcb, 0x38, 0x42, 0x24, 0xe9, 0x97, + 0x49, 0x40, 0xf4, 0xb2, 0xa7, 0x12, 0xbd, 0x2d, 0xc9, 0xeb, 0x35, 0x16, 0x2c, 0x6a, 0x81, 0x46, + 0x15, 0xc7, 0x30, 0x09, 0x4c, 0x50, 0x1d, 0x3f, 0x2a, 0x94, 0x0f, 0xbf, 0xc8, 0x3b, 0x6a, 0xfb, + 0x1b, 0x84, 0x77, 0x87, 0xab, 0x21, 0x89, 0x75, 0x22, 0x70, 0x43, 0x75, 0xfc, 0x17, 0xde, 0x85, + 0x8c, 0x4b, 0xf9, 0xbc, 0xab, 0xb4, 0x52, 0xa5, 0x99, 0xd3, 0x93, 0xa5, 0x34, 0x13, 0x56, 0x5b, + 0x95, 0xd3, 0x2c, 0xb9, 0xa6, 0xa3, 0x65, 0xc8, 0x1a, 0xa6, 0xe3, 0xaa, 0xa4, 0x48, 0x7c, 0x33, + 0x29, 0xc7, 0xee, 0x3a, 0xa8, 0x71, 0x72, 0x6d, 0x55, 0x06, 0x2f, 0x4b, 0x4d, 0x47, 0x35, 0xb8, + 0x68, 0xe2, 0x63, 0x57, 0xa1, 0xe1, 0xe4, 0x61, 0xd6, 0x34, 0x65, 0xbd, 0x74, 0x7a, 0xb2, 0x84, + 0xb6, 0xf0, 0xb1, 0xbb, 0x61, 0x1c, 0xe1, 0x90, 0x08, 0x64, 0xf6, 0xd3, 0x74, 0xd2, 0x0b, 0x7c, + 0x01, 0xec, 0xa0, 0x89, 0x86, 0xd2, 0xcb, 0x39, 0x8f, 0xca, 0x8e, 0x8f, 0x2e, 0xc3, 0xb4, 0x83, + 0x5f, 0x50, 0xa3, 0x63, 0x9a, 0x1a, 0x1d, 0x53, 0x0e, 0xbd, 0xe3, 0x16, 0x49, 0xfe, 0x39, 0x15, + 0xd6, 0x15, 0xd2, 0x49, 0x59, 0x3c, 0x51, 0xd6, 0x27, 0xca, 0x3b, 0xe8, 0x39, 0xf0, 0xed, 0x0d, + 0x7f, 0x4b, 0xa5, 0x8d, 0x6d, 0xc3, 0xf2, 0xee, 0x8d, 0x18, 0x6b, 0x97, 0x66, 0x8e, 0x49, 0xe0, + 0xda, 0xdd, 0xa6, 0xfc, 0xd2, 0x9f, 0x08, 0x30, 0xd7, 0xd3, 0xd6, 0xdc, 0x82, 0xf9, 0xa8, 0x77, + 0x66, 0x0e, 0x9b, 0xf1, 0x94, 0xee, 0xd0, 0x23, 0x7d, 0xcd, 0xd2, 0xf9, 0x94, 0xd6, 0x33, 0x43, + 0xa3, 0x45, 0xc8, 0xee, 0xdb, 0xaa, 0xe9, 0x55, 0x28, 0x41, 0x2b, 0x94, 0xe1, 0x24, 0x79, 0x07, + 0x6d, 0x81, 0x48, 0x80, 0xf5, 0x61, 0x13, 0x2b, 0x3a, 0x2f, 0x28, 0x9f, 0x3c, 0xc7, 0xaa, 0x49, + 0x81, 0x33, 0x7b, 0x49, 0x64, 0x30, 0xef, 0xa9, 0xcd, 0xe6, 0xae, 0xaa, 0x1d, 0x2a, 0x74, 0xe3, + 0x9a, 0xee, 0x0a, 0xca, 0x33, 0x1e, 0x51, 0x56, 0x5d, 0x2c, 0x35, 0xa0, 0xb0, 0x66, 0x19, 0xe6, + 0x96, 0xa5, 0xfb, 0xc1, 0x50, 0x2b, 0x90, 0xdf, 0x35, 0x4c, 0xd5, 0xee, 0x2a, 0x5e, 0x3c, 0x93, + 0x70, 0x56, 0x3c, 0x93, 0x9c, 0x63, 0x1c, 0xfc, 0x51, 0xfa, 0x99, 0x00, 0x62, 0x20, 0x96, 0x6b, + 0xef, 0x4d, 0x00, 0xad, 0xd9, 0x71, 0x5c, 0x6c, 0x7b, 0xd3, 0xf7, 0x0c, 0x0b, 0xc5, 0x2e, 0x33, + 0x6a, 0x6d, 0x55, 0xce, 0xf0, 0x0c, 0x35, 0x1d, 0xdd, 0xe8, 0xbd, 0x30, 0x6e, 0xb2, 0x04, 0xa7, + 0x03, 0xd7, 0xc4, 0x91, 0xf5, 0xc0, 0x71, 0x2d, 0xdb, 0xef, 0xde, 0x7c, 0x3d, 0xf0, 0xae, 0xd2, + 0xa4, 0x57, 0x46, 0x61, 0x7a, 0xdb, 0x42, 0x9e, 0x80, 0x93, 0x23, 0xec, 0x57, 0x29, 0x75, 0x76, + 0x95, 0x18, 0x87, 0x57, 0xa5, 0x3f, 0x10, 0xa0, 0x50, 0x66, 0xd3, 0xb4, 0x3f, 0xf5, 0x8f, 0x30, + 0x75, 0x56, 0x21, 0xed, 0x1e, 0x9b, 0x4a, 0x0b, 0xfb, 0x5f, 0x7f, 0x3c, 0xc7, 0x5d, 0xd7, 0xd3, + 0x2e, 0x7b, 0xa4, 0x1f, 0x14, 0x7f, 0x85, 0xae, 0xe0, 0x33, 0x71, 0x94, 0xf1, 0xaf, 0x04, 0xc8, + 0xd4, 0x35, 0xd5, 0xf4, 0xe3, 0x4d, 0xcd, 0x4e, 0x8b, 0x1d, 0x0d, 0xec, 0xa9, 0x1a, 0x56, 0x1c, + 0x8c, 0x0f, 0x1d, 0x7e, 0x45, 0xce, 0xac, 0xd9, 0x69, 0xd5, 0xbc, 0x94, 0x3a, 0x49, 0x40, 0x6f, + 0x02, 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, 0x73, 0xaf, 0x4e, 0x96, 0xf7, 0x01, 0x5c, 0x83, 0xf2, 0x00, 0xa1, + 0xcf, 0x94, 0x4e, 0xd0, 0x8f, 0x97, 0x56, 0x56, 0x56, 0x95, 0x9d, 0xad, 0xf2, 0x93, 0xcd, 0xcd, + 0x5a, 0xa3, 0x51, 0x59, 0x15, 0x05, 0x24, 0xc2, 0x4c, 0xcf, 0x47, 0x4e, 0x13, 0x0b, 0xa9, 0x1f, + 0xfc, 0xde, 0xe2, 0xc4, 0xbd, 0xb7, 0x20, 0xd7, 0xb3, 0xbf, 0x8c, 0x0a, 0x90, 0xdd, 0xa8, 0xac, + 0xd4, 0x2b, 0xd5, 0x27, 0x1b, 0xab, 0x74, 0x5b, 0x25, 0x0b, 0xd3, 0x5b, 0x95, 0x15, 0xb9, 0x52, + 0x6f, 0x88, 0xc2, 0x3d, 0x13, 0x66, 0xd8, 0x4d, 0x62, 0x32, 0xbd, 0x2c, 0x01, 0x21, 0xc8, 0xcb, + 0x95, 0xfa, 0xce, 0x66, 0x45, 0xd9, 0xd9, 0x5a, 0xdf, 0x7a, 0xf2, 0x7c, 0xcb, 0x2b, 0x00, 0xa5, + 0xad, 0x57, 0x3e, 0x51, 0x36, 0x6a, 0x9b, 0xb5, 0x86, 0x28, 0xa0, 0x8b, 0x30, 0xcb, 0xa9, 0xa5, + 0x4f, 0x1a, 0x15, 0x4e, 0x4e, 0xd0, 0x6f, 0xb3, 0x32, 0x72, 0x6d, 0x8b, 0x7e, 0x2b, 0x95, 0x25, + 0x24, 0x79, 0xf1, 0xfe, 0x1c, 0x40, 0xf0, 0x39, 0x67, 0x52, 0x55, 0x22, 0xf2, 0xd9, 0xca, 0xc6, + 0x4e, 0xa5, 0x2e, 0x4e, 0x90, 0xb7, 0x97, 0x56, 0x1a, 0xe5, 0xaa, 0x22, 0x57, 0xea, 0xdb, 0x4f, + 0xb6, 0xea, 0x15, 0x51, 0xe0, 0x7c, 0xab, 0x30, 0x13, 0xbe, 0x30, 0x15, 0xcd, 0x41, 0xa1, 0x5c, + 0xad, 0x94, 0xd7, 0x95, 0x67, 0xb5, 0x15, 0xe5, 0xe9, 0x4e, 0x65, 0xa7, 0x22, 0x4e, 0x50, 0xcd, + 0x51, 0xe2, 0xa3, 0x9d, 0x8d, 0x0d, 0x51, 0x20, 0x55, 0x67, 0xcf, 0xf4, 0x7b, 0xad, 0x62, 0xe2, + 0xde, 0x26, 0x64, 0x43, 0x1f, 0x72, 0x21, 0xaf, 0xdb, 0xde, 0xa9, 0x57, 0x95, 0x46, 0x6d, 0xb3, + 0x52, 0x6f, 0xac, 0x6c, 0x6e, 0x33, 0x19, 0x94, 0xb6, 0x52, 0x7a, 0x22, 0x93, 0x6a, 0x7a, 0xcf, + 0x8d, 0x27, 0x3b, 0xe5, 0xaa, 0xa7, 0x65, 0x29, 0x95, 0x4e, 0x8a, 0xc9, 0x7b, 0xc7, 0x70, 0x79, + 0xc8, 0xdd, 0xa1, 0x44, 0xc9, 0x3b, 0x26, 0xfd, 0xa8, 0x85, 0x38, 0x81, 0x72, 0x90, 0x21, 0x23, + 0x9d, 0x5e, 0xc7, 0x23, 0x0a, 0x28, 0x0d, 0xa9, 0x03, 0xd7, 0x6d, 0x8b, 0x09, 0x34, 0x05, 0x09, + 0xe7, 0xa1, 0x98, 0x24, 0xff, 0xf7, 0x1d, 0x31, 0x85, 0x32, 0x30, 0xa9, 0x7e, 0xd6, 0xb1, 0xb1, + 0x38, 0x89, 0x66, 0x20, 0xdd, 0x71, 0xb0, 0xbd, 0x67, 0x34, 0xb1, 0x38, 0x4d, 0x58, 0xcc, 0x4e, + 0xb3, 0x29, 0xa6, 0xa5, 0x54, 0x7a, 0x4a, 0x9c, 0xba, 0x77, 0x1d, 0x42, 0xd7, 0xb5, 0x21, 0x80, + 0xa9, 0x0d, 0xd5, 0xc5, 0x8e, 0x2b, 0x4e, 0xa0, 0x69, 0x48, 0xae, 0x34, 0x9b, 0xa2, 0xf0, 0xe0, + 0x7f, 0x4c, 0x41, 0xda, 0xeb, 0x75, 0x68, 0x03, 0x26, 0xe9, 0x3e, 0x0f, 0x5a, 0x1a, 0xbe, 0x03, + 0x44, 0xa7, 0xcd, 0x85, 0x6b, 0x67, 0x6d, 0x11, 0x49, 0x13, 0xe8, 0xcf, 0x43, 0x36, 0x84, 0x8c, + 0xd1, 0xd0, 0xf3, 0xd0, 0x9e, 0xdd, 0x80, 0x85, 0x5b, 0x67, 0x65, 0xf3, 0xe5, 0x3f, 0x87, 0x8c, + 0x6f, 0x54, 0xa3, 0x1b, 0xa3, 0x4c, 0x6e, 0x4f, 0xf6, 0x68, 0xbb, 0x9c, 0xcc, 0x72, 0xd2, 0xc4, + 0xdb, 0x02, 0xb2, 0x01, 0x0d, 0xda, 0xbf, 0x28, 0x2a, 0x40, 0x62, 0xa8, 0x81, 0xbd, 0x70, 0x6f, + 0xac, 0xdc, 0xc1, 0x3b, 0x89, 0xb2, 0x02, 0x23, 0x3e, 0x5a, 0x59, 0x03, 0x10, 0x21, 0x5a, 0x59, + 0x11, 0x58, 0x80, 0x36, 0x46, 0x68, 0x91, 0x8f, 0x94, 0x3f, 0x68, 0xf0, 0x45, 0xca, 0x8f, 0xb0, + 0x15, 0xa4, 0x09, 0xf4, 0x14, 0x52, 0x64, 0x0d, 0x44, 0x51, 0xf0, 0xba, 0x6f, 0xcd, 0x5d, 0xb8, + 0x31, 0x32, 0x8f, 0x2f, 0x72, 0x1f, 0xf2, 0x8f, 0x31, 0x3d, 0x9c, 0x61, 0x77, 0xaf, 0x3a, 0xe8, + 0x4e, 0xb4, 0xd7, 0x6a, 0x28, 0x8b, 0xf7, 0x8a, 0xbb, 0x63, 0xe4, 0xf4, 0x5f, 0xd4, 0x86, 0x59, + 0xf6, 0xd5, 0x9a, 0xf0, 0xbb, 0xa2, 0x5c, 0x65, 0x06, 0x72, 0x79, 0xaf, 0x7b, 0x73, 0xbc, 0xcc, + 0xde, 0x1b, 0x4b, 0x77, 0xbf, 0xf8, 0x0f, 0x8b, 0x13, 0x5f, 0x9c, 0x2e, 0x0a, 0x3f, 0x3b, 0x5d, + 0x14, 0xfe, 0xe8, 0x74, 0x51, 0xf8, 0xe3, 0xd3, 0x45, 0xe1, 0x87, 0xbf, 0x58, 0x9c, 0xf8, 0xd9, + 0x2f, 0x16, 0x27, 0xfe, 0xe8, 0x17, 0x8b, 0x13, 0x9f, 0x4e, 0x73, 0x31, 0xbb, 0x53, 0x74, 0xed, + 0x7b, 0xf8, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x11, 0x23, 0x0b, 0xde, 0x60, 0x8f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -13613,6 +13639,26 @@ func (m *AddSSTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Blind { + i-- + if m.Blind { + 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 { @@ -19554,6 +19600,12 @@ func (m *AddSSTableRequest) Size() (n int) { if m.IngestAsWrites { n += 2 } + if m.WriteAtRequestTimestamp { + n += 2 + } + if m.Blind { + n += 2 + } return n } @@ -34340,6 +34392,46 @@ 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 Blind", 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.Blind = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 0bdfc2f97339..779a23b473ef 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1591,6 +1591,33 @@ message AddSSTableRequest { // the usual write pipeline (on-disk raft log, WAL, etc). // TODO(dt): https://github.com/cockroachdb/cockroach/issues/34579#issuecomment-544627193 bool ingest_as_writes = 5; + + // WriteAtRequestTimestamp will update all MVCC timestamps in the SST to be at + // the request timestamp, even if the request gets pushed. The writes will + // comply with MVCC, in that they cannot write below the timestamp of an + // existing key or below an intent, and they respect the closed timestamp. All + // SST entries must have a non-zero MVCC timestamp, and cannot be tombstones. + // + // Use of this parameter requires the MVCCAddSSTable version gate. + bool write_at_request_timestamp = 6; + + // Blind will write the SST blindly when WriteAtRequestTimestamp is set, + // without checking for conflicting keys and intents, and without + // synchronizing with transactions by taking out locks. This is still + // serializable because: + // + // * AddSSTable cannot be in a transaction nor write intents. + // * AddSSTable cannot be split across ranges, and is always alone in a batch. + // * Write latches provide isolation from concurrent requests. + // * The timestamp cache ensures noone has read above the write timestamp. + // * The write timestamp is globally unique. (FIXME: verify this) + // + // It also complies with MVCC, because the timestamp cache ensures noone + // has observed the absence of the value, and with the closed timestamp. + // + // However, it will not update the MVCC statistics accurately, overcounting + // any keys that already exist and setting ContainsEstimates. + bool blind = 7; } // AddSSTableResponse is the response to a AddSSTable() operation. diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 797d9803591d..1419c175ca5f 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 d87dd9156c5c..f5fb867534f4 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 6287d867642f..08447e839e85 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4181,145 +4181,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..fc6ff7ecc5e4 --- /dev/null +++ b/pkg/storage/sst.go @@ -0,0 +1,224 @@ +// 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. +// +// The given SST cannot contain intents or inline values (i.e. zero timestamps), +// nor tombstones (i.e. empty values). 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, + maxIntents int64, +) (enginepb.MVCCStats, error) { + var statsDiff enginepb.MVCCStats + var intents []roachpb.Intent + + 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 && ctx.Err() == nil { + 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. + 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 + } + } + + // If the ingested key/value pair is identical to an existing one then we + // do not consider it to be a collision. + // + // TODO(erikgrinaker): We retain this behavior mostly for backwards + // compatibility, and only use it when disallowShadowing is true. It can + // result in e.g. emitting rangefeed events below the closed timestamp, so + // it should be removed in 22.2 if possible. + if disallowShadowing && 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 the existing key has a timestamp at or above the SST key, return a + // write error. + if sstKey.Timestamp.LessEq(extKey.Timestamp) { + // FIXME: Make sure returning this is OK, in particular with 22.1 clients. + return enginepb.MVCCStats{}, roachpb.NewWriteTooOldError( + sstKey.Timestamp, extKey.Timestamp.Next()) + } + + // If requested, check that we're not shadowing a live key. + if disallowShadowing && len(extValue) != 0 { + return enginepb.MVCCStats{}, errors.Errorf( + "SST key %q shadows existing key %q", sstKey, extKey) + } + + // If we are shadowing an existing key, we must update the live stats + // accordingly to take into account the existing KV pair. + if len(extValue) != 0 { + // FIXME: Verify and test this. + statsDiff.LiveCount-- + statsDiff.LiveBytes -= int64(len(extKey.Key) + 1) + statsDiff.LiveBytes -= int64(len(extValue)) + MVCCVersionTimestampSize + } + + extIter.NextKey() + extOK, extErr = extIter.Valid() + } + + switch { + case extErr != nil: + return enginepb.MVCCStats{}, extErr + case sstErr != nil: + return enginepb.MVCCStats{}, sstErr + case len(intents) > 0: + return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} + case ctx.Err() != nil: + return enginepb.MVCCStats{}, ctx.Err() + } + + return statsDiff, nil +} + +// UpdateSSTTimestamps replaces all MVCC timestamp in the provided SST with the +// given timestamp. All keys must have an existing timestamp, otherwise an error +// is returned, to protect against accidental inclusion of intents or inline +// values. +// +// 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") + } + 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_test.go b/pkg/storage/sst_test.go new file mode 100644 index 000000000000..9a429ec3b185 --- /dev/null +++ b/pkg/storage/sst_test.go @@ -0,0 +1,168 @@ +// 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 TestCheckSSTKeyCollisionsMaxIntents(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, true, 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() + } + + 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) + } +}