From a69000658814f6768a94bfbe5f1cde2bc05f00ef Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Thu, 10 Jun 2021 16:04:28 -0400 Subject: [PATCH] kv,storage: use a BytesMonitor to track memory allocations for scans The goal is to track memory allocations for: - non-local SQL=>KV requests: this can happen with joins, multi-tenant clusters, and if ranges move between DistSQL planning and execution. - local SQL=>KV requests for the first request by a fetcher: in this case the fetcher reserves a modest 1KB which can be significantly exceeded by KV allocations. Only allocations in pebbleMVCCScanner for kv pairs and intents are tracked. The memory is released when returning from executeReadOnlyBatchWithServersideRefreshes since the chain of returns will end up in gRPC response processing and we can't hook into where that memory is released. This should still help for some cases of OOMs, and give some signal of memory overload that we can use elsewhere (e.g. admission control). The BytesMonitor is used to construct a BoundAccount that is wrapped in a narrower ScannerMemoryMonitor that is passed via the EvalContext interface. The other alternative would be for the engine to have a BytesMonitor at initialization time that it can use to construct a BoundAccount for each MVCC scan, and pass it back via MVCCScanResult. This would mean multiple BoundAccounts for a batch (since we don't want to release memory until all the requests in the batch are processed), and would be harder to extend to track additional request types compared to embedding in EvalContext. The rootSQLMonitor is reused for this memory allocation tracking. This tracking is always done for non-local requests, and for the first request by a fetcher for a local request. This is to avoid double-counting, the first request issued by a SQL fetcher only reserves 1KB, but subsequent ones have already reserved what was returned in the first response. So there is room to tighten this if we knew what had been reserved on the local client (there are complications because the batch may have been split to send to different nodes, only one of which was local). The AdmissionHeader.SourceLocation field is used to mark local requests and is set in rpc.internalClientAdapter. The first request is marked using the AdmissionHeader.NoMemoryReservedAtSource bit. Informs #19721 Release note (ops change): The memory pool used for SQL is now also used to cover KV memory used for scans. --- pkg/kv/kvserver/batcheval/cmd_get.go | 1 + pkg/kv/kvserver/batcheval/cmd_get_test.go | 12 +- pkg/kv/kvserver/batcheval/cmd_reverse_scan.go | 1 + pkg/kv/kvserver/batcheval/cmd_scan.go | 1 + pkg/kv/kvserver/batcheval/eval_context.go | 9 + pkg/kv/kvserver/replica.go | 7 + pkg/kv/kvserver/replica_eval_context_span.go | 6 + pkg/kv/kvserver/replica_read.go | 62 + pkg/kv/kvserver/store.go | 11 + pkg/kv/txn.go | 4 +- pkg/roachpb/api.pb.go | 1144 +++++++++-------- pkg/roachpb/api.proto | 30 + pkg/rpc/context.go | 5 + pkg/server/server.go | 22 + pkg/server/server_sql.go | 58 +- pkg/server/tenant.go | 5 + .../logictest/testdata/logic_test/mem_limit | 47 + pkg/sql/row/kv_batch_fetcher.go | 11 + pkg/storage/BUILD.bazel | 1 + pkg/storage/mvcc.go | 47 +- pkg/storage/pebble_mvcc_scanner.go | 80 +- pkg/storage/pebble_mvcc_scanner_test.go | 4 +- 22 files changed, 1005 insertions(+), 563 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/mem_limit diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index 5a803f6e531f..6eb7b988b543 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -52,6 +52,7 @@ func Get( Txn: h.Txn, FailOnMoreRecent: args.KeyLocking != lock.None, LocalUncertaintyLimit: cArgs.LocalUncertaintyLimit, + MemoryMonitor: cArgs.EvalCtx.GetResponseMemoryAccount(), }) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_get_test.go b/pkg/kv/kvserver/batcheval/cmd_get_test.go index dba0ffb8732d..8eadafed0e9a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_get_test.go @@ -33,7 +33,8 @@ func TestGetResumeSpan(t *testing.T) { defer db.Close() _, err := Put(ctx, db, CommandArgs{ - Header: roachpb.Header{TargetBytes: -1}, + EvalCtx: (&MockEvalCtx{}).EvalContext(), + Header: roachpb.Header{TargetBytes: -1}, Args: &roachpb.PutRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, @@ -45,7 +46,8 @@ func TestGetResumeSpan(t *testing.T) { // Case 1: Check that a negative TargetBytes causes a resume span. _, err = Get(ctx, db, CommandArgs{ - Header: roachpb.Header{TargetBytes: -1}, + EvalCtx: (&MockEvalCtx{}).EvalContext(), + Header: roachpb.Header{TargetBytes: -1}, Args: &roachpb.GetRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, @@ -62,7 +64,8 @@ func TestGetResumeSpan(t *testing.T) { resp = &roachpb.GetResponse{} // Case 2: Check that a negative MaxSpanRequestKeys causes a resume span. _, err = Get(ctx, db, CommandArgs{ - Header: roachpb.Header{MaxSpanRequestKeys: -1}, + EvalCtx: (&MockEvalCtx{}).EvalContext(), + Header: roachpb.Header{MaxSpanRequestKeys: -1}, Args: &roachpb.GetRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, @@ -79,7 +82,8 @@ func TestGetResumeSpan(t *testing.T) { resp = &roachpb.GetResponse{} // Case 3: Check that a positive limit causes a normal return. _, err = Get(ctx, db, CommandArgs{ - Header: roachpb.Header{MaxSpanRequestKeys: 10, TargetBytes: 100}, + EvalCtx: (&MockEvalCtx{}).EvalContext(), + Header: roachpb.Header{MaxSpanRequestKeys: 10, TargetBytes: 100}, Args: &roachpb.GetRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index c30ab3ef9431..5f1ae3128f0d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -47,6 +47,7 @@ func ReverseScan( TargetBytes: h.TargetBytes, FailOnMoreRecent: args.KeyLocking != lock.None, Reverse: true, + MemoryMonitor: cArgs.EvalCtx.GetResponseMemoryAccount(), } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index 131ffe52471f..0e72c9e0520b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -48,6 +48,7 @@ func Scan( TargetBytes: h.TargetBytes, FailOnMoreRecent: args.KeyLocking != lock.None, Reverse: false, + MemoryMonitor: cArgs.EvalCtx.GetResponseMemoryAccount(), } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 20281dcf90b8..c7fca32129e4 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -127,6 +128,11 @@ type EvalContext interface { // WatchForMerge arranges to block all requests until the in-progress merge // completes. Returns an error if no in-progress merge is detected. WatchForMerge(ctx context.Context) error + + // GetResponseMemoryAccount returns a memory account to be used when + // generating BatchResponses. Currently only used for MVCC scans, and only + // initialized to be a real account on those paths. + GetResponseMemoryAccount() storage.ResponseMemoryAccount } // MockEvalCtx is a dummy implementation of EvalContext for testing purposes. @@ -257,3 +263,6 @@ func (m *mockEvalCtxImpl) RevokeLease(_ context.Context, seq roachpb.LeaseSequen func (m *mockEvalCtxImpl) WatchForMerge(ctx context.Context) error { panic("unimplemented") } +func (m *mockEvalCtxImpl) GetResponseMemoryAccount() storage.ResponseMemoryAccount { + return storage.ResponseMemoryAccount{} +} diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index b1d7f7b02a7a..da307b69c7b8 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1866,6 +1866,13 @@ func (r *Replica) markSystemConfigGossipFailed() { r.mu.failureToGossipSystemConfig = true } +// GetResponseMemoryAccount implements the batcheval.EvalContext interface. +func (r *Replica) GetResponseMemoryAccount() storage.ResponseMemoryAccount { + // Return an empty account. Places where a real account is needed use a + // wrapper for Replica as the EvalContext. + return storage.ResponseMemoryAccount{} +} + func init() { tracing.RegisterTagRemapping("r", "range") } diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 882f30b5659f..7592cf9fb323 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -249,3 +250,8 @@ func (rec *SpanSetReplicaEvalContext) RevokeLease(ctx context.Context, seq roach func (rec *SpanSetReplicaEvalContext) WatchForMerge(ctx context.Context) error { return rec.i.WatchForMerge(ctx) } + +// GetResponseMemoryAccount implements the batcheval.EvalContext interface. +func (rec *SpanSetReplicaEvalContext) GetResponseMemoryAccount() storage.ResponseMemoryAccount { + return rec.i.GetResponseMemoryAccount() +} diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index ab97286ceeb6..422226d2cb59 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/kr/pretty" ) @@ -180,6 +181,25 @@ func (r *Replica) executeReadOnlyBatch( return br, nil, pErr } +// evalContextWithAccount wraps an EvalContext to provide a real +// ResponseMemoryAccount. This wrapping is conditional on various factors, and +// specific to a request (see executeReadOnlyBatchWithServersideRefreshes), +// which is why the implementation of EvalContext by Replica does not by +// default provide a real ResponseMemoryAccount. +// +// If we start using evalContextWithAccount on more code paths we should +// consider using it everywhere and lift it to an earlier point in the code. +// Then code that decides whether or not a real ResponseMemoryAccount can set +// a field instead of wrapping. +type evalContextWithAccount struct { + batcheval.EvalContext + memMonitor storage.ResponseMemoryAccount +} + +func (e evalContextWithAccount) GetResponseMemoryAccount() storage.ResponseMemoryAccount { + return e.memMonitor +} + // executeReadOnlyBatchWithServersideRefreshes invokes evaluateBatch and retries // at a higher timestamp in the event of some retriable errors if allowed by the // batch/txn. @@ -193,8 +213,50 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( ) (br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { log.Event(ctx, "executing read-only batch") + var rootMonitor *mon.BytesMonitor + // Only do memory allocation accounting if the request did not originate + // locally, or for a local request that has reserved no memory. Local + // requests (originating in DistSQL) do memory accounting before issuing the + // request. Even though the accounting for the first request in the caller + // is small (the NoMemoryReservedAtSource=true case), subsequent ones use + // the size of the response for subsequent requests (see row.txnKVFetcher). + // Note that we could additionally add an OR-clause with + // ba.AdmissionHeader.Source != FROM_SQL for the if-block that does memory + // accounting. We don't do that currently since there are some SQL requests + // that are not marked as FROM_SQL. + // + // This whole scheme could be tightened, both in terms of marking, and + // compensating for the amount of memory reserved at the source. + // + // TODO(sumeer): for multi-tenant KV we should be accounting on a per-tenant + // basis and not letting a single tenant consume all the memory (we could + // place a limit equal to total/2). + if ba.AdmissionHeader.SourceLocation != roachpb.AdmissionHeader_LOCAL || + ba.AdmissionHeader.NoMemoryReservedAtSource { + // rootMonitor will never be nil in production settings, but it can be nil + // for tests that do not have a monitor. + rootMonitor = r.store.getRootMemoryMonitorForKV() + } + var boundAccount mon.BoundAccount + if rootMonitor != nil { + boundAccount = rootMonitor.MakeBoundAccount() + // Memory is not actually released when this function returns, but at + // least the batch is fully evaluated. Ideally we would like to release + // after grpc has sent the response, but there are no interceptors at that + // stage. The interceptors execute before the response is marshaled in + // Server.processUnaryRPC by calling sendResponse. + // We are intentionally not using finalizers because they delay GC and + // because they have had bugs in the past (and can prevent GC of objects + // with cyclic references). + defer boundAccount.Close(ctx) + rec = evalContextWithAccount{ + EvalContext: rec, memMonitor: storage.ResponseMemoryAccount{B: &boundAccount}} + } + for retries := 0; ; retries++ { if retries > 0 { + // It is safe to call Clear on an uninitialized BoundAccount. + boundAccount.Clear(ctx) log.VEventf(ctx, 2, "server-side retry of batch") } br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, lul, true /* readOnly */) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index b0dc01daa456..47e6857e7538 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -64,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/limit" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -723,6 +724,10 @@ type StoreConfig struct { // subsystem. It is queried during the GC process and in the handling of // AdminVerifyProtectedTimestampRequest. ProtectedTimestampCache protectedts.Cache + + // KV Memory Monitor. Must be non-nil for production, and can be nil in some + // tests. + KVMemoryMonitor *mon.BytesMonitor } // ConsistencyTestingKnobs is a BatchEvalTestingKnobs struct used to control the @@ -2897,6 +2902,12 @@ func (s *Store) unregisterLeaseholderByID(ctx context.Context, rangeID roachpb.R } } +// getRootMemoryMonitorForKV returns a BytesMonitor to use for KV memory +// tracking. +func (s *Store) getRootMemoryMonitorForKV() *mon.BytesMonitor { + return s.cfg.KVMemoryMonitor +} + // WriteClusterVersion writes the given cluster version to the store-local // cluster version key. We only accept a raw engine to ensure we're persisting // the write durably. diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 1e4e3a449c6b..688d0ee41d1c 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -972,8 +972,10 @@ func (txn *Txn) Send( } // Some callers have not initialized ba using a Batch constructed using - // Txn.NewBatch. So we fallback to initializing here. + // Txn.NewBatch. So we fallback to partially overwriting here. + noMem := ba.AdmissionHeader.NoMemoryReservedAtSource ba.AdmissionHeader = txn.admissionHeader + ba.AdmissionHeader.NoMemoryReservedAtSource = noMem txn.mu.Lock() requestTxnID := txn.mu.ID diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 5a0160e269fd..a64e7e666405 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -371,6 +371,35 @@ func (AdmissionHeader_Source) EnumDescriptor() ([]byte, []int) { return fileDescriptor_e08772acc330f58b, []int{95, 0} } +// SourceLocation specifies physically where the call originated. LOCAL +// means the client is collocated on the same node as the server. It is set +// on codepaths that use internalClientAdapter which avoids using gRPC for +// local calls to the KV API. +type AdmissionHeader_SourceLocation int32 + +const ( + AdmissionHeader_REMOTE AdmissionHeader_SourceLocation = 0 + AdmissionHeader_LOCAL AdmissionHeader_SourceLocation = 1 +) + +var AdmissionHeader_SourceLocation_name = map[int32]string{ + 0: "REMOTE", + 1: "LOCAL", +} + +var AdmissionHeader_SourceLocation_value = map[string]int32{ + "REMOTE": 0, + "LOCAL": 1, +} + +func (x AdmissionHeader_SourceLocation) String() string { + return proto.EnumName(AdmissionHeader_SourceLocation_name, int32(x)) +} + +func (AdmissionHeader_SourceLocation) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{95, 1} +} + // RequestHeader is supplied with every storage node request. type RequestHeader struct { // The key for request. If the request operates on a range, this @@ -6259,8 +6288,27 @@ type AdmissionHeader struct { // CreateTime is equivalent to Time.UnixNano() at the creation time of this // request or a parent request. See admission.WorkInfo.Priority for details. // It is used to give preference to older requests. - CreateTime int64 `protobuf:"varint,2,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` - Source AdmissionHeader_Source `protobuf:"varint,3,opt,name=source,proto3,enum=cockroach.roachpb.AdmissionHeader_Source" json:"source,omitempty"` + CreateTime int64 `protobuf:"varint,2,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + Source AdmissionHeader_Source `protobuf:"varint,3,opt,name=source,proto3,enum=cockroach.roachpb.AdmissionHeader_Source" json:"source,omitempty"` + SourceLocation AdmissionHeader_SourceLocation `protobuf:"varint,4,opt,name=source_location,json=sourceLocation,proto3,enum=cockroach.roachpb.AdmissionHeader_SourceLocation" json:"source_location,omitempty"` + // NoMemoryReservedAtSource is set by the source/client when it has + // effectively reserved close to 0 bytes. It is read by the server only when + // SourceLocation=LOCAL, to differentiate this 0 reservation case from the + // case where the client has already reserved enough memory based on + // previous responses. In the latter case the server avoids reserving more + // since it would result in double counting. Not setting this defaults to + // turning off server reserving more memory -- this optimistic choice was + // made to err on the side of avoiding double counting in case we forget to + // instrument some calling path. + // + // NOTE: This field is a temporary field until we move to comprehensive + // accounting at the client, by reserving all the bytes for responses, and + // explicitly propagating these through DistSender to the servers, so the + // servers can (best-effort) respect these values when producing responses. + // In that future world, the local server will explicitly know what has + // already been accounted for, and can start reserving more only when it + // exceeds. + NoMemoryReservedAtSource bool `protobuf:"varint,5,opt,name=no_memory_reserved_at_source,json=noMemoryReservedAtSource,proto3" json:"no_memory_reserved_at_source,omitempty"` } func (m *AdmissionHeader) Reset() { *m = AdmissionHeader{} } @@ -6998,6 +7046,7 @@ func init() { proto.RegisterEnum("cockroach.roachpb.ResponseHeader_ResumeReason", ResponseHeader_ResumeReason_name, ResponseHeader_ResumeReason_value) proto.RegisterEnum("cockroach.roachpb.CheckConsistencyResponse_Status", CheckConsistencyResponse_Status_name, CheckConsistencyResponse_Status_value) proto.RegisterEnum("cockroach.roachpb.AdmissionHeader_Source", AdmissionHeader_Source_name, AdmissionHeader_Source_value) + proto.RegisterEnum("cockroach.roachpb.AdmissionHeader_SourceLocation", AdmissionHeader_SourceLocation_name, AdmissionHeader_SourceLocation_value) proto.RegisterType((*RequestHeader)(nil), "cockroach.roachpb.RequestHeader") proto.RegisterType((*ResponseHeader)(nil), "cockroach.roachpb.ResponseHeader") proto.RegisterType((*GetRequest)(nil), "cockroach.roachpb.GetRequest") @@ -7129,520 +7178,525 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 8195 bytes of a gzipped FileDescriptorProto + // 8277 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5f, 0x6c, 0x23, 0x49, - 0x7a, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0xa3, 0x44, 0xb6, 0x4a, 0xf3, 0x47, 0xa3, 0xdd, 0x95, 0x66, - 0x7a, 0x76, 0xfe, 0xfa, 0x96, 0xda, 0x99, 0xb9, 0xcb, 0xad, 0x77, 0xd7, 0x7b, 0x16, 0x29, 0xce, - 0x90, 0xd2, 0x48, 0xa3, 0x69, 0x52, 0x33, 0xd8, 0xf5, 0x39, 0xed, 0x56, 0x77, 0x89, 0xea, 0x13, - 0xd9, 0xcd, 0xe9, 0x6e, 0x6a, 0xc4, 0x05, 0xf2, 0x90, 0xc4, 0x70, 0xee, 0x29, 0xb8, 0x00, 0x01, - 0x7c, 0x07, 0x07, 0xc1, 0x39, 0x36, 0x92, 0x87, 0x3c, 0x24, 0x88, 0x83, 0xfc, 0x43, 0x12, 0x23, - 0x4f, 0x39, 0x04, 0x87, 0xdc, 0x19, 0xc8, 0x83, 0x11, 0x20, 0x8a, 0xa3, 0xcb, 0x43, 0x0c, 0x23, - 0x48, 0x10, 0x04, 0x30, 0xb0, 0x40, 0x82, 0xa0, 0xfe, 0xf4, 0x3f, 0xb2, 0x49, 0x51, 0xb3, 0x7d, - 0xf1, 0x02, 0x7e, 0x91, 0xd8, 0x5f, 0xd5, 0xf7, 0x75, 0xd5, 0x57, 0x55, 0x5f, 0x7d, 0xbf, 0xaa, - 0xaf, 0xaa, 0x61, 0xc1, 0xb6, 0x54, 0xed, 0xb0, 0xbb, 0xbf, 0xa6, 0x76, 0x8d, 0x52, 0xd7, 0xb6, - 0x5c, 0x0b, 0x2d, 0x68, 0x96, 0x76, 0x44, 0xc9, 0x25, 0x9e, 0xb8, 0x7c, 0xff, 0xe8, 0x78, 0xed, - 0xe8, 0xd8, 0xc1, 0xf6, 0x31, 0xb6, 0xd7, 0x34, 0xcb, 0xd4, 0x7a, 0xb6, 0x8d, 0x4d, 0xad, 0xbf, - 0xd6, 0xb6, 0xb4, 0x23, 0xfa, 0xc7, 0x30, 0x5b, 0x8c, 0x3d, 0x9a, 0xd7, 0xc6, 0xaa, 0xee, 0xf4, - 0x3a, 0x1d, 0xd5, 0xee, 0xaf, 0xd9, 0x4e, 0x77, 0x7f, 0x8d, 0x3f, 0xf0, 0xbc, 0xc8, 0x7b, 0xbb, - 0xae, 0xba, 0x2a, 0xa7, 0x5d, 0xf2, 0x68, 0xd8, 0xb6, 0x2d, 0xdb, 0xe1, 0xd4, 0x2b, 0x1e, 0xb5, - 0x83, 0x5d, 0x35, 0x94, 0xfb, 0x2d, 0xc7, 0xb5, 0x6c, 0xb5, 0x85, 0xd7, 0xb0, 0xd9, 0x32, 0x4c, - 0x4c, 0x32, 0x1c, 0x6b, 0x1a, 0x4f, 0x7c, 0x3b, 0x36, 0xf1, 0x11, 0x4f, 0x5d, 0xea, 0xb9, 0x46, - 0x7b, 0xed, 0xb0, 0xad, 0xad, 0xb9, 0x46, 0x07, 0x3b, 0xae, 0xda, 0xe9, 0x7a, 0x55, 0xa0, 0x29, - 0xae, 0xad, 0x6a, 0x86, 0xd9, 0xf2, 0xfe, 0x77, 0xf7, 0xd7, 0x6c, 0xac, 0x59, 0xb6, 0x8e, 0x75, - 0xc5, 0xe9, 0xaa, 0xa6, 0x57, 0xdc, 0x96, 0xd5, 0xb2, 0xe8, 0xcf, 0x35, 0xf2, 0x8b, 0x53, 0x57, - 0x5a, 0x96, 0xd5, 0x6a, 0xe3, 0x35, 0xfa, 0xb4, 0xdf, 0x3b, 0x58, 0xd3, 0x7b, 0xb6, 0xea, 0x1a, - 0x16, 0xe7, 0x92, 0xfe, 0x89, 0x00, 0xf3, 0x32, 0x7e, 0xd5, 0xc3, 0x8e, 0x5b, 0xc3, 0xaa, 0x8e, - 0x6d, 0x74, 0x0d, 0xd2, 0x47, 0xb8, 0xbf, 0x94, 0xbe, 0x2e, 0xdc, 0x9d, 0x2b, 0xcf, 0x7e, 0x71, - 0xba, 0x9a, 0xde, 0xc2, 0x7d, 0x99, 0xd0, 0xd0, 0x75, 0x98, 0xc5, 0xa6, 0xae, 0x90, 0xe4, 0x4c, - 0x34, 0x79, 0x06, 0x9b, 0xfa, 0x16, 0xee, 0xa3, 0x6f, 0x43, 0xd6, 0x21, 0xd2, 0x4c, 0x0d, 0x2f, - 0x4d, 0x5f, 0x17, 0xee, 0x4e, 0x97, 0x7f, 0xf9, 0x8b, 0xd3, 0xd5, 0x8f, 0x5b, 0x86, 0x7b, 0xd8, - 0xdb, 0x2f, 0x69, 0x56, 0x67, 0xcd, 0x6f, 0x53, 0x7d, 0x3f, 0xf8, 0xbd, 0xd6, 0x3d, 0x6a, 0xad, - 0x0d, 0xea, 0xa8, 0xd4, 0x3c, 0x31, 0x1b, 0xf8, 0x95, 0xec, 0x4b, 0xdc, 0xcc, 0x64, 0x05, 0x31, - 0xb5, 0x99, 0xc9, 0xa6, 0xc4, 0xb4, 0xf4, 0x93, 0x14, 0x14, 0x64, 0xec, 0x74, 0x2d, 0xd3, 0xc1, - 0xbc, 0xe4, 0xef, 0x43, 0xda, 0x3d, 0x31, 0x69, 0xc9, 0xf3, 0x0f, 0x57, 0x4a, 0x43, 0xbd, 0xa7, - 0xd4, 0xb4, 0x55, 0xd3, 0x51, 0x35, 0x52, 0x7d, 0x99, 0x64, 0x45, 0x1f, 0x40, 0xde, 0xc6, 0x4e, - 0xaf, 0x83, 0xa9, 0x22, 0x69, 0xa5, 0xf2, 0x0f, 0xaf, 0xc6, 0x70, 0x36, 0xba, 0xaa, 0x29, 0x03, - 0xcb, 0x4b, 0x7e, 0xa3, 0x06, 0xcc, 0x73, 0x4e, 0x1b, 0xab, 0x8e, 0x65, 0x2e, 0xcd, 0x5e, 0x17, - 0xee, 0x16, 0x1e, 0x96, 0x62, 0x78, 0xa3, 0xa5, 0x24, 0x8f, 0xbd, 0x0e, 0x96, 0x29, 0x97, 0x3c, - 0x67, 0x87, 0x9e, 0xd0, 0x35, 0xc8, 0x9a, 0xbd, 0x0e, 0xd1, 0xaf, 0x43, 0xb5, 0x97, 0x96, 0x67, - 0xcd, 0x5e, 0x67, 0x0b, 0xf7, 0x1d, 0xf4, 0x16, 0xe4, 0x48, 0xd2, 0x7e, 0xdf, 0xc5, 0xce, 0x52, - 0x96, 0xa6, 0x91, 0xbc, 0x65, 0xf2, 0x2c, 0x7d, 0x02, 0x73, 0x61, 0xa9, 0x08, 0x41, 0x41, 0xae, - 0x36, 0xf6, 0xb6, 0xab, 0xca, 0xde, 0xce, 0xd6, 0xce, 0xb3, 0x97, 0x3b, 0xe2, 0x14, 0xba, 0x04, - 0x22, 0xa7, 0x6d, 0x55, 0x3f, 0x55, 0x9e, 0xd6, 0xb7, 0xeb, 0x4d, 0x51, 0x58, 0xce, 0x7c, 0xf7, - 0x77, 0x56, 0xa6, 0x36, 0x33, 0xd9, 0x19, 0x71, 0x56, 0xfa, 0x1d, 0x01, 0xe0, 0x09, 0x76, 0x79, - 0x6f, 0x40, 0x65, 0x98, 0x39, 0xa4, 0x25, 0x5e, 0x12, 0xa8, 0x5a, 0xae, 0xc7, 0x56, 0x2d, 0xd4, - 0x73, 0xca, 0xd9, 0x1f, 0x9d, 0xae, 0x4e, 0xfd, 0xf4, 0x74, 0x55, 0x90, 0x39, 0x27, 0x7a, 0x0e, - 0xf9, 0x23, 0xdc, 0x57, 0xf8, 0xb8, 0x5c, 0x4a, 0x51, 0x1d, 0xbd, 0x1f, 0x12, 0x74, 0x74, 0x5c, - 0xf2, 0x86, 0x68, 0x29, 0x34, 0x9c, 0x4b, 0x84, 0xa3, 0xd4, 0x70, 0x6d, 0x6c, 0xb6, 0xdc, 0x43, - 0x19, 0x8e, 0x70, 0xff, 0x29, 0x93, 0x21, 0xfd, 0xbe, 0x00, 0x79, 0x5a, 0x4a, 0xa6, 0x54, 0x54, - 0x19, 0x28, 0xe6, 0x8d, 0x73, 0x5b, 0x20, 0xa6, 0x9c, 0x25, 0x98, 0x3e, 0x56, 0xdb, 0x3d, 0x4c, - 0x4b, 0x98, 0x7f, 0xb8, 0x14, 0x23, 0xe3, 0x05, 0x49, 0x97, 0x59, 0x36, 0xf4, 0x11, 0xcc, 0x19, - 0xa6, 0x8b, 0x4d, 0x57, 0x61, 0x6c, 0xe9, 0x73, 0xd8, 0xf2, 0x2c, 0x37, 0x7d, 0x90, 0xfe, 0xb1, - 0x00, 0xb0, 0xdb, 0x4b, 0x54, 0xcf, 0x5f, 0x9f, 0xb0, 0xfc, 0xe5, 0x0c, 0x61, 0xf5, 0x6a, 0x71, - 0x05, 0x66, 0x0c, 0xb3, 0x6d, 0x98, 0xac, 0xfc, 0x59, 0x99, 0x3f, 0xa1, 0x4b, 0x30, 0xbd, 0xdf, - 0x36, 0x4c, 0x9d, 0x8e, 0x87, 0xac, 0xcc, 0x1e, 0x24, 0x19, 0xf2, 0xb4, 0xd4, 0x09, 0xea, 0x5d, - 0x3a, 0x4d, 0xc1, 0xe5, 0x8a, 0x65, 0xea, 0x06, 0x19, 0x92, 0x6a, 0xfb, 0x2b, 0xa1, 0x95, 0x4d, - 0xb8, 0xa4, 0xe3, 0xae, 0x8d, 0x35, 0xd5, 0xc5, 0xba, 0x82, 0x4f, 0xba, 0x13, 0xb6, 0x31, 0x0a, - 0xb8, 0xaa, 0x27, 0x5d, 0x4a, 0x23, 0xa3, 0x96, 0x08, 0x60, 0xa3, 0x76, 0x86, 0x98, 0x4c, 0x39, - 0x8b, 0x4f, 0xba, 0x74, 0xd4, 0xc6, 0xab, 0x19, 0x7d, 0x1d, 0xae, 0xaa, 0xed, 0xb6, 0xf5, 0x5a, - 0x31, 0x0e, 0x14, 0xdd, 0xc2, 0x8e, 0x62, 0x5a, 0xae, 0x82, 0x4f, 0x0c, 0xc7, 0xa5, 0x26, 0x21, - 0x2b, 0x2f, 0xd2, 0xe4, 0xfa, 0xc1, 0x86, 0x85, 0x9d, 0x1d, 0xcb, 0xad, 0x92, 0xa4, 0x50, 0x53, - 0xce, 0x86, 0x9b, 0x52, 0xfa, 0x55, 0xb8, 0x32, 0xa8, 0xdf, 0x24, 0xdb, 0xef, 0xc7, 0x02, 0x14, - 0xea, 0xa6, 0xe1, 0x7e, 0x25, 0x1a, 0xce, 0xd7, 0x67, 0x3a, 0xac, 0xcf, 0xfb, 0x20, 0x1e, 0xa8, - 0x46, 0xfb, 0x99, 0xd9, 0xb4, 0x3a, 0xfb, 0x8e, 0x6b, 0x99, 0xd8, 0xe1, 0x0a, 0x1f, 0xa2, 0x4b, - 0x2f, 0xa0, 0xe8, 0xd7, 0x26, 0x49, 0x35, 0xb9, 0x20, 0xd6, 0x4d, 0xcd, 0xc6, 0x1d, 0x6c, 0x26, - 0xaa, 0xa7, 0xb7, 0x21, 0x67, 0x78, 0x72, 0xa9, 0xae, 0xd2, 0x72, 0x40, 0x90, 0x7a, 0xb0, 0x10, - 0x7a, 0x6b, 0x92, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0xb5, 0x12, 0xb4, 0x11, 0x99, 0x8c, 0xf0, 0x6b, - 0x66, 0xde, 0x1a, 0x30, 0xbf, 0x81, 0xdb, 0xd8, 0xc5, 0x09, 0xd6, 0x54, 0xda, 0x83, 0x82, 0x27, - 0x34, 0xc9, 0x86, 0xf9, 0x4d, 0x01, 0x10, 0x97, 0xab, 0x9a, 0xad, 0x24, 0x4b, 0x8c, 0x56, 0x89, - 0x6b, 0xe1, 0xf6, 0x6c, 0x93, 0x4d, 0xe7, 0xac, 0x4f, 0x02, 0x23, 0xd1, 0x19, 0x3d, 0x18, 0xb2, - 0x99, 0xf0, 0x90, 0xe5, 0xee, 0xcd, 0x6b, 0x58, 0x8c, 0x14, 0x2c, 0xd9, 0xe6, 0xcb, 0xd0, 0x32, - 0xa5, 0xae, 0xa7, 0xc3, 0x3e, 0x1c, 0x25, 0x4a, 0xdf, 0x17, 0x60, 0xa1, 0xd2, 0xc6, 0xaa, 0x9d, - 0xb8, 0x46, 0xbe, 0x05, 0x59, 0x1d, 0xab, 0x3a, 0xad, 0x32, 0x1b, 0xd8, 0xef, 0x84, 0xa4, 0x10, - 0x4f, 0xb7, 0x74, 0xd8, 0xd6, 0x4a, 0x4d, 0xcf, 0x07, 0xe6, 0xa3, 0xdb, 0x67, 0x92, 0x3e, 0x05, - 0x14, 0x2e, 0x59, 0x92, 0x1d, 0xe1, 0x77, 0x53, 0x80, 0x64, 0x7c, 0x8c, 0x6d, 0x37, 0xf1, 0x6a, - 0x6f, 0x40, 0xde, 0x55, 0xed, 0x16, 0x76, 0x15, 0xe2, 0xdd, 0x5f, 0xa4, 0xe6, 0xc0, 0xf8, 0x08, - 0x19, 0x35, 0xe1, 0x0e, 0x36, 0xd5, 0xfd, 0x36, 0xa6, 0x52, 0x94, 0x7d, 0xab, 0x67, 0xea, 0x8a, - 0xe1, 0x62, 0x5b, 0x75, 0x2d, 0x5b, 0xb1, 0xba, 0xae, 0xd1, 0x31, 0x3e, 0xa7, 0x8e, 0x3d, 0xef, - 0x6a, 0x37, 0x59, 0x76, 0xc2, 0x5c, 0x26, 0x99, 0xeb, 0x3c, 0xef, 0xb3, 0x50, 0x56, 0x54, 0x82, - 0x45, 0xa3, 0x65, 0x5a, 0x36, 0x56, 0x5a, 0x9a, 0xe2, 0x1e, 0xda, 0xd8, 0x39, 0xb4, 0xda, 0xde, - 0x84, 0xb4, 0xc0, 0x92, 0x9e, 0x68, 0x4d, 0x2f, 0x41, 0xfa, 0x0c, 0x16, 0x23, 0x5a, 0x4a, 0xb2, - 0x09, 0xfe, 0xa7, 0x00, 0xf9, 0x86, 0xa6, 0x9a, 0x49, 0xea, 0xfe, 0x13, 0xc8, 0x3b, 0x9a, 0x6a, - 0x2a, 0x07, 0x96, 0xdd, 0x51, 0x5d, 0x5a, 0xaf, 0x42, 0x44, 0xf7, 0xbe, 0x7f, 0xaf, 0xa9, 0xe6, - 0x63, 0x9a, 0x49, 0x06, 0xc7, 0xff, 0x3d, 0xe8, 0xbf, 0x4e, 0x7f, 0x79, 0xff, 0x95, 0x0d, 0xef, - 0xcd, 0x4c, 0x36, 0x2d, 0x66, 0xa4, 0x3f, 0x15, 0x60, 0x8e, 0x55, 0x39, 0xc9, 0xe1, 0xfd, 0x0d, - 0xc8, 0xd8, 0xd6, 0x6b, 0x36, 0xbc, 0xf3, 0x0f, 0xdf, 0x8a, 0x11, 0xb1, 0x85, 0xfb, 0xe1, 0xf9, - 0x93, 0x66, 0x47, 0x65, 0xe0, 0x5e, 0xaa, 0x42, 0xb9, 0xd3, 0x93, 0x72, 0x03, 0xe3, 0x92, 0x89, - 0x8c, 0x3b, 0x50, 0xdc, 0x57, 0x5d, 0xed, 0x50, 0xb1, 0x79, 0x21, 0xc9, 0x5c, 0x9b, 0xbe, 0x3b, - 0x27, 0x17, 0x28, 0xd9, 0x2b, 0xba, 0x43, 0x6a, 0xce, 0xc6, 0x9b, 0x83, 0xff, 0x9c, 0xb5, 0xf9, - 0xff, 0x15, 0xf8, 0x18, 0xf2, 0x6a, 0xfe, 0xe7, 0xad, 0xe9, 0x7f, 0x90, 0x82, 0xab, 0x95, 0x43, - 0xac, 0x1d, 0x55, 0x2c, 0xd3, 0x31, 0x1c, 0x97, 0xe8, 0x2e, 0xc9, 0xf6, 0x7f, 0x0b, 0x72, 0xaf, - 0x0d, 0xf7, 0x50, 0xd1, 0x8d, 0x83, 0x03, 0x6a, 0x6d, 0xb3, 0x72, 0x96, 0x10, 0x36, 0x8c, 0x83, - 0x03, 0xf4, 0x08, 0x32, 0x1d, 0x4b, 0x67, 0xce, 0x7c, 0xe1, 0xe1, 0x6a, 0x8c, 0x78, 0x5a, 0x34, - 0xa7, 0xd7, 0xd9, 0xb6, 0x74, 0x2c, 0xd3, 0xcc, 0x68, 0x05, 0x40, 0x23, 0xd4, 0xae, 0x65, 0x98, - 0x2e, 0x37, 0x8e, 0x21, 0x0a, 0xaa, 0x41, 0xce, 0xc5, 0x76, 0xc7, 0x30, 0x55, 0x17, 0x2f, 0x4d, - 0x53, 0xe5, 0xbd, 0x1b, 0x5b, 0xf0, 0x6e, 0xdb, 0xd0, 0xd4, 0x0d, 0xec, 0x68, 0xb6, 0xd1, 0x75, - 0x2d, 0x9b, 0x6b, 0x31, 0x60, 0x96, 0xfe, 0x7a, 0x06, 0x96, 0x86, 0x75, 0x93, 0x64, 0x0f, 0xd9, - 0x85, 0x19, 0x1b, 0x3b, 0xbd, 0xb6, 0xcb, 0xfb, 0xc8, 0xc3, 0x51, 0x2a, 0x88, 0x29, 0x01, 0x5d, - 0xba, 0x68, 0xbb, 0xbc, 0xd8, 0x5c, 0xce, 0xf2, 0xbf, 0x14, 0x60, 0x86, 0x25, 0xa0, 0x07, 0x90, - 0xb5, 0xc9, 0xc4, 0xa0, 0x18, 0x3a, 0x2d, 0x63, 0xba, 0x7c, 0xe5, 0xec, 0x74, 0x75, 0x96, 0x4e, - 0x16, 0xf5, 0x8d, 0x2f, 0x82, 0x9f, 0xf2, 0x2c, 0xcd, 0x57, 0xd7, 0x49, 0x6b, 0x39, 0xae, 0x6a, - 0xbb, 0x74, 0x51, 0x29, 0xc5, 0x10, 0x12, 0x25, 0x6c, 0xe1, 0x3e, 0xda, 0x84, 0x19, 0xc7, 0x55, - 0xdd, 0x9e, 0xc3, 0xdb, 0xeb, 0x42, 0x85, 0x6d, 0x50, 0x4e, 0x99, 0x4b, 0x20, 0xee, 0x96, 0x8e, - 0x5d, 0xd5, 0x68, 0xd3, 0x06, 0xcc, 0xc9, 0xfc, 0x49, 0xfa, 0x2d, 0x01, 0x66, 0x58, 0x56, 0x74, - 0x15, 0x16, 0xe5, 0xf5, 0x9d, 0x27, 0x55, 0xa5, 0xbe, 0xb3, 0x51, 0x6d, 0x56, 0xe5, 0xed, 0xfa, - 0xce, 0x7a, 0xb3, 0x2a, 0x4e, 0xa1, 0x2b, 0x80, 0xbc, 0x84, 0xca, 0xb3, 0x9d, 0x46, 0xbd, 0xd1, - 0xac, 0xee, 0x34, 0x45, 0x81, 0xae, 0xa9, 0x50, 0x7a, 0x88, 0x9a, 0x42, 0xef, 0xc2, 0xf5, 0x41, - 0xaa, 0xd2, 0x68, 0xae, 0x37, 0x1b, 0x4a, 0xb5, 0xd1, 0xac, 0x6f, 0xaf, 0x37, 0xab, 0x1b, 0x62, - 0x7a, 0x4c, 0x2e, 0xf2, 0x12, 0x59, 0xae, 0x56, 0x9a, 0x62, 0x46, 0x72, 0xe1, 0xb2, 0x8c, 0x35, - 0xab, 0xd3, 0xed, 0xb9, 0x98, 0x94, 0xd2, 0x49, 0x72, 0xa4, 0x5c, 0x85, 0x59, 0xdd, 0xee, 0x2b, - 0x76, 0xcf, 0xe4, 0xe3, 0x64, 0x46, 0xb7, 0xfb, 0x72, 0xcf, 0x94, 0xfe, 0xa1, 0x00, 0x57, 0x06, - 0x5f, 0x9b, 0x64, 0x27, 0x7c, 0x0e, 0x79, 0x55, 0xd7, 0xb1, 0xae, 0xe8, 0xb8, 0xed, 0xaa, 0xdc, - 0x25, 0xba, 0x1f, 0x92, 0xc4, 0x97, 0x02, 0x4b, 0xfe, 0x52, 0xe0, 0xf6, 0x8b, 0x4a, 0x85, 0x16, - 0x64, 0x83, 0x70, 0x78, 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xe9, 0x07, 0x19, 0x98, 0xaf, 0x9a, 0x7a, - 0xf3, 0x24, 0xd1, 0xb9, 0xe4, 0x0a, 0xcc, 0x68, 0x56, 0xa7, 0x63, 0xb8, 0x9e, 0x82, 0xd8, 0x13, - 0xfa, 0xc5, 0x90, 0x2b, 0x9b, 0x9e, 0xc0, 0xa1, 0x0b, 0x9c, 0x58, 0xf4, 0x6b, 0x70, 0x95, 0x58, - 0x4d, 0xdb, 0x54, 0xdb, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xd1, 0x6a, 0x61, 0x9b, 0x2f, 0x3f, 0xde, - 0x8d, 0x29, 0x67, 0x9d, 0x73, 0x54, 0x28, 0x43, 0x93, 0xe5, 0x97, 0x2f, 0x1b, 0x71, 0x64, 0xf4, - 0x31, 0x00, 0x99, 0x8a, 0xe8, 0x92, 0xa6, 0xc3, 0xed, 0xd1, 0xa8, 0x35, 0x4d, 0xcf, 0x04, 0x11, - 0x06, 0xf2, 0xec, 0xa0, 0xe7, 0x20, 0x1a, 0xa6, 0x72, 0xd0, 0x36, 0x5a, 0x87, 0xae, 0xf2, 0xda, - 0x36, 0x5c, 0xec, 0x2c, 0x2d, 0x50, 0x19, 0x71, 0x4d, 0xdd, 0xe0, 0x4b, 0xb3, 0xfa, 0x4b, 0x92, - 0x93, 0x4b, 0x2b, 0x18, 0xe6, 0x63, 0xca, 0x4f, 0x89, 0x0e, 0x5a, 0x23, 0x50, 0xe8, 0x55, 0xcf, - 0xb0, 0xb1, 0xf2, 0xa0, 0xab, 0xd1, 0x75, 0x90, 0x6c, 0xb9, 0x70, 0x76, 0xba, 0x0a, 0x32, 0x23, - 0x3f, 0xd8, 0xad, 0x10, 0x68, 0xc4, 0x7e, 0x77, 0x35, 0xa2, 0xf6, 0xae, 0x65, 0x38, 0x96, 0xb9, - 0x94, 0x63, 0x6a, 0x67, 0x4f, 0xe8, 0x1e, 0x88, 0xee, 0x89, 0xa9, 0x1c, 0x62, 0xd5, 0x76, 0xf7, - 0xb1, 0xea, 0x92, 0xf9, 0x19, 0x68, 0x8e, 0xa2, 0x7b, 0x62, 0xd6, 0x42, 0xe4, 0xcd, 0x4c, 0x76, - 0x56, 0xcc, 0x6e, 0x66, 0xb2, 0x59, 0x31, 0x27, 0xfd, 0x27, 0x01, 0x0a, 0x5e, 0xdf, 0x48, 0xb2, - 0x1b, 0xdf, 0x05, 0xd1, 0x32, 0xb1, 0xd2, 0x3d, 0x54, 0x1d, 0xcc, 0xdb, 0x92, 0xcf, 0x0e, 0x05, - 0xcb, 0xc4, 0xbb, 0x84, 0xcc, 0x5a, 0x06, 0xed, 0xc2, 0x82, 0xe3, 0xaa, 0x2d, 0xc3, 0x6c, 0x29, - 0xfe, 0x12, 0x3f, 0xf5, 0x2c, 0x26, 0x44, 0x02, 0x22, 0xe7, 0xf6, 0xe9, 0x11, 0x97, 0xe2, 0x0f, - 0x04, 0x58, 0x58, 0xd7, 0x3b, 0x86, 0xd9, 0xe8, 0xb6, 0x8d, 0x44, 0x17, 0x18, 0xde, 0x85, 0x9c, - 0x43, 0x64, 0x06, 0xd6, 0x39, 0x80, 0x8b, 0x59, 0x9a, 0x42, 0xcc, 0xf4, 0x53, 0x28, 0xe2, 0x93, - 0xae, 0xc1, 0xf6, 0x15, 0x18, 0xca, 0xc9, 0x4c, 0x5e, 0xb7, 0x42, 0xc0, 0x4b, 0x92, 0x78, 0x9d, - 0x3e, 0x05, 0x14, 0xae, 0x52, 0x92, 0x40, 0xe3, 0x53, 0x58, 0xa4, 0xa2, 0xf7, 0x4c, 0x27, 0x61, - 0x7d, 0x49, 0xbf, 0x02, 0x97, 0xa2, 0xa2, 0x93, 0x2c, 0xf7, 0x4b, 0xde, 0xca, 0xdb, 0xd8, 0x4e, - 0x14, 0xa1, 0xfa, 0xba, 0xe6, 0x82, 0x93, 0x2c, 0xf3, 0xaf, 0x0b, 0x70, 0x8d, 0xca, 0xa6, 0x5b, - 0x2f, 0x07, 0xd8, 0x7e, 0x8a, 0x55, 0x27, 0x51, 0x78, 0x7d, 0x13, 0x66, 0x18, 0x4c, 0xa6, 0xfd, - 0x73, 0xba, 0x9c, 0x27, 0x6e, 0x46, 0xc3, 0xb5, 0x6c, 0xe2, 0x66, 0xf0, 0x24, 0x49, 0x85, 0xe5, - 0xb8, 0x52, 0x24, 0x59, 0xd3, 0xbf, 0x2d, 0xc0, 0x02, 0xf7, 0xf0, 0x48, 0x57, 0xae, 0x1c, 0x12, - 0x07, 0x07, 0x55, 0x21, 0xaf, 0xd1, 0x5f, 0x8a, 0xdb, 0xef, 0x62, 0x2a, 0xbf, 0x30, 0xce, 0x39, - 0x64, 0x6c, 0xcd, 0x7e, 0x17, 0x13, 0x0f, 0xd3, 0xfb, 0x4d, 0x14, 0x15, 0xaa, 0xe4, 0x58, 0xf7, - 0x92, 0x8e, 0x23, 0x9a, 0xd7, 0xf3, 0xd3, 0xb8, 0x0e, 0x7e, 0x2f, 0xcd, 0x95, 0xc0, 0xde, 0xc1, - 0xb3, 0x27, 0xea, 0x50, 0x7c, 0x06, 0x57, 0x42, 0x4b, 0xe7, 0xe1, 0x8a, 0xa7, 0x2e, 0x50, 0xf1, - 0xd0, 0xf2, 0x7b, 0x40, 0x45, 0x9f, 0x42, 0x68, 0x81, 0x5d, 0x61, 0x75, 0xf2, 0xa0, 0xca, 0x45, - 0xd4, 0xb1, 0x10, 0x48, 0x61, 0x74, 0x07, 0x55, 0x20, 0x8b, 0x4f, 0xba, 0x8a, 0x8e, 0x1d, 0x8d, - 0x1b, 0x2e, 0x29, 0x4e, 0x20, 0x29, 0xca, 0x90, 0xf3, 0x3e, 0x8b, 0x4f, 0xba, 0x84, 0x88, 0xf6, - 0xc8, 0xbc, 0xe9, 0xcd, 0xeb, 0xb4, 0xd8, 0xce, 0xf9, 0x58, 0x20, 0xe8, 0x29, 0x5c, 0x5c, 0xd1, - 0x9f, 0xd2, 0x99, 0x08, 0xe9, 0x87, 0x02, 0xbc, 0x15, 0xdb, 0x6a, 0x49, 0x4e, 0x64, 0x1f, 0x43, - 0x86, 0x56, 0x3e, 0x75, 0xc1, 0xca, 0x53, 0x2e, 0xe9, 0xbb, 0x29, 0x3e, 0xc6, 0x65, 0xdc, 0xb6, - 0x88, 0x62, 0x13, 0x5f, 0x42, 0x7b, 0x06, 0xf3, 0xc7, 0x96, 0x8b, 0x6d, 0xbf, 0xd9, 0x53, 0x17, - 0x6e, 0xf6, 0x39, 0x2a, 0xc0, 0x6b, 0xf1, 0x17, 0xb0, 0x60, 0x5a, 0xa6, 0x12, 0x15, 0x7a, 0xf1, - 0xbe, 0x54, 0x34, 0x2d, 0xf3, 0x45, 0x48, 0xae, 0x6f, 0x67, 0x06, 0x34, 0x91, 0xa4, 0x9d, 0xf9, - 0x9e, 0x00, 0x8b, 0xbe, 0xa7, 0x93, 0xb0, 0xbb, 0xfb, 0x0d, 0x48, 0x9b, 0xd6, 0xeb, 0x8b, 0x2c, - 0x51, 0x92, 0xfc, 0x64, 0xd6, 0x8b, 0x96, 0x28, 0xc9, 0xfa, 0xfe, 0xab, 0x14, 0xe4, 0x9e, 0x54, - 0x92, 0xac, 0xe5, 0xc7, 0x7c, 0xf9, 0x9b, 0xb5, 0x77, 0x5c, 0x6f, 0xf7, 0xdf, 0x57, 0x7a, 0x52, - 0xd9, 0xc2, 0x7d, 0xaf, 0xb7, 0x13, 0x2e, 0xb4, 0x0e, 0xb9, 0xe8, 0x42, 0xe9, 0x84, 0x9a, 0x0a, - 0xb8, 0x96, 0x31, 0x4c, 0x53, 0xb9, 0x5e, 0xa8, 0x85, 0x10, 0x13, 0x6a, 0x41, 0x5e, 0xe3, 0x7b, - 0x8a, 0xa9, 0x8b, 0xbc, 0x26, 0xe4, 0x22, 0x4e, 0x8b, 0x33, 0xd2, 0x73, 0x00, 0x52, 0x9d, 0x24, - 0x9b, 0xe4, 0x37, 0xd2, 0x50, 0xd8, 0xed, 0x39, 0x87, 0x09, 0xf7, 0xbe, 0x0a, 0x40, 0xb7, 0xe7, - 0x1c, 0x92, 0x11, 0x79, 0x62, 0xf2, 0x3a, 0x9f, 0x13, 0xc5, 0xe1, 0x55, 0x9a, 0xf1, 0x35, 0x4f, - 0x4c, 0x54, 0xe3, 0x42, 0xb0, 0x12, 0x84, 0x82, 0xdc, 0x1c, 0x87, 0x2c, 0x9b, 0x27, 0xe6, 0x36, - 0xf6, 0x21, 0x25, 0x93, 0x84, 0x89, 0xa4, 0x8f, 0x61, 0x96, 0x3c, 0x28, 0xae, 0x75, 0x91, 0x66, - 0x9e, 0x21, 0x3c, 0x4d, 0x0b, 0x7d, 0x04, 0x39, 0xc6, 0x4d, 0x66, 0xbf, 0x19, 0x3a, 0xfb, 0xc5, - 0xd5, 0x85, 0xab, 0x91, 0xce, 0x7b, 0x59, 0xca, 0x4a, 0xe6, 0xba, 0x4b, 0x30, 0x7d, 0x60, 0xd9, - 0x9a, 0xb7, 0x99, 0xcb, 0x1e, 0x58, 0x7b, 0x32, 0x48, 0xb3, 0x99, 0xc9, 0xe6, 0x44, 0x90, 0x7e, - 0x4b, 0x80, 0xa2, 0xdf, 0x10, 0x49, 0x4e, 0x08, 0x95, 0x88, 0x16, 0x2f, 0xde, 0x14, 0x44, 0x81, - 0xd2, 0xbf, 0xa5, 0x1e, 0x91, 0x66, 0x1d, 0xd3, 0x96, 0x49, 0xb2, 0xa7, 0x7c, 0xc4, 0x02, 0x7d, - 0x52, 0x17, 0x6d, 0x5d, 0x1a, 0xf3, 0xf3, 0x00, 0x2e, 0x19, 0x1d, 0x62, 0xcf, 0x0d, 0xb7, 0xdd, - 0xe7, 0xb0, 0xcd, 0xc5, 0xde, 0xae, 0xf1, 0x62, 0x90, 0x56, 0xf1, 0x92, 0xa4, 0xdf, 0xa5, 0xab, - 0xd5, 0x41, 0x4d, 0x92, 0x54, 0x75, 0x1d, 0xe6, 0x6d, 0x26, 0x9a, 0xb8, 0x35, 0x17, 0xd4, 0xf6, - 0x9c, 0xcf, 0x4a, 0x14, 0xfe, 0xdb, 0x29, 0x28, 0x3e, 0xef, 0x61, 0xbb, 0xff, 0x55, 0x52, 0xf7, - 0x6d, 0x28, 0xbe, 0x56, 0x0d, 0x57, 0x39, 0xb0, 0x6c, 0xa5, 0xd7, 0xd5, 0x55, 0xd7, 0x8b, 0x36, - 0x99, 0x27, 0xe4, 0xc7, 0x96, 0xbd, 0x47, 0x89, 0x08, 0x03, 0x3a, 0x32, 0xad, 0xd7, 0xa6, 0x42, - 0xc8, 0x14, 0x28, 0x9f, 0x98, 0x7c, 0x09, 0xb9, 0xfc, 0xcd, 0xff, 0x78, 0xba, 0xfa, 0x68, 0xa2, - 0x18, 0x32, 0x1a, 0x2f, 0xd7, 0xeb, 0x19, 0x7a, 0x69, 0x6f, 0xaf, 0xbe, 0x21, 0x8b, 0x54, 0xe4, - 0x4b, 0x26, 0xb1, 0x79, 0x62, 0x3a, 0xd2, 0xdf, 0x4d, 0x81, 0x18, 0xe8, 0x28, 0xc9, 0x86, 0xac, - 0x42, 0xfe, 0x55, 0x0f, 0xdb, 0xc6, 0x1b, 0x34, 0x23, 0x70, 0x46, 0x62, 0x76, 0xee, 0xc3, 0x82, - 0x7b, 0x62, 0x2a, 0x2c, 0xc2, 0x8f, 0x05, 0x7e, 0x78, 0x01, 0x0b, 0x45, 0x97, 0x94, 0x99, 0xd0, - 0x69, 0xd0, 0x87, 0x83, 0x3e, 0x83, 0xb9, 0x88, 0xb6, 0xd2, 0x5f, 0x4e, 0x5b, 0xf9, 0xd7, 0x21, - 0x45, 0xfd, 0xbe, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0xd6, 0xf8, 0xbf, 0x2a, 0xfd, 0xe9, 0x2e, 0x88, - 0x34, 0x1e, 0x53, 0x31, 0x0e, 0x94, 0x8e, 0xe1, 0x38, 0x86, 0xd9, 0xe2, 0x1d, 0xaa, 0x40, 0xe9, - 0xf5, 0x83, 0x6d, 0x46, 0x95, 0xfe, 0x12, 0x2c, 0x46, 0x2a, 0x90, 0x64, 0x63, 0xdf, 0x80, 0xb9, - 0x03, 0xb6, 0x05, 0x4b, 0x85, 0xf3, 0xe5, 0xc1, 0x3c, 0xa5, 0xb1, 0xf7, 0x49, 0x7f, 0x92, 0x82, - 0x4b, 0x32, 0x76, 0xac, 0xf6, 0x31, 0x4e, 0x5e, 0x85, 0x35, 0xe0, 0x7b, 0x2f, 0xca, 0x1b, 0x69, - 0x32, 0xc7, 0x98, 0xd9, 0x34, 0x17, 0x5d, 0x63, 0x7f, 0x77, 0x7c, 0x8f, 0x1d, 0x5e, 0x55, 0xe7, - 0x2b, 0x75, 0x99, 0xc8, 0x4a, 0x9d, 0x05, 0x45, 0xb6, 0x7b, 0xac, 0x2b, 0x0e, 0x7e, 0x65, 0xf6, - 0x3a, 0x1e, 0x18, 0x2a, 0x8d, 0x2b, 0x64, 0x9d, 0xb1, 0x34, 0xf0, 0xab, 0x9d, 0x5e, 0x87, 0xfa, - 0xce, 0xe5, 0x2b, 0xa4, 0xbc, 0x67, 0xa7, 0xab, 0x85, 0x48, 0x9a, 0x23, 0x17, 0x0c, 0xff, 0x99, - 0x48, 0x97, 0xbe, 0x0d, 0x97, 0x07, 0x94, 0x9d, 0xa4, 0xc7, 0xf3, 0x2f, 0xd2, 0x70, 0x2d, 0x2a, - 0x3e, 0x69, 0x88, 0xf3, 0x55, 0x6f, 0xd0, 0x1a, 0xcc, 0x77, 0x0c, 0xf3, 0xcd, 0x56, 0x2f, 0xe7, - 0x3a, 0x86, 0xe9, 0xd3, 0xe2, 0xba, 0xc6, 0xcc, 0xcf, 0xb5, 0x6b, 0xa8, 0xb0, 0x1c, 0xd7, 0x76, - 0x49, 0xf6, 0x8f, 0xef, 0x0a, 0x30, 0x97, 0xf4, 0xb2, 0xdc, 0x9b, 0x45, 0xc1, 0x49, 0x4d, 0x98, - 0xff, 0x39, 0xac, 0xe3, 0xfd, 0xb6, 0x00, 0xa8, 0x69, 0xf7, 0x4c, 0x02, 0x6a, 0x9f, 0x5a, 0xad, - 0x24, 0xab, 0x79, 0x09, 0xa6, 0x0d, 0x53, 0xc7, 0x27, 0xb4, 0x9a, 0x19, 0x99, 0x3d, 0x44, 0xb6, - 0x12, 0xd3, 0x13, 0x6d, 0x25, 0x4a, 0x9f, 0xc1, 0x62, 0xa4, 0x88, 0x49, 0xd6, 0xff, 0x8f, 0x53, - 0xb0, 0xc8, 0x2b, 0x92, 0xf8, 0x0a, 0xe6, 0xd7, 0x61, 0xba, 0x4d, 0x64, 0x8e, 0x69, 0x67, 0xfa, - 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0x4b, 0x00, 0x5d, 0x1b, 0x1f, 0x2b, 0x8c, 0x35, 0x3d, 0x11, - 0x6b, 0x8e, 0x70, 0x50, 0x02, 0xfa, 0xbe, 0x00, 0x45, 0x32, 0xa0, 0xbb, 0xb6, 0xd5, 0xb5, 0x1c, - 0xe2, 0xb3, 0x38, 0x93, 0xc1, 0x9c, 0xe7, 0x67, 0xa7, 0xab, 0xf3, 0xdb, 0x86, 0xb9, 0xcb, 0x19, - 0x9b, 0x8d, 0x89, 0x03, 0xfc, 0xbd, 0x63, 0x0e, 0xa5, 0x4a, 0xdb, 0xd2, 0x8e, 0x82, 0xcd, 0x31, - 0x62, 0x59, 0x7c, 0x71, 0x8e, 0xf4, 0x13, 0x01, 0x2e, 0xfd, 0xdc, 0x96, 0x8b, 0xff, 0x2c, 0x94, - 0x2d, 0xbd, 0x00, 0x91, 0xfe, 0xa8, 0x9b, 0x07, 0x56, 0x92, 0x0b, 0xf7, 0xff, 0x47, 0x80, 0x85, - 0x90, 0xe0, 0x24, 0x1d, 0x9c, 0x37, 0xd5, 0xd3, 0x3c, 0x0b, 0x87, 0x71, 0x27, 0x53, 0x95, 0x3c, - 0xc7, 0xb3, 0xb3, 0x4e, 0x59, 0x82, 0x39, 0x4c, 0xac, 0x18, 0x5d, 0xe2, 0xdd, 0x67, 0x87, 0x4c, - 0x06, 0x56, 0xf4, 0xf3, 0x7e, 0x86, 0x72, 0x5f, 0xfa, 0x15, 0xe2, 0x61, 0x85, 0x07, 0x65, 0x92, - 0x43, 0xfe, 0x9f, 0xa6, 0xe0, 0x4a, 0x85, 0x6d, 0x81, 0x7b, 0x31, 0x21, 0x49, 0x76, 0xc4, 0x25, - 0x98, 0x3d, 0xc6, 0xb6, 0x63, 0x58, 0x6c, 0xb6, 0x9f, 0x97, 0xbd, 0x47, 0xb4, 0x0c, 0x59, 0xc7, - 0x54, 0xbb, 0xce, 0xa1, 0xe5, 0x6d, 0x27, 0xfa, 0xcf, 0x7e, 0xfc, 0xca, 0xf4, 0x9b, 0xc7, 0xaf, - 0xcc, 0x8c, 0x8f, 0x5f, 0x99, 0xfd, 0x12, 0xf1, 0x2b, 0x7c, 0xef, 0xee, 0xdf, 0x09, 0x70, 0x75, - 0x48, 0x73, 0x49, 0x76, 0xce, 0xef, 0x40, 0x5e, 0xe3, 0x82, 0xc9, 0xfc, 0xc0, 0x36, 0x26, 0xeb, - 0x24, 0xdb, 0x1b, 0x42, 0x9f, 0xb3, 0xd3, 0x55, 0xf0, 0x8a, 0x5a, 0xdf, 0xe0, 0xca, 0x21, 0xbf, - 0x75, 0xe9, 0xbf, 0x03, 0x14, 0xab, 0x27, 0x6c, 0x51, 0xbe, 0xc1, 0xbc, 0x12, 0xf4, 0x18, 0xb2, - 0x5d, 0xdb, 0x3a, 0x36, 0xbc, 0x6a, 0x14, 0x22, 0xc1, 0x0b, 0x5e, 0x35, 0x06, 0xb8, 0x76, 0x39, - 0x87, 0xec, 0xf3, 0xa2, 0x26, 0xe4, 0x9e, 0x5a, 0x9a, 0xda, 0x7e, 0x6c, 0xb4, 0xbd, 0x81, 0xf6, - 0xfe, 0xf9, 0x82, 0x4a, 0x3e, 0xcf, 0xae, 0xea, 0x1e, 0x7a, 0x8d, 0xe0, 0x13, 0x51, 0x1d, 0xb2, - 0x35, 0xd7, 0xed, 0x92, 0x44, 0x3e, 0xfe, 0xee, 0x4c, 0x20, 0x94, 0xb0, 0x78, 0x11, 0xb7, 0x1e, - 0x3b, 0x6a, 0xc2, 0xc2, 0x13, 0x7a, 0x7e, 0xac, 0xd2, 0xb6, 0x7a, 0x7a, 0xc5, 0x32, 0x0f, 0x8c, - 0x16, 0x9f, 0x26, 0x6e, 0x4f, 0x20, 0xf3, 0x49, 0xa5, 0x21, 0x0f, 0x0b, 0x40, 0xeb, 0x90, 0x6d, - 0x3c, 0xe2, 0xc2, 0x98, 0x1b, 0x79, 0x6b, 0x02, 0x61, 0x8d, 0x47, 0xb2, 0xcf, 0x86, 0x36, 0x21, - 0xbf, 0xfe, 0x79, 0xcf, 0xc6, 0x5c, 0xca, 0xcc, 0xc8, 0xc8, 0x89, 0x41, 0x29, 0x94, 0x4b, 0x0e, - 0x33, 0xa3, 0x6f, 0x43, 0x91, 0xe8, 0xad, 0xa9, 0xee, 0xb7, 0x3d, 0x79, 0x59, 0x2a, 0xef, 0x6b, - 0x13, 0xc8, 0xf3, 0x39, 0xbd, 0x2d, 0x81, 0x01, 0x51, 0xcb, 0x32, 0xcc, 0x47, 0xda, 0x0b, 0x21, - 0xc8, 0x74, 0x49, 0xd3, 0x08, 0x34, 0x0c, 0x89, 0xfe, 0x46, 0xef, 0xc1, 0xac, 0x69, 0xe9, 0xd8, - 0xeb, 0xcc, 0xf3, 0xe5, 0x4b, 0x67, 0xa7, 0xab, 0x33, 0x3b, 0x96, 0xce, 0x7c, 0x1d, 0xfe, 0x4b, - 0x9e, 0x21, 0x99, 0xea, 0xfa, 0xf2, 0x75, 0xc8, 0x90, 0x26, 0x22, 0x36, 0x64, 0x5f, 0x75, 0xf0, - 0x9e, 0x6d, 0x70, 0x69, 0xde, 0xe3, 0xf2, 0x3f, 0x48, 0x41, 0xaa, 0xf1, 0x88, 0x78, 0xf3, 0xfb, - 0x3d, 0xed, 0x08, 0xbb, 0x3c, 0x9d, 0x3f, 0x51, 0x2f, 0xdf, 0xc6, 0x07, 0x06, 0x73, 0xba, 0x72, - 0x32, 0x7f, 0x42, 0xef, 0x00, 0xa8, 0x9a, 0x86, 0x1d, 0x47, 0xf1, 0x8e, 0x00, 0xe6, 0xe4, 0x1c, - 0xa3, 0x6c, 0xe1, 0x3e, 0x61, 0x73, 0xb0, 0x66, 0x63, 0xd7, 0x8b, 0xa1, 0x62, 0x4f, 0x84, 0xcd, - 0xc5, 0x9d, 0xae, 0xe2, 0x5a, 0x47, 0xd8, 0xa4, 0x4d, 0x9a, 0x23, 0x56, 0xa1, 0xd3, 0x6d, 0x12, - 0x02, 0x31, 0x68, 0xd8, 0xd4, 0x03, 0xeb, 0x93, 0x93, 0xfd, 0x67, 0x22, 0xd2, 0xc6, 0x2d, 0x83, - 0x1f, 0xa0, 0xcb, 0xc9, 0xfc, 0x89, 0x68, 0x49, 0xed, 0xb9, 0x87, 0xb4, 0x25, 0x72, 0x32, 0xfd, - 0x8d, 0x6e, 0x43, 0x91, 0x85, 0x5d, 0x2a, 0xd8, 0xd4, 0x14, 0x6a, 0x07, 0x73, 0x34, 0x79, 0x9e, - 0x91, 0xab, 0xa6, 0x46, 0xac, 0x1e, 0x7a, 0x04, 0x9c, 0xa0, 0x1c, 0x75, 0x1c, 0xa2, 0x53, 0x20, - 0xb9, 0xca, 0xc5, 0xb3, 0xd3, 0xd5, 0x7c, 0x83, 0x26, 0x6c, 0x6d, 0x37, 0xc8, 0x5c, 0xc2, 0x72, - 0x6d, 0x75, 0x9c, 0xba, 0xbe, 0xfc, 0x37, 0x05, 0x48, 0x3f, 0xa9, 0x34, 0x2e, 0xac, 0x32, 0xaf, - 0xa0, 0xe9, 0x50, 0x41, 0xef, 0x40, 0x71, 0xdf, 0x68, 0xb7, 0x0d, 0xb3, 0x45, 0xfc, 0xab, 0xef, - 0x60, 0xcd, 0x53, 0x58, 0x81, 0x93, 0x77, 0x19, 0x15, 0x5d, 0x87, 0xbc, 0x66, 0x63, 0x1d, 0x9b, - 0xae, 0xa1, 0xb6, 0x1d, 0xae, 0xb9, 0x30, 0x69, 0xf9, 0x2f, 0x0b, 0x30, 0x4d, 0x3b, 0x2b, 0x7a, - 0x1b, 0x72, 0x9a, 0x65, 0xba, 0xaa, 0x61, 0x72, 0xab, 0x93, 0x93, 0x03, 0xc2, 0xc8, 0xe2, 0xdd, - 0x80, 0x39, 0x55, 0xd3, 0xac, 0x9e, 0xe9, 0x2a, 0xa6, 0xda, 0xc1, 0xbc, 0x98, 0x79, 0x4e, 0xdb, - 0x51, 0x3b, 0x18, 0xad, 0x82, 0xf7, 0xe8, 0x9f, 0xec, 0xcc, 0xc9, 0xc0, 0x49, 0x5b, 0xb8, 0xbf, - 0x8c, 0x21, 0xe7, 0xf7, 0x6a, 0x52, 0xdf, 0x9e, 0xe3, 0x97, 0x80, 0xfe, 0x46, 0xef, 0xc3, 0xa5, - 0x57, 0x3d, 0xb5, 0x6d, 0x1c, 0xd0, 0xc5, 0x2f, 0x1a, 0xa5, 0x4e, 0x5f, 0xc6, 0x8a, 0x82, 0xfc, - 0x34, 0x2a, 0x81, 0xbe, 0xd3, 0x1b, 0x04, 0xe9, 0x60, 0x10, 0xb0, 0x90, 0x1d, 0xa9, 0x0f, 0x0b, - 0x32, 0x76, 0xed, 0x7e, 0x93, 0x1d, 0x76, 0xad, 0x1e, 0x63, 0xd3, 0x25, 0x75, 0xb7, 0xba, 0x98, - 0x05, 0x89, 0x78, 0x75, 0xf7, 0x09, 0xe8, 0x16, 0x14, 0x54, 0x97, 0x74, 0x37, 0x57, 0x31, 0x7b, - 0x9d, 0x7d, 0x6c, 0xb3, 0x50, 0x00, 0x79, 0x9e, 0x53, 0x77, 0x28, 0x91, 0x9f, 0xc8, 0xb0, 0xfb, - 0x0a, 0x5d, 0x27, 0xe2, 0xaf, 0x06, 0x4a, 0xaa, 0x12, 0x8a, 0x74, 0x0f, 0x2e, 0x93, 0x7a, 0x56, - 0x4d, 0xcd, 0xee, 0x77, 0x89, 0xe4, 0x67, 0xf4, 0xaf, 0x83, 0xc4, 0xd0, 0x3e, 0x0d, 0xdd, 0x9e, - 0x91, 0x7e, 0x3c, 0x03, 0xf3, 0xd5, 0x93, 0xae, 0x65, 0x27, 0xba, 0xaa, 0x53, 0x86, 0x59, 0x0e, - 0x7c, 0xc7, 0x6c, 0xc5, 0x0e, 0x58, 0x20, 0x6f, 0x1f, 0x9a, 0x33, 0xa2, 0x32, 0x00, 0x0b, 0xa8, - 0xa4, 0x71, 0x38, 0xe9, 0x0b, 0xec, 0x1c, 0x51, 0x36, 0x7a, 0xd8, 0x60, 0x07, 0xf2, 0x9d, 0x63, - 0x4d, 0x53, 0x0e, 0x8c, 0xb6, 0xcb, 0xe3, 0xd2, 0xe2, 0x43, 0xa8, 0xb7, 0x5f, 0x54, 0x2a, 0x8f, - 0x69, 0x26, 0x16, 0xcf, 0x15, 0x3c, 0xcb, 0x40, 0x24, 0xb0, 0xdf, 0xe8, 0x6b, 0xc0, 0x0f, 0xbe, - 0x28, 0x8e, 0x77, 0x8c, 0xad, 0x3c, 0x7f, 0x76, 0xba, 0x9a, 0x93, 0x29, 0xb5, 0xd1, 0x68, 0xca, - 0x39, 0x96, 0xa1, 0xe1, 0xb8, 0x17, 0x39, 0xea, 0x30, 0x3b, 0xf9, 0x51, 0x87, 0xbf, 0x26, 0xc0, - 0x15, 0xae, 0x23, 0x65, 0x9f, 0x86, 0x77, 0xab, 0x6d, 0xc3, 0xed, 0x2b, 0x47, 0xc7, 0x4b, 0x59, - 0xea, 0xf2, 0xfc, 0x62, 0xac, 0xae, 0x43, 0x4d, 0x5c, 0xf2, 0x34, 0xde, 0x7f, 0xca, 0x99, 0xb7, - 0x8e, 0xab, 0xa6, 0x6b, 0xf7, 0xcb, 0x57, 0xcf, 0x4e, 0x57, 0x17, 0x87, 0x53, 0x5f, 0xc8, 0x8b, - 0xce, 0x30, 0x0b, 0xaa, 0x01, 0x60, 0xbf, 0x8b, 0x51, 0x0b, 0x16, 0x3f, 0x75, 0xc5, 0xf6, 0x45, - 0x39, 0xc4, 0x8b, 0xee, 0x82, 0xc8, 0x8f, 0x96, 0x1c, 0x18, 0x6d, 0xac, 0x38, 0xc6, 0xe7, 0x98, - 0xda, 0xba, 0xb4, 0x5c, 0x60, 0x74, 0x22, 0xa2, 0x61, 0x7c, 0x8e, 0xd1, 0x03, 0xb8, 0x1c, 0xb4, - 0x80, 0xb2, 0x8f, 0xdb, 0xd6, 0x6b, 0x96, 0x3d, 0x4f, 0xb3, 0x23, 0x5f, 0xfb, 0x65, 0x92, 0x44, - 0x58, 0x96, 0xbf, 0x03, 0x4b, 0xa3, 0x2a, 0x1c, 0x1e, 0x10, 0x39, 0xb6, 0x5f, 0xf9, 0x41, 0x74, - 0xb1, 0x62, 0x82, 0x8e, 0xcb, 0x17, 0x2c, 0x3e, 0x4c, 0x7d, 0x20, 0x48, 0x7f, 0x3f, 0x05, 0xf3, - 0xe5, 0x5e, 0xfb, 0xe8, 0x59, 0xb7, 0xc1, 0x8e, 0xe5, 0xa3, 0xb7, 0x20, 0xa7, 0xab, 0xae, 0xca, - 0x0a, 0x29, 0xb0, 0x23, 0x66, 0x84, 0x40, 0x6b, 0x73, 0x07, 0x8a, 0xa1, 0x58, 0x10, 0x1e, 0xf1, - 0x4e, 0xab, 0x1d, 0x90, 0x69, 0x50, 0xfa, 0x07, 0xb0, 0x14, 0xca, 0x48, 0x57, 0x16, 0x14, 0x6c, - 0xba, 0xb6, 0x81, 0xd9, 0xea, 0x58, 0x5a, 0x0e, 0x05, 0xac, 0xd4, 0x49, 0x72, 0x95, 0xa5, 0xa2, - 0x26, 0xcc, 0x91, 0x8c, 0x7d, 0x85, 0x5a, 0x41, 0x6f, 0xf5, 0xf2, 0x41, 0x4c, 0xb5, 0x22, 0xe5, - 0x2e, 0x51, 0xfd, 0x54, 0x28, 0x0f, 0xfd, 0x29, 0xe7, 0x71, 0x40, 0x59, 0xfe, 0x04, 0xc4, 0xc1, - 0x0c, 0x61, 0x5d, 0x66, 0x98, 0x2e, 0x2f, 0x85, 0x75, 0x99, 0x0e, 0xe9, 0x69, 0x33, 0x93, 0xcd, - 0x88, 0xd3, 0xd2, 0x4f, 0xd2, 0x50, 0xf0, 0x7a, 0x66, 0x92, 0x6e, 0x75, 0x19, 0xa6, 0x49, 0x3f, - 0xf2, 0xc2, 0x2b, 0x6e, 0x8f, 0x19, 0x10, 0x3c, 0xc6, 0x9a, 0xf4, 0x2f, 0x0f, 0x01, 0x52, 0xd6, - 0x24, 0xcc, 0xcf, 0xf2, 0xff, 0x10, 0x20, 0x43, 0x3d, 0xd9, 0x07, 0x90, 0xa1, 0xe7, 0xf2, 0x85, - 0xb1, 0xe7, 0xf2, 0xbd, 0xed, 0x79, 0x92, 0xd5, 0x9f, 0x58, 0x52, 0x21, 0xef, 0xaa, 0x4c, 0xe3, - 0x7b, 0x2c, 0xdb, 0xc5, 0x3a, 0xf7, 0x14, 0xaf, 0x9f, 0xd7, 0x8e, 0x9e, 0x27, 0xec, 0xf1, 0xa1, - 0x6b, 0x90, 0x26, 0xb6, 0x6b, 0x96, 0x6d, 0xd5, 0x9f, 0x9d, 0xae, 0xa6, 0x89, 0xd5, 0x22, 0x34, - 0xb4, 0x06, 0xf9, 0xa8, 0x35, 0x21, 0xce, 0x06, 0x35, 0x87, 0x21, 0x4b, 0x00, 0x6d, 0x7f, 0x08, - 0x31, 0x94, 0xc4, 0xda, 0x92, 0x6f, 0xd2, 0xff, 0xba, 0xc0, 0x63, 0x12, 0x1b, 0x1a, 0x99, 0xb3, - 0xec, 0x24, 0x27, 0x95, 0x7b, 0x20, 0xda, 0xaa, 0xa9, 0x5b, 0x1d, 0xe3, 0x73, 0xcc, 0x50, 0xb9, - 0xc3, 0xb7, 0x2b, 0x8a, 0x3e, 0x9d, 0xc2, 0x67, 0x47, 0xfa, 0x6f, 0x02, 0x8f, 0x5f, 0xf4, 0x8b, - 0x91, 0xec, 0xa6, 0x72, 0x9e, 0x2f, 0xe9, 0x99, 0x07, 0x96, 0x17, 0x7e, 0xf1, 0xf6, 0xa8, 0x60, - 0xa3, 0xba, 0x79, 0x60, 0x79, 0xdb, 0x63, 0xb6, 0x47, 0x70, 0x96, 0x7f, 0x19, 0xa6, 0x69, 0xf2, - 0x1b, 0xf4, 0x0d, 0x3f, 0x66, 0x36, 0x25, 0xa6, 0xa5, 0x3f, 0x4a, 0xc1, 0xbb, 0xb4, 0xaa, 0x2f, - 0xb0, 0x6d, 0x1c, 0xf4, 0x77, 0x6d, 0xcb, 0xc5, 0x9a, 0x8b, 0xf5, 0x60, 0x55, 0x2a, 0xc1, 0x26, - 0xd0, 0x21, 0xc7, 0xf7, 0xf3, 0x0c, 0x9d, 0xdf, 0x9c, 0xf1, 0xe4, 0xcb, 0xa1, 0xd5, 0x2c, 0xdb, - 0x07, 0xac, 0x6f, 0xc8, 0x59, 0x26, 0xb9, 0xae, 0xa3, 0x75, 0xc8, 0x75, 0xbd, 0x6a, 0x5c, 0x28, - 0x64, 0xc4, 0xe7, 0x42, 0x5b, 0x50, 0xe4, 0x05, 0x55, 0xdb, 0xc6, 0x31, 0x56, 0x54, 0xf7, 0x22, - 0x43, 0x78, 0x9e, 0xf1, 0xae, 0x13, 0xd6, 0x75, 0x57, 0xfa, 0x1b, 0x19, 0xb8, 0x75, 0x8e, 0x8a, - 0x93, 0xec, 0x5e, 0xcb, 0x90, 0x3d, 0x26, 0x2f, 0x32, 0x78, 0xed, 0xb3, 0xb2, 0xff, 0x8c, 0xf6, - 0x23, 0xf3, 0xc0, 0x81, 0x6a, 0xb4, 0xc9, 0xbc, 0xc1, 0x82, 0xf4, 0x46, 0x87, 0x01, 0xc5, 0x07, - 0xbd, 0x85, 0x66, 0x8c, 0xc7, 0x54, 0x10, 0xcd, 0xe6, 0xa0, 0xef, 0x0a, 0xb0, 0xcc, 0x5e, 0xc8, - 0x22, 0xc5, 0x06, 0x5e, 0x93, 0xa1, 0xaf, 0xd9, 0x88, 0x79, 0xcd, 0x44, 0x3a, 0x2a, 0x85, 0xde, - 0xc5, 0x0b, 0xb2, 0x14, 0x7e, 0x5b, 0xb8, 0x28, 0xcb, 0xbf, 0x29, 0x40, 0x3e, 0x44, 0x40, 0xb7, - 0x87, 0xce, 0xe5, 0xe4, 0xcf, 0xe2, 0x0e, 0xe3, 0xdc, 0x1a, 0x3a, 0x8c, 0x53, 0xce, 0x7e, 0x71, - 0xba, 0x9a, 0x91, 0x59, 0xbc, 0xb7, 0x77, 0x2c, 0xe7, 0x46, 0x70, 0x0d, 0x4c, 0x7a, 0x20, 0x93, - 0x77, 0x0f, 0x0c, 0x85, 0x75, 0xaa, 0xb7, 0x8d, 0x44, 0x61, 0x1d, 0x79, 0x92, 0x7e, 0x90, 0x82, - 0x85, 0x75, 0x5d, 0x6f, 0x34, 0x28, 0x14, 0x48, 0x72, 0x8c, 0x21, 0xc8, 0x10, 0xff, 0x80, 0x9f, - 0x21, 0xa2, 0xbf, 0xd1, 0x7b, 0x80, 0x74, 0xc3, 0x61, 0xd7, 0x29, 0x38, 0x87, 0xaa, 0x6e, 0xbd, - 0x0e, 0x76, 0x8b, 0x17, 0xbc, 0x94, 0x86, 0x97, 0x80, 0x1a, 0x40, 0x9d, 0x56, 0xc5, 0x71, 0x55, - 0x7f, 0x35, 0xfc, 0xd6, 0x44, 0xa7, 0x52, 0x98, 0x37, 0xeb, 0x3f, 0xca, 0x39, 0x22, 0x87, 0xfe, - 0x24, 0x3e, 0x9a, 0x41, 0x1a, 0xc5, 0x55, 0x54, 0xc7, 0x3b, 0x4f, 0xc1, 0x2e, 0x72, 0x28, 0x30, - 0xfa, 0xba, 0xc3, 0x8e, 0x49, 0xb0, 0x30, 0xec, 0x40, 0x35, 0x49, 0xae, 0x65, 0xfe, 0x1d, 0x01, - 0x0a, 0x32, 0x3e, 0xb0, 0xb1, 0x73, 0x98, 0xa4, 0xce, 0x1f, 0xc3, 0x9c, 0xcd, 0xa4, 0x2a, 0x07, - 0xb6, 0xd5, 0xb9, 0x88, 0xad, 0xc8, 0x73, 0xc6, 0xc7, 0xb6, 0xd5, 0xe1, 0x26, 0xf9, 0x05, 0x14, - 0xfd, 0x32, 0x26, 0x59, 0xf9, 0xbf, 0x47, 0x4f, 0x5c, 0x32, 0xc1, 0x49, 0x6f, 0xdb, 0x26, 0xab, - 0x01, 0xba, 0x9e, 0x1d, 0x2e, 0x68, 0x92, 0x6a, 0xf8, 0xaf, 0x02, 0x14, 0x1a, 0xbd, 0x7d, 0x76, - 0x4d, 0x50, 0x72, 0x1a, 0xa8, 0x42, 0xae, 0x8d, 0x0f, 0x5c, 0xe5, 0x8d, 0x02, 0x88, 0xb3, 0x84, - 0x95, 0x86, 0x4f, 0x3f, 0x01, 0xb0, 0xe9, 0x91, 0x23, 0x2a, 0x27, 0x7d, 0x41, 0x39, 0x39, 0xca, - 0x4b, 0xc8, 0x64, 0xd6, 0x29, 0xfa, 0xd5, 0x4c, 0x72, 0x7e, 0x79, 0x19, 0xb1, 0x0e, 0xe9, 0x8b, - 0x58, 0x87, 0x05, 0xbe, 0x53, 0x1d, 0x6f, 0x21, 0x4a, 0xb0, 0x48, 0xdd, 0x32, 0x45, 0xed, 0x76, - 0xdb, 0x86, 0x87, 0x53, 0xa8, 0xfd, 0xc9, 0xc8, 0x0b, 0x34, 0x69, 0x9d, 0xa5, 0x50, 0x84, 0x82, - 0x7e, 0x43, 0x80, 0xb9, 0x03, 0x1b, 0xe3, 0xcf, 0xb1, 0x42, 0x4d, 0xf2, 0x64, 0x5b, 0xf1, 0x1b, - 0xa4, 0x0c, 0x5f, 0x7a, 0xab, 0x2e, 0xcf, 0x5e, 0xdc, 0x20, 0xef, 0x45, 0x3b, 0x20, 0x6a, 0x6d, - 0xb6, 0x79, 0xe8, 0x87, 0x05, 0xcc, 0x4c, 0x3e, 0x00, 0x8a, 0x8c, 0x39, 0x88, 0x0c, 0x78, 0x4e, - 0x06, 0x93, 0xaa, 0x2b, 0xfc, 0x6a, 0x36, 0xea, 0x6c, 0x47, 0xc3, 0x02, 0xc2, 0x47, 0xaf, 0x43, - 0x37, 0xba, 0x95, 0x64, 0xac, 0xea, 0xdc, 0x73, 0x27, 0xe3, 0xca, 0x7f, 0xe0, 0xe3, 0xea, 0x25, - 0x2c, 0xd0, 0x7e, 0x93, 0xf4, 0x09, 0x4a, 0xe9, 0x1f, 0xa5, 0x01, 0x85, 0x25, 0xff, 0xfc, 0xfa, - 0x5b, 0x2a, 0xb9, 0xfe, 0xb6, 0x09, 0x52, 0xc8, 0x19, 0x6a, 0xab, 0x8e, 0xab, 0xb0, 0xf8, 0x33, - 0x47, 0xe9, 0x62, 0x5b, 0x71, 0xb0, 0x66, 0xf1, 0x4b, 0x74, 0x04, 0x79, 0x25, 0xc8, 0xf9, 0x54, - 0x75, 0xdc, 0xe7, 0x2c, 0xdf, 0x2e, 0xb6, 0x1b, 0x34, 0x17, 0x7a, 0x04, 0x57, 0x3a, 0xea, 0x49, - 0x1c, 0xff, 0x34, 0xe5, 0x5f, 0xec, 0xa8, 0x27, 0x43, 0x4c, 0x1f, 0xc2, 0x72, 0x3c, 0x93, 0xe2, - 0x60, 0x6f, 0x7f, 0xea, 0x4a, 0x0c, 0x63, 0x03, 0xbb, 0x68, 0x1d, 0x20, 0x00, 0x11, 0x7c, 0x8e, - 0x9e, 0x04, 0x43, 0xe4, 0x7c, 0x0c, 0x21, 0x7d, 0x4f, 0x80, 0xc2, 0xb6, 0xd1, 0xb2, 0xd5, 0x44, - 0xaf, 0xa8, 0x41, 0x1f, 0x46, 0x37, 0xf4, 0xf2, 0x0f, 0x97, 0xe3, 0x02, 0x36, 0x58, 0x0e, 0x6f, - 0xd1, 0x8e, 0x33, 0x90, 0xa9, 0xcf, 0x2f, 0x51, 0x92, 0x36, 0xff, 0x3f, 0x5c, 0x83, 0x39, 0x5e, - 0xee, 0x3d, 0xd3, 0xb0, 0x4c, 0xf4, 0x00, 0xd2, 0x2d, 0xbe, 0xb4, 0x9d, 0x8f, 0x5d, 0xd1, 0x0b, - 0x2e, 0x80, 0xab, 0x4d, 0xc9, 0x24, 0x2f, 0x61, 0xe9, 0xf6, 0xdc, 0x18, 0x40, 0x11, 0xc4, 0x30, - 0x87, 0x59, 0xba, 0x3d, 0x17, 0x35, 0xa0, 0xa8, 0x05, 0xb7, 0x4e, 0x29, 0x84, 0x3d, 0x3d, 0x72, - 0x99, 0x2b, 0xf6, 0xfe, 0xaf, 0xda, 0x94, 0x5c, 0xd0, 0x22, 0x09, 0xa8, 0x12, 0xbe, 0xec, 0x28, - 0x33, 0x14, 0x20, 0x15, 0x1c, 0x95, 0x8d, 0x5e, 0xb4, 0x54, 0x9b, 0x0a, 0xdd, 0x89, 0x84, 0x3e, - 0x84, 0x19, 0x9d, 0x5e, 0xab, 0xc3, 0x8d, 0x66, 0x5c, 0x43, 0x47, 0x6e, 0x2f, 0xaa, 0x4d, 0xc9, - 0x9c, 0x03, 0x6d, 0xc2, 0x1c, 0xfb, 0xc5, 0x5c, 0x7a, 0x6e, 0xea, 0x6e, 0x8d, 0x96, 0x10, 0x72, - 0x36, 0x6a, 0x53, 0x72, 0x5e, 0x0f, 0xa8, 0xe8, 0x09, 0xe4, 0xb5, 0x36, 0x56, 0x6d, 0x2e, 0xea, - 0xf6, 0xc8, 0x53, 0x5d, 0x43, 0x57, 0xf1, 0xd4, 0xa6, 0x64, 0xd0, 0x7c, 0x22, 0x29, 0x94, 0x4d, - 0x6f, 0x64, 0xe1, 0x92, 0xde, 0x1f, 0x59, 0xa8, 0xe1, 0xeb, 0x6d, 0x6a, 0xd4, 0x09, 0xf1, 0xa9, - 0xe8, 0xeb, 0x90, 0x71, 0x34, 0xd5, 0xe4, 0x76, 0x77, 0x65, 0xc4, 0x95, 0x19, 0x01, 0x33, 0xcd, - 0x8d, 0x3e, 0x62, 0x68, 0xc0, 0x3d, 0xf1, 0xd6, 0x32, 0xe3, 0x74, 0x1a, 0x39, 0x9a, 0x4d, 0x74, - 0x8a, 0x29, 0x81, 0xe8, 0x41, 0x25, 0xf0, 0x47, 0xa1, 0xe7, 0x25, 0xe9, 0xe2, 0x65, 0xbc, 0x1e, - 0x86, 0xce, 0xb7, 0xd6, 0xe8, 0xf9, 0x6f, 0x8f, 0x88, 0xb6, 0x61, 0x9e, 0x09, 0xea, 0xb1, 0xa3, - 0x97, 0x4b, 0x6b, 0x23, 0x77, 0x29, 0x63, 0x0e, 0x7f, 0xd6, 0xa6, 0xe4, 0x39, 0x35, 0x44, 0x0e, - 0xca, 0xd5, 0xc1, 0x76, 0x8b, 0xad, 0x92, 0x8e, 0x29, 0x57, 0x38, 0xf4, 0xcb, 0x2f, 0x17, 0x25, - 0xa2, 0x5f, 0x83, 0x4b, 0x4c, 0x90, 0xcb, 0x23, 0x5a, 0x78, 0x60, 0xc4, 0x3b, 0x23, 0x77, 0x18, - 0x47, 0x1e, 0x97, 0xac, 0x4d, 0xc9, 0x48, 0x1d, 0x4a, 0x44, 0x1a, 0x5c, 0x66, 0x6f, 0xe0, 0xe7, - 0xed, 0x6c, 0x7e, 0x44, 0x6c, 0xe9, 0x26, 0x7d, 0xc5, 0x7b, 0xa3, 0x5e, 0x11, 0x7b, 0x0c, 0xb0, - 0x36, 0x25, 0x2f, 0xaa, 0xc3, 0xa9, 0x41, 0x35, 0x6c, 0x7e, 0xb2, 0x89, 0x77, 0xb7, 0xf7, 0xc6, - 0x57, 0x23, 0xee, 0x44, 0x98, 0x5f, 0x8d, 0x48, 0x22, 0x69, 0x40, 0xff, 0x5c, 0x37, 0xed, 0x4c, - 0x73, 0x23, 0x1b, 0x30, 0xe6, 0xf8, 0x13, 0x69, 0xc0, 0xc3, 0x10, 0x19, 0x95, 0x20, 0xd5, 0xd2, - 0x96, 0xe6, 0x47, 0xce, 0x0f, 0xfe, 0x11, 0x9f, 0xda, 0x94, 0x9c, 0x6a, 0x69, 0xe8, 0x13, 0xc8, - 0xb2, 0xf3, 0x1a, 0x27, 0xe6, 0x52, 0x61, 0xa4, 0xc1, 0x8d, 0x9e, 0x7a, 0xa9, 0x4d, 0xc9, 0xf4, - 0x88, 0x08, 0xef, 0xc8, 0x3c, 0x16, 0x9f, 0x8a, 0x28, 0x8d, 0x39, 0xa6, 0x39, 0x70, 0x22, 0x82, - 0x74, 0x18, 0xdb, 0x27, 0xa2, 0x5d, 0x28, 0xd8, 0x2c, 0x5a, 0xd1, 0x8b, 0x2d, 0x16, 0x47, 0xee, - 0xe1, 0xc7, 0x85, 0x17, 0xd7, 0xe8, 0x3a, 0x4c, 0x88, 0x4e, 0xda, 0x2e, 0x2a, 0x91, 0xb7, 0xdd, - 0xc2, 0xc8, 0xb6, 0x1b, 0x19, 0xea, 0x4a, 0xda, 0xce, 0x1e, 0x4a, 0x44, 0xdf, 0x84, 0x69, 0x36, - 0x4e, 0x10, 0x15, 0x19, 0x17, 0x96, 0x32, 0x30, 0x44, 0x58, 0x7e, 0x62, 0xbd, 0x5c, 0x1e, 0xb2, - 0xa7, 0xb4, 0xad, 0xd6, 0xd2, 0xe2, 0x48, 0xeb, 0x35, 0x1c, 0x7c, 0x48, 0xac, 0x97, 0x1b, 0x50, - 0x49, 0x07, 0xb2, 0x59, 0x0a, 0x1f, 0x62, 0x97, 0x46, 0x76, 0xa0, 0x98, 0x48, 0xbe, 0x1a, 0x3d, - 0x4c, 0x11, 0x90, 0x7d, 0xc3, 0xea, 0x60, 0x85, 0x1a, 0xc5, 0xcb, 0xe3, 0x0d, 0x6b, 0xe4, 0x1e, - 0x23, 0xdf, 0xb0, 0x32, 0x2a, 0x7a, 0x01, 0x22, 0xbf, 0x4c, 0x43, 0xf1, 0x22, 0x4b, 0x96, 0xae, - 0x50, 0x79, 0xf7, 0x62, 0x27, 0xc4, 0xb8, 0xa0, 0xa3, 0x1a, 0x71, 0x98, 0xa3, 0x29, 0xe8, 0x53, - 0x58, 0xa0, 0xf2, 0x14, 0x2d, 0xb8, 0xff, 0x64, 0x69, 0x69, 0xe8, 0x36, 0x8d, 0xd1, 0x57, 0xa5, - 0x78, 0x92, 0x45, 0x6d, 0x20, 0x89, 0x8c, 0x07, 0xc3, 0x34, 0x5c, 0x3a, 0x77, 0x2f, 0x8f, 0x1c, - 0x0f, 0xd1, 0xbb, 0x1f, 0xc9, 0x78, 0x30, 0x18, 0x85, 0x74, 0xe3, 0x01, 0x8b, 0xf7, 0xf6, 0xc8, - 0x6e, 0x3c, 0xc2, 0xd8, 0xcd, 0xbb, 0x11, 0x3b, 0xb7, 0x01, 0xc0, 0x60, 0x12, 0xf5, 0xfc, 0x56, - 0x46, 0x3a, 0x00, 0x83, 0x91, 0x76, 0xc4, 0x01, 0x68, 0x7b, 0x34, 0xe2, 0x00, 0xb0, 0x35, 0xfd, - 0xa5, 0xeb, 0xa3, 0x27, 0xab, 0xf0, 0xae, 0x1f, 0x9d, 0xac, 0x28, 0x01, 0xad, 0x43, 0x8e, 0xf8, - 0xac, 0x7d, 0x3a, 0xc2, 0x6f, 0x8c, 0x84, 0xa8, 0x03, 0x47, 0x70, 0x6a, 0x53, 0x72, 0xf6, 0x15, - 0x27, 0x91, 0x5e, 0xc5, 0x44, 0xf0, 0xb1, 0x7d, 0x7f, 0x64, 0xaf, 0x1a, 0x3e, 0x7b, 0x41, 0x7a, - 0xd5, 0xab, 0x80, 0x1a, 0x4c, 0x79, 0x0e, 0x5b, 0xad, 0x5f, 0x7a, 0x77, 0xfc, 0x94, 0x17, 0xdd, - 0x5b, 0xf0, 0xa7, 0x3c, 0x4e, 0x66, 0x53, 0x9e, 0xae, 0x38, 0x0e, 0xdd, 0xc0, 0x5f, 0xba, 0x35, - 0x66, 0xca, 0x1b, 0x58, 0xbf, 0x63, 0x53, 0x9e, 0xde, 0x60, 0x9c, 0xc4, 0xfb, 0xb3, 0xbd, 0xcb, - 0x63, 0x38, 0x7a, 0xb9, 0x33, 0xd2, 0xfb, 0x8b, 0xbd, 0xdd, 0x86, 0x78, 0x7f, 0x76, 0x24, 0x01, - 0xfd, 0x12, 0xcc, 0xf2, 0xf5, 0x92, 0xa5, 0xbb, 0x63, 0xfc, 0xe1, 0xf0, 0x12, 0x17, 0xe9, 0x8e, - 0x9c, 0x87, 0x19, 0x07, 0xb6, 0x4e, 0xc3, 0x8c, 0xdf, 0xbd, 0x31, 0xc6, 0x61, 0x68, 0xa9, 0x88, - 0x19, 0x87, 0x80, 0x4c, 0x4a, 0xe3, 0xb0, 0x35, 0x86, 0xa5, 0x5f, 0x18, 0x59, 0x9a, 0xe8, 0x62, - 0x0b, 0x29, 0x0d, 0xe7, 0xa1, 0x93, 0x05, 0x9d, 0xab, 0x99, 0x76, 0xbe, 0x36, 0x7a, 0xb2, 0x18, - 0x44, 0xad, 0x35, 0x6f, 0x37, 0x84, 0x69, 0xe5, 0xaf, 0x08, 0x70, 0x9d, 0xf5, 0x01, 0xba, 0x16, - 0xdc, 0x57, 0xfc, 0xa5, 0xfc, 0x10, 0x24, 0x7f, 0x40, 0xc5, 0x7f, 0xf3, 0xe2, 0x2b, 0xcf, 0xde, - 0x1b, 0xdf, 0x51, 0xc7, 0xe5, 0x23, 0xca, 0xe8, 0x30, 0xf0, 0xb2, 0xf4, 0x70, 0xa4, 0x32, 0xa2, - 0x80, 0x8b, 0x28, 0x83, 0xf3, 0x94, 0x67, 0xf9, 0x6e, 0xa7, 0x7f, 0xb2, 0xb1, 0x28, 0x8a, 0x9b, - 0x99, 0xec, 0x55, 0x71, 0x69, 0x33, 0x93, 0xbd, 0x26, 0x2e, 0x6f, 0x66, 0xb2, 0x6f, 0x89, 0x6f, - 0x6f, 0x66, 0xb2, 0xab, 0xe2, 0xf5, 0xcd, 0x4c, 0x56, 0x12, 0x6f, 0x4a, 0xbf, 0x77, 0x0d, 0xe6, - 0x3d, 0xec, 0xc3, 0x70, 0xcd, 0xc3, 0x30, 0xae, 0x59, 0x19, 0x85, 0x6b, 0x38, 0x5a, 0xe2, 0xc0, - 0xe6, 0x61, 0x18, 0xd8, 0xac, 0x8c, 0x02, 0x36, 0x01, 0x0f, 0x41, 0x36, 0xcd, 0x51, 0xc8, 0xe6, - 0xde, 0x04, 0xc8, 0xc6, 0x17, 0x35, 0x08, 0x6d, 0x36, 0x86, 0xa1, 0xcd, 0xbb, 0xe3, 0xa1, 0x8d, - 0x2f, 0x2a, 0x84, 0x6d, 0x3e, 0x1a, 0xc0, 0x36, 0x37, 0xc6, 0x60, 0x1b, 0x9f, 0xdf, 0x03, 0x37, - 0x5b, 0xb1, 0xe0, 0xe6, 0xf6, 0x79, 0xe0, 0xc6, 0x97, 0x13, 0x41, 0x37, 0xb5, 0x38, 0x74, 0x73, - 0xeb, 0x1c, 0x74, 0xe3, 0x8b, 0x0a, 0xc3, 0x9b, 0xad, 0x58, 0x78, 0x73, 0xfb, 0x3c, 0x78, 0x13, - 0x14, 0x2b, 0x8c, 0x6f, 0xbe, 0x11, 0xc1, 0x37, 0xab, 0x23, 0xf1, 0x8d, 0xcf, 0xcd, 0x00, 0xce, - 0xc7, 0x83, 0x00, 0xe7, 0xc6, 0x18, 0x80, 0x13, 0x28, 0x96, 0x23, 0x9c, 0x5a, 0x1c, 0xc2, 0xb9, - 0x75, 0x0e, 0xc2, 0x09, 0x74, 0x11, 0x82, 0x38, 0x3b, 0xf1, 0x10, 0xe7, 0xce, 0xb9, 0x10, 0xc7, - 0x97, 0x16, 0xc5, 0x38, 0xb5, 0x38, 0x8c, 0x73, 0xeb, 0x1c, 0x8c, 0x33, 0x50, 0x32, 0x06, 0x72, - 0xd4, 0xb1, 0x20, 0xe7, 0xbd, 0x09, 0x41, 0x8e, 0x2f, 0x3a, 0x0e, 0xe5, 0xe8, 0xe3, 0x51, 0x4e, - 0x69, 0x52, 0x94, 0xe3, 0xbf, 0x24, 0x16, 0xe6, 0xa8, 0x63, 0x61, 0xce, 0x7b, 0x13, 0xc2, 0x9c, - 0x81, 0x8a, 0x44, 0x71, 0xce, 0x4e, 0x3c, 0xce, 0xb9, 0x73, 0x2e, 0xce, 0x09, 0x5a, 0x31, 0x02, - 0x74, 0xd6, 0x42, 0x40, 0xe7, 0x9d, 0x11, 0x40, 0xc7, 0x67, 0x25, 0x48, 0xe7, 0x5b, 0x43, 0x48, - 0x47, 0x1a, 0x87, 0x74, 0x7c, 0x5e, 0x1f, 0xea, 0xd4, 0xe2, 0xa0, 0xce, 0xad, 0x73, 0xa0, 0x4e, - 0xd0, 0x6f, 0x42, 0x58, 0xe7, 0xf9, 0x08, 0xac, 0x73, 0xf7, 0x7c, 0xac, 0xe3, 0xcb, 0x1b, 0x00, - 0x3b, 0xea, 0x58, 0xb0, 0xf3, 0xde, 0x84, 0x60, 0x27, 0x68, 0xc1, 0x18, 0xb4, 0xf3, 0x41, 0x14, - 0xed, 0x5c, 0x1f, 0x8d, 0x76, 0x7c, 0x31, 0x1c, 0xee, 0x6c, 0xc5, 0xc2, 0x9d, 0xdb, 0xe7, 0xc1, - 0x9d, 0xc0, 0x9a, 0x85, 0xf1, 0xce, 0x4e, 0x3c, 0xde, 0xb9, 0x73, 0x2e, 0xde, 0x09, 0x3a, 0x52, - 0x04, 0xf0, 0x6c, 0xc5, 0x02, 0x9e, 0xdb, 0xe7, 0x01, 0x9e, 0x01, 0x53, 0xcb, 0x11, 0xcf, 0xcb, - 0x91, 0x88, 0xe7, 0xfe, 0x24, 0x88, 0xc7, 0x17, 0x3a, 0x04, 0x79, 0x3e, 0x1b, 0x0d, 0x79, 0x7e, - 0xe1, 0x02, 0xb7, 0x43, 0xc6, 0x62, 0x9e, 0x6f, 0x0d, 0x61, 0x1e, 0x69, 0x1c, 0xe6, 0x09, 0x46, - 0x86, 0x07, 0x7a, 0xaa, 0x31, 0x10, 0xe5, 0xdd, 0xf1, 0x10, 0x25, 0x98, 0xc8, 0x03, 0x8c, 0xf2, - 0xd1, 0x00, 0x46, 0xb9, 0x71, 0x6e, 0x20, 0x56, 0x08, 0xa4, 0x94, 0x87, 0x41, 0xca, 0xcd, 0xb1, - 0x20, 0xc5, 0x97, 0x10, 0xa0, 0x94, 0xad, 0x58, 0x94, 0x72, 0xfb, 0x3c, 0x94, 0x12, 0x74, 0x85, - 0x30, 0x4c, 0xd9, 0x89, 0x87, 0x29, 0x77, 0xce, 0x85, 0x29, 0x03, 0xd3, 0x96, 0x87, 0x53, 0x6a, - 0x71, 0x38, 0xe5, 0xd6, 0x39, 0x38, 0x25, 0x3c, 0x6d, 0xf9, 0x40, 0xa5, 0x39, 0x0a, 0xa8, 0xdc, - 0x9b, 0x00, 0xa8, 0x04, 0xce, 0xdc, 0x00, 0x52, 0xf9, 0x64, 0x10, 0xa9, 0x48, 0xe3, 0x90, 0x4a, - 0xd0, 0x89, 0x3c, 0xa8, 0xb2, 0x13, 0x0f, 0x55, 0xee, 0x9c, 0x0b, 0x55, 0xc2, 0xe3, 0x3a, 0x84, - 0x55, 0x3e, 0x19, 0xc4, 0x2a, 0xd2, 0x38, 0xac, 0x12, 0x94, 0xc7, 0x03, 0x2b, 0xb5, 0x38, 0xb0, - 0x72, 0xeb, 0x1c, 0xb0, 0x12, 0x32, 0xf7, 0x01, 0x5a, 0xf9, 0xab, 0x93, 0xa3, 0x95, 0x0f, 0xde, - 0x34, 0x4e, 0xe6, 0x7c, 0xb8, 0xf2, 0xc9, 0x20, 0x5c, 0x91, 0xc6, 0xc1, 0x95, 0x40, 0x1f, 0x17, - 0xc6, 0x2b, 0x6f, 0x8b, 0xef, 0x44, 0x50, 0xcb, 0x9f, 0xcc, 0xc0, 0x0c, 0xff, 0x52, 0x51, 0xe4, - 0x76, 0x1f, 0xe1, 0x4d, 0x6e, 0xf7, 0x41, 0x1b, 0xa4, 0x9b, 0x51, 0xb7, 0xe5, 0xfc, 0x3b, 0xe1, - 0x86, 0x6f, 0x2d, 0xe3, 0xac, 0x6f, 0x70, 0xcc, 0x16, 0x7d, 0x03, 0xe6, 0x7b, 0x0e, 0xb6, 0x95, - 0xae, 0x6d, 0x58, 0xb6, 0xe1, 0xb2, 0x03, 0x03, 0x42, 0x59, 0xfc, 0xe2, 0x74, 0x75, 0x6e, 0xcf, - 0xc1, 0xf6, 0x2e, 0xa7, 0xcb, 0x73, 0xbd, 0xd0, 0x93, 0xf7, 0x71, 0xa6, 0xe9, 0xc9, 0x3f, 0xce, - 0xf4, 0x1c, 0x44, 0xba, 0x1d, 0x1c, 0xb6, 0xf4, 0xec, 0x26, 0x9d, 0xf8, 0x49, 0x49, 0xd5, 0x43, - 0xc6, 0x9c, 0xde, 0xa8, 0x53, 0xb4, 0xa3, 0x44, 0xd4, 0x00, 0x7a, 0xc7, 0x85, 0xd2, 0xb5, 0xda, - 0x86, 0xd6, 0xa7, 0x13, 0x78, 0xf4, 0x56, 0xe1, 0xb1, 0x77, 0x7b, 0xbf, 0x54, 0x0d, 0x77, 0x97, - 0x72, 0xca, 0xf0, 0xda, 0xff, 0x8d, 0x1e, 0xc0, 0xe5, 0x8e, 0x7a, 0x42, 0xaf, 0x5b, 0x55, 0xbc, - 0x19, 0x99, 0x5e, 0x30, 0xc5, 0x3e, 0xd3, 0x84, 0x3a, 0xea, 0x09, 0xfd, 0x7c, 0x14, 0x4b, 0xa2, - 0xdf, 0x7e, 0xb8, 0x01, 0x73, 0x3c, 0x70, 0x9b, 0x7d, 0x1a, 0xa6, 0x48, 0x73, 0xf2, 0xef, 0x04, - 0xb0, 0xaf, 0xc3, 0xdc, 0x82, 0x82, 0x6e, 0x38, 0xae, 0x61, 0x6a, 0x2e, 0xbf, 0xc9, 0x95, 0xdd, - 0x85, 0x3a, 0xef, 0x51, 0xd9, 0x75, 0xad, 0x4d, 0x58, 0xd0, 0xda, 0x86, 0xef, 0xe7, 0xb0, 0x99, - 0x67, 0x61, 0x64, 0xbf, 0xae, 0xd0, 0xbc, 0x83, 0x9b, 0xa3, 0x45, 0x2d, 0x4a, 0x46, 0x15, 0x28, - 0xb6, 0x54, 0x17, 0xbf, 0x56, 0xfb, 0x8a, 0x77, 0x2e, 0x29, 0x4f, 0xcf, 0x62, 0xbe, 0x75, 0x76, - 0xba, 0x3a, 0xff, 0x84, 0x25, 0x0d, 0x1d, 0x4f, 0x9a, 0x6f, 0x85, 0x12, 0x74, 0x74, 0x07, 0x8a, - 0xaa, 0xd3, 0x37, 0x35, 0xda, 0x80, 0xd8, 0x74, 0x7a, 0x0e, 0x75, 0x53, 0xb3, 0x72, 0x81, 0x92, - 0x2b, 0x1e, 0x15, 0x7d, 0x04, 0xcb, 0xfc, 0xc2, 0xf6, 0xd7, 0xaa, 0xad, 0x2b, 0xb4, 0xd1, 0x83, - 0xe1, 0x21, 0x52, 0x9e, 0xab, 0xec, 0x82, 0x76, 0x92, 0x81, 0xb4, 0x74, 0xf8, 0x22, 0x54, 0x76, - 0xd1, 0x2b, 0x88, 0xf9, 0xcd, 0x4c, 0x76, 0x4e, 0x9c, 0xdf, 0xcc, 0x64, 0x0b, 0x62, 0x51, 0xfa, - 0x37, 0x02, 0x14, 0x89, 0xc5, 0x70, 0x1c, 0xc3, 0x32, 0x6b, 0x7e, 0x88, 0xa0, 0xdf, 0x6b, 0x05, - 0x7a, 0x44, 0xc4, 0x7f, 0x46, 0xab, 0xf4, 0x28, 0x0e, 0xf1, 0xcc, 0xfc, 0xcf, 0x34, 0xa4, 0x65, - 0x60, 0x24, 0x7a, 0x28, 0x62, 0x1d, 0x66, 0x1c, 0xab, 0x67, 0x6b, 0xde, 0xe5, 0xe1, 0xf7, 0x46, - 0x98, 0xa8, 0xd0, 0x0b, 0x4b, 0x0d, 0xca, 0x20, 0x73, 0x46, 0xa9, 0x04, 0x33, 0x8c, 0x82, 0x72, - 0x30, 0xfd, 0xac, 0x59, 0xab, 0xca, 0xe2, 0x14, 0x9a, 0x83, 0xec, 0x63, 0xf9, 0xd9, 0xb6, 0xd2, - 0x78, 0xfe, 0x54, 0x14, 0x50, 0x1e, 0x66, 0xe5, 0x67, 0xcf, 0x9a, 0xca, 0xd6, 0x0b, 0x31, 0x25, - 0xfd, 0xb1, 0x00, 0x73, 0x65, 0x76, 0x7b, 0x3a, 0xdb, 0xa7, 0xfe, 0x68, 0x60, 0x53, 0xf8, 0x5a, - 0x3c, 0x6a, 0x88, 0xdf, 0xa0, 0x5e, 0x87, 0x2c, 0xef, 0x9e, 0x5e, 0x84, 0xf7, 0xea, 0x68, 0x5f, - 0x91, 0x2e, 0xab, 0x78, 0x51, 0x3a, 0x1e, 0x1b, 0x6a, 0x80, 0xa8, 0x7a, 0x55, 0x54, 0x78, 0x49, - 0x46, 0xc7, 0xea, 0x0c, 0x68, 0xc3, 0xeb, 0x6c, 0x6a, 0x94, 0xfc, 0x61, 0xe6, 0xfb, 0x3f, 0x5c, - 0x9d, 0x92, 0xfe, 0x34, 0x03, 0xf3, 0xe5, 0xf0, 0x4d, 0xf1, 0xa8, 0x3e, 0x50, 0xd9, 0xb8, 0x19, - 0x30, 0xc2, 0x51, 0x1a, 0xf3, 0x0d, 0x8e, 0x5c, 0x70, 0x2d, 0x3d, 0xab, 0xfb, 0xf5, 0x31, 0xfb, - 0xe9, 0xe1, 0xca, 0x07, 0x8c, 0xcb, 0xff, 0x3e, 0xed, 0x1b, 0xf0, 0x12, 0x4c, 0xb3, 0x53, 0x44, - 0xc2, 0xd0, 0x01, 0x67, 0x6a, 0x42, 0x88, 0x87, 0x46, 0xd2, 0x65, 0x96, 0x8d, 0x18, 0xfc, 0xe6, - 0x1b, 0x5d, 0xe7, 0x16, 0xcc, 0x5b, 0x17, 0xff, 0xba, 0x5d, 0x8f, 0x5d, 0xe7, 0xf7, 0xff, 0x31, - 0x3c, 0x88, 0xbc, 0x0f, 0xfd, 0x2a, 0x14, 0x35, 0xab, 0xdd, 0x66, 0x13, 0x3b, 0x33, 0x5d, 0xc3, - 0x17, 0x7c, 0xd0, 0x22, 0xf0, 0x0f, 0x1a, 0x96, 0xfc, 0x0f, 0x1b, 0x96, 0x64, 0xfe, 0x61, 0xc3, - 0x50, 0xec, 0x76, 0xc1, 0x17, 0xc6, 0x2c, 0xde, 0x40, 0x18, 0xf9, 0xec, 0x9b, 0x84, 0x91, 0xb3, - 0xe0, 0x7b, 0xde, 0xf3, 0xfe, 0x40, 0xe0, 0x41, 0x3c, 0x4f, 0x2d, 0xeb, 0xa8, 0xe7, 0x07, 0x7e, - 0x2f, 0x87, 0x2f, 0xe7, 0x0b, 0x22, 0x5c, 0xe9, 0x09, 0x8d, 0xb8, 0xa9, 0x29, 0xf5, 0xe5, 0xa6, - 0xa6, 0x1b, 0x30, 0xd7, 0xb5, 0xf1, 0x01, 0x76, 0xb5, 0x43, 0xc5, 0xec, 0x75, 0xf8, 0xf1, 0x94, - 0xbc, 0x47, 0xdb, 0xe9, 0x75, 0xd0, 0x3d, 0x10, 0xfd, 0x2c, 0x1c, 0x6c, 0x79, 0x37, 0x43, 0x79, - 0x74, 0x0e, 0xcd, 0xa4, 0xff, 0x25, 0xc0, 0x62, 0xa4, 0x4e, 0x7c, 0x4c, 0x6d, 0x42, 0x5e, 0xf7, - 0x9d, 0x01, 0x67, 0x49, 0xb8, 0x60, 0xec, 0x73, 0x98, 0x19, 0x29, 0x70, 0xc5, 0x7b, 0x2d, 0xbd, - 0xca, 0x3d, 0x10, 0x9b, 0xba, 0xa0, 0xd8, 0xcb, 0x81, 0x9c, 0x8d, 0xd0, 0x0b, 0xfc, 0x41, 0x96, - 0x9e, 0x68, 0x90, 0x49, 0xff, 0x5b, 0x00, 0x91, 0xbe, 0xe0, 0x31, 0xc6, 0x7a, 0x22, 0x26, 0xd3, - 0x3b, 0x64, 0x90, 0x9a, 0xfc, 0x00, 0x4a, 0xe4, 0xf3, 0x13, 0xe9, 0x81, 0xcf, 0x4f, 0xc4, 0xd9, - 0xcf, 0xcc, 0x97, 0xb4, 0x9f, 0xd2, 0x0f, 0x05, 0x28, 0xf8, 0xd5, 0x66, 0xdf, 0x9d, 0x1b, 0x73, - 0xb1, 0xe4, 0x9b, 0x7d, 0x5b, 0xcd, 0xbb, 0x00, 0x63, 0xa2, 0x4f, 0xe1, 0x85, 0x2f, 0xc0, 0x60, - 0xdf, 0x04, 0xfb, 0x5b, 0x5e, 0x77, 0x24, 0x45, 0xac, 0x04, 0x37, 0x0f, 0xbc, 0xc1, 0x01, 0x1f, - 0x99, 0x7e, 0xb2, 0xd3, 0x6a, 0x1f, 0xb3, 0x3b, 0x4b, 0x26, 0xb2, 0xa5, 0x88, 0xc7, 0xc3, 0x01, - 0x5f, 0xeb, 0xd1, 0x9b, 0x0d, 0xfa, 0x31, 0x4f, 0xf6, 0xdb, 0x91, 0x1e, 0x87, 0x14, 0x48, 0x7b, - 0x14, 0xd1, 0xd2, 0x44, 0xf6, 0xdd, 0xd3, 0x12, 0xeb, 0x80, 0x3f, 0x0e, 0xb7, 0x04, 0x3b, 0xb9, - 0xfa, 0x08, 0xd2, 0xc7, 0x6a, 0x7b, 0x5c, 0x0c, 0x57, 0xa4, 0xe5, 0x64, 0x92, 0x1b, 0x3d, 0x8e, - 0x5c, 0xd8, 0x90, 0x1a, 0xbd, 0x10, 0x33, 0xac, 0xd2, 0xc8, 0xc5, 0x0e, 0xdf, 0x8c, 0x0e, 0xa0, - 0xb1, 0xaf, 0x0f, 0x8f, 0xa4, 0x0f, 0x33, 0x3f, 0xfa, 0xe1, 0xaa, 0x20, 0x7d, 0x0c, 0x48, 0xc6, - 0x0e, 0x76, 0x9f, 0xf7, 0x2c, 0x3b, 0xb8, 0xfc, 0x62, 0xf0, 0x30, 0xc1, 0x74, 0xfc, 0x61, 0x02, - 0xe9, 0x32, 0x2c, 0x46, 0xb8, 0x99, 0x05, 0x92, 0xbe, 0x09, 0xd7, 0x9e, 0x58, 0x8e, 0x63, 0x74, - 0x09, 0xe4, 0xa4, 0x43, 0x9d, 0xcc, 0x57, 0xbe, 0xcd, 0xcd, 0x76, 0x29, 0xca, 0x37, 0x99, 0x6d, - 0xca, 0xc9, 0xfe, 0xb3, 0xf4, 0xaf, 0x05, 0xb8, 0x3a, 0xcc, 0xc9, 0xb4, 0x1c, 0x77, 0x1e, 0x71, - 0x56, 0xb3, 0x82, 0xbb, 0xd9, 0xce, 0xef, 0xad, 0x5e, 0x76, 0xe2, 0xb6, 0xf2, 0x77, 0x2a, 0x1d, - 0x95, 0xda, 0x24, 0x7e, 0x54, 0xb8, 0xc0, 0xc9, 0xdb, 0x8c, 0x1a, 0x98, 0xa7, 0xcc, 0x64, 0xe6, - 0xa9, 0x09, 0xc5, 0x4d, 0xcb, 0x30, 0x89, 0x77, 0xec, 0xd5, 0x77, 0x1d, 0x0a, 0xfb, 0x86, 0xa9, - 0xda, 0x7d, 0xc5, 0x0b, 0x1d, 0x14, 0xce, 0x0b, 0x1d, 0x94, 0xe7, 0x19, 0x07, 0x7f, 0x94, 0x7e, - 0x2a, 0x80, 0x18, 0x88, 0xe5, 0x66, 0xfe, 0x6b, 0x00, 0x5a, 0xbb, 0xe7, 0xb8, 0xd8, 0xf6, 0x5a, - 0x69, 0x8e, 0x1d, 0x51, 0xa8, 0x30, 0x6a, 0x7d, 0x43, 0xce, 0xf1, 0x0c, 0x75, 0x1d, 0xdd, 0x8c, - 0xde, 0x3e, 0x30, 0x5d, 0x86, 0xb3, 0xa1, 0x3b, 0x07, 0x48, 0xb3, 0x3b, 0xae, 0x65, 0xfb, 0x48, - 0x91, 0x37, 0xbb, 0x77, 0x2f, 0x0b, 0x3d, 0x7f, 0x8c, 0xe9, 0x29, 0xa4, 0x02, 0xf1, 0x41, 0x8e, - 0xb1, 0x5f, 0xa5, 0xcc, 0xf9, 0x55, 0x62, 0x1c, 0x5e, 0x95, 0xfe, 0xb9, 0x00, 0xc5, 0x0a, 0x6b, - 0x0d, 0xbf, 0x85, 0xc7, 0x58, 0xb4, 0x0d, 0xc8, 0xba, 0x27, 0xa6, 0xd2, 0xc1, 0xfe, 0xa7, 0x44, - 0x2e, 0x70, 0x71, 0xda, 0xac, 0xcb, 0x1e, 0xe9, 0xd7, 0xe9, 0xf8, 0xa7, 0x91, 0xf9, 0x70, 0xb9, - 0x56, 0x62, 0xdf, 0x4e, 0x2e, 0x79, 0xdf, 0x4e, 0x2e, 0x6d, 0xf0, 0x0c, 0x6c, 0xa6, 0xf8, 0xfe, - 0x7f, 0x5e, 0x15, 0x64, 0x9f, 0x89, 0x39, 0x13, 0xf7, 0x1b, 0xa4, 0xd7, 0x0f, 0x4d, 0xf7, 0xa8, - 0x00, 0x10, 0xfa, 0x46, 0x0c, 0xff, 0x1a, 0xef, 0xfa, 0x86, 0xb2, 0xb7, 0x53, 0x79, 0xb6, 0xbd, - 0x5d, 0x6f, 0x36, 0xab, 0x1b, 0xa2, 0x80, 0x44, 0x98, 0x8b, 0x7c, 0x61, 0x26, 0xc5, 0xbe, 0xcf, - 0x7b, 0xff, 0x2f, 0x00, 0x04, 0x1f, 0xab, 0x22, 0xb2, 0xb6, 0xaa, 0x9f, 0x2a, 0x2f, 0xd6, 0x9f, - 0xee, 0x55, 0x1b, 0xe2, 0x14, 0x42, 0x50, 0x28, 0xaf, 0x37, 0x2b, 0x35, 0x45, 0xae, 0x36, 0x76, - 0x9f, 0xed, 0x34, 0xaa, 0xde, 0x77, 0x7d, 0xef, 0x6f, 0xc0, 0x5c, 0xf8, 0x3a, 0x18, 0xb4, 0x08, - 0xc5, 0x4a, 0xad, 0x5a, 0xd9, 0x52, 0x5e, 0xd4, 0xd7, 0x95, 0xe7, 0x7b, 0xd5, 0xbd, 0xaa, 0x38, - 0x45, 0x8b, 0x46, 0x89, 0x8f, 0xf7, 0x9e, 0x12, 0x04, 0x52, 0x84, 0x3c, 0x7b, 0xa6, 0x5f, 0xa3, - 0x11, 0x53, 0xf7, 0xb7, 0x21, 0x1f, 0xba, 0xa6, 0x96, 0xbc, 0x6e, 0x77, 0xaf, 0x51, 0x53, 0x9a, - 0xf5, 0xed, 0x6a, 0xa3, 0xb9, 0xbe, 0xbd, 0xcb, 0x64, 0x50, 0xda, 0x7a, 0xf9, 0x99, 0xdc, 0x14, - 0x05, 0xff, 0xb9, 0xf9, 0x6c, 0xaf, 0x52, 0xf3, 0xaa, 0x21, 0x65, 0xb2, 0x69, 0x31, 0x7d, 0xff, - 0x04, 0xae, 0x8e, 0xb8, 0x19, 0x85, 0x80, 0x9f, 0x3d, 0x93, 0x5e, 0xd9, 0x29, 0x4e, 0xa1, 0x79, - 0xc8, 0x91, 0xae, 0x47, 0xcf, 0x4d, 0x8a, 0x02, 0xca, 0x42, 0xe6, 0xd0, 0x75, 0xbb, 0x62, 0x0a, - 0xcd, 0x40, 0xca, 0x79, 0x24, 0xa6, 0xc9, 0xff, 0x96, 0x23, 0x66, 0x08, 0x96, 0x52, 0x3f, 0xef, - 0xd9, 0x58, 0x9c, 0x26, 0x58, 0xaa, 0xe7, 0x60, 0xfb, 0xc0, 0x68, 0x63, 0x71, 0x96, 0xb0, 0x98, - 0xbd, 0x76, 0x5b, 0xcc, 0x4a, 0x99, 0xec, 0x8c, 0x38, 0x73, 0xff, 0x06, 0x84, 0x0e, 0xa8, 0x23, - 0x80, 0x99, 0xa7, 0xaa, 0x8b, 0x1d, 0x57, 0x9c, 0x42, 0xb3, 0x90, 0x5e, 0x6f, 0xb7, 0x45, 0xe1, - 0xe1, 0x3f, 0xcb, 0x40, 0xd6, 0xfb, 0xd8, 0x0a, 0x7a, 0x0a, 0xd3, 0x14, 0x5f, 0xa0, 0xd5, 0xd1, - 0xc8, 0x83, 0x8e, 0xe3, 0xe5, 0xeb, 0xe7, 0x41, 0x13, 0x69, 0x0a, 0xfd, 0x45, 0xc8, 0x87, 0x3c, - 0x32, 0x34, 0x72, 0x05, 0x2d, 0xe2, 0x85, 0x2e, 0xdf, 0x3e, 0x2f, 0x9b, 0x2f, 0xff, 0x25, 0xe4, - 0x7c, 0x63, 0x8e, 0x6e, 0x8e, 0x33, 0xf5, 0x9e, 0xec, 0xf1, 0xf3, 0x01, 0x19, 0x76, 0xd2, 0xd4, - 0xfb, 0x02, 0xb2, 0x01, 0x0d, 0xdb, 0x5d, 0x14, 0x17, 0x37, 0x36, 0xd2, 0xb0, 0x2f, 0xdf, 0x9f, - 0x28, 0x77, 0xf0, 0x4e, 0xa2, 0xac, 0x60, 0xf2, 0x88, 0x57, 0xd6, 0xd0, 0xd4, 0x14, 0xaf, 0xac, - 0x98, 0x39, 0x68, 0x0a, 0x3d, 0x87, 0x0c, 0x31, 0x9a, 0x28, 0xce, 0xed, 0x1a, 0x30, 0xd2, 0xcb, - 0x37, 0xc7, 0xe6, 0xf1, 0x44, 0x96, 0xef, 0xfd, 0xe8, 0xbf, 0xac, 0x4c, 0xfd, 0xe8, 0x6c, 0x45, - 0xf8, 0xe9, 0xd9, 0x8a, 0xf0, 0x87, 0x67, 0x2b, 0xc2, 0x1f, 0x9d, 0xad, 0x08, 0xdf, 0xfb, 0xd9, - 0xca, 0xd4, 0x4f, 0x7f, 0xb6, 0x32, 0xf5, 0x87, 0x3f, 0x5b, 0x99, 0xfa, 0x6c, 0x96, 0x73, 0xef, - 0xcf, 0x50, 0x8b, 0xf2, 0xe8, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x47, 0x4a, 0x72, 0x7d, 0xf4, - 0x7e, 0x00, 0x00, + 0x7a, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0xa3, 0x44, 0xb6, 0x4a, 0xf3, 0x87, 0xa3, 0xdd, 0x1d, 0xcd, + 0xf4, 0xec, 0xfc, 0xf5, 0x2d, 0xb5, 0x33, 0x73, 0x97, 0x5b, 0xef, 0xae, 0xf7, 0x2c, 0x52, 0x9c, + 0x21, 0xa5, 0x91, 0x46, 0xd3, 0xa4, 0x66, 0xb0, 0xeb, 0x73, 0xda, 0xad, 0xee, 0x12, 0xd5, 0x27, + 0xb2, 0x9b, 0xd3, 0xdd, 0xd4, 0x88, 0x0b, 0xe4, 0x21, 0x89, 0x71, 0xb9, 0xa7, 0xe0, 0x02, 0x04, + 0xf0, 0x1d, 0x1c, 0x04, 0xe7, 0xd8, 0x48, 0x1e, 0xf2, 0x90, 0x20, 0x0e, 0xf2, 0x0f, 0x49, 0xfc, + 0x98, 0x43, 0x70, 0xc8, 0x9d, 0x81, 0x3c, 0x18, 0x01, 0xa2, 0x38, 0x73, 0x79, 0x88, 0x61, 0x04, + 0x09, 0x82, 0x00, 0x06, 0x16, 0x48, 0x10, 0xd4, 0x9f, 0xfe, 0x47, 0x36, 0x29, 0x6a, 0xb6, 0x2f, + 0x59, 0xc0, 0x2f, 0x12, 0xfb, 0xab, 0xfa, 0xbe, 0xae, 0xfa, 0xaa, 0xea, 0xab, 0xef, 0x57, 0xf5, + 0x55, 0x35, 0x2c, 0xd9, 0x96, 0xaa, 0x1d, 0xf6, 0xf6, 0xd7, 0xd4, 0x9e, 0x51, 0xee, 0xd9, 0x96, + 0x6b, 0xa1, 0x25, 0xcd, 0xd2, 0x8e, 0x28, 0xb9, 0xcc, 0x13, 0x57, 0xee, 0x1d, 0x1d, 0xaf, 0x1d, + 0x1d, 0x3b, 0xd8, 0x3e, 0xc6, 0xf6, 0x9a, 0x66, 0x99, 0x5a, 0xdf, 0xb6, 0xb1, 0xa9, 0x0d, 0xd6, + 0x3a, 0x96, 0x76, 0x44, 0xff, 0x18, 0x66, 0x9b, 0xb1, 0x47, 0xf3, 0xda, 0x58, 0xd5, 0x9d, 0x7e, + 0xb7, 0xab, 0xda, 0x83, 0x35, 0xdb, 0xe9, 0xed, 0xaf, 0xf1, 0x07, 0x9e, 0x17, 0x79, 0x6f, 0xd7, + 0x55, 0x57, 0xe5, 0xb4, 0x0b, 0x1e, 0x0d, 0xdb, 0xb6, 0x65, 0x3b, 0x9c, 0x7a, 0xc9, 0xa3, 0x76, + 0xb1, 0xab, 0x86, 0x72, 0xbf, 0xe5, 0xb8, 0x96, 0xad, 0xb6, 0xf1, 0x1a, 0x36, 0xdb, 0x86, 0x89, + 0x49, 0x86, 0x63, 0x4d, 0xe3, 0x89, 0x6f, 0xc7, 0x26, 0x3e, 0xe4, 0xa9, 0xa5, 0xbe, 0x6b, 0x74, + 0xd6, 0x0e, 0x3b, 0xda, 0x9a, 0x6b, 0x74, 0xb1, 0xe3, 0xaa, 0xdd, 0x9e, 0x57, 0x05, 0x9a, 0xe2, + 0xda, 0xaa, 0x66, 0x98, 0x6d, 0xef, 0x7f, 0x6f, 0x7f, 0xcd, 0xc6, 0x9a, 0x65, 0xeb, 0x58, 0x57, + 0x9c, 0x9e, 0x6a, 0x7a, 0xc5, 0x6d, 0x5b, 0x6d, 0x8b, 0xfe, 0x5c, 0x23, 0xbf, 0x38, 0xf5, 0x6a, + 0xdb, 0xb2, 0xda, 0x1d, 0xbc, 0x46, 0x9f, 0xf6, 0xfb, 0x07, 0x6b, 0x7a, 0xdf, 0x56, 0x5d, 0xc3, + 0xe2, 0x5c, 0xd2, 0x3f, 0x11, 0x60, 0x51, 0xc6, 0x2f, 0xfb, 0xd8, 0x71, 0xeb, 0x58, 0xd5, 0xb1, + 0x8d, 0xae, 0x40, 0xfa, 0x08, 0x0f, 0x4a, 0xe9, 0x6b, 0xc2, 0x9d, 0x85, 0xca, 0xfc, 0x17, 0xa7, + 0xab, 0xe9, 0x2d, 0x3c, 0x90, 0x09, 0x0d, 0x5d, 0x83, 0x79, 0x6c, 0xea, 0x0a, 0x49, 0xce, 0x44, + 0x93, 0xe7, 0xb0, 0xa9, 0x6f, 0xe1, 0x01, 0xfa, 0x36, 0x64, 0x1d, 0x22, 0xcd, 0xd4, 0x70, 0x69, + 0xf6, 0x9a, 0x70, 0x67, 0xb6, 0xf2, 0xab, 0x5f, 0x9c, 0xae, 0x7e, 0xdc, 0x36, 0xdc, 0xc3, 0xfe, + 0x7e, 0x59, 0xb3, 0xba, 0x6b, 0x7e, 0x9b, 0xea, 0xfb, 0xc1, 0xef, 0xb5, 0xde, 0x51, 0x7b, 0x6d, + 0x58, 0x47, 0xe5, 0xd6, 0x89, 0xd9, 0xc4, 0x2f, 0x65, 0x5f, 0xe2, 0x66, 0x26, 0x2b, 0x88, 0xa9, + 0xcd, 0x4c, 0x36, 0x25, 0xa6, 0xa5, 0x9f, 0xa6, 0xa0, 0x20, 0x63, 0xa7, 0x67, 0x99, 0x0e, 0xe6, + 0x25, 0x7f, 0x1f, 0xd2, 0xee, 0x89, 0x49, 0x4b, 0x9e, 0x7f, 0x70, 0xb5, 0x3c, 0xd2, 0x7b, 0xca, + 0x2d, 0x5b, 0x35, 0x1d, 0x55, 0x23, 0xd5, 0x97, 0x49, 0x56, 0xf4, 0x01, 0xe4, 0x6d, 0xec, 0xf4, + 0xbb, 0x98, 0x2a, 0x92, 0x56, 0x2a, 0xff, 0xe0, 0x72, 0x0c, 0x67, 0xb3, 0xa7, 0x9a, 0x32, 0xb0, + 0xbc, 0xe4, 0x37, 0x6a, 0xc2, 0x22, 0xe7, 0xb4, 0xb1, 0xea, 0x58, 0x66, 0x69, 0xfe, 0x9a, 0x70, + 0xa7, 0xf0, 0xa0, 0x1c, 0xc3, 0x1b, 0x2d, 0x25, 0x79, 0xec, 0x77, 0xb1, 0x4c, 0xb9, 0xe4, 0x05, + 0x3b, 0xf4, 0x84, 0xae, 0x40, 0xd6, 0xec, 0x77, 0x89, 0x7e, 0x1d, 0xaa, 0xbd, 0xb4, 0x3c, 0x6f, + 0xf6, 0xbb, 0x5b, 0x78, 0xe0, 0xa0, 0xb7, 0x20, 0x47, 0x92, 0xf6, 0x07, 0x2e, 0x76, 0x4a, 0x59, + 0x9a, 0x46, 0xf2, 0x56, 0xc8, 0xb3, 0xf4, 0x09, 0x2c, 0x84, 0xa5, 0x22, 0x04, 0x05, 0xb9, 0xd6, + 0xdc, 0xdb, 0xae, 0x29, 0x7b, 0x3b, 0x5b, 0x3b, 0x4f, 0x5f, 0xec, 0x88, 0x33, 0xe8, 0x02, 0x88, + 0x9c, 0xb6, 0x55, 0xfb, 0x54, 0x79, 0xd2, 0xd8, 0x6e, 0xb4, 0x44, 0x61, 0x25, 0xf3, 0xbd, 0xdf, + 0xbd, 0x3a, 0xb3, 0x99, 0xc9, 0xce, 0x89, 0xf3, 0xd2, 0xef, 0x0a, 0x00, 0x8f, 0xb1, 0xcb, 0x7b, + 0x03, 0xaa, 0xc0, 0xdc, 0x21, 0x2d, 0x71, 0x49, 0xa0, 0x6a, 0xb9, 0x16, 0x5b, 0xb5, 0x50, 0xcf, + 0xa9, 0x64, 0x7f, 0x7c, 0xba, 0x3a, 0xf3, 0xb3, 0xd3, 0x55, 0x41, 0xe6, 0x9c, 0xe8, 0x19, 0xe4, + 0x8f, 0xf0, 0x40, 0xe1, 0xe3, 0xb2, 0x94, 0xa2, 0x3a, 0x7a, 0x3f, 0x24, 0xe8, 0xe8, 0xb8, 0xec, + 0x0d, 0xd1, 0x72, 0x68, 0x38, 0x97, 0x09, 0x47, 0xb9, 0xe9, 0xda, 0xd8, 0x6c, 0xbb, 0x87, 0x32, + 0x1c, 0xe1, 0xc1, 0x13, 0x26, 0x43, 0xfa, 0x03, 0x01, 0xf2, 0xb4, 0x94, 0x4c, 0xa9, 0xa8, 0x3a, + 0x54, 0xcc, 0xeb, 0x67, 0xb6, 0x40, 0x4c, 0x39, 0xcb, 0x30, 0x7b, 0xac, 0x76, 0xfa, 0x98, 0x96, + 0x30, 0xff, 0xa0, 0x14, 0x23, 0xe3, 0x39, 0x49, 0x97, 0x59, 0x36, 0xf4, 0x11, 0x2c, 0x18, 0xa6, + 0x8b, 0x4d, 0x57, 0x61, 0x6c, 0xe9, 0x33, 0xd8, 0xf2, 0x2c, 0x37, 0x7d, 0x90, 0xfe, 0xb1, 0x00, + 0xb0, 0xdb, 0x4f, 0x54, 0xcf, 0x5f, 0x9f, 0xb2, 0xfc, 0x95, 0x0c, 0x61, 0xf5, 0x6a, 0x71, 0x09, + 0xe6, 0x0c, 0xb3, 0x63, 0x98, 0xac, 0xfc, 0x59, 0x99, 0x3f, 0xa1, 0x0b, 0x30, 0xbb, 0xdf, 0x31, + 0x4c, 0x9d, 0x8e, 0x87, 0xac, 0xcc, 0x1e, 0x24, 0x19, 0xf2, 0xb4, 0xd4, 0x09, 0xea, 0x5d, 0x3a, + 0x4d, 0xc1, 0xc5, 0xaa, 0x65, 0xea, 0x06, 0x19, 0x92, 0x6a, 0xe7, 0x2b, 0xa1, 0x95, 0x4d, 0xb8, + 0xa0, 0xe3, 0x9e, 0x8d, 0x35, 0xd5, 0xc5, 0xba, 0x82, 0x4f, 0x7a, 0x53, 0xb6, 0x31, 0x0a, 0xb8, + 0x6a, 0x27, 0x3d, 0x4a, 0x23, 0xa3, 0x96, 0x08, 0x60, 0xa3, 0x76, 0x8e, 0x98, 0x4c, 0x39, 0x8b, + 0x4f, 0x7a, 0x74, 0xd4, 0xc6, 0xab, 0x19, 0x7d, 0x1d, 0x2e, 0xab, 0x9d, 0x8e, 0xf5, 0x4a, 0x31, + 0x0e, 0x14, 0xdd, 0xc2, 0x8e, 0x62, 0x5a, 0xae, 0x82, 0x4f, 0x0c, 0xc7, 0xa5, 0x26, 0x21, 0x2b, + 0x2f, 0xd3, 0xe4, 0xc6, 0xc1, 0x86, 0x85, 0x9d, 0x1d, 0xcb, 0xad, 0x91, 0xa4, 0x50, 0x53, 0xce, + 0x87, 0x9b, 0x52, 0xfa, 0x75, 0xb8, 0x34, 0xac, 0xdf, 0x24, 0xdb, 0xef, 0x27, 0x02, 0x14, 0x1a, + 0xa6, 0xe1, 0x7e, 0x25, 0x1a, 0xce, 0xd7, 0x67, 0x3a, 0xac, 0xcf, 0x7b, 0x20, 0x1e, 0xa8, 0x46, + 0xe7, 0xa9, 0xd9, 0xb2, 0xba, 0xfb, 0x8e, 0x6b, 0x99, 0xd8, 0xe1, 0x0a, 0x1f, 0xa1, 0x4b, 0xcf, + 0xa1, 0xe8, 0xd7, 0x26, 0x49, 0x35, 0xb9, 0x20, 0x36, 0x4c, 0xcd, 0xc6, 0x5d, 0x6c, 0x26, 0xaa, + 0xa7, 0xb7, 0x21, 0x67, 0x78, 0x72, 0xa9, 0xae, 0xd2, 0x72, 0x40, 0x90, 0xfa, 0xb0, 0x14, 0x7a, + 0x6b, 0x92, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0x95, 0x12, 0xb4, 0x11, 0x99, 0x8c, 0xf0, 0x2b, 0x66, + 0xde, 0x9a, 0xb0, 0xb8, 0x81, 0x3b, 0xd8, 0xc5, 0x09, 0xd6, 0x54, 0xda, 0x83, 0x82, 0x27, 0x34, + 0xc9, 0x86, 0xf9, 0x2d, 0x01, 0x10, 0x97, 0xab, 0x9a, 0xed, 0x24, 0x4b, 0x8c, 0x56, 0x89, 0x6b, + 0xe1, 0xf6, 0x6d, 0x93, 0x4d, 0xe7, 0xac, 0x4f, 0x02, 0x23, 0xd1, 0x19, 0x3d, 0x18, 0xb2, 0x99, + 0xf0, 0x90, 0xe5, 0xee, 0xcd, 0x2b, 0x58, 0x8e, 0x14, 0x2c, 0xd9, 0xe6, 0xcb, 0xd0, 0x32, 0xa5, + 0xae, 0xa5, 0xc3, 0x3e, 0x1c, 0x25, 0x4a, 0x3f, 0x10, 0x60, 0xa9, 0xda, 0xc1, 0xaa, 0x9d, 0xb8, + 0x46, 0xbe, 0x05, 0x59, 0x1d, 0xab, 0x3a, 0xad, 0x32, 0x1b, 0xd8, 0xef, 0x84, 0xa4, 0x10, 0x4f, + 0xb7, 0x7c, 0xd8, 0xd1, 0xca, 0x2d, 0xcf, 0x07, 0xe6, 0xa3, 0xdb, 0x67, 0x92, 0x3e, 0x05, 0x14, + 0x2e, 0x59, 0x92, 0x1d, 0xe1, 0xf7, 0x52, 0x80, 0x64, 0x7c, 0x8c, 0x6d, 0x37, 0xf1, 0x6a, 0x6f, + 0x40, 0xde, 0x55, 0xed, 0x36, 0x76, 0x15, 0xe2, 0xdd, 0x9f, 0xa7, 0xe6, 0xc0, 0xf8, 0x08, 0x19, + 0xb5, 0xe0, 0x36, 0x36, 0xd5, 0xfd, 0x0e, 0xa6, 0x52, 0x94, 0x7d, 0xab, 0x6f, 0xea, 0x8a, 0xe1, + 0x62, 0x5b, 0x75, 0x2d, 0x5b, 0xb1, 0x7a, 0xae, 0xd1, 0x35, 0x3e, 0xa7, 0x8e, 0x3d, 0xef, 0x6a, + 0x37, 0x58, 0x76, 0xc2, 0x5c, 0x21, 0x99, 0x1b, 0x3c, 0xef, 0xd3, 0x50, 0x56, 0x54, 0x86, 0x65, + 0xa3, 0x6d, 0x5a, 0x36, 0x56, 0xda, 0x9a, 0xe2, 0x1e, 0xda, 0xd8, 0x39, 0xb4, 0x3a, 0xde, 0x84, + 0xb4, 0xc4, 0x92, 0x1e, 0x6b, 0x2d, 0x2f, 0x41, 0xfa, 0x0c, 0x96, 0x23, 0x5a, 0x4a, 0xb2, 0x09, + 0xfe, 0x87, 0x00, 0xf9, 0xa6, 0xa6, 0x9a, 0x49, 0xea, 0xfe, 0x13, 0xc8, 0x3b, 0x9a, 0x6a, 0x2a, + 0x07, 0x96, 0xdd, 0x55, 0x5d, 0x5a, 0xaf, 0x42, 0x44, 0xf7, 0xbe, 0x7f, 0xaf, 0xa9, 0xe6, 0x23, + 0x9a, 0x49, 0x06, 0xc7, 0xff, 0x3d, 0xec, 0xbf, 0xce, 0x7e, 0x79, 0xff, 0x95, 0x0d, 0xef, 0xcd, + 0x4c, 0x36, 0x2d, 0x66, 0xa4, 0x3f, 0x13, 0x60, 0x81, 0x55, 0x39, 0xc9, 0xe1, 0xfd, 0x0d, 0xc8, + 0xd8, 0xd6, 0x2b, 0x36, 0xbc, 0xf3, 0x0f, 0xde, 0x8a, 0x11, 0xb1, 0x85, 0x07, 0xe1, 0xf9, 0x93, + 0x66, 0x47, 0x15, 0xe0, 0x5e, 0xaa, 0x42, 0xb9, 0xd3, 0xd3, 0x72, 0x03, 0xe3, 0x92, 0x89, 0x8c, + 0xdb, 0x50, 0xdc, 0x57, 0x5d, 0xed, 0x50, 0xb1, 0x79, 0x21, 0xc9, 0x5c, 0x9b, 0xbe, 0xb3, 0x20, + 0x17, 0x28, 0xd9, 0x2b, 0xba, 0x43, 0x6a, 0xce, 0xc6, 0x9b, 0x83, 0xff, 0x9c, 0xb5, 0xf9, 0xff, + 0x11, 0xf8, 0x18, 0xf2, 0x6a, 0xfe, 0xe7, 0xad, 0xe9, 0x7f, 0x98, 0x82, 0xcb, 0xd5, 0x43, 0xac, + 0x1d, 0x55, 0x2d, 0xd3, 0x31, 0x1c, 0x97, 0xe8, 0x2e, 0xc9, 0xf6, 0x7f, 0x0b, 0x72, 0xaf, 0x0c, + 0xf7, 0x50, 0xd1, 0x8d, 0x83, 0x03, 0x6a, 0x6d, 0xb3, 0x72, 0x96, 0x10, 0x36, 0x8c, 0x83, 0x03, + 0xf4, 0x10, 0x32, 0x5d, 0x4b, 0x67, 0xce, 0x7c, 0xe1, 0xc1, 0x6a, 0x8c, 0x78, 0x5a, 0x34, 0xa7, + 0xdf, 0xdd, 0xb6, 0x74, 0x2c, 0xd3, 0xcc, 0xe8, 0x2a, 0x80, 0x46, 0xa8, 0x3d, 0xcb, 0x30, 0x5d, + 0x6e, 0x1c, 0x43, 0x14, 0x54, 0x87, 0x9c, 0x8b, 0xed, 0xae, 0x61, 0xaa, 0x2e, 0x2e, 0xcd, 0x52, + 0xe5, 0xbd, 0x1b, 0x5b, 0xf0, 0x5e, 0xc7, 0xd0, 0xd4, 0x0d, 0xec, 0x68, 0xb6, 0xd1, 0x73, 0x2d, + 0x9b, 0x6b, 0x31, 0x60, 0x96, 0xfe, 0x7a, 0x06, 0x4a, 0xa3, 0xba, 0x49, 0xb2, 0x87, 0xec, 0xc2, + 0x9c, 0x8d, 0x9d, 0x7e, 0xc7, 0xe5, 0x7d, 0xe4, 0xc1, 0x38, 0x15, 0xc4, 0x94, 0x80, 0x2e, 0x5d, + 0x74, 0x5c, 0x5e, 0x6c, 0x2e, 0x67, 0xe5, 0x5f, 0x0a, 0x30, 0xc7, 0x12, 0xd0, 0x7d, 0xc8, 0xda, + 0x64, 0x62, 0x50, 0x0c, 0x9d, 0x96, 0x31, 0x5d, 0xb9, 0xf4, 0xfa, 0x74, 0x75, 0x9e, 0x4e, 0x16, + 0x8d, 0x8d, 0x2f, 0x82, 0x9f, 0xf2, 0x3c, 0xcd, 0xd7, 0xd0, 0x49, 0x6b, 0x39, 0xae, 0x6a, 0xbb, + 0x74, 0x51, 0x29, 0xc5, 0x10, 0x12, 0x25, 0x6c, 0xe1, 0x01, 0xda, 0x84, 0x39, 0xc7, 0x55, 0xdd, + 0xbe, 0xc3, 0xdb, 0xeb, 0x5c, 0x85, 0x6d, 0x52, 0x4e, 0x99, 0x4b, 0x20, 0xee, 0x96, 0x8e, 0x5d, + 0xd5, 0xe8, 0xd0, 0x06, 0xcc, 0xc9, 0xfc, 0x49, 0xfa, 0x6d, 0x01, 0xe6, 0x58, 0x56, 0x74, 0x19, + 0x96, 0xe5, 0xf5, 0x9d, 0xc7, 0x35, 0xa5, 0xb1, 0xb3, 0x51, 0x6b, 0xd5, 0xe4, 0xed, 0xc6, 0xce, + 0x7a, 0xab, 0x26, 0xce, 0xa0, 0x4b, 0x80, 0xbc, 0x84, 0xea, 0xd3, 0x9d, 0x66, 0xa3, 0xd9, 0xaa, + 0xed, 0xb4, 0x44, 0x81, 0xae, 0xa9, 0x50, 0x7a, 0x88, 0x9a, 0x42, 0xef, 0xc2, 0xb5, 0x61, 0xaa, + 0xd2, 0x6c, 0xad, 0xb7, 0x9a, 0x4a, 0xad, 0xd9, 0x6a, 0x6c, 0xaf, 0xb7, 0x6a, 0x1b, 0x62, 0x7a, + 0x42, 0x2e, 0xf2, 0x12, 0x59, 0xae, 0x55, 0x5b, 0x62, 0x46, 0x72, 0xe1, 0xa2, 0x8c, 0x35, 0xab, + 0xdb, 0xeb, 0xbb, 0x98, 0x94, 0xd2, 0x49, 0x72, 0xa4, 0x5c, 0x86, 0x79, 0xdd, 0x1e, 0x28, 0x76, + 0xdf, 0xe4, 0xe3, 0x64, 0x4e, 0xb7, 0x07, 0x72, 0xdf, 0x94, 0xfe, 0xa1, 0x00, 0x97, 0x86, 0x5f, + 0x9b, 0x64, 0x27, 0x7c, 0x06, 0x79, 0x55, 0xd7, 0xb1, 0xae, 0xe8, 0xb8, 0xe3, 0xaa, 0xdc, 0x25, + 0xba, 0x17, 0x92, 0xc4, 0x97, 0x02, 0xcb, 0xfe, 0x52, 0xe0, 0xf6, 0xf3, 0x6a, 0x95, 0x16, 0x64, + 0x83, 0x70, 0x78, 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xe9, 0x87, 0x19, 0x58, 0xac, 0x99, 0x7a, 0xeb, + 0x24, 0xd1, 0xb9, 0xe4, 0x12, 0xcc, 0x69, 0x56, 0xb7, 0x6b, 0xb8, 0x9e, 0x82, 0xd8, 0x13, 0xfa, + 0xe5, 0x90, 0x2b, 0x9b, 0x9e, 0xc2, 0xa1, 0x0b, 0x9c, 0x58, 0xf4, 0x1b, 0x70, 0x99, 0x58, 0x4d, + 0xdb, 0x54, 0x3b, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xd1, 0x6e, 0x63, 0x9b, 0x2f, 0x3f, 0xde, 0x89, + 0x29, 0x67, 0x83, 0x73, 0x54, 0x29, 0x43, 0x8b, 0xe5, 0x97, 0x2f, 0x1a, 0x71, 0x64, 0xf4, 0x31, + 0x00, 0x99, 0x8a, 0xe8, 0x92, 0xa6, 0xc3, 0xed, 0xd1, 0xb8, 0x35, 0x4d, 0xcf, 0x04, 0x11, 0x06, + 0xf2, 0xec, 0xa0, 0x67, 0x20, 0x1a, 0xa6, 0x72, 0xd0, 0x31, 0xda, 0x87, 0xae, 0xf2, 0xca, 0x36, + 0x5c, 0xec, 0x94, 0x96, 0xa8, 0x8c, 0xb8, 0xa6, 0x6e, 0xf2, 0xa5, 0x59, 0xfd, 0x05, 0xc9, 0xc9, + 0xa5, 0x15, 0x0c, 0xf3, 0x11, 0xe5, 0xa7, 0x44, 0x07, 0xad, 0x11, 0x28, 0xf4, 0xb2, 0x6f, 0xd8, + 0x58, 0xb9, 0xdf, 0xd3, 0xe8, 0x3a, 0x48, 0xb6, 0x52, 0x78, 0x7d, 0xba, 0x0a, 0x32, 0x23, 0xdf, + 0xdf, 0xad, 0x12, 0x68, 0xc4, 0x7e, 0xf7, 0x34, 0xa2, 0xf6, 0x9e, 0x65, 0x38, 0x96, 0x59, 0xca, + 0x31, 0xb5, 0xb3, 0x27, 0x74, 0x17, 0x44, 0xf7, 0xc4, 0x54, 0x0e, 0xb1, 0x6a, 0xbb, 0xfb, 0x58, + 0x75, 0xc9, 0xfc, 0x0c, 0x34, 0x47, 0xd1, 0x3d, 0x31, 0xeb, 0x21, 0xf2, 0x66, 0x26, 0x3b, 0x2f, + 0x66, 0x37, 0x33, 0xd9, 0xac, 0x98, 0x93, 0xfe, 0xa3, 0x00, 0x05, 0xaf, 0x6f, 0x24, 0xd9, 0x8d, + 0xef, 0x80, 0x68, 0x99, 0x58, 0xe9, 0x1d, 0xaa, 0x0e, 0xe6, 0x6d, 0xc9, 0x67, 0x87, 0x82, 0x65, + 0xe2, 0x5d, 0x42, 0x66, 0x2d, 0x83, 0x76, 0x61, 0xc9, 0x71, 0xd5, 0xb6, 0x61, 0xb6, 0x15, 0x7f, + 0x89, 0x9f, 0x7a, 0x16, 0x53, 0x22, 0x01, 0x91, 0x73, 0xfb, 0xf4, 0x88, 0x4b, 0xf1, 0x87, 0x02, + 0x2c, 0xad, 0xeb, 0x5d, 0xc3, 0x6c, 0xf6, 0x3a, 0x46, 0xa2, 0x0b, 0x0c, 0xef, 0x42, 0xce, 0x21, + 0x32, 0x03, 0xeb, 0x1c, 0xc0, 0xc5, 0x2c, 0x4d, 0x21, 0x66, 0xfa, 0x09, 0x14, 0xf1, 0x49, 0xcf, + 0x60, 0xfb, 0x0a, 0x0c, 0xe5, 0x64, 0xa6, 0xaf, 0x5b, 0x21, 0xe0, 0x25, 0x49, 0xbc, 0x4e, 0x9f, + 0x02, 0x0a, 0x57, 0x29, 0x49, 0xa0, 0xf1, 0x29, 0x2c, 0x53, 0xd1, 0x7b, 0xa6, 0x93, 0xb0, 0xbe, + 0xa4, 0x5f, 0x83, 0x0b, 0x51, 0xd1, 0x49, 0x96, 0xfb, 0x05, 0x6f, 0xe5, 0x6d, 0x6c, 0x27, 0x8a, + 0x50, 0x7d, 0x5d, 0x73, 0xc1, 0x49, 0x96, 0xf9, 0x37, 0x05, 0xb8, 0x42, 0x65, 0xd3, 0xad, 0x97, + 0x03, 0x6c, 0x3f, 0xc1, 0xaa, 0x93, 0x28, 0xbc, 0xbe, 0x01, 0x73, 0x0c, 0x26, 0xd3, 0xfe, 0x39, + 0x5b, 0xc9, 0x13, 0x37, 0xa3, 0xe9, 0x5a, 0x36, 0x71, 0x33, 0x78, 0x92, 0xa4, 0xc2, 0x4a, 0x5c, + 0x29, 0x92, 0xac, 0xe9, 0xdf, 0x16, 0x60, 0x89, 0x7b, 0x78, 0xa4, 0x2b, 0x57, 0x0f, 0x89, 0x83, + 0x83, 0x6a, 0x90, 0xd7, 0xe8, 0x2f, 0xc5, 0x1d, 0xf4, 0x30, 0x95, 0x5f, 0x98, 0xe4, 0x1c, 0x32, + 0xb6, 0xd6, 0xa0, 0x87, 0x89, 0x87, 0xe9, 0xfd, 0x26, 0x8a, 0x0a, 0x55, 0x72, 0xa2, 0x7b, 0x49, + 0xc7, 0x11, 0xcd, 0xeb, 0xf9, 0x69, 0x5c, 0x07, 0xbf, 0x9f, 0xe6, 0x4a, 0x60, 0xef, 0xe0, 0xd9, + 0x13, 0x75, 0x28, 0x3e, 0x83, 0x4b, 0xa1, 0xa5, 0xf3, 0x70, 0xc5, 0x53, 0xe7, 0xa8, 0x78, 0x68, + 0xf9, 0x3d, 0xa0, 0xa2, 0x4f, 0x21, 0xb4, 0xc0, 0xae, 0xb0, 0x3a, 0x79, 0x50, 0xe5, 0x3c, 0xea, + 0x58, 0x0a, 0xa4, 0x30, 0xba, 0x83, 0xaa, 0x90, 0xc5, 0x27, 0x3d, 0x45, 0xc7, 0x8e, 0xc6, 0x0d, + 0x97, 0x14, 0x27, 0x90, 0x14, 0x65, 0xc4, 0x79, 0x9f, 0xc7, 0x27, 0x3d, 0x42, 0x44, 0x7b, 0x64, + 0xde, 0xf4, 0xe6, 0x75, 0x5a, 0x6c, 0xe7, 0x6c, 0x2c, 0x10, 0xf4, 0x14, 0x2e, 0xae, 0xe8, 0x4f, + 0xe9, 0x4c, 0x84, 0xf4, 0x23, 0x01, 0xde, 0x8a, 0x6d, 0xb5, 0x24, 0x27, 0xb2, 0x8f, 0x21, 0x43, + 0x2b, 0x9f, 0x3a, 0x67, 0xe5, 0x29, 0x97, 0xf4, 0xbd, 0x14, 0x1f, 0xe3, 0x32, 0xee, 0x58, 0x44, + 0xb1, 0x89, 0x2f, 0xa1, 0x3d, 0x85, 0xc5, 0x63, 0xcb, 0xc5, 0xb6, 0xdf, 0xec, 0xa9, 0x73, 0x37, + 0xfb, 0x02, 0x15, 0xe0, 0xb5, 0xf8, 0x73, 0x58, 0x32, 0x2d, 0x53, 0x89, 0x0a, 0x3d, 0x7f, 0x5f, + 0x2a, 0x9a, 0x96, 0xf9, 0x3c, 0x24, 0xd7, 0xb7, 0x33, 0x43, 0x9a, 0x48, 0xd2, 0xce, 0x7c, 0x5f, + 0x80, 0x65, 0xdf, 0xd3, 0x49, 0xd8, 0xdd, 0xfd, 0x06, 0xa4, 0x4d, 0xeb, 0xd5, 0x79, 0x96, 0x28, + 0x49, 0x7e, 0x32, 0xeb, 0x45, 0x4b, 0x94, 0x64, 0x7d, 0xff, 0x55, 0x0a, 0x72, 0x8f, 0xab, 0x49, + 0xd6, 0xf2, 0x63, 0xbe, 0xfc, 0xcd, 0xda, 0x3b, 0xae, 0xb7, 0xfb, 0xef, 0x2b, 0x3f, 0xae, 0x6e, + 0xe1, 0x81, 0xd7, 0xdb, 0x09, 0x17, 0x5a, 0x87, 0x5c, 0x74, 0xa1, 0x74, 0x4a, 0x4d, 0x05, 0x5c, + 0x2b, 0x18, 0x66, 0xa9, 0x5c, 0x2f, 0xd4, 0x42, 0x88, 0x09, 0xb5, 0x20, 0xaf, 0xf1, 0x3d, 0xc5, + 0xd4, 0x79, 0x5e, 0x13, 0x72, 0x11, 0x67, 0xc5, 0x39, 0xe9, 0x19, 0x00, 0xa9, 0x4e, 0x92, 0x4d, + 0xf2, 0xdd, 0x34, 0x14, 0x76, 0xfb, 0xce, 0x61, 0xc2, 0xbd, 0xaf, 0x0a, 0xd0, 0xeb, 0x3b, 0x87, + 0x64, 0x44, 0x9e, 0x98, 0xbc, 0xce, 0x67, 0x44, 0x71, 0x78, 0x95, 0x66, 0x7c, 0xad, 0x13, 0x13, + 0xd5, 0xb9, 0x10, 0xac, 0x04, 0xa1, 0x20, 0x37, 0x26, 0x21, 0xcb, 0xd6, 0x89, 0xb9, 0x8d, 0x7d, + 0x48, 0xc9, 0x24, 0x61, 0x22, 0xe9, 0x63, 0x98, 0x27, 0x0f, 0x8a, 0x6b, 0x9d, 0xa7, 0x99, 0xe7, + 0x08, 0x4f, 0xcb, 0x42, 0x1f, 0x41, 0x8e, 0x71, 0x93, 0xd9, 0x6f, 0x8e, 0xce, 0x7e, 0x71, 0x75, + 0xe1, 0x6a, 0xa4, 0xf3, 0x5e, 0x96, 0xb2, 0x92, 0xb9, 0xee, 0x02, 0xcc, 0x1e, 0x58, 0xb6, 0xe6, + 0x6d, 0xe6, 0xb2, 0x07, 0xd6, 0x9e, 0x0c, 0xd2, 0x6c, 0x66, 0xb2, 0x39, 0x11, 0xa4, 0xdf, 0x16, + 0xa0, 0xe8, 0x37, 0x44, 0x92, 0x13, 0x42, 0x35, 0xa2, 0xc5, 0xf3, 0x37, 0x05, 0x51, 0xa0, 0xf4, + 0x6f, 0xa8, 0x47, 0xa4, 0x59, 0xc7, 0xb4, 0x65, 0x92, 0xec, 0x29, 0x1f, 0xb1, 0x40, 0x9f, 0xd4, + 0x79, 0x5b, 0x97, 0xc6, 0xfc, 0xdc, 0x87, 0x0b, 0x46, 0x97, 0xd8, 0x73, 0xc3, 0xed, 0x0c, 0x38, + 0x6c, 0x73, 0xb1, 0xb7, 0x6b, 0xbc, 0x1c, 0xa4, 0x55, 0xbd, 0x24, 0xe9, 0xf7, 0xe8, 0x6a, 0x75, + 0x50, 0x93, 0x24, 0x55, 0xdd, 0x80, 0x45, 0x9b, 0x89, 0x26, 0x6e, 0xcd, 0x39, 0xb5, 0xbd, 0xe0, + 0xb3, 0x12, 0x85, 0xff, 0x4e, 0x0a, 0x8a, 0xcf, 0xfa, 0xd8, 0x1e, 0x7c, 0x95, 0xd4, 0x7d, 0x0b, + 0x8a, 0xaf, 0x54, 0xc3, 0x55, 0x0e, 0x2c, 0x5b, 0xe9, 0xf7, 0x74, 0xd5, 0xf5, 0xa2, 0x4d, 0x16, + 0x09, 0xf9, 0x91, 0x65, 0xef, 0x51, 0x22, 0xc2, 0x80, 0x8e, 0x4c, 0xeb, 0x95, 0xa9, 0x10, 0x32, + 0x05, 0xca, 0x27, 0x26, 0x5f, 0x42, 0xae, 0x7c, 0xf3, 0x3f, 0x9c, 0xae, 0x3e, 0x9c, 0x2a, 0x86, + 0x8c, 0xc6, 0xcb, 0xf5, 0xfb, 0x86, 0x5e, 0xde, 0xdb, 0x6b, 0x6c, 0xc8, 0x22, 0x15, 0xf9, 0x82, + 0x49, 0x6c, 0x9d, 0x98, 0x8e, 0xf4, 0x77, 0x53, 0x20, 0x06, 0x3a, 0x4a, 0xb2, 0x21, 0x6b, 0x90, + 0x7f, 0xd9, 0xc7, 0xb6, 0xf1, 0x06, 0xcd, 0x08, 0x9c, 0x91, 0x98, 0x9d, 0x7b, 0xb0, 0xe4, 0x9e, + 0x98, 0x0a, 0x8b, 0xf0, 0x63, 0x81, 0x1f, 0x5e, 0xc0, 0x42, 0xd1, 0x25, 0x65, 0x26, 0x74, 0x1a, + 0xf4, 0xe1, 0xa0, 0xcf, 0x60, 0x21, 0xa2, 0xad, 0xf4, 0x97, 0xd3, 0x56, 0xfe, 0x55, 0x48, 0x51, + 0x7f, 0x20, 0x00, 0xa2, 0x8a, 0x6a, 0xb0, 0x35, 0xfe, 0xaf, 0x4a, 0x7f, 0xba, 0x03, 0x22, 0x8d, + 0xc7, 0x54, 0x8c, 0x03, 0xa5, 0x6b, 0x38, 0x8e, 0x61, 0xb6, 0x79, 0x87, 0x2a, 0x50, 0x7a, 0xe3, + 0x60, 0x9b, 0x51, 0xa5, 0xbf, 0x04, 0xcb, 0x91, 0x0a, 0x24, 0xd9, 0xd8, 0xd7, 0x61, 0xe1, 0x80, + 0x6d, 0xc1, 0x52, 0xe1, 0x7c, 0x79, 0x30, 0x4f, 0x69, 0xec, 0x7d, 0xd2, 0x9f, 0xa6, 0xe0, 0x82, + 0x8c, 0x1d, 0xab, 0x73, 0x8c, 0x93, 0x57, 0x61, 0x1d, 0xf8, 0xde, 0x8b, 0xf2, 0x46, 0x9a, 0xcc, + 0x31, 0x66, 0x36, 0xcd, 0x45, 0xd7, 0xd8, 0xdf, 0x9d, 0xdc, 0x63, 0x47, 0x57, 0xd5, 0xf9, 0x4a, + 0x5d, 0x26, 0xb2, 0x52, 0x67, 0x41, 0x91, 0xed, 0x1e, 0xeb, 0x8a, 0x83, 0x5f, 0x9a, 0xfd, 0xae, + 0x07, 0x86, 0xca, 0x93, 0x0a, 0xd9, 0x60, 0x2c, 0x4d, 0xfc, 0x72, 0xa7, 0xdf, 0xa5, 0xbe, 0x73, + 0xe5, 0x12, 0x29, 0xef, 0xeb, 0xd3, 0xd5, 0x42, 0x24, 0xcd, 0x91, 0x0b, 0x86, 0xff, 0x4c, 0xa4, + 0x4b, 0xdf, 0x86, 0x8b, 0x43, 0xca, 0x4e, 0xd2, 0xe3, 0xf9, 0x17, 0x69, 0xb8, 0x12, 0x15, 0x9f, + 0x34, 0xc4, 0xf9, 0xaa, 0x37, 0x68, 0x1d, 0x16, 0xbb, 0x86, 0xf9, 0x66, 0xab, 0x97, 0x0b, 0x5d, + 0xc3, 0xf4, 0x69, 0x71, 0x5d, 0x63, 0xee, 0x17, 0xda, 0x35, 0x54, 0x58, 0x89, 0x6b, 0xbb, 0x24, + 0xfb, 0xc7, 0xf7, 0x04, 0x58, 0x48, 0x7a, 0x59, 0xee, 0xcd, 0xa2, 0xe0, 0xa4, 0x16, 0x2c, 0xfe, + 0x02, 0xd6, 0xf1, 0x7e, 0x47, 0x00, 0xd4, 0xb2, 0xfb, 0x26, 0x01, 0xb5, 0x4f, 0xac, 0x76, 0x92, + 0xd5, 0xbc, 0x00, 0xb3, 0x86, 0xa9, 0xe3, 0x13, 0x5a, 0xcd, 0x8c, 0xcc, 0x1e, 0x22, 0x5b, 0x89, + 0xe9, 0xa9, 0xb6, 0x12, 0xa5, 0xcf, 0x60, 0x39, 0x52, 0xc4, 0x24, 0xeb, 0xff, 0x27, 0x29, 0x58, + 0xe6, 0x15, 0x49, 0x7c, 0x05, 0xf3, 0xeb, 0x30, 0xdb, 0x21, 0x32, 0x27, 0xb4, 0x33, 0x7d, 0xa7, + 0xd7, 0xce, 0x34, 0x33, 0xfa, 0x15, 0x80, 0x9e, 0x8d, 0x8f, 0x15, 0xc6, 0x9a, 0x9e, 0x8a, 0x35, + 0x47, 0x38, 0x28, 0x01, 0xfd, 0x40, 0x80, 0x22, 0x19, 0xd0, 0x3d, 0xdb, 0xea, 0x59, 0x0e, 0xf1, + 0x59, 0x9c, 0xe9, 0x60, 0xce, 0xb3, 0xd7, 0xa7, 0xab, 0x8b, 0xdb, 0x86, 0xb9, 0xcb, 0x19, 0x5b, + 0xcd, 0xa9, 0x03, 0xfc, 0xbd, 0x63, 0x0e, 0xe5, 0x6a, 0xc7, 0xd2, 0x8e, 0x82, 0xcd, 0x31, 0x62, + 0x59, 0x7c, 0x71, 0x8e, 0xf4, 0x53, 0x01, 0x2e, 0xfc, 0xc2, 0x96, 0x8b, 0xff, 0x7f, 0x28, 0x5b, + 0x7a, 0x0e, 0x22, 0xfd, 0xd1, 0x30, 0x0f, 0xac, 0x24, 0x17, 0xee, 0xff, 0xb7, 0x00, 0x4b, 0x21, + 0xc1, 0x49, 0x3a, 0x38, 0x6f, 0xaa, 0xa7, 0x45, 0x16, 0x0e, 0xe3, 0x4e, 0xa7, 0x2a, 0x79, 0x81, + 0x67, 0x67, 0x9d, 0xb2, 0x0c, 0x0b, 0x98, 0x58, 0x31, 0xba, 0xc4, 0xbb, 0xcf, 0x0e, 0x99, 0x0c, + 0xad, 0xe8, 0xe7, 0xfd, 0x0c, 0x95, 0x81, 0xf4, 0x6b, 0xc4, 0xc3, 0x0a, 0x0f, 0xca, 0x24, 0x87, + 0xfc, 0x3f, 0x4d, 0xc1, 0xa5, 0x2a, 0xdb, 0x02, 0xf7, 0x62, 0x42, 0x92, 0xec, 0x88, 0x25, 0x98, + 0x3f, 0xc6, 0xb6, 0x63, 0x58, 0x6c, 0xb6, 0x5f, 0x94, 0xbd, 0x47, 0xb4, 0x02, 0x59, 0xc7, 0x54, + 0x7b, 0xce, 0xa1, 0xe5, 0x6d, 0x27, 0xfa, 0xcf, 0x7e, 0xfc, 0xca, 0xec, 0x9b, 0xc7, 0xaf, 0xcc, + 0x4d, 0x8e, 0x5f, 0x99, 0xff, 0x12, 0xf1, 0x2b, 0x7c, 0xef, 0xee, 0xdf, 0x0a, 0x70, 0x79, 0x44, + 0x73, 0x49, 0x76, 0xce, 0xef, 0x40, 0x5e, 0xe3, 0x82, 0xc9, 0xfc, 0xc0, 0x36, 0x26, 0x1b, 0x24, + 0xdb, 0x1b, 0x42, 0x9f, 0xd7, 0xa7, 0xab, 0xe0, 0x15, 0xb5, 0xb1, 0xc1, 0x95, 0x43, 0x7e, 0xeb, + 0xd2, 0x7f, 0x03, 0x28, 0xd6, 0x4e, 0xd8, 0xa2, 0x7c, 0x93, 0x79, 0x25, 0xe8, 0x11, 0x64, 0x7b, + 0xb6, 0x75, 0x6c, 0x78, 0xd5, 0x28, 0x44, 0x82, 0x17, 0xbc, 0x6a, 0x0c, 0x71, 0xed, 0x72, 0x0e, + 0xd9, 0xe7, 0x45, 0x2d, 0xc8, 0x3d, 0xb1, 0x34, 0xb5, 0xf3, 0xc8, 0xe8, 0x78, 0x03, 0xed, 0xfd, + 0xb3, 0x05, 0x95, 0x7d, 0x9e, 0x5d, 0xd5, 0x3d, 0xf4, 0x1a, 0xc1, 0x27, 0xa2, 0x06, 0x64, 0xeb, + 0xae, 0xdb, 0x23, 0x89, 0x7c, 0xfc, 0xdd, 0x9e, 0x42, 0x28, 0x61, 0xf1, 0x22, 0x6e, 0x3d, 0x76, + 0xd4, 0x82, 0xa5, 0xc7, 0xf4, 0xfc, 0x58, 0xb5, 0x63, 0xf5, 0xf5, 0xaa, 0x65, 0x1e, 0x18, 0x6d, + 0x3e, 0x4d, 0xdc, 0x9a, 0x42, 0xe6, 0xe3, 0x6a, 0x53, 0x1e, 0x15, 0x80, 0xd6, 0x21, 0xdb, 0x7c, + 0xc8, 0x85, 0x31, 0x37, 0xf2, 0xe6, 0x14, 0xc2, 0x9a, 0x0f, 0x65, 0x9f, 0x0d, 0x6d, 0x42, 0x7e, + 0xfd, 0xf3, 0xbe, 0x8d, 0xb9, 0x94, 0xb9, 0xb1, 0x91, 0x13, 0xc3, 0x52, 0x28, 0x97, 0x1c, 0x66, + 0x46, 0xdf, 0x86, 0x22, 0xd1, 0x5b, 0x4b, 0xdd, 0xef, 0x78, 0xf2, 0xb2, 0x54, 0xde, 0xd7, 0xa6, + 0x90, 0xe7, 0x73, 0x7a, 0x5b, 0x02, 0x43, 0xa2, 0x56, 0x64, 0x58, 0x8c, 0xb4, 0x17, 0x42, 0x90, + 0xe9, 0x91, 0xa6, 0x11, 0x68, 0x18, 0x12, 0xfd, 0x8d, 0xde, 0x83, 0x79, 0xd3, 0xd2, 0xb1, 0xd7, + 0x99, 0x17, 0x2b, 0x17, 0x5e, 0x9f, 0xae, 0xce, 0xed, 0x58, 0x3a, 0xf3, 0x75, 0xf8, 0x2f, 0x79, + 0x8e, 0x64, 0x6a, 0xe8, 0x2b, 0xd7, 0x20, 0x43, 0x9a, 0x88, 0xd8, 0x90, 0x7d, 0xd5, 0xc1, 0x7b, + 0xb6, 0xc1, 0xa5, 0x79, 0x8f, 0x2b, 0xff, 0x20, 0x05, 0xa9, 0xe6, 0x43, 0xe2, 0xcd, 0xef, 0xf7, + 0xb5, 0x23, 0xec, 0xf2, 0x74, 0xfe, 0x44, 0xbd, 0x7c, 0x1b, 0x1f, 0x18, 0xcc, 0xe9, 0xca, 0xc9, + 0xfc, 0x09, 0xbd, 0x03, 0xa0, 0x6a, 0x1a, 0x76, 0x1c, 0xc5, 0x3b, 0x02, 0x98, 0x93, 0x73, 0x8c, + 0xb2, 0x85, 0x07, 0x84, 0xcd, 0xc1, 0x9a, 0x8d, 0x5d, 0x2f, 0x86, 0x8a, 0x3d, 0x11, 0x36, 0x17, + 0x77, 0x7b, 0x8a, 0x6b, 0x1d, 0x61, 0x93, 0x36, 0x69, 0x8e, 0x58, 0x85, 0x6e, 0xaf, 0x45, 0x08, + 0xc4, 0xa0, 0x61, 0x53, 0x0f, 0xac, 0x4f, 0x4e, 0xf6, 0x9f, 0x89, 0x48, 0x1b, 0xb7, 0x0d, 0x7e, + 0x80, 0x2e, 0x27, 0xf3, 0x27, 0xa2, 0x25, 0xb5, 0xef, 0x1e, 0xd2, 0x96, 0xc8, 0xc9, 0xf4, 0x37, + 0xba, 0x05, 0x45, 0x16, 0x76, 0xa9, 0x60, 0x53, 0x53, 0xa8, 0x1d, 0xcc, 0xd1, 0xe4, 0x45, 0x46, + 0xae, 0x99, 0x1a, 0xb1, 0x7a, 0xe8, 0x21, 0x70, 0x82, 0x72, 0xd4, 0x75, 0x88, 0x4e, 0x81, 0xe4, + 0xaa, 0x14, 0x5f, 0x9f, 0xae, 0xe6, 0x9b, 0x34, 0x61, 0x6b, 0xbb, 0x49, 0xe6, 0x12, 0x96, 0x6b, + 0xab, 0xeb, 0x34, 0xf4, 0x95, 0xbf, 0x29, 0x40, 0xfa, 0x71, 0xb5, 0x79, 0x6e, 0x95, 0x79, 0x05, + 0x4d, 0x87, 0x0a, 0x7a, 0x1b, 0x8a, 0xfb, 0x46, 0xa7, 0x63, 0x98, 0x6d, 0xe2, 0x5f, 0x7d, 0x07, + 0x6b, 0x9e, 0xc2, 0x0a, 0x9c, 0xbc, 0xcb, 0xa8, 0xe8, 0x1a, 0xe4, 0x35, 0x1b, 0xeb, 0xd8, 0x74, + 0x0d, 0xb5, 0xe3, 0x70, 0xcd, 0x85, 0x49, 0x2b, 0x7f, 0x59, 0x80, 0x59, 0xda, 0x59, 0xd1, 0xdb, + 0x90, 0xd3, 0x2c, 0xd3, 0x55, 0x0d, 0x93, 0x5b, 0x9d, 0x9c, 0x1c, 0x10, 0xc6, 0x16, 0xef, 0x3a, + 0x2c, 0xa8, 0x9a, 0x66, 0xf5, 0x4d, 0x57, 0x31, 0xd5, 0x2e, 0xe6, 0xc5, 0xcc, 0x73, 0xda, 0x8e, + 0xda, 0xc5, 0x68, 0x15, 0xbc, 0x47, 0xff, 0x64, 0x67, 0x4e, 0x06, 0x4e, 0xda, 0xc2, 0x83, 0x15, + 0x0c, 0x39, 0xbf, 0x57, 0x93, 0xfa, 0xf6, 0x1d, 0xbf, 0x04, 0xf4, 0x37, 0x7a, 0x1f, 0x2e, 0xbc, + 0xec, 0xab, 0x1d, 0xe3, 0x80, 0x2e, 0x7e, 0xd1, 0x28, 0x75, 0xfa, 0x32, 0x56, 0x14, 0xe4, 0xa7, + 0x51, 0x09, 0xf4, 0x9d, 0xde, 0x20, 0x48, 0x07, 0x83, 0x80, 0x85, 0xec, 0x48, 0x03, 0x58, 0x92, + 0xb1, 0x6b, 0x0f, 0x5a, 0xec, 0xb0, 0x6b, 0xed, 0x18, 0x9b, 0x2e, 0xa9, 0xbb, 0xd5, 0xc3, 0x2c, + 0x48, 0xc4, 0xab, 0xbb, 0x4f, 0x40, 0x37, 0xa1, 0xa0, 0xba, 0xa4, 0xbb, 0xb9, 0x8a, 0xd9, 0xef, + 0xee, 0x63, 0x9b, 0x85, 0x02, 0xc8, 0x8b, 0x9c, 0xba, 0x43, 0x89, 0xfc, 0x44, 0x86, 0x3d, 0x50, + 0xe8, 0x3a, 0x11, 0x7f, 0x35, 0x50, 0x52, 0x8d, 0x50, 0xa4, 0xbb, 0x70, 0x91, 0xd4, 0xb3, 0x66, + 0x6a, 0xf6, 0xa0, 0x47, 0x24, 0x3f, 0xa5, 0x7f, 0x1d, 0x24, 0x86, 0xf6, 0x69, 0xe8, 0xf6, 0x8c, + 0xf4, 0x93, 0x39, 0x58, 0xac, 0x9d, 0xf4, 0x2c, 0x3b, 0xd1, 0x55, 0x9d, 0x0a, 0xcc, 0x73, 0xe0, + 0x3b, 0x61, 0x2b, 0x76, 0xc8, 0x02, 0x79, 0xfb, 0xd0, 0x9c, 0x11, 0x55, 0x00, 0x58, 0x40, 0x25, + 0x8d, 0xc3, 0x49, 0x9f, 0x63, 0xe7, 0x88, 0xb2, 0xd1, 0xc3, 0x06, 0x3b, 0x90, 0xef, 0x1e, 0x6b, + 0x9a, 0x72, 0x60, 0x74, 0x5c, 0x1e, 0x97, 0x16, 0x1f, 0x42, 0xbd, 0xfd, 0xbc, 0x5a, 0x7d, 0x44, + 0x33, 0xb1, 0x78, 0xae, 0xe0, 0x59, 0x06, 0x22, 0x81, 0xfd, 0x46, 0x5f, 0x03, 0x7e, 0xf0, 0x45, + 0x71, 0xbc, 0x63, 0x6c, 0x95, 0xc5, 0xd7, 0xa7, 0xab, 0x39, 0x99, 0x52, 0x9b, 0xcd, 0x96, 0x9c, + 0x63, 0x19, 0x9a, 0x8e, 0x7b, 0x9e, 0xa3, 0x0e, 0xf3, 0xd3, 0x1f, 0x75, 0xf8, 0x6b, 0x02, 0x5c, + 0xe2, 0x3a, 0x52, 0xf6, 0x69, 0x78, 0xb7, 0xda, 0x31, 0xdc, 0x81, 0x72, 0x74, 0x5c, 0xca, 0x52, + 0x97, 0xe7, 0x97, 0x63, 0x75, 0x1d, 0x6a, 0xe2, 0xb2, 0xa7, 0xf1, 0xc1, 0x13, 0xce, 0xbc, 0x75, + 0x5c, 0x33, 0x5d, 0x7b, 0x50, 0xb9, 0xfc, 0xfa, 0x74, 0x75, 0x79, 0x34, 0xf5, 0xb9, 0xbc, 0xec, + 0x8c, 0xb2, 0xa0, 0x3a, 0x00, 0xf6, 0xbb, 0x18, 0xb5, 0x60, 0xf1, 0x53, 0x57, 0x6c, 0x5f, 0x94, + 0x43, 0xbc, 0xe8, 0x0e, 0x88, 0xfc, 0x68, 0xc9, 0x81, 0xd1, 0xc1, 0x8a, 0x63, 0x7c, 0x8e, 0xa9, + 0xad, 0x4b, 0xcb, 0x05, 0x46, 0x27, 0x22, 0x9a, 0xc6, 0xe7, 0x18, 0xdd, 0x87, 0x8b, 0x41, 0x0b, + 0x28, 0xfb, 0xb8, 0x63, 0xbd, 0x62, 0xd9, 0xf3, 0x34, 0x3b, 0xf2, 0xb5, 0x5f, 0x21, 0x49, 0x84, + 0x65, 0xe5, 0x3b, 0x50, 0x1a, 0x57, 0xe1, 0xf0, 0x80, 0xc8, 0xb1, 0xfd, 0xca, 0x0f, 0xa2, 0x8b, + 0x15, 0x53, 0x74, 0x5c, 0xbe, 0x60, 0xf1, 0x61, 0xea, 0x03, 0x41, 0xfa, 0xfb, 0x29, 0x58, 0xac, + 0xf4, 0x3b, 0x47, 0x4f, 0x7b, 0x4d, 0x76, 0x2c, 0x1f, 0xbd, 0x05, 0x39, 0x5d, 0x75, 0x55, 0x56, + 0x48, 0x81, 0x1d, 0x31, 0x23, 0x04, 0x5a, 0x9b, 0xdb, 0x50, 0x0c, 0xc5, 0x82, 0xf0, 0x88, 0x77, + 0x5a, 0xed, 0x80, 0x4c, 0x83, 0xd2, 0x3f, 0x80, 0x52, 0x28, 0x23, 0x5d, 0x59, 0x50, 0xb0, 0xe9, + 0xda, 0x06, 0x66, 0xab, 0x63, 0x69, 0x39, 0x14, 0xb0, 0xd2, 0x20, 0xc9, 0x35, 0x96, 0x8a, 0x5a, + 0xb0, 0x40, 0x32, 0x0e, 0x14, 0x6a, 0x05, 0xbd, 0xd5, 0xcb, 0xfb, 0x31, 0xd5, 0x8a, 0x94, 0xbb, + 0x4c, 0xf5, 0x53, 0xa5, 0x3c, 0xf4, 0xa7, 0x9c, 0xc7, 0x01, 0x65, 0xe5, 0x13, 0x10, 0x87, 0x33, + 0x84, 0x75, 0x99, 0x61, 0xba, 0xbc, 0x10, 0xd6, 0x65, 0x3a, 0xa4, 0xa7, 0xcd, 0x4c, 0x36, 0x23, + 0xce, 0x4a, 0x3f, 0x4d, 0x43, 0xc1, 0xeb, 0x99, 0x49, 0xba, 0xd5, 0x15, 0x98, 0x25, 0xfd, 0xc8, + 0x0b, 0xaf, 0xb8, 0x35, 0x61, 0x40, 0xf0, 0x18, 0x6b, 0xd2, 0xbf, 0x3c, 0x04, 0x48, 0x59, 0x93, + 0x30, 0x3f, 0x2b, 0xff, 0x5d, 0x80, 0x0c, 0xf5, 0x64, 0xef, 0x43, 0x86, 0x9e, 0xcb, 0x17, 0x26, + 0x9e, 0xcb, 0xf7, 0xb6, 0xe7, 0x49, 0x56, 0x7f, 0x62, 0x49, 0x85, 0xbc, 0xab, 0x0a, 0x8d, 0xef, + 0xb1, 0x6c, 0x17, 0xeb, 0xdc, 0x53, 0xbc, 0x76, 0x56, 0x3b, 0x7a, 0x9e, 0xb0, 0xc7, 0x87, 0xae, + 0x40, 0x9a, 0xd8, 0xae, 0x79, 0xb6, 0x55, 0xff, 0xfa, 0x74, 0x35, 0x4d, 0xac, 0x16, 0xa1, 0xa1, + 0x35, 0xc8, 0x47, 0xad, 0x09, 0x71, 0x36, 0xa8, 0x39, 0x0c, 0x59, 0x02, 0xe8, 0xf8, 0x43, 0x88, + 0xa1, 0x24, 0xd6, 0x96, 0x7c, 0x93, 0xfe, 0x37, 0x05, 0x1e, 0x93, 0xd8, 0xd4, 0xc8, 0x9c, 0x65, + 0x27, 0x39, 0xa9, 0xdc, 0x05, 0xd1, 0x56, 0x4d, 0xdd, 0xea, 0x1a, 0x9f, 0x63, 0x86, 0xca, 0x1d, + 0xbe, 0x5d, 0x51, 0xf4, 0xe9, 0x14, 0x3e, 0x3b, 0xd2, 0x7f, 0x15, 0x78, 0xfc, 0xa2, 0x5f, 0x8c, + 0x64, 0x37, 0x95, 0xf3, 0x7c, 0x49, 0xcf, 0x3c, 0xb0, 0xbc, 0xf0, 0x8b, 0xb7, 0xc7, 0x05, 0x1b, + 0x35, 0xcc, 0x03, 0xcb, 0xdb, 0x1e, 0xb3, 0x3d, 0x82, 0xb3, 0xf2, 0xab, 0x30, 0x4b, 0x93, 0xdf, + 0xa0, 0x6f, 0xf8, 0x31, 0xb3, 0x29, 0x31, 0x2d, 0xfd, 0x71, 0x0a, 0xde, 0xa5, 0x55, 0x7d, 0x8e, + 0x6d, 0xe3, 0x60, 0xb0, 0x6b, 0x5b, 0x2e, 0xd6, 0x5c, 0xac, 0x07, 0xab, 0x52, 0x09, 0x36, 0x81, + 0x0e, 0x39, 0xbe, 0x9f, 0x67, 0xe8, 0xfc, 0xe6, 0x8c, 0xc7, 0x5f, 0x0e, 0xad, 0x66, 0xd9, 0x3e, + 0x60, 0x63, 0x43, 0xce, 0x32, 0xc9, 0x0d, 0x1d, 0xad, 0x43, 0xae, 0xe7, 0x55, 0xe3, 0x5c, 0x21, + 0x23, 0x3e, 0x17, 0xda, 0x82, 0x22, 0x2f, 0xa8, 0xda, 0x31, 0x8e, 0xb1, 0xa2, 0xba, 0xe7, 0x19, + 0xc2, 0x8b, 0x8c, 0x77, 0x9d, 0xb0, 0xae, 0xbb, 0xd2, 0xdf, 0xc8, 0xc0, 0xcd, 0x33, 0x54, 0x9c, + 0x64, 0xf7, 0x5a, 0x81, 0xec, 0x31, 0x79, 0x91, 0xc1, 0x6b, 0x9f, 0x95, 0xfd, 0x67, 0xb4, 0x1f, + 0x99, 0x07, 0x0e, 0x54, 0xa3, 0x43, 0xe6, 0x0d, 0x16, 0xa4, 0x37, 0x3e, 0x0c, 0x28, 0x3e, 0xe8, + 0x2d, 0x34, 0x63, 0x3c, 0xa2, 0x82, 0x68, 0x36, 0x07, 0x7d, 0x4f, 0x80, 0x15, 0xf6, 0x42, 0x16, + 0x29, 0x36, 0xf4, 0x9a, 0x0c, 0x7d, 0xcd, 0x46, 0xcc, 0x6b, 0xa6, 0xd2, 0x51, 0x39, 0xf4, 0x2e, + 0x5e, 0x90, 0x52, 0xf8, 0x6d, 0xe1, 0xa2, 0xac, 0xfc, 0x96, 0x00, 0xf9, 0x10, 0x01, 0xdd, 0x1a, + 0x39, 0x97, 0x93, 0x7f, 0x1d, 0x77, 0x18, 0xe7, 0xe6, 0xc8, 0x61, 0x9c, 0x4a, 0xf6, 0x8b, 0xd3, + 0xd5, 0x8c, 0xcc, 0xe2, 0xbd, 0xbd, 0x63, 0x39, 0xd7, 0x83, 0x6b, 0x60, 0xd2, 0x43, 0x99, 0xbc, + 0x7b, 0x60, 0x28, 0xac, 0x53, 0xbd, 0x6d, 0x24, 0x0a, 0xeb, 0xc8, 0x93, 0xf4, 0xc3, 0x14, 0x2c, + 0xad, 0xeb, 0x7a, 0xb3, 0x49, 0xa1, 0x40, 0x92, 0x63, 0x0c, 0x41, 0x86, 0xf8, 0x07, 0xfc, 0x0c, + 0x11, 0xfd, 0x8d, 0xde, 0x03, 0xa4, 0x1b, 0x0e, 0xbb, 0x4e, 0xc1, 0x39, 0x54, 0x75, 0xeb, 0x55, + 0xb0, 0x5b, 0xbc, 0xe4, 0xa5, 0x34, 0xbd, 0x04, 0xd4, 0x04, 0xea, 0xb4, 0x2a, 0x8e, 0xab, 0xfa, + 0xab, 0xe1, 0x37, 0xa7, 0x3a, 0x95, 0xc2, 0xbc, 0x59, 0xff, 0x51, 0xce, 0x11, 0x39, 0xf4, 0x27, + 0xf1, 0xd1, 0x0c, 0xd2, 0x28, 0xae, 0xa2, 0x3a, 0xde, 0x79, 0x0a, 0x76, 0x91, 0x43, 0x81, 0xd1, + 0xd7, 0x1d, 0x76, 0x4c, 0x82, 0x85, 0x61, 0x07, 0xaa, 0x49, 0x72, 0x2d, 0xf3, 0xef, 0x08, 0x50, + 0x90, 0xf1, 0x81, 0x8d, 0x9d, 0xc3, 0x24, 0x75, 0xfe, 0x08, 0x16, 0x6c, 0x26, 0x55, 0x39, 0xb0, + 0xad, 0xee, 0x79, 0x6c, 0x45, 0x9e, 0x33, 0x3e, 0xb2, 0xad, 0x2e, 0x37, 0xc9, 0xcf, 0xa1, 0xe8, + 0x97, 0x31, 0xc9, 0xca, 0xff, 0x3d, 0x7a, 0xe2, 0x92, 0x09, 0x4e, 0x7a, 0xdb, 0x36, 0x59, 0x0d, + 0xd0, 0xf5, 0xec, 0x70, 0x41, 0x93, 0x54, 0xc3, 0x7f, 0x11, 0xa0, 0xd0, 0xec, 0xef, 0xb3, 0x6b, + 0x82, 0x92, 0xd3, 0x40, 0x0d, 0x72, 0x1d, 0x7c, 0xe0, 0x2a, 0x6f, 0x14, 0x40, 0x9c, 0x25, 0xac, + 0x34, 0x7c, 0xfa, 0x31, 0x80, 0x4d, 0x8f, 0x1c, 0x51, 0x39, 0xe9, 0x73, 0xca, 0xc9, 0x51, 0x5e, + 0x42, 0x26, 0xb3, 0x4e, 0xd1, 0xaf, 0x66, 0x92, 0xf3, 0xcb, 0x8b, 0x88, 0x75, 0x48, 0x9f, 0xc7, + 0x3a, 0x2c, 0xf1, 0x9d, 0xea, 0x78, 0x0b, 0x51, 0x86, 0x65, 0xea, 0x96, 0x29, 0x6a, 0xaf, 0xd7, + 0x31, 0x3c, 0x9c, 0x42, 0xed, 0x4f, 0x46, 0x5e, 0xa2, 0x49, 0xeb, 0x2c, 0x85, 0x22, 0x14, 0xf4, + 0x5d, 0x01, 0x16, 0x0e, 0x6c, 0x8c, 0x3f, 0xc7, 0x0a, 0x35, 0xc9, 0xd3, 0x6d, 0xc5, 0x6f, 0x90, + 0x32, 0x7c, 0xe9, 0xad, 0xba, 0x3c, 0x7b, 0x71, 0x93, 0xbc, 0x17, 0xed, 0x80, 0xa8, 0x75, 0xd8, + 0xe6, 0xa1, 0x1f, 0x16, 0x30, 0x37, 0xfd, 0x00, 0x28, 0x32, 0xe6, 0x20, 0x32, 0xe0, 0x19, 0x19, + 0x4c, 0xaa, 0xae, 0xf0, 0xab, 0xd9, 0xa8, 0xb3, 0x1d, 0x0d, 0x0b, 0x08, 0x1f, 0xbd, 0x0e, 0xdd, + 0xe8, 0x56, 0x96, 0xb1, 0xaa, 0x73, 0xcf, 0x9d, 0x8c, 0x2b, 0xff, 0x81, 0x8f, 0xab, 0x17, 0xb0, + 0x44, 0xfb, 0x4d, 0xd2, 0x27, 0x28, 0xa5, 0x7f, 0x94, 0x06, 0x14, 0x96, 0xfc, 0x8b, 0xeb, 0x6f, + 0xa9, 0xe4, 0xfa, 0xdb, 0x26, 0x48, 0x21, 0x67, 0xa8, 0xa3, 0x3a, 0xae, 0xc2, 0xe2, 0xcf, 0x1c, + 0xa5, 0x87, 0x6d, 0xc5, 0xc1, 0x9a, 0xc5, 0x2f, 0xd1, 0x11, 0xe4, 0xab, 0x41, 0xce, 0x27, 0xaa, + 0xe3, 0x3e, 0x63, 0xf9, 0x76, 0xb1, 0xdd, 0xa4, 0xb9, 0xd0, 0x43, 0xb8, 0xd4, 0x55, 0x4f, 0xe2, + 0xf8, 0x67, 0x29, 0xff, 0x72, 0x57, 0x3d, 0x19, 0x61, 0xfa, 0x10, 0x56, 0xe2, 0x99, 0x14, 0x07, + 0x7b, 0xfb, 0x53, 0x97, 0x62, 0x18, 0x9b, 0xd8, 0x45, 0xeb, 0x00, 0x01, 0x88, 0xe0, 0x73, 0xf4, + 0x34, 0x18, 0x22, 0xe7, 0x63, 0x08, 0xe9, 0xfb, 0x02, 0x14, 0xb6, 0x8d, 0xb6, 0xad, 0x26, 0x7a, + 0x45, 0x0d, 0xfa, 0x30, 0xba, 0xa1, 0x97, 0x7f, 0xb0, 0x12, 0x17, 0xb0, 0xc1, 0x72, 0x78, 0x8b, + 0x76, 0x9c, 0x81, 0x4c, 0x7d, 0x7e, 0x89, 0x92, 0xb4, 0xf9, 0xff, 0xfe, 0x0a, 0x2c, 0xf0, 0x72, + 0xef, 0x99, 0x86, 0x65, 0xa2, 0xfb, 0x90, 0x6e, 0xf3, 0xa5, 0xed, 0x7c, 0xec, 0x8a, 0x5e, 0x70, + 0x01, 0x5c, 0x7d, 0x46, 0x26, 0x79, 0x09, 0x4b, 0xaf, 0xef, 0xc6, 0x00, 0x8a, 0x20, 0x86, 0x39, + 0xcc, 0xd2, 0xeb, 0xbb, 0xa8, 0x09, 0x45, 0x2d, 0xb8, 0x75, 0x4a, 0x21, 0xec, 0xe9, 0xb1, 0xcb, + 0x5c, 0xb1, 0xf7, 0x7f, 0xd5, 0x67, 0xe4, 0x82, 0x16, 0x49, 0x40, 0xd5, 0xf0, 0x65, 0x47, 0x99, + 0x91, 0x00, 0xa9, 0xe0, 0xa8, 0x6c, 0xf4, 0xa2, 0xa5, 0xfa, 0x4c, 0xe8, 0x4e, 0x24, 0xf4, 0x21, + 0xcc, 0xe9, 0xf4, 0x5a, 0x1d, 0x6e, 0x34, 0xe3, 0x1a, 0x3a, 0x72, 0x7b, 0x51, 0x7d, 0x46, 0xe6, + 0x1c, 0x68, 0x13, 0x16, 0xd8, 0x2f, 0xe6, 0xd2, 0x73, 0x53, 0x77, 0x73, 0xbc, 0x84, 0x90, 0xb3, + 0x51, 0x9f, 0x91, 0xf3, 0x7a, 0x40, 0x45, 0x8f, 0x21, 0xaf, 0x75, 0xb0, 0x6a, 0x73, 0x51, 0xb7, + 0xc6, 0x9e, 0xea, 0x1a, 0xb9, 0x8a, 0xa7, 0x3e, 0x23, 0x83, 0xe6, 0x13, 0x49, 0xa1, 0x6c, 0x7a, + 0x23, 0x0b, 0x97, 0xf4, 0xfe, 0xd8, 0x42, 0x8d, 0x5e, 0x6f, 0x53, 0xa7, 0x4e, 0x88, 0x4f, 0x45, + 0x5f, 0x87, 0x8c, 0xa3, 0xa9, 0x26, 0xb7, 0xbb, 0x57, 0xc7, 0x5c, 0x99, 0x11, 0x30, 0xd3, 0xdc, + 0xe8, 0x23, 0x86, 0x06, 0xdc, 0x13, 0x6f, 0x2d, 0x33, 0x4e, 0xa7, 0x91, 0xa3, 0xd9, 0x44, 0xa7, + 0x98, 0x12, 0x88, 0x1e, 0x54, 0x02, 0x7f, 0x14, 0x7a, 0x5e, 0x92, 0x2e, 0x5e, 0xc6, 0xeb, 0x61, + 0xe4, 0x7c, 0x6b, 0x9d, 0x9e, 0xff, 0xf6, 0x88, 0x68, 0x1b, 0x16, 0x99, 0xa0, 0x3e, 0x3b, 0x7a, + 0x59, 0x5a, 0x1b, 0xbb, 0x4b, 0x19, 0x73, 0xf8, 0xb3, 0x3e, 0x23, 0x2f, 0xa8, 0x21, 0x72, 0x50, + 0xae, 0x2e, 0xb6, 0xdb, 0x6c, 0x95, 0x74, 0x42, 0xb9, 0xc2, 0xa1, 0x5f, 0x7e, 0xb9, 0x28, 0x11, + 0xfd, 0x06, 0x5c, 0x60, 0x82, 0x5c, 0x1e, 0xd1, 0xc2, 0x03, 0x23, 0xde, 0x19, 0xbb, 0xc3, 0x38, + 0xf6, 0xb8, 0x64, 0x7d, 0x46, 0x46, 0xea, 0x48, 0x22, 0xd2, 0xe0, 0x22, 0x7b, 0x03, 0x3f, 0x6f, + 0x67, 0xf3, 0x23, 0x62, 0xa5, 0x1b, 0xf4, 0x15, 0xef, 0x8d, 0x7b, 0x45, 0xec, 0x31, 0xc0, 0xfa, + 0x8c, 0xbc, 0xac, 0x8e, 0xa6, 0x06, 0xd5, 0xb0, 0xf9, 0xc9, 0x26, 0xde, 0xdd, 0xde, 0x9b, 0x5c, + 0x8d, 0xb8, 0x13, 0x61, 0x7e, 0x35, 0x22, 0x89, 0xa4, 0x01, 0xfd, 0x73, 0xdd, 0xb4, 0x33, 0x2d, + 0x8c, 0x6d, 0xc0, 0x98, 0xe3, 0x4f, 0xa4, 0x01, 0x0f, 0x43, 0x64, 0x54, 0x86, 0x54, 0x5b, 0x2b, + 0x2d, 0x8e, 0x9d, 0x1f, 0xfc, 0x23, 0x3e, 0xf5, 0x19, 0x39, 0xd5, 0xd6, 0xd0, 0x27, 0x90, 0x65, + 0xe7, 0x35, 0x4e, 0xcc, 0x52, 0x61, 0xac, 0xc1, 0x8d, 0x9e, 0x7a, 0xa9, 0xcf, 0xc8, 0xf4, 0x88, + 0x08, 0xef, 0xc8, 0x3c, 0x16, 0x9f, 0x8a, 0x28, 0x4f, 0x38, 0xa6, 0x39, 0x74, 0x22, 0x82, 0x74, + 0x18, 0xdb, 0x27, 0xa2, 0x5d, 0x28, 0xd8, 0x2c, 0x5a, 0xd1, 0x8b, 0x2d, 0x16, 0xc7, 0xee, 0xe1, + 0xc7, 0x85, 0x17, 0xd7, 0xe9, 0x3a, 0x4c, 0x88, 0x4e, 0xda, 0x2e, 0x2a, 0x91, 0xb7, 0xdd, 0xd2, + 0xd8, 0xb6, 0x1b, 0x1b, 0xea, 0x4a, 0xda, 0xce, 0x1e, 0x49, 0x44, 0xdf, 0x84, 0x59, 0x36, 0x4e, + 0x10, 0x15, 0x19, 0x17, 0x96, 0x32, 0x34, 0x44, 0x58, 0x7e, 0x62, 0xbd, 0x5c, 0x1e, 0xb2, 0xa7, + 0x74, 0xac, 0x76, 0x69, 0x79, 0xac, 0xf5, 0x1a, 0x0d, 0x3e, 0x24, 0xd6, 0xcb, 0x0d, 0xa8, 0xa4, + 0x03, 0xd9, 0x2c, 0x85, 0x0f, 0xb1, 0x0b, 0x63, 0x3b, 0x50, 0x4c, 0x24, 0x5f, 0x9d, 0x1e, 0xa6, + 0x08, 0xc8, 0xbe, 0x61, 0x75, 0xb0, 0x42, 0x8d, 0xe2, 0xc5, 0xc9, 0x86, 0x35, 0x72, 0x8f, 0x91, + 0x6f, 0x58, 0x19, 0x15, 0x3d, 0x07, 0x91, 0x5f, 0xa6, 0xa1, 0x78, 0x91, 0x25, 0xa5, 0x4b, 0x54, + 0xde, 0xdd, 0xd8, 0x09, 0x31, 0x2e, 0xe8, 0xa8, 0x4e, 0x1c, 0xe6, 0x68, 0x0a, 0xfa, 0x14, 0x96, + 0xa8, 0x3c, 0x45, 0x0b, 0xee, 0x3f, 0x29, 0x95, 0x46, 0x6e, 0xd3, 0x18, 0x7f, 0x55, 0x8a, 0x27, + 0x59, 0xd4, 0x86, 0x92, 0xc8, 0x78, 0x30, 0x4c, 0xc3, 0xa5, 0x73, 0xf7, 0xca, 0xd8, 0xf1, 0x10, + 0xbd, 0xfb, 0x91, 0x8c, 0x07, 0x83, 0x51, 0x48, 0x37, 0x1e, 0xb2, 0x78, 0x6f, 0x8f, 0xed, 0xc6, + 0x63, 0x8c, 0xdd, 0xa2, 0x1b, 0xb1, 0x73, 0x1b, 0x00, 0x0c, 0x26, 0x51, 0xcf, 0xef, 0xea, 0x58, + 0x07, 0x60, 0x38, 0xd2, 0x8e, 0x38, 0x00, 0x1d, 0x8f, 0x46, 0x1c, 0x00, 0xb6, 0xa6, 0x5f, 0xba, + 0x36, 0x7e, 0xb2, 0x0a, 0xef, 0xfa, 0xd1, 0xc9, 0x8a, 0x12, 0xd0, 0x3a, 0xe4, 0x88, 0xcf, 0x3a, + 0xa0, 0x23, 0xfc, 0xfa, 0x58, 0x88, 0x3a, 0x74, 0x04, 0xa7, 0x3e, 0x23, 0x67, 0x5f, 0x72, 0x12, + 0xe9, 0x55, 0x4c, 0x04, 0x1f, 0xdb, 0xf7, 0xc6, 0xf6, 0xaa, 0xd1, 0xb3, 0x17, 0xa4, 0x57, 0xbd, + 0x0c, 0xa8, 0xc1, 0x94, 0xe7, 0xb0, 0xd5, 0xfa, 0xd2, 0xbb, 0x93, 0xa7, 0xbc, 0xe8, 0xde, 0x82, + 0x3f, 0xe5, 0x71, 0x32, 0x9b, 0xf2, 0x74, 0xc5, 0x71, 0xe8, 0x06, 0x7e, 0xe9, 0xe6, 0x84, 0x29, + 0x6f, 0x68, 0xfd, 0x8e, 0x4d, 0x79, 0x7a, 0x93, 0x71, 0x12, 0xef, 0xcf, 0xf6, 0x2e, 0x8f, 0xe1, + 0xe8, 0xe5, 0xf6, 0x58, 0xef, 0x2f, 0xf6, 0x76, 0x1b, 0xe2, 0xfd, 0xd9, 0x91, 0x04, 0xf4, 0x2b, + 0x30, 0xcf, 0xd7, 0x4b, 0x4a, 0x77, 0x26, 0xf8, 0xc3, 0xe1, 0x25, 0x2e, 0xd2, 0x1d, 0x39, 0x0f, + 0x33, 0x0e, 0x6c, 0x9d, 0x86, 0x19, 0xbf, 0xbb, 0x13, 0x8c, 0xc3, 0xc8, 0x52, 0x11, 0x33, 0x0e, + 0x01, 0x99, 0x94, 0xc6, 0x61, 0x6b, 0x0c, 0xa5, 0x5f, 0x1a, 0x5b, 0x9a, 0xe8, 0x62, 0x0b, 0x29, + 0x0d, 0xe7, 0xa1, 0x93, 0x05, 0x9d, 0xab, 0x99, 0x76, 0xbe, 0x36, 0x7e, 0xb2, 0x18, 0x46, 0xad, + 0x75, 0x6f, 0x37, 0x84, 0x69, 0xe5, 0xaf, 0x08, 0x70, 0x8d, 0xf5, 0x01, 0xba, 0x16, 0x3c, 0x50, + 0xfc, 0xa5, 0xfc, 0x10, 0x24, 0xbf, 0x4f, 0xc5, 0x7f, 0xf3, 0xfc, 0x2b, 0xcf, 0xde, 0x1b, 0xdf, + 0x51, 0x27, 0xe5, 0x23, 0xca, 0xe8, 0x32, 0xf0, 0x52, 0x7a, 0x30, 0x56, 0x19, 0x51, 0xc0, 0x45, + 0x94, 0xc1, 0x79, 0x2a, 0xf3, 0x7c, 0xb7, 0xd3, 0x3f, 0xd9, 0x58, 0x14, 0xc5, 0xcd, 0x4c, 0xf6, + 0xb2, 0x58, 0xda, 0xcc, 0x64, 0xaf, 0x88, 0x2b, 0x9b, 0x99, 0xec, 0x5b, 0xe2, 0xdb, 0x9b, 0x99, + 0xec, 0xaa, 0x78, 0x6d, 0x33, 0x93, 0x95, 0xc4, 0x1b, 0xd2, 0xef, 0x5f, 0x81, 0x45, 0x0f, 0xfb, + 0x30, 0x5c, 0xf3, 0x20, 0x8c, 0x6b, 0xae, 0x8e, 0xc3, 0x35, 0x1c, 0x2d, 0x71, 0x60, 0xf3, 0x20, + 0x0c, 0x6c, 0xae, 0x8e, 0x03, 0x36, 0x01, 0x0f, 0x41, 0x36, 0xad, 0x71, 0xc8, 0xe6, 0xee, 0x14, + 0xc8, 0xc6, 0x17, 0x35, 0x0c, 0x6d, 0x36, 0x46, 0xa1, 0xcd, 0xbb, 0x93, 0xa1, 0x8d, 0x2f, 0x2a, + 0x84, 0x6d, 0x3e, 0x1a, 0xc2, 0x36, 0xd7, 0x27, 0x60, 0x1b, 0x9f, 0xdf, 0x03, 0x37, 0x5b, 0xb1, + 0xe0, 0xe6, 0xd6, 0x59, 0xe0, 0xc6, 0x97, 0x13, 0x41, 0x37, 0xf5, 0x38, 0x74, 0x73, 0xf3, 0x0c, + 0x74, 0xe3, 0x8b, 0x0a, 0xc3, 0x9b, 0xad, 0x58, 0x78, 0x73, 0xeb, 0x2c, 0x78, 0x13, 0x14, 0x2b, + 0x8c, 0x6f, 0xbe, 0x11, 0xc1, 0x37, 0xab, 0x63, 0xf1, 0x8d, 0xcf, 0xcd, 0x00, 0xce, 0xc7, 0xc3, + 0x00, 0xe7, 0xfa, 0x04, 0x80, 0x13, 0x28, 0x96, 0x23, 0x9c, 0x7a, 0x1c, 0xc2, 0xb9, 0x79, 0x06, + 0xc2, 0x09, 0x74, 0x11, 0x82, 0x38, 0x3b, 0xf1, 0x10, 0xe7, 0xf6, 0x99, 0x10, 0xc7, 0x97, 0x16, + 0xc5, 0x38, 0xf5, 0x38, 0x8c, 0x73, 0xf3, 0x0c, 0x8c, 0x33, 0x54, 0x32, 0x06, 0x72, 0xd4, 0x89, + 0x20, 0xe7, 0xbd, 0x29, 0x41, 0x8e, 0x2f, 0x3a, 0x0e, 0xe5, 0xe8, 0x93, 0x51, 0x4e, 0x79, 0x5a, + 0x94, 0xe3, 0xbf, 0x24, 0x16, 0xe6, 0xa8, 0x13, 0x61, 0xce, 0x7b, 0x53, 0xc2, 0x9c, 0xa1, 0x8a, + 0x44, 0x71, 0xce, 0x4e, 0x3c, 0xce, 0xb9, 0x7d, 0x26, 0xce, 0x09, 0x5a, 0x31, 0x02, 0x74, 0xd6, + 0x42, 0x40, 0xe7, 0x9d, 0x31, 0x40, 0xc7, 0x67, 0x25, 0x48, 0xe7, 0x5b, 0x23, 0x48, 0x47, 0x9a, + 0x84, 0x74, 0x7c, 0x5e, 0x1f, 0xea, 0xd4, 0xe3, 0xa0, 0xce, 0xcd, 0x33, 0xa0, 0x4e, 0xd0, 0x6f, + 0x42, 0x58, 0xe7, 0xd9, 0x18, 0xac, 0x73, 0xe7, 0x6c, 0xac, 0xe3, 0xcb, 0x1b, 0x02, 0x3b, 0xea, + 0x44, 0xb0, 0xf3, 0xde, 0x94, 0x60, 0x27, 0x68, 0xc1, 0x18, 0xb4, 0xf3, 0x41, 0x14, 0xed, 0x5c, + 0x1b, 0x8f, 0x76, 0x7c, 0x31, 0x1c, 0xee, 0x6c, 0xc5, 0xc2, 0x9d, 0x5b, 0x67, 0xc1, 0x9d, 0xc0, + 0x9a, 0x85, 0xf1, 0xce, 0x4e, 0x3c, 0xde, 0xb9, 0x7d, 0x26, 0xde, 0x09, 0x3a, 0x52, 0x04, 0xf0, + 0x6c, 0xc5, 0x02, 0x9e, 0x5b, 0x67, 0x01, 0x9e, 0x21, 0x53, 0xcb, 0x11, 0xcf, 0x8b, 0xb1, 0x88, + 0xe7, 0xde, 0x34, 0x88, 0xc7, 0x17, 0x3a, 0x02, 0x79, 0x3e, 0x1b, 0x0f, 0x79, 0x7e, 0xe9, 0x1c, + 0xb7, 0x43, 0xc6, 0x62, 0x9e, 0x6f, 0x8d, 0x60, 0x1e, 0x69, 0x12, 0xe6, 0x09, 0x46, 0x86, 0x07, + 0x7a, 0x6a, 0x31, 0x10, 0xe5, 0xdd, 0xc9, 0x10, 0x25, 0x98, 0xc8, 0x03, 0x8c, 0xf2, 0xd1, 0x10, + 0x46, 0xb9, 0x7e, 0x66, 0x20, 0x56, 0x08, 0xa4, 0x54, 0x46, 0x41, 0xca, 0x8d, 0x89, 0x20, 0xc5, + 0x97, 0x10, 0xa0, 0x94, 0xad, 0x58, 0x94, 0x72, 0xeb, 0x2c, 0x94, 0x12, 0x74, 0x85, 0x30, 0x4c, + 0xd9, 0x89, 0x87, 0x29, 0xb7, 0xcf, 0x84, 0x29, 0x43, 0xd3, 0x96, 0x87, 0x53, 0xea, 0x71, 0x38, + 0xe5, 0xe6, 0x19, 0x38, 0x25, 0x3c, 0x6d, 0xf9, 0x40, 0xa5, 0x35, 0x0e, 0xa8, 0xdc, 0x9d, 0x02, + 0xa8, 0x04, 0xce, 0xdc, 0x10, 0x52, 0xf9, 0x64, 0x18, 0xa9, 0x48, 0x93, 0x90, 0x4a, 0xd0, 0x89, + 0x3c, 0xa8, 0xb2, 0x13, 0x0f, 0x55, 0x6e, 0x9f, 0x09, 0x55, 0xc2, 0xe3, 0x3a, 0x84, 0x55, 0x3e, + 0x19, 0xc6, 0x2a, 0xd2, 0x24, 0xac, 0x12, 0x94, 0xc7, 0x03, 0x2b, 0xf5, 0x38, 0xb0, 0x72, 0xf3, + 0x0c, 0xb0, 0x12, 0x32, 0xf7, 0x01, 0x5a, 0xf9, 0xab, 0xd3, 0xa3, 0x95, 0x0f, 0xde, 0x34, 0x4e, + 0xe6, 0x6c, 0xb8, 0xf2, 0xc9, 0x30, 0x5c, 0x91, 0x26, 0xc1, 0x95, 0x40, 0x1f, 0xe7, 0xc6, 0x2b, + 0x6f, 0x8b, 0xef, 0x44, 0x50, 0xcb, 0x9f, 0xce, 0xc1, 0x1c, 0xff, 0x52, 0x51, 0xe4, 0x76, 0x1f, + 0xe1, 0x4d, 0x6e, 0xf7, 0x41, 0x1b, 0xa4, 0x9b, 0x51, 0xb7, 0xe5, 0xec, 0x3b, 0xe1, 0x46, 0x6f, + 0x2d, 0xe3, 0xac, 0x6f, 0x70, 0xcc, 0x16, 0x7d, 0x03, 0x16, 0xfb, 0x0e, 0xb6, 0x95, 0x9e, 0x6d, + 0x58, 0xb6, 0xe1, 0xb2, 0x03, 0x03, 0x42, 0x45, 0xfc, 0xe2, 0x74, 0x75, 0x61, 0xcf, 0xc1, 0xf6, + 0x2e, 0xa7, 0xcb, 0x0b, 0xfd, 0xd0, 0x93, 0xf7, 0x71, 0xa6, 0xd9, 0xe9, 0x3f, 0xce, 0xf4, 0x0c, + 0x44, 0xba, 0x1d, 0x1c, 0xb6, 0xf4, 0xec, 0x26, 0x9d, 0xf8, 0x49, 0x49, 0xd5, 0x43, 0xc6, 0x9c, + 0xde, 0xa8, 0x53, 0xb4, 0xa3, 0x44, 0xd4, 0x04, 0x7a, 0xc7, 0x85, 0xd2, 0xb3, 0x3a, 0x86, 0x36, + 0xa0, 0x13, 0x78, 0xf4, 0x56, 0xe1, 0x89, 0x77, 0x7b, 0xbf, 0x50, 0x0d, 0x77, 0x97, 0x72, 0xca, + 0xf0, 0xca, 0xff, 0x8d, 0xee, 0xc3, 0xc5, 0xae, 0x7a, 0x42, 0xaf, 0x5b, 0x55, 0xbc, 0x19, 0x99, + 0x5e, 0x30, 0xc5, 0x3e, 0xd3, 0x84, 0xba, 0xea, 0x09, 0xfd, 0x7c, 0x14, 0x4b, 0xa2, 0xdf, 0x7e, + 0xb8, 0x0e, 0x0b, 0x3c, 0x70, 0x9b, 0x7d, 0x1a, 0xa6, 0x48, 0x73, 0xf2, 0xef, 0x04, 0xb0, 0xaf, + 0xc3, 0xdc, 0x84, 0x82, 0x6e, 0x38, 0xae, 0x61, 0x6a, 0x2e, 0xbf, 0xc9, 0x95, 0xdd, 0x85, 0xba, + 0xe8, 0x51, 0xd9, 0x75, 0xad, 0x2d, 0x58, 0xd2, 0x3a, 0x86, 0xef, 0xe7, 0xb0, 0x99, 0x67, 0x69, + 0x6c, 0xbf, 0xae, 0xd2, 0xbc, 0xc3, 0x9b, 0xa3, 0x45, 0x2d, 0x4a, 0x46, 0x55, 0x28, 0xb6, 0x55, + 0x17, 0xbf, 0x52, 0x07, 0x8a, 0x77, 0x2e, 0x29, 0x4f, 0xcf, 0x62, 0xbe, 0xf5, 0xfa, 0x74, 0x75, + 0xf1, 0x31, 0x4b, 0x1a, 0x39, 0x9e, 0xb4, 0xd8, 0x0e, 0x25, 0xe8, 0xe8, 0x36, 0x14, 0x55, 0x67, + 0x60, 0x6a, 0xb4, 0x01, 0xb1, 0xe9, 0xf4, 0x1d, 0xea, 0xa6, 0x66, 0xe5, 0x02, 0x25, 0x57, 0x3d, + 0x2a, 0xfa, 0x08, 0x56, 0xf8, 0x85, 0xed, 0xaf, 0x54, 0x5b, 0x57, 0x68, 0xa3, 0x07, 0xc3, 0x43, + 0xa4, 0x3c, 0x97, 0xd9, 0x05, 0xed, 0x24, 0x03, 0x69, 0xe9, 0xf0, 0x45, 0xa8, 0xec, 0xa2, 0x57, + 0x10, 0xf3, 0x9b, 0x99, 0xec, 0x82, 0xb8, 0xb8, 0x99, 0xc9, 0x16, 0xc4, 0xa2, 0xf4, 0xdd, 0x34, + 0x14, 0x89, 0xc5, 0x70, 0x1c, 0xc3, 0x32, 0xeb, 0x7e, 0x88, 0xa0, 0xdf, 0x6b, 0x05, 0x7a, 0x44, + 0xc4, 0x7f, 0x46, 0xab, 0xf4, 0x28, 0x0e, 0xf1, 0xcc, 0xfc, 0xcf, 0x34, 0xa4, 0x65, 0x60, 0x24, + 0x7a, 0x28, 0x62, 0x1d, 0xe6, 0x1c, 0xab, 0x6f, 0x6b, 0xde, 0xe5, 0xe1, 0x77, 0xc7, 0x98, 0xa8, + 0xd0, 0x0b, 0xcb, 0x4d, 0xca, 0x20, 0x73, 0x46, 0xf4, 0x19, 0x14, 0xd9, 0x2f, 0x7a, 0xfc, 0x80, + 0x86, 0xff, 0xb3, 0xb3, 0x15, 0xf7, 0xa7, 0x96, 0xf5, 0x84, 0x33, 0xca, 0x05, 0x27, 0xf2, 0x8c, + 0x3e, 0x81, 0xb7, 0x4d, 0x4b, 0xe9, 0xe2, 0xae, 0x65, 0x0f, 0x14, 0x1b, 0xd3, 0x3e, 0xac, 0x2b, + 0xaa, 0xab, 0xf0, 0x42, 0xb3, 0x98, 0xb3, 0x92, 0x69, 0x6d, 0xd3, 0x2c, 0x32, 0xcf, 0xb1, 0xee, + 0x32, 0xb9, 0x52, 0x19, 0xe6, 0xd8, 0x2f, 0x94, 0x83, 0xd9, 0xa7, 0xad, 0x7a, 0x4d, 0x16, 0x67, + 0xd0, 0x02, 0x64, 0x1f, 0xc9, 0x4f, 0xb7, 0x95, 0xe6, 0xb3, 0x27, 0xa2, 0x80, 0xf2, 0x30, 0x2f, + 0x3f, 0x7d, 0xda, 0x52, 0xb6, 0x9e, 0x8b, 0x29, 0xe9, 0x36, 0x14, 0xa2, 0x25, 0x42, 0x00, 0x73, + 0x72, 0x6d, 0xfb, 0x29, 0xbd, 0x31, 0x3b, 0x07, 0xb3, 0x4f, 0x9e, 0x56, 0xd7, 0x9f, 0x88, 0x82, + 0xf4, 0x27, 0x02, 0x2c, 0x54, 0xd8, 0x15, 0xf0, 0x6c, 0xb3, 0xfd, 0xa3, 0xa1, 0x9d, 0xed, 0x2b, + 0xf1, 0xd0, 0x27, 0x7e, 0x97, 0x7d, 0x1d, 0xb2, 0x7c, 0x8c, 0x79, 0x61, 0xea, 0xab, 0xe3, 0x1d, + 0x5e, 0xba, 0x36, 0xe4, 0x85, 0x1a, 0x79, 0x6c, 0xa8, 0x09, 0xa2, 0xea, 0xe9, 0x56, 0xe1, 0x25, + 0x19, 0x1f, 0x70, 0x34, 0xd4, 0x0c, 0xde, 0x88, 0x51, 0xa3, 0xe4, 0x0f, 0x33, 0x3f, 0xf8, 0xd1, + 0xea, 0x8c, 0xf4, 0x67, 0x19, 0x58, 0xac, 0x84, 0xaf, 0xbb, 0x47, 0x8d, 0xa1, 0xca, 0xc6, 0x4d, + 0xe3, 0x11, 0x8e, 0xf2, 0x84, 0x0f, 0x89, 0xe4, 0x82, 0xbb, 0xf5, 0x59, 0xdd, 0xaf, 0x4d, 0x08, + 0x0a, 0x08, 0x57, 0x3e, 0x60, 0x5c, 0xf9, 0x77, 0x69, 0x7f, 0x16, 0x2a, 0xc3, 0x2c, 0x3b, 0x0a, + 0x25, 0x8c, 0x9c, 0xd2, 0xa6, 0x76, 0x90, 0xb8, 0x99, 0x24, 0x5d, 0x66, 0xd9, 0xc8, 0xac, 0xd5, + 0x7a, 0xa3, 0x3b, 0xe9, 0x82, 0xc9, 0xf7, 0xfc, 0x9f, 0xe8, 0xeb, 0xb3, 0x3b, 0x09, 0xff, 0x1f, + 0xc6, 0x38, 0x91, 0xf7, 0xa1, 0x5f, 0x87, 0xa2, 0x66, 0x75, 0x3a, 0xcc, 0x3b, 0x61, 0xf6, 0x77, + 0xf4, 0x96, 0x12, 0x5a, 0x04, 0xfe, 0x55, 0xc6, 0xb2, 0xff, 0x75, 0xc6, 0xb2, 0xcc, 0xbf, 0xce, + 0x18, 0x0a, 0x40, 0x2f, 0xf8, 0xc2, 0x98, 0xd9, 0x1e, 0x8a, 0x85, 0x9f, 0x7f, 0x93, 0x58, 0x78, + 0x76, 0x82, 0x80, 0xf7, 0xbc, 0x3f, 0x14, 0x78, 0x24, 0xd2, 0x13, 0xcb, 0x3a, 0xea, 0xfb, 0xd1, + 0xeb, 0x2b, 0xe1, 0x1b, 0x06, 0x83, 0x30, 0x5d, 0x7a, 0xcc, 0x24, 0x6e, 0x7e, 0x4d, 0x7d, 0xb9, + 0xf9, 0xf5, 0x3a, 0x2c, 0xf4, 0x6c, 0x7c, 0x80, 0x5d, 0xed, 0x50, 0x31, 0xfb, 0x5d, 0x7e, 0xc6, + 0x26, 0xef, 0xd1, 0x76, 0xfa, 0x5d, 0x74, 0x17, 0x44, 0x3f, 0x0b, 0x47, 0x8c, 0xde, 0xf5, 0x56, + 0x1e, 0x9d, 0xe3, 0x4b, 0xe9, 0x7f, 0x0a, 0xb0, 0x1c, 0xa9, 0x13, 0x1f, 0x53, 0x9b, 0x90, 0xd7, + 0x7d, 0x8f, 0xc6, 0x29, 0x09, 0xe7, 0x0c, 0xe0, 0x0e, 0x33, 0x23, 0x05, 0x2e, 0x79, 0xaf, 0xa5, + 0xf7, 0xd1, 0x07, 0x62, 0x53, 0xe7, 0x14, 0x7b, 0x31, 0x90, 0xb3, 0x11, 0x7a, 0x81, 0x3f, 0xc8, + 0xd2, 0x53, 0x0d, 0x32, 0xe9, 0x7f, 0x09, 0x20, 0xd2, 0x17, 0x3c, 0xc2, 0x58, 0x4f, 0xc4, 0x64, + 0x7a, 0x27, 0x25, 0x52, 0xd3, 0x9f, 0xa2, 0x89, 0x7c, 0x43, 0x23, 0x3d, 0xf4, 0x0d, 0x8d, 0x38, + 0xfb, 0x99, 0xf9, 0x92, 0xf6, 0x53, 0xfa, 0x91, 0x00, 0x05, 0xbf, 0xda, 0xec, 0xe3, 0x79, 0x13, + 0x6e, 0xc7, 0x7c, 0xb3, 0x0f, 0xc4, 0x79, 0xb7, 0x78, 0x4c, 0xf5, 0x3d, 0xbf, 0xf0, 0x2d, 0x1e, + 0xec, 0xc3, 0x66, 0x7f, 0xcb, 0xeb, 0x8e, 0xa4, 0x88, 0xd5, 0xe0, 0xfa, 0x84, 0x37, 0x38, 0xa5, + 0x24, 0xd3, 0xef, 0x8e, 0x5a, 0x9d, 0x63, 0x76, 0xf1, 0xca, 0x54, 0xb6, 0x14, 0xf1, 0xa0, 0x3e, + 0xe0, 0x0b, 0x56, 0x7a, 0xab, 0x49, 0xbf, 0x48, 0xca, 0x7e, 0x3b, 0xd2, 0xa3, 0x90, 0x02, 0x69, + 0x8f, 0x22, 0x5a, 0x9a, 0xca, 0xbe, 0x7b, 0x5a, 0x62, 0x1d, 0xf0, 0x27, 0xe1, 0x96, 0x60, 0xc7, + 0x6f, 0x1f, 0x42, 0xfa, 0x58, 0xed, 0x4c, 0x0a, 0x44, 0x8b, 0xb4, 0x9c, 0x4c, 0x72, 0xa3, 0x47, + 0x91, 0x5b, 0x27, 0x52, 0xe3, 0x57, 0x93, 0x46, 0x55, 0x1a, 0xb9, 0x9d, 0xe2, 0x9b, 0xd1, 0x01, + 0x34, 0xf1, 0xf5, 0xe1, 0x91, 0xf4, 0x61, 0xe6, 0xc7, 0x3f, 0x5a, 0x15, 0xa4, 0x8f, 0x01, 0x11, + 0x5f, 0xc7, 0x7d, 0xd6, 0xb7, 0xec, 0xe0, 0x06, 0x8f, 0xe1, 0x13, 0x11, 0xb3, 0xf1, 0x27, 0x22, + 0xa4, 0x8b, 0xb0, 0x1c, 0xe1, 0x66, 0x16, 0x48, 0xfa, 0x26, 0x5c, 0x79, 0x6c, 0x39, 0x8e, 0xd1, + 0x23, 0xb8, 0x99, 0x0e, 0x75, 0x32, 0x5f, 0xf9, 0x36, 0x37, 0xdb, 0xa3, 0x4b, 0x15, 0x26, 0xb3, + 0x4d, 0x39, 0xd9, 0x7f, 0x96, 0xfe, 0xb5, 0x00, 0x97, 0x47, 0x39, 0x99, 0x96, 0xe3, 0x0e, 0x55, + 0xce, 0x6b, 0x56, 0x70, 0xc1, 0xdc, 0xd9, 0xbd, 0xd5, 0xcb, 0x4e, 0x7c, 0x6f, 0xfe, 0x4e, 0xa5, + 0xab, 0x52, 0x9b, 0xc4, 0xcf, 0x3b, 0x17, 0x38, 0x79, 0x9b, 0x51, 0x03, 0xf3, 0x94, 0x99, 0xce, + 0x3c, 0xb5, 0xa0, 0xb8, 0x69, 0x19, 0x26, 0x71, 0xf1, 0xbd, 0xfa, 0xae, 0x43, 0x61, 0xdf, 0x30, + 0x55, 0x7b, 0xa0, 0x78, 0xf1, 0x8f, 0xc2, 0x59, 0xf1, 0x8f, 0xf2, 0x22, 0xe3, 0xe0, 0x8f, 0xd2, + 0xcf, 0x04, 0x10, 0x03, 0xb1, 0xdc, 0xcc, 0x7f, 0x0d, 0x40, 0xeb, 0xf4, 0x1d, 0x17, 0xdb, 0x5e, + 0x2b, 0x2d, 0xb0, 0x73, 0x16, 0x55, 0x46, 0x6d, 0x6c, 0xc8, 0x39, 0x9e, 0xa1, 0xa1, 0xa3, 0x1b, + 0xd1, 0x2b, 0x14, 0x66, 0x2b, 0xf0, 0x7a, 0xe4, 0xe2, 0x04, 0xd2, 0xec, 0x8e, 0x6b, 0xd9, 0x3e, + 0xdc, 0xe5, 0xcd, 0xee, 0x5d, 0x2e, 0x43, 0x0f, 0x51, 0x63, 0x7a, 0x94, 0xaa, 0x40, 0x7c, 0x90, + 0x63, 0xec, 0x57, 0x29, 0x73, 0x76, 0x95, 0x18, 0x87, 0x57, 0xa5, 0x7f, 0x2e, 0x40, 0xb1, 0xca, + 0x5a, 0xc3, 0x6f, 0xe1, 0x09, 0x16, 0x6d, 0x03, 0xb2, 0xee, 0x89, 0xa9, 0x74, 0xb1, 0xff, 0x3d, + 0x94, 0x73, 0xdc, 0xfe, 0x36, 0xef, 0xb2, 0x47, 0xfa, 0x89, 0x3d, 0xfe, 0x7d, 0x67, 0x3e, 0x5c, + 0xae, 0x94, 0xd9, 0x07, 0xa0, 0xcb, 0xde, 0x07, 0xa0, 0xcb, 0x1b, 0x3c, 0x03, 0x9b, 0x29, 0x7e, + 0xf0, 0x9f, 0x56, 0x05, 0xd9, 0x67, 0x62, 0xce, 0xc4, 0xbd, 0x26, 0xe9, 0xf5, 0x23, 0xd3, 0x3d, + 0x2a, 0x00, 0x84, 0x3e, 0x74, 0xc3, 0x3f, 0x29, 0xbc, 0xbe, 0xa1, 0xec, 0xed, 0x54, 0x9f, 0x6e, + 0x6f, 0x37, 0x5a, 0xad, 0xda, 0x86, 0x28, 0x20, 0x11, 0x16, 0x22, 0x9f, 0xc9, 0x49, 0xb1, 0x8f, + 0x0c, 0xdf, 0xfb, 0x0b, 0x00, 0xc1, 0x17, 0xb7, 0x88, 0xac, 0xad, 0xda, 0xa7, 0xca, 0xf3, 0xf5, + 0x27, 0x7b, 0xb5, 0xa6, 0x38, 0x83, 0x10, 0x14, 0x2a, 0xeb, 0xad, 0x6a, 0x5d, 0x91, 0x6b, 0xcd, + 0xdd, 0xa7, 0x3b, 0xcd, 0x9a, 0xf7, 0x71, 0xe2, 0x7b, 0x1b, 0xb0, 0x10, 0xbe, 0xd3, 0x06, 0x2d, + 0x43, 0xb1, 0x5a, 0xaf, 0x55, 0xb7, 0x94, 0xe7, 0x8d, 0x75, 0xe5, 0xd9, 0x5e, 0x6d, 0x8f, 0xe0, + 0x0d, 0x52, 0x34, 0x4a, 0x7c, 0xb4, 0xf7, 0x84, 0x40, 0x95, 0x22, 0xe4, 0xd9, 0x33, 0xfd, 0xa4, + 0x8e, 0x98, 0xba, 0xb7, 0x0d, 0xf9, 0xd0, 0x5d, 0xbb, 0xe4, 0x75, 0xbb, 0x7b, 0xcd, 0xba, 0xd2, + 0x6a, 0x6c, 0xd7, 0x9a, 0xad, 0xf5, 0xed, 0x5d, 0x26, 0x83, 0xd2, 0xd6, 0x2b, 0x4f, 0xe5, 0x96, + 0x28, 0xf8, 0xcf, 0xad, 0xa7, 0x7b, 0xd5, 0xba, 0x57, 0x0d, 0x29, 0x93, 0x4d, 0x8b, 0xe9, 0x7b, + 0x27, 0x70, 0x79, 0xcc, 0xf5, 0x2e, 0x04, 0x25, 0xed, 0x99, 0xf4, 0xde, 0x51, 0x71, 0x06, 0x2d, + 0x42, 0x8e, 0x74, 0x3d, 0x7a, 0xf8, 0x53, 0x14, 0x50, 0x16, 0x32, 0x87, 0xae, 0xdb, 0x13, 0x53, + 0x68, 0x0e, 0x52, 0xce, 0x43, 0x31, 0x4d, 0xfe, 0xb7, 0x1d, 0x31, 0x43, 0x00, 0x93, 0xfa, 0x79, + 0xdf, 0xc6, 0xe2, 0x2c, 0x01, 0x5d, 0x7d, 0x07, 0xdb, 0x07, 0x46, 0x07, 0x8b, 0xf3, 0x84, 0xc5, + 0xec, 0x77, 0x3a, 0x62, 0x56, 0xca, 0x64, 0xe7, 0xc4, 0xb9, 0x7b, 0xd7, 0x21, 0x74, 0xca, 0x9e, + 0x60, 0xae, 0x27, 0xaa, 0x8b, 0x1d, 0x57, 0x9c, 0x41, 0xf3, 0x90, 0x5e, 0xef, 0x74, 0x44, 0xe1, + 0xc1, 0x3f, 0xcb, 0x40, 0xd6, 0xfb, 0x62, 0x0c, 0x7a, 0x02, 0xb3, 0x14, 0x5f, 0xa0, 0xd5, 0xf1, + 0xc8, 0x83, 0x8e, 0xe3, 0x95, 0x6b, 0x67, 0x41, 0x13, 0x69, 0x06, 0xfd, 0x45, 0xc8, 0x87, 0x3c, + 0x32, 0x34, 0x76, 0x19, 0x30, 0xe2, 0x85, 0xae, 0xdc, 0x3a, 0x2b, 0x9b, 0x2f, 0xff, 0x05, 0xe4, + 0x7c, 0x63, 0x8e, 0x6e, 0x4c, 0x32, 0xf5, 0x9e, 0xec, 0xc9, 0xf3, 0x01, 0x19, 0x76, 0xd2, 0xcc, + 0xfb, 0x02, 0xb2, 0x01, 0x8d, 0xda, 0x5d, 0x14, 0x17, 0xfc, 0x36, 0xd6, 0xb0, 0xaf, 0xdc, 0x9b, + 0x2a, 0x77, 0xf0, 0x4e, 0xa2, 0xac, 0x60, 0xf2, 0x88, 0x57, 0xd6, 0xc8, 0xd4, 0x14, 0xaf, 0xac, + 0x98, 0x39, 0x68, 0x06, 0x3d, 0x83, 0x0c, 0x31, 0x9a, 0x28, 0xce, 0xed, 0x1a, 0x32, 0xd2, 0x2b, + 0x37, 0x26, 0xe6, 0xf1, 0x44, 0x56, 0xee, 0xfe, 0xf8, 0x3f, 0x5f, 0x9d, 0xf9, 0xf1, 0xeb, 0xab, + 0xc2, 0xcf, 0x5e, 0x5f, 0x15, 0xfe, 0xe8, 0xf5, 0x55, 0xe1, 0x8f, 0x5f, 0x5f, 0x15, 0xbe, 0xff, + 0xf3, 0xab, 0x33, 0x3f, 0xfb, 0xf9, 0xd5, 0x99, 0x3f, 0xfa, 0xf9, 0xd5, 0x99, 0xcf, 0xe6, 0x39, + 0xf7, 0xfe, 0x1c, 0xb5, 0x28, 0x0f, 0xff, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x66, 0x1d, 0x87, + 0xfe, 0xb9, 0x7f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -15264,6 +15318,21 @@ func (m *AdmissionHeader) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.NoMemoryReservedAtSource { + i-- + if m.NoMemoryReservedAtSource { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } + if m.SourceLocation != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.SourceLocation)) + i-- + dAtA[i] = 0x20 + } if m.Source != 0 { i = encodeVarintApi(dAtA, i, uint64(m.Source)) i-- @@ -19062,6 +19131,12 @@ func (m *AdmissionHeader) Size() (n int) { if m.Source != 0 { n += 1 + sovApi(uint64(m.Source)) } + if m.SourceLocation != 0 { + n += 1 + sovApi(uint64(m.SourceLocation)) + } + if m.NoMemoryReservedAtSource { + n += 2 + } return n } @@ -37080,6 +37155,45 @@ func (m *AdmissionHeader) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceLocation", wireType) + } + m.SourceLocation = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SourceLocation |= AdmissionHeader_SourceLocation(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NoMemoryReservedAtSource", 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.NoMemoryReservedAtSource = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 5ebe6346c4d5..9fbd978e85f8 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1997,6 +1997,7 @@ message AdmissionHeader { // request or a parent request. See admission.WorkInfo.Priority for details. // It is used to give preference to older requests. int64 create_time = 2; + // Source represents the immediate source of a request. FROM_SQL represents // a KV request originating in SQL, and ROOT_KV represents a request // originating within KV, but at the root of the tree of requests. @@ -2013,6 +2014,35 @@ message AdmissionHeader { ROOT_KV = 2; } Source source = 3; + + // SourceLocation specifies physically where the call originated. LOCAL + // means the client is collocated on the same node as the server. It is set + // on codepaths that use internalClientAdapter which avoids using gRPC for + // local calls to the KV API. + enum SourceLocation { + REMOTE = 0; + LOCAL = 1; + } + SourceLocation source_location = 4; + + // NoMemoryReservedAtSource is set by the source/client when it has + // effectively reserved close to 0 bytes. It is read by the server only when + // SourceLocation=LOCAL, to differentiate this 0 reservation case from the + // case where the client has already reserved enough memory based on + // previous responses. In the latter case the server avoids reserving more + // since it would result in double counting. Not setting this defaults to + // turning off server reserving more memory -- this optimistic choice was + // made to err on the side of avoiding double counting in case we forget to + // instrument some calling path. + // + // NOTE: This field is a temporary field until we move to comprehensive + // accounting at the client, by reserving all the bytes for responses, and + // explicitly propagating these through DistSender to the servers, so the + // servers can (best-effort) respect these values when producing responses. + // In that future world, the local server will explicitly know what has + // already been accounted for, and can start reserving more only when it + // exceeds. + bool no_memory_reserved_at_source = 5; } // A BatchRequest contains one or more requests to be executed in diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index f2c38589375e..dc57770e5a4d 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -459,6 +459,9 @@ type internalClientAdapter struct { func (a internalClientAdapter) Batch( ctx context.Context, ba *roachpb.BatchRequest, _ ...grpc.CallOption, ) (*roachpb.BatchResponse, error) { + // Mark this as originating locally, which is useful for the decision about + // memory allocation tracking. + ba.AdmissionHeader.SourceLocation = roachpb.AdmissionHeader_LOCAL return a.InternalServer.Batch(ctx, ba) } @@ -568,6 +571,8 @@ func (a internalClientAdapter) RangeFeed( respStreamClientAdapter: makeRespStreamClientAdapter(ctx), } + // Mark this as originating locally. + args.AdmissionHeader.SourceLocation = roachpb.AdmissionHeader_LOCAL go func() { defer cancel() err := a.InternalServer.RangeFeed(args, rfAdapter) diff --git a/pkg/server/server.go b/pkg/server/server.go index 3e6a621e3e40..0cddee7ded96 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -88,6 +88,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -178,6 +179,10 @@ type Server struct { externalStorageBuilder *externalStorageBuilder gcoord *admission.GrantCoordinator + // kvMemoryMonitor is a child of the rootSQLMemoryMonitor and is used to + // account for and bound the memory used for request processing in the KV + // layer. + kvMemoryMonitor *mon.BytesMonitor // The following fields are populated at start time, i.e. in `(*Server).Start`. startTime time.Time @@ -555,6 +560,20 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // ClosedTimestamp), but the Node needs a StoreConfig to be made. var lateBoundNode *Node + // Break a circular dependency: we need the rootSQLMemoryMonitor to construct + // the KV memory monitor for the StoreConfig. + sqlMonitorAndMetrics := newRootSQLMemoryMonitor(monitorAndMetricsOptions{ + memoryPoolSize: cfg.MemoryPoolSize, + histogramWindowInterval: cfg.HistogramWindowInterval(), + settings: cfg.Settings, + }) + kvMemoryMonitor := mon.NewMonitorInheritWithLimit( + "kv-mem", 0 /* limit */, sqlMonitorAndMetrics.rootSQLMemoryMonitor) + kvMemoryMonitor.Start(ctx, sqlMonitorAndMetrics.rootSQLMemoryMonitor, mon.BoundAccount{}) + stopper.AddCloser(stop.CloserFn(func() { + kvMemoryMonitor.Stop(ctx) + })) + storeCfg := kvserver.StoreConfig{ DefaultZoneConfig: &cfg.DefaultZoneConfig, Settings: st, @@ -603,6 +622,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { ExternalStorage: externalStorage, ExternalStorageFromURI: externalStorageFromURI, ProtectedTimestampCache: protectedtsProvider, + KVMemoryMonitor: kvMemoryMonitor, } if storeTestingKnobs := cfg.TestingKnobs.Store; storeTestingKnobs != nil { storeCfg.TestingKnobs = *storeTestingKnobs.(*kvserver.StoreTestingKnobs) @@ -737,6 +757,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { protectedtsProvider: protectedtsProvider, rangeFeedFactory: rangeFeedFactory, sqlStatusServer: sStatus, + monitorAndMetrics: sqlMonitorAndMetrics, }) if err != nil { return nil, err @@ -783,6 +804,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { drainSleepFn: drainSleepFn, externalStorageBuilder: externalStorageBuilder, gcoord: gcoord, + kvMemoryMonitor: kvMemoryMonitor, } return lateBoundServer, err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 3a787e99e31c..1a27c7371020 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -16,6 +16,7 @@ import ( "net" "os" "path/filepath" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/blobs" @@ -46,6 +47,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/settingswatcher" "github.com/cockroachdb/cockroach/pkg/server/status" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -250,6 +252,46 @@ type sqlServerArgs struct { // Used to watch settings and descriptor changes. rangeFeedFactory *rangefeed.Factory + + // monitorAndMetrics contains the return value of newRootSQLMemoryMonitor. + monitorAndMetrics monitorAndMetrics +} + +type monitorAndMetrics struct { + rootSQLMemoryMonitor *mon.BytesMonitor + rootSQLMetrics sql.BaseMemoryMetrics +} + +type monitorAndMetricsOptions struct { + memoryPoolSize int64 + histogramWindowInterval time.Duration + settings *cluster.Settings +} + +// newRootSQLMemoryMonitor returns a started BytesMonitor and corresponding +// metrics. +func newRootSQLMemoryMonitor(opts monitorAndMetricsOptions) monitorAndMetrics { + rootSQLMetrics := sql.MakeBaseMemMetrics("root", opts.histogramWindowInterval) + // We do not set memory monitors or a noteworthy limit because the children of + // this monitor will be setting their own noteworthy limits. + rootSQLMemoryMonitor := mon.NewMonitor( + "root", + mon.MemoryResource, + rootSQLMetrics.CurBytesCount, + rootSQLMetrics.MaxBytesHist, + -1, /* increment: use default increment */ + math.MaxInt64, /* noteworthy */ + opts.settings, + ) + // Set the limit to the memoryPoolSize. Note that this memory monitor also + // serves as a parent for a memory monitor that accounts for memory used in + // the KV layer at the same node. + rootSQLMemoryMonitor.Start( + context.Background(), nil, mon.MakeStandaloneBudget(opts.memoryPoolSize)) + return monitorAndMetrics{ + rootSQLMemoryMonitor: rootSQLMemoryMonitor, + rootSQLMetrics: rootSQLMetrics, + } } func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { @@ -330,25 +372,15 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { ) cfg.registry.AddMetricStruct(leaseMgr.MetricsStruct()) - rootSQLMetrics := sql.MakeBaseMemMetrics("root", cfg.HistogramWindowInterval()) + rootSQLMetrics := cfg.monitorAndMetrics.rootSQLMetrics cfg.registry.AddMetricStruct(rootSQLMetrics) // Set up internal memory metrics for use by internal SQL executors. internalMemMetrics := sql.MakeMemMetrics("internal", cfg.HistogramWindowInterval()) cfg.registry.AddMetricStruct(internalMemMetrics) - // We do not set memory monitors or a noteworthy limit because the children of - // this monitor will be setting their own noteworthy limits. - rootSQLMemoryMonitor := mon.NewMonitor( - "root", - mon.MemoryResource, - rootSQLMetrics.CurBytesCount, - rootSQLMetrics.MaxBytesHist, - -1, /* increment: use default increment */ - math.MaxInt64, /* noteworthy */ - cfg.Settings, - ) - rootSQLMemoryMonitor.Start(context.Background(), nil, mon.MakeStandaloneBudget(cfg.MemoryPoolSize)) + rootSQLMemoryMonitor := cfg.monitorAndMetrics.rootSQLMemoryMonitor + // bulkMemoryMonitor is the parent to all child SQL monitors tracking bulk // operations (IMPORT, index backfill). It is itself a child of the // ParentMemoryMonitor. diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 2d99fc8147ed..d9e0e843e369 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -57,6 +57,11 @@ func StartTenant( if err != nil { return nil, "", "", err } + args.monitorAndMetrics = newRootSQLMemoryMonitor(monitorAndMetricsOptions{ + memoryPoolSize: args.MemoryPoolSize, + histogramWindowInterval: args.HistogramWindowInterval(), + settings: args.Settings, + }) s, err := newSQLServer(ctx, args) if err != nil { return nil, "", "", err diff --git a/pkg/sql/logictest/testdata/logic_test/mem_limit b/pkg/sql/logictest/testdata/logic_test/mem_limit new file mode 100644 index 000000000000..fb627f3ad9a3 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/mem_limit @@ -0,0 +1,47 @@ +# LogicTest: local + +# logicTest.newCluster uses a 192MB limit. We can insert multiple large rows +# individually, but reading them out in a single scan caused by an index join +# trips the memory limit in KV during the scan. +# +# Once we improve the handling for a single query as discussed in +# https://github.com/cockroachdb/cockroach/issues/54680 we will need to +# rewrite this test to issue multiple concurrent queries in a loop until we +# trigger a condition where the aggregate memory across the concurrently +# executing queries exceeds the limit, and one of them returns an error. + +statement ok +CREATE TABLE foo (id INT PRIMARY KEY, attribute INT, blob TEXT, INDEX(attribute)) + +statement ok +INSERT INTO foo SELECT 1, 10, repeat('a', 60000000) + +statement ok +INSERT INTO foo SELECT 2, 10, repeat('a', 60000000) + +statement ok +INSERT INTO foo SELECT 3, 10, repeat('a', 60000000) + +statement ok +INSERT INTO foo SELECT 4, 10, repeat('a', 60000000) + +query T +EXPLAIN SELECT * FROM foo@foo_attribute_idx WHERE attribute=10 AND blob LIKE 'blah%' +---- +distribution: local +vectorized: true +· +• filter +│ filter: blob LIKE 'blah%' +│ +└── • index join + │ table: foo@primary + │ + └── • scan + missing stats + table: foo@foo_attribute_idx + spans: [/10 - /10] + +query error scan with start key .* memory budget exceeded +SELECT * FROM foo@foo_attribute_idx WHERE attribute=10 AND blob LIKE 'blah%' + diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index d7ebbffec952..799bde959b84 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -427,6 +427,14 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { monitoring := f.acc.Monitor() != nil const tokenFetchAllocation = 1 << 10 + if !monitoring || f.acc.Used() < tokenFetchAllocation { + // In case part of this batch ends up being evaluated locally, we want + // that local evaluation to do memory accounting since we have reserved + // negligible bytes. Ideally, we would split the memory reserved across + // the various servers that DistSender will split this batch into, but we + // do not yet have that capability. + ba.AdmissionHeader.NoMemoryReservedAtSource = true + } if monitoring && f.acc.Used() < tokenFetchAllocation { // Pre-reserve a token fraction of the maximum amount of memory this scan // could return. Most of the time, scans won't use this amount of memory, @@ -444,6 +452,9 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { if err != nil { return err } + // TODO(sumeer): move admission control after the memory accounting below, + // since we are using the memory and must account for it even if the rest of + // the response processing has to wait. if f.responseAdmissionQ != nil { responseAdmission := admission.WorkInfo{ TenantID: roachpb.SystemTenantID, diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 08123d6c0cb7..2f4d0002cd78 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -91,6 +91,7 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/iterutil", "//pkg/util/log", + "//pkg/util/mon", "//pkg/util/protoutil", "//pkg/util/stop", "//pkg/util/syncutil", diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 87526347b305..a8023ac66e2b 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -803,6 +804,8 @@ type MVCCGetOptions struct { // // The field is only set if Txn is also set. LocalUncertaintyLimit hlc.Timestamp + // MemoryMonitor is used for tracking memory allocations. + MemoryMonitor ResponseMemoryAccount } func (opts *MVCCGetOptions) validate() error { @@ -881,6 +884,7 @@ func mvccGet( // key different than the start key. This is a bit of a hack. *mvccScanner = pebbleMVCCScanner{ parent: iter, + memMonitor: opts.MemoryMonitor, start: key, ts: timestamp, maxKeys: 1, @@ -891,7 +895,7 @@ func mvccGet( } mvccScanner.init(opts.Txn, opts.LocalUncertaintyLimit) - mvccScanner.get() + mvccScanner.get(ctx) if mvccScanner.err != nil { return optionalValue{}, nil, mvccScanner.err @@ -2374,6 +2378,7 @@ func mvccScanToBytes( *mvccScanner = pebbleMVCCScanner{ parent: iter, + memMonitor: opts.MemoryMonitor, reverse: opts.Reverse, start: key, end: endKey, @@ -2391,7 +2396,7 @@ func mvccScanToBytes( var res MVCCScanResult var err error - res.ResumeSpan, err = mvccScanner.scan() + res.ResumeSpan, err = mvccScanner.scan(ctx) if err != nil { return MVCCScanResult{}, err @@ -2471,6 +2476,42 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { return intents, nil } +// ResponseMemoryAccount is used to track memory allocations when producing a +// response from the MVCC layer. The Grow method is used to track memory added +// to the response. The provider of this monitor knows when the memory is no +// longer needed, as multiple responses are typically coalesced into a batch +// response. So it is typical to not need to use Shrink, though it is provided +// as a convenience. +// +// An empty initialized ResponseMemoryAccount is safe to use, and is utilized +// when the call path does not wish to do memory accounting, or has not yet +// been instrumented to do such accounting (so using +// MVCCScanOptions.MemoryMonitor always "works"). This also avoids up to two +// memory allocations on paths that don't do memory accounting -- to construct +// a NewUnlimitedMonitor and a BoundAccount. +type ResponseMemoryAccount struct { + B *mon.BoundAccount +} + +// Grow grows the reserved memory by x bytes. +func (b ResponseMemoryAccount) Grow(ctx context.Context, x int64) error { + if b.B == nil { + return nil + } + if err := b.B.Grow(ctx, x); err != nil { + return errors.Wrapf(err, "used bytes %d", b.B.Used()) + } + return nil +} + +// Shrink releases part of the cumulated allocations by the specified size. +func (b ResponseMemoryAccount) Shrink(ctx context.Context, delta int64) { + if b.B == nil { + return + } + b.B.Shrink(ctx, delta) +} + // MVCCScanOptions bundles options for the MVCCScan family of functions. type MVCCScanOptions struct { // See the documentation for MVCCScan for information on these parameters. @@ -2523,6 +2564,8 @@ type MVCCScanOptions struct { // Not used in inconsistent scans. // The zero value indicates no limit. MaxIntents int64 + // MemoryMonitor is used for tracking memory allocations. + MemoryMonitor ResponseMemoryAccount } func (opts *MVCCScanOptions) validate() error { diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 0d4d5a977a30..1176269f62b6 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -12,6 +12,7 @@ package storage import ( "bytes" + "context" "encoding/binary" "sort" "sync" @@ -45,7 +46,9 @@ func (p *pebbleResults) clear() { // The repr that MVCCScan / MVCCGet expects to provide as output goes: // // This function adds to repr in that format. -func (p *pebbleResults) put(key []byte, value []byte) { +func (p *pebbleResults) put( + ctx context.Context, key []byte, value []byte, monitor ResponseMemoryAccount, +) error { // Key value lengths take up 8 bytes (2 x Uint32). const kvLenSize = 8 const minSize = 16 @@ -76,6 +79,9 @@ func (p *pebbleResults) put(key []byte, value []byte) { if len(p.repr) > 0 { p.bufs = append(p.bufs, p.repr) } + if err := monitor.Grow(ctx, int64(newSize)); err != nil { + return err + } p.repr = nonZeroingMakeByteSlice(newSize)[:0] } @@ -87,6 +93,7 @@ func (p *pebbleResults) put(key []byte, value []byte) { copy(p.repr[startIdx+kvLenSize+lenKey:], value) p.count++ p.bytes += int64(lenToAdd) + return nil } func (p *pebbleResults) finish() [][]byte { @@ -100,9 +107,10 @@ func (p *pebbleResults) finish() [][]byte { // Go port of mvccScanner in libroach/mvcc.h. Stores all variables relating to // one MVCCGet / MVCCScan call. type pebbleMVCCScanner struct { - parent MVCCIterator - reverse bool - peeked bool + parent MVCCIterator + memMonitor ResponseMemoryAccount + reverse bool + peeked bool // Iteration bounds. Does not contain MVCC timestamp. start, end roachpb.Key // Timestamp with which MVCCScan/MVCCGet was called. @@ -194,19 +202,19 @@ func (p *pebbleMVCCScanner) init(txn *roachpb.Transaction, localUncertaintyLimit } // get iterates exactly once and adds one KV to the result set. -func (p *pebbleMVCCScanner) get() { +func (p *pebbleMVCCScanner) get(ctx context.Context) { p.isGet = true p.parent.SeekGE(MVCCKey{Key: p.start}) if !p.updateCurrent() { return } - p.getAndAdvance() + p.getAndAdvance(ctx) p.maybeFailOnMoreRecent() } // scan iterates until maxKeys records are in results, or the underlying // iterator is exhausted, or an error is encountered. -func (p *pebbleMVCCScanner) scan() (*roachpb.Span, error) { +func (p *pebbleMVCCScanner) scan(ctx context.Context) (*roachpb.Span, error) { p.isGet = false if p.reverse { if !p.iterSeekReverse(MVCCKey{Key: p.end}) { @@ -218,7 +226,7 @@ func (p *pebbleMVCCScanner) scan() (*roachpb.Span, error) { } } - for p.getAndAdvance() { + for p.getAndAdvance(ctx) { } p.maybeFailOnMoreRecent() @@ -326,13 +334,13 @@ func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) bool { // Emit a tuple and return true if we have reason to believe iteration can // continue. -func (p *pebbleMVCCScanner) getAndAdvance() bool { +func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { if !p.curUnsafeKey.Timestamp.IsEmpty() { // ts < read_ts if p.curUnsafeKey.Timestamp.Less(p.ts) { // 1. Fast path: there is no intent and our read timestamp is newer // than the most recent version's timestamp. - return p.addAndAdvance(p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curRawKey, p.curValue) } // ts == read_ts @@ -350,7 +358,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // 3. There is no intent and our read timestamp is equal to the most // recent version's timestamp. - return p.addAndAdvance(p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curRawKey, p.curValue) } // ts > read_ts @@ -376,7 +384,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // This value is not within the reader's uncertainty window, but // there could be other uncertain committed values, so seek and // check uncertainty using globalUncertaintyLimit. - return p.seekVersion(p.globalUncertaintyLimit, true) + return p.seekVersion(ctx, p.globalUncertaintyLimit, true) } // 6. Our txn's read timestamp is greater than or equal to the @@ -384,7 +392,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // unnecessary. We need to seek to the desired version of the // value (i.e. one with a timestamp earlier than our read // timestamp). - return p.seekVersion(p.ts, false) + return p.seekVersion(ctx, p.ts, false) } if len(p.curValue) == 0 { @@ -398,7 +406,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { } if len(p.meta.RawBytes) != 0 { // 7. Emit immediately if the value is inline. - return p.addAndAdvance(p.curRawKey, p.meta.RawBytes) + return p.addAndAdvance(ctx, p.curRawKey, p.meta.RawBytes) } if p.meta.Txn == nil { @@ -435,9 +443,9 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // The intent is not within the uncertainty window, but there could // be an uncertain committed value, so seek and check uncertainty // using globalUncertaintyLimit. - return p.seekVersion(p.globalUncertaintyLimit, true) + return p.seekVersion(ctx, p.globalUncertaintyLimit, true) } - return p.seekVersion(p.ts, false) + return p.seekVersion(ctx, p.ts, false) } if p.inconsistent { @@ -459,12 +467,19 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // that lie before the resume key. return false } + // p.intents is a pebble.Batch which grows its byte slice capacity in + // chunks to amortize allocations. The memMonitor is under-counting here + // by only accounting for the key and value bytes. + if p.err = p.memMonitor.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) + return false + } p.err = p.intents.Set(p.curRawKey, p.curValue, nil) if p.err != nil { return false } - return p.seekVersion(prevTS, false) + return p.seekVersion(ctx, prevTS, false) } if !ownIntent { @@ -479,6 +494,14 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // Note that this will trigger an error higher up the stack. We // continue scanning so that we can return all of the intents // in the scan range. + // + // p.intents is a pebble.Batch which grows its byte slice capacity in + // chunks to amortize allocations. The memMonitor is under-counting here + // by only accounting for the key and value bytes. + if p.err = p.memMonitor.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) + return false + } p.err = p.intents.Set(p.curRawKey, p.curValue, nil) if p.err != nil { return false @@ -496,7 +519,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // Note that we read at the intent timestamp, not at our read timestamp // as the intent timestamp may have been pushed forward by another // transaction. Txn's always need to read their own writes. - return p.seekVersion(metaTS, false) + return p.seekVersion(ctx, metaTS, false) } // 12. We're reading our own txn's intent at a lower sequence than is @@ -518,14 +541,14 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // addAndAdvance to take an MVCCKey explicitly. p.curUnsafeKey.Timestamp = metaTS p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], p.curUnsafeKey) - return p.addAndAdvance(p.keyBuf, value) + return p.addAndAdvance(ctx, p.keyBuf, value) } // 13. If no value in the intent history has a sequence number equal to // or less than the read, we must ignore the intents laid down by the // transaction all together. We ignore the intent by insisting that the // timestamp we're reading at is a historical timestamp < the intent // timestamp. - return p.seekVersion(prevTS, false) + return p.seekVersion(ctx, prevTS, false) } if p.txnEpoch < p.meta.Txn.Epoch { @@ -543,7 +566,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // restarted and an earlier iteration wrote the value we're now // reading. In this case, we ignore the intent and read the // previous value as if the transaction were starting fresh. - return p.seekVersion(prevTS, false) + return p.seekVersion(ctx, prevTS, false) } // nextKey advances to the next user key. @@ -669,11 +692,14 @@ func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool { // Adds the specified key and value to the result set, excluding tombstones unless // p.tombstones is true. Advances to the next key unless we've reached the max // results limit. -func (p *pebbleMVCCScanner) addAndAdvance(rawKey []byte, val []byte) bool { +func (p *pebbleMVCCScanner) addAndAdvance(ctx context.Context, rawKey []byte, val []byte) bool { // Don't include deleted versions len(val) == 0, unless we've been instructed // to include tombstones in the results. if len(val) > 0 || p.tombstones { - p.results.put(rawKey, val) + if err := p.results.put(ctx, rawKey, val, p.memMonitor); err != nil { + p.err = errors.Wrapf(err, "scan with start key %s", p.start) + return false + } if p.targetBytes > 0 && p.results.bytes >= p.targetBytes { // When the target bytes are met or exceeded, stop producing more // keys. We implement this by reducing maxKeys to the current @@ -692,7 +718,9 @@ func (p *pebbleMVCCScanner) addAndAdvance(rawKey []byte, val []byte) bool { // Seeks to the latest revision of the current key that's still less than or // equal to the specified timestamp, adds it to the result set, then moves onto // the next user key. -func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck bool) bool { +func (p *pebbleMVCCScanner) seekVersion( + ctx context.Context, seekTS hlc.Timestamp, uncertaintyCheck bool, +) bool { seekKey := MVCCKey{Key: p.curUnsafeKey.Key, Timestamp: seekTS} p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], seekKey) origKey := p.keyBuf[:len(p.curUnsafeKey.Key)] @@ -713,7 +741,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b if p.curUnsafeKey.Timestamp.LessEq(seekTS) { p.incrementItersBeforeSeek() if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curRawKey, p.curValue) } // Iterate through uncertainty interval. Though we found a value in // the interval, it may not be uncertainty. This is because seekTS @@ -738,7 +766,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b return p.advanceKeyAtNewKey(origKey) } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curRawKey, p.curValue) } // Iterate through uncertainty interval. See the comment above about why // a value in this interval is not necessarily cause for an uncertainty diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 934042d88e74..cce412008141 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -90,7 +90,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { failOnMoreRecent: false, } mvccScanner.init(nil /* txn */, hlc.Timestamp{}) - _, err = mvccScanner.scan() + _, err = mvccScanner.scan(context.Background()) require.NoError(t, err) kvData := mvccScanner.results.finish() @@ -149,7 +149,7 @@ func TestMVCCScanWithLargeKeyValue(t *testing.T) { ts: ts, } mvccScanner.init(nil /* txn */, hlc.Timestamp{}) - _, err := mvccScanner.scan() + _, err := mvccScanner.scan(context.Background()) require.NoError(t, err) kvData := mvccScanner.results.finish()