From 57ad801fb267fb7f2f594c28c6c3e958923d5ba0 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 11 Jun 2021 23:03:09 -0400 Subject: [PATCH] server,roachpb: introduce span config rpcs Part of #67679; these RPCs are exposed through the `kvtenant.Connector` interface for tenants and also sit on `pkg/server.(*Node)` for the host tenant. The basic type in these RPCs is the `SpanConfig` proto, which is the same as our existing `ZoneConfig` proto but without any inheritance business. // GetSpanConfigsRequest is used to fetch the span configurations // over the specified keyspans. message GetSpanConfigsRequest { // Spans to request the configurations for. The spans listed here // are not allowed to overlap with one another. repeated Span spans = 1 [(gogoproto.nullable) = false]; }; // GetSpanConfigsResponse lists out the span configurations that // overlap with the requested spans. message GetSpanConfigsResponse { // SpanConfigEntries capture the span configurations over the // requested spans. The results for each Span in the matching // GetSpanConfigsRequest are flattened out into a single slice. It's // possible for there to be no configurations for a given span; // there'll simply be no entries for it. repeated SpanConfigEntry span_config_entries = 1 [(gogoproto.nullable) = false]; }; // UpdateSpanConfigsRequest is used to update the span // configurations over the given spans. // // This is a "targeted" API: the spans being deleted are expected to // have been present with the same bounds (same start/end key); the // same is true for spans being updated with new configs. If spans // are being added, they're expected to not overlap with any // existing spans. When divvying up an existing span into multiple // others, callers are expected to delete the old and upsert the new // ones. This can happen as part of the same request; we delete the // spans marked for deletion before upserting whatever was // requested. // // Spans are not allowed to overlap with other spans in the same // list but can across lists. This is necessary to support the // delete+upsert semantics described above. message UpdateSpanConfigsRequest { // ToDelete lists out the spans we want to delete span configs // for. repeated Span to_delete = 1 [(gogoproto.nullable) = false]; // ToUpsert lists out the spans we want to upsert and the configs // we want to upsert with. repeated SpanConfigEntry to_upsert = 2 [(gogoproto.nullable) = false]; }; The RPCs are backed by a new host-tenant only `system.span_configurations` table. Future PRs will wire a view of this table into KV with an eye towards replacing our use of `config.SystemConfig`. CREATE TABLE system.span_configurations ( start_key BYTES NOT NULL PRIMARY KEY, end_key BYTES NOT NULL, config BYTES NOT NULL, CONSTRAINT check_bounds CHECK (start_key < end_key), FAMILY "primary" (start_key, end_key, config) ) --- While here, we also introduce a `crdb_internal.pretty_span` builtin to help with the readability of this table. In future PRs we'll make use of this built-in for datadriven tests asserting on the state of the table. Release note (sql change): We've added a `system.span_configurations` table. This will later be used to store authoritative span configs that KV has decided to apply. Release justification: non-production code changes --- BUILD.bazel | 1 + .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- docs/generated/sql/functions.md | 2 + pkg/BUILD.bazel | 1 + pkg/ccl/backupccl/system_schema.go | 3 + .../changefeedccl/helpers_tenant_shim_test.go | 1 + pkg/ccl/kvccl/kvtenantccl/BUILD.bazel | 1 + pkg/ccl/kvccl/kvtenantccl/connector.go | 72 +- pkg/ccl/kvccl/kvtenantccl/connector_test.go | 12 + pkg/cli/testdata/doctor/test_examine_cluster | 2 +- pkg/cli/testdata/zip/partial1 | 6 +- pkg/cli/testdata/zip/partial1_excluded | 6 +- pkg/cli/testdata/zip/partial2 | 6 +- pkg/cli/testdata/zip/testzip | 3 +- pkg/cli/testdata/zip/testzip_concurrent | 18 +- pkg/clusterversion/cockroach_versions.go | 9 + pkg/clusterversion/key_string.go | 5 +- pkg/config/zonepb/BUILD.bazel | 2 + pkg/config/zonepb/zone.go | 108 + pkg/config/zonepb/zone_test.go | 297 ++ pkg/keys/constants.go | 1 + pkg/kv/kvclient/kvcoord/send_test.go | 12 + pkg/kv/kvclient/kvcoord/transport_test.go | 12 + pkg/kv/kvclient/kvtenant/BUILD.bazel | 1 + pkg/kv/kvclient/kvtenant/connector.go | 26 +- pkg/migration/migrations/BUILD.bazel | 1 + pkg/migration/migrations/migrations.go | 6 + .../migrations/span_configurations.go | 32 + pkg/roachpb/BUILD.bazel | 2 + pkg/roachpb/api.pb.go | 1208 +++++---- pkg/roachpb/api.proto | 9 + pkg/roachpb/mocks_generated.go | 40 + pkg/roachpb/span_config.go | 16 + pkg/roachpb/span_config.pb.go | 2392 +++++++++++++++++ pkg/roachpb/span_config.proto | 186 ++ pkg/rpc/auth_tenant.go | 55 + pkg/rpc/auth_test.go | 96 + pkg/rpc/context.go | 14 + pkg/rpc/context_test.go | 12 + pkg/server/BUILD.bazel | 1 + pkg/server/node.go | 55 +- pkg/server/server.go | 11 +- pkg/server/server_sql.go | 4 + pkg/server/tenant.go | 1 + pkg/server/testserver.go | 5 + pkg/settings/bool.go | 3 - pkg/spanconfig/BUILD.bazel | 2 +- pkg/spanconfig/spanconfig.go | 46 +- .../spanconfigkvaccessor/BUILD.bazel | 46 + .../spanconfigkvaccessor/kvaccessor.go | 415 +++ .../spanconfigkvaccessor/kvaccessor_test.go | 183 ++ .../spanconfigkvaccessor/main_test.go | 31 + .../spanconfigkvaccessor/validation_test.go | 97 + pkg/spanconfig/spanconfigmanager/BUILD.bazel | 1 + pkg/spanconfig/spanconfigmanager/manager.go | 33 +- .../spanconfigmanager/manager_test.go | 2 + pkg/sql/catalog/bootstrap/metadata.go | 1 + pkg/sql/catalog/catconstants/constants.go | 1 + pkg/sql/catalog/catprivilege/system.go | 1 + pkg/sql/catalog/systemschema/BUILD.bazel | 1 + pkg/sql/catalog/systemschema/system.go | 44 + .../logictest/testdata/logic_test/grant_table | 35 +- .../testdata/logic_test/information_schema | 36 + .../logictest/testdata/logic_test/pg_catalog | 2 + pkg/sql/logictest/testdata/logic_test/ranges | Bin 29253 -> 29599 bytes .../logictest/testdata/logic_test/show_source | 6 +- pkg/sql/logictest/testdata/logic_test/system | 12 + .../testdata/logic_test/system_namespace | 1 + .../testdata/autocommit_nonmetamorphic | 118 +- pkg/sql/opt/exec/execbuilder/testdata/delete | 6 +- .../testdata/show_trace_nonmetamorphic | 24 +- pkg/sql/pgwire/pgwire_test.go | 2 +- pkg/sql/sem/builtins/builtins.go | 26 + pkg/sql/tests/system_table_test.go | 7 +- pkg/sql/tests/testdata/initial_keys | 7 +- pkg/testutils/serverutils/test_server_shim.go | 4 + 77 files changed, 5215 insertions(+), 735 deletions(-) create mode 100644 pkg/migration/migrations/span_configurations.go create mode 100644 pkg/roachpb/span_config.go create mode 100644 pkg/roachpb/span_config.pb.go create mode 100644 pkg/roachpb/span_config.proto create mode 100644 pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel create mode 100644 pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go create mode 100644 pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go create mode 100644 pkg/spanconfig/spanconfigkvaccessor/main_test.go create mode 100644 pkg/spanconfig/spanconfigkvaccessor/validation_test.go diff --git a/BUILD.bazel b/BUILD.bazel index e6ca552a7712..0a65ebfa2e4e 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -67,6 +67,7 @@ load("@bazel_gazelle//:def.bzl", "gazelle") # gazelle:resolve proto go roachpb/internal_raft.proto //pkg/roachpb:with-mocks # gazelle:resolve proto go roachpb/io-formats.proto //pkg/roachpb:with-mocks # gazelle:resolve proto go roachpb/metadata.proto //pkg/roachpb:with-mocks +# gazelle:resolve proto go roachpb/span_config.proto //pkg/roachpb:with-mocks # gazelle:exclude pkg/roachpb/batch_generated.go # gazelle:exclude pkg/roachpb/batch_generated-gen.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 53244c1531d4..6b2ff848473d 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -156,4 +156,4 @@ trace.datadog.project string CockroachDB the project under which traces will be trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -version version 21.1-152 set the active cluster version in the format '.' +version version 21.1-154 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index f049f81b8049..8146ab46e26a 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -160,6 +160,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -versionversion21.1-152set the active cluster version in the format '.' +versionversion21.1-154set the active cluster version in the format '.' diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 2a8892592353..8f74a49fb6c2 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2858,6 +2858,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)

crdb_internal.pretty_key(raw_key: bytes, skip_fields: int) → string

This function is used only by CockroachDB’s developers for testing purposes.

+crdb_internal.pretty_span(raw_key_start: bytes, raw_key_end: bytes, skip_fields: int) → string

This function is used only by CockroachDB’s developers for testing purposes.

+
crdb_internal.range_stats(key: bytes) → jsonb

This function is used to retrieve range statistics information as a JSON object.

crdb_internal.reset_sql_stats() → bool

This function is used to clear the collected SQL statistics.

diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 4ac713f59f76..60403a74ab5d 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -181,6 +181,7 @@ ALL_TESTS = [ "//pkg/server/tracedumper:tracedumper_test", "//pkg/server:server_test", "//pkg/settings:settings_test", + "//pkg/spanconfig/spanconfigkvaccessor:spanconfigkvaccessor_test", "//pkg/spanconfig/spanconfigmanager:spanconfigmanager_test", "//pkg/sql/catalog/catalogkeys:catalogkeys_test", "//pkg/sql/catalog/catalogkv:catalogkv_test", diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index e552ff40ef1e..dd887ac46012 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -352,6 +352,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.SQLInstancesTable.GetName(): { shouldIncludeInClusterBackup: optOutOfClusterBackup, }, + systemschema.SpanConfigurationsTable.GetName(): { + shouldIncludeInClusterBackup: optOutOfClusterBackup, + }, } // GetSystemTablesToIncludeInClusterBackup returns a set of system table names that diff --git a/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go b/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go index 7c4034dd6dfc..c493b13ec8da 100644 --- a/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go +++ b/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go @@ -69,6 +69,7 @@ func (t *testServerShim) RangeFeedFactory() interface{} { panic(unsuppor func (t *testServerShim) Clock() *hlc.Clock { panic(unsupportedShimMethod) } func (t *testServerShim) DistSenderI() interface{} { panic(unsupportedShimMethod) } func (t *testServerShim) MigrationServer() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SpanConfigAccessor() interface{} { panic(unsupportedShimMethod) } func (t *testServerShim) SQLServer() interface{} { panic(unsupportedShimMethod) } func (t *testServerShim) SQLLivenessProvider() interface{} { panic(unsupportedShimMethod) } func (t *testServerShim) StartupMigrationsManager() interface{} { panic(unsupportedShimMethod) } diff --git a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel index b6d9c6ea2bc8..f024cc9a03ad 100644 --- a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/roachpb:with-mocks", "//pkg/rpc", "//pkg/server/serverpb", + "//pkg/spanconfig", "//pkg/util/contextutil", "//pkg/util/grpcutil", "//pkg/util/log", diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index f214c0e3f6d5..ae634c985a45 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -108,6 +109,9 @@ var _ config.SystemConfigProvider = (*Connector)(nil) // multi-region primitives. var _ serverpb.RegionsServer = (*Connector)(nil) +// Connector is capable of accessing span configurations for secondary tenants. +var _ spanconfig.KVAccessor = (*Connector)(nil) + // NewConnector creates a new Connector. // NOTE: Calling Start will set cfg.RPCContext.ClusterID. func NewConnector(cfg kvtenant.ConnectorConfig, addrs []string) *Connector { @@ -367,16 +371,16 @@ func (c *Connector) RangeLookup( // Regions implements the serverpb.RegionsServer interface. func (c *Connector) Regions( ctx context.Context, req *serverpb.RegionsRequest, -) (*serverpb.RegionsResponse, error) { - ctx = c.AnnotateCtx(ctx) - for ctx.Err() == nil { - client, err := c.getClient(ctx) - if err != nil { - continue - } - return client.Regions(ctx, req) +) (resp *serverpb.RegionsResponse, _ error) { + if err := c.withClient(ctx, func(ctx context.Context, c *client) error { + var err error + resp, err = c.Regions(ctx, req) + return err + }); err != nil { + return nil, err } - return nil, ctx.Err() + + return resp, nil } // FirstRange implements the kvcoord.RangeDescriptorDB interface. @@ -415,6 +419,56 @@ func (c *Connector) TokenBucket( return nil, ctx.Err() } +// GetSpanConfigEntriesFor implements the spanconfig.KVAccessor interface. +func (c *Connector) GetSpanConfigEntriesFor( + ctx context.Context, spans []roachpb.Span, +) (entries []roachpb.SpanConfigEntry, _ error) { + if err := c.withClient(ctx, func(ctx context.Context, c *client) error { + resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{ + Spans: spans, + }) + if err != nil { + return err + } + + entries = resp.SpanConfigEntries + return nil + }); err != nil { + return nil, err + } + return entries, nil +} + +// UpdateSpanConfigEntries implements the spanconfig.KVAccessor +// interface. +func (c *Connector) UpdateSpanConfigEntries( + ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, +) error { + return c.withClient(ctx, func(ctx context.Context, c *client) error { + _, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{ + ToDelete: toDelete, + ToUpsert: toUpsert, + }) + return err + }) +} + +// withClient is a convenience wrapper that executes the given closure while +// papering over InternalClient retrieval errors. +func (c *Connector) withClient( + ctx context.Context, f func(ctx context.Context, c *client) error, +) error { + ctx = c.AnnotateCtx(ctx) + for ctx.Err() == nil { + c, err := c.getClient(ctx) + if err != nil { + continue + } + return f(ctx, c) + } + return ctx.Err() +} + // getClient returns the singleton InternalClient if one is currently active. If // not, the method attempts to dial one of the configured addresses. The method // blocks until either a connection is successfully established or the provided diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index bbbb171dae18..ec8322910a0e 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -89,6 +89,18 @@ func (*mockServer) TokenBucket( panic("unimplemented") } +func (m *mockServer) GetSpanConfigs( + context.Context, *roachpb.GetSpanConfigsRequest, +) (*roachpb.GetSpanConfigsResponse, error) { + panic("unimplemented") +} + +func (m *mockServer) UpdateSpanConfigs( + context.Context, *roachpb.UpdateSpanConfigsRequest, +) (*roachpb.UpdateSpanConfigsResponse, error) { + panic("unimplemented") +} + func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent { return &roachpb.GossipSubscriptionEvent{ Key: gossip.KeyClusterID, diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster index 24d84a13b232..1c2777f765b6 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster +++ b/pkg/cli/testdata/doctor/test_examine_cluster @@ -1,7 +1,7 @@ debug doctor examine cluster ---- debug doctor examine cluster -Examining 40 descriptors and 41 namespace entries... +Examining 41 descriptors and 42 namespace entries... ParentID 50, ParentSchemaID 29: relation "foo" (53): expected matching namespace entry, found none Examining 4 jobs... ERROR: validation failed diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 56b866b0b9be..c0571101116d 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 42 ranges found +[node 1] 43 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -114,6 +114,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done [node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done [node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done +[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done [node 2] node status... converting to JSON... writing binary output: debug/nodes/2/status.json... done [node 2] using SQL connection URL: postgresql://... [node 2] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/2/crdb_internal.feature_usage.txt... @@ -236,7 +237,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 42 ranges found +[node 3] 43 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -279,5 +280,6 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done +[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 9312defa7601..b389032406f7 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -71,7 +71,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 42 ranges found +[node 1] 43 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -114,6 +114,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done [node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done [node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done +[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done [node 2] skipping node... writing binary output: debug/nodes/2.skipped... done [node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done [node 3] using SQL connection URL: postgresql://... @@ -151,7 +152,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 42 ranges found +[node 3] 43 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -194,5 +195,6 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done +[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index 7f3cd4908eb3..ae6607c7f803 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 42 ranges found +[node 1] 43 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -114,6 +114,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done [node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done [node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done +[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done [node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done [node 3] using SQL connection URL: postgresql://... [node 3] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/3/crdb_internal.feature_usage.txt... done @@ -150,7 +151,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 42 ranges found +[node 3] 43 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -193,5 +194,6 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done +[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 195ee929d31d..57ebd79f4eab 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] requesting log file ... [node 1] 0 log file ... [node 1] requesting ranges... received response... done -[node 1] 42 ranges found +[node 1] 43 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -114,5 +114,6 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done [node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done [node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done +[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index 97118aa6fa67..c3485811410a 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -118,7 +118,7 @@ zip [cluster] using SQL address: ... [cluster] using SQL address: ... [node 1] 1 log file ... -[node 1] 42 ranges found +[node 1] 43 ranges found [node 1] [log file ... [node 1] [log file ... [node 1] [log file ... @@ -369,6 +369,10 @@ zip [node 1] writing range 42: converting to JSON... [node 1] writing range 42: done [node 1] writing range 42: writing binary output: debug/nodes/1/ranges/42.json... +[node 1] writing range 43... +[node 1] writing range 43: converting to JSON... +[node 1] writing range 43: done +[node 1] writing range 43: writing binary output: debug/nodes/1/ranges/43.json... [node 1] writing range 4: converting to JSON... [node 1] writing range 4: done [node 1] writing range 4: writing binary output: debug/nodes/1/ranges/4.json... @@ -393,7 +397,7 @@ zip [node 1] writing range 9: done [node 1] writing range 9: writing binary output: debug/nodes/1/ranges/9.json... [node 2] 1 log file ... -[node 2] 42 ranges found +[node 2] 43 ranges found [node 2] [log file ... [node 2] [log file ... [node 2] [log file ... @@ -644,6 +648,10 @@ zip [node 2] writing range 42: converting to JSON... [node 2] writing range 42: done [node 2] writing range 42: writing binary output: debug/nodes/2/ranges/42.json... +[node 2] writing range 43... +[node 2] writing range 43: converting to JSON... +[node 2] writing range 43: done +[node 2] writing range 43: writing binary output: debug/nodes/2/ranges/43.json... [node 2] writing range 4: converting to JSON... [node 2] writing range 4: done [node 2] writing range 4: writing binary output: debug/nodes/2/ranges/4.json... @@ -668,7 +676,7 @@ zip [node 2] writing range 9: done [node 2] writing range 9: writing binary output: debug/nodes/2/ranges/9.json... [node 3] 1 log file ... -[node 3] 42 ranges found +[node 3] 43 ranges found [node 3] [log file ... [node 3] [log file ... [node 3] [log file ... @@ -919,6 +927,10 @@ zip [node 3] writing range 42: converting to JSON... [node 3] writing range 42: done [node 3] writing range 42: writing binary output: debug/nodes/3/ranges/42.json... +[node 3] writing range 43... +[node 3] writing range 43: converting to JSON... +[node 3] writing range 43: done +[node 3] writing range 43: writing binary output: debug/nodes/3/ranges/43.json... [node 3] writing range 4: converting to JSON... [node 3] writing range 4: done [node 3] writing range 4: writing binary output: debug/nodes/3/ranges/4.json... diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 28d871db8332..c79b5bf6a72e 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -310,6 +310,10 @@ const ( // OnUpdateExpressions setting ON UPDATE column expressions is supported in // this version. OnUpdateExpressions + // SpanConfigurationsTable adds the span configurations system table, to + // store all KV span configs. + SpanConfigurationsTable + // Step (1): Add new versions here. ) @@ -533,6 +537,11 @@ var versionsSingleton = keyedVersions{ Key: OnUpdateExpressions, Version: roachpb.Version{Major: 21, Minor: 1, Internal: 152}, }, + { + Key: SpanConfigurationsTable, + Version: roachpb.Version{Major: 21, Minor: 1, Internal: 154}, + }, + // Step (2): Add new versions here. } diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index a517cd15d1b2..dd85d60694d1 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -55,11 +55,12 @@ func _() { _ = x[DatabasePlacementPolicy-44] _ = x[GeneratedAsIdentity-45] _ = x[OnUpdateExpressions-46] + _ = x[SpanConfigurationsTable-47] } -const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressions" +const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTable" -var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 719, 744, 773, 804, 824, 855, 882, 907, 924, 953, 986, 1009, 1028, 1047} +var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 719, 744, 773, 804, 824, 855, 882, 907, 924, 953, 986, 1009, 1028, 1047, 1070} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/config/zonepb/BUILD.bazel b/pkg/config/zonepb/BUILD.bazel index 61a38afb9c64..c3240f84a361 100644 --- a/pkg/config/zonepb/BUILD.bazel +++ b/pkg/config/zonepb/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/roachpb:with-mocks", "//pkg/sql/opt/cat", "//pkg/sql/sem/tree", + "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", "@in_gopkg_yaml_v2//:yaml_v2", @@ -30,6 +31,7 @@ go_test( embed = [":zonepb"], deps = [ "//pkg/keys", + "//pkg/roachpb:with-mocks", "//pkg/sql/sem/tree", "//pkg/testutils", "//pkg/util/leaktest", diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zonepb/zone.go index 31d6d8f7a9dc..69d876e4904d 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zonepb/zone.go @@ -12,6 +12,7 @@ package zonepb import ( "bytes" + "context" "fmt" "strings" "time" @@ -21,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" ) @@ -1152,3 +1154,109 @@ func (c *Constraint) GetValue() string { func (m *GCPolicy) TTL() time.Duration { return time.Duration(m.TTLSeconds) * time.Second } + +// EnsureFullyHydrated returns an assertion error if the zone config is not +// fully hydrated. A fully hydrated zone configuration must have all required +// fields set, which are RangeMaxBytes, RangeMinBytes, GC, and NumReplicas. +func (z *ZoneConfig) EnsureFullyHydrated() error { + var unsetFields []string + if z.RangeMaxBytes == nil { + unsetFields = append(unsetFields, "RangeMaxBytes") + } + if z.RangeMinBytes == nil { + unsetFields = append(unsetFields, "RangeMinBytes") + } + if z.GC == nil { + unsetFields = append(unsetFields, "GCPolicy") + } + if z.NumReplicas == nil { + unsetFields = append(unsetFields, "NumReplicas") + } + + if len(unsetFields) > 0 { + return errors.AssertionFailedf("expected hydrated zone config: %s unset", strings.Join(unsetFields, ", ")) + } + return nil +} + +// AsSpanConfig converts a fully hydrated zone configuration to an equivalent +// SpanConfig. It fatals if the zone config hasn't been fully hydrated (fields +// are expected to have been cascaded through parent zone configs). +func (z *ZoneConfig) AsSpanConfig() roachpb.SpanConfig { + spanConfig, err := z.toSpanConfig() + if err != nil { + log.Fatalf(context.Background(), "%v", err) + } + return spanConfig +} + +func (z *ZoneConfig) toSpanConfig() (roachpb.SpanConfig, error) { + var sc roachpb.SpanConfig + var err error + + if err = z.EnsureFullyHydrated(); err != nil { + return sc, err + } + + // Copy over the values. + sc.RangeMinBytes = *z.RangeMinBytes + sc.RangeMaxBytes = *z.RangeMaxBytes + sc.GCTTL = z.GC.TTLSeconds + + // GlobalReads is false by default. + if z.GlobalReads != nil { + sc.GlobalReads = *z.GlobalReads + } + sc.NumReplicas = *z.NumReplicas + if z.NumVoters != nil { + sc.NumVoters = *z.NumVoters + } + + toSpanConfigConstraints := func(src []Constraint) ([]roachpb.Constraint, error) { + spanConfigConstraints := make([]roachpb.Constraint, len(src)) + for i, c := range src { + switch c.Type { + case Constraint_REQUIRED: + spanConfigConstraints[i].Type = roachpb.Constraint_REQUIRED + case Constraint_PROHIBITED: + spanConfigConstraints[i].Type = roachpb.Constraint_PROHIBITED + default: + return nil, errors.AssertionFailedf("unknown constraint type: %v", c.Type) + } + spanConfigConstraints[i].Key = c.Key + spanConfigConstraints[i].Value = c.Value + } + return spanConfigConstraints, nil + } + + toSpanConfigConstraintsConjunction := func(src []ConstraintsConjunction) ([]roachpb.ConstraintsConjunction, error) { + constraintsConjunction := make([]roachpb.ConstraintsConjunction, len(src)) + for i, constraint := range src { + constraintsConjunction[i].NumReplicas = constraint.NumReplicas + constraintsConjunction[i].Constraints, err = toSpanConfigConstraints(constraint.Constraints) + if err != nil { + return nil, err + } + } + return constraintsConjunction, nil + } + + sc.Constraints = make([]roachpb.ConstraintsConjunction, len(z.Constraints)) + sc.Constraints, err = toSpanConfigConstraintsConjunction(z.Constraints) + if err != nil { + return roachpb.SpanConfig{}, err + } + sc.VoterConstraints, err = toSpanConfigConstraintsConjunction(z.VoterConstraints) + if err != nil { + return roachpb.SpanConfig{}, err + } + + sc.LeasePreferences = make([]roachpb.LeasePreference, len(z.LeasePreferences)) + for i, leasePreference := range z.LeasePreferences { + sc.LeasePreferences[i].Constraints, err = toSpanConfigConstraints(leasePreference.Constraints) + if err != nil { + return roachpb.SpanConfig{}, err + } + } + return sc, nil +} diff --git a/pkg/config/zonepb/zone_test.go b/pkg/config/zonepb/zone_test.go index b5706d4a420a..3359b9470b6c 100644 --- a/pkg/config/zonepb/zone_test.go +++ b/pkg/config/zonepb/zone_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -1120,3 +1121,299 @@ func TestDefaultRangeSizesAreSane(t *testing.T) { require.Regexp(t, "range_min_bytes:134217728 range_max_bytes:536870912", DefaultZoneConfigRef().String()) } + +// TestZoneConfigToSpanConfigConversion tests zone configurations are correctly +// translated to span configurations using `toSpanConfig`. +func TestZoneConfigToSpanConfigConversion(t *testing.T) { + testCases := []struct { + zoneConfig ZoneConfig + errorStr string + expectSpanConfig roachpb.SpanConfig + }{ + { + zoneConfig: ZoneConfig{}, + errorStr: "expected hydrated zone config: RangeMaxBytes, RangeMinBytes, GCPolicy, NumReplicas unset", + }, + { + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + NumReplicas: proto.Int32(3), + }, + errorStr: "expected hydrated zone config: RangeMaxBytes unset", + }, + { + zoneConfig: ZoneConfig{ + RangeMaxBytes: proto.Int64(200000), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + NumReplicas: proto.Int32(3), + }, + errorStr: "expected hydrated zone config: RangeMinBytes unset", + }, + { + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + }, + errorStr: "expected hydrated zone config: GCPolicy unset", + }, + { + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + }, + errorStr: "expected hydrated zone config: NumReplicas unset", + }, + { + // Basic sanity check test. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, + VoterConstraints: []roachpb.ConstraintsConjunction{}, + LeasePreferences: []roachpb.LeasePreference{}, + }, + }, + { + // Test GlobalReads set to true. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GlobalReads: proto.Bool(true), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: true, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, + VoterConstraints: []roachpb.ConstraintsConjunction{}, + LeasePreferences: []roachpb.LeasePreference{}, + }, + }, + { + // Test GlobalReads set to false (explicitly). + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GlobalReads: proto.Bool(false), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, + VoterConstraints: []roachpb.ConstraintsConjunction{}, + LeasePreferences: []roachpb.LeasePreference{}, + }, + }, + { + // Test `DEPRECATED_POSITIVE` constraints throw an error. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GlobalReads: proto.Bool(false), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + Constraints: []ConstraintsConjunction{ + { + NumReplicas: 1, + Constraints: []Constraint{ + {Type: Constraint_DEPRECATED_POSITIVE, Key: "region", Value: "region_a"}, + }, + }, + }, + }, + errorStr: "unknown constraint type", + }, + { + // Test Constraints are translated correctly, both positive and negative. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + Constraints: []ConstraintsConjunction{ + { + NumReplicas: 1, + Constraints: []Constraint{ + {Type: Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []Constraint{ + {Type: Constraint_PROHIBITED, Key: "region", Value: "region_b"}, + }, + }, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{ + { + NumReplicas: 1, + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_PROHIBITED, Key: "region", Value: "region_b"}, + }, + }, + }, + VoterConstraints: []roachpb.ConstraintsConjunction{}, + LeasePreferences: []roachpb.LeasePreference{}, + }, + }, + { + // Test VoterConstraints are translated properly. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + VoterConstraints: []ConstraintsConjunction{ + { + Constraints: []Constraint{ + {Type: Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []Constraint{ + {Type: Constraint_PROHIBITED, Key: "region", Value: "region_b"}, + }, + }, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, + VoterConstraints: []roachpb.ConstraintsConjunction{ + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_PROHIBITED, Key: "region", Value: "region_b"}, + }, + }, + }, + LeasePreferences: []roachpb.LeasePreference{}, + }, + }, + { + // Test LeasePreferences are translated properly. + zoneConfig: ZoneConfig{ + RangeMinBytes: proto.Int64(100000), + RangeMaxBytes: proto.Int64(200000), + NumReplicas: proto.Int32(3), + GC: &GCPolicy{ + TTLSeconds: 2400, + }, + LeasePreferences: []LeasePreference{ + { + Constraints: []Constraint{ + {Type: Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []Constraint{ + {Type: Constraint_REQUIRED, Key: "region", Value: "region_b"}, + }, + }, + { + Constraints: []Constraint{ + {Type: Constraint_PROHIBITED, Key: "region", Value: "region_c"}, + }, + }, + }, + }, + expectSpanConfig: roachpb.SpanConfig{ + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCTTL: 2400, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, + VoterConstraints: []roachpb.ConstraintsConjunction{}, + LeasePreferences: []roachpb.LeasePreference{ + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "region", Value: "region_a"}, + }, + }, + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "region", Value: "region_b"}, + }, + }, + { + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_PROHIBITED, Key: "region", Value: "region_c"}, + }, + }, + }, + }, + }, + } + for _, tc := range testCases { + spanConfig, err := tc.zoneConfig.toSpanConfig() + if tc.errorStr != "" { + require.Truef(t, testutils.IsError(err, tc.errorStr), "mismatched errors: expected %q got %q", tc.errorStr, err.Error()) + } + require.Equal(t, tc.expectSpanConfig, spanConfig) + } +} diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 401457ad1692..e65d1be779b8 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -412,6 +412,7 @@ const ( DatabaseRoleSettingsTableID = 44 TenantUsageTableID = 45 SQLInstancesTableID = 46 + SpanConfigurationsTableID = 47 // CommentType is type for system.comments DatabaseCommentType = 0 diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index 5fec0a764389..5e1e85bb771a 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -80,6 +80,18 @@ func (n Node) TokenBucket( panic("unimplemented") } +func (n Node) GetSpanConfigs( + _ context.Context, _ *roachpb.GetSpanConfigsRequest, +) (*roachpb.GetSpanConfigsResponse, error) { + panic("unimplemented") +} + +func (n Node) UpdateSpanConfigs( + _ context.Context, _ *roachpb.UpdateSpanConfigsRequest, +) (*roachpb.UpdateSpanConfigsResponse, error) { + panic("unimplemented") +} + // TestSendToOneClient verifies that Send correctly sends a request // to one server using the heartbeat RPC. func TestSendToOneClient(t *testing.T) { diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index 3ef145a9f41b..7e7ad5e2a64f 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -197,3 +197,15 @@ func (m *mockInternalClient) TokenBucket( ) (*roachpb.TokenBucketResponse, error) { return nil, fmt.Errorf("unsupported TokenBucket call") } + +func (m *mockInternalClient) GetSpanConfigs( + _ context.Context, _ *roachpb.GetSpanConfigsRequest, _ ...grpc.CallOption, +) (*roachpb.GetSpanConfigsResponse, error) { + return nil, fmt.Errorf("unsupported GetSpanConfigs call") +} + +func (m *mockInternalClient) UpdateSpanConfigs( + _ context.Context, _ *roachpb.UpdateSpanConfigsRequest, _ ...grpc.CallOption, +) (*roachpb.UpdateSpanConfigsResponse, error) { + return nil, fmt.Errorf("unsupported UpdateSpanConfigs call") +} diff --git a/pkg/kv/kvclient/kvtenant/BUILD.bazel b/pkg/kv/kvclient/kvtenant/BUILD.bazel index eb6020b68f5d..51a07e7c3020 100644 --- a/pkg/kv/kvclient/kvtenant/BUILD.bazel +++ b/pkg/kv/kvclient/kvtenant/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/rpc", "//pkg/rpc/nodedialer", "//pkg/server/serverpb", + "//pkg/spanconfig", "//pkg/util/log", "//pkg/util/retry", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index 54a9711b5df5..a3b0b41b8aac 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" @@ -41,32 +42,37 @@ type Connector interface { // Start starts the connector. Start(context.Context) error - // Connector is capable of providing information on each of the KV nodes in - // the cluster in the form of NodeDescriptors. This obviates the need for - // SQL-only tenant processes to join the cluster-wide gossip network. + // NodeDescStore provides information on each of the KV nodes in the cluster + // in the form of NodeDescriptors. This obviates the need for SQL-only + // tenant processes to join the cluster-wide gossip network. kvcoord.NodeDescStore - // Connector is capable of providing Range addressing information in the - // form of RangeDescriptors through delegated RangeLookup requests. This is + // RangeDescriptorDB provides range addressing information in the form of + // RangeDescriptors through delegated RangeLookup requests. This is // necessary because SQL-only tenants are restricted from reading Range // Metadata keys directly. Instead, the RangeLookup requests are proxied // through existing KV nodes while being subject to additional validation // (e.g. is the Range being requested owned by the requesting tenant?). rangecache.RangeDescriptorDB - // Connector is capable of providing a filtered view of the SystemConfig + // SystemConfigProvider provides a filtered view of the SystemConfig // containing only information applicable to secondary tenants. This // obviates the need for SQL-only tenant processes to join the cluster-wide // gossip network. config.SystemConfigProvider - // Connector is capable of knowing every region in the cluster. - // This is necessary for region validation for zone configurations and - // multi-region primitives. + // RegionsServer provides access to a tenant's available regions. This is + // necessary for region validation for zone configurations and multi-region + // primitives. serverpb.RegionsServer - // Connector is capable of providing an endpoint for the TokenBucket API. + // TokenBucketProvider provides access to the tenant cost control token + // bucket. TokenBucketProvider + + // KVAccessor provides access to the subset of the cluster's span configs + // applicable to secondary tenants. + spanconfig.KVAccessor } // TokenBucketProvider supplies an endpoint (to tenants) for the TokenBucket API diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 1888f2d6a813..18d7ee1abf43 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "records_based_registry.go", "retry_jobs_with_exponential_backoff.go", "separated_intents.go", + "span_configurations.go", "sql_instances.go", "sql_stats.go", "tenant_usage.go", diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index a4b34ff8cf07..c46f6af34ce2 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -127,6 +127,12 @@ var migrations = []migration.Migration{ NoPrecondition, zonesTableForSecondaryTenants, ), + migration.NewTenantMigration( + "add the system.span_configurations table to system tenant", + toCV(clusterversion.SpanConfigurationsTable), + NoPrecondition, + spanConfigurationsTableMigration, + ), } func init() { diff --git a/pkg/migration/migrations/span_configurations.go b/pkg/migration/migrations/span_configurations.go new file mode 100644 index 000000000000..5b4c3a3c075f --- /dev/null +++ b/pkg/migration/migrations/span_configurations.go @@ -0,0 +1,32 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migrations + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/startupmigrations" +) + +func spanConfigurationsTableMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, +) error { + if !d.Codec.ForSystemTenant() { + return nil + } + + return startupmigrations.CreateSystemTable( + ctx, d.DB, d.Codec, d.Settings, systemschema.SpanConfigurationsTable, + ) +} diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index d034ef7d8c41..74c6227d8a11 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "metadata.go", "metadata_replicas.go", "method.go", + "span_config.go", "span_group.go", "tenant.go", "version.go", @@ -212,6 +213,7 @@ proto_library( "internal_raft.proto", "io-formats.proto", "metadata.proto", + "span_config.proto", ], strip_import_prefix = "/pkg", visibility = ["//visibility:public"], diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8aaa89efc58a..026c0f3bf3dc 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -7813,571 +7813,575 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 9018 bytes of a gzipped FileDescriptorProto + // 9074 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x57, 0x96, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x24, 0x4b, 0x57, 0xfd, 0xa3, 0x56, 0xdb, 0x52, 0x77, - 0xb5, 0xfb, 0x77, 0xc7, 0x94, 0xbb, 0xdb, 0x8e, 0x3d, 0xb6, 0xc7, 0xb3, 0x22, 0xc5, 0x6e, 0x52, - 0x7f, 0xad, 0x2e, 0x52, 0xdd, 0xb0, 0x77, 0x36, 0xb5, 0xa5, 0xaa, 0x2b, 0xa9, 0x46, 0x64, 0x15, - 0xbb, 0xaa, 0xa8, 0x16, 0x0d, 0xe4, 0x21, 0xc9, 0x62, 0xe2, 0xa7, 0x60, 0x82, 0x04, 0x58, 0x0f, + 0xb5, 0xfb, 0x77, 0x6d, 0xca, 0xdd, 0x6d, 0xc7, 0x1e, 0xdb, 0xe3, 0x59, 0x91, 0x62, 0x37, 0xa9, + 0xbf, 0x56, 0x17, 0xa9, 0x6e, 0xd8, 0x3b, 0x9b, 0xda, 0x52, 0xd5, 0x95, 0x54, 0x23, 0xb2, 0x8a, + 0x5d, 0x55, 0x54, 0x8b, 0x06, 0xf2, 0x90, 0x64, 0x31, 0xf1, 0x53, 0x30, 0x41, 0x02, 0xec, 0x0c, 0x36, 0x58, 0x78, 0x33, 0x8b, 0xe4, 0x21, 0x0f, 0x09, 0x92, 0x20, 0xc1, 0x06, 0xc9, 0xee, 0x63, - 0x8c, 0x60, 0x91, 0x9d, 0x79, 0x5b, 0x04, 0x88, 0xb2, 0xd1, 0xe4, 0x21, 0x8b, 0x20, 0x48, 0x10, - 0x04, 0xd8, 0xc0, 0x40, 0x7e, 0x70, 0x7f, 0xea, 0x87, 0x64, 0x91, 0xa2, 0xda, 0xe5, 0xc4, 0xc0, - 0xbc, 0x48, 0xac, 0x73, 0xef, 0x39, 0x75, 0xef, 0xb9, 0x7f, 0xe7, 0xbb, 0xf7, 0x9c, 0x5b, 0x30, - 0x6b, 0x5b, 0xaa, 0x76, 0xd0, 0xde, 0x5d, 0x56, 0xdb, 0x46, 0xb1, 0x6d, 0x5b, 0xae, 0x85, 0x66, - 0x35, 0x4b, 0x3b, 0xa4, 0xe4, 0x22, 0x4f, 0x5c, 0xb8, 0x88, 0x6d, 0xdb, 0xb2, 0x9d, 0xf6, 0xee, - 0x32, 0xfb, 0xc1, 0x72, 0x2e, 0xdc, 0x3b, 0x3c, 0x5a, 0x3e, 0x3c, 0x72, 0xb0, 0x7d, 0x84, 0xed, - 0x65, 0xcd, 0x32, 0xb5, 0x8e, 0x6d, 0x63, 0x53, 0xeb, 0x2e, 0x37, 0x2d, 0xed, 0x90, 0xfe, 0x31, - 0xcc, 0xfd, 0xa8, 0xbc, 0x36, 0x56, 0x75, 0xa7, 0xd3, 0x6a, 0xa9, 0x76, 0x77, 0x99, 0x8a, 0xe5, - 0x0f, 0x3c, 0x2f, 0xf2, 0x0a, 0xa5, 0xab, 0xae, 0xca, 0x69, 0x17, 0x3c, 0x5a, 0x4f, 0x09, 0x2e, - 0x79, 0xd4, 0x16, 0x76, 0xd5, 0x50, 0xee, 0xab, 0x8e, 0x6b, 0xd9, 0xea, 0x3e, 0x5e, 0xc6, 0xe6, - 0xbe, 0x61, 0x62, 0x92, 0xe1, 0x48, 0xd3, 0x78, 0xe2, 0x6b, 0x91, 0x89, 0x0f, 0x79, 0xea, 0x7c, - 0xc7, 0x35, 0x9a, 0xcb, 0x07, 0x4d, 0x6d, 0xd9, 0x35, 0x5a, 0xd8, 0x71, 0xd5, 0x56, 0xdb, 0xab, - 0x02, 0x4d, 0x71, 0x6d, 0x55, 0x33, 0xcc, 0x7d, 0xef, 0x7f, 0x7b, 0x77, 0xd9, 0xc6, 0x9a, 0x65, - 0xeb, 0x58, 0x57, 0x9c, 0xb6, 0x6a, 0x7a, 0xc5, 0xdd, 0xb7, 0xf6, 0x2d, 0xfa, 0x73, 0x99, 0xfc, - 0xe2, 0xd4, 0xc5, 0x7d, 0xcb, 0xda, 0x6f, 0xe2, 0x65, 0xfa, 0xb4, 0xdb, 0xd9, 0x5b, 0xd6, 0x3b, - 0xb6, 0xea, 0x1a, 0x16, 0xe7, 0x92, 0xfe, 0xa9, 0x00, 0x39, 0x19, 0xbf, 0xe8, 0x60, 0xc7, 0xad, - 0x62, 0x55, 0xc7, 0x36, 0xba, 0x02, 0xc9, 0x43, 0xdc, 0x9d, 0x4f, 0x5e, 0x13, 0xee, 0xcc, 0x94, - 0xa6, 0xbf, 0x3a, 0x59, 0x4a, 0xae, 0xe3, 0xae, 0x4c, 0x68, 0xe8, 0x1a, 0x4c, 0x63, 0x53, 0x57, - 0x48, 0x72, 0xaa, 0x37, 0x79, 0x0a, 0x9b, 0xfa, 0x3a, 0xee, 0xa2, 0x1f, 0x40, 0xda, 0x21, 0xd2, - 0x4c, 0x0d, 0xcf, 0x4f, 0x5e, 0x13, 0xee, 0x4c, 0x96, 0x7e, 0xf5, 0xab, 0x93, 0xa5, 0x0f, 0xf7, - 0x0d, 0xf7, 0xa0, 0xb3, 0x5b, 0xd4, 0xac, 0xd6, 0xb2, 0xdf, 0xd4, 0xfa, 0x6e, 0xf0, 0x7b, 0xb9, - 0x7d, 0xb8, 0xbf, 0xdc, 0xaf, 0xa3, 0x62, 0xe3, 0xd8, 0xac, 0xe3, 0x17, 0xb2, 0x2f, 0x71, 0x2d, - 0x95, 0x16, 0xc4, 0xc4, 0x5a, 0x2a, 0x9d, 0x10, 0x93, 0xd2, 0x1f, 0x27, 0x20, 0x2f, 0x63, 0xa7, - 0x6d, 0x99, 0x0e, 0xe6, 0x25, 0x7f, 0x0b, 0x92, 0xee, 0xb1, 0x49, 0x4b, 0x9e, 0x7d, 0xb0, 0x58, - 0x1c, 0xe8, 0x54, 0xc5, 0x86, 0xad, 0x9a, 0x8e, 0xaa, 0x91, 0xea, 0xcb, 0x24, 0x2b, 0x7a, 0x0f, - 0xb2, 0x36, 0x76, 0x3a, 0x2d, 0x4c, 0x15, 0x49, 0x2b, 0x95, 0x7d, 0x70, 0x39, 0x82, 0xb3, 0xde, - 0x56, 0x4d, 0x19, 0x58, 0x5e, 0xf2, 0x1b, 0xd5, 0x21, 0xc7, 0x39, 0x6d, 0xac, 0x3a, 0x96, 0x39, - 0x3f, 0x7d, 0x4d, 0xb8, 0x93, 0x7f, 0x50, 0x8c, 0xe0, 0xed, 0x2d, 0x25, 0x79, 0xec, 0xb4, 0xb0, - 0x4c, 0xb9, 0xe4, 0x19, 0x3b, 0xf4, 0x84, 0xae, 0x40, 0xda, 0xec, 0xb4, 0x88, 0x7e, 0x1d, 0xaa, - 0xbd, 0xa4, 0x3c, 0x6d, 0x76, 0x5a, 0xeb, 0xb8, 0xeb, 0xa0, 0xab, 0x90, 0x21, 0x49, 0xbb, 0x5d, - 0x17, 0x3b, 0xf3, 0x69, 0x9a, 0x46, 0xf2, 0x96, 0xc8, 0xb3, 0xf4, 0x11, 0xcc, 0x84, 0xa5, 0x22, - 0x04, 0x79, 0xb9, 0x52, 0xdf, 0xd9, 0xac, 0x28, 0x3b, 0x5b, 0xeb, 0x5b, 0x4f, 0x9e, 0x6f, 0x89, - 0x13, 0xe8, 0x02, 0x88, 0x9c, 0xb6, 0x5e, 0xf9, 0x58, 0xd9, 0xa8, 0x6d, 0xd6, 0x1a, 0xa2, 0xb0, - 0x90, 0xfa, 0xec, 0xa7, 0x8b, 0x13, 0x6b, 0xa9, 0xf4, 0x94, 0x38, 0x2d, 0xfd, 0x54, 0x00, 0x78, - 0x8c, 0x5d, 0xde, 0x1b, 0x50, 0x09, 0xa6, 0x0e, 0x68, 0x89, 0xe7, 0x05, 0xaa, 0x96, 0x6b, 0x91, - 0x55, 0x0b, 0xf5, 0x9c, 0x52, 0xfa, 0xcb, 0x93, 0xa5, 0x89, 0x9f, 0x9d, 0x2c, 0x09, 0x32, 0xe7, - 0x44, 0x4f, 0x21, 0x7b, 0x88, 0xbb, 0x0a, 0x1f, 0x97, 0xf3, 0x09, 0xaa, 0xa3, 0xb7, 0x42, 0x82, - 0x0e, 0x8f, 0x8a, 0xde, 0x10, 0x2d, 0x86, 0x86, 0x73, 0x91, 0x70, 0x14, 0xeb, 0xae, 0x8d, 0xcd, - 0x7d, 0xf7, 0x40, 0x86, 0x43, 0xdc, 0xdd, 0x60, 0x32, 0xa4, 0x3f, 0x14, 0x20, 0x4b, 0x4b, 0xc9, - 0x94, 0x8a, 0xca, 0x7d, 0xc5, 0xbc, 0x7e, 0x66, 0x0b, 0x44, 0x94, 0xb3, 0x08, 0x93, 0x47, 0x6a, - 0xb3, 0x83, 0x69, 0x09, 0xb3, 0x0f, 0xe6, 0x23, 0x64, 0x3c, 0x23, 0xe9, 0x32, 0xcb, 0x86, 0x3e, - 0x80, 0x19, 0xc3, 0x74, 0xb1, 0xe9, 0x2a, 0x8c, 0x2d, 0x79, 0x06, 0x5b, 0x96, 0xe5, 0xa6, 0x0f, - 0xd2, 0x3f, 0x11, 0x00, 0xb6, 0x3b, 0xb1, 0xea, 0xf9, 0xed, 0x31, 0xcb, 0x5f, 0x4a, 0x11, 0x56, - 0xaf, 0x16, 0x97, 0x60, 0xca, 0x30, 0x9b, 0x86, 0xc9, 0xca, 0x9f, 0x96, 0xf9, 0x13, 0xba, 0x00, - 0x93, 0xbb, 0x4d, 0xc3, 0xd4, 0xe9, 0x78, 0x48, 0xcb, 0xec, 0x41, 0x92, 0x21, 0x4b, 0x4b, 0x1d, - 0xa3, 0xde, 0xa5, 0x93, 0x04, 0x5c, 0x2c, 0x5b, 0xa6, 0x6e, 0x90, 0x21, 0xa9, 0x36, 0xbf, 0x15, - 0x5a, 0x59, 0x83, 0x0b, 0x3a, 0x6e, 0xdb, 0x58, 0x53, 0x5d, 0xac, 0x2b, 0xf8, 0xb8, 0x3d, 0x66, - 0x1b, 0xa3, 0x80, 0xab, 0x72, 0xdc, 0xa6, 0x34, 0x32, 0x6a, 0x89, 0x00, 0x36, 0x6a, 0xa7, 0xc8, - 0x94, 0x29, 0xa7, 0xf1, 0x71, 0x9b, 0x8e, 0xda, 0x68, 0x35, 0xa3, 0xb7, 0xe1, 0xb2, 0xda, 0x6c, - 0x5a, 0x2f, 0x15, 0x63, 0x4f, 0xd1, 0x2d, 0xec, 0x28, 0xa6, 0xe5, 0x2a, 0xf8, 0xd8, 0x70, 0x5c, - 0x3a, 0x25, 0xa4, 0xe5, 0x39, 0x9a, 0x5c, 0xdb, 0x5b, 0xb5, 0xb0, 0xb3, 0x65, 0xb9, 0x15, 0x92, - 0x14, 0x6a, 0xca, 0xe9, 0x70, 0x53, 0x4a, 0xbf, 0x0e, 0x97, 0xfa, 0xf5, 0x1b, 0x67, 0xfb, 0xfd, - 0x91, 0x00, 0xf9, 0x9a, 0x69, 0xb8, 0xdf, 0x8a, 0x86, 0xf3, 0xf5, 0x99, 0x0c, 0xeb, 0xf3, 0x1e, - 0x88, 0x7b, 0xaa, 0xd1, 0x7c, 0x62, 0x36, 0xac, 0xd6, 0xae, 0xe3, 0x5a, 0x26, 0x76, 0xb8, 0xc2, - 0x07, 0xe8, 0xd2, 0x33, 0x28, 0xf8, 0xb5, 0x89, 0x53, 0x4d, 0x2e, 0x88, 0x35, 0x53, 0xb3, 0x71, - 0x0b, 0x9b, 0xb1, 0xea, 0xe9, 0x35, 0xc8, 0x18, 0x9e, 0x5c, 0xaa, 0xab, 0xa4, 0x1c, 0x10, 0xa4, - 0x0e, 0xcc, 0x86, 0xde, 0x1a, 0xe7, 0x74, 0x49, 0x16, 0x23, 0xfc, 0x52, 0x09, 0xda, 0x88, 0x2c, - 0x46, 0xf8, 0x25, 0x9b, 0xde, 0xea, 0x90, 0x5b, 0xc5, 0x4d, 0xec, 0xe2, 0x18, 0x6b, 0x2a, 0xed, - 0x40, 0xde, 0x13, 0x1a, 0x67, 0xc3, 0xfc, 0x96, 0x00, 0x88, 0xcb, 0x55, 0xcd, 0xfd, 0x38, 0x4b, - 0x8c, 0x96, 0x88, 0x69, 0xe1, 0x76, 0x6c, 0x93, 0x2d, 0xe7, 0xac, 0x4f, 0x02, 0x23, 0xd1, 0x15, - 0x3d, 0x18, 0xb2, 0xa9, 0xf0, 0x90, 0xe5, 0xe6, 0xcd, 0x4b, 0x98, 0xeb, 0x29, 0x58, 0xbc, 0xcd, - 0x97, 0xa2, 0x65, 0x4a, 0x5c, 0x4b, 0x86, 0x6d, 0x38, 0x4a, 0x94, 0x3e, 0x17, 0x60, 0xb6, 0xdc, - 0xc4, 0xaa, 0x1d, 0xbb, 0x46, 0xbe, 0x0f, 0x69, 0x1d, 0xab, 0x3a, 0xad, 0x32, 0x1b, 0xd8, 0xaf, - 0x87, 0xa4, 0x10, 0x4b, 0xb7, 0x78, 0xd0, 0xd4, 0x8a, 0x0d, 0xcf, 0x06, 0xe6, 0xa3, 0xdb, 0x67, - 0x92, 0x3e, 0x06, 0x14, 0x2e, 0x59, 0x9c, 0x1d, 0xe1, 0xf7, 0x12, 0x80, 0x64, 0x7c, 0x84, 0x6d, - 0x37, 0xf6, 0x6a, 0xaf, 0x42, 0xd6, 0x55, 0xed, 0x7d, 0xec, 0x2a, 0xc4, 0xba, 0x3f, 0x4f, 0xcd, - 0x81, 0xf1, 0x11, 0x32, 0x6a, 0xc0, 0x6d, 0x6c, 0xaa, 0xbb, 0x4d, 0x4c, 0xa5, 0x28, 0xbb, 0x56, - 0xc7, 0xd4, 0x15, 0xc3, 0xc5, 0xb6, 0xea, 0x5a, 0xb6, 0x62, 0xb5, 0x5d, 0xa3, 0x65, 0x7c, 0x4a, - 0x0d, 0x7b, 0xde, 0xd5, 0x6e, 0xb0, 0xec, 0x84, 0xb9, 0x44, 0x32, 0xd7, 0x78, 0xde, 0x27, 0xa1, - 0xac, 0xa8, 0x08, 0x73, 0xc6, 0xbe, 0x69, 0xd9, 0x58, 0xd9, 0xd7, 0x14, 0xf7, 0xc0, 0xc6, 0xce, - 0x81, 0xd5, 0xf4, 0x16, 0xa4, 0x59, 0x96, 0xf4, 0x58, 0x6b, 0x78, 0x09, 0xd2, 0x27, 0x30, 0xd7, - 0xa3, 0xa5, 0x38, 0x9b, 0xe0, 0xbf, 0x09, 0x90, 0xad, 0x6b, 0xaa, 0x19, 0xa7, 0xee, 0x3f, 0x82, - 0xac, 0xa3, 0xa9, 0xa6, 0xb2, 0x67, 0xd9, 0x2d, 0xd5, 0xa5, 0xf5, 0xca, 0xf7, 0xe8, 0xde, 0xb7, - 0xef, 0x35, 0xd5, 0x7c, 0x44, 0x33, 0xc9, 0xe0, 0xf8, 0xbf, 0xfb, 0xed, 0xd7, 0xc9, 0xaf, 0x6f, - 0xbf, 0xb2, 0xe1, 0xbd, 0x96, 0x4a, 0x27, 0xc5, 0x94, 0xf4, 0xe7, 0x02, 0xcc, 0xb0, 0x2a, 0xc7, - 0x39, 0xbc, 0xdf, 0x81, 0x94, 0x6d, 0xbd, 0x64, 0xc3, 0x3b, 0xfb, 0xe0, 0x6a, 0x84, 0x88, 0x75, - 0xdc, 0x0d, 0xaf, 0x9f, 0x34, 0x3b, 0x2a, 0x01, 0xb7, 0x52, 0x15, 0xca, 0x9d, 0x1c, 0x97, 0x1b, - 0x18, 0x97, 0x4c, 0x64, 0xdc, 0x86, 0xc2, 0xae, 0xea, 0x6a, 0x07, 0x8a, 0xcd, 0x0b, 0x49, 0xd6, - 0xda, 0xe4, 0x9d, 0x19, 0x39, 0x4f, 0xc9, 0x5e, 0xd1, 0x1d, 0x52, 0x73, 0x36, 0xde, 0x1c, 0xfc, - 0x4b, 0xd6, 0xe6, 0xff, 0x5b, 0xe0, 0x63, 0xc8, 0xab, 0xf9, 0x2f, 0x5b, 0xd3, 0xff, 0x24, 0x01, - 0x97, 0xcb, 0x07, 0x58, 0x3b, 0x2c, 0x5b, 0xa6, 0x63, 0x38, 0x2e, 0xd1, 0x5d, 0x9c, 0xed, 0x7f, - 0x15, 0x32, 0x2f, 0x0d, 0xf7, 0x40, 0xd1, 0x8d, 0xbd, 0x3d, 0x3a, 0xdb, 0xa6, 0xe5, 0x34, 0x21, - 0xac, 0x1a, 0x7b, 0x7b, 0xe8, 0x21, 0xa4, 0x5a, 0x96, 0xce, 0x8c, 0xf9, 0xfc, 0x83, 0xa5, 0x08, - 0xf1, 0xb4, 0x68, 0x4e, 0xa7, 0xb5, 0x69, 0xe9, 0x58, 0xa6, 0x99, 0xd1, 0x22, 0x80, 0x46, 0xa8, - 0x6d, 0xcb, 0x30, 0x5d, 0x3e, 0x39, 0x86, 0x28, 0xa8, 0x0a, 0x19, 0x17, 0xdb, 0x2d, 0xc3, 0x54, - 0x5d, 0x3c, 0x3f, 0x49, 0x95, 0xf7, 0x46, 0x64, 0xc1, 0xdb, 0x4d, 0x43, 0x53, 0x57, 0xb1, 0xa3, - 0xd9, 0x46, 0xdb, 0xb5, 0x6c, 0xae, 0xc5, 0x80, 0x59, 0xfa, 0xeb, 0x29, 0x98, 0x1f, 0xd4, 0x4d, - 0x9c, 0x3d, 0x64, 0x1b, 0xa6, 0x6c, 0xec, 0x74, 0x9a, 0x2e, 0xef, 0x23, 0x0f, 0x86, 0xa9, 0x20, - 0xa2, 0x04, 0x74, 0xeb, 0xa2, 0xe9, 0xf2, 0x62, 0x73, 0x39, 0x0b, 0xff, 0x42, 0x80, 0x29, 0x96, - 0x80, 0xee, 0x43, 0xda, 0x26, 0x0b, 0x83, 0x62, 0xe8, 0xb4, 0x8c, 0xc9, 0xd2, 0xa5, 0xd3, 0x93, - 0xa5, 0x69, 0xba, 0x58, 0xd4, 0x56, 0xbf, 0x0a, 0x7e, 0xca, 0xd3, 0x34, 0x5f, 0x4d, 0x27, 0xad, - 0xe5, 0xb8, 0xaa, 0xed, 0xd2, 0x4d, 0xa5, 0x04, 0x43, 0x48, 0x94, 0xb0, 0x8e, 0xbb, 0x68, 0x0d, - 0xa6, 0x1c, 0x57, 0x75, 0x3b, 0x0e, 0x6f, 0xaf, 0x73, 0x15, 0xb6, 0x4e, 0x39, 0x65, 0x2e, 0x81, - 0x98, 0x5b, 0x3a, 0x76, 0x55, 0xa3, 0x49, 0x1b, 0x30, 0x23, 0xf3, 0x27, 0xe9, 0xb7, 0x05, 0x98, - 0x62, 0x59, 0xd1, 0x65, 0x98, 0x93, 0x57, 0xb6, 0x1e, 0x57, 0x94, 0xda, 0xd6, 0x6a, 0xa5, 0x51, - 0x91, 0x37, 0x6b, 0x5b, 0x2b, 0x8d, 0x8a, 0x38, 0x81, 0x2e, 0x01, 0xf2, 0x12, 0xca, 0x4f, 0xb6, - 0xea, 0xb5, 0x7a, 0xa3, 0xb2, 0xd5, 0x10, 0x05, 0xba, 0xa7, 0x42, 0xe9, 0x21, 0x6a, 0x02, 0xbd, - 0x01, 0xd7, 0xfa, 0xa9, 0x4a, 0xbd, 0xb1, 0xd2, 0xa8, 0x2b, 0x95, 0x7a, 0xa3, 0xb6, 0xb9, 0xd2, - 0xa8, 0xac, 0x8a, 0xc9, 0x11, 0xb9, 0xc8, 0x4b, 0x64, 0xb9, 0x52, 0x6e, 0x88, 0x29, 0xc9, 0x85, - 0x8b, 0x32, 0xd6, 0xac, 0x56, 0xbb, 0xe3, 0x62, 0x52, 0x4a, 0x27, 0xce, 0x91, 0x72, 0x19, 0xa6, - 0x75, 0xbb, 0xab, 0xd8, 0x1d, 0x93, 0x8f, 0x93, 0x29, 0xdd, 0xee, 0xca, 0x1d, 0x53, 0xfa, 0x87, - 0x02, 0x5c, 0xea, 0x7f, 0x6d, 0x9c, 0x9d, 0xf0, 0x29, 0x64, 0x55, 0x5d, 0xc7, 0xba, 0xa2, 0xe3, - 0xa6, 0xab, 0x72, 0x93, 0xe8, 0x5e, 0x48, 0x12, 0xdf, 0x0a, 0x2c, 0xfa, 0x5b, 0x81, 0x9b, 0xcf, - 0xca, 0x65, 0x5a, 0x90, 0x55, 0xc2, 0xe1, 0x4d, 0x3f, 0x54, 0x08, 0xa5, 0x48, 0x3f, 0x49, 0x41, - 0xae, 0x62, 0xea, 0x8d, 0xe3, 0x58, 0xd7, 0x92, 0x4b, 0x30, 0xa5, 0x59, 0xad, 0x96, 0xe1, 0x7a, - 0x0a, 0x62, 0x4f, 0xe8, 0xbb, 0x21, 0x53, 0x36, 0x39, 0x86, 0x41, 0x17, 0x18, 0xb1, 0xe8, 0x37, - 0xe0, 0x32, 0x99, 0x35, 0x6d, 0x53, 0x6d, 0x2a, 0x4c, 0x9a, 0xe2, 0xda, 0xc6, 0xfe, 0x3e, 0xb6, - 0xf9, 0xf6, 0xe3, 0x9d, 0x88, 0x72, 0xd6, 0x38, 0x47, 0x99, 0x32, 0x34, 0x58, 0x7e, 0xf9, 0xa2, - 0x11, 0x45, 0x46, 0x1f, 0x02, 0x90, 0xa5, 0x88, 0x6e, 0x69, 0x3a, 0x7c, 0x3e, 0x1a, 0xb6, 0xa7, - 0xe9, 0x4d, 0x41, 0x84, 0x81, 0x3c, 0x3b, 0xe8, 0x29, 0x88, 0x86, 0xa9, 0xec, 0x35, 0x8d, 0xfd, - 0x03, 0x57, 0x79, 0x69, 0x1b, 0x2e, 0x76, 0xe6, 0x67, 0xa9, 0x8c, 0xa8, 0xa6, 0xae, 0xf3, 0xad, - 0x59, 0xfd, 0x39, 0xc9, 0xc9, 0xa5, 0xe5, 0x0d, 0xf3, 0x11, 0xe5, 0xa7, 0x44, 0x07, 0x2d, 0x13, - 0x28, 0xf4, 0xa2, 0x63, 0xd8, 0x58, 0xb9, 0xdf, 0xd6, 0xe8, 0x3e, 0x48, 0xba, 0x94, 0x3f, 0x3d, - 0x59, 0x02, 0x99, 0x91, 0xef, 0x6f, 0x97, 0x09, 0x34, 0x62, 0xbf, 0xdb, 0x1a, 0x51, 0x7b, 0xdb, - 0x32, 0x1c, 0xcb, 0x9c, 0xcf, 0x30, 0xb5, 0xb3, 0x27, 0x74, 0x17, 0x44, 0xf7, 0xd8, 0x54, 0x0e, - 0xb0, 0x6a, 0xbb, 0xbb, 0x58, 0x75, 0xc9, 0xfa, 0x0c, 0x34, 0x47, 0xc1, 0x3d, 0x36, 0xab, 0x21, - 0xf2, 0x5a, 0x2a, 0x3d, 0x2d, 0xa6, 0xd7, 0x52, 0xe9, 0xb4, 0x98, 0x91, 0xfe, 0x9d, 0x00, 0x79, - 0xaf, 0x6f, 0xc4, 0xd9, 0x8d, 0xef, 0x80, 0x68, 0x99, 0x58, 0x69, 0x1f, 0xa8, 0x0e, 0xe6, 0x6d, - 0xc9, 0x57, 0x87, 0xbc, 0x65, 0xe2, 0x6d, 0x42, 0x66, 0x2d, 0x83, 0xb6, 0x61, 0xd6, 0x71, 0xd5, - 0x7d, 0xc3, 0xdc, 0x57, 0xfc, 0x2d, 0x7e, 0x6a, 0x59, 0x8c, 0x89, 0x04, 0x44, 0xce, 0xed, 0xd3, - 0x7b, 0x4c, 0x8a, 0x9f, 0x0b, 0x30, 0xbb, 0xa2, 0xb7, 0x0c, 0xb3, 0xde, 0x6e, 0x1a, 0xb1, 0x6e, - 0x30, 0xbc, 0x01, 0x19, 0x87, 0xc8, 0x0c, 0x66, 0xe7, 0x00, 0x2e, 0xa6, 0x69, 0x0a, 0x99, 0xa6, - 0x37, 0xa0, 0x80, 0x8f, 0xdb, 0x06, 0x3b, 0x57, 0x60, 0x28, 0x27, 0x35, 0x7e, 0xdd, 0xf2, 0x01, - 0x2f, 0x49, 0xe2, 0x75, 0xfa, 0x18, 0x50, 0xb8, 0x4a, 0x71, 0x02, 0x8d, 0x8f, 0x61, 0x8e, 0x8a, - 0xde, 0x31, 0x9d, 0x98, 0xf5, 0x25, 0xfd, 0x1a, 0x5c, 0xe8, 0x15, 0x1d, 0x67, 0xb9, 0x9f, 0xf3, - 0x56, 0xde, 0xc4, 0x76, 0xac, 0x08, 0xd5, 0xd7, 0x35, 0x17, 0x1c, 0x67, 0x99, 0x7f, 0x53, 0x80, - 0x2b, 0x54, 0x36, 0x3d, 0x7a, 0xd9, 0xc3, 0xf6, 0x06, 0x56, 0x9d, 0x58, 0xe1, 0xf5, 0x0d, 0x98, - 0x62, 0x30, 0x99, 0xf6, 0xcf, 0xc9, 0x52, 0x96, 0x98, 0x19, 0x75, 0xd7, 0xb2, 0x89, 0x99, 0xc1, - 0x93, 0x24, 0x15, 0x16, 0xa2, 0x4a, 0x11, 0x67, 0x4d, 0x7f, 0x47, 0x80, 0x59, 0x6e, 0xe1, 0x91, - 0xae, 0x5c, 0x3e, 0x20, 0x06, 0x0e, 0xaa, 0x40, 0x56, 0xa3, 0xbf, 0x14, 0xb7, 0xdb, 0xc6, 0x54, - 0x7e, 0x7e, 0x94, 0x71, 0xc8, 0xd8, 0x1a, 0xdd, 0x36, 0x26, 0x16, 0xa6, 0xf7, 0x9b, 0x28, 0x2a, - 0x54, 0xc9, 0x91, 0xe6, 0x25, 0x1d, 0x47, 0x34, 0xaf, 0x67, 0xa7, 0x71, 0x1d, 0xfc, 0xe3, 0x24, - 0x57, 0x02, 0x7b, 0x07, 0xcf, 0x1e, 0xab, 0x41, 0xf1, 0x09, 0x5c, 0x0a, 0x6d, 0x9d, 0x87, 0x2b, - 0x9e, 0x38, 0x47, 0xc5, 0x43, 0xdb, 0xef, 0x01, 0x15, 0x7d, 0x0c, 0xa1, 0x0d, 0x76, 0x85, 0xd5, - 0xc9, 0x83, 0x2a, 0xe7, 0x51, 0xc7, 0x6c, 0x20, 0x85, 0xd1, 0x1d, 0x54, 0x86, 0x34, 0x3e, 0x6e, - 0x2b, 0x3a, 0x76, 0x34, 0x3e, 0x71, 0x49, 0x51, 0x02, 0x49, 0x51, 0x06, 0x8c, 0xf7, 0x69, 0x7c, - 0xdc, 0x26, 0x44, 0xb4, 0x43, 0xd6, 0x4d, 0x6f, 0x5d, 0xa7, 0xc5, 0x76, 0xce, 0xc6, 0x02, 0x41, - 0x4f, 0xe1, 0xe2, 0x0a, 0xfe, 0x92, 0xce, 0x44, 0x48, 0x5f, 0x08, 0x70, 0x35, 0xb2, 0xd5, 0xe2, - 0x5c, 0xc8, 0x3e, 0x84, 0x14, 0xad, 0x7c, 0xe2, 0x9c, 0x95, 0xa7, 0x5c, 0xd2, 0x67, 0x09, 0x3e, - 0xc6, 0x65, 0xdc, 0xb4, 0x88, 0x62, 0x63, 0xdf, 0x42, 0x7b, 0x02, 0xb9, 0x23, 0xcb, 0xc5, 0xb6, - 0xdf, 0xec, 0x89, 0x73, 0x37, 0xfb, 0x0c, 0x15, 0xe0, 0xb5, 0xf8, 0x33, 0x98, 0x35, 0x2d, 0x53, - 0xe9, 0x15, 0x7a, 0xfe, 0xbe, 0x54, 0x30, 0x2d, 0xf3, 0x59, 0x48, 0xae, 0x3f, 0xcf, 0xf4, 0x69, - 0x22, 0xce, 0x79, 0xe6, 0xc7, 0x02, 0xcc, 0xf9, 0x96, 0x4e, 0xcc, 0xe6, 0xee, 0x3b, 0x90, 0x34, - 0xad, 0x97, 0xe7, 0xd9, 0xa2, 0x24, 0xf9, 0xc9, 0xaa, 0xd7, 0x5b, 0xa2, 0x38, 0xeb, 0xfb, 0x2f, - 0x13, 0x90, 0x79, 0x5c, 0x8e, 0xb3, 0x96, 0x1f, 0xf2, 0xed, 0x6f, 0xd6, 0xde, 0x51, 0xbd, 0xdd, - 0x7f, 0x5f, 0xf1, 0x71, 0x79, 0x1d, 0x77, 0xbd, 0xde, 0x4e, 0xb8, 0xd0, 0x0a, 0x64, 0x7a, 0x37, - 0x4a, 0xc7, 0xd4, 0x54, 0xc0, 0xb5, 0x80, 0x61, 0x92, 0xca, 0xf5, 0x5c, 0x2d, 0x84, 0x08, 0x57, - 0x0b, 0xf2, 0x1a, 0xdf, 0x52, 0x4c, 0x9c, 0xe7, 0x35, 0x21, 0x13, 0x71, 0x52, 0x9c, 0x92, 0x9e, - 0x02, 0x90, 0xea, 0xc4, 0xd9, 0x24, 0x3f, 0x4a, 0x42, 0x7e, 0xbb, 0xe3, 0x1c, 0xc4, 0xdc, 0xfb, - 0xca, 0x00, 0xed, 0x8e, 0x73, 0x40, 0x46, 0xe4, 0xb1, 0xc9, 0xeb, 0x7c, 0x86, 0x17, 0x87, 0x57, - 0x69, 0xc6, 0xd7, 0x38, 0x36, 0x51, 0x95, 0x0b, 0xc1, 0x4a, 0xe0, 0x0a, 0x72, 0x63, 0x14, 0xb2, - 0x6c, 0x1c, 0x9b, 0x9b, 0xd8, 0x87, 0x94, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x84, 0x69, 0xf2, 0xa0, - 0xb8, 0xd6, 0x79, 0x9a, 0x79, 0x8a, 0xf0, 0x34, 0x2c, 0xf4, 0x01, 0x64, 0x18, 0x37, 0x59, 0xfd, - 0xa6, 0xe8, 0xea, 0x17, 0x55, 0x17, 0xae, 0x46, 0xba, 0xee, 0xa5, 0x29, 0x2b, 0x59, 0xeb, 0x2e, - 0xc0, 0xe4, 0x9e, 0x65, 0x6b, 0xde, 0x61, 0x2e, 0x7b, 0x60, 0xed, 0xc9, 0x20, 0xcd, 0x5a, 0x2a, - 0x9d, 0x11, 0x41, 0xfa, 0x6d, 0x01, 0x0a, 0x7e, 0x43, 0xc4, 0xb9, 0x20, 0x94, 0x7b, 0xb4, 0x78, - 0xfe, 0xa6, 0x20, 0x0a, 0x94, 0xfe, 0x15, 0xb5, 0x88, 0x34, 0xeb, 0x88, 0xb6, 0x4c, 0x9c, 0x3d, - 0xe5, 0x03, 0xe6, 0xe8, 0x93, 0x38, 0x6f, 0xeb, 0x52, 0x9f, 0x9f, 0xfb, 0x70, 0xc1, 0x68, 0x91, - 0xf9, 0xdc, 0x70, 0x9b, 0x5d, 0x0e, 0xdb, 0x5c, 0xec, 0x9d, 0x1a, 0xcf, 0x05, 0x69, 0x65, 0x2f, - 0x49, 0xfa, 0x3d, 0xba, 0x5b, 0x1d, 0xd4, 0x24, 0x4e, 0x55, 0xd7, 0x20, 0x67, 0x33, 0xd1, 0xc4, - 0xac, 0x39, 0xa7, 0xb6, 0x67, 0x7c, 0x56, 0xa2, 0xf0, 0xdf, 0x4d, 0x40, 0xe1, 0x69, 0x07, 0xdb, - 0xdd, 0x6f, 0x93, 0xba, 0x6f, 0x41, 0xe1, 0xa5, 0x6a, 0xb8, 0xca, 0x9e, 0x65, 0x2b, 0x9d, 0xb6, - 0xae, 0xba, 0x9e, 0xb7, 0x49, 0x8e, 0x90, 0x1f, 0x59, 0xf6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xd0, - 0xb4, 0x5e, 0x9a, 0x0a, 0x21, 0x53, 0xa0, 0x7c, 0x6c, 0xf2, 0x2d, 0xe4, 0xd2, 0xbb, 0xff, 0xf6, - 0x64, 0xe9, 0xe1, 0x58, 0x3e, 0x64, 0xd4, 0x5f, 0xae, 0xd3, 0x31, 0xf4, 0xe2, 0xce, 0x4e, 0x6d, - 0x55, 0x16, 0xa9, 0xc8, 0xe7, 0x4c, 0x62, 0xe3, 0xd8, 0x74, 0xa4, 0xbf, 0x9b, 0x00, 0x31, 0xd0, - 0x51, 0x9c, 0x0d, 0x59, 0x81, 0xec, 0x8b, 0x0e, 0xb6, 0x8d, 0x57, 0x68, 0x46, 0xe0, 0x8c, 0x64, - 0xda, 0xb9, 0x07, 0xb3, 0xee, 0xb1, 0xa9, 0x30, 0x0f, 0x3f, 0xe6, 0xf8, 0xe1, 0x39, 0x2c, 0x14, - 0x5c, 0x52, 0x66, 0x42, 0xa7, 0x4e, 0x1f, 0x0e, 0xfa, 0x04, 0x66, 0x7a, 0xb4, 0x95, 0xfc, 0x7a, - 0xda, 0xca, 0xbe, 0x0c, 0x29, 0xea, 0x0f, 0x05, 0x40, 0x54, 0x51, 0x35, 0xb6, 0xc7, 0xff, 0x6d, - 0xe9, 0x4f, 0x77, 0x40, 0xa4, 0xfe, 0x98, 0x8a, 0xb1, 0xa7, 0xb4, 0x0c, 0xc7, 0x31, 0xcc, 0x7d, - 0xde, 0xa1, 0xf2, 0x94, 0x5e, 0xdb, 0xdb, 0x64, 0x54, 0xe9, 0x2f, 0xc1, 0x5c, 0x4f, 0x05, 0xe2, - 0x6c, 0xec, 0xeb, 0x30, 0xb3, 0xc7, 0x8e, 0x60, 0xa9, 0x70, 0xbe, 0x3d, 0x98, 0xa5, 0x34, 0xf6, - 0x3e, 0xe9, 0x3f, 0x27, 0xe0, 0x82, 0x8c, 0x1d, 0xab, 0x79, 0x84, 0xe3, 0x57, 0x61, 0x15, 0xf8, - 0xd9, 0x8b, 0xf2, 0x4a, 0x9a, 0xcc, 0x30, 0x66, 0xb6, 0xcc, 0xf5, 0xee, 0xb1, 0xbf, 0x31, 0xba, - 0xc7, 0x0e, 0xee, 0xaa, 0xf3, 0x9d, 0xba, 0x54, 0xcf, 0x4e, 0x9d, 0x05, 0x05, 0x76, 0x7a, 0xac, - 0x2b, 0x0e, 0x7e, 0x61, 0x76, 0x5a, 0x1e, 0x18, 0x2a, 0x8e, 0x2a, 0x64, 0x8d, 0xb1, 0xd4, 0xf1, - 0x8b, 0xad, 0x4e, 0x8b, 0xda, 0xce, 0xa5, 0x4b, 0xa4, 0xbc, 0xa7, 0x27, 0x4b, 0xf9, 0x9e, 0x34, - 0x47, 0xce, 0x1b, 0xfe, 0x33, 0x91, 0x2e, 0xfd, 0x00, 0x2e, 0xf6, 0x29, 0x3b, 0x4e, 0x8b, 0xe7, - 0x9f, 0x27, 0xe1, 0x4a, 0xaf, 0xf8, 0xb8, 0x21, 0xce, 0xb7, 0xbd, 0x41, 0xab, 0x90, 0x6b, 0x19, - 0xe6, 0xab, 0xed, 0x5e, 0xce, 0xb4, 0x0c, 0xd3, 0xa7, 0x45, 0x75, 0x8d, 0xa9, 0x6f, 0xb4, 0x6b, - 0xa8, 0xb0, 0x10, 0xd5, 0x76, 0x71, 0xf6, 0x8f, 0xcf, 0x04, 0x98, 0x89, 0x7b, 0x5b, 0xee, 0xd5, - 0xbc, 0xe0, 0xa4, 0x06, 0xe4, 0xbe, 0x81, 0x7d, 0xbc, 0xdf, 0x15, 0x00, 0x35, 0xec, 0x8e, 0x49, - 0x40, 0xed, 0x86, 0xb5, 0x1f, 0x67, 0x35, 0x2f, 0xc0, 0xa4, 0x61, 0xea, 0xf8, 0x98, 0x56, 0x33, - 0x25, 0xb3, 0x87, 0x9e, 0xa3, 0xc4, 0xe4, 0x58, 0x47, 0x89, 0xd2, 0x27, 0x30, 0xd7, 0x53, 0xc4, - 0x38, 0xeb, 0xff, 0x67, 0x09, 0x98, 0xe3, 0x15, 0x89, 0x7d, 0x07, 0xf3, 0x6d, 0x98, 0x6c, 0x12, - 0x99, 0x23, 0xda, 0x99, 0xbe, 0xd3, 0x6b, 0x67, 0x9a, 0x19, 0x7d, 0x0f, 0xa0, 0x6d, 0xe3, 0x23, - 0x85, 0xb1, 0x26, 0xc7, 0x62, 0xcd, 0x10, 0x0e, 0x4a, 0x40, 0x9f, 0x0b, 0x50, 0x20, 0x03, 0xba, - 0x6d, 0x5b, 0x6d, 0xcb, 0x21, 0x36, 0x8b, 0x33, 0x1e, 0xcc, 0x79, 0x7a, 0x7a, 0xb2, 0x94, 0xdb, - 0x34, 0xcc, 0x6d, 0xce, 0xd8, 0xa8, 0x8f, 0xed, 0xe0, 0xef, 0x85, 0x39, 0x14, 0xcb, 0x4d, 0x4b, - 0x3b, 0x0c, 0x0e, 0xc7, 0xc8, 0xcc, 0xe2, 0x8b, 0x73, 0xa4, 0x3f, 0x16, 0xe0, 0xc2, 0x37, 0xb6, - 0x5d, 0xfc, 0xff, 0x43, 0xd9, 0xd2, 0x33, 0x10, 0xe9, 0x8f, 0x9a, 0xb9, 0x67, 0xc5, 0xb9, 0x71, - 0xff, 0xbf, 0x04, 0x98, 0x0d, 0x09, 0x8e, 0xd3, 0xc0, 0x79, 0x55, 0x3d, 0xe5, 0x98, 0x3b, 0x8c, - 0x3b, 0x9e, 0xaa, 0xe4, 0x19, 0x9e, 0x9d, 0x75, 0xca, 0x22, 0xcc, 0x60, 0x32, 0x8b, 0xd1, 0x2d, - 0xde, 0x5d, 0x16, 0x64, 0xd2, 0xb7, 0xa3, 0x9f, 0xf5, 0x33, 0x94, 0xba, 0xd2, 0xaf, 0x11, 0x0b, - 0x2b, 0x3c, 0x28, 0xe3, 0x1c, 0xf2, 0xff, 0x2c, 0x01, 0x97, 0xca, 0xec, 0x08, 0xdc, 0xf3, 0x09, - 0x89, 0xb3, 0x23, 0xce, 0xc3, 0xf4, 0x11, 0xb6, 0x1d, 0xc3, 0x62, 0xab, 0x7d, 0x4e, 0xf6, 0x1e, - 0xd1, 0x02, 0xa4, 0x1d, 0x53, 0x6d, 0x3b, 0x07, 0x96, 0x77, 0x9c, 0xe8, 0x3f, 0xfb, 0xfe, 0x2b, - 0x93, 0xaf, 0xee, 0xbf, 0x32, 0x35, 0xda, 0x7f, 0x65, 0xfa, 0x6b, 0xf8, 0xaf, 0xf0, 0xb3, 0xbb, - 0x7f, 0x2d, 0xc0, 0xe5, 0x01, 0xcd, 0xc5, 0xd9, 0x39, 0x7f, 0x08, 0x59, 0x8d, 0x0b, 0x26, 0xeb, - 0x03, 0x3b, 0x98, 0xac, 0x91, 0x6c, 0xaf, 0x08, 0x7d, 0x4e, 0x4f, 0x96, 0xc0, 0x2b, 0x6a, 0x6d, - 0x95, 0x2b, 0x87, 0xfc, 0xd6, 0xa5, 0xff, 0x02, 0x50, 0xa8, 0x1c, 0xb3, 0x4d, 0xf9, 0x3a, 0xb3, - 0x4a, 0xd0, 0x23, 0x48, 0xb7, 0x6d, 0xeb, 0xc8, 0xf0, 0xaa, 0x91, 0xef, 0x71, 0x5e, 0xf0, 0xaa, - 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x64, 0x9f, 0x17, 0x35, 0x20, 0xb3, 0x61, 0x69, 0x6a, 0xf3, 0x91, - 0xd1, 0xf4, 0x06, 0xda, 0x5b, 0x67, 0x0b, 0x2a, 0xfa, 0x3c, 0xdb, 0xaa, 0x7b, 0xe0, 0x35, 0x82, - 0x4f, 0x44, 0x35, 0x48, 0x57, 0x5d, 0xb7, 0x4d, 0x12, 0xf9, 0xf8, 0xbb, 0x3d, 0x86, 0x50, 0xc2, - 0xe2, 0x79, 0xdc, 0x7a, 0xec, 0xa8, 0x01, 0xb3, 0x8f, 0x69, 0xfc, 0x58, 0xb9, 0x69, 0x75, 0xf4, - 0xb2, 0x65, 0xee, 0x19, 0xfb, 0x7c, 0x99, 0xb8, 0x35, 0x86, 0xcc, 0xc7, 0xe5, 0xba, 0x3c, 0x28, - 0x00, 0xad, 0x40, 0xba, 0xfe, 0x90, 0x0b, 0x63, 0x66, 0xe4, 0xcd, 0x31, 0x84, 0xd5, 0x1f, 0xca, - 0x3e, 0x1b, 0x5a, 0x83, 0xec, 0xca, 0xa7, 0x1d, 0x1b, 0x73, 0x29, 0x53, 0x43, 0x3d, 0x27, 0xfa, - 0xa5, 0x50, 0x2e, 0x39, 0xcc, 0x8c, 0x7e, 0x00, 0x05, 0xa2, 0xb7, 0x86, 0xba, 0xdb, 0xf4, 0xe4, - 0xa5, 0xa9, 0xbc, 0xef, 0x8c, 0x21, 0xcf, 0xe7, 0xf4, 0x8e, 0x04, 0xfa, 0x44, 0x2d, 0xc8, 0x90, - 0xeb, 0x69, 0x2f, 0x84, 0x20, 0xd5, 0x26, 0x4d, 0x23, 0x50, 0x37, 0x24, 0xfa, 0x1b, 0xbd, 0x09, - 0xd3, 0xa6, 0xa5, 0x63, 0xaf, 0x33, 0xe7, 0x4a, 0x17, 0x4e, 0x4f, 0x96, 0xa6, 0xb6, 0x2c, 0x9d, - 0xd9, 0x3a, 0xfc, 0x97, 0x3c, 0x45, 0x32, 0xd5, 0xf4, 0x85, 0x6b, 0x90, 0x22, 0x4d, 0x44, 0xe6, - 0x90, 0x5d, 0xd5, 0xc1, 0x3b, 0xb6, 0xc1, 0xa5, 0x79, 0x8f, 0x0b, 0xff, 0x20, 0x01, 0x89, 0xfa, - 0x43, 0x62, 0xcd, 0xef, 0x76, 0xb4, 0x43, 0xec, 0xf2, 0x74, 0xfe, 0x44, 0xad, 0x7c, 0x1b, 0xef, - 0x19, 0xcc, 0xe8, 0xca, 0xc8, 0xfc, 0x09, 0xbd, 0x0e, 0xa0, 0x6a, 0x1a, 0x76, 0x1c, 0xc5, 0x0b, - 0x01, 0xcc, 0xc8, 0x19, 0x46, 0x59, 0xc7, 0x5d, 0xc2, 0xe6, 0x60, 0xcd, 0xc6, 0xae, 0xe7, 0x43, - 0xc5, 0x9e, 0x08, 0x9b, 0x8b, 0x5b, 0x6d, 0xc5, 0xb5, 0x0e, 0xb1, 0x49, 0x9b, 0x34, 0x43, 0x66, - 0x85, 0x56, 0xbb, 0x41, 0x08, 0x64, 0x42, 0xc3, 0xa6, 0x1e, 0xcc, 0x3e, 0x19, 0xd9, 0x7f, 0x26, - 0x22, 0x6d, 0xbc, 0x6f, 0xf0, 0x00, 0xba, 0x8c, 0xcc, 0x9f, 0x88, 0x96, 0xd4, 0x8e, 0x7b, 0x40, - 0x5b, 0x22, 0x23, 0xd3, 0xdf, 0xe8, 0x16, 0x14, 0x98, 0xdb, 0xa5, 0x82, 0x4d, 0x4d, 0xa1, 0xf3, - 0x60, 0x86, 0x26, 0xe7, 0x18, 0xb9, 0x62, 0x6a, 0x64, 0xd6, 0x43, 0x0f, 0x81, 0x13, 0x94, 0xc3, - 0x96, 0x43, 0x74, 0x0a, 0x24, 0x57, 0xa9, 0x70, 0x7a, 0xb2, 0x94, 0xad, 0xd3, 0x84, 0xf5, 0xcd, - 0x3a, 0x59, 0x4b, 0x58, 0xae, 0xf5, 0x96, 0x53, 0xd3, 0x17, 0xfe, 0x96, 0x00, 0xc9, 0xc7, 0xe5, - 0xfa, 0xb9, 0x55, 0xe6, 0x15, 0x34, 0x19, 0x2a, 0xe8, 0x6d, 0x28, 0xec, 0x1a, 0xcd, 0xa6, 0x61, - 0xee, 0x13, 0xfb, 0xea, 0x87, 0x58, 0xf3, 0x14, 0x96, 0xe7, 0xe4, 0x6d, 0x46, 0x45, 0xd7, 0x20, - 0xab, 0xd9, 0x58, 0xc7, 0xa6, 0x6b, 0xa8, 0x4d, 0x87, 0x6b, 0x2e, 0x4c, 0x5a, 0xf8, 0xcb, 0x02, - 0x4c, 0xd2, 0xce, 0x8a, 0x5e, 0x83, 0x8c, 0x66, 0x99, 0xae, 0x6a, 0x98, 0x7c, 0xd6, 0xc9, 0xc8, - 0x01, 0x61, 0x68, 0xf1, 0xae, 0xc3, 0x8c, 0xaa, 0x69, 0x56, 0xc7, 0x74, 0x15, 0x53, 0x6d, 0x61, - 0x5e, 0xcc, 0x2c, 0xa7, 0x6d, 0xa9, 0x2d, 0x8c, 0x96, 0xc0, 0x7b, 0xf4, 0x23, 0x3b, 0x33, 0x32, - 0x70, 0xd2, 0x3a, 0xee, 0x2e, 0x60, 0xc8, 0xf8, 0xbd, 0x9a, 0xd4, 0xb7, 0xe3, 0xf8, 0x25, 0xa0, - 0xbf, 0xd1, 0x5b, 0x70, 0xe1, 0x45, 0x47, 0x6d, 0x1a, 0x7b, 0x74, 0xf3, 0x8b, 0x7a, 0xa9, 0xd3, - 0x97, 0xb1, 0xa2, 0x20, 0x3f, 0x8d, 0x4a, 0xa0, 0xef, 0xf4, 0x06, 0x41, 0x32, 0x18, 0x04, 0xcc, - 0x65, 0x47, 0xea, 0xc2, 0xac, 0x8c, 0x5d, 0xbb, 0xdb, 0x60, 0xc1, 0xae, 0x95, 0x23, 0x6c, 0xba, - 0xa4, 0xee, 0x56, 0x1b, 0x33, 0x27, 0x11, 0xaf, 0xee, 0x3e, 0x01, 0xdd, 0x84, 0xbc, 0xea, 0x92, - 0xee, 0xe6, 0x2a, 0x66, 0xa7, 0xb5, 0x8b, 0x6d, 0xe6, 0x0a, 0x20, 0xe7, 0x38, 0x75, 0x8b, 0x12, - 0x79, 0x44, 0x86, 0xdd, 0x55, 0xe8, 0x3e, 0x11, 0x7f, 0x35, 0x50, 0x52, 0x85, 0x50, 0xa4, 0xbb, - 0x70, 0x91, 0xd4, 0xb3, 0x62, 0x6a, 0x76, 0xb7, 0x4d, 0x24, 0x3f, 0xa1, 0x7f, 0x1d, 0x24, 0x86, - 0xce, 0x69, 0xe8, 0xf1, 0x8c, 0xf4, 0xe5, 0x34, 0xe4, 0x2a, 0xc7, 0x6d, 0xcb, 0x8e, 0x75, 0x57, - 0xa7, 0x04, 0xd3, 0x1c, 0xf8, 0x8e, 0x38, 0x8a, 0xed, 0x9b, 0x81, 0xbc, 0x73, 0x68, 0xce, 0x88, - 0x9e, 0xf9, 0x81, 0xa9, 0x87, 0xb8, 0x4b, 0xac, 0xfa, 0x99, 0x71, 0xac, 0xfa, 0x39, 0x8e, 0xa5, - 0xb3, 0x2c, 0x8e, 0x74, 0x1d, 0x77, 0x1b, 0x75, 0x99, 0xc7, 0xc6, 0x92, 0x07, 0x07, 0x6d, 0x41, - 0xb6, 0x75, 0xa4, 0x69, 0xca, 0x9e, 0xd1, 0x74, 0xb9, 0xaf, 0x5a, 0xb4, 0x5b, 0xf5, 0xe6, 0xb3, - 0x72, 0xf9, 0x11, 0xcd, 0xc4, 0x7c, 0xbc, 0x82, 0x67, 0x19, 0x88, 0x04, 0xf6, 0x1b, 0x95, 0x00, - 0x98, 0xe3, 0x27, 0xf5, 0x17, 0x4a, 0x9e, 0xe3, 0x84, 0x8b, 0xb2, 0xd1, 0xa0, 0x08, 0x09, 0x72, - 0xcc, 0x3d, 0xa9, 0x65, 0xb0, 0xa8, 0xe4, 0x1c, 0xdb, 0x86, 0xa3, 0xc4, 0x4d, 0x83, 0x46, 0x24, - 0x7f, 0x07, 0x78, 0xd0, 0x8d, 0xe2, 0x78, 0x21, 0x74, 0xa5, 0xdc, 0xe9, 0xc9, 0x52, 0x46, 0xa6, - 0xd4, 0x7a, 0xbd, 0x21, 0x67, 0x58, 0x86, 0xba, 0xe3, 0x9e, 0x27, 0xcc, 0x62, 0x7a, 0xfc, 0x30, - 0x8b, 0xbf, 0x26, 0xc0, 0x25, 0xde, 0x3e, 0xca, 0x2e, 0x75, 0x2d, 0x57, 0x9b, 0x86, 0xdb, 0x55, - 0x0e, 0x8f, 0xe6, 0xd3, 0xd4, 0xdc, 0xfa, 0x6e, 0x64, 0x3b, 0x87, 0xba, 0x57, 0xd1, 0x6b, 0xed, - 0xee, 0x06, 0x67, 0x5e, 0x3f, 0xaa, 0x98, 0xae, 0xdd, 0x2d, 0x5d, 0x3e, 0x3d, 0x59, 0x9a, 0x1b, - 0x4c, 0x7d, 0x26, 0xcf, 0x39, 0x83, 0x2c, 0xa8, 0x0a, 0x80, 0xfd, 0xee, 0x4d, 0x67, 0xcf, 0xe8, - 0x65, 0x33, 0x72, 0x1c, 0xc8, 0x21, 0x5e, 0x74, 0x07, 0x44, 0x1e, 0xd6, 0xb2, 0x67, 0x34, 0xb1, - 0xe2, 0x18, 0x9f, 0x62, 0x3a, 0xcf, 0x26, 0xe5, 0x3c, 0xa3, 0x13, 0x11, 0x75, 0xe3, 0x53, 0x8c, - 0xee, 0xc3, 0xc5, 0xa0, 0x05, 0x94, 0x5d, 0xdc, 0xb4, 0x5e, 0xb2, 0xec, 0x59, 0x9a, 0x1d, 0xf9, - 0xda, 0x2f, 0x91, 0x24, 0xc2, 0xb2, 0xf0, 0x43, 0x98, 0x1f, 0x56, 0xe1, 0xf0, 0x60, 0xcc, 0xb0, - 0xb3, 0xd2, 0xf7, 0x7a, 0x37, 0x4a, 0xc6, 0x18, 0x34, 0x7c, 0xb3, 0xe4, 0xfd, 0xc4, 0x7b, 0x82, - 0xf4, 0xf7, 0x13, 0x90, 0x2b, 0x75, 0x9a, 0x87, 0x4f, 0xda, 0x75, 0x76, 0x25, 0x00, 0xba, 0x0a, - 0x19, 0x5d, 0x75, 0x55, 0x56, 0x48, 0x81, 0x85, 0xb7, 0x11, 0x02, 0xad, 0xcd, 0x6d, 0x28, 0x84, - 0xfc, 0x50, 0xb8, 0xb7, 0x3d, 0xad, 0x76, 0x40, 0xa6, 0x0e, 0xf1, 0xef, 0xc1, 0x7c, 0x28, 0x23, - 0xdd, 0xd5, 0x50, 0xb0, 0xe9, 0xda, 0x06, 0x66, 0x3b, 0x73, 0x49, 0x39, 0xe4, 0x2c, 0x53, 0x23, - 0xc9, 0x15, 0x96, 0x8a, 0x1a, 0x30, 0x43, 0x32, 0x76, 0x15, 0x3a, 0x03, 0x7b, 0x3b, 0xa7, 0xf7, - 0x23, 0xaa, 0xd5, 0x53, 0xee, 0x22, 0xd5, 0x4f, 0x99, 0xf2, 0xd0, 0x9f, 0x72, 0x16, 0x07, 0x94, - 0x85, 0x8f, 0x40, 0xec, 0xcf, 0x10, 0xd6, 0x65, 0x8a, 0xe9, 0xf2, 0x42, 0x58, 0x97, 0xc9, 0x90, - 0x9e, 0xd6, 0x52, 0xe9, 0x94, 0x38, 0x29, 0xfd, 0x4e, 0x0a, 0xf2, 0x5e, 0xcf, 0x8c, 0xd3, 0xa4, - 0x2f, 0xc1, 0x24, 0xe9, 0x47, 0x9e, 0x6b, 0xc7, 0xad, 0x11, 0x03, 0x82, 0xfb, 0x77, 0x93, 0xfe, - 0xe5, 0xa1, 0x4f, 0xca, 0x1a, 0xc7, 0x94, 0xb2, 0xf0, 0x07, 0x09, 0x48, 0x51, 0x2b, 0xfa, 0x3e, - 0xa4, 0xe8, 0x9d, 0x00, 0xc2, 0xc8, 0x3b, 0x01, 0x3c, 0xd7, 0x00, 0x92, 0x15, 0x6d, 0x92, 0xc1, - 0xa5, 0x7b, 0xf3, 0x6e, 0x66, 0x9c, 0xf7, 0x8b, 0x7c, 0xde, 0x4d, 0x57, 0xe8, 0xfd, 0x09, 0x8d, - 0x3a, 0x35, 0x8d, 0xd8, 0x8c, 0xeb, 0xad, 0x91, 0x89, 0x90, 0xa1, 0x58, 0xa2, 0xae, 0x4a, 0x96, - 0xed, 0x62, 0x9d, 0x1b, 0xbd, 0xd7, 0xce, 0xea, 0x16, 0x9e, 0x51, 0xef, 0xf1, 0xa1, 0x2b, 0x90, - 0x24, 0x53, 0xe1, 0x34, 0xf3, 0x3a, 0x38, 0x3d, 0x59, 0x4a, 0x92, 0x49, 0x90, 0xd0, 0xd0, 0x32, - 0x64, 0x7b, 0x27, 0x27, 0x62, 0x37, 0xd1, 0x59, 0x3c, 0x34, 0xb1, 0x40, 0xd3, 0x1f, 0x91, 0x0c, - 0xf0, 0xb1, 0xae, 0xc1, 0xfd, 0x0d, 0x7e, 0x53, 0xe0, 0xee, 0x95, 0x75, 0x8d, 0x2c, 0xbf, 0x76, - 0x9c, 0xeb, 0xe3, 0x5d, 0x10, 0x6d, 0xd5, 0xd4, 0xad, 0x96, 0xf1, 0x29, 0x66, 0x1b, 0x0c, 0x0e, - 0x3f, 0x79, 0x29, 0xf8, 0x74, 0xba, 0x13, 0xe0, 0x48, 0xff, 0x49, 0xe0, 0xae, 0x98, 0x7e, 0x31, - 0xe2, 0x3d, 0x1f, 0xcf, 0xf2, 0xdd, 0x49, 0x73, 0xcf, 0xf2, 0x3c, 0x49, 0x5e, 0x1b, 0xe6, 0x37, - 0x55, 0x33, 0xf7, 0x2c, 0xef, 0xa4, 0xcf, 0xf6, 0x08, 0xce, 0xc2, 0xaf, 0xc2, 0x24, 0x4d, 0x7e, - 0x85, 0xae, 0xe6, 0xbb, 0xff, 0x26, 0xc4, 0xa4, 0xf4, 0xa7, 0x09, 0x78, 0x83, 0x56, 0xf5, 0x19, - 0xb6, 0x8d, 0xbd, 0xee, 0xb6, 0x6d, 0xb9, 0x58, 0x73, 0xb1, 0x1e, 0x6c, 0xb0, 0xc5, 0xd8, 0x04, - 0x3a, 0x64, 0xf8, 0xd1, 0xa4, 0xa1, 0xf3, 0x4b, 0x40, 0x1e, 0x7f, 0x3d, 0xe0, 0x9d, 0x66, 0x47, - 0x9a, 0xb5, 0x55, 0x39, 0xcd, 0x24, 0xd7, 0x74, 0xb4, 0x02, 0x99, 0xb6, 0x57, 0x8d, 0x73, 0x79, - 0xbf, 0xf8, 0x5c, 0x68, 0x1d, 0x0a, 0xbc, 0xa0, 0x6a, 0xd3, 0x38, 0xc2, 0x8a, 0xea, 0x9e, 0x67, - 0x46, 0xc8, 0x31, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x4a, 0x7f, 0x23, 0x05, 0x37, 0xcf, 0x50, 0x71, - 0x9c, 0xdd, 0x6b, 0x01, 0xd2, 0x47, 0xe4, 0x45, 0x06, 0xaf, 0x7d, 0x5a, 0xf6, 0x9f, 0xd1, 0x6e, - 0xcf, 0xb2, 0xb2, 0xa7, 0x1a, 0x4d, 0xb2, 0x0c, 0x31, 0x7f, 0xc3, 0xe1, 0x1e, 0x4d, 0xd1, 0xfe, - 0x7b, 0xa1, 0x05, 0xe8, 0x11, 0x15, 0x44, 0xb3, 0x39, 0xe8, 0x33, 0x01, 0x16, 0xd8, 0x0b, 0x99, - 0xd3, 0x5b, 0xdf, 0x6b, 0x52, 0xf4, 0x35, 0xab, 0x11, 0xaf, 0x19, 0x4b, 0x47, 0xc5, 0xd0, 0xbb, - 0x78, 0x41, 0xe6, 0xc3, 0x6f, 0x0b, 0x17, 0x65, 0xe1, 0xb7, 0x04, 0xc8, 0x86, 0x08, 0xe8, 0xd6, - 0x40, 0x88, 0x51, 0xf6, 0x34, 0x2a, 0xae, 0xe8, 0xe6, 0x40, 0x5c, 0x51, 0x29, 0xfd, 0xd5, 0xc9, - 0x52, 0x4a, 0x66, 0xae, 0xeb, 0x5e, 0x84, 0xd1, 0xf5, 0xe0, 0x46, 0x9b, 0x64, 0x5f, 0x26, 0xef, - 0x4a, 0x1b, 0x8a, 0x50, 0x55, 0xef, 0x44, 0x8c, 0x22, 0x54, 0xf2, 0x24, 0xfd, 0x24, 0x01, 0xb3, - 0x2b, 0xba, 0x5e, 0xaf, 0x53, 0x54, 0x13, 0xe7, 0x18, 0x43, 0x90, 0x22, 0xe6, 0x06, 0x0f, 0x87, - 0xa2, 0xbf, 0xd1, 0x9b, 0x80, 0x74, 0xc3, 0x61, 0x37, 0x43, 0x38, 0x07, 0xaa, 0x6e, 0xbd, 0x0c, - 0x0e, 0xbe, 0x67, 0xbd, 0x94, 0xba, 0x97, 0x80, 0xea, 0x40, 0x6d, 0x6d, 0xc5, 0x71, 0x55, 0x7f, - 0x63, 0xff, 0xe6, 0x58, 0x01, 0x36, 0xcc, 0x38, 0xf6, 0x1f, 0xe5, 0x0c, 0x91, 0x43, 0x7f, 0x12, - 0x93, 0xcf, 0x20, 0x8d, 0xe2, 0x2a, 0xaa, 0xe3, 0x85, 0x86, 0xb0, 0x3b, 0x29, 0xf2, 0x8c, 0xbe, - 0xe2, 0xb0, 0x88, 0x0f, 0xe6, 0x51, 0x1e, 0xa8, 0x26, 0xce, 0x6d, 0xd9, 0xbf, 0x23, 0x40, 0x5e, - 0xc6, 0x7b, 0x36, 0x76, 0x0e, 0xe2, 0xd4, 0xf9, 0x23, 0x98, 0xb1, 0x99, 0x54, 0x65, 0xcf, 0xb6, - 0x5a, 0xe7, 0x99, 0x2b, 0xb2, 0x9c, 0xf1, 0x91, 0x6d, 0xb5, 0xf8, 0x94, 0xfc, 0x0c, 0x0a, 0x7e, - 0x19, 0xe3, 0xac, 0xfc, 0xdf, 0xa3, 0xc1, 0xa3, 0x4c, 0x70, 0xdc, 0x27, 0xd0, 0xf1, 0x6a, 0x80, - 0x6e, 0xcd, 0x87, 0x0b, 0x1a, 0xa7, 0x1a, 0xfe, 0xa3, 0x00, 0xf9, 0x7a, 0x67, 0x97, 0xdd, 0x78, - 0x14, 0x9f, 0x06, 0x2a, 0x90, 0x69, 0xe2, 0x3d, 0x57, 0x79, 0x25, 0x5f, 0xe8, 0x34, 0x61, 0xa5, - 0x9e, 0xe0, 0x8f, 0x01, 0x6c, 0x1a, 0x3d, 0x45, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x43, 0x79, 0x09, - 0x99, 0xac, 0x3a, 0x05, 0xbf, 0x9a, 0x71, 0xae, 0x2f, 0xcf, 0x7b, 0x66, 0x87, 0xe4, 0x79, 0x66, - 0x87, 0x59, 0x6e, 0xb0, 0x46, 0xcf, 0x10, 0x45, 0x98, 0xa3, 0x66, 0x99, 0xa2, 0xb6, 0xdb, 0x4d, - 0xc3, 0x83, 0x3d, 0x74, 0xfe, 0x49, 0xc9, 0xb3, 0x34, 0x69, 0x85, 0xa5, 0x50, 0xc0, 0x83, 0x7e, - 0x24, 0xc0, 0xcc, 0x9e, 0x8d, 0xf1, 0xa7, 0x58, 0xa1, 0x53, 0xf2, 0x78, 0x5e, 0x05, 0xab, 0xa4, - 0x0c, 0x5f, 0xfb, 0xd4, 0x31, 0xcb, 0x5e, 0x5c, 0x27, 0xef, 0x45, 0x5b, 0x20, 0x6a, 0x4d, 0x76, - 0x0e, 0xea, 0x7b, 0x38, 0x4c, 0x8d, 0x3f, 0x00, 0x0a, 0x8c, 0x39, 0x70, 0x72, 0x78, 0x4a, 0x06, - 0x93, 0xaa, 0x2b, 0xfc, 0x96, 0x39, 0x6a, 0x6c, 0xf7, 0x7a, 0x38, 0x84, 0xa3, 0xc8, 0x43, 0x97, - 0xd3, 0x15, 0x65, 0xac, 0xea, 0xdc, 0x72, 0x27, 0xe3, 0xca, 0x7f, 0xe0, 0xe3, 0xea, 0x39, 0xcc, - 0xd2, 0x7e, 0x13, 0x77, 0x30, 0xa8, 0xf4, 0x8f, 0x92, 0x80, 0xc2, 0x92, 0xbf, 0xb9, 0xfe, 0x96, - 0x88, 0xaf, 0xbf, 0xad, 0x81, 0x14, 0x32, 0x86, 0x9a, 0xaa, 0xe3, 0x2a, 0xcc, 0x95, 0xce, 0x51, - 0xda, 0xd8, 0x56, 0x1c, 0xac, 0x59, 0xfc, 0x3e, 0x20, 0x41, 0x5e, 0x0c, 0x72, 0x6e, 0xa8, 0x8e, - 0xfb, 0x94, 0xe5, 0xdb, 0xc6, 0x76, 0x9d, 0xe6, 0x42, 0x0f, 0xe1, 0x52, 0x4b, 0x3d, 0x8e, 0xe2, - 0x9f, 0xa4, 0xfc, 0x73, 0x2d, 0xf5, 0x78, 0x80, 0xe9, 0x7d, 0x58, 0x88, 0x66, 0x52, 0x1c, 0xec, - 0x1d, 0xb5, 0x5d, 0x8a, 0x60, 0xac, 0x63, 0x17, 0xad, 0x00, 0x04, 0x20, 0x82, 0xaf, 0xd1, 0xe3, - 0x60, 0x88, 0x8c, 0x8f, 0x21, 0xa4, 0x1f, 0x0b, 0x90, 0xdf, 0x34, 0xf6, 0x6d, 0x35, 0xd6, 0xdb, - 0x76, 0xd0, 0xfb, 0xbd, 0x67, 0x93, 0xd9, 0x07, 0x0b, 0x51, 0xbe, 0x27, 0x2c, 0x87, 0xb7, 0xff, - 0xc8, 0x19, 0xc8, 0xd2, 0xe7, 0x97, 0x28, 0xce, 0x39, 0x5f, 0x83, 0xd7, 0xa9, 0x37, 0x1f, 0x77, - 0xe5, 0xf9, 0x46, 0xd0, 0x8d, 0xf4, 0xfb, 0x02, 0x2c, 0x0e, 0x7b, 0x4b, 0x9c, 0x03, 0x42, 0xa6, - 0xf7, 0x0e, 0xd2, 0x37, 0x28, 0xfe, 0x88, 0x38, 0x63, 0xa6, 0x41, 0x7c, 0x24, 0x80, 0x5f, 0xb6, - 0x3a, 0xbd, 0x91, 0x90, 0xfd, 0x76, 0x88, 0x82, 0xea, 0x9a, 0x6a, 0xd2, 0x50, 0xe1, 0x26, 0x56, - 0x8f, 0x30, 0x77, 0x44, 0x8c, 0x75, 0x96, 0xf8, 0x42, 0x80, 0xc5, 0x61, 0x6f, 0x89, 0x53, 0x41, - 0xdf, 0x85, 0x69, 0xe6, 0xd1, 0xe6, 0x81, 0x9a, 0x2b, 0x43, 0xa2, 0xa2, 0x4d, 0x2f, 0x16, 0xc7, - 0xcb, 0x2f, 0x35, 0x20, 0x5f, 0x52, 0x6d, 0xdb, 0x88, 0x75, 0xeb, 0x41, 0xfa, 0x89, 0x00, 0x05, - 0x5f, 0x6c, 0x9c, 0x35, 0xfd, 0xfa, 0x81, 0x1e, 0xd2, 0x7f, 0xbd, 0x0a, 0x33, 0xbc, 0xfc, 0x3b, - 0xa6, 0x61, 0x99, 0xe8, 0x3e, 0x24, 0xf7, 0xf9, 0x01, 0x56, 0x36, 0x72, 0x8f, 0x3e, 0xb8, 0xe6, - 0xb1, 0x3a, 0x21, 0x93, 0xbc, 0x84, 0xa5, 0xdd, 0x71, 0x23, 0x0a, 0x10, 0x44, 0x2a, 0x84, 0x59, - 0xda, 0x1d, 0x17, 0xd5, 0xa1, 0xa0, 0x05, 0x77, 0xcb, 0x29, 0x84, 0x3d, 0x39, 0x74, 0x43, 0x39, - 0xf2, 0x96, 0xbf, 0xea, 0x84, 0x9c, 0xd7, 0x7a, 0x12, 0x50, 0x39, 0x7c, 0xa5, 0x59, 0x6a, 0xc0, - 0x0d, 0x32, 0x68, 0xfa, 0xde, 0xeb, 0xd4, 0xaa, 0x13, 0xa1, 0x9b, 0xcf, 0xd0, 0xfb, 0x30, 0xa5, - 0xd3, 0xcb, 0xb3, 0xb8, 0x3d, 0x11, 0xd5, 0xe0, 0x3d, 0x77, 0x94, 0x55, 0x27, 0x64, 0xce, 0x81, - 0xd6, 0x60, 0x86, 0xfd, 0x62, 0x68, 0x97, 0x5b, 0x01, 0x37, 0x87, 0x4b, 0x08, 0xd9, 0xe1, 0xd5, - 0x09, 0x39, 0xab, 0x07, 0x54, 0xf4, 0x18, 0xb2, 0x5a, 0x13, 0xab, 0x36, 0x17, 0x75, 0x6b, 0x68, - 0xec, 0xe6, 0xc0, 0x85, 0x5b, 0xd5, 0x09, 0x19, 0x34, 0x9f, 0x48, 0x0a, 0x65, 0xd3, 0x7b, 0x97, - 0xb8, 0xa4, 0xb7, 0x86, 0x16, 0x6a, 0xf0, 0x12, 0xab, 0x2a, 0xb5, 0xcf, 0x7d, 0x2a, 0x7a, 0x1b, - 0x52, 0x8e, 0xa6, 0x9a, 0xdc, 0x24, 0x59, 0x1c, 0x72, 0x31, 0x4e, 0xc0, 0x4c, 0x73, 0xa3, 0x0f, - 0x18, 0x50, 0x76, 0x8f, 0xbd, 0x53, 0x83, 0x28, 0x9d, 0xf6, 0x5c, 0xc0, 0x40, 0x74, 0x8a, 0x29, - 0x81, 0xe8, 0x41, 0xd5, 0x5b, 0x86, 0xa9, 0xd0, 0x83, 0x19, 0x7a, 0x4c, 0x10, 0xad, 0x87, 0x81, - 0x28, 0xf6, 0x2a, 0xbd, 0xe5, 0xc1, 0x23, 0xa2, 0x4d, 0xc8, 0x31, 0x41, 0x1d, 0x16, 0x60, 0x3d, - 0xbf, 0x3c, 0xd4, 0x17, 0x21, 0x22, 0xc4, 0xbb, 0x3a, 0x21, 0xcf, 0xa8, 0x21, 0x72, 0x50, 0xae, - 0x16, 0xb6, 0xf7, 0xd9, 0x79, 0xc4, 0x88, 0x72, 0x85, 0x1d, 0x3c, 0xfd, 0x72, 0x51, 0x22, 0xfa, - 0x0d, 0xb8, 0xc0, 0x04, 0xb9, 0xdc, 0x6f, 0x8d, 0xbb, 0x3f, 0xbd, 0x3e, 0xd4, 0x8f, 0x60, 0x68, - 0x50, 0x74, 0x75, 0x42, 0x46, 0xea, 0x40, 0x22, 0xd2, 0xe0, 0x22, 0x7b, 0x03, 0x8f, 0xaa, 0xb5, - 0x79, 0x20, 0xe8, 0xfc, 0x0d, 0xfa, 0x8a, 0x37, 0x87, 0xbd, 0x22, 0x32, 0xd8, 0xb7, 0x3a, 0x21, - 0xcf, 0xa9, 0x83, 0xa9, 0x41, 0x35, 0x6c, 0x1e, 0xbf, 0xc8, 0xbb, 0xdb, 0x9b, 0xa3, 0xab, 0x11, - 0x15, 0xf7, 0xe9, 0x57, 0xa3, 0x27, 0x91, 0x34, 0xa0, 0x7f, 0x7b, 0x03, 0xed, 0x4c, 0x33, 0x43, - 0x1b, 0x30, 0x22, 0xc8, 0x91, 0x34, 0xe0, 0x41, 0x88, 0x8c, 0x8a, 0x90, 0xd8, 0xd7, 0xe8, 0xa9, - 0x5f, 0xb4, 0xe9, 0xe4, 0x07, 0xf2, 0x55, 0x27, 0xe4, 0xc4, 0xbe, 0x86, 0x3e, 0x82, 0x34, 0x8b, - 0xca, 0x3a, 0x36, 0xe7, 0xf3, 0x43, 0xe7, 0xec, 0xde, 0xd8, 0xb6, 0xea, 0x84, 0x4c, 0x03, 0xc1, - 0x78, 0x47, 0xe6, 0x11, 0x37, 0x54, 0x44, 0x71, 0x44, 0x30, 0x76, 0x5f, 0xdc, 0x13, 0xe9, 0x30, - 0xb6, 0x4f, 0x44, 0xdb, 0x90, 0xe7, 0x4b, 0xb7, 0x17, 0x41, 0x20, 0x0e, 0xf5, 0xd4, 0x89, 0x0a, - 0x22, 0xa8, 0xd2, 0x2d, 0xca, 0x10, 0x9d, 0xb4, 0x5d, 0xaf, 0x44, 0xde, 0x76, 0xb3, 0x43, 0xdb, - 0x6e, 0xa8, 0x43, 0x3b, 0x69, 0x3b, 0x7b, 0x20, 0x11, 0xbd, 0x0b, 0x93, 0x6c, 0x9c, 0x20, 0x2a, - 0x32, 0xca, 0xf9, 0xac, 0x6f, 0x88, 0xb0, 0xfc, 0x64, 0xf6, 0x72, 0xb9, 0x63, 0xae, 0xd2, 0xb4, - 0xf6, 0xe7, 0xe7, 0x86, 0xce, 0x5e, 0x83, 0x2e, 0xc6, 0x64, 0xf6, 0x72, 0x03, 0x2a, 0xe9, 0x40, - 0x36, 0x4b, 0xe1, 0x43, 0xec, 0xc2, 0xd0, 0x0e, 0x14, 0xe1, 0xaf, 0x5b, 0xa5, 0x21, 0x53, 0x01, - 0xd9, 0x9f, 0x58, 0x1d, 0xac, 0xd0, 0x49, 0xf1, 0xe2, 0xe8, 0x89, 0xb5, 0xe7, 0xb6, 0x32, 0x7f, - 0x62, 0x65, 0x54, 0xf4, 0x0c, 0x44, 0x7e, 0x65, 0x8e, 0xe2, 0xf9, 0x8f, 0xcd, 0x5f, 0xa2, 0xf2, - 0xee, 0x46, 0x2e, 0x88, 0x51, 0xae, 0x85, 0x55, 0x82, 0x25, 0x7b, 0x53, 0xd0, 0xc7, 0x30, 0x4b, - 0xe5, 0x29, 0x5a, 0x70, 0xcb, 0xd1, 0xfc, 0xfc, 0xc0, 0x9d, 0x39, 0xc3, 0x2f, 0x44, 0xf2, 0x24, - 0x8b, 0x5a, 0x5f, 0x12, 0x19, 0x0f, 0x86, 0x69, 0xb8, 0x74, 0xed, 0x5e, 0x18, 0x3a, 0x1e, 0x7a, - 0x6f, 0x78, 0xad, 0x52, 0x5b, 0x8b, 0x52, 0x48, 0x37, 0xee, 0x9b, 0xf1, 0x5e, 0x1b, 0xda, 0x8d, - 0x87, 0x4c, 0x76, 0x39, 0xb7, 0x67, 0x9e, 0x5b, 0x05, 0x60, 0x3b, 0x08, 0x14, 0x14, 0x2d, 0x0e, - 0x35, 0x00, 0xfa, 0xfd, 0x69, 0x89, 0x01, 0xd0, 0xf4, 0x68, 0xc4, 0x00, 0x60, 0xc7, 0x5d, 0xf3, - 0xd7, 0x86, 0x2f, 0x56, 0xe1, 0xf3, 0x75, 0xba, 0x58, 0x51, 0x02, 0x31, 0xc8, 0x08, 0x9c, 0xeb, - 0xd2, 0x11, 0x7e, 0x7d, 0xe8, 0xee, 0x4d, 0x5f, 0xa0, 0x5d, 0x75, 0x42, 0x4e, 0xbf, 0xe0, 0x24, - 0xd2, 0xab, 0x98, 0x08, 0x3e, 0xb6, 0xef, 0x0d, 0xed, 0x55, 0x83, 0x11, 0x56, 0xa4, 0x57, 0xbd, - 0x08, 0xa8, 0xc1, 0x92, 0xe7, 0xb0, 0x83, 0xac, 0xf9, 0x37, 0x46, 0x2f, 0x79, 0xbd, 0xc7, 0x6e, - 0xfe, 0x92, 0xc7, 0xc9, 0x6c, 0xc9, 0xd3, 0x15, 0xc7, 0xa1, 0x6e, 0x3a, 0xf3, 0x37, 0x47, 0x2c, - 0x79, 0x7d, 0x5b, 0xdb, 0x6c, 0xc9, 0xd3, 0xeb, 0x8c, 0x93, 0x58, 0x7f, 0xb6, 0x77, 0x45, 0x14, - 0x07, 0xf6, 0xb7, 0x87, 0x5a, 0x7f, 0x91, 0x77, 0x58, 0x11, 0xeb, 0xcf, 0xee, 0x49, 0x40, 0xdf, - 0x83, 0x69, 0xbe, 0x95, 0x38, 0x7f, 0x67, 0x84, 0x49, 0x1d, 0xde, 0xfd, 0x25, 0xdd, 0x91, 0xf3, - 0xb0, 0xc9, 0x81, 0x6d, 0x61, 0xb2, 0xc9, 0xef, 0xee, 0x88, 0xc9, 0x61, 0x60, 0x17, 0x95, 0x4d, - 0x0e, 0x01, 0x99, 0x94, 0xc6, 0x61, 0xdb, 0x6f, 0xf3, 0xbf, 0x32, 0xb4, 0x34, 0xbd, 0xfb, 0x90, - 0xa4, 0x34, 0x9c, 0x87, 0x2e, 0x16, 0x74, 0xad, 0x66, 0xda, 0xf9, 0xce, 0xf0, 0xc5, 0xa2, 0x7f, - 0x43, 0xa7, 0xea, 0x1d, 0x14, 0x32, 0xad, 0xfc, 0x15, 0x01, 0xae, 0xb1, 0x3e, 0x40, 0x8f, 0x49, - 0xba, 0x8a, 0x7f, 0xca, 0x15, 0xda, 0xad, 0xba, 0x4f, 0xc5, 0xbf, 0x7b, 0xfe, 0x43, 0x19, 0xef, - 0x8d, 0xaf, 0xab, 0xa3, 0xf2, 0x11, 0x65, 0xb4, 0x18, 0xae, 0x9f, 0x7f, 0x30, 0x54, 0x19, 0xbd, - 0x7b, 0x11, 0x44, 0x19, 0x9c, 0x07, 0x35, 0x61, 0x9e, 0x0d, 0x89, 0x00, 0xf7, 0xfa, 0x45, 0x7f, - 0x38, 0xd4, 0xf3, 0x75, 0x24, 0xe2, 0xaf, 0x4e, 0xc8, 0x97, 0x5e, 0x44, 0x66, 0x20, 0x6f, 0xa3, - 0x77, 0x40, 0x1a, 0x01, 0x4c, 0x55, 0x3c, 0x3c, 0xf9, 0xf6, 0xd0, 0xb7, 0x8d, 0x84, 0xcf, 0xe4, - 0x6d, 0x4e, 0x64, 0x06, 0xa2, 0x9a, 0x5d, 0x06, 0x0d, 0xe7, 0xdf, 0x19, 0xaa, 0x9a, 0x5e, 0x4c, - 0x4a, 0x54, 0xc3, 0x79, 0x4a, 0xd3, 0xdc, 0xe5, 0xc2, 0x0f, 0xed, 0x2e, 0x88, 0xe2, 0x5a, 0x2a, - 0x7d, 0x59, 0x9c, 0x5f, 0x4b, 0xa5, 0xaf, 0x88, 0x0b, 0x6b, 0xa9, 0xf4, 0x55, 0xf1, 0xb5, 0xb5, - 0x54, 0x7a, 0x49, 0xbc, 0xb6, 0x96, 0x4a, 0x4b, 0xe2, 0x0d, 0xe9, 0xe7, 0x57, 0x21, 0xe7, 0x21, - 0x4b, 0x06, 0xf9, 0x1e, 0x84, 0x21, 0xdf, 0xe2, 0x30, 0xc8, 0xc7, 0xb1, 0x28, 0xc7, 0x7c, 0x0f, - 0xc2, 0x98, 0x6f, 0x71, 0x18, 0xe6, 0x0b, 0x78, 0x08, 0xe8, 0x6b, 0x0c, 0x03, 0x7d, 0x77, 0xc7, - 0x00, 0x7d, 0xbe, 0xa8, 0x7e, 0xd4, 0xb7, 0x3a, 0x88, 0xfa, 0xde, 0x18, 0x8d, 0xfa, 0x7c, 0x51, - 0x21, 0xd8, 0xf7, 0x41, 0x1f, 0xec, 0xbb, 0x3e, 0x02, 0xf6, 0xf9, 0xfc, 0x1e, 0xee, 0x5b, 0x8f, - 0xc4, 0x7d, 0xb7, 0xce, 0xc2, 0x7d, 0xbe, 0x9c, 0x1e, 0xe0, 0x57, 0x8d, 0x02, 0x7e, 0x37, 0xcf, - 0x00, 0x7e, 0xbe, 0xa8, 0x30, 0xf2, 0x5b, 0x8f, 0x44, 0x7e, 0xb7, 0xce, 0x42, 0x7e, 0x41, 0xb1, - 0xc2, 0xd0, 0xef, 0x9d, 0x1e, 0xe8, 0xb7, 0x34, 0x14, 0xfa, 0xf9, 0xdc, 0x0c, 0xfb, 0x7d, 0xd8, - 0x8f, 0xfd, 0xae, 0x8f, 0xc0, 0x7e, 0x81, 0x62, 0x39, 0xf8, 0xab, 0x46, 0x81, 0xbf, 0x9b, 0x67, - 0x80, 0xbf, 0x40, 0x17, 0x21, 0xf4, 0xb7, 0x15, 0x8d, 0xfe, 0x6e, 0x9f, 0x89, 0xfe, 0x7c, 0x69, - 0xbd, 0xf0, 0xaf, 0x1a, 0x05, 0xff, 0x6e, 0x9e, 0x01, 0xff, 0xfa, 0x4a, 0xc6, 0xf0, 0x9f, 0x3a, - 0x12, 0xff, 0xbd, 0x39, 0x26, 0xfe, 0xf3, 0x45, 0x47, 0x01, 0x40, 0x7d, 0x34, 0x00, 0x2c, 0x8e, - 0x0b, 0x00, 0xfd, 0x97, 0x44, 0x22, 0x40, 0x75, 0x24, 0x02, 0x7c, 0x73, 0x4c, 0x04, 0xd8, 0x57, - 0x91, 0x5e, 0x08, 0xb8, 0x15, 0x0d, 0x01, 0x6f, 0x9f, 0x09, 0x01, 0x83, 0x56, 0xec, 0xc1, 0x80, - 0xcb, 0x21, 0x0c, 0xf8, 0xfa, 0x10, 0x0c, 0xe8, 0xb3, 0x12, 0x10, 0xf8, 0xfd, 0x01, 0x10, 0x28, - 0x8d, 0x02, 0x81, 0x3e, 0xaf, 0x8f, 0x02, 0xab, 0x51, 0x28, 0xf0, 0xe6, 0x19, 0x28, 0x30, 0xe8, - 0x37, 0x21, 0x18, 0xf8, 0x74, 0x08, 0x0c, 0xbc, 0x73, 0x36, 0x0c, 0xf4, 0xe5, 0xf5, 0xe1, 0x40, - 0x75, 0x24, 0x0e, 0x7c, 0x73, 0x4c, 0x1c, 0x18, 0xb4, 0x60, 0x04, 0x10, 0x7c, 0xaf, 0x17, 0x08, - 0x5e, 0x1b, 0x0e, 0x04, 0x7d, 0x31, 0x1c, 0x09, 0xae, 0x47, 0x22, 0xc1, 0x5b, 0x67, 0x21, 0xc1, - 0x60, 0x36, 0x0b, 0x43, 0xc1, 0xad, 0x68, 0x28, 0x78, 0xfb, 0x4c, 0x28, 0x18, 0x74, 0xa4, 0x1e, - 0x2c, 0xb8, 0x1e, 0x89, 0x05, 0x6f, 0x9d, 0x85, 0x05, 0xfb, 0xa6, 0x5a, 0x0e, 0x06, 0x9f, 0x0f, - 0x05, 0x83, 0xf7, 0xc6, 0x01, 0x83, 0xbe, 0xd0, 0x01, 0x34, 0xf8, 0xc9, 0x70, 0x34, 0xf8, 0x2b, - 0xe7, 0xb8, 0x1e, 0x37, 0x12, 0x0e, 0x7e, 0x7f, 0x00, 0x0e, 0x4a, 0xa3, 0xe0, 0x60, 0x30, 0x32, - 0x3c, 0x3c, 0x58, 0x89, 0x40, 0x6f, 0x6f, 0x8c, 0x46, 0x6f, 0xc1, 0x42, 0x1e, 0xc0, 0xb7, 0x0f, - 0xfa, 0xe0, 0xdb, 0xf5, 0x33, 0xbd, 0x41, 0x43, 0xf8, 0xad, 0x34, 0x88, 0xdf, 0x6e, 0x8c, 0xc4, - 0x6f, 0xbe, 0x84, 0x00, 0xc0, 0xad, 0x47, 0x02, 0xb8, 0x5b, 0x67, 0x01, 0xb8, 0xa0, 0x2b, 0x84, - 0x11, 0xdc, 0x56, 0x34, 0x82, 0xbb, 0x7d, 0x26, 0x82, 0xeb, 0x5b, 0xb6, 0x3c, 0x08, 0x57, 0x8d, - 0x82, 0x70, 0x37, 0xcf, 0x80, 0x70, 0xe1, 0x65, 0xcb, 0xc7, 0x70, 0x8d, 0x61, 0x18, 0xee, 0xee, - 0x18, 0x18, 0x2e, 0x30, 0xe6, 0xfa, 0x40, 0xdc, 0x47, 0xfd, 0x20, 0x4e, 0x1a, 0x05, 0xe2, 0x82, - 0x4e, 0xe4, 0xa1, 0xb8, 0xad, 0x68, 0x14, 0x77, 0xfb, 0x4c, 0x14, 0x17, 0x1e, 0xd7, 0x21, 0x18, - 0xf7, 0x51, 0x3f, 0x8c, 0x93, 0x46, 0xc1, 0xb8, 0xa0, 0x3c, 0x1e, 0x8e, 0xab, 0x46, 0xe1, 0xb8, - 0x9b, 0x67, 0xe0, 0xb8, 0xd0, 0x74, 0x1f, 0x00, 0xb9, 0xbf, 0x3a, 0x3e, 0x90, 0x7b, 0xef, 0x55, - 0xbd, 0xeb, 0xce, 0x46, 0x72, 0x1f, 0xf5, 0x23, 0x39, 0x69, 0x14, 0x92, 0x0b, 0xf4, 0xe1, 0x41, - 0xb9, 0xd6, 0x99, 0x50, 0xee, 0xfe, 0x39, 0xa0, 0x9c, 0x2f, 0x7f, 0x18, 0x96, 0x6b, 0x9d, 0x89, - 0xe5, 0xee, 0x9f, 0x03, 0xcb, 0x05, 0xaf, 0x1b, 0x02, 0xe6, 0x3e, 0xea, 0x07, 0x73, 0xd2, 0x28, - 0x30, 0x17, 0x68, 0xe7, 0xdc, 0x68, 0xee, 0x35, 0xf1, 0xf5, 0x1e, 0x4c, 0xf7, 0x37, 0x33, 0x30, - 0x55, 0x8d, 0x38, 0x13, 0x14, 0x5e, 0xe5, 0x4c, 0x10, 0x7d, 0x0f, 0xae, 0xfa, 0x0f, 0xd4, 0x9f, - 0x4b, 0xe1, 0x21, 0x71, 0x5a, 0xd3, 0xd2, 0x0e, 0xe9, 0x52, 0x99, 0x96, 0xe7, 0xfd, 0x2c, 0x8f, - 0x6c, 0xab, 0xc5, 0x42, 0xe3, 0xa8, 0x97, 0x0c, 0x5a, 0x25, 0x63, 0x98, 0xda, 0x84, 0x67, 0xdf, - 0x38, 0x3a, 0x78, 0x27, 0x26, 0x67, 0x7d, 0x85, 0x4b, 0x1c, 0xd0, 0x3b, 0x90, 0xeb, 0x38, 0xd8, - 0x56, 0xda, 0xb6, 0x61, 0xd9, 0x86, 0xcb, 0xc2, 0xd1, 0x84, 0x92, 0xf8, 0xd5, 0xc9, 0xd2, 0xcc, - 0x8e, 0x83, 0xed, 0x6d, 0x4e, 0x97, 0x67, 0x3a, 0xa1, 0x27, 0xef, 0xd3, 0x7f, 0x93, 0xe3, 0x7f, - 0xfa, 0xef, 0x29, 0x88, 0xd4, 0x43, 0x27, 0xbc, 0x8c, 0xb2, 0x7b, 0xda, 0xa2, 0x57, 0x7c, 0x55, - 0x0f, 0xad, 0x94, 0xf4, 0xbe, 0xb6, 0x82, 0xdd, 0x4b, 0x44, 0x8f, 0x21, 0x6f, 0x5b, 0x1d, 0x7a, - 0x1d, 0x53, 0xdb, 0x6a, 0x1a, 0x5a, 0x97, 0xda, 0x37, 0xf9, 0xe8, 0xf3, 0x66, 0x96, 0x71, 0x9b, - 0xe6, 0x93, 0x73, 0x76, 0xf8, 0x11, 0xd5, 0x81, 0x5e, 0xc5, 0xe4, 0x49, 0x41, 0x03, 0x97, 0xdf, - 0x8f, 0xfc, 0x04, 0xc5, 0x73, 0xd5, 0x70, 0xb9, 0x5c, 0x78, 0xe9, 0xff, 0x46, 0x8f, 0x60, 0x86, - 0x5e, 0x0b, 0x4e, 0xda, 0xdc, 0xea, 0xb8, 0xdc, 0xbc, 0xb9, 0x52, 0x64, 0x1f, 0x88, 0x2c, 0x7a, - 0x1f, 0x88, 0x2c, 0xae, 0xf2, 0x0f, 0x44, 0xb2, 0xb3, 0xea, 0xcf, 0xff, 0xfd, 0x92, 0x20, 0x67, - 0x3d, 0xcf, 0x29, 0xab, 0xe3, 0xa2, 0xfb, 0x70, 0xb1, 0xa5, 0x1e, 0xd3, 0xdb, 0xc5, 0x15, 0xcf, - 0xfe, 0xa2, 0xf7, 0x29, 0xb2, 0xaf, 0x12, 0xa2, 0x96, 0x7a, 0x4c, 0xbf, 0x96, 0xc8, 0x92, 0xe8, - 0xa7, 0x8e, 0xae, 0xc3, 0x0c, 0x8f, 0x15, 0x62, 0x5f, 0x42, 0x2b, 0xd0, 0x9c, 0xfc, 0xb3, 0x38, - 0xec, 0x63, 0x68, 0x37, 0x21, 0xaf, 0x1b, 0x8e, 0x6b, 0x98, 0x9a, 0xcb, 0x2f, 0x2e, 0x67, 0x57, - 0x7f, 0xe7, 0x3c, 0x2a, 0xbb, 0x9d, 0xbc, 0x01, 0xb3, 0x5a, 0xd3, 0xf0, 0xad, 0x5a, 0x66, 0x67, - 0xcc, 0x0e, 0x1d, 0xa7, 0x65, 0x9a, 0xb7, 0xdf, 0x81, 0xa6, 0xa0, 0xf5, 0x92, 0x51, 0x19, 0x0a, - 0xfb, 0xaa, 0x8b, 0x5f, 0xaa, 0x5d, 0xc5, 0x0b, 0xc3, 0xcd, 0xd2, 0xab, 0x07, 0xae, 0x9e, 0x9e, - 0x2c, 0xe5, 0x1e, 0xb3, 0xa4, 0x81, 0x68, 0xdc, 0xdc, 0x7e, 0x28, 0x41, 0x47, 0xb7, 0xa1, 0xa0, - 0x3a, 0x5d, 0x53, 0xa3, 0x3d, 0x0a, 0x9b, 0x4e, 0xc7, 0xe1, 0xe1, 0x68, 0x79, 0x4a, 0x2e, 0x7b, - 0x54, 0xf4, 0x01, 0x2c, 0xf0, 0xef, 0x93, 0xbc, 0x54, 0x6d, 0x5d, 0xa1, 0xbd, 0x30, 0x18, 0xee, - 0x22, 0xe5, 0xb9, 0xcc, 0xbe, 0x47, 0x42, 0x32, 0x90, 0xae, 0x17, 0xcc, 0x86, 0xcf, 0x60, 0x96, - 0x06, 0xa5, 0x61, 0x9d, 0x2c, 0x47, 0x4d, 0x6c, 0x62, 0xc7, 0x19, 0x71, 0xca, 0x50, 0x62, 0x79, - 0xeb, 0x5e, 0x56, 0x36, 0xc1, 0xc8, 0xe2, 0x6e, 0x1f, 0xdd, 0xbf, 0x2f, 0x1d, 0xc4, 0xec, 0x5a, - 0x2a, 0x3d, 0x23, 0xe6, 0xd6, 0x52, 0xe9, 0xbc, 0x58, 0x90, 0xfe, 0x8f, 0x00, 0x97, 0xa2, 0x85, - 0xa0, 0x3a, 0xcc, 0xf5, 0x5c, 0x09, 0xc4, 0xa2, 0xe5, 0xce, 0x33, 0x5f, 0xcd, 0x86, 0x2f, 0x06, - 0xa2, 0x6f, 0x21, 0xca, 0x89, 0x10, 0xaa, 0x38, 0xae, 0x6d, 0x68, 0xde, 0x35, 0x5b, 0x97, 0x07, - 0xd8, 0xea, 0x34, 0x99, 0x96, 0x48, 0x3d, 0x1e, 0x28, 0x51, 0xf2, 0x3c, 0x25, 0x52, 0x8f, 0x7b, - 0x45, 0x4b, 0x3f, 0x4a, 0x42, 0x81, 0xac, 0xbc, 0x8e, 0x63, 0x58, 0x66, 0xd5, 0x77, 0xd0, 0xf7, - 0x27, 0x28, 0x81, 0xc6, 0x9a, 0xfa, 0xcf, 0x68, 0x89, 0xc6, 0xf4, 0x12, 0x84, 0xe3, 0x7f, 0xef, - 0x29, 0x29, 0x03, 0x23, 0xd1, 0xa8, 0xc5, 0x15, 0x98, 0x72, 0xac, 0x8e, 0xad, 0x79, 0x5f, 0x21, - 0xb9, 0x3b, 0x64, 0xa9, 0x0f, 0xbd, 0xb0, 0x58, 0xa7, 0x0c, 0x32, 0x67, 0x44, 0x9f, 0x40, 0x81, - 0xfd, 0xa2, 0xb1, 0x84, 0x34, 0x96, 0x8f, 0x05, 0x64, 0xde, 0x1f, 0x5b, 0xd6, 0x06, 0x67, 0x94, - 0xf3, 0x4e, 0xcf, 0x33, 0xfa, 0x08, 0x5e, 0x33, 0x2d, 0xa5, 0x85, 0x5b, 0x16, 0x5b, 0xe2, 0xc9, - 0x2c, 0xa3, 0x2b, 0xaa, 0xab, 0xf0, 0x42, 0x33, 0x8f, 0xef, 0x79, 0xd3, 0xda, 0xa4, 0x59, 0x64, - 0x9e, 0x63, 0xc5, 0x65, 0x72, 0xa5, 0x22, 0x4c, 0xb1, 0x5f, 0x28, 0x03, 0x93, 0x4f, 0x1a, 0xd5, - 0x8a, 0x2c, 0x4e, 0xa0, 0x19, 0x48, 0x3f, 0x92, 0x9f, 0x6c, 0x2a, 0xf5, 0xa7, 0x1b, 0xa2, 0x80, - 0xb2, 0x30, 0x2d, 0x3f, 0x79, 0xd2, 0x50, 0xd6, 0x9f, 0x89, 0x09, 0xe9, 0x36, 0xe4, 0x7b, 0x4b, - 0x84, 0x00, 0xa6, 0xe4, 0xca, 0xe6, 0x13, 0xfa, 0xe9, 0x8d, 0x0c, 0x4c, 0x6e, 0x3c, 0x29, 0xaf, - 0x6c, 0x88, 0x82, 0xf4, 0x67, 0x02, 0xcc, 0x94, 0xd8, 0xb7, 0x64, 0x98, 0x5f, 0xcf, 0x07, 0x7d, - 0xfe, 0x37, 0x57, 0xa2, 0xb7, 0x10, 0x86, 0xf9, 0xdd, 0xa4, 0xf9, 0xec, 0xe5, 0xc5, 0x9c, 0x2d, - 0x0d, 0x07, 0x8e, 0x74, 0x8f, 0xd5, 0x73, 0xf4, 0xf5, 0xd8, 0x50, 0x1d, 0x44, 0xd5, 0xd3, 0xad, - 0xc2, 0x4b, 0x32, 0xdc, 0xdd, 0xb7, 0xaf, 0x19, 0xbc, 0xb9, 0x48, 0xed, 0x25, 0xbf, 0x9f, 0xfa, - 0xfc, 0x8b, 0xa5, 0x09, 0xe9, 0xcf, 0x53, 0x90, 0x2b, 0x85, 0xbf, 0x9b, 0x83, 0x6a, 0x7d, 0x95, - 0xbd, 0x1d, 0x69, 0x96, 0x84, 0x38, 0x8a, 0x23, 0xbe, 0x48, 0x96, 0x09, 0x3e, 0xd2, 0xc3, 0xea, - 0x7e, 0x6d, 0x84, 0xeb, 0x52, 0xb8, 0xf2, 0x01, 0xe3, 0xc2, 0xbf, 0x49, 0xfa, 0xf6, 0x4a, 0x11, - 0x26, 0x59, 0x4c, 0xb5, 0x30, 0x70, 0xdd, 0x0b, 0x5d, 0xa9, 0x08, 0x5c, 0x23, 0xe9, 0x32, 0xcb, - 0x46, 0xec, 0x9b, 0xc6, 0x2b, 0xf9, 0x3c, 0x05, 0xf3, 0xe0, 0xf9, 0xbf, 0xf5, 0xdb, 0x61, 0x97, - 0x1b, 0xff, 0x3f, 0xf4, 0x30, 0x26, 0xef, 0x43, 0xbf, 0x0e, 0x05, 0xcd, 0x6a, 0x36, 0x99, 0x95, - 0xcf, 0x56, 0xb6, 0xc1, 0xeb, 0xce, 0x68, 0x11, 0xf8, 0xe7, 0x9d, 0x8b, 0xfe, 0x67, 0x9e, 0x8b, - 0x32, 0xff, 0xcc, 0x73, 0x28, 0xfc, 0x2b, 0xef, 0x0b, 0x63, 0x0b, 0x62, 0x5f, 0x24, 0xda, 0xf4, - 0xab, 0x44, 0xa2, 0xb1, 0xf8, 0x3d, 0xde, 0xf3, 0x7e, 0x2e, 0x70, 0x3f, 0xe0, 0x0d, 0xcb, 0x3a, - 0xec, 0xf8, 0xde, 0x95, 0x0b, 0xe1, 0xab, 0x8a, 0x83, 0x20, 0x19, 0x1a, 0x33, 0x1a, 0x65, 0x4a, - 0x25, 0xbe, 0x9e, 0x29, 0x75, 0x1d, 0x66, 0xda, 0x36, 0xde, 0xc3, 0xae, 0x76, 0xa0, 0x98, 0x9d, - 0x16, 0x0f, 0x98, 0xcd, 0x7a, 0xb4, 0xad, 0x4e, 0x0b, 0xdd, 0x05, 0xd1, 0xcf, 0xc2, 0x77, 0x5e, - 0xbc, 0x7b, 0x32, 0x3d, 0x3a, 0xdf, 0xa7, 0x91, 0xfe, 0xbb, 0x00, 0x73, 0x3d, 0x75, 0xe2, 0x63, - 0x6a, 0x0d, 0xb2, 0xba, 0x6f, 0xbc, 0x3a, 0xf3, 0xc2, 0x39, 0xc3, 0xa7, 0xc2, 0xcc, 0x48, 0x81, - 0x4b, 0xde, 0x6b, 0xe9, 0x87, 0x6d, 0x02, 0xb1, 0x89, 0x73, 0x8a, 0xbd, 0x18, 0xc8, 0x59, 0x0d, - 0xbd, 0xc0, 0x1f, 0x64, 0xc9, 0xb1, 0x06, 0x99, 0xf4, 0x3f, 0x04, 0x10, 0xe9, 0x0b, 0x1e, 0x61, - 0xac, 0xc7, 0x32, 0x65, 0x7a, 0x71, 0x8a, 0x89, 0xf1, 0x43, 0x62, 0x7b, 0x3e, 0xc6, 0x95, 0xec, - 0xfb, 0x18, 0x57, 0xd4, 0xfc, 0x99, 0xfa, 0x9a, 0xf3, 0xa7, 0xf4, 0x85, 0x00, 0x79, 0xbf, 0xda, - 0xec, 0x2b, 0xbc, 0x23, 0xae, 0xd9, 0x7e, 0xb5, 0x2f, 0xcd, 0x7a, 0xd7, 0x81, 0x8d, 0xf5, 0x61, - 0xe0, 0xf0, 0x75, 0x60, 0xec, 0x0b, 0xa9, 0x7f, 0xdb, 0xeb, 0x8e, 0xa4, 0x88, 0xe5, 0xe0, 0x1e, - 0xa6, 0x57, 0x08, 0x39, 0xfe, 0x26, 0x1c, 0x89, 0x1f, 0x85, 0x14, 0x48, 0x7b, 0x14, 0xd1, 0xd2, - 0x58, 0xf3, 0xbb, 0xa7, 0x25, 0xd6, 0x01, 0xff, 0x28, 0xdc, 0x12, 0xec, 0x1e, 0x8f, 0x87, 0x90, - 0x3c, 0x52, 0x9b, 0xa3, 0xdc, 0x65, 0x7b, 0x5a, 0x4e, 0x26, 0xb9, 0xd1, 0xa3, 0x9e, 0xeb, 0xab, - 0x12, 0xc3, 0x77, 0x65, 0x07, 0x55, 0xda, 0x73, 0xcd, 0xd5, 0xbb, 0xbd, 0x03, 0x68, 0xe4, 0xeb, - 0xc3, 0x23, 0xe9, 0xfd, 0xd4, 0x97, 0x5f, 0x2c, 0x09, 0xd2, 0x87, 0x80, 0x88, 0xad, 0xe3, 0x3e, - 0xed, 0x58, 0x76, 0x70, 0x15, 0x58, 0x7f, 0x3c, 0xe2, 0x64, 0x74, 0x3c, 0xa2, 0x74, 0x11, 0xe6, - 0x7a, 0xb8, 0xd9, 0x0c, 0x24, 0xbd, 0x0b, 0x57, 0x1e, 0x5b, 0x8e, 0x63, 0xb4, 0xeb, 0x9d, 0x5d, - 0x36, 0xd4, 0xc9, 0x7a, 0xe5, 0xcf, 0xb9, 0xe9, 0x36, 0xdd, 0xf2, 0x33, 0xd9, 0xdc, 0x94, 0x91, - 0xfd, 0x67, 0xe9, 0x0f, 0x04, 0xb8, 0x3c, 0xc8, 0xc9, 0xb4, 0x1c, 0x75, 0x43, 0xc2, 0xb4, 0x66, - 0x05, 0x37, 0xd5, 0x9e, 0xdd, 0x5b, 0xbd, 0xec, 0x04, 0xd5, 0xf0, 0x77, 0x2a, 0x2d, 0x95, 0xce, - 0x49, 0xfc, 0xe2, 0x94, 0x3c, 0x27, 0x6f, 0x32, 0x6a, 0x30, 0x3d, 0xa5, 0xc6, 0x9b, 0x9e, 0xfe, - 0xa7, 0x00, 0xb3, 0x0d, 0x6c, 0xaa, 0xa6, 0x4b, 0xe6, 0xfd, 0x4e, 0x8b, 0xdd, 0x2a, 0x51, 0x80, - 0xa4, 0xad, 0x74, 0x68, 0xd1, 0x05, 0x39, 0x61, 0xef, 0xa0, 0x1b, 0x90, 0xa3, 0x6b, 0x4b, 0xc8, - 0x56, 0x13, 0xee, 0xa4, 0x64, 0x1a, 0x5d, 0x23, 0x7b, 0x86, 0xd8, 0xeb, 0x00, 0x34, 0x13, 0x43, - 0x97, 0x49, 0x9a, 0x23, 0x43, 0x28, 0x3e, 0xb6, 0xa4, 0xd1, 0x8a, 0x81, 0x10, 0x16, 0x90, 0x94, - 0xa3, 0x54, 0x5f, 0xca, 0x12, 0x64, 0x59, 0x36, 0x26, 0x66, 0x92, 0xe6, 0x01, 0x4a, 0x62, 0x72, - 0x1e, 0xc1, 0x05, 0xe7, 0x45, 0x53, 0x69, 0x5b, 0xba, 0xa3, 0x68, 0xed, 0x0e, 0x8f, 0xf4, 0x60, - 0x1f, 0xf6, 0x16, 0x4a, 0x17, 0x4f, 0x4f, 0x96, 0x66, 0xeb, 0x4f, 0x37, 0xb6, 0x2d, 0xdd, 0x29, - 0x6f, 0xef, 0xb0, 0x38, 0x0f, 0x47, 0x9e, 0x75, 0x5e, 0x34, 0x29, 0xa9, 0xdd, 0xe1, 0x24, 0xe9, - 0xb3, 0x04, 0x20, 0x7a, 0x33, 0x52, 0x89, 0x5e, 0x2d, 0xe4, 0x35, 0xb7, 0x05, 0x8b, 0x5a, 0xa0, - 0x0a, 0xc5, 0x31, 0x4c, 0x62, 0xdf, 0xab, 0x8e, 0xeb, 0x95, 0x9b, 0x8f, 0x9b, 0xc8, 0x0b, 0x5d, - 0xfb, 0x35, 0xc9, 0xdb, 0xf1, 0x6a, 0x48, 0x62, 0x9d, 0x08, 0xdc, 0x50, 0x1d, 0xff, 0x85, 0x77, - 0x21, 0xe3, 0x52, 0x3e, 0xef, 0xde, 0xa9, 0x54, 0x69, 0xe6, 0xf4, 0x64, 0x29, 0xcd, 0x84, 0xd5, - 0x56, 0xe5, 0x34, 0x4b, 0xae, 0xe9, 0x68, 0x19, 0xb2, 0x86, 0xe9, 0xb8, 0x2a, 0x29, 0x12, 0xdf, - 0xcc, 0xc9, 0xb1, 0x8b, 0x01, 0x6a, 0x9c, 0x5c, 0x5b, 0x95, 0xc1, 0xcb, 0x42, 0xe3, 0x6f, 0xf3, - 0x3e, 0x03, 0x3b, 0x9d, 0xa1, 0xc1, 0xe2, 0x72, 0xce, 0xa3, 0xb2, 0x9b, 0x11, 0xeb, 0x30, 0xd7, - 0xa3, 0x09, 0xbe, 0x30, 0x7f, 0xd8, 0x3b, 0xe1, 0x84, 0xad, 0x53, 0x4a, 0x77, 0xe8, 0x81, 0xb3, - 0x66, 0xe9, 0x7c, 0xa4, 0xf6, 0x4e, 0x3c, 0x0d, 0x28, 0xac, 0x59, 0x86, 0x49, 0x70, 0xb9, 0x57, - 0xd5, 0x15, 0xc8, 0xef, 0x1a, 0xa6, 0x6a, 0x77, 0x15, 0x2f, 0xb0, 0x45, 0x38, 0x2b, 0xb0, 0x45, - 0xce, 0x31, 0x0e, 0xfe, 0x28, 0xfd, 0x4c, 0x00, 0x31, 0x10, 0xcb, 0x0b, 0xfa, 0x1d, 0x00, 0xad, - 0xd9, 0x71, 0x5c, 0x6c, 0x7b, 0x13, 0xc0, 0x0c, 0x0b, 0xa0, 0x2d, 0x33, 0x6a, 0x6d, 0x55, 0xce, - 0xf0, 0x0c, 0x35, 0x1d, 0xdd, 0xe8, 0xbd, 0xe6, 0x6b, 0xb2, 0x04, 0xa7, 0x03, 0x97, 0x7b, 0x91, - 0x19, 0xc5, 0x71, 0x2d, 0xdb, 0xd7, 0x33, 0x9f, 0x51, 0xbc, 0x0b, 0x10, 0xe9, 0x45, 0x3f, 0x98, - 0xc6, 0xc8, 0xe7, 0x89, 0x79, 0x7b, 0x84, 0xfd, 0x2a, 0xa5, 0xce, 0xae, 0x12, 0xe3, 0xf0, 0xaa, - 0xf4, 0xfb, 0x02, 0x14, 0xca, 0x6c, 0xa0, 0xfb, 0x93, 0xc7, 0x88, 0xc5, 0x72, 0x15, 0xd2, 0xee, - 0xb1, 0xa9, 0xb4, 0xb0, 0xff, 0xcd, 0xbe, 0x73, 0xdc, 0x50, 0x3c, 0xed, 0xb2, 0x47, 0xfa, 0x19, - 0x68, 0xbe, 0xc5, 0xc4, 0x67, 0xe2, 0xb1, 0xf6, 0xa0, 0x7c, 0x26, 0x66, 0xa7, 0xde, 0xab, 0x93, - 0x09, 0x75, 0xc0, 0x92, 0x44, 0x79, 0x80, 0xd0, 0xc7, 0x18, 0x27, 0xe8, 0x27, 0x1a, 0x2b, 0x2b, - 0xab, 0xca, 0xce, 0x56, 0xf9, 0xc9, 0xe6, 0x66, 0xad, 0xd1, 0xa8, 0xac, 0x8a, 0x02, 0x12, 0x61, - 0xa6, 0xe7, 0x53, 0x8e, 0x89, 0x85, 0xd4, 0x67, 0x3f, 0x5d, 0x9c, 0xb8, 0xf7, 0x26, 0xe4, 0x7a, - 0x36, 0xe6, 0x50, 0x01, 0xb2, 0x1b, 0x95, 0x95, 0x7a, 0xa5, 0xfa, 0x64, 0x63, 0x95, 0x02, 0xd9, - 0x2c, 0x4c, 0x6f, 0x55, 0x56, 0xe4, 0x4a, 0xbd, 0x21, 0x0a, 0xf7, 0xfe, 0x02, 0x40, 0xf0, 0x11, - 0x59, 0xf2, 0xea, 0xf5, 0xca, 0xc7, 0xca, 0xb3, 0x95, 0x8d, 0x9d, 0x4a, 0x5d, 0x9c, 0x40, 0x08, - 0xf2, 0xa5, 0x95, 0x46, 0xb9, 0xaa, 0xc8, 0x95, 0xfa, 0xf6, 0x93, 0xad, 0x7a, 0x45, 0x14, 0xf8, - 0x6b, 0x56, 0x61, 0x26, 0x7c, 0x4d, 0x23, 0x9a, 0x83, 0x42, 0xb9, 0x5a, 0x29, 0xaf, 0x2b, 0xcf, - 0x6a, 0x2b, 0xca, 0xd3, 0x9d, 0xca, 0x0e, 0x41, 0xbe, 0xa4, 0x26, 0x94, 0xf8, 0x68, 0x67, 0x83, - 0x80, 0xe6, 0x02, 0x64, 0xd9, 0x33, 0xfd, 0x4a, 0xa4, 0x98, 0xb8, 0xb7, 0x09, 0xd9, 0xd0, 0xe7, - 0x23, 0xc8, 0xeb, 0xb6, 0x77, 0xea, 0x55, 0xa5, 0x51, 0xdb, 0xac, 0xd4, 0x1b, 0x2b, 0x9b, 0xdb, - 0x4c, 0x06, 0xa5, 0xad, 0x94, 0x9e, 0xc8, 0x0d, 0x51, 0xf0, 0x9f, 0x1b, 0x4f, 0x76, 0xca, 0x55, - 0xaf, 0xd6, 0x52, 0x2a, 0x9d, 0x14, 0x93, 0xf7, 0x8e, 0xe1, 0xf2, 0x90, 0x1b, 0x0b, 0x49, 0xa5, - 0x77, 0x4c, 0x7a, 0x95, 0xbe, 0x38, 0x81, 0x72, 0x90, 0x21, 0x3d, 0x95, 0x5e, 0x02, 0x22, 0x0a, - 0x28, 0x0d, 0xa9, 0x03, 0xd7, 0x6d, 0x8b, 0x09, 0x34, 0x05, 0x09, 0xe7, 0xa1, 0x98, 0x24, 0xff, - 0xf7, 0x1d, 0x31, 0x45, 0xa0, 0xbb, 0xfa, 0x69, 0xc7, 0xc6, 0xe2, 0x24, 0x81, 0xff, 0x1d, 0x07, - 0xdb, 0x7b, 0x46, 0x13, 0x8b, 0xd3, 0x84, 0xc5, 0xec, 0x34, 0x9b, 0x62, 0x5a, 0x4a, 0xa5, 0xa7, - 0xc4, 0xa9, 0x7b, 0xd7, 0x21, 0x74, 0x49, 0x14, 0x41, 0xff, 0x1b, 0xaa, 0x8b, 0x1d, 0x57, 0x9c, - 0x40, 0xd3, 0x90, 0x5c, 0x69, 0x36, 0x45, 0xe1, 0xc1, 0xe7, 0x93, 0x90, 0xf6, 0x3e, 0x82, 0x88, - 0x36, 0x60, 0x92, 0x22, 0x5d, 0xb4, 0x34, 0x1c, 0x03, 0xd3, 0x61, 0xbf, 0x70, 0xed, 0x2c, 0x90, - 0x2c, 0x4d, 0xa0, 0xbf, 0x08, 0xd9, 0x10, 0x36, 0x40, 0x43, 0x0f, 0x76, 0x7a, 0xf0, 0xd0, 0xc2, - 0xad, 0xb3, 0xb2, 0xf9, 0xf2, 0x9f, 0x43, 0xc6, 0x37, 0x2b, 0xd0, 0x8d, 0x51, 0x46, 0x87, 0x27, - 0x7b, 0xb4, 0x65, 0x42, 0x46, 0xa9, 0x34, 0xf1, 0x96, 0x80, 0x6c, 0x40, 0x83, 0x16, 0x00, 0x8a, - 0xf2, 0xf4, 0x1e, 0x6a, 0x62, 0x2c, 0xdc, 0x1b, 0x2b, 0x77, 0xf0, 0x4e, 0xa2, 0xac, 0xc0, 0x8c, - 0x89, 0x56, 0xd6, 0x80, 0x91, 0x14, 0xad, 0xac, 0x08, 0x6b, 0x88, 0x36, 0x46, 0x68, 0x3d, 0x88, - 0x94, 0x3f, 0xb8, 0x72, 0x46, 0xca, 0x8f, 0x58, 0x56, 0xa4, 0x09, 0xf4, 0x14, 0x52, 0x64, 0x0e, - 0x47, 0x51, 0x00, 0xa3, 0x6f, 0xcd, 0x58, 0xb8, 0x31, 0x32, 0x8f, 0x27, 0xb2, 0x74, 0xf7, 0xcb, - 0xff, 0xb0, 0x38, 0xf1, 0xe5, 0xe9, 0xa2, 0xf0, 0xb3, 0xd3, 0x45, 0xe1, 0x4f, 0x4e, 0x17, 0x85, - 0x3f, 0x3d, 0x5d, 0x14, 0x7e, 0xfc, 0x8b, 0xc5, 0x89, 0x9f, 0xfd, 0x62, 0x71, 0xe2, 0x4f, 0x7e, - 0xb1, 0x38, 0xf1, 0xc9, 0x34, 0xe7, 0xde, 0x9d, 0xa2, 0x13, 0xdc, 0xc3, 0xff, 0x1b, 0x00, 0x00, - 0xff, 0xff, 0xac, 0x93, 0x65, 0xde, 0x03, 0x8b, 0x00, 0x00, + 0x8c, 0x60, 0x91, 0x9d, 0x79, 0x5b, 0x04, 0x88, 0xb2, 0xd1, 0xe4, 0x21, 0x8b, 0x20, 0x48, 0x90, + 0x04, 0xd8, 0xc0, 0x40, 0x7e, 0x70, 0x7f, 0xea, 0x87, 0x64, 0x91, 0xa2, 0xda, 0xe5, 0xac, 0x81, + 0x7d, 0x91, 0x58, 0xa7, 0xee, 0x39, 0xf7, 0xde, 0x73, 0xff, 0xce, 0x77, 0xef, 0x39, 0xb7, 0x60, + 0xd6, 0xb6, 0x54, 0xed, 0xa0, 0xbd, 0xbb, 0xac, 0xb6, 0x8d, 0x62, 0xdb, 0xb6, 0x5c, 0x0b, 0xcd, + 0x6a, 0x96, 0x76, 0x48, 0xc9, 0x45, 0xfe, 0x72, 0xe1, 0x22, 0xb6, 0x6d, 0xcb, 0x76, 0xda, 0xbb, + 0xcb, 0xec, 0x07, 0x4b, 0xb9, 0x70, 0xef, 0xf0, 0x68, 0xf9, 0xf0, 0xc8, 0xc1, 0xf6, 0x11, 0xb6, + 0x97, 0x35, 0xcb, 0xd4, 0x3a, 0xb6, 0x8d, 0x4d, 0xad, 0xbb, 0xdc, 0xb4, 0xb4, 0x43, 0xfa, 0xc7, + 0x30, 0xf7, 0xa3, 0xd2, 0xda, 0x58, 0xd5, 0x9d, 0x4e, 0xab, 0xa5, 0xda, 0xdd, 0x65, 0x2a, 0x96, + 0x3f, 0xf0, 0xb4, 0xc8, 0x2b, 0x94, 0xae, 0xba, 0x2a, 0xa7, 0x5d, 0xf0, 0x68, 0x3d, 0x25, 0xb8, + 0xe4, 0x51, 0x5b, 0xd8, 0x55, 0x43, 0xa9, 0xaf, 0x78, 0x74, 0xa7, 0xad, 0x9a, 0x8a, 0x66, 0x99, + 0x7b, 0x86, 0x57, 0x90, 0xab, 0x8e, 0x6b, 0xd9, 0xea, 0x3e, 0x5e, 0xc6, 0xe6, 0xbe, 0x61, 0x62, + 0xc2, 0x7b, 0xa4, 0x69, 0xfc, 0xe5, 0x6b, 0x91, 0x2f, 0x1f, 0xf2, 0xb7, 0xf3, 0x1d, 0xd7, 0x68, + 0x2e, 0x1f, 0x34, 0xb5, 0x65, 0xd7, 0x68, 0x61, 0xc7, 0x55, 0x5b, 0x6d, 0xaf, 0x76, 0xf4, 0x8d, + 0x6b, 0xab, 0x9a, 0x61, 0xee, 0x7b, 0xff, 0xdb, 0xbb, 0xcb, 0x36, 0xd6, 0x2c, 0x5b, 0xc7, 0xba, + 0x42, 0xca, 0xe1, 0xd5, 0x64, 0xdf, 0xda, 0xb7, 0xe8, 0xcf, 0x65, 0xf2, 0x8b, 0x53, 0x17, 0xf7, + 0x2d, 0x6b, 0xbf, 0x89, 0x97, 0xe9, 0xd3, 0x6e, 0x67, 0x6f, 0x59, 0xef, 0xd8, 0xaa, 0x6b, 0x58, + 0x9c, 0x4b, 0xfa, 0xa7, 0x02, 0xe4, 0x64, 0xfc, 0xa2, 0x83, 0x1d, 0xb7, 0x8a, 0x55, 0x1d, 0xdb, + 0xe8, 0x0a, 0x24, 0x0f, 0x71, 0x77, 0x3e, 0x79, 0x4d, 0xb8, 0x33, 0x53, 0x9a, 0xfe, 0xea, 0x64, + 0x29, 0xb9, 0x8e, 0xbb, 0x32, 0xa1, 0xa1, 0x6b, 0x30, 0x8d, 0x4d, 0x5d, 0x21, 0xaf, 0x53, 0xbd, + 0xaf, 0xa7, 0xb0, 0xa9, 0xaf, 0xe3, 0x2e, 0xfa, 0x3e, 0xa4, 0x1d, 0x22, 0xcd, 0xd4, 0xf0, 0xfc, + 0xe4, 0x35, 0xe1, 0xce, 0x64, 0xe9, 0x97, 0xbf, 0x3a, 0x59, 0xfa, 0x68, 0xdf, 0x70, 0x0f, 0x3a, + 0xbb, 0x45, 0xcd, 0x6a, 0x2d, 0xfb, 0xbd, 0x40, 0xdf, 0x0d, 0x7e, 0x2f, 0xb7, 0x0f, 0xf7, 0x97, + 0xfb, 0x75, 0x54, 0x6c, 0x1c, 0x9b, 0x75, 0xfc, 0x42, 0xf6, 0x25, 0xae, 0xa5, 0xd2, 0x82, 0x98, + 0x58, 0x4b, 0xa5, 0x13, 0x62, 0x52, 0xfa, 0xc3, 0x04, 0xe4, 0x65, 0xec, 0xb4, 0x2d, 0xd3, 0xc1, + 0xbc, 0xe4, 0x6f, 0x43, 0xd2, 0x3d, 0x36, 0x69, 0xc9, 0xb3, 0x0f, 0x16, 0x8b, 0x03, 0xfd, 0xad, + 0xd8, 0xb0, 0x55, 0xd3, 0x51, 0x35, 0x52, 0x7d, 0x99, 0x24, 0x45, 0xef, 0x43, 0xd6, 0xc6, 0x4e, + 0xa7, 0x85, 0xa9, 0x22, 0x69, 0xa5, 0xb2, 0x0f, 0x2e, 0x47, 0x70, 0xd6, 0xdb, 0xaa, 0x29, 0x03, + 0x4b, 0x4b, 0x7e, 0xa3, 0x3a, 0xe4, 0x38, 0xa7, 0x8d, 0x55, 0xc7, 0x32, 0xe7, 0xa7, 0xaf, 0x09, + 0x77, 0xf2, 0x0f, 0x8a, 0x11, 0xbc, 0xbd, 0xa5, 0x24, 0x8f, 0x9d, 0x16, 0x96, 0x29, 0x97, 0x3c, + 0x63, 0x87, 0x9e, 0xd0, 0x15, 0x48, 0x9b, 0x9d, 0x16, 0xd1, 0xaf, 0x43, 0xb5, 0x97, 0x94, 0xa7, + 0xcd, 0x4e, 0x6b, 0x1d, 0x77, 0x1d, 0x74, 0x15, 0x32, 0xe4, 0xd5, 0x6e, 0xd7, 0xc5, 0xce, 0x7c, + 0x9a, 0xbe, 0x23, 0x69, 0x4b, 0xe4, 0x59, 0xfa, 0x18, 0x66, 0xc2, 0x52, 0x11, 0x82, 0xbc, 0x5c, + 0xa9, 0xef, 0x6c, 0x56, 0x94, 0x9d, 0xad, 0xf5, 0xad, 0x27, 0xcf, 0xb7, 0xc4, 0x09, 0x74, 0x01, + 0x44, 0x4e, 0x5b, 0xaf, 0x7c, 0xa2, 0x6c, 0xd4, 0x36, 0x6b, 0x0d, 0x51, 0x58, 0x48, 0x7d, 0xfe, + 0xd3, 0xc5, 0x89, 0xb5, 0x54, 0x7a, 0x4a, 0x9c, 0x96, 0x7e, 0x2a, 0x00, 0x3c, 0xc6, 0x2e, 0xef, + 0x0d, 0xa8, 0x04, 0x53, 0x07, 0xb4, 0xc4, 0xf3, 0x02, 0x55, 0xcb, 0xb5, 0xc8, 0xaa, 0x85, 0x7a, + 0x4e, 0x29, 0xfd, 0xe5, 0xc9, 0xd2, 0xc4, 0xcf, 0x4e, 0x96, 0x04, 0x99, 0x73, 0xa2, 0xa7, 0x90, + 0x3d, 0xc4, 0x5d, 0x85, 0x0f, 0xd9, 0xf9, 0x04, 0xd5, 0xd1, 0xdb, 0x21, 0x41, 0x87, 0x47, 0x45, + 0x6f, 0xf4, 0x16, 0x43, 0x23, 0xbd, 0x48, 0x38, 0x8a, 0x75, 0xd7, 0xc6, 0xe6, 0xbe, 0x7b, 0x20, + 0xc3, 0x21, 0xee, 0x6e, 0x30, 0x19, 0xd2, 0xef, 0x0b, 0x90, 0xa5, 0xa5, 0x64, 0x4a, 0x45, 0xe5, + 0xbe, 0x62, 0x5e, 0x3f, 0xb3, 0x05, 0x22, 0xca, 0x59, 0x84, 0xc9, 0x23, 0xb5, 0xd9, 0xc1, 0xb4, + 0x84, 0xd9, 0x07, 0xf3, 0x11, 0x32, 0x9e, 0x91, 0xf7, 0x32, 0x4b, 0x86, 0x3e, 0x84, 0x19, 0xc3, + 0x74, 0xb1, 0xe9, 0x2a, 0x8c, 0x2d, 0x79, 0x06, 0x5b, 0x96, 0xa5, 0xa6, 0x0f, 0xd2, 0x3f, 0x11, + 0x00, 0xb6, 0x3b, 0xb1, 0xea, 0xf9, 0x9d, 0x31, 0xcb, 0x5f, 0x4a, 0x11, 0x56, 0xaf, 0x16, 0x97, + 0x60, 0xca, 0x30, 0x9b, 0x86, 0xc9, 0xca, 0x9f, 0x96, 0xf9, 0x13, 0xba, 0x00, 0x93, 0xbb, 0x4d, + 0xc3, 0xd4, 0xe9, 0x78, 0x48, 0xcb, 0xec, 0x41, 0x92, 0x21, 0x4b, 0x4b, 0x1d, 0xa3, 0xde, 0xa5, + 0x93, 0x04, 0x5c, 0x2c, 0x5b, 0xa6, 0x6e, 0x90, 0x21, 0xa9, 0x36, 0xbf, 0x15, 0x5a, 0x59, 0x83, + 0x0b, 0x3a, 0x6e, 0xdb, 0x58, 0x53, 0x5d, 0xac, 0x2b, 0xf8, 0xb8, 0x3d, 0x66, 0x1b, 0xa3, 0x80, + 0xab, 0x72, 0xdc, 0xa6, 0x34, 0x32, 0x6a, 0x89, 0x00, 0x36, 0x6a, 0xa7, 0xc8, 0x94, 0x29, 0xa7, + 0xf1, 0x71, 0x9b, 0x8e, 0xda, 0x68, 0x35, 0xa3, 0x77, 0xe0, 0xb2, 0xda, 0x6c, 0x5a, 0x2f, 0x15, + 0x63, 0x4f, 0xd1, 0x2d, 0xec, 0x28, 0xa6, 0xe5, 0x2a, 0xf8, 0xd8, 0x70, 0x5c, 0x3a, 0x25, 0xa4, + 0xe5, 0x39, 0xfa, 0xba, 0xb6, 0xb7, 0x6a, 0x61, 0x67, 0xcb, 0x72, 0x2b, 0xe4, 0x55, 0xa8, 0x29, + 0xa7, 0xc3, 0x4d, 0x29, 0xfd, 0x2a, 0x5c, 0xea, 0xd7, 0x6f, 0x9c, 0xed, 0xf7, 0x07, 0x02, 0xe4, + 0x6b, 0xa6, 0xe1, 0x7e, 0x2b, 0x1a, 0xce, 0xd7, 0x67, 0x32, 0xac, 0xcf, 0x7b, 0x20, 0xee, 0xa9, + 0x46, 0xf3, 0x89, 0xd9, 0xb0, 0x5a, 0xbb, 0x8e, 0x6b, 0x99, 0xd8, 0xe1, 0x0a, 0x1f, 0xa0, 0x4b, + 0xcf, 0xa0, 0xe0, 0xd7, 0x26, 0x4e, 0x35, 0xb9, 0x20, 0xd6, 0x4c, 0xcd, 0xc6, 0x2d, 0x6c, 0xc6, + 0xaa, 0xa7, 0xd7, 0x20, 0x63, 0x78, 0x72, 0xa9, 0xae, 0x92, 0x72, 0x40, 0x90, 0x3a, 0x30, 0x1b, + 0xca, 0x35, 0xce, 0xe9, 0x92, 0x2c, 0x46, 0xf8, 0xa5, 0x12, 0xb4, 0x11, 0x59, 0x8c, 0xf0, 0x4b, + 0x36, 0xbd, 0xd5, 0x21, 0xb7, 0x8a, 0x9b, 0xd8, 0xc5, 0x31, 0xd6, 0x54, 0xda, 0x81, 0xbc, 0x27, + 0x34, 0xce, 0x86, 0xf9, 0x0d, 0x01, 0x10, 0x97, 0xab, 0x9a, 0xfb, 0x71, 0x96, 0x18, 0x2d, 0x11, + 0xd3, 0xc2, 0xed, 0xd8, 0x26, 0x5b, 0xce, 0x59, 0x9f, 0x04, 0x46, 0xa2, 0x2b, 0x7a, 0x30, 0x64, + 0x53, 0xe1, 0x21, 0xcb, 0xcd, 0x9b, 0x97, 0x30, 0xd7, 0x53, 0xb0, 0x78, 0x9b, 0x2f, 0x45, 0xcb, + 0x94, 0xb8, 0x96, 0x0c, 0xdb, 0x70, 0x94, 0x28, 0xfd, 0x58, 0x80, 0xd9, 0x72, 0x13, 0xab, 0x76, + 0xec, 0x1a, 0xf9, 0x1e, 0xa4, 0x75, 0xac, 0xea, 0xb4, 0xca, 0x6c, 0x60, 0xbf, 0x1e, 0x92, 0x42, + 0x2c, 0xdd, 0xe2, 0x41, 0x53, 0x2b, 0x36, 0x3c, 0x1b, 0x98, 0x8f, 0x6e, 0x9f, 0x49, 0xfa, 0x04, + 0x50, 0xb8, 0x64, 0x71, 0x76, 0x84, 0xdf, 0x49, 0x00, 0x92, 0xf1, 0x11, 0xb6, 0xdd, 0xd8, 0xab, + 0xbd, 0x0a, 0x59, 0x57, 0xb5, 0xf7, 0xb1, 0xab, 0x10, 0xeb, 0xfe, 0x3c, 0x35, 0x07, 0xc6, 0x47, + 0xc8, 0xa8, 0x01, 0xb7, 0xb1, 0xa9, 0xee, 0x36, 0x31, 0x95, 0xa2, 0xec, 0x5a, 0x1d, 0x53, 0x57, + 0x0c, 0x17, 0xdb, 0xaa, 0x6b, 0xd9, 0x8a, 0xd5, 0x76, 0x8d, 0x96, 0xf1, 0x19, 0x35, 0xec, 0x79, + 0x57, 0xbb, 0xc1, 0x92, 0x13, 0xe6, 0x12, 0x49, 0x5c, 0xe3, 0x69, 0x9f, 0x84, 0x92, 0xa2, 0x22, + 0xcc, 0x19, 0xfb, 0xa6, 0x65, 0x63, 0x65, 0x5f, 0x53, 0xdc, 0x03, 0x1b, 0x3b, 0x07, 0x56, 0xd3, + 0x5b, 0x90, 0x66, 0xd9, 0xab, 0xc7, 0x5a, 0xc3, 0x7b, 0x21, 0x7d, 0x0a, 0x73, 0x3d, 0x5a, 0x8a, + 0xb3, 0x09, 0xfe, 0x9b, 0x00, 0xd9, 0xba, 0xa6, 0x9a, 0x71, 0xea, 0xfe, 0x63, 0xc8, 0x3a, 0x9a, + 0x6a, 0x2a, 0x7b, 0x96, 0xdd, 0x52, 0x5d, 0x5a, 0xaf, 0x7c, 0x8f, 0xee, 0x7d, 0xfb, 0x5e, 0x53, + 0xcd, 0x47, 0x34, 0x91, 0x0c, 0x8e, 0xff, 0xbb, 0xdf, 0x7e, 0x9d, 0xfc, 0xfa, 0xf6, 0x2b, 0x1b, + 0xde, 0x6b, 0xa9, 0x74, 0x52, 0x4c, 0x49, 0x7f, 0x2a, 0xc0, 0x0c, 0xab, 0x72, 0x9c, 0xc3, 0xfb, + 0x5d, 0x48, 0xd9, 0xd6, 0x4b, 0x36, 0xbc, 0xb3, 0x0f, 0xae, 0x46, 0x88, 0x58, 0xc7, 0xdd, 0xf0, + 0xfa, 0x49, 0x93, 0xa3, 0x12, 0x70, 0x2b, 0x55, 0xa1, 0xdc, 0xc9, 0x71, 0xb9, 0x81, 0x71, 0xc9, + 0x44, 0xc6, 0x6d, 0x28, 0xec, 0xaa, 0xae, 0x76, 0xa0, 0xd8, 0xbc, 0x90, 0x64, 0xad, 0x4d, 0xde, + 0x99, 0x91, 0xf3, 0x94, 0xec, 0x15, 0xdd, 0x21, 0x35, 0x67, 0xe3, 0xcd, 0xc1, 0x7f, 0xce, 0xda, + 0xfc, 0xff, 0x08, 0x7c, 0x0c, 0x79, 0x35, 0xff, 0xf3, 0xd6, 0xf4, 0x3f, 0x49, 0xc0, 0xe5, 0xf2, + 0x01, 0xd6, 0x0e, 0xcb, 0x96, 0xe9, 0x18, 0x8e, 0x4b, 0x74, 0x17, 0x67, 0xfb, 0x5f, 0x85, 0xcc, + 0x4b, 0xc3, 0x3d, 0x50, 0x74, 0x63, 0x6f, 0x8f, 0xce, 0xb6, 0x69, 0x39, 0x4d, 0x08, 0xab, 0xc6, + 0xde, 0x1e, 0x7a, 0x08, 0xa9, 0x96, 0xa5, 0x33, 0x63, 0x3e, 0xff, 0x60, 0x29, 0x42, 0x3c, 0x2d, + 0x9a, 0xd3, 0x69, 0x6d, 0x5a, 0x3a, 0x96, 0x69, 0x62, 0xb4, 0x08, 0xa0, 0x11, 0x6a, 0xdb, 0x32, + 0x4c, 0x97, 0x4f, 0x8e, 0x21, 0x0a, 0xaa, 0x42, 0xc6, 0xc5, 0x76, 0xcb, 0x30, 0x55, 0x17, 0xcf, + 0x4f, 0x52, 0xe5, 0xbd, 0x11, 0x59, 0xf0, 0x76, 0xd3, 0xd0, 0xd4, 0x55, 0xec, 0x68, 0xb6, 0xd1, + 0x76, 0x2d, 0x9b, 0x6b, 0x31, 0x60, 0x96, 0xfe, 0x7a, 0x0a, 0xe6, 0x07, 0x75, 0x13, 0x67, 0x0f, + 0xd9, 0x86, 0x29, 0x1b, 0x3b, 0x9d, 0xa6, 0xcb, 0xfb, 0xc8, 0x83, 0x61, 0x2a, 0x88, 0x28, 0x01, + 0xdd, 0xba, 0x68, 0xba, 0xbc, 0xd8, 0x5c, 0xce, 0xc2, 0xbf, 0x10, 0x60, 0x8a, 0xbd, 0x40, 0xf7, + 0x21, 0x6d, 0x93, 0x85, 0x41, 0x31, 0x74, 0x5a, 0xc6, 0x64, 0xe9, 0xd2, 0xe9, 0xc9, 0xd2, 0x34, + 0x5d, 0x2c, 0x6a, 0xab, 0x5f, 0x05, 0x3f, 0xe5, 0x69, 0x9a, 0xae, 0xa6, 0x93, 0xd6, 0x72, 0x5c, + 0xd5, 0x76, 0xe9, 0xa6, 0x52, 0x82, 0x21, 0x24, 0x4a, 0x58, 0xc7, 0x5d, 0xb4, 0x06, 0x53, 0x8e, + 0xab, 0xba, 0x1d, 0x87, 0xb7, 0xd7, 0xb9, 0x0a, 0x5b, 0xa7, 0x9c, 0x32, 0x97, 0x40, 0xcc, 0x2d, + 0x1d, 0xbb, 0xaa, 0xd1, 0xa4, 0x0d, 0x98, 0x91, 0xf9, 0x93, 0xf4, 0x9b, 0x02, 0x4c, 0xb1, 0xa4, + 0xe8, 0x32, 0xcc, 0xc9, 0x2b, 0x5b, 0x8f, 0x2b, 0x4a, 0x6d, 0x6b, 0xb5, 0xd2, 0xa8, 0xc8, 0x9b, + 0xb5, 0xad, 0x95, 0x46, 0x45, 0x9c, 0x40, 0x97, 0x00, 0x79, 0x2f, 0xca, 0x4f, 0xb6, 0xea, 0xb5, + 0x7a, 0xa3, 0xb2, 0xd5, 0x10, 0x05, 0xba, 0xa7, 0x42, 0xe9, 0x21, 0x6a, 0x02, 0xbd, 0x01, 0xd7, + 0xfa, 0xa9, 0x4a, 0xbd, 0xb1, 0xd2, 0xa8, 0x2b, 0x95, 0x7a, 0xa3, 0xb6, 0xb9, 0xd2, 0xa8, 0xac, + 0x8a, 0xc9, 0x11, 0xa9, 0x48, 0x26, 0xb2, 0x5c, 0x29, 0x37, 0xc4, 0x94, 0xe4, 0xc2, 0x45, 0x19, + 0x6b, 0x56, 0xab, 0xdd, 0x71, 0x31, 0x29, 0xa5, 0x13, 0xe7, 0x48, 0xb9, 0x0c, 0xd3, 0xba, 0xdd, + 0x55, 0xec, 0x8e, 0xc9, 0xc7, 0xc9, 0x94, 0x6e, 0x77, 0xe5, 0x8e, 0x29, 0xfd, 0x43, 0x01, 0x2e, + 0xf5, 0x67, 0x1b, 0x67, 0x27, 0x7c, 0x0a, 0x59, 0x55, 0xd7, 0xb1, 0xae, 0xe8, 0xb8, 0xe9, 0xaa, + 0xdc, 0x24, 0xba, 0x17, 0x92, 0xc4, 0xb7, 0x02, 0x8b, 0xfe, 0x56, 0xe0, 0xe6, 0xb3, 0x72, 0x99, + 0x16, 0x64, 0x95, 0x70, 0x78, 0xd3, 0x0f, 0x15, 0x42, 0x29, 0xd2, 0x4f, 0x52, 0x90, 0xab, 0x98, + 0x7a, 0xe3, 0x38, 0xd6, 0xb5, 0xe4, 0x12, 0x4c, 0x69, 0x56, 0xab, 0x65, 0xb8, 0x9e, 0x82, 0xd8, + 0x13, 0xfa, 0x4e, 0xc8, 0x94, 0x4d, 0x8e, 0x61, 0xd0, 0x05, 0x46, 0x2c, 0xfa, 0x35, 0xb8, 0x4c, + 0x66, 0x4d, 0xdb, 0x54, 0x9b, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xb1, 0xbf, 0x8f, 0x6d, 0xbe, 0xfd, + 0x78, 0x27, 0xa2, 0x9c, 0x35, 0xce, 0x51, 0xa6, 0x0c, 0x0d, 0x96, 0x5e, 0xbe, 0x68, 0x44, 0x91, + 0xd1, 0x47, 0x00, 0x64, 0x29, 0xa2, 0x5b, 0x9a, 0x0e, 0x9f, 0x8f, 0x86, 0xed, 0x69, 0x7a, 0x53, + 0x10, 0x61, 0x20, 0xcf, 0x0e, 0x7a, 0x0a, 0xa2, 0x61, 0x2a, 0x7b, 0x4d, 0x63, 0xff, 0xc0, 0x55, + 0x5e, 0xda, 0x86, 0x8b, 0x9d, 0xf9, 0x59, 0x2a, 0x23, 0xaa, 0xa9, 0xeb, 0x7c, 0x6b, 0x56, 0x7f, + 0x4e, 0x52, 0x72, 0x69, 0x79, 0xc3, 0x7c, 0x44, 0xf9, 0x29, 0xd1, 0x41, 0xcb, 0x04, 0x0a, 0xbd, + 0xe8, 0x18, 0x36, 0x56, 0xee, 0xb7, 0x35, 0xba, 0x0f, 0x92, 0x2e, 0xe5, 0x4f, 0x4f, 0x96, 0x40, + 0x66, 0xe4, 0xfb, 0xdb, 0x65, 0x02, 0x8d, 0xd8, 0xef, 0xb6, 0x46, 0xd4, 0xde, 0xb6, 0x0c, 0xc7, + 0x32, 0xe7, 0x33, 0x4c, 0xed, 0xec, 0x09, 0xdd, 0x05, 0xd1, 0x3d, 0x36, 0x95, 0x03, 0xac, 0xda, + 0xee, 0x2e, 0x56, 0x5d, 0xb2, 0x3e, 0x03, 0x4d, 0x51, 0x70, 0x8f, 0xcd, 0x6a, 0x88, 0xbc, 0x96, + 0x4a, 0x4f, 0x8b, 0xe9, 0xb5, 0x54, 0x3a, 0x2d, 0x66, 0xa4, 0x7f, 0x27, 0x40, 0xde, 0xeb, 0x1b, + 0x71, 0x76, 0xe3, 0x3b, 0x20, 0x5a, 0x26, 0x56, 0xda, 0x07, 0xaa, 0x83, 0x79, 0x5b, 0xf2, 0xd5, + 0x21, 0x6f, 0x99, 0x78, 0x9b, 0x90, 0x59, 0xcb, 0xa0, 0x6d, 0x98, 0x75, 0x5c, 0x75, 0xdf, 0x30, + 0xf7, 0x15, 0x7f, 0x8b, 0x9f, 0x5a, 0x16, 0x63, 0x22, 0x01, 0x91, 0x73, 0xfb, 0xf4, 0x1e, 0x93, + 0xe2, 0xe7, 0x02, 0xcc, 0xae, 0xe8, 0x2d, 0xc3, 0xac, 0xb7, 0x9b, 0x46, 0xac, 0x1b, 0x0c, 0x6f, + 0x40, 0xc6, 0x21, 0x32, 0x83, 0xd9, 0x39, 0x80, 0x8b, 0x69, 0xfa, 0x86, 0x4c, 0xd3, 0x1b, 0x50, + 0xc0, 0xc7, 0x6d, 0x83, 0x9d, 0x2b, 0x30, 0x94, 0x93, 0x1a, 0xbf, 0x6e, 0xf9, 0x80, 0x97, 0xbc, + 0xe2, 0x75, 0xfa, 0x04, 0x50, 0xb8, 0x4a, 0x71, 0x02, 0x8d, 0x4f, 0x60, 0x8e, 0x8a, 0xde, 0x31, + 0x9d, 0x98, 0xf5, 0x25, 0xfd, 0x0a, 0x5c, 0xe8, 0x15, 0x1d, 0x67, 0xb9, 0x9f, 0xf3, 0x56, 0xde, + 0xc4, 0x76, 0xac, 0x08, 0xd5, 0xd7, 0x35, 0x17, 0x1c, 0x67, 0x99, 0x7f, 0x5d, 0x80, 0x2b, 0x54, + 0x36, 0x3d, 0x7a, 0xd9, 0xc3, 0xf6, 0x06, 0x56, 0x9d, 0x58, 0xe1, 0xf5, 0x0d, 0x98, 0x62, 0x30, + 0x99, 0xf6, 0xcf, 0xc9, 0x52, 0x96, 0x98, 0x19, 0x75, 0xd7, 0xb2, 0x89, 0x99, 0xc1, 0x5f, 0x49, + 0x2a, 0x2c, 0x44, 0x95, 0x22, 0xce, 0x9a, 0xfe, 0x96, 0x00, 0xb3, 0xdc, 0xc2, 0x23, 0x5d, 0xb9, + 0x7c, 0x40, 0x0c, 0x1c, 0x54, 0x81, 0xac, 0x46, 0x7f, 0x29, 0x6e, 0xb7, 0x8d, 0xa9, 0xfc, 0xfc, + 0x28, 0xe3, 0x90, 0xb1, 0x35, 0xba, 0x6d, 0x4c, 0x2c, 0x4c, 0xef, 0x37, 0x51, 0x54, 0xa8, 0x92, + 0x23, 0xcd, 0x4b, 0x3a, 0x8e, 0x68, 0x5a, 0xcf, 0x4e, 0xe3, 0x3a, 0xf8, 0xc7, 0x49, 0xae, 0x04, + 0x96, 0x07, 0x4f, 0x1e, 0xab, 0x41, 0xf1, 0x29, 0x5c, 0x0a, 0x6d, 0x9d, 0x87, 0x2b, 0x9e, 0x38, + 0x47, 0xc5, 0x43, 0xdb, 0xef, 0x01, 0x15, 0x7d, 0x02, 0xa1, 0x0d, 0x76, 0x85, 0xd5, 0xc9, 0x83, + 0x2a, 0xe7, 0x51, 0xc7, 0x6c, 0x20, 0x85, 0xd1, 0x1d, 0x54, 0x86, 0x34, 0x3e, 0x6e, 0x2b, 0x3a, + 0x76, 0x34, 0x3e, 0x71, 0x49, 0x51, 0x02, 0x49, 0x51, 0x06, 0x8c, 0xf7, 0x69, 0x7c, 0xdc, 0x26, + 0x44, 0xb4, 0x43, 0xd6, 0x4d, 0x6f, 0x5d, 0xa7, 0xc5, 0x76, 0xce, 0xc6, 0x02, 0x41, 0x4f, 0xe1, + 0xe2, 0x0a, 0xfe, 0x92, 0xce, 0x44, 0x48, 0x5f, 0x08, 0x70, 0x35, 0xb2, 0xd5, 0xe2, 0x5c, 0xc8, + 0x3e, 0x82, 0x14, 0xad, 0x7c, 0xe2, 0x9c, 0x95, 0xa7, 0x5c, 0xd2, 0xe7, 0x09, 0x3e, 0xc6, 0x65, + 0xdc, 0xb4, 0x88, 0x62, 0x63, 0xdf, 0x42, 0x7b, 0x02, 0xb9, 0x23, 0xcb, 0xc5, 0xb6, 0xdf, 0xec, + 0x89, 0x73, 0x37, 0xfb, 0x0c, 0x15, 0xe0, 0xb5, 0xf8, 0x33, 0x98, 0x35, 0x2d, 0x53, 0xe9, 0x15, + 0x7a, 0xfe, 0xbe, 0x54, 0x30, 0x2d, 0xf3, 0x59, 0x48, 0xae, 0x3f, 0xcf, 0xf4, 0x69, 0x22, 0xce, + 0x79, 0xe6, 0x47, 0x02, 0xcc, 0xf9, 0x96, 0x4e, 0xcc, 0xe6, 0xee, 0xbb, 0x90, 0x34, 0xad, 0x97, + 0xe7, 0xd9, 0xa2, 0x24, 0xe9, 0xc9, 0xaa, 0xd7, 0x5b, 0xa2, 0x38, 0xeb, 0xfb, 0x2f, 0x13, 0x90, + 0x79, 0x5c, 0x8e, 0xb3, 0x96, 0x1f, 0xf1, 0xed, 0x6f, 0xd6, 0xde, 0x51, 0xbd, 0xdd, 0xcf, 0xaf, + 0xf8, 0xb8, 0xbc, 0x8e, 0xbb, 0x5e, 0x6f, 0x27, 0x5c, 0x68, 0x05, 0x32, 0xbd, 0x1b, 0xa5, 0x63, + 0x6a, 0x2a, 0xe0, 0x5a, 0xc0, 0x30, 0x49, 0xe5, 0x7a, 0xae, 0x16, 0x42, 0x84, 0xab, 0x05, 0xc9, + 0xc6, 0xb7, 0x14, 0x13, 0xe7, 0xc9, 0x26, 0x64, 0x22, 0x4e, 0x8a, 0x53, 0xd2, 0x53, 0x00, 0x52, + 0x9d, 0x38, 0x9b, 0xe4, 0x87, 0x49, 0xc8, 0x6f, 0x77, 0x9c, 0x83, 0x98, 0x7b, 0x5f, 0x19, 0xa0, + 0xdd, 0x71, 0x0e, 0xc8, 0x88, 0x3c, 0x36, 0x79, 0x9d, 0xcf, 0xf0, 0xe2, 0xf0, 0x2a, 0xcd, 0xf8, + 0x1a, 0xc7, 0x26, 0xaa, 0x72, 0x21, 0x58, 0x09, 0x5c, 0x41, 0x6e, 0x8c, 0x42, 0x96, 0x8d, 0x63, + 0x73, 0x13, 0xfb, 0x90, 0x92, 0x49, 0xc2, 0x44, 0xd2, 0x47, 0x30, 0x4d, 0x1e, 0x14, 0xd7, 0x3a, + 0x4f, 0x33, 0x4f, 0x11, 0x9e, 0x86, 0x85, 0x3e, 0x84, 0x0c, 0xe3, 0x26, 0xab, 0xdf, 0x14, 0x5d, + 0xfd, 0xa2, 0xea, 0xc2, 0xd5, 0x48, 0xd7, 0xbd, 0x34, 0x65, 0x25, 0x6b, 0xdd, 0x05, 0x98, 0xdc, + 0xb3, 0x6c, 0xcd, 0x3b, 0xcc, 0x65, 0x0f, 0xac, 0x3d, 0x19, 0xa4, 0x59, 0x4b, 0xa5, 0x33, 0x22, + 0x48, 0xbf, 0x29, 0x40, 0xc1, 0x6f, 0x88, 0x38, 0x17, 0x84, 0x72, 0x8f, 0x16, 0xcf, 0xdf, 0x14, + 0x44, 0x81, 0xd2, 0xbf, 0xa2, 0x16, 0x91, 0x66, 0x1d, 0xd1, 0x96, 0x89, 0xb3, 0xa7, 0x7c, 0xc8, + 0x1c, 0x7d, 0x12, 0xe7, 0x6d, 0x5d, 0xea, 0xf3, 0x73, 0x1f, 0x2e, 0x18, 0x2d, 0x32, 0x9f, 0x1b, + 0x6e, 0xb3, 0xcb, 0x61, 0x9b, 0x8b, 0xbd, 0x53, 0xe3, 0xb9, 0xe0, 0x5d, 0xd9, 0x7b, 0x25, 0xfd, + 0x0e, 0xdd, 0xad, 0x0e, 0x6a, 0x12, 0xa7, 0xaa, 0x6b, 0x90, 0xb3, 0x99, 0x68, 0x62, 0xd6, 0x9c, + 0x53, 0xdb, 0x33, 0x3e, 0x2b, 0x51, 0xf8, 0x6f, 0x27, 0xa0, 0xf0, 0xb4, 0x83, 0xed, 0xee, 0xb7, + 0x49, 0xdd, 0xb7, 0xa0, 0xf0, 0x52, 0x35, 0x5c, 0x65, 0xcf, 0xb2, 0x95, 0x4e, 0x5b, 0x57, 0x5d, + 0xcf, 0xdb, 0x24, 0x47, 0xc8, 0x8f, 0x2c, 0x7b, 0x87, 0x12, 0x11, 0x06, 0x74, 0x68, 0x5a, 0x2f, + 0x4d, 0x85, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf9, 0x16, 0x72, 0xe9, 0xbd, 0x7f, 0x7b, 0xb2, 0xf4, + 0x70, 0x2c, 0x1f, 0x32, 0xea, 0x2f, 0xd7, 0xe9, 0x18, 0x7a, 0x71, 0x67, 0xa7, 0xb6, 0x2a, 0x8b, + 0x54, 0xe4, 0x73, 0x26, 0xb1, 0x71, 0x6c, 0x3a, 0xd2, 0xdf, 0x4d, 0x80, 0x18, 0xe8, 0x28, 0xce, + 0x86, 0xac, 0x40, 0xf6, 0x45, 0x07, 0xdb, 0xc6, 0x2b, 0x34, 0x23, 0x70, 0x46, 0x32, 0xed, 0xdc, + 0x83, 0x59, 0xf7, 0xd8, 0x54, 0x98, 0x87, 0x1f, 0x73, 0xfc, 0xf0, 0x1c, 0x16, 0x0a, 0x2e, 0x29, + 0x33, 0xa1, 0x53, 0xa7, 0x0f, 0x07, 0x7d, 0x0a, 0x33, 0x3d, 0xda, 0x4a, 0x7e, 0x3d, 0x6d, 0x65, + 0x5f, 0x86, 0x14, 0xf5, 0xfb, 0x02, 0x20, 0xaa, 0xa8, 0x1a, 0xdb, 0xe3, 0xff, 0xb6, 0xf4, 0xa7, + 0x3b, 0x20, 0x52, 0x57, 0x4d, 0xc5, 0xd8, 0x53, 0x5a, 0x86, 0xe3, 0x18, 0xe6, 0x3e, 0xef, 0x50, + 0x79, 0x4a, 0xaf, 0xed, 0x6d, 0x32, 0xaa, 0xf4, 0x97, 0x60, 0xae, 0xa7, 0x02, 0x71, 0x36, 0xf6, + 0x75, 0x98, 0xd9, 0x63, 0x47, 0xb0, 0x54, 0x38, 0xdf, 0x1e, 0xcc, 0x52, 0x1a, 0xcb, 0x4f, 0xfa, + 0xcf, 0x09, 0xb8, 0x20, 0x63, 0xc7, 0x6a, 0x1e, 0xe1, 0xf8, 0x55, 0x58, 0x05, 0x7e, 0xf6, 0xa2, + 0xbc, 0x92, 0x26, 0x33, 0x8c, 0x99, 0x2d, 0x73, 0xbd, 0x7b, 0xec, 0x6f, 0x8c, 0xee, 0xb1, 0x83, + 0xbb, 0xea, 0x7c, 0xa7, 0x2e, 0xd5, 0xb3, 0x53, 0x67, 0x41, 0x81, 0x9d, 0x1e, 0xeb, 0x8a, 0x83, + 0x5f, 0x98, 0x9d, 0x96, 0x07, 0x86, 0x8a, 0xa3, 0x0a, 0x59, 0x63, 0x2c, 0x75, 0xfc, 0x62, 0xab, + 0xd3, 0xa2, 0xb6, 0x73, 0xe9, 0x12, 0x29, 0xef, 0xe9, 0xc9, 0x52, 0xbe, 0xe7, 0x9d, 0x23, 0xe7, + 0x0d, 0xff, 0x99, 0x48, 0x97, 0xbe, 0x0f, 0x17, 0xfb, 0x94, 0x1d, 0xa7, 0xc5, 0xf3, 0xcf, 0x93, + 0x70, 0xa5, 0x57, 0x7c, 0xdc, 0x10, 0xe7, 0xdb, 0xde, 0xa0, 0x55, 0xc8, 0xb5, 0x0c, 0xf3, 0xd5, + 0x76, 0x2f, 0x67, 0x5a, 0x86, 0xe9, 0xd3, 0xa2, 0xba, 0xc6, 0xd4, 0x37, 0xda, 0x35, 0x54, 0x58, + 0x88, 0x6a, 0xbb, 0x38, 0xfb, 0xc7, 0xe7, 0x02, 0xcc, 0xc4, 0xbd, 0x2d, 0xf7, 0x6a, 0x5e, 0x70, + 0x52, 0x03, 0x72, 0xdf, 0xc0, 0x3e, 0xde, 0x6f, 0x0b, 0x80, 0x1a, 0x76, 0xc7, 0x24, 0xa0, 0x76, + 0xc3, 0xda, 0x8f, 0xb3, 0x9a, 0x17, 0x60, 0xd2, 0x30, 0x75, 0x7c, 0x4c, 0xab, 0x99, 0x92, 0xd9, + 0x43, 0xcf, 0x51, 0x62, 0x72, 0xac, 0xa3, 0x44, 0xe9, 0x53, 0x98, 0xeb, 0x29, 0x62, 0x9c, 0xf5, + 0xff, 0x93, 0x04, 0xcc, 0xf1, 0x8a, 0xc4, 0xbe, 0x83, 0xf9, 0x0e, 0x4c, 0x36, 0x89, 0xcc, 0x11, + 0xed, 0x4c, 0xf3, 0xf4, 0xda, 0x99, 0x26, 0x46, 0xdf, 0x05, 0x68, 0xdb, 0xf8, 0x48, 0x61, 0xac, + 0xc9, 0xb1, 0x58, 0x33, 0x84, 0x83, 0x12, 0xd0, 0x8f, 0x05, 0x28, 0x90, 0x01, 0xdd, 0xb6, 0xad, + 0xb6, 0xe5, 0x10, 0x9b, 0xc5, 0x19, 0x0f, 0xe6, 0x3c, 0x3d, 0x3d, 0x59, 0xca, 0x6d, 0x1a, 0xe6, + 0x36, 0x67, 0x6c, 0xd4, 0xc7, 0x76, 0xf0, 0xf7, 0xc2, 0x1c, 0x8a, 0xe5, 0xa6, 0xa5, 0x1d, 0x06, + 0x87, 0x63, 0x64, 0x66, 0xf1, 0xc5, 0x39, 0xd2, 0x1f, 0x0a, 0x70, 0xe1, 0x1b, 0xdb, 0x2e, 0xfe, + 0xb3, 0x50, 0xb6, 0xf4, 0x0c, 0x44, 0xfa, 0xa3, 0x66, 0xee, 0x59, 0x71, 0x6e, 0xdc, 0xff, 0x6f, + 0x01, 0x66, 0x43, 0x82, 0xe3, 0x34, 0x70, 0x5e, 0x55, 0x4f, 0x39, 0xe6, 0x0e, 0xe3, 0x8e, 0xa7, + 0x2a, 0x79, 0x86, 0x27, 0x67, 0x9d, 0xb2, 0x08, 0x33, 0x98, 0xcc, 0x62, 0x74, 0x8b, 0x77, 0x97, + 0x05, 0x99, 0xf4, 0xed, 0xe8, 0x67, 0xfd, 0x04, 0xa5, 0xae, 0xf4, 0x2b, 0xc4, 0xc2, 0x0a, 0x0f, + 0xca, 0x38, 0x87, 0xfc, 0x3f, 0x4b, 0xc0, 0xa5, 0x32, 0x3b, 0x02, 0xf7, 0x7c, 0x42, 0xe2, 0xec, + 0x88, 0xf3, 0x30, 0x7d, 0x84, 0x6d, 0xc7, 0xb0, 0xd8, 0x6a, 0x9f, 0x93, 0xbd, 0x47, 0xb4, 0x00, + 0x69, 0xc7, 0x54, 0xdb, 0xce, 0x81, 0xe5, 0x1d, 0x27, 0xfa, 0xcf, 0xbe, 0xff, 0xca, 0xe4, 0xab, + 0xfb, 0xaf, 0x4c, 0x8d, 0xf6, 0x5f, 0x99, 0xfe, 0x1a, 0xfe, 0x2b, 0xfc, 0xec, 0xee, 0x5f, 0x0b, + 0x70, 0x79, 0x40, 0x73, 0x71, 0x76, 0xce, 0x1f, 0x40, 0x56, 0xe3, 0x82, 0xc9, 0xfa, 0xc0, 0x0e, + 0x26, 0x6b, 0x24, 0xd9, 0x2b, 0x42, 0x9f, 0xd3, 0x93, 0x25, 0xf0, 0x8a, 0x5a, 0x5b, 0xe5, 0xca, + 0x21, 0xbf, 0x75, 0xe9, 0xbf, 0x00, 0x14, 0x2a, 0xc7, 0x6c, 0x53, 0xbe, 0xce, 0xac, 0x12, 0xf4, + 0x08, 0xd2, 0x6d, 0xdb, 0x3a, 0x32, 0xbc, 0x6a, 0xe4, 0x7b, 0x9c, 0x17, 0xbc, 0x6a, 0xf4, 0x71, + 0x6d, 0x73, 0x0e, 0xd9, 0xe7, 0x45, 0x0d, 0xc8, 0x6c, 0x58, 0x9a, 0xda, 0x7c, 0x64, 0x34, 0xbd, + 0x81, 0xf6, 0xf6, 0xd9, 0x82, 0x8a, 0x3e, 0xcf, 0xb6, 0xea, 0x1e, 0x78, 0x8d, 0xe0, 0x13, 0x51, + 0x0d, 0xd2, 0x55, 0xd7, 0x6d, 0x93, 0x97, 0x7c, 0xfc, 0xdd, 0x1e, 0x43, 0x28, 0x61, 0xf1, 0x3c, + 0x6e, 0x3d, 0x76, 0xd4, 0x80, 0xd9, 0xc7, 0x34, 0x7e, 0xac, 0xdc, 0xb4, 0x3a, 0x7a, 0x99, 0xc6, + 0xbb, 0xf1, 0x65, 0xe2, 0xd6, 0x18, 0x32, 0x1f, 0x97, 0xeb, 0xf2, 0xa0, 0x00, 0xb4, 0x02, 0xe9, + 0xfa, 0x43, 0x2e, 0x8c, 0x99, 0x91, 0x37, 0xc7, 0x10, 0x56, 0x7f, 0x28, 0xfb, 0x6c, 0x68, 0x0d, + 0xb2, 0x2b, 0x9f, 0x75, 0x6c, 0xcc, 0xa5, 0x4c, 0x0d, 0xf5, 0x9c, 0xe8, 0x97, 0x42, 0xb9, 0xe4, + 0x30, 0x33, 0xfa, 0x3e, 0x14, 0x88, 0xde, 0x1a, 0xea, 0x6e, 0xd3, 0x93, 0x97, 0xa6, 0xf2, 0xde, + 0x1c, 0x43, 0x9e, 0xcf, 0xe9, 0x1d, 0x09, 0xf4, 0x89, 0x5a, 0x90, 0x21, 0xd7, 0xd3, 0x5e, 0x08, + 0x41, 0xaa, 0x4d, 0x9a, 0x46, 0xa0, 0x6e, 0x48, 0xf4, 0x37, 0x7a, 0x0b, 0xa6, 0x4d, 0x4b, 0xc7, + 0x5e, 0x67, 0xce, 0x95, 0x2e, 0x9c, 0x9e, 0x2c, 0x4d, 0x6d, 0x59, 0x3a, 0xb3, 0x75, 0xf8, 0x2f, + 0x79, 0x8a, 0x24, 0xaa, 0xe9, 0x0b, 0xd7, 0x20, 0x45, 0x9a, 0x88, 0xcc, 0x21, 0xbb, 0xaa, 0x83, + 0x77, 0x6c, 0x83, 0x4b, 0xf3, 0x1e, 0x17, 0xfe, 0x41, 0x02, 0x12, 0xf5, 0x87, 0xc4, 0x9a, 0xdf, + 0xed, 0x68, 0x87, 0xd8, 0xe5, 0xef, 0xf9, 0x13, 0xb5, 0xf2, 0x6d, 0xbc, 0x67, 0x30, 0xa3, 0x2b, + 0x23, 0xf3, 0x27, 0xf4, 0x3a, 0x80, 0xaa, 0x69, 0xd8, 0x71, 0x14, 0x2f, 0x04, 0x30, 0x23, 0x67, + 0x18, 0x65, 0x1d, 0x77, 0x09, 0x9b, 0x83, 0x35, 0x1b, 0xbb, 0x9e, 0x0f, 0x15, 0x7b, 0x22, 0x6c, + 0x2e, 0x6e, 0xb5, 0x15, 0xd7, 0x3a, 0xc4, 0x26, 0x6d, 0xd2, 0x0c, 0x99, 0x15, 0x5a, 0xed, 0x06, + 0x21, 0x90, 0x09, 0x0d, 0x9b, 0x7a, 0x30, 0xfb, 0x64, 0x64, 0xff, 0x99, 0x88, 0xb4, 0xf1, 0xbe, + 0xc1, 0x03, 0xe8, 0x32, 0x32, 0x7f, 0x22, 0x5a, 0x52, 0x3b, 0xee, 0x01, 0x6d, 0x89, 0x8c, 0x4c, + 0x7f, 0xa3, 0x5b, 0x50, 0x60, 0x6e, 0x97, 0x0a, 0x36, 0x35, 0x85, 0xce, 0x83, 0x19, 0xfa, 0x3a, + 0xc7, 0xc8, 0x15, 0x53, 0x23, 0xb3, 0x1e, 0x7a, 0x08, 0x9c, 0xa0, 0x1c, 0xb6, 0x1c, 0xa2, 0x53, + 0x20, 0xa9, 0x4a, 0x85, 0xd3, 0x93, 0xa5, 0x6c, 0x9d, 0xbe, 0x58, 0xdf, 0xac, 0x93, 0xb5, 0x84, + 0xa5, 0x5a, 0x6f, 0x39, 0x35, 0x7d, 0xe1, 0x6f, 0x09, 0x90, 0x7c, 0x5c, 0xae, 0x9f, 0x5b, 0x65, + 0x5e, 0x41, 0x93, 0xa1, 0x82, 0xde, 0x86, 0xc2, 0xae, 0xd1, 0x6c, 0x1a, 0xe6, 0x3e, 0xb1, 0xaf, + 0x7e, 0x80, 0x35, 0x4f, 0x61, 0x79, 0x4e, 0xde, 0x66, 0x54, 0x74, 0x0d, 0xb2, 0x9a, 0x8d, 0x75, + 0x6c, 0xba, 0x86, 0xda, 0x74, 0xb8, 0xe6, 0xc2, 0xa4, 0x85, 0xbf, 0x2c, 0xc0, 0x24, 0xed, 0xac, + 0xe8, 0x35, 0xc8, 0x68, 0x96, 0xe9, 0xaa, 0x86, 0xc9, 0x67, 0x9d, 0x8c, 0x1c, 0x10, 0x86, 0x16, + 0xef, 0x3a, 0xcc, 0xa8, 0x9a, 0x66, 0x75, 0x4c, 0x57, 0x31, 0xd5, 0x16, 0xe6, 0xc5, 0xcc, 0x72, + 0xda, 0x96, 0xda, 0xc2, 0x68, 0x09, 0xbc, 0x47, 0x3f, 0xb2, 0x33, 0x23, 0x03, 0x27, 0xad, 0xe3, + 0xee, 0x02, 0x86, 0x8c, 0xdf, 0xab, 0x49, 0x7d, 0x3b, 0x8e, 0x5f, 0x02, 0xfa, 0x1b, 0xbd, 0x0d, + 0x17, 0x5e, 0x74, 0xd4, 0xa6, 0xb1, 0x47, 0x37, 0xbf, 0xa8, 0x97, 0x3a, 0xcd, 0x8c, 0x15, 0x05, + 0xf9, 0xef, 0xa8, 0x04, 0x9a, 0xa7, 0x37, 0x08, 0x92, 0xc1, 0x20, 0x60, 0x2e, 0x3b, 0x52, 0x17, + 0x66, 0x65, 0xec, 0xda, 0xdd, 0x06, 0x0b, 0x76, 0xad, 0x1c, 0x61, 0xd3, 0x25, 0x75, 0xb7, 0xda, + 0x98, 0x39, 0x89, 0x78, 0x75, 0xf7, 0x09, 0xe8, 0x26, 0xe4, 0x55, 0x97, 0x74, 0x37, 0x57, 0x31, + 0x3b, 0xad, 0x5d, 0x6c, 0x33, 0x57, 0x00, 0x39, 0xc7, 0xa9, 0x5b, 0x94, 0xc8, 0x23, 0x32, 0xec, + 0xae, 0x42, 0xf7, 0x89, 0x78, 0xd6, 0x40, 0x49, 0x15, 0x42, 0x91, 0xee, 0xc2, 0x45, 0x52, 0xcf, + 0x8a, 0xa9, 0xd9, 0xdd, 0x36, 0x91, 0xfc, 0x84, 0xfe, 0x75, 0x90, 0x18, 0x3a, 0xa7, 0xa1, 0xc7, + 0x33, 0xd2, 0x97, 0xd3, 0x90, 0xab, 0x1c, 0xb7, 0x2d, 0x3b, 0xd6, 0x5d, 0x9d, 0x12, 0x4c, 0x73, + 0xe0, 0x3b, 0xe2, 0x28, 0xb6, 0x6f, 0x06, 0xf2, 0xce, 0xa1, 0x39, 0x23, 0x7a, 0xe6, 0x07, 0xa6, + 0x1e, 0xe2, 0x2e, 0xb1, 0xea, 0x67, 0xc6, 0xb1, 0xea, 0xe7, 0x38, 0x96, 0xce, 0xb2, 0x38, 0xd2, + 0x75, 0xdc, 0x6d, 0xd4, 0x65, 0x1e, 0x1b, 0x4b, 0x1e, 0x1c, 0xb4, 0x05, 0xd9, 0xd6, 0x91, 0xa6, + 0x29, 0x7b, 0x46, 0xd3, 0xe5, 0xbe, 0x6a, 0xd1, 0x6e, 0xd5, 0x9b, 0xcf, 0xca, 0xe5, 0x47, 0x34, + 0x11, 0xf3, 0xf1, 0x0a, 0x9e, 0x65, 0x20, 0x12, 0xd8, 0x6f, 0x54, 0x02, 0x60, 0x8e, 0x9f, 0xd4, + 0x5f, 0x28, 0x79, 0x8e, 0x13, 0x2e, 0xca, 0x46, 0x83, 0x22, 0x24, 0xc8, 0x31, 0xf7, 0xa4, 0x96, + 0xc1, 0xa2, 0x92, 0x73, 0x6c, 0x1b, 0x8e, 0x12, 0x37, 0x0d, 0x1a, 0x91, 0xfc, 0x26, 0xf0, 0xa0, + 0x1b, 0xc5, 0xf1, 0x42, 0xe8, 0x4a, 0xb9, 0xd3, 0x93, 0xa5, 0x8c, 0x4c, 0xa9, 0xf5, 0x7a, 0x43, + 0xce, 0xb0, 0x04, 0x75, 0xc7, 0x3d, 0x4f, 0x98, 0xc5, 0xf4, 0xf8, 0x61, 0x16, 0x7f, 0x4d, 0x80, + 0x4b, 0xbc, 0x7d, 0x94, 0x5d, 0xea, 0x5a, 0xae, 0x36, 0x0d, 0xb7, 0xab, 0x1c, 0x1e, 0xcd, 0xa7, + 0xa9, 0xb9, 0xf5, 0x9d, 0xc8, 0x76, 0x0e, 0x75, 0xaf, 0xa2, 0xd7, 0xda, 0xdd, 0x0d, 0xce, 0xbc, + 0x7e, 0x54, 0x31, 0x5d, 0xbb, 0x5b, 0xba, 0x7c, 0x7a, 0xb2, 0x34, 0x37, 0xf8, 0xf6, 0x99, 0x3c, + 0xe7, 0x0c, 0xb2, 0xa0, 0x2a, 0x00, 0xf6, 0xbb, 0x37, 0x9d, 0x3d, 0xa3, 0x97, 0xcd, 0xc8, 0x71, + 0x20, 0x87, 0x78, 0xd1, 0x1d, 0x10, 0x79, 0x58, 0xcb, 0x9e, 0xd1, 0xc4, 0x8a, 0x63, 0x7c, 0x86, + 0xe9, 0x3c, 0x9b, 0x94, 0xf3, 0x8c, 0x4e, 0x44, 0xd4, 0x8d, 0xcf, 0x30, 0xba, 0x0f, 0x17, 0x83, + 0x16, 0x50, 0x76, 0x71, 0xd3, 0x7a, 0xc9, 0x92, 0x67, 0x69, 0x72, 0xe4, 0x6b, 0xbf, 0x44, 0x5e, + 0x11, 0x96, 0x85, 0x1f, 0xc0, 0xfc, 0xb0, 0x0a, 0x87, 0x07, 0x63, 0x86, 0x9d, 0x95, 0xbe, 0xdf, + 0xbb, 0x51, 0x32, 0xc6, 0xa0, 0xe1, 0x9b, 0x25, 0x1f, 0x24, 0xde, 0x17, 0xa4, 0xbf, 0x9f, 0x80, + 0x5c, 0xa9, 0xd3, 0x3c, 0x7c, 0xd2, 0xae, 0xb3, 0xdb, 0x02, 0xd0, 0x55, 0xc8, 0xe8, 0xaa, 0xab, + 0xb2, 0x42, 0x0a, 0x2c, 0xbc, 0x8d, 0x10, 0x68, 0x6d, 0x6e, 0x43, 0x21, 0xe4, 0x87, 0xc2, 0xbd, + 0xed, 0x69, 0xb5, 0x03, 0x32, 0x75, 0x88, 0x7f, 0x1f, 0xe6, 0x43, 0x09, 0xe9, 0xae, 0x86, 0x82, + 0x4d, 0xd7, 0x36, 0x30, 0xdb, 0x99, 0x4b, 0xca, 0x21, 0x67, 0x99, 0x1a, 0x79, 0x5d, 0x61, 0x6f, + 0x51, 0x03, 0x66, 0x48, 0xc2, 0xae, 0x42, 0x67, 0x60, 0x6f, 0xe7, 0xf4, 0x7e, 0x44, 0xb5, 0x7a, + 0xca, 0x5d, 0xa4, 0xfa, 0x29, 0x53, 0x1e, 0xfa, 0x53, 0xce, 0xe2, 0x80, 0xb2, 0xf0, 0x31, 0x88, + 0xfd, 0x09, 0xc2, 0xba, 0x4c, 0x31, 0x5d, 0x5e, 0x08, 0xeb, 0x32, 0x19, 0xd2, 0xd3, 0x5a, 0x2a, + 0x9d, 0x12, 0x27, 0xa5, 0xdf, 0x4a, 0x41, 0xde, 0xeb, 0x99, 0x71, 0x9a, 0xf4, 0x25, 0x98, 0x24, + 0xfd, 0xc8, 0x73, 0xed, 0xb8, 0x35, 0x62, 0x40, 0x70, 0xff, 0x6e, 0xd2, 0xbf, 0x3c, 0xf4, 0x49, + 0x59, 0xe3, 0x98, 0x52, 0x16, 0x7e, 0x2f, 0x01, 0x29, 0x6a, 0x45, 0xdf, 0x87, 0x14, 0xbd, 0x13, + 0x40, 0x18, 0x79, 0x27, 0x80, 0xe7, 0x1a, 0x40, 0x92, 0xa2, 0x4d, 0x32, 0xb8, 0x74, 0x6f, 0xde, + 0xcd, 0x8c, 0x93, 0xbf, 0xc8, 0xe7, 0xdd, 0x74, 0x85, 0xde, 0x9f, 0xd0, 0xa8, 0x53, 0xd3, 0x88, + 0xcd, 0xb8, 0xde, 0x1a, 0x99, 0x08, 0x19, 0x8a, 0x25, 0xea, 0xaa, 0x64, 0xd9, 0x2e, 0xd6, 0xb9, + 0xd1, 0x7b, 0xed, 0xac, 0x6e, 0xe1, 0x19, 0xf5, 0x1e, 0x1f, 0xba, 0x02, 0x49, 0x32, 0x15, 0x4e, + 0x33, 0xaf, 0x83, 0xd3, 0x93, 0xa5, 0x24, 0x99, 0x04, 0x09, 0x0d, 0x2d, 0x43, 0xb6, 0x77, 0x72, + 0x22, 0x76, 0x13, 0x9d, 0xc5, 0x43, 0x13, 0x0b, 0x34, 0xfd, 0x11, 0xc9, 0x00, 0x1f, 0xeb, 0x1a, + 0xdc, 0xdf, 0xe0, 0xd7, 0x05, 0xee, 0x5e, 0x59, 0xd7, 0xc8, 0xf2, 0x6b, 0xc7, 0xb9, 0x3e, 0xde, + 0x05, 0xd1, 0x56, 0x4d, 0xdd, 0x6a, 0x19, 0x9f, 0x61, 0xb6, 0xc1, 0xe0, 0xf0, 0x93, 0x97, 0x82, + 0x4f, 0xa7, 0x3b, 0x01, 0x8e, 0xf4, 0x9f, 0x04, 0xee, 0x8a, 0xe9, 0x17, 0x23, 0xde, 0xf3, 0xf1, + 0x2c, 0xdf, 0x9d, 0x34, 0xf7, 0x2c, 0xcf, 0x93, 0xe4, 0xb5, 0x61, 0x7e, 0x53, 0x35, 0x73, 0xcf, + 0xf2, 0x4e, 0xfa, 0x6c, 0x8f, 0xe0, 0x2c, 0xfc, 0x32, 0x4c, 0xd2, 0xd7, 0xaf, 0xd0, 0xd5, 0x7c, + 0xf7, 0xdf, 0x84, 0x98, 0x94, 0xfe, 0x38, 0x01, 0x6f, 0xd0, 0xaa, 0x3e, 0xc3, 0xb6, 0xb1, 0xd7, + 0xdd, 0xb6, 0x2d, 0x17, 0x6b, 0x2e, 0xd6, 0x83, 0x0d, 0xb6, 0x18, 0x9b, 0x40, 0x87, 0x0c, 0x3f, + 0x9a, 0x34, 0x74, 0x7e, 0x09, 0xc8, 0xe3, 0xaf, 0x07, 0xbc, 0xd3, 0xec, 0x48, 0xb3, 0xb6, 0x2a, + 0xa7, 0x99, 0xe4, 0x9a, 0x8e, 0x56, 0x20, 0xd3, 0xf6, 0xaa, 0x71, 0x2e, 0xef, 0x17, 0x9f, 0x0b, + 0xad, 0x43, 0x81, 0x17, 0x54, 0x6d, 0x1a, 0x47, 0x58, 0x51, 0xdd, 0xf3, 0xcc, 0x08, 0x39, 0xc6, + 0xbb, 0x42, 0x58, 0x57, 0x5c, 0xe9, 0x6f, 0xa4, 0xe0, 0xe6, 0x19, 0x2a, 0x8e, 0xb3, 0x7b, 0x2d, + 0x40, 0xfa, 0x88, 0x64, 0x64, 0xf0, 0xda, 0xa7, 0x65, 0xff, 0x19, 0xed, 0xf6, 0x2c, 0x2b, 0x7b, + 0xaa, 0xd1, 0x24, 0xcb, 0x10, 0xf3, 0x37, 0x1c, 0xee, 0xd1, 0x14, 0xed, 0xbf, 0x17, 0x5a, 0x80, + 0x1e, 0x51, 0x41, 0x34, 0x99, 0x83, 0x3e, 0x17, 0x60, 0x81, 0x65, 0xc8, 0x9c, 0xde, 0xfa, 0xb2, + 0x49, 0xd1, 0x6c, 0x56, 0x23, 0xb2, 0x19, 0x4b, 0x47, 0xc5, 0x50, 0x5e, 0xbc, 0x20, 0xf3, 0xe1, + 0xdc, 0xc2, 0x45, 0x59, 0xf8, 0x0d, 0x01, 0xb2, 0x21, 0x02, 0xba, 0x35, 0x10, 0x62, 0x94, 0x3d, + 0x8d, 0x8a, 0x2b, 0xba, 0x39, 0x10, 0x57, 0x54, 0x4a, 0x7f, 0x75, 0xb2, 0x94, 0x92, 0x99, 0xeb, + 0xba, 0x17, 0x61, 0x74, 0x3d, 0xb8, 0xd1, 0x26, 0xd9, 0x97, 0xc8, 0xbb, 0xd2, 0x86, 0x22, 0x54, + 0xd5, 0x3b, 0x11, 0xa3, 0x08, 0x95, 0x3c, 0x49, 0x3f, 0x49, 0xc0, 0xec, 0x8a, 0xae, 0xd7, 0xeb, + 0x14, 0xd5, 0xc4, 0x39, 0xc6, 0x10, 0xa4, 0x88, 0xb9, 0xc1, 0xc3, 0xa1, 0xe8, 0x6f, 0xf4, 0x16, + 0x20, 0xdd, 0x70, 0xd8, 0xcd, 0x10, 0xce, 0x81, 0xaa, 0x5b, 0x2f, 0x83, 0x83, 0xef, 0x59, 0xef, + 0x4d, 0xdd, 0x7b, 0x81, 0xea, 0x40, 0x6d, 0x6d, 0xc5, 0x71, 0x55, 0x7f, 0x63, 0xff, 0xe6, 0x58, + 0x01, 0x36, 0xcc, 0x38, 0xf6, 0x1f, 0xe5, 0x0c, 0x91, 0x43, 0x7f, 0x12, 0x93, 0xcf, 0x20, 0x8d, + 0xe2, 0x2a, 0xaa, 0xe3, 0x85, 0x86, 0xb0, 0x3b, 0x29, 0xf2, 0x8c, 0xbe, 0xe2, 0xb0, 0x88, 0x0f, + 0xe6, 0x51, 0x1e, 0xa8, 0x26, 0xce, 0x6d, 0xd9, 0xbf, 0x23, 0x40, 0x5e, 0xc6, 0x7b, 0x36, 0x76, + 0x0e, 0xe2, 0xd4, 0xf9, 0x23, 0x98, 0xb1, 0x99, 0x54, 0x65, 0xcf, 0xb6, 0x5a, 0xe7, 0x99, 0x2b, + 0xb2, 0x9c, 0xf1, 0x91, 0x6d, 0xb5, 0xf8, 0x94, 0xfc, 0x0c, 0x0a, 0x7e, 0x19, 0xe3, 0xac, 0xfc, + 0xdf, 0xa3, 0xc1, 0xa3, 0x4c, 0x70, 0xdc, 0x27, 0xd0, 0xf1, 0x6a, 0x80, 0x6e, 0xcd, 0x87, 0x0b, + 0x1a, 0xa7, 0x1a, 0xfe, 0xa3, 0x00, 0xf9, 0x7a, 0x67, 0x97, 0xdd, 0x78, 0x14, 0x9f, 0x06, 0x2a, + 0x90, 0x69, 0xe2, 0x3d, 0x57, 0x79, 0x25, 0x5f, 0xe8, 0x34, 0x61, 0xa5, 0x9e, 0xe0, 0x8f, 0x01, + 0x6c, 0x1a, 0x3d, 0x45, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x43, 0x79, 0x09, 0x99, 0xac, 0x3a, 0x05, + 0xbf, 0x9a, 0x71, 0xae, 0x2f, 0xcf, 0x7b, 0x66, 0x87, 0xe4, 0x79, 0x66, 0x87, 0x59, 0x6e, 0xb0, + 0x46, 0xcf, 0x10, 0x45, 0x98, 0xa3, 0x66, 0x99, 0xa2, 0xb6, 0xdb, 0x4d, 0xc3, 0x83, 0x3d, 0x74, + 0xfe, 0x49, 0xc9, 0xb3, 0xf4, 0xd5, 0x0a, 0x7b, 0x43, 0x01, 0x0f, 0xfa, 0xa1, 0x00, 0x33, 0x7b, + 0x36, 0xc6, 0x9f, 0x61, 0x85, 0x4e, 0xc9, 0xe3, 0x79, 0x15, 0xac, 0x92, 0x32, 0x7c, 0xed, 0x53, + 0xc7, 0x2c, 0xcb, 0xb8, 0x4e, 0xf2, 0x45, 0x5b, 0x20, 0x6a, 0x4d, 0x76, 0x0e, 0xea, 0x7b, 0x38, + 0x4c, 0x8d, 0x3f, 0x00, 0x0a, 0x8c, 0x39, 0x70, 0x72, 0x78, 0x4a, 0x06, 0x93, 0xaa, 0x2b, 0xfc, + 0x02, 0x3a, 0x6a, 0x6c, 0xf7, 0x7a, 0x38, 0x84, 0xa3, 0xc8, 0x43, 0xf7, 0xd6, 0x15, 0x65, 0xac, + 0xea, 0xdc, 0x72, 0x27, 0xe3, 0xca, 0x7f, 0xe0, 0xe3, 0xea, 0x39, 0xcc, 0xd2, 0x7e, 0x13, 0x77, + 0x30, 0xa8, 0xf4, 0x8f, 0x92, 0x80, 0xc2, 0x92, 0xbf, 0xb9, 0xfe, 0x96, 0x88, 0xaf, 0xbf, 0xad, + 0x81, 0x14, 0x32, 0x86, 0x9a, 0xaa, 0xe3, 0x2a, 0xcc, 0x95, 0xce, 0x51, 0xda, 0xd8, 0x56, 0x1c, + 0xac, 0x59, 0xfc, 0x3e, 0x20, 0x41, 0x5e, 0x0c, 0x52, 0x6e, 0xa8, 0x8e, 0xfb, 0x94, 0xa5, 0xdb, + 0xc6, 0x76, 0x9d, 0xa6, 0x42, 0x0f, 0xe1, 0x52, 0x4b, 0x3d, 0x8e, 0xe2, 0x9f, 0xa4, 0xfc, 0x73, + 0x2d, 0xf5, 0x78, 0x80, 0xe9, 0x03, 0x58, 0x88, 0x66, 0x52, 0x1c, 0xec, 0x1d, 0xb5, 0x5d, 0x8a, + 0x60, 0xac, 0x63, 0x17, 0xad, 0x00, 0x04, 0x20, 0x82, 0xaf, 0xd1, 0xe3, 0x60, 0x88, 0x8c, 0x8f, + 0x21, 0xa4, 0x1f, 0x09, 0x90, 0xdf, 0x34, 0xf6, 0x6d, 0x35, 0xd6, 0xdb, 0x76, 0xd0, 0x07, 0xbd, + 0x67, 0x93, 0xd9, 0x07, 0x0b, 0x51, 0xbe, 0x27, 0x2c, 0x85, 0xb7, 0xff, 0xc8, 0x19, 0xc8, 0xd2, + 0xe7, 0x97, 0x28, 0xce, 0x39, 0x5f, 0x83, 0xd7, 0xa9, 0x37, 0x1f, 0x77, 0xe5, 0xf9, 0x46, 0xd0, + 0x8d, 0xf4, 0xbb, 0x02, 0x2c, 0x0e, 0xcb, 0x25, 0xce, 0x01, 0x21, 0xd3, 0x7b, 0x07, 0x69, 0x0e, + 0x8a, 0x3f, 0x22, 0xce, 0x98, 0x69, 0x10, 0x1f, 0x09, 0xe0, 0x97, 0xad, 0x4e, 0x6f, 0x24, 0x64, + 0xbf, 0x1d, 0xa2, 0xa0, 0xba, 0xa6, 0x9a, 0x34, 0x54, 0xb8, 0x89, 0xd5, 0x23, 0xcc, 0x1d, 0x11, + 0x63, 0x9d, 0x25, 0xbe, 0x10, 0x60, 0x71, 0x58, 0x2e, 0x71, 0x2a, 0xe8, 0x3b, 0x30, 0xcd, 0x3c, + 0xda, 0x3c, 0x50, 0x73, 0x65, 0x48, 0x54, 0xb4, 0xe9, 0xc5, 0xe2, 0x78, 0xe9, 0xa5, 0x06, 0xe4, + 0x4b, 0xaa, 0x6d, 0x1b, 0xb1, 0x6e, 0x3d, 0x48, 0x3f, 0x11, 0xa0, 0xe0, 0x8b, 0x8d, 0xb3, 0xa6, + 0x5f, 0x3f, 0xd0, 0x43, 0xfa, 0xaf, 0x57, 0x61, 0x86, 0x97, 0x7f, 0xc7, 0x34, 0x2c, 0x13, 0xdd, + 0x87, 0xe4, 0x3e, 0x3f, 0xc0, 0xca, 0x46, 0xee, 0xd1, 0x07, 0xd7, 0x3c, 0x56, 0x27, 0x64, 0x92, + 0x96, 0xb0, 0xb4, 0x3b, 0x6e, 0x44, 0x01, 0x82, 0x48, 0x85, 0x30, 0x4b, 0xbb, 0xe3, 0xa2, 0x3a, + 0x14, 0xb4, 0xe0, 0x6e, 0x39, 0x85, 0xb0, 0x27, 0x87, 0x6e, 0x28, 0x47, 0xde, 0xf2, 0x57, 0x9d, + 0x90, 0xf3, 0x5a, 0xcf, 0x0b, 0x54, 0x0e, 0x5f, 0x69, 0x96, 0x1a, 0x70, 0x83, 0x0c, 0x9a, 0xbe, + 0xf7, 0x3a, 0xb5, 0xea, 0x44, 0xe8, 0xe6, 0x33, 0xf4, 0x01, 0x4c, 0xe9, 0xf4, 0xf2, 0x2c, 0x6e, + 0x4f, 0x44, 0x35, 0x78, 0xcf, 0x1d, 0x65, 0xd5, 0x09, 0x99, 0x73, 0xa0, 0x35, 0x98, 0x61, 0xbf, + 0x18, 0xda, 0xe5, 0x56, 0xc0, 0xcd, 0xe1, 0x12, 0x42, 0x76, 0x78, 0x75, 0x42, 0xce, 0xea, 0x01, + 0x15, 0x3d, 0x86, 0xac, 0xd6, 0xc4, 0xaa, 0xcd, 0x45, 0xdd, 0x1a, 0x1a, 0xbb, 0x39, 0x70, 0xe1, + 0x56, 0x75, 0x42, 0x06, 0xcd, 0x27, 0x92, 0x42, 0xd9, 0xf4, 0xde, 0x25, 0x2e, 0xe9, 0xed, 0xa1, + 0x85, 0x1a, 0xbc, 0xc4, 0xaa, 0x4a, 0xed, 0x73, 0x9f, 0x8a, 0xde, 0x81, 0x94, 0xa3, 0xa9, 0x26, + 0x37, 0x49, 0x16, 0x87, 0x5c, 0x8c, 0x13, 0x30, 0xd3, 0xd4, 0xe8, 0x43, 0x06, 0x94, 0xdd, 0x63, + 0xef, 0xd4, 0x20, 0x4a, 0xa7, 0x3d, 0x17, 0x30, 0x10, 0x9d, 0x62, 0x4a, 0x20, 0x7a, 0x50, 0xf5, + 0x96, 0x61, 0x2a, 0xf4, 0x60, 0x86, 0x1e, 0x13, 0x44, 0xeb, 0x61, 0x20, 0x8a, 0xbd, 0x4a, 0x6f, + 0x79, 0xf0, 0x88, 0x68, 0x13, 0x72, 0x4c, 0x50, 0x87, 0x05, 0x58, 0xcf, 0x2f, 0x0f, 0xf5, 0x45, + 0x88, 0x08, 0xf1, 0xae, 0x4e, 0xc8, 0x33, 0x6a, 0x88, 0x1c, 0x94, 0xab, 0x85, 0xed, 0x7d, 0x76, + 0x1e, 0x31, 0xa2, 0x5c, 0x61, 0x07, 0x4f, 0xbf, 0x5c, 0x94, 0x88, 0x7e, 0x0d, 0x2e, 0x30, 0x41, + 0x2e, 0xf7, 0x5b, 0xe3, 0xee, 0x4f, 0xaf, 0x0f, 0xf5, 0x23, 0x18, 0x1a, 0x14, 0x5d, 0x9d, 0x90, + 0x91, 0x3a, 0xf0, 0x12, 0x69, 0x70, 0x91, 0xe5, 0xc0, 0xa3, 0x6a, 0x6d, 0x1e, 0x08, 0x3a, 0x7f, + 0x83, 0x66, 0xf1, 0xd6, 0xb0, 0x2c, 0x22, 0x83, 0x7d, 0xab, 0x13, 0xf2, 0x9c, 0x3a, 0xf8, 0x36, + 0xa8, 0x86, 0xcd, 0xe3, 0x17, 0x79, 0x77, 0x7b, 0x6b, 0x74, 0x35, 0xa2, 0xe2, 0x3e, 0xfd, 0x6a, + 0xf4, 0xbc, 0x24, 0x0d, 0xe8, 0xdf, 0xde, 0x40, 0x3b, 0xd3, 0xcc, 0xd0, 0x06, 0x8c, 0x08, 0x72, + 0x24, 0x0d, 0x78, 0x10, 0x22, 0xa3, 0x22, 0x24, 0xf6, 0x35, 0x7a, 0xea, 0x17, 0x6d, 0x3a, 0xf9, + 0x81, 0x7c, 0xd5, 0x09, 0x39, 0xb1, 0xaf, 0xa1, 0x8f, 0x21, 0xcd, 0xa2, 0xb2, 0x8e, 0xcd, 0xf9, + 0xfc, 0xd0, 0x39, 0xbb, 0x37, 0xb6, 0xad, 0x3a, 0x21, 0xd3, 0x40, 0x30, 0xde, 0x91, 0x79, 0xc4, + 0x0d, 0x15, 0x51, 0x1c, 0x11, 0x8c, 0xdd, 0x17, 0xf7, 0x44, 0x3a, 0x8c, 0xed, 0x13, 0xd1, 0x36, + 0xe4, 0xf9, 0xd2, 0xed, 0x45, 0x10, 0x88, 0x43, 0x3d, 0x75, 0xa2, 0x82, 0x08, 0xaa, 0x74, 0x8b, + 0x32, 0x44, 0x27, 0x6d, 0xd7, 0x2b, 0x91, 0xb7, 0xdd, 0xec, 0xd0, 0xb6, 0x1b, 0xea, 0xd0, 0x4e, + 0xda, 0xce, 0x1e, 0x78, 0x89, 0xde, 0x83, 0x49, 0x36, 0x4e, 0x10, 0x15, 0x19, 0xe5, 0x7c, 0xd6, + 0x37, 0x44, 0x58, 0x7a, 0x32, 0x7b, 0xb9, 0xdc, 0x31, 0x57, 0x69, 0x5a, 0xfb, 0xf3, 0x73, 0x43, + 0x67, 0xaf, 0x41, 0x17, 0x63, 0x32, 0x7b, 0xb9, 0x01, 0x95, 0x74, 0x20, 0x9b, 0xbd, 0xe1, 0x43, + 0xec, 0xc2, 0xd0, 0x0e, 0x14, 0xe1, 0xaf, 0x5b, 0xa5, 0x21, 0x53, 0x01, 0xd9, 0x9f, 0x58, 0x1d, + 0xac, 0xd0, 0x49, 0xf1, 0xe2, 0xe8, 0x89, 0xb5, 0xe7, 0xb6, 0x32, 0x7f, 0x62, 0x65, 0x54, 0xf4, + 0x0c, 0x44, 0x7e, 0x65, 0x8e, 0xe2, 0xf9, 0x8f, 0xcd, 0x5f, 0xa2, 0xf2, 0xee, 0x46, 0x2e, 0x88, + 0x51, 0xae, 0x85, 0x55, 0x82, 0x25, 0x7b, 0xdf, 0xa0, 0x4f, 0x60, 0x96, 0xca, 0x53, 0xb4, 0xe0, + 0x96, 0xa3, 0xf9, 0xf9, 0x81, 0x3b, 0x73, 0x86, 0x5f, 0x88, 0xe4, 0x49, 0x16, 0xb5, 0xbe, 0x57, + 0x64, 0x3c, 0x18, 0xa6, 0xe1, 0xd2, 0xb5, 0x7b, 0x61, 0xe8, 0x78, 0xe8, 0xbd, 0xe1, 0xb5, 0x4a, + 0x6d, 0x2d, 0x4a, 0x21, 0xdd, 0xb8, 0x6f, 0xc6, 0x7b, 0x6d, 0x68, 0x37, 0x1e, 0x32, 0xd9, 0xe5, + 0xdc, 0x9e, 0x79, 0x6e, 0x15, 0x80, 0xed, 0x20, 0x50, 0x50, 0xb4, 0x38, 0xd4, 0x00, 0xe8, 0xf7, + 0xa7, 0x25, 0x06, 0x40, 0xd3, 0xa3, 0x11, 0x03, 0x80, 0x1d, 0x77, 0xcd, 0x5f, 0x1b, 0xbe, 0x58, + 0x85, 0xcf, 0xd7, 0xe9, 0x62, 0x45, 0x09, 0xc4, 0x20, 0x23, 0x70, 0xae, 0x4b, 0x47, 0xf8, 0xf5, + 0xa1, 0xbb, 0x37, 0x7d, 0x81, 0x76, 0xd5, 0x09, 0x39, 0xfd, 0x82, 0x93, 0x48, 0xaf, 0x62, 0x22, + 0xf8, 0xd8, 0xbe, 0x37, 0xb4, 0x57, 0x0d, 0x46, 0x58, 0x91, 0x5e, 0xf5, 0x22, 0xa0, 0x06, 0x4b, + 0x9e, 0xc3, 0x0e, 0xb2, 0xe6, 0xdf, 0x18, 0xbd, 0xe4, 0xf5, 0x1e, 0xbb, 0xf9, 0x4b, 0x1e, 0x27, + 0xb3, 0x25, 0x4f, 0x57, 0x1c, 0x87, 0xba, 0xe9, 0xcc, 0xdf, 0x1c, 0xb1, 0xe4, 0xf5, 0x6d, 0x6d, + 0xb3, 0x25, 0x4f, 0xaf, 0x33, 0x4e, 0x62, 0xfd, 0xd9, 0xde, 0x15, 0x51, 0x1c, 0xd8, 0xdf, 0x1e, + 0x6a, 0xfd, 0x45, 0xde, 0x61, 0x45, 0xac, 0x3f, 0xbb, 0xe7, 0x05, 0xfa, 0x2e, 0x4c, 0xf3, 0xad, + 0xc4, 0xf9, 0x3b, 0x23, 0x4c, 0xea, 0xf0, 0xee, 0x2f, 0xe9, 0x8e, 0x9c, 0x87, 0x4d, 0x0e, 0x6c, + 0x0b, 0x93, 0x4d, 0x7e, 0x77, 0x47, 0x4c, 0x0e, 0x03, 0xbb, 0xa8, 0x6c, 0x72, 0x08, 0xc8, 0xa4, + 0x34, 0x0e, 0xdb, 0x7e, 0x9b, 0xff, 0xa5, 0xa1, 0xa5, 0xe9, 0xdd, 0x87, 0x24, 0xa5, 0xe1, 0x3c, + 0x74, 0xb1, 0xa0, 0x6b, 0x35, 0xd3, 0xce, 0x9b, 0xc3, 0x17, 0x8b, 0xfe, 0x0d, 0x9d, 0xaa, 0x77, + 0x50, 0xc8, 0xb4, 0xf2, 0x57, 0x04, 0xb8, 0xc6, 0xfa, 0x00, 0x3d, 0x26, 0xe9, 0x2a, 0xfe, 0x29, + 0x57, 0x68, 0xb7, 0xea, 0x3e, 0x15, 0xff, 0xde, 0xf9, 0x0f, 0x65, 0xbc, 0x1c, 0x5f, 0x57, 0x47, + 0xa5, 0x23, 0xca, 0x68, 0x31, 0x5c, 0x3f, 0xff, 0x60, 0xa8, 0x32, 0x7a, 0xf7, 0x22, 0x88, 0x32, + 0x38, 0x0f, 0x6a, 0xc2, 0x3c, 0x1b, 0x12, 0x01, 0xee, 0xf5, 0x8b, 0xfe, 0x70, 0xa8, 0xe7, 0xeb, + 0x48, 0xc4, 0x5f, 0x9d, 0x90, 0x2f, 0xbd, 0x88, 0x4c, 0x40, 0x72, 0xa3, 0x77, 0x40, 0x1a, 0x01, + 0x4c, 0x55, 0x3c, 0x3c, 0xf9, 0xce, 0xd0, 0xdc, 0x46, 0xc2, 0x67, 0x92, 0x9b, 0x13, 0x99, 0x80, + 0xa8, 0x66, 0x97, 0x41, 0xc3, 0xf9, 0x77, 0x87, 0xaa, 0xa6, 0x17, 0x93, 0x12, 0xd5, 0x70, 0x9e, + 0xd2, 0x34, 0x77, 0xb9, 0xf0, 0x43, 0xbb, 0x0b, 0xa2, 0xb8, 0x96, 0x4a, 0x5f, 0x16, 0xe7, 0xd7, + 0x52, 0xe9, 0x2b, 0xe2, 0xc2, 0x5a, 0x2a, 0x7d, 0x55, 0x7c, 0x6d, 0x2d, 0x95, 0x5e, 0x12, 0xaf, + 0xad, 0xa5, 0xd2, 0x92, 0x78, 0x43, 0xfa, 0xf9, 0x55, 0xc8, 0x79, 0xc8, 0x92, 0x41, 0xbe, 0x07, + 0x61, 0xc8, 0xb7, 0x38, 0x0c, 0xf2, 0x71, 0x2c, 0xca, 0x31, 0xdf, 0x83, 0x30, 0xe6, 0x5b, 0x1c, + 0x86, 0xf9, 0x02, 0x1e, 0x02, 0xfa, 0x1a, 0xc3, 0x40, 0xdf, 0xdd, 0x31, 0x40, 0x9f, 0x2f, 0xaa, + 0x1f, 0xf5, 0xad, 0x0e, 0xa2, 0xbe, 0x37, 0x46, 0xa3, 0x3e, 0x5f, 0x54, 0x08, 0xf6, 0x7d, 0xd8, + 0x07, 0xfb, 0xae, 0x8f, 0x80, 0x7d, 0x3e, 0xbf, 0x87, 0xfb, 0xd6, 0x23, 0x71, 0xdf, 0xad, 0xb3, + 0x70, 0x9f, 0x2f, 0xa7, 0x07, 0xf8, 0x55, 0xa3, 0x80, 0xdf, 0xcd, 0x33, 0x80, 0x9f, 0x2f, 0x2a, + 0x8c, 0xfc, 0xd6, 0x23, 0x91, 0xdf, 0xad, 0xb3, 0x90, 0x5f, 0x50, 0xac, 0x30, 0xf4, 0x7b, 0xb7, + 0x07, 0xfa, 0x2d, 0x0d, 0x85, 0x7e, 0x3e, 0x37, 0xc3, 0x7e, 0x1f, 0xf5, 0x63, 0xbf, 0xeb, 0x23, + 0xb0, 0x5f, 0xa0, 0x58, 0x0e, 0xfe, 0xaa, 0x51, 0xe0, 0xef, 0xe6, 0x19, 0xe0, 0x2f, 0xd0, 0x45, + 0x08, 0xfd, 0x6d, 0x45, 0xa3, 0xbf, 0xdb, 0x67, 0xa2, 0x3f, 0x5f, 0x5a, 0x2f, 0xfc, 0xab, 0x46, + 0xc1, 0xbf, 0x9b, 0x67, 0xc0, 0xbf, 0xbe, 0x92, 0x31, 0xfc, 0xa7, 0x8e, 0xc4, 0x7f, 0x6f, 0x8d, + 0x89, 0xff, 0x7c, 0xd1, 0x51, 0x00, 0x50, 0x1f, 0x0d, 0x00, 0x8b, 0xe3, 0x02, 0x40, 0x3f, 0x93, + 0x48, 0x04, 0xa8, 0x8e, 0x44, 0x80, 0x6f, 0x8d, 0x89, 0x00, 0xfb, 0x2a, 0xd2, 0x0b, 0x01, 0xb7, + 0xa2, 0x21, 0xe0, 0xed, 0x33, 0x21, 0x60, 0xd0, 0x8a, 0x3d, 0x18, 0x70, 0x39, 0x84, 0x01, 0x5f, + 0x1f, 0x82, 0x01, 0x7d, 0x56, 0x02, 0x02, 0xbf, 0x37, 0x00, 0x02, 0xa5, 0x51, 0x20, 0xd0, 0xe7, + 0xf5, 0x51, 0x60, 0x35, 0x0a, 0x05, 0xde, 0x3c, 0x03, 0x05, 0x06, 0xfd, 0x26, 0x04, 0x03, 0x9f, + 0x0e, 0x81, 0x81, 0x77, 0xce, 0x86, 0x81, 0xbe, 0xbc, 0x3e, 0x1c, 0xa8, 0x8e, 0xc4, 0x81, 0x6f, + 0x8d, 0x89, 0x03, 0x83, 0x16, 0x8c, 0x00, 0x82, 0xef, 0xf7, 0x02, 0xc1, 0x6b, 0xc3, 0x81, 0xa0, + 0x2f, 0x86, 0x23, 0xc1, 0xf5, 0x48, 0x24, 0x78, 0xeb, 0x2c, 0x24, 0x18, 0xcc, 0x66, 0x61, 0x28, + 0xb8, 0x15, 0x0d, 0x05, 0x6f, 0x9f, 0x09, 0x05, 0x83, 0x8e, 0xd4, 0x83, 0x05, 0xd7, 0x23, 0xb1, + 0xe0, 0xad, 0xb3, 0xb0, 0x60, 0xdf, 0x54, 0xcb, 0xc1, 0xe0, 0xf3, 0xa1, 0x60, 0xf0, 0xde, 0x38, + 0x60, 0xd0, 0x17, 0x3a, 0x80, 0x06, 0x3f, 0x1d, 0x8e, 0x06, 0x7f, 0xe9, 0x1c, 0xd7, 0xe3, 0x46, + 0xc2, 0xc1, 0xef, 0x0d, 0xc0, 0x41, 0x69, 0x14, 0x1c, 0x0c, 0x46, 0x86, 0x87, 0x07, 0x2b, 0x11, + 0xe8, 0xed, 0x8d, 0xd1, 0xe8, 0x2d, 0x58, 0xc8, 0x03, 0xf8, 0xf6, 0x61, 0x1f, 0x7c, 0xbb, 0x7e, + 0xa6, 0x37, 0x68, 0x08, 0xbf, 0x95, 0x06, 0xf1, 0xdb, 0x8d, 0x91, 0xf8, 0xcd, 0x97, 0x10, 0x00, + 0xb8, 0xf5, 0x48, 0x00, 0x77, 0xeb, 0x2c, 0x00, 0x17, 0x74, 0x85, 0x30, 0x82, 0xdb, 0x8a, 0x46, + 0x70, 0xb7, 0xcf, 0x44, 0x70, 0x7d, 0xcb, 0x96, 0x07, 0xe1, 0xaa, 0x51, 0x10, 0xee, 0xe6, 0x19, + 0x10, 0x2e, 0xbc, 0x6c, 0xf9, 0x18, 0xae, 0x31, 0x0c, 0xc3, 0xdd, 0x1d, 0x03, 0xc3, 0x05, 0xc6, + 0x5c, 0x1f, 0x88, 0xfb, 0xb8, 0x1f, 0xc4, 0x49, 0xa3, 0x40, 0x5c, 0xd0, 0x89, 0x3c, 0x14, 0xb7, + 0x15, 0x8d, 0xe2, 0x6e, 0x9f, 0x89, 0xe2, 0xc2, 0xe3, 0x3a, 0x04, 0xe3, 0x3e, 0xee, 0x87, 0x71, + 0xd2, 0x28, 0x18, 0x17, 0x94, 0xc7, 0xc3, 0x71, 0xd5, 0x28, 0x1c, 0x77, 0xf3, 0x0c, 0x1c, 0x17, + 0x9a, 0xee, 0x03, 0x20, 0xf7, 0x57, 0xc7, 0x07, 0x72, 0xef, 0xbf, 0xaa, 0x77, 0xdd, 0xd9, 0x48, + 0xee, 0xe3, 0x7e, 0x24, 0x27, 0x8d, 0x42, 0x72, 0x81, 0x3e, 0x3c, 0x28, 0xd7, 0x3a, 0x13, 0xca, + 0xdd, 0x3f, 0x07, 0x94, 0xf3, 0xe5, 0x0f, 0xc3, 0x72, 0xad, 0x33, 0xb1, 0xdc, 0xfd, 0x73, 0x60, + 0xb9, 0x20, 0xbb, 0x21, 0x60, 0xee, 0xe3, 0x7e, 0x30, 0x27, 0x8d, 0x02, 0x73, 0x81, 0x76, 0xce, + 0x8d, 0xe6, 0x5e, 0x13, 0x5f, 0xef, 0xc1, 0x74, 0x7f, 0x33, 0x03, 0x53, 0xd5, 0x88, 0x33, 0x41, + 0xe1, 0x55, 0xce, 0x04, 0xd1, 0x77, 0xe1, 0xaa, 0xff, 0x40, 0xfd, 0xb9, 0x14, 0x1e, 0x12, 0xa7, + 0x35, 0x2d, 0xed, 0x90, 0x2e, 0x95, 0x69, 0x79, 0xde, 0x4f, 0xf2, 0xc8, 0xb6, 0x5a, 0x2c, 0x34, + 0x8e, 0x7a, 0xc9, 0xa0, 0x55, 0x32, 0x86, 0xa9, 0x4d, 0x78, 0xf6, 0x8d, 0xa3, 0x83, 0x77, 0x62, + 0x72, 0xd6, 0x57, 0xb8, 0xc4, 0x01, 0xbd, 0x0b, 0xb9, 0x8e, 0x83, 0x6d, 0xa5, 0x6d, 0x1b, 0x96, + 0x6d, 0xb8, 0x2c, 0x1c, 0x4d, 0x28, 0x89, 0x5f, 0x9d, 0x2c, 0xcd, 0xec, 0x38, 0xd8, 0xde, 0xe6, + 0x74, 0x79, 0xa6, 0x13, 0x7a, 0xf2, 0x3e, 0xfd, 0x37, 0x39, 0xfe, 0xa7, 0xff, 0x9e, 0x82, 0x48, + 0x3d, 0x74, 0xc2, 0xcb, 0x28, 0xbb, 0xa7, 0x2d, 0x7a, 0xc5, 0x57, 0xf5, 0xd0, 0x4a, 0x49, 0xef, + 0x6b, 0x2b, 0xd8, 0xbd, 0x44, 0xf4, 0x18, 0xf2, 0xb6, 0xd5, 0xa1, 0xd7, 0x31, 0xb5, 0xad, 0xa6, + 0xa1, 0x75, 0xa9, 0x7d, 0x93, 0x8f, 0x3e, 0x6f, 0x66, 0x09, 0xb7, 0x69, 0x3a, 0x39, 0x67, 0x87, + 0x1f, 0x51, 0x1d, 0xe8, 0x55, 0x4c, 0x9e, 0x14, 0x34, 0x70, 0xf9, 0xfd, 0xc8, 0x4f, 0x50, 0x3c, + 0x57, 0x0d, 0x97, 0xcb, 0x85, 0x97, 0xfe, 0x6f, 0xf4, 0x08, 0x66, 0xe8, 0xb5, 0xe0, 0xa4, 0xcd, + 0xad, 0x8e, 0xcb, 0xcd, 0x9b, 0x2b, 0x45, 0xf6, 0x81, 0xc8, 0xa2, 0xf7, 0x81, 0xc8, 0xe2, 0x2a, + 0xff, 0x40, 0x24, 0x3b, 0xab, 0xfe, 0xf1, 0xbf, 0x5f, 0x12, 0xe4, 0xac, 0xe7, 0x39, 0x65, 0x75, + 0x5c, 0x74, 0x1f, 0x2e, 0xb6, 0xd4, 0x63, 0x7a, 0xbb, 0xb8, 0xe2, 0xd9, 0x5f, 0xf4, 0x3e, 0x45, + 0xf6, 0x55, 0x42, 0xd4, 0x52, 0x8f, 0xe9, 0xd7, 0x12, 0xd9, 0x2b, 0xfa, 0xa9, 0xa3, 0xeb, 0x30, + 0xc3, 0x63, 0x85, 0xd8, 0x97, 0xd0, 0x0a, 0x34, 0x25, 0xff, 0x2c, 0x0e, 0xfb, 0x18, 0xda, 0x4d, + 0xc8, 0xeb, 0x86, 0xe3, 0x1a, 0xa6, 0xe6, 0xf2, 0x8b, 0xcb, 0xd9, 0xd5, 0xdf, 0x39, 0x8f, 0xca, + 0x6e, 0x27, 0x6f, 0xc0, 0xac, 0xd6, 0x34, 0x7c, 0xab, 0x96, 0xd9, 0x19, 0xb3, 0x43, 0xc7, 0x69, + 0x99, 0xa6, 0xed, 0x77, 0xa0, 0x29, 0x68, 0xbd, 0x64, 0x54, 0x86, 0xc2, 0xbe, 0xea, 0xe2, 0x97, + 0x6a, 0x57, 0xf1, 0xc2, 0x70, 0xb3, 0xf4, 0xea, 0x81, 0xab, 0xa7, 0x27, 0x4b, 0xb9, 0xc7, 0xec, + 0xd5, 0x40, 0x34, 0x6e, 0x6e, 0x3f, 0xf4, 0x42, 0x47, 0xb7, 0xa1, 0xa0, 0x3a, 0x5d, 0x53, 0xa3, + 0x3d, 0x0a, 0x9b, 0x4e, 0xc7, 0xe1, 0xe1, 0x68, 0x79, 0x4a, 0x2e, 0x7b, 0x54, 0xf4, 0x21, 0x2c, + 0xf0, 0xef, 0x93, 0xbc, 0x54, 0x6d, 0x5d, 0xa1, 0xbd, 0x30, 0x18, 0xee, 0x22, 0xe5, 0xb9, 0xcc, + 0xbe, 0x47, 0x42, 0x12, 0x90, 0xae, 0x17, 0xcc, 0x86, 0xcf, 0x60, 0x96, 0x06, 0xa5, 0x61, 0x9d, + 0x2c, 0x47, 0x4d, 0x6c, 0x62, 0xc7, 0x19, 0x71, 0xca, 0x50, 0x62, 0x69, 0xeb, 0x5e, 0x52, 0x36, + 0xc1, 0xc8, 0xe2, 0x6e, 0x1f, 0xdd, 0xbf, 0x2f, 0x1d, 0xc4, 0xec, 0x5a, 0x2a, 0x3d, 0x23, 0xe6, + 0xd6, 0x52, 0xe9, 0xbc, 0x58, 0x90, 0xfe, 0xaf, 0x00, 0x97, 0xa2, 0x85, 0xa0, 0x3a, 0xcc, 0xf5, + 0x5c, 0x09, 0xc4, 0xa2, 0xe5, 0xce, 0x33, 0x5f, 0xcd, 0x86, 0x2f, 0x06, 0xa2, 0xb9, 0x10, 0xe5, + 0x44, 0x08, 0x55, 0x1c, 0xd7, 0x36, 0x34, 0xef, 0x9a, 0xad, 0xcb, 0x03, 0x6c, 0x75, 0xfa, 0x9a, + 0x96, 0x48, 0x3d, 0x1e, 0x28, 0x51, 0xf2, 0x3c, 0x25, 0x52, 0x8f, 0x7b, 0x45, 0x4b, 0x3f, 0x4c, + 0x42, 0x81, 0xac, 0xbc, 0x8e, 0x63, 0x58, 0x66, 0xd5, 0x77, 0xd0, 0xf7, 0x27, 0x28, 0x81, 0xc6, + 0x9a, 0xfa, 0xcf, 0x68, 0x89, 0xc6, 0xf4, 0x12, 0x84, 0xe3, 0x7f, 0xef, 0x29, 0x29, 0x03, 0x23, + 0xd1, 0xa8, 0xc5, 0x15, 0x98, 0x72, 0xac, 0x8e, 0xad, 0x79, 0x5f, 0x21, 0xb9, 0x3b, 0x64, 0xa9, + 0x0f, 0x65, 0x58, 0xac, 0x53, 0x06, 0x99, 0x33, 0xa2, 0x4f, 0xa1, 0xc0, 0x7e, 0xd1, 0x58, 0x42, + 0x1a, 0xcb, 0xc7, 0x02, 0x32, 0xef, 0x8f, 0x2d, 0x6b, 0x83, 0x33, 0xca, 0x79, 0xa7, 0xe7, 0x19, + 0x7d, 0x0c, 0xaf, 0x99, 0x96, 0xd2, 0xc2, 0x2d, 0x8b, 0x2d, 0xf1, 0x64, 0x96, 0xd1, 0x15, 0xd5, + 0x55, 0x78, 0xa1, 0x99, 0xc7, 0xf7, 0xbc, 0x69, 0x6d, 0xd2, 0x24, 0x32, 0x4f, 0xb1, 0xe2, 0x32, + 0xb9, 0x52, 0x11, 0xa6, 0xd8, 0x2f, 0x94, 0x81, 0xc9, 0x27, 0x8d, 0x6a, 0x45, 0x16, 0x27, 0xd0, + 0x0c, 0xa4, 0x1f, 0xc9, 0x4f, 0x36, 0x95, 0xfa, 0xd3, 0x0d, 0x51, 0x40, 0x59, 0x98, 0x96, 0x9f, + 0x3c, 0x69, 0x28, 0xeb, 0xcf, 0xc4, 0x84, 0x74, 0x1b, 0xf2, 0xbd, 0x25, 0x42, 0x00, 0x53, 0x72, + 0x65, 0xf3, 0x09, 0xfd, 0xf4, 0x46, 0x06, 0x26, 0x37, 0x9e, 0x94, 0x57, 0x36, 0x44, 0x41, 0xfa, + 0x13, 0x01, 0x66, 0x4a, 0xec, 0x5b, 0x32, 0xcc, 0xaf, 0xe7, 0xc3, 0x3e, 0xff, 0x9b, 0x2b, 0xd1, + 0x5b, 0x08, 0xc3, 0xfc, 0x6e, 0xd2, 0x7c, 0xf6, 0xf2, 0x62, 0xce, 0x96, 0x86, 0x03, 0x47, 0xba, + 0xc7, 0xea, 0x39, 0xfa, 0x7a, 0x6c, 0xa8, 0x0e, 0xa2, 0xea, 0xe9, 0x56, 0xe1, 0x25, 0x19, 0xee, + 0xee, 0xdb, 0xd7, 0x0c, 0xde, 0x5c, 0xa4, 0xf6, 0x92, 0x3f, 0x48, 0xfd, 0xf8, 0x8b, 0xa5, 0x09, + 0xe9, 0x4f, 0x53, 0x90, 0x2b, 0x85, 0xbf, 0x9b, 0x83, 0x6a, 0x7d, 0x95, 0xbd, 0x1d, 0x69, 0x96, + 0x84, 0x38, 0x8a, 0x23, 0xbe, 0x48, 0x96, 0x09, 0x3e, 0xd2, 0xc3, 0xea, 0x7e, 0x6d, 0x84, 0xeb, + 0x52, 0xb8, 0xf2, 0x01, 0xe3, 0xc2, 0xbf, 0x49, 0xfa, 0xf6, 0x4a, 0x11, 0x26, 0x59, 0x4c, 0xb5, + 0x30, 0x70, 0xdd, 0x0b, 0x5d, 0xa9, 0x08, 0x5c, 0x23, 0xef, 0x65, 0x96, 0x8c, 0xd8, 0x37, 0x8d, + 0x57, 0xf2, 0x79, 0x0a, 0xe6, 0xc1, 0xf3, 0x7f, 0xeb, 0xb7, 0xc3, 0x2e, 0x37, 0xfe, 0xff, 0xe8, + 0x61, 0x4c, 0xf2, 0x43, 0xbf, 0x0a, 0x05, 0xcd, 0x6a, 0x36, 0x99, 0x95, 0xcf, 0x56, 0xb6, 0xc1, + 0xeb, 0xce, 0x68, 0x11, 0xf8, 0xe7, 0x9d, 0x8b, 0xfe, 0x67, 0x9e, 0x8b, 0x32, 0xff, 0xcc, 0x73, + 0x28, 0xfc, 0x2b, 0xef, 0x0b, 0x63, 0x0b, 0x62, 0x5f, 0x24, 0xda, 0xf4, 0xab, 0x44, 0xa2, 0xb1, + 0xf8, 0x3d, 0xde, 0xf3, 0x7e, 0x2e, 0x70, 0x3f, 0xe0, 0x0d, 0xcb, 0x3a, 0xec, 0xf8, 0xde, 0x95, + 0x0b, 0xe1, 0xab, 0x8a, 0x83, 0x20, 0x19, 0x1a, 0x33, 0x1a, 0x65, 0x4a, 0x25, 0xbe, 0x9e, 0x29, + 0x75, 0x1d, 0x66, 0xda, 0x36, 0xde, 0xc3, 0xae, 0x76, 0xa0, 0x98, 0x9d, 0x16, 0x0f, 0x98, 0xcd, + 0x7a, 0xb4, 0xad, 0x4e, 0x0b, 0xdd, 0x05, 0xd1, 0x4f, 0xc2, 0x77, 0x5e, 0xbc, 0x7b, 0x32, 0x3d, + 0x3a, 0xdf, 0xa7, 0x91, 0xfe, 0xbb, 0x00, 0x73, 0x3d, 0x75, 0xe2, 0x63, 0x6a, 0x0d, 0xb2, 0xba, + 0x6f, 0xbc, 0x3a, 0xf3, 0xc2, 0x39, 0xc3, 0xa7, 0xc2, 0xcc, 0x48, 0x81, 0x4b, 0x5e, 0xb6, 0xf4, + 0xc3, 0x36, 0x81, 0xd8, 0xc4, 0x39, 0xc5, 0x5e, 0x0c, 0xe4, 0xac, 0x86, 0x32, 0xf0, 0x07, 0x59, + 0x72, 0xac, 0x41, 0x26, 0xfd, 0x4f, 0x01, 0x44, 0x9a, 0xc1, 0x23, 0x8c, 0xf5, 0x58, 0xa6, 0x4c, + 0x2f, 0x4e, 0x31, 0x31, 0x7e, 0x48, 0x6c, 0xcf, 0xc7, 0xb8, 0x92, 0x7d, 0x1f, 0xe3, 0x8a, 0x9a, + 0x3f, 0x53, 0x5f, 0x73, 0xfe, 0x94, 0xbe, 0x10, 0x20, 0xef, 0x57, 0x9b, 0x7d, 0x85, 0x77, 0xc4, + 0x35, 0xdb, 0xaf, 0xf6, 0xa5, 0x59, 0xef, 0x3a, 0xb0, 0xb1, 0x3e, 0x0c, 0x1c, 0xbe, 0x0e, 0x8c, + 0x7d, 0x21, 0xf5, 0x6f, 0x7b, 0xdd, 0x91, 0x14, 0xb1, 0x1c, 0xdc, 0xc3, 0xf4, 0x0a, 0x21, 0xc7, + 0xdf, 0x84, 0x23, 0xf1, 0xa3, 0x90, 0x02, 0x69, 0x8f, 0x22, 0x5a, 0x1a, 0x6b, 0x7e, 0xf7, 0xb4, + 0xc4, 0x3a, 0xe0, 0x1f, 0x84, 0x5b, 0x82, 0xdd, 0xe3, 0xf1, 0x10, 0x92, 0x47, 0x6a, 0x73, 0x94, + 0xbb, 0x6c, 0x4f, 0xcb, 0xc9, 0x24, 0x35, 0x7a, 0xd4, 0x73, 0x7d, 0x55, 0x62, 0xf8, 0xae, 0xec, + 0xa0, 0x4a, 0x7b, 0xae, 0xb9, 0x7a, 0xaf, 0x77, 0x00, 0x8d, 0xcc, 0x3e, 0x3c, 0x92, 0x3e, 0x48, + 0x7d, 0xf9, 0xc5, 0x92, 0x20, 0x7d, 0x04, 0x88, 0xd8, 0x3a, 0xee, 0xd3, 0x8e, 0x65, 0x07, 0x57, + 0x81, 0xf5, 0xc7, 0x23, 0x4e, 0x46, 0xc7, 0x23, 0x4a, 0x17, 0x61, 0xae, 0x87, 0x9b, 0xcd, 0x40, + 0xd2, 0x7b, 0x70, 0xe5, 0xb1, 0xe5, 0x38, 0x46, 0xbb, 0xde, 0xd9, 0x65, 0x43, 0x9d, 0xac, 0x57, + 0xfe, 0x9c, 0x9b, 0x6e, 0xd3, 0x2d, 0x3f, 0x93, 0xcd, 0x4d, 0x19, 0xd9, 0x7f, 0x96, 0x7e, 0x4f, + 0x80, 0xcb, 0x83, 0x9c, 0x4c, 0xcb, 0x51, 0x37, 0x24, 0x4c, 0x6b, 0x56, 0x70, 0x53, 0xed, 0xd9, + 0xbd, 0xd5, 0x4b, 0x4e, 0x50, 0x0d, 0xcf, 0x53, 0x69, 0xa9, 0x74, 0x4e, 0xe2, 0x17, 0xa7, 0xe4, + 0x39, 0x79, 0x93, 0x51, 0x83, 0xe9, 0x29, 0x35, 0xde, 0xf4, 0xf4, 0xbf, 0x04, 0x98, 0x6d, 0x60, + 0x53, 0x35, 0x5d, 0x32, 0xef, 0x77, 0x5a, 0xec, 0x56, 0x89, 0x02, 0x24, 0x6d, 0xa5, 0x43, 0x8b, + 0x2e, 0xc8, 0x09, 0x7b, 0x07, 0xdd, 0x80, 0x1c, 0x5d, 0x5b, 0x42, 0xb6, 0x9a, 0x70, 0x27, 0x25, + 0xd3, 0xe8, 0x1a, 0xd9, 0x33, 0xc4, 0x5e, 0x07, 0xa0, 0x89, 0x18, 0xba, 0x4c, 0xd2, 0x14, 0x19, + 0x42, 0xf1, 0xb1, 0x25, 0x8d, 0x56, 0x0c, 0x84, 0xb0, 0x80, 0xa4, 0x1c, 0xa5, 0xfa, 0x52, 0x96, + 0x20, 0xcb, 0x92, 0x31, 0x31, 0x93, 0x34, 0x0d, 0x50, 0x12, 0x93, 0xf3, 0x08, 0x2e, 0x38, 0x2f, + 0x9a, 0x4a, 0xdb, 0xd2, 0x1d, 0x45, 0x6b, 0x77, 0x78, 0xa4, 0x07, 0xfb, 0xb0, 0xb7, 0x50, 0xba, + 0x78, 0x7a, 0xb2, 0x34, 0x5b, 0x7f, 0xba, 0xb1, 0x6d, 0xe9, 0x4e, 0x79, 0x7b, 0x87, 0xc5, 0x79, + 0x38, 0xf2, 0xac, 0xf3, 0xa2, 0x49, 0x49, 0xed, 0x0e, 0x27, 0x49, 0x9f, 0x27, 0x00, 0xd1, 0x9b, + 0x91, 0x4a, 0xf4, 0x6a, 0x21, 0xaf, 0xb9, 0x2d, 0x58, 0xd4, 0x02, 0x55, 0x28, 0x8e, 0x61, 0x12, + 0xfb, 0x5e, 0x75, 0x5c, 0xaf, 0xdc, 0x7c, 0xdc, 0x44, 0x5e, 0xe8, 0xda, 0xaf, 0x49, 0xde, 0x8e, + 0x57, 0x43, 0x12, 0xeb, 0x44, 0xe0, 0x86, 0xea, 0xf8, 0x19, 0xde, 0x85, 0x8c, 0x4b, 0xf9, 0xbc, + 0x7b, 0xa7, 0x52, 0xa5, 0x99, 0xd3, 0x93, 0xa5, 0x34, 0x13, 0x56, 0x5b, 0x95, 0xd3, 0xec, 0x75, + 0x4d, 0x47, 0xcb, 0x90, 0x35, 0x4c, 0xc7, 0x55, 0x49, 0x91, 0xf8, 0x66, 0x4e, 0x8e, 0x5d, 0x0c, + 0x50, 0xe3, 0xe4, 0xda, 0xaa, 0x0c, 0x5e, 0x12, 0x1a, 0x7f, 0x9b, 0xf7, 0x19, 0xd8, 0xe9, 0x0c, + 0x0d, 0x16, 0x97, 0x73, 0x1e, 0x95, 0xdd, 0x8c, 0x58, 0x87, 0xb9, 0x1e, 0x4d, 0xf0, 0x85, 0xf9, + 0xa3, 0xde, 0x09, 0x27, 0x6c, 0x9d, 0x52, 0xba, 0x43, 0x0f, 0x9c, 0x35, 0x4b, 0xe7, 0x23, 0xb5, + 0x77, 0xe2, 0x69, 0x40, 0x61, 0xcd, 0x32, 0x4c, 0x82, 0xcb, 0xbd, 0xaa, 0xae, 0x40, 0x7e, 0xd7, + 0x30, 0x55, 0xbb, 0xab, 0x78, 0x81, 0x2d, 0xc2, 0x59, 0x81, 0x2d, 0x72, 0x8e, 0x71, 0xf0, 0x47, + 0xe9, 0x67, 0x02, 0x88, 0x81, 0x58, 0x5e, 0xd0, 0x37, 0x01, 0xb4, 0x66, 0xc7, 0x71, 0xb1, 0xed, + 0x4d, 0x00, 0x33, 0x2c, 0x80, 0xb6, 0xcc, 0xa8, 0xb5, 0x55, 0x39, 0xc3, 0x13, 0xd4, 0x74, 0x74, + 0xa3, 0xf7, 0x9a, 0xaf, 0xc9, 0x12, 0x9c, 0x0e, 0x5c, 0xee, 0x45, 0x66, 0x14, 0xc7, 0xb5, 0x6c, + 0x5f, 0xcf, 0x7c, 0x46, 0xf1, 0x2e, 0x40, 0xa4, 0x17, 0xfd, 0x60, 0x1a, 0x23, 0x9f, 0x27, 0xe6, + 0xed, 0x11, 0xf6, 0xab, 0x94, 0x3a, 0xbb, 0x4a, 0x8c, 0xc3, 0xab, 0xd2, 0xef, 0x0a, 0x50, 0x28, + 0xb3, 0x81, 0xee, 0x4f, 0x1e, 0x23, 0x16, 0xcb, 0x55, 0x48, 0xbb, 0xc7, 0xa6, 0xd2, 0xc2, 0xfe, + 0x37, 0xfb, 0xce, 0x71, 0x43, 0xf1, 0xb4, 0xcb, 0x1e, 0xe9, 0x67, 0xa0, 0xf9, 0x16, 0x13, 0x9f, + 0x89, 0xc7, 0xda, 0x83, 0xf2, 0x99, 0x98, 0x9d, 0x7a, 0xaf, 0x4e, 0x26, 0xd4, 0x01, 0x4b, 0x12, + 0xe5, 0x01, 0x42, 0x1f, 0x63, 0x9c, 0xa0, 0x9f, 0x68, 0xac, 0xac, 0xac, 0x2a, 0x3b, 0x5b, 0xe5, + 0x27, 0x9b, 0x9b, 0xb5, 0x46, 0xa3, 0xb2, 0x2a, 0x0a, 0x48, 0x84, 0x99, 0x9e, 0x4f, 0x39, 0x26, + 0x16, 0x52, 0x9f, 0xff, 0x74, 0x71, 0xe2, 0xde, 0x5b, 0x90, 0xeb, 0xd9, 0x98, 0x43, 0x05, 0xc8, + 0x6e, 0x54, 0x56, 0xea, 0x95, 0xea, 0x93, 0x8d, 0x55, 0x0a, 0x64, 0xb3, 0x30, 0xbd, 0x55, 0x59, + 0x91, 0x2b, 0xf5, 0x86, 0x28, 0xdc, 0xfb, 0x0b, 0x00, 0xc1, 0x47, 0x64, 0x49, 0xd6, 0xeb, 0x95, + 0x4f, 0x94, 0x67, 0x2b, 0x1b, 0x3b, 0x95, 0xba, 0x38, 0x81, 0x10, 0xe4, 0x4b, 0x2b, 0x8d, 0x72, + 0x55, 0x91, 0x2b, 0xf5, 0xed, 0x27, 0x5b, 0xf5, 0x8a, 0x28, 0xf0, 0x6c, 0x56, 0x61, 0x26, 0x7c, + 0x4d, 0x23, 0x9a, 0x83, 0x42, 0xb9, 0x5a, 0x29, 0xaf, 0x2b, 0xcf, 0x6a, 0x2b, 0xca, 0xd3, 0x9d, + 0xca, 0x0e, 0x41, 0xbe, 0xa4, 0x26, 0x94, 0xf8, 0x68, 0x67, 0x83, 0x80, 0xe6, 0x02, 0x64, 0xd9, + 0x33, 0xfd, 0x4a, 0xa4, 0x98, 0xb8, 0xb7, 0x09, 0xd9, 0xd0, 0xe7, 0x23, 0x48, 0x76, 0xdb, 0x3b, + 0xf5, 0xaa, 0xd2, 0xa8, 0x6d, 0x56, 0xea, 0x8d, 0x95, 0xcd, 0x6d, 0x26, 0x83, 0xd2, 0x56, 0x4a, + 0x4f, 0xe4, 0x86, 0x28, 0xf8, 0xcf, 0x8d, 0x27, 0x3b, 0xe5, 0xaa, 0x57, 0x6b, 0x29, 0x95, 0x4e, + 0x8a, 0xc9, 0x7b, 0xc7, 0x70, 0x79, 0xc8, 0x8d, 0x85, 0xa4, 0xd2, 0x3b, 0x26, 0xbd, 0x4a, 0x5f, + 0x9c, 0x40, 0x39, 0xc8, 0x90, 0x9e, 0x4a, 0x2f, 0x01, 0x11, 0x05, 0x94, 0x86, 0xd4, 0x81, 0xeb, + 0xb6, 0xc5, 0x04, 0x9a, 0x82, 0x84, 0xf3, 0x50, 0x4c, 0x92, 0xff, 0xfb, 0x8e, 0x98, 0x22, 0xd0, + 0x5d, 0xfd, 0xac, 0x63, 0x63, 0x71, 0x92, 0xc0, 0xff, 0x8e, 0x83, 0xed, 0x3d, 0xa3, 0x89, 0xc5, + 0x69, 0xc2, 0x62, 0x76, 0x9a, 0x4d, 0x31, 0x2d, 0xa5, 0xd2, 0x53, 0xe2, 0xd4, 0xbd, 0xeb, 0x10, + 0xba, 0x24, 0x8a, 0xa0, 0xff, 0x0d, 0xd5, 0xc5, 0x8e, 0x2b, 0x4e, 0xa0, 0x69, 0x48, 0xae, 0x34, + 0x9b, 0xa2, 0xf0, 0xe0, 0x7f, 0x4c, 0x41, 0xda, 0xfb, 0x08, 0x22, 0xda, 0x80, 0x49, 0x8a, 0x74, + 0xd1, 0xd2, 0x70, 0x0c, 0x4c, 0x87, 0xfd, 0xc2, 0xb5, 0xb3, 0x40, 0xb2, 0x34, 0x81, 0xfe, 0x22, + 0x64, 0x43, 0xd8, 0x00, 0x0d, 0x3d, 0xd8, 0xe9, 0xc1, 0x43, 0x0b, 0xb7, 0xce, 0x4a, 0xe6, 0xcb, + 0x7f, 0x0e, 0x19, 0xdf, 0xac, 0x40, 0x37, 0x46, 0x19, 0x1d, 0x9e, 0xec, 0xd1, 0x96, 0x09, 0x19, + 0xa5, 0xd2, 0xc4, 0xdb, 0x02, 0xb2, 0x01, 0x0d, 0x5a, 0x00, 0x28, 0xca, 0xd3, 0x7b, 0xa8, 0x89, + 0xb1, 0x70, 0x6f, 0xac, 0xd4, 0x41, 0x9e, 0x44, 0x59, 0x81, 0x19, 0x13, 0xad, 0xac, 0x01, 0x23, + 0x29, 0x5a, 0x59, 0x11, 0xd6, 0x10, 0x6d, 0x8c, 0xd0, 0x7a, 0x10, 0x29, 0x7f, 0x70, 0xe5, 0x8c, + 0x94, 0x1f, 0xb1, 0xac, 0x48, 0x13, 0xe8, 0x29, 0xa4, 0xc8, 0x1c, 0x8e, 0xa2, 0x00, 0x46, 0xdf, + 0x9a, 0xb1, 0x70, 0x63, 0x64, 0x1a, 0x5f, 0xe4, 0x3e, 0xe4, 0x1f, 0x63, 0xba, 0x3d, 0xcd, 0x6e, + 0x7c, 0x74, 0xd0, 0x9d, 0x68, 0xf7, 0xbb, 0x50, 0x12, 0x2f, 0x8b, 0xbb, 0x63, 0xa4, 0xf4, 0x33, + 0x6a, 0xc3, 0x2c, 0xfb, 0x56, 0x46, 0x38, 0xaf, 0xa8, 0x33, 0xff, 0x81, 0x54, 0x5e, 0x76, 0x6f, + 0x8e, 0x97, 0xd8, 0xcb, 0xb1, 0x74, 0xf7, 0xcb, 0xff, 0xb0, 0x38, 0xf1, 0xe5, 0xe9, 0xa2, 0xf0, + 0xb3, 0xd3, 0x45, 0xe1, 0x8f, 0x4e, 0x17, 0x85, 0x3f, 0x3e, 0x5d, 0x14, 0x7e, 0xf4, 0x8b, 0xc5, + 0x89, 0x9f, 0xfd, 0x62, 0x71, 0xe2, 0x8f, 0x7e, 0xb1, 0x38, 0xf1, 0xe9, 0x34, 0x17, 0xb3, 0x3b, + 0x45, 0xe7, 0xee, 0x87, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x45, 0x51, 0x7a, 0x3b, 0xf9, 0x8b, + 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -8403,6 +8407,12 @@ type InternalClient interface { // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. Join(ctx context.Context, in *JoinNodeRequest, opts ...grpc.CallOption) (*JoinNodeResponse, error) + // GetSpanConfigs is used to fetch the span configurations over a given + // keyspan. + GetSpanConfigs(ctx context.Context, in *GetSpanConfigsRequest, opts ...grpc.CallOption) (*GetSpanConfigsResponse, error) + // UpdateSpanConfigs is used to update the span configurations over given + // keyspans. + UpdateSpanConfigs(ctx context.Context, in *UpdateSpanConfigsRequest, opts ...grpc.CallOption) (*UpdateSpanConfigsResponse, error) } type internalClient struct { @@ -8522,6 +8532,24 @@ func (c *internalClient) Join(ctx context.Context, in *JoinNodeRequest, opts ... return out, nil } +func (c *internalClient) GetSpanConfigs(ctx context.Context, in *GetSpanConfigsRequest, opts ...grpc.CallOption) (*GetSpanConfigsResponse, error) { + out := new(GetSpanConfigsResponse) + err := c.cc.Invoke(ctx, "/cockroach.roachpb.Internal/GetSpanConfigs", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *internalClient) UpdateSpanConfigs(ctx context.Context, in *UpdateSpanConfigsRequest, opts ...grpc.CallOption) (*UpdateSpanConfigsResponse, error) { + out := new(UpdateSpanConfigsResponse) + err := c.cc.Invoke(ctx, "/cockroach.roachpb.Internal/UpdateSpanConfigs", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // InternalServer is the server API for Internal service. type InternalServer interface { Batch(context.Context, *BatchRequest) (*BatchResponse, error) @@ -8535,6 +8563,12 @@ type InternalServer interface { // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. Join(context.Context, *JoinNodeRequest) (*JoinNodeResponse, error) + // GetSpanConfigs is used to fetch the span configurations over a given + // keyspan. + GetSpanConfigs(context.Context, *GetSpanConfigsRequest) (*GetSpanConfigsResponse, error) + // UpdateSpanConfigs is used to update the span configurations over given + // keyspans. + UpdateSpanConfigs(context.Context, *UpdateSpanConfigsRequest) (*UpdateSpanConfigsResponse, error) } // UnimplementedInternalServer can be embedded to have forward compatible implementations. @@ -8562,6 +8596,12 @@ func (*UnimplementedInternalServer) TokenBucket(ctx context.Context, req *TokenB func (*UnimplementedInternalServer) Join(ctx context.Context, req *JoinNodeRequest) (*JoinNodeResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Join not implemented") } +func (*UnimplementedInternalServer) GetSpanConfigs(ctx context.Context, req *GetSpanConfigsRequest) (*GetSpanConfigsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSpanConfigs not implemented") +} +func (*UnimplementedInternalServer) UpdateSpanConfigs(ctx context.Context, req *UpdateSpanConfigsRequest) (*UpdateSpanConfigsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateSpanConfigs not implemented") +} func RegisterInternalServer(s *grpc.Server, srv InternalServer) { s.RegisterService(&_Internal_serviceDesc, srv) @@ -8699,6 +8739,42 @@ func _Internal_Join_Handler(srv interface{}, ctx context.Context, dec func(inter return interceptor(ctx, in, info, handler) } +func _Internal_GetSpanConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSpanConfigsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(InternalServer).GetSpanConfigs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.roachpb.Internal/GetSpanConfigs", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(InternalServer).GetSpanConfigs(ctx, req.(*GetSpanConfigsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Internal_UpdateSpanConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateSpanConfigsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(InternalServer).UpdateSpanConfigs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.roachpb.Internal/UpdateSpanConfigs", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(InternalServer).UpdateSpanConfigs(ctx, req.(*UpdateSpanConfigsRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Internal_serviceDesc = grpc.ServiceDesc{ ServiceName: "cockroach.roachpb.Internal", HandlerType: (*InternalServer)(nil), @@ -8723,6 +8799,14 @@ var _Internal_serviceDesc = grpc.ServiceDesc{ MethodName: "Join", Handler: _Internal_Join_Handler, }, + { + MethodName: "GetSpanConfigs", + Handler: _Internal_GetSpanConfigs_Handler, + }, + { + MethodName: "UpdateSpanConfigs", + Handler: _Internal_UpdateSpanConfigs_Handler, + }, }, Streams: []grpc.StreamDesc{ { diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 2218207a5dcc..ae159ac4a3ae 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -18,6 +18,7 @@ import "kv/kvserver/readsummary/rspb/summary.proto"; import "roachpb/data.proto"; import "roachpb/errors.proto"; import "roachpb/metadata.proto"; +import "roachpb/span_config.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/mvcc3.proto"; import "util/hlc/timestamp.proto"; @@ -2527,6 +2528,14 @@ service Internal { // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. rpc Join(JoinNodeRequest) returns (JoinNodeResponse) { } + + // GetSpanConfigs is used to fetch the span configurations over a given + // keyspan. + rpc GetSpanConfigs (GetSpanConfigsRequest) returns (GetSpanConfigsResponse) { } + + // UpdateSpanConfigs is used to update the span configurations over given + // keyspans. + rpc UpdateSpanConfigs (UpdateSpanConfigsRequest) returns (UpdateSpanConfigsResponse) { } } // ContentionEvent is a message that will be attached to BatchResponses diff --git a/pkg/roachpb/mocks_generated.go b/pkg/roachpb/mocks_generated.go index 3fc8d811d546..7344ee7bb366 100644 --- a/pkg/roachpb/mocks_generated.go +++ b/pkg/roachpb/mocks_generated.go @@ -56,6 +56,26 @@ func (mr *MockInternalClientMockRecorder) Batch(arg0, arg1 interface{}, arg2 ... return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Batch", reflect.TypeOf((*MockInternalClient)(nil).Batch), varargs...) } +// GetSpanConfigs mocks base method. +func (m *MockInternalClient) GetSpanConfigs(arg0 context.Context, arg1 *GetSpanConfigsRequest, arg2 ...grpc.CallOption) (*GetSpanConfigsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetSpanConfigs", varargs...) + ret0, _ := ret[0].(*GetSpanConfigsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetSpanConfigs indicates an expected call of GetSpanConfigs. +func (mr *MockInternalClientMockRecorder) GetSpanConfigs(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{arg0, arg1}, arg2...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).GetSpanConfigs), varargs...) +} + // GossipSubscription mocks base method. func (m *MockInternalClient) GossipSubscription(arg0 context.Context, arg1 *GossipSubscriptionRequest, arg2 ...grpc.CallOption) (Internal_GossipSubscriptionClient, error) { m.ctrl.T.Helper() @@ -176,6 +196,26 @@ func (mr *MockInternalClientMockRecorder) TokenBucket(arg0, arg1 interface{}, ar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TokenBucket", reflect.TypeOf((*MockInternalClient)(nil).TokenBucket), varargs...) } +// UpdateSpanConfigs mocks base method. +func (m *MockInternalClient) UpdateSpanConfigs(arg0 context.Context, arg1 *UpdateSpanConfigsRequest, arg2 ...grpc.CallOption) (*UpdateSpanConfigsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateSpanConfigs", varargs...) + ret0, _ := ret[0].(*UpdateSpanConfigsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateSpanConfigs indicates an expected call of UpdateSpanConfigs. +func (mr *MockInternalClientMockRecorder) UpdateSpanConfigs(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{arg0, arg1}, arg2...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSpanConfigs", reflect.TypeOf((*MockInternalClient)(nil).UpdateSpanConfigs), varargs...) +} + // MockInternal_RangeFeedClient is a mock of Internal_RangeFeedClient interface. type MockInternal_RangeFeedClient struct { ctrl *gomock.Controller diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go new file mode 100644 index 000000000000..c506586314c4 --- /dev/null +++ b/pkg/roachpb/span_config.go @@ -0,0 +1,16 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package roachpb + +// Equal compares two span config entries. +func (e SpanConfigEntry) Equal(other SpanConfigEntry) bool { + return e.Span.Equal(other.Span) && e.Config.Equal(other.Config) +} diff --git a/pkg/roachpb/span_config.pb.go b/pkg/roachpb/span_config.pb.go new file mode 100644 index 000000000000..159b3ec02002 --- /dev/null +++ b/pkg/roachpb/span_config.pb.go @@ -0,0 +1,2392 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: roachpb/span_config.proto + +package roachpb + +import ( + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type Constraint_Type int32 + +const ( + // REQUIRED ensures all replicas are placed on stores with locality tags + // that match the constraint. Replication will fail if there aren't any such + // stores. + Constraint_REQUIRED Constraint_Type = 0 + // PROHIBITED prevents replicas from being placed on stores with locality + // tags that match the constrain. + Constraint_PROHIBITED Constraint_Type = 1 +) + +var Constraint_Type_name = map[int32]string{ + 0: "REQUIRED", + 1: "PROHIBITED", +} + +var Constraint_Type_value = map[string]int32{ + "REQUIRED": 0, + "PROHIBITED": 1, +} + +func (x Constraint_Type) String() string { + return proto.EnumName(Constraint_Type_name, int32(x)) +} + +func (Constraint_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{0, 0} +} + +// Constraint constrains the stores that a replica can be stored on. It +// parallels the definition found in zonepb/zone.proto. +type Constraint struct { + // Type captures the kind of constraint this is: required or prohibited. + Type Constraint_Type `protobuf:"varint,1,opt,name=type,proto3,enum=cockroach.roachpb.Constraint_Type" json:"type,omitempty"` + // Key captures the locality tag key we're constraining against. + Key string `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + // Value is the locality tag value we're constraining against. + Value string `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` +} + +func (m *Constraint) Reset() { *m = Constraint{} } +func (m *Constraint) String() string { return proto.CompactTextString(m) } +func (*Constraint) ProtoMessage() {} +func (*Constraint) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{0} +} +func (m *Constraint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Constraint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *Constraint) XXX_Merge(src proto.Message) { + xxx_messageInfo_Constraint.Merge(m, src) +} +func (m *Constraint) XXX_Size() int { + return m.Size() +} +func (m *Constraint) XXX_DiscardUnknown() { + xxx_messageInfo_Constraint.DiscardUnknown(m) +} + +var xxx_messageInfo_Constraint proto.InternalMessageInfo + +// ConstraintsConjunction is the set of constraints that need to be satisfied +// together by replicas. It parallels the definition found in zonepb/zone.proto. +type ConstraintsConjunction struct { + // NumReplicas is the number of replicas that should abide by the constraints + // below. If set to zero, the constraints will apply to all replicas of the + // range. + // + // NB: Only REQUIRED constraints are allowed when the number of replicas is + // non-zero. + NumReplicas int32 `protobuf:"varint,1,opt,name=num_replicas,json=numReplicas,proto3" json:"num_replicas,omitempty"` + // Constraints is the set that needs to be satisfied by the store in order for + // us to place replicas on it. + Constraints []Constraint `protobuf:"bytes,2,rep,name=constraints,proto3" json:"constraints"` +} + +func (m *ConstraintsConjunction) Reset() { *m = ConstraintsConjunction{} } +func (m *ConstraintsConjunction) String() string { return proto.CompactTextString(m) } +func (*ConstraintsConjunction) ProtoMessage() {} +func (*ConstraintsConjunction) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{1} +} +func (m *ConstraintsConjunction) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConstraintsConjunction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *ConstraintsConjunction) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConstraintsConjunction.Merge(m, src) +} +func (m *ConstraintsConjunction) XXX_Size() int { + return m.Size() +} +func (m *ConstraintsConjunction) XXX_DiscardUnknown() { + xxx_messageInfo_ConstraintsConjunction.DiscardUnknown(m) +} + +var xxx_messageInfo_ConstraintsConjunction proto.InternalMessageInfo + +// LeasePreference specifies a preference about where range leases should be +// located. It parallels the definition found in zonepb/zone.proto. +type LeasePreference struct { + // Constraints is the set that needs to be satisfied by the store in order for + // us to prefer placing leases on it. + Constraints []Constraint `protobuf:"bytes,1,rep,name=constraints,proto3" json:"constraints"` +} + +func (m *LeasePreference) Reset() { *m = LeasePreference{} } +func (m *LeasePreference) String() string { return proto.CompactTextString(m) } +func (*LeasePreference) ProtoMessage() {} +func (*LeasePreference) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{2} +} +func (m *LeasePreference) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeasePreference) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *LeasePreference) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeasePreference.Merge(m, src) +} +func (m *LeasePreference) XXX_Size() int { + return m.Size() +} +func (m *LeasePreference) XXX_DiscardUnknown() { + xxx_messageInfo_LeasePreference.DiscardUnknown(m) +} + +var xxx_messageInfo_LeasePreference proto.InternalMessageInfo + +// SpanConfig holds the configuration that applies to a given keyspan. It +// parallels the definition found in zonepb/zone.proto. +type SpanConfig struct { + // RangeMinBytes is the minimum size, in bytes, for a range in the given + // keyspan. When a range is less than this size, it'll be merged with an + // adjacent range. + RangeMinBytes int64 `protobuf:"varint,1,opt,name=range_min_bytes,json=rangeMinBytes,proto3" json:"range_min_bytes,omitempty"` + // RangeMaxBytes is the maximum size, in bytes, for a range in the given + // keyspan. When a range is more than this size, it'll split into two ranges. + RangeMaxBytes int64 `protobuf:"varint,2,opt,name=range_max_bytes,json=rangeMaxBytes,proto3" json:"range_max_bytes,omitempty"` + // GCTTL is the number of seconds overwritten values will be retained before + // garbage collection. A value <= 0 means older versions are never GC-ed. + GCTTL int32 `protobuf:"varint,3,opt,name=gc_ttl,json=gcTtl,proto3" json:"gc_ttl,omitempty"` + // GlobalReads specifies whether transactions operating over the range(s) + // should be configured to provide non-blocking behavior, meaning that reads + // can be served consistently from all replicas and do not block on writes. In + // exchange, writes get pushed into the future and must wait on commit to + // ensure linearizability. For more, see #52745. + GlobalReads bool `protobuf:"varint,4,opt,name=global_reads,json=globalReads,proto3" json:"global_reads,omitempty"` + // NumReplicas specifies the number of replicas, including both voting and + // non-voting kinds. + NumReplicas int32 `protobuf:"varint,5,opt,name=num_replicas,json=numReplicas,proto3" json:"num_replicas,omitempty"` + // NumVoters specifies the number of voter replicas. If set to zero, we'll + // consider NumReplicas to be the voter replica count instead (i.e. no + // non-voting replicas). + NumVoters int32 `protobuf:"varint,6,opt,name=num_voters,json=numVoters,proto3" json:"num_voters,omitempty"` + // Constraints constrain which stores the both voting and non-voting replicas + // can be placed on. + // + // NB: The NumReplicas fields in Constraints must either: + // - add up to at most SpanConfig.NumReplicas (pinning the sum of replicas and + // having the rest float freely); + // - all be zero (applying each constraint to all replicas). + Constraints []ConstraintsConjunction `protobuf:"bytes,7,rep,name=constraints,proto3" json:"constraints"` + // VoterConstraints constrains which stores the voting replicas can be placed + // on. This must be compatible with the Constraints field above, but not + // necessarily a subset. It's compatible as long as there are no prohibitive + // constraints above that are required here. + VoterConstraints []ConstraintsConjunction `protobuf:"bytes,8,rep,name=voter_constraints,json=voterConstraints,proto3" json:"voter_constraints"` + // LeasePreference captures the preference for how range leases are to be + // placed. They're allowed to be placed elsewhere if needed, but will follow + // the stated preferences when possible. + // + // More than one lease preference is allowed; they're considered as the most + // preferred option to least. The first preference that an existing replica of + // a range matches will take priority for the lease. + LeasePreferences []LeasePreference `protobuf:"bytes,9,rep,name=lease_preferences,json=leasePreferences,proto3" json:"lease_preferences"` +} + +func (m *SpanConfig) Reset() { *m = SpanConfig{} } +func (m *SpanConfig) String() string { return proto.CompactTextString(m) } +func (*SpanConfig) ProtoMessage() {} +func (*SpanConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{3} +} +func (m *SpanConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SpanConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *SpanConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_SpanConfig.Merge(m, src) +} +func (m *SpanConfig) XXX_Size() int { + return m.Size() +} +func (m *SpanConfig) XXX_DiscardUnknown() { + xxx_messageInfo_SpanConfig.DiscardUnknown(m) +} + +var xxx_messageInfo_SpanConfig proto.InternalMessageInfo + +// SpanConfigEntry ties a span to its corresponding config. +type SpanConfigEntry struct { + // Span is the keyspan the config is said to apply over. + Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + // Config is the set of attributes that apply over the corresponding keyspan. + Config SpanConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config"` +} + +func (m *SpanConfigEntry) Reset() { *m = SpanConfigEntry{} } +func (m *SpanConfigEntry) String() string { return proto.CompactTextString(m) } +func (*SpanConfigEntry) ProtoMessage() {} +func (*SpanConfigEntry) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{4} +} +func (m *SpanConfigEntry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SpanConfigEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *SpanConfigEntry) XXX_Merge(src proto.Message) { + xxx_messageInfo_SpanConfigEntry.Merge(m, src) +} +func (m *SpanConfigEntry) XXX_Size() int { + return m.Size() +} +func (m *SpanConfigEntry) XXX_DiscardUnknown() { + xxx_messageInfo_SpanConfigEntry.DiscardUnknown(m) +} + +var xxx_messageInfo_SpanConfigEntry proto.InternalMessageInfo + +// GetSpanConfigsRequest is used to fetch the span configurations over the +// specified keyspans. +type GetSpanConfigsRequest struct { + // Spans to request the configurations for. The spans listed here are not + // allowed to overlap with one another. + Spans []Span `protobuf:"bytes,1,rep,name=spans,proto3" json:"spans"` +} + +func (m *GetSpanConfigsRequest) Reset() { *m = GetSpanConfigsRequest{} } +func (m *GetSpanConfigsRequest) String() string { return proto.CompactTextString(m) } +func (*GetSpanConfigsRequest) ProtoMessage() {} +func (*GetSpanConfigsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{5} +} +func (m *GetSpanConfigsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetSpanConfigsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *GetSpanConfigsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSpanConfigsRequest.Merge(m, src) +} +func (m *GetSpanConfigsRequest) XXX_Size() int { + return m.Size() +} +func (m *GetSpanConfigsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetSpanConfigsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSpanConfigsRequest proto.InternalMessageInfo + +// GetSpanConfigsResponse lists out the span configurations that overlap with +// the requested spans. +type GetSpanConfigsResponse struct { + // SpanConfigEntries capture the span configurations over the requested spans. + // The results for each Span in the matching GetSpanConfigsRequest are + // flattened out into a single slice, and follow the same ordering. It's + // possible for there to be no configurations for a given span; there'll + // simply be no entries for it. + SpanConfigEntries []SpanConfigEntry `protobuf:"bytes,1,rep,name=span_config_entries,json=spanConfigEntries,proto3" json:"span_config_entries"` +} + +func (m *GetSpanConfigsResponse) Reset() { *m = GetSpanConfigsResponse{} } +func (m *GetSpanConfigsResponse) String() string { return proto.CompactTextString(m) } +func (*GetSpanConfigsResponse) ProtoMessage() {} +func (*GetSpanConfigsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{6} +} +func (m *GetSpanConfigsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetSpanConfigsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *GetSpanConfigsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSpanConfigsResponse.Merge(m, src) +} +func (m *GetSpanConfigsResponse) XXX_Size() int { + return m.Size() +} +func (m *GetSpanConfigsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetSpanConfigsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSpanConfigsResponse proto.InternalMessageInfo + +// UpdateSpanConfigsRequest is used to update the span configurations over the +// given spans. +// +// This is a "targeted" API: the spans being deleted are expected to have been +// present with the same bounds (same start/end key); the same is true for spans +// being upserted with new configs. If bounds are mismatched, an error is +// returned. If spans are being added, they're expected to not overlap with any +// existing spans. When divvying up an existing span into multiple others, +// callers are expected to delete the old and upsert the new ones. This can +// happen as part of the same request; we delete the spans marked for deletion +// before upserting whatever was requested. +// +// Spans are not allowed to overlap with other spans in the same list but can +// across lists. This is necessary to support the delete+upsert semantics +// described above. +type UpdateSpanConfigsRequest struct { + // ToDelete captures the spans we want to delete configs for. + ToDelete []Span `protobuf:"bytes,1,rep,name=to_delete,json=toDelete,proto3" json:"to_delete"` + // ToUpsert captures the spans we want to upsert and the configs we want to + // upsert with. + ToUpsert []SpanConfigEntry `protobuf:"bytes,2,rep,name=to_upsert,json=toUpsert,proto3" json:"to_upsert"` +} + +func (m *UpdateSpanConfigsRequest) Reset() { *m = UpdateSpanConfigsRequest{} } +func (m *UpdateSpanConfigsRequest) String() string { return proto.CompactTextString(m) } +func (*UpdateSpanConfigsRequest) ProtoMessage() {} +func (*UpdateSpanConfigsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{7} +} +func (m *UpdateSpanConfigsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UpdateSpanConfigsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *UpdateSpanConfigsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateSpanConfigsRequest.Merge(m, src) +} +func (m *UpdateSpanConfigsRequest) XXX_Size() int { + return m.Size() +} +func (m *UpdateSpanConfigsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_UpdateSpanConfigsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_UpdateSpanConfigsRequest proto.InternalMessageInfo + +type UpdateSpanConfigsResponse struct { +} + +func (m *UpdateSpanConfigsResponse) Reset() { *m = UpdateSpanConfigsResponse{} } +func (m *UpdateSpanConfigsResponse) String() string { return proto.CompactTextString(m) } +func (*UpdateSpanConfigsResponse) ProtoMessage() {} +func (*UpdateSpanConfigsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{8} +} +func (m *UpdateSpanConfigsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UpdateSpanConfigsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *UpdateSpanConfigsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateSpanConfigsResponse.Merge(m, src) +} +func (m *UpdateSpanConfigsResponse) XXX_Size() int { + return m.Size() +} +func (m *UpdateSpanConfigsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_UpdateSpanConfigsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_UpdateSpanConfigsResponse proto.InternalMessageInfo + +func init() { + proto.RegisterEnum("cockroach.roachpb.Constraint_Type", Constraint_Type_name, Constraint_Type_value) + proto.RegisterType((*Constraint)(nil), "cockroach.roachpb.Constraint") + proto.RegisterType((*ConstraintsConjunction)(nil), "cockroach.roachpb.ConstraintsConjunction") + proto.RegisterType((*LeasePreference)(nil), "cockroach.roachpb.LeasePreference") + proto.RegisterType((*SpanConfig)(nil), "cockroach.roachpb.SpanConfig") + proto.RegisterType((*SpanConfigEntry)(nil), "cockroach.roachpb.SpanConfigEntry") + proto.RegisterType((*GetSpanConfigsRequest)(nil), "cockroach.roachpb.GetSpanConfigsRequest") + proto.RegisterType((*GetSpanConfigsResponse)(nil), "cockroach.roachpb.GetSpanConfigsResponse") + proto.RegisterType((*UpdateSpanConfigsRequest)(nil), "cockroach.roachpb.UpdateSpanConfigsRequest") + proto.RegisterType((*UpdateSpanConfigsResponse)(nil), "cockroach.roachpb.UpdateSpanConfigsResponse") +} + +func init() { proto.RegisterFile("roachpb/span_config.proto", fileDescriptor_91c9f1dcea14470a) } + +var fileDescriptor_91c9f1dcea14470a = []byte{ + // 691 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xcf, 0x4f, 0x13, 0x4d, + 0x18, 0xee, 0xd0, 0x1f, 0xb4, 0x6f, 0xf9, 0xa0, 0x9d, 0x8f, 0x8f, 0x6f, 0xc1, 0xb0, 0xd4, 0x8d, + 0x31, 0xe5, 0x52, 0x22, 0x24, 0x1e, 0xf0, 0xd6, 0xd2, 0x20, 0x09, 0x46, 0x58, 0x5b, 0x63, 0x8c, + 0x71, 0x33, 0xdd, 0x0e, 0x75, 0x65, 0x3b, 0xb3, 0xee, 0xcc, 0x12, 0x7a, 0xf4, 0xe6, 0xd1, 0xa3, + 0x17, 0x13, 0xff, 0x1c, 0x8e, 0x1c, 0x39, 0x11, 0x2d, 0x17, 0xaf, 0xfe, 0x07, 0x66, 0x67, 0xb7, + 0x76, 0x81, 0x06, 0x89, 0xde, 0x66, 0x9f, 0x7d, 0xde, 0xe7, 0x7d, 0xe7, 0x79, 0x66, 0x06, 0x16, + 0x7d, 0x4e, 0xec, 0x37, 0x5e, 0x67, 0x4d, 0x78, 0x84, 0x59, 0x36, 0x67, 0x07, 0x4e, 0xaf, 0xe6, + 0xf9, 0x5c, 0x72, 0x5c, 0xb6, 0xb9, 0x7d, 0xa8, 0x7e, 0xd7, 0x62, 0xd2, 0x12, 0x1e, 0xb1, 0xbb, + 0x44, 0x92, 0x88, 0xb6, 0x34, 0xdf, 0xe3, 0x3d, 0xae, 0x96, 0x6b, 0xe1, 0x2a, 0x42, 0x8d, 0x4f, + 0x08, 0xa0, 0xc1, 0x99, 0x90, 0x3e, 0x71, 0x98, 0xc4, 0x0f, 0x21, 0x23, 0x07, 0x1e, 0xd5, 0x50, + 0x05, 0x55, 0x67, 0xd7, 0x8d, 0xda, 0x35, 0xe9, 0xda, 0x98, 0x5c, 0x6b, 0x0d, 0x3c, 0x6a, 0x2a, + 0x3e, 0x2e, 0x41, 0xfa, 0x90, 0x0e, 0xb4, 0xa9, 0x0a, 0xaa, 0x16, 0xcc, 0x70, 0x89, 0xe7, 0x21, + 0x7b, 0x44, 0xdc, 0x80, 0x6a, 0x69, 0x85, 0x45, 0x1f, 0xc6, 0x3d, 0xc8, 0x84, 0x55, 0x78, 0x06, + 0xf2, 0x66, 0x73, 0xbf, 0xbd, 0x63, 0x36, 0xb7, 0x4a, 0x29, 0x3c, 0x0b, 0xb0, 0x67, 0x3e, 0x7d, + 0xbc, 0x53, 0xdf, 0x69, 0x35, 0xb7, 0x4a, 0x68, 0x33, 0xf3, 0xfd, 0xcb, 0x0a, 0x32, 0x3e, 0x20, + 0x58, 0x18, 0x77, 0x13, 0x0d, 0xce, 0xde, 0x06, 0xcc, 0x96, 0x0e, 0x67, 0xf8, 0x2e, 0xcc, 0xb0, + 0xa0, 0x6f, 0xf9, 0xd4, 0x73, 0x1d, 0x9b, 0x08, 0x35, 0x6e, 0xd6, 0x2c, 0xb2, 0xa0, 0x6f, 0xc6, + 0x10, 0x6e, 0x42, 0xd1, 0x1e, 0x17, 0x6b, 0x53, 0x95, 0x74, 0xb5, 0xb8, 0xbe, 0x7c, 0xe3, 0x86, + 0xea, 0x99, 0x93, 0xf3, 0x95, 0x94, 0x99, 0xac, 0x8b, 0x47, 0x79, 0x0d, 0x73, 0xbb, 0x94, 0x08, + 0xba, 0xe7, 0xd3, 0x03, 0xea, 0x53, 0x66, 0xd3, 0xab, 0xfa, 0xe8, 0xaf, 0xf4, 0x7f, 0xa4, 0x01, + 0x9e, 0x79, 0x84, 0x35, 0x54, 0xae, 0xf8, 0x3e, 0xcc, 0xf9, 0x84, 0xf5, 0xa8, 0xd5, 0x77, 0x98, + 0xd5, 0x19, 0x48, 0x1a, 0xed, 0x30, 0x6d, 0xfe, 0xa3, 0xe0, 0x27, 0x0e, 0xab, 0x87, 0x60, 0x82, + 0x47, 0x8e, 0x63, 0xde, 0x54, 0x92, 0x47, 0x8e, 0x23, 0x5e, 0x05, 0x72, 0x3d, 0xdb, 0x92, 0xd2, + 0x55, 0x61, 0x64, 0xeb, 0x85, 0xe1, 0xf9, 0x4a, 0x76, 0xbb, 0xd1, 0x6a, 0xed, 0x9a, 0xd9, 0x9e, + 0xdd, 0x92, 0x6e, 0x68, 0x68, 0xcf, 0xe5, 0x1d, 0xe2, 0x5a, 0x3e, 0x25, 0x5d, 0xa1, 0x65, 0x2a, + 0xa8, 0x9a, 0x37, 0x8b, 0x11, 0x66, 0x86, 0xd0, 0x35, 0xcf, 0xb3, 0xd7, 0x3d, 0x5f, 0x06, 0x08, + 0x29, 0x47, 0x5c, 0x52, 0x5f, 0x68, 0x39, 0x45, 0x28, 0xb0, 0xa0, 0xff, 0x5c, 0x01, 0x78, 0xff, + 0xb2, 0x65, 0xd3, 0xca, 0xb2, 0xd5, 0x1b, 0x2d, 0x4b, 0xa6, 0x3e, 0xc1, 0x3e, 0xfc, 0x0a, 0xca, + 0xaa, 0x9b, 0x95, 0x14, 0xce, 0xff, 0x99, 0x70, 0x49, 0x29, 0x25, 0x28, 0xb8, 0x0d, 0x65, 0x37, + 0x8c, 0xdd, 0xf2, 0x7e, 0xe5, 0x2e, 0xb4, 0x82, 0x52, 0x9f, 0x74, 0x35, 0xae, 0x1c, 0x91, 0x91, + 0xac, 0x7b, 0x19, 0x1e, 0x65, 0xfe, 0x1e, 0xc1, 0xdc, 0x38, 0xf3, 0x26, 0x93, 0xfe, 0x00, 0x3f, + 0x80, 0x4c, 0x78, 0xbf, 0x55, 0xda, 0xc5, 0xf5, 0xff, 0x27, 0xf4, 0x08, 0x2b, 0x62, 0x61, 0x45, + 0xc5, 0x8f, 0x20, 0x17, 0xbd, 0x06, 0x2a, 0xfa, 0xc9, 0x47, 0x70, 0xdc, 0x26, 0x2e, 0x8d, 0x4b, + 0x8c, 0x5d, 0xf8, 0x6f, 0x9b, 0xca, 0xf1, 0x6f, 0x61, 0xd2, 0x77, 0x01, 0x15, 0x12, 0x6f, 0x40, + 0x36, 0x54, 0x1f, 0x9d, 0xeb, 0xdf, 0x4c, 0x12, 0x71, 0x0d, 0x1f, 0x16, 0xae, 0xaa, 0x09, 0x8f, + 0x33, 0x41, 0xf1, 0x0b, 0xf8, 0x37, 0xf1, 0x6e, 0x59, 0x94, 0x49, 0xdf, 0xa1, 0x23, 0x71, 0xe3, + 0xc6, 0x89, 0x95, 0x31, 0x71, 0x9f, 0xb2, 0xb8, 0x04, 0x3b, 0x54, 0x18, 0x9f, 0x11, 0x68, 0x6d, + 0xaf, 0x4b, 0x24, 0x9d, 0xb0, 0x8b, 0x4d, 0x28, 0x48, 0x6e, 0x75, 0xa9, 0x4b, 0x25, 0xbd, 0xdd, + 0x4e, 0xf2, 0x92, 0x6f, 0x29, 0x3a, 0x6e, 0xaa, 0xda, 0xc0, 0x13, 0xd4, 0x97, 0xf1, 0xeb, 0x71, + 0xfb, 0x41, 0xf3, 0x92, 0xb7, 0x55, 0xa5, 0x71, 0x07, 0x16, 0x27, 0x8c, 0x17, 0xd9, 0x52, 0x5f, + 0x3d, 0xf9, 0xa6, 0xa7, 0x4e, 0x86, 0x3a, 0x3a, 0x1d, 0xea, 0xe8, 0x6c, 0xa8, 0xa3, 0xaf, 0x43, + 0x1d, 0x7d, 0xbc, 0xd0, 0x53, 0xa7, 0x17, 0x7a, 0xea, 0xec, 0x42, 0x4f, 0xbd, 0x9c, 0x8e, 0xfb, + 0x74, 0x72, 0xea, 0xb9, 0xde, 0xf8, 0x19, 0x00, 0x00, 0xff, 0xff, 0x1a, 0x72, 0xb9, 0x59, 0x08, + 0x06, 0x00, 0x00, +} + +func (this *Constraint) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Constraint) + if !ok { + that2, ok := that.(Constraint) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if this.Key != that1.Key { + return false + } + if this.Value != that1.Value { + return false + } + return true +} +func (this *ConstraintsConjunction) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ConstraintsConjunction) + if !ok { + that2, ok := that.(ConstraintsConjunction) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.NumReplicas != that1.NumReplicas { + return false + } + if len(this.Constraints) != len(that1.Constraints) { + return false + } + for i := range this.Constraints { + if !this.Constraints[i].Equal(&that1.Constraints[i]) { + return false + } + } + return true +} +func (this *LeasePreference) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LeasePreference) + if !ok { + that2, ok := that.(LeasePreference) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Constraints) != len(that1.Constraints) { + return false + } + for i := range this.Constraints { + if !this.Constraints[i].Equal(&that1.Constraints[i]) { + return false + } + } + return true +} +func (this *SpanConfig) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SpanConfig) + if !ok { + that2, ok := that.(SpanConfig) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.RangeMinBytes != that1.RangeMinBytes { + return false + } + if this.RangeMaxBytes != that1.RangeMaxBytes { + return false + } + if this.GCTTL != that1.GCTTL { + return false + } + if this.GlobalReads != that1.GlobalReads { + return false + } + if this.NumReplicas != that1.NumReplicas { + return false + } + if this.NumVoters != that1.NumVoters { + return false + } + if len(this.Constraints) != len(that1.Constraints) { + return false + } + for i := range this.Constraints { + if !this.Constraints[i].Equal(&that1.Constraints[i]) { + return false + } + } + if len(this.VoterConstraints) != len(that1.VoterConstraints) { + return false + } + for i := range this.VoterConstraints { + if !this.VoterConstraints[i].Equal(&that1.VoterConstraints[i]) { + return false + } + } + if len(this.LeasePreferences) != len(that1.LeasePreferences) { + return false + } + for i := range this.LeasePreferences { + if !this.LeasePreferences[i].Equal(&that1.LeasePreferences[i]) { + return false + } + } + return true +} +func (m *Constraint) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Constraint) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Constraint) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintSpanConfig(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x1a + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintSpanConfig(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0x12 + } + if m.Type != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ConstraintsConjunction) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ConstraintsConjunction) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ConstraintsConjunction) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Constraints) > 0 { + for iNdEx := len(m.Constraints) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Constraints[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.NumReplicas != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.NumReplicas)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LeasePreference) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LeasePreference) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LeasePreference) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Constraints) > 0 { + for iNdEx := len(m.Constraints) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Constraints[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SpanConfig) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SpanConfig) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SpanConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.LeasePreferences) > 0 { + for iNdEx := len(m.LeasePreferences) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.LeasePreferences[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } + if len(m.VoterConstraints) > 0 { + for iNdEx := len(m.VoterConstraints) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.VoterConstraints[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + if len(m.Constraints) > 0 { + for iNdEx := len(m.Constraints) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Constraints[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if m.NumVoters != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.NumVoters)) + i-- + dAtA[i] = 0x30 + } + if m.NumReplicas != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.NumReplicas)) + i-- + dAtA[i] = 0x28 + } + if m.GlobalReads { + i-- + if m.GlobalReads { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if m.GCTTL != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.GCTTL)) + i-- + dAtA[i] = 0x18 + } + if m.RangeMaxBytes != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.RangeMaxBytes)) + i-- + dAtA[i] = 0x10 + } + if m.RangeMinBytes != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.RangeMinBytes)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SpanConfigEntry) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SpanConfigEntry) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SpanConfigEntry) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Config.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.Span.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *GetSpanConfigsRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetSpanConfigsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetSpanConfigsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Spans) > 0 { + for iNdEx := len(m.Spans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Spans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *GetSpanConfigsResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetSpanConfigsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetSpanConfigsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.SpanConfigEntries) > 0 { + for iNdEx := len(m.SpanConfigEntries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SpanConfigEntries[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *UpdateSpanConfigsRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpdateSpanConfigsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateSpanConfigsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ToUpsert) > 0 { + for iNdEx := len(m.ToUpsert) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ToUpsert[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.ToDelete) > 0 { + for iNdEx := len(m.ToDelete) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ToDelete[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *UpdateSpanConfigsResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpdateSpanConfigsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateSpanConfigsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + +func encodeVarintSpanConfig(dAtA []byte, offset int, v uint64) int { + offset -= sovSpanConfig(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Constraint) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovSpanConfig(uint64(m.Type)) + } + l = len(m.Key) + if l > 0 { + n += 1 + l + sovSpanConfig(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovSpanConfig(uint64(l)) + } + return n +} + +func (m *ConstraintsConjunction) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NumReplicas != 0 { + n += 1 + sovSpanConfig(uint64(m.NumReplicas)) + } + if len(m.Constraints) > 0 { + for _, e := range m.Constraints { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *LeasePreference) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Constraints) > 0 { + for _, e := range m.Constraints { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *SpanConfig) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RangeMinBytes != 0 { + n += 1 + sovSpanConfig(uint64(m.RangeMinBytes)) + } + if m.RangeMaxBytes != 0 { + n += 1 + sovSpanConfig(uint64(m.RangeMaxBytes)) + } + if m.GCTTL != 0 { + n += 1 + sovSpanConfig(uint64(m.GCTTL)) + } + if m.GlobalReads { + n += 2 + } + if m.NumReplicas != 0 { + n += 1 + sovSpanConfig(uint64(m.NumReplicas)) + } + if m.NumVoters != 0 { + n += 1 + sovSpanConfig(uint64(m.NumVoters)) + } + if len(m.Constraints) > 0 { + for _, e := range m.Constraints { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + if len(m.VoterConstraints) > 0 { + for _, e := range m.VoterConstraints { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + if len(m.LeasePreferences) > 0 { + for _, e := range m.LeasePreferences { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *SpanConfigEntry) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.Span.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + l = m.Config.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + return n +} + +func (m *GetSpanConfigsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Spans) > 0 { + for _, e := range m.Spans { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *GetSpanConfigsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.SpanConfigEntries) > 0 { + for _, e := range m.SpanConfigEntries { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *UpdateSpanConfigsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.ToDelete) > 0 { + for _, e := range m.ToDelete { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + if len(m.ToUpsert) > 0 { + for _, e := range m.ToUpsert { + l = e.Size() + n += 1 + l + sovSpanConfig(uint64(l)) + } + } + return n +} + +func (m *UpdateSpanConfigsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func sovSpanConfig(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozSpanConfig(x uint64) (n int) { + return sovSpanConfig(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Constraint) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Constraint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Constraint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= Constraint_Type(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ConstraintsConjunction) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConstraintsConjunction: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConstraintsConjunction: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumReplicas", wireType) + } + m.NumReplicas = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumReplicas |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Constraints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Constraints = append(m.Constraints, Constraint{}) + if err := m.Constraints[len(m.Constraints)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LeasePreference) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LeasePreference: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LeasePreference: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Constraints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Constraints = append(m.Constraints, Constraint{}) + if err := m.Constraints[len(m.Constraints)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SpanConfig) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SpanConfig: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SpanConfig: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RangeMinBytes", wireType) + } + m.RangeMinBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RangeMinBytes |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RangeMaxBytes", wireType) + } + m.RangeMaxBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RangeMaxBytes |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GCTTL", wireType) + } + m.GCTTL = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GCTTL |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GlobalReads", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.GlobalReads = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumReplicas", wireType) + } + m.NumReplicas = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumReplicas |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumVoters", wireType) + } + m.NumVoters = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumVoters |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Constraints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Constraints = append(m.Constraints, ConstraintsConjunction{}) + if err := m.Constraints[len(m.Constraints)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoterConstraints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoterConstraints = append(m.VoterConstraints, ConstraintsConjunction{}) + if err := m.VoterConstraints[len(m.VoterConstraints)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LeasePreferences", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LeasePreferences = append(m.LeasePreferences, LeasePreference{}) + if err := m.LeasePreferences[len(m.LeasePreferences)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SpanConfigEntry) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SpanConfigEntry: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SpanConfigEntry: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Span", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Span.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Config", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Config.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetSpanConfigsRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetSpanConfigsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSpanConfigsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Spans = append(m.Spans, Span{}) + if err := m.Spans[len(m.Spans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetSpanConfigsResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetSpanConfigsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSpanConfigsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SpanConfigEntries", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SpanConfigEntries = append(m.SpanConfigEntries, SpanConfigEntry{}) + if err := m.SpanConfigEntries[len(m.SpanConfigEntries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpdateSpanConfigsRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpdateSpanConfigsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpdateSpanConfigsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToDelete", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ToDelete = append(m.ToDelete, Span{}) + if err := m.ToDelete[len(m.ToDelete)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToUpsert", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ToUpsert = append(m.ToUpsert, SpanConfigEntry{}) + if err := m.ToUpsert[len(m.ToUpsert)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpdateSpanConfigsResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpdateSpanConfigsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpdateSpanConfigsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipSpanConfig(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthSpanConfig + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupSpanConfig + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthSpanConfig + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthSpanConfig = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSpanConfig = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupSpanConfig = fmt.Errorf("proto: unexpected end of group") +) diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto new file mode 100644 index 000000000000..3963442d6dde --- /dev/null +++ b/pkg/roachpb/span_config.proto @@ -0,0 +1,186 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.roachpb; +option go_package = "roachpb"; + +import "roachpb/data.proto"; +import "gogoproto/gogo.proto"; + +// TODO(irfansharif): We could have the proto definitions in pkg/config/zonepb +// use these messages instead of duplicating everything. + +// Constraint constrains the stores that a replica can be stored on. It +// parallels the definition found in zonepb/zone.proto. +message Constraint { + option (gogoproto.equal) = true; + + enum Type { + // REQUIRED ensures all replicas are placed on stores with locality tags + // that match the constraint. Replication will fail if there aren't any such + // stores. + REQUIRED = 0; + // PROHIBITED prevents replicas from being placed on stores with locality + // tags that match the constrain. + PROHIBITED = 1; + } + + // Type captures the kind of constraint this is: required or prohibited. + Type type = 1; + + // Key captures the locality tag key we're constraining against. + string key = 2; + + // Value is the locality tag value we're constraining against. + string value = 3; +} + +// ConstraintsConjunction is the set of constraints that need to be satisfied +// together by replicas. It parallels the definition found in zonepb/zone.proto. +message ConstraintsConjunction { + option (gogoproto.equal) = true; + + // NumReplicas is the number of replicas that should abide by the constraints + // below. If set to zero, the constraints will apply to all replicas of the + // range. + // + // NB: Only REQUIRED constraints are allowed when the number of replicas is + // non-zero. + int32 num_replicas = 1; + + // Constraints is the set that needs to be satisfied by the store in order for + // us to place replicas on it. + repeated Constraint constraints = 2 [(gogoproto.nullable) = false]; +} + +// LeasePreference specifies a preference about where range leases should be +// located. It parallels the definition found in zonepb/zone.proto. +message LeasePreference { + option (gogoproto.equal) = true; + + // Constraints is the set that needs to be satisfied by the store in order for + // us to prefer placing leases on it. + repeated Constraint constraints = 1 [(gogoproto.nullable) = false]; +} + +// SpanConfig holds the configuration that applies to a given keyspan. It +// parallels the definition found in zonepb/zone.proto. +message SpanConfig { + option (gogoproto.equal) = true; + + // RangeMinBytes is the minimum size, in bytes, for a range in the given + // keyspan. When a range is less than this size, it'll be merged with an + // adjacent range. + int64 range_min_bytes = 1; + + // RangeMaxBytes is the maximum size, in bytes, for a range in the given + // keyspan. When a range is more than this size, it'll split into two ranges. + int64 range_max_bytes = 2; + + // GCTTL is the number of seconds overwritten values will be retained before + // garbage collection. A value <= 0 means older versions are never GC-ed. + int32 gc_ttl = 3 [(gogoproto.customname) = "GCTTL"]; + + // GlobalReads specifies whether transactions operating over the range(s) + // should be configured to provide non-blocking behavior, meaning that reads + // can be served consistently from all replicas and do not block on writes. In + // exchange, writes get pushed into the future and must wait on commit to + // ensure linearizability. For more, see #52745. + bool global_reads = 4; + + // NumReplicas specifies the number of replicas, including both voting and + // non-voting kinds. + int32 num_replicas = 5; + + // NumVoters specifies the number of voter replicas. If set to zero, we'll + // consider NumReplicas to be the voter replica count instead (i.e. no + // non-voting replicas). + int32 num_voters = 6; + + // Constraints constrain which stores the both voting and non-voting replicas + // can be placed on. + // + // NB: The NumReplicas fields in Constraints must either: + // - add up to at most SpanConfig.NumReplicas (pinning the sum of replicas and + // having the rest float freely); + // - all be zero (applying each constraint to all replicas). + repeated ConstraintsConjunction constraints = 7 [(gogoproto.nullable) = false]; + + // VoterConstraints constrains which stores the voting replicas can be placed + // on. This must be compatible with the Constraints field above, but not + // necessarily a subset. It's compatible as long as there are no prohibitive + // constraints above that are required here. + repeated ConstraintsConjunction voter_constraints = 8 [(gogoproto.nullable) = false]; + + // LeasePreference captures the preference for how range leases are to be + // placed. They're allowed to be placed elsewhere if needed, but will follow + // the stated preferences when possible. + // + // More than one lease preference is allowed; they're considered as the most + // preferred option to least. The first preference that an existing replica of + // a range matches will take priority for the lease. + repeated LeasePreference lease_preferences = 9 [(gogoproto.nullable) = false]; +} + +// SpanConfigEntry ties a span to its corresponding config. +message SpanConfigEntry { + // Span is the keyspan the config is said to apply over. + Span span = 1 [(gogoproto.nullable) = false]; + + // Config is the set of attributes that apply over the corresponding keyspan. + SpanConfig config = 2 [(gogoproto.nullable) = false]; +}; + +// GetSpanConfigsRequest is used to fetch the span configurations over the +// specified keyspans. +message GetSpanConfigsRequest { + // Spans to request the configurations for. The spans listed here are not + // allowed to overlap with one another. + repeated Span spans = 1 [(gogoproto.nullable) = false]; +}; + +// GetSpanConfigsResponse lists out the span configurations that overlap with +// the requested spans. +message GetSpanConfigsResponse { + // SpanConfigEntries capture the span configurations over the requested spans. + // The results for each Span in the matching GetSpanConfigsRequest are + // flattened out into a single slice, and follow the same ordering. It's + // possible for there to be no configurations for a given span; there'll + // simply be no entries for it. + repeated SpanConfigEntry span_config_entries = 1 [(gogoproto.nullable) = false]; +}; + +// UpdateSpanConfigsRequest is used to update the span configurations over the +// given spans. +// +// This is a "targeted" API: the spans being deleted are expected to have been +// present with the same bounds (same start/end key); the same is true for spans +// being upserted with new configs. If bounds are mismatched, an error is +// returned. If spans are being added, they're expected to not overlap with any +// existing spans. When divvying up an existing span into multiple others, +// callers are expected to delete the old and upsert the new ones. This can +// happen as part of the same request; we delete the spans marked for deletion +// before upserting whatever was requested. +// +// Spans are not allowed to overlap with other spans in the same list but can +// across lists. This is necessary to support the delete+upsert semantics +// described above. +message UpdateSpanConfigsRequest { + // ToDelete captures the spans we want to delete configs for. + repeated Span to_delete = 1 [(gogoproto.nullable) = false]; + + // ToUpsert captures the spans we want to upsert and the configs we want to + // upsert with. + repeated SpanConfigEntry to_upsert = 2 [(gogoproto.nullable) = false]; +}; + +message UpdateSpanConfigsResponse { }; + diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index a27ef36b72a6..cd45fb4b0090 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -71,6 +71,12 @@ func (a tenantAuthorizer) authorize( case "/cockroach.server.serverpb.Status/ListSessions": return a.authTenant(tenID) + case "/cockroach.roachpb.Internal/GetSpanConfigs": + return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest)) + + case "/cockroach.roachpb.Internal/UpdateSpanConfigs": + return a.authUpdateSpanConfigs(tenID, req.(*roachpb.UpdateSpanConfigsRequest)) + default: return authErrorf("unknown method %q", fullMethod) } @@ -206,6 +212,55 @@ func (a tenantAuthorizer) authTokenBucket( return nil } +// authGetSpanConfigs authorizes the provided tenant to invoke the +// GetSpanConfigs RPC with the provided args. +func (a tenantAuthorizer) authGetSpanConfigs( + tenID roachpb.TenantID, args *roachpb.GetSpanConfigsRequest, +) error { + tenSpan := tenantPrefix(tenID) + for _, sp := range args.Spans { + rSpan, err := keys.SpanAddr(sp) + if err != nil { + return authError(err.Error()) + } + if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { + return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan) + } + } + return nil +} + +// authUpdateSpanConfigs authorizes the provided tenant to invoke the +// UpdateSpanConfigs RPC with the provided args. +func (a tenantAuthorizer) authUpdateSpanConfigs( + tenID roachpb.TenantID, args *roachpb.UpdateSpanConfigsRequest, +) error { + tenSpan := tenantPrefix(tenID) + validate := func(sp roachpb.Span) error { + rSpan, err := keys.SpanAddr(sp) + if err != nil { + return authError(err.Error()) + } + if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { + return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan) + } + return nil + } + + for _, entry := range args.ToUpsert { + if err := validate(entry.Span); err != nil { + return err + } + } + for _, span := range args.ToDelete { + if err := validate(span); err != nil { + return err + } + } + + return nil +} + func contextWithTenant(ctx context.Context, tenID roachpb.TenantID) context.Context { ctx = roachpb.NewContextForTenant(ctx, tenID) ctx = logtags.AddTag(ctx, "tenant", tenID.String()) diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go index abc33e495f86..0e6d9f9dc73d 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -175,6 +175,17 @@ func TestTenantAuthRequest(t *testing.T) { } return ru } + makeGetSpanConfigsReq := func(key, endKey string) *roachpb.GetSpanConfigsRequest { + sp := makeSpan(key, endKey) + return &roachpb.GetSpanConfigsRequest{Spans: []roachpb.Span{sp}} + } + makeUpdateSpanConfigsReq := func(key, endKey string, delete bool) *roachpb.UpdateSpanConfigsRequest { + sp := makeSpan(key, endKey) + if delete { + return &roachpb.UpdateSpanConfigsRequest{ToDelete: []roachpb.Span{sp}} + } + return &roachpb.UpdateSpanConfigsRequest{ToUpsert: []roachpb.SpanConfigEntry{{Span: sp}}} + } const noError = "" for method, tests := range map[string][]struct { @@ -374,6 +385,91 @@ func TestTenantAuthRequest(t *testing.T) { expErr: `token bucket request with unspecified tenant not permitted`, }, }, + "/cockroach.roachpb.Internal/GetSpanConfigs": { + { + req: &roachpb.GetSpanConfigsRequest{}, + expErr: noError, + }, + { + req: makeGetSpanConfigsReq("a", "b"), + expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeGetSpanConfigsReq(prefix(5, "a"), prefix(5, "b")), + expErr: `requested key span /Tenant/5"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeGetSpanConfigsReq(prefix(10, "a"), prefix(10, "b")), + expErr: noError, + }, + { + req: makeGetSpanConfigsReq(prefix(50, "a"), prefix(50, "b")), + expErr: `requested key span /Tenant/50"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeGetSpanConfigsReq("a", prefix(10, "b")), + expErr: `requested key span {a-/Tenant/10"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeGetSpanConfigsReq(prefix(10, "a"), prefix(20, "b")), + expErr: `requested key span /Tenant/{10"a"-20"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + }, + "/cockroach.roachpb.Internal/UpdateSpanConfigs": { + { + req: &roachpb.UpdateSpanConfigsRequest{}, + expErr: noError, + }, + { + req: makeUpdateSpanConfigsReq("a", "b", true), + expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(5, "a"), prefix(5, "b"), true), + expErr: `requested key span /Tenant/5"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(10, "a"), prefix(10, "b"), true), + expErr: noError, + }, + { + req: makeUpdateSpanConfigsReq(prefix(50, "a"), prefix(50, "b"), true), + expErr: `requested key span /Tenant/50"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq("a", prefix(10, "b"), true), + expErr: `requested key span {a-/Tenant/10"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(10, "a"), prefix(20, "b"), true), + expErr: `requested key span /Tenant/{10"a"-20"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq("a", "b", false), + expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(5, "a"), prefix(5, "b"), false), + expErr: `requested key span /Tenant/5"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(10, "a"), prefix(10, "b"), false), + expErr: noError, + }, + { + req: makeUpdateSpanConfigsReq(prefix(50, "a"), prefix(50, "b"), false), + expErr: `requested key span /Tenant/50"{a"-b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq("a", prefix(10, "b"), false), + expErr: `requested key span {a-/Tenant/10"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeUpdateSpanConfigsReq(prefix(10, "a"), prefix(20, "b"), false), + expErr: `requested key span /Tenant/{10"a"-20"b"} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + }, + "/cockroach.rpc.Heartbeat/Ping": { {req: &PingRequest{}, expErr: noError}, }, diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 7db845efe1bb..99dd0fe558d1 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -497,6 +497,20 @@ func (a internalClientAdapter) TokenBucket( return a.server.TokenBucket(ctx, in) } +// GetSpanConfigs is part of the roachpb.InternalClient interface. +func (a internalClientAdapter) GetSpanConfigs( + ctx context.Context, req *roachpb.GetSpanConfigsRequest, _ ...grpc.CallOption, +) (*roachpb.GetSpanConfigsResponse, error) { + return a.server.GetSpanConfigs(ctx, req) +} + +// UpdateSpanConfigs is part of the roachpb.InternalClient interface. +func (a internalClientAdapter) UpdateSpanConfigs( + ctx context.Context, req *roachpb.UpdateSpanConfigsRequest, _ ...grpc.CallOption, +) (*roachpb.UpdateSpanConfigsResponse, error) { + return a.server.UpdateSpanConfigs(ctx, req) +} + type respStreamClientAdapter struct { ctx context.Context respC chan interface{} diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 702fa6a19d55..d3881a461923 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -262,6 +262,18 @@ func (*internalServer) TokenBucket( panic("unimplemented") } +func (*internalServer) GetSpanConfigs( + context.Context, *roachpb.GetSpanConfigsRequest, +) (*roachpb.GetSpanConfigsResponse, error) { + panic("unimplemented") +} + +func (*internalServer) UpdateSpanConfigs( + context.Context, *roachpb.UpdateSpanConfigsRequest, +) (*roachpb.UpdateSpanConfigsResponse, error) { + panic("unimplemented") +} + // TestInternalServerAddress verifies that RPCContext uses AdvertiseAddr, not Addr, to // determine whether to apply the local server optimization. // diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index c43376d0a443..67675ea16206 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -112,6 +112,7 @@ go_library( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/spanconfig/spanconfigjob", + "//pkg/spanconfig/spanconfigkvaccessor", "//pkg/spanconfig/spanconfigmanager", "//pkg/sql", "//pkg/sql/catalog/bootstrap", diff --git a/pkg/server/node.go b/pkg/server/node.go index 5acc64fe3b9d..e9e468b7fa92 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/storage" @@ -166,7 +167,7 @@ type Node struct { recorder *status.MetricsRecorder startedAt int64 lastUp int64 - initialStart bool // True if this is the first time this node has started. + initialStart bool // true if this is the first time this node has started txnMetrics kvcoord.TxnMetrics // Used to signal when additional stores, if any, have been initialized. @@ -179,6 +180,8 @@ type Node struct { storeGrantCoords *admission.StoreGrantCoordinators tenantUsage multitenant.TenantUsageServer + + spanConfigAccessor spanconfig.KVAccessor // powers the span configuration RPCs } var _ roachpb.InternalServer = &Node{} @@ -302,23 +305,25 @@ func NewNode( kvAdmissionQ *admission.WorkQueue, storeGrantCoords *admission.StoreGrantCoordinators, tenantUsage multitenant.TenantUsageServer, + spanConfigAccessor spanconfig.KVAccessor, ) *Node { var sqlExec *sql.InternalExecutor if execCfg != nil { sqlExec = execCfg.InternalExecutor } n := &Node{ - storeCfg: cfg, - stopper: stopper, - recorder: recorder, - metrics: makeNodeMetrics(reg, cfg.HistogramWindowInterval), - stores: stores, - txnMetrics: txnMetrics, - sqlExec: sqlExec, - clusterID: clusterID, - kvAdmissionQ: kvAdmissionQ, - storeGrantCoords: storeGrantCoords, - tenantUsage: tenantUsage, + storeCfg: cfg, + stopper: stopper, + recorder: recorder, + metrics: makeNodeMetrics(reg, cfg.HistogramWindowInterval), + stores: stores, + txnMetrics: txnMetrics, + sqlExec: sqlExec, + clusterID: clusterID, + kvAdmissionQ: kvAdmissionQ, + storeGrantCoords: storeGrantCoords, + tenantUsage: tenantUsage, + spanConfigAccessor: spanConfigAccessor, } n.perReplicaServer = kvserver.MakeServer(&n.Descriptor, n.stores) return n @@ -1440,3 +1445,29 @@ type emptyMetricStruct struct{} var _ metric.Struct = emptyMetricStruct{} func (emptyMetricStruct) MetricStruct() {} + +// GetSpanConfigs implements the roachpb.InternalServer interface. +func (n *Node) GetSpanConfigs( + ctx context.Context, req *roachpb.GetSpanConfigsRequest, +) (*roachpb.GetSpanConfigsResponse, error) { + entries, err := n.spanConfigAccessor.GetSpanConfigEntriesFor(ctx, req.Spans) + if err != nil { + return nil, err + } + + return &roachpb.GetSpanConfigsResponse{SpanConfigEntries: entries}, nil +} + +// UpdateSpanConfigs implements the roachpb.InternalServer interface. +func (n *Node) UpdateSpanConfigs( + ctx context.Context, req *roachpb.UpdateSpanConfigsRequest, +) (*roachpb.UpdateSpanConfigsResponse, error) { + // TODO(irfansharif): We want to protect ourselves from tenants creating + // outlandishly large string buffers here and OOM-ing the host cluster. Is + // the maximum protobuf message size enough of a safeguard? + err := n.spanConfigAccessor.UpdateSpanConfigEntries(ctx, req.ToDelete, req.ToUpsert) + if err != nil { + return nil, err + } + return &roachpb.UpdateSpanConfigsResponse{}, nil +} diff --git a/pkg/server/server.go b/pkg/server/server.go index 28d27f9281c4..e13ab1bc482d 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -69,7 +69,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" _ "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" // register jobs declared outside of pkg/sql + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" _ "github.com/cockroachdb/cockroach/pkg/sql/gcjob" // register jobs declared outside of pkg/sql @@ -652,11 +654,17 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { tenantUsage := NewTenantUsageServer(db, internalExecutor) registry.AddMetricStruct(tenantUsage.Metrics()) + + spanConfigAccessor := spanconfigkvaccessor.New( + db, internalExecutor, cfg.Settings, + systemschema.SpanConfigurationsTableName.FQString(), + ) + node := NewNode( storeCfg, recorder, registry, stopper, txnMetrics, stores, nil /* execCfg */, &rpcContext.ClusterID, gcoords.Regular.GetWorkQueue(admission.KVWork), gcoords.Stores, - tenantUsage, + tenantUsage, spanConfigAccessor, ) lateBoundNode = node roachpb.RegisterInternalServer(grpcServer.Server, node) @@ -752,6 +760,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { rpcContext: rpcContext, nodeDescs: g, systemConfigProvider: g, + spanConfigAccessor: spanConfigAccessor, nodeDialer: nodeDialer, distSender: distSender, db: db, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index fd6ece24f77e..3da7f7c087a8 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -220,6 +220,9 @@ type sqlServerArgs struct { // Used by the executor config. systemConfigProvider config.SystemConfigProvider + // Used by the span config reconciliation job. + spanConfigAccessor spanconfig.KVAccessor + // Used by DistSQLPlanner. nodeDialer *nodedialer.Dialer @@ -828,6 +831,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { cfg.circularInternalExecutor, cfg.stopper, cfg.Settings, + cfg.spanConfigAccessor, knobs, ) execCfg.SpanConfigReconciliationJobDeps = spanconfigMgr diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index e5067911eb8f..5ce7e3eb2d0b 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -431,6 +431,7 @@ func makeTenantSQLServerArgs( rpcContext: rpcContext, nodeDescs: tenantConnect, systemConfigProvider: tenantConnect, + spanConfigAccessor: tenantConnect, nodeDialer: nodeDialer, distSender: ds, db: db, diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index e8805ae3c228..1e4fedf8b0ce 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -928,6 +928,11 @@ func (ts *TestServer) MigrationServer() interface{} { return ts.migrationServer } +// SpanConfigAccessor is part of TestServerInterface. +func (ts *TestServer) SpanConfigAccessor() interface{} { + return ts.Server.node.spanConfigAccessor +} + // SQLServer is part of TestServerInterface. func (ts *TestServer) SQLServer() interface{} { return ts.PGServer().SQLServer diff --git a/pkg/settings/bool.go b/pkg/settings/bool.go index a25ea51e96bc..589849fa5cfb 100644 --- a/pkg/settings/bool.go +++ b/pkg/settings/bool.go @@ -98,9 +98,6 @@ func (b *BoolSetting) WithSystemOnly() *BoolSetting { return b } -// Defeat the linter. -var _ = (*BoolSetting).WithSystemOnly - // RegisterBoolSetting defines a new setting with type bool. func RegisterBoolSetting(key, desc string, defaultValue bool) *BoolSetting { setting := &BoolSetting{defaultValue: defaultValue} diff --git a/pkg/spanconfig/BUILD.bazel b/pkg/spanconfig/BUILD.bazel index be35186fa41d..e949d5d82f05 100644 --- a/pkg/spanconfig/BUILD.bazel +++ b/pkg/spanconfig/BUILD.bazel @@ -10,6 +10,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/settings", + "//pkg/roachpb:with-mocks", ], ) diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 864995b1f2da..c75739da996c 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -11,34 +11,38 @@ package spanconfig import ( - "time" + "context" - "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/roachpb" ) -// CheckAndStartReconciliationJobInterval is a cluster setting to control how -// often the existence of the automatic span config reconciliation job will be -// checked. If the check concludes that the job doesn't exist it will be started. -var CheckAndStartReconciliationJobInterval = settings.RegisterDurationSetting( - "spanconfig.reconciliation_job.start_interval", - "how often to check for the span config reconciliation job exists and start it if it doesn't", - 10*time.Minute, - settings.NonNegativeDuration, -) +// KVAccessor mediates access to KV span configurations pertaining to a given +// tenant. +type KVAccessor interface { + // GetSpanConfigEntriesFor returns the span configurations that overlap with + // the given spans. + GetSpanConfigEntriesFor(ctx context.Context, spans []roachpb.Span) ([]roachpb.SpanConfigEntry, error) + + // UpdateSpanConfigEntries updates configurations for the given spans. This + // is a "targeted" API: the spans being deleted are expected to have been + // present with the exact same bounds; if spans are being updated with new + // configs, they're expected to have been present with the same bounds. When + // divvying up an existing span into multiple others with distinct configs, + // callers are to issue a delete for the previous span and upserts for the + // new ones. + UpdateSpanConfigEntries(ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry) error +} // ReconciliationDependencies captures what's needed by the span config // reconciliation job to perform its task. The job is responsible for // reconciling a tenant's zone configurations with the clusters span // configurations. type ReconciliationDependencies interface { - // TODO(zcfgs-pod): Placeholder comment until subsequent PRs add useful - // interfaces here. - // The job will want access to two interfaces to reconcile. - // 1. spanconfig.KVAccessor -- this will expose RPCs the job can use to fetch - // span configs from KV and update them. It'll be implemented by Node for the - // host tenant and the Connector for secondary tenants. - // 2. spanconfig.SQLWatcher -- this will maintain a rangefeed over - // system.{descriptors, zones} and be responsible for generating span config - // updates. The job will respond to these updates by issuing RPCs using the - // KVAccessor + KVAccessor + + // TODO(irfansharif): We'll also want access to a "SQLWatcher", something + // that watches for changes to system.{descriptor,zones} and be responsible + // for generating corresponding span config updates. Put together, the + // reconciliation job will react to these updates by installing them into KV + // through the KVAccessor. } diff --git a/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel new file mode 100644 index 000000000000..1794734ec43a --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel @@ -0,0 +1,46 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "spanconfigkvaccessor", + srcs = ["kvaccessor.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/spanconfig", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "spanconfigkvaccessor_test", + srcs = [ + "kvaccessor_test.go", + "main_test.go", + "validation_test.go", + ], + embed = [":spanconfigkvaccessor"], + deps = [ + "//pkg/base", + "//pkg/config/zonepb", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql/sqlutil", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go new file mode 100644 index 000000000000..ac2b25ae6ac7 --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go @@ -0,0 +1,415 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigkvaccessor + +import ( + "context" + "fmt" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// KVAccessor provides read/write access to all the span configurations for a +// CRDB cluster. It's a concrete implementation of the KVAccessor interface. +type KVAccessor struct { + db *kv.DB + ie sqlutil.InternalExecutor + settings *cluster.Settings + tableName string // typically system.span_configurations, but overridable for testing purposes +} + +var _ spanconfig.KVAccessor = &KVAccessor{} + +// New constructs a new Manager. +func New( + db *kv.DB, ie sqlutil.InternalExecutor, settings *cluster.Settings, tableFQN string, +) *KVAccessor { + return &KVAccessor{ + db: db, + ie: ie, + settings: settings, + tableName: tableFQN, + } +} + +var kvAccessorEnabled = settings.RegisterBoolSetting( + "spanconfig.kvaccessor_experimental.enabled", + "enable the use of the kv accessor", false).WithSystemOnly() + +// GetSpanConfigEntriesFor is part of the KVAccessor interface. +func (k *KVAccessor) GetSpanConfigEntriesFor( + ctx context.Context, spans []roachpb.Span, +) (resp []roachpb.SpanConfigEntry, retErr error) { + if kvAccessorEnabled.Get(&k.settings.SV) { + return nil, errors.New("use of span configs disabled") + } + if len(spans) == 0 { + return resp, nil + } + if err := validateSpans(spans); err != nil { + return nil, err + } + + getStmt, getQueryArgs := k.constructGetStmtAndArgs(spans) + it, err := k.ie.QueryIteratorEx(ctx, "get-span-cfgs", nil, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + getStmt, getQueryArgs..., + ) + if err != nil { + return nil, err + } + defer func() { + if closeErr := it.Close(); closeErr != nil { + resp, retErr = nil, errors.CombineErrors(retErr, closeErr) + } + }() + + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() + span := roachpb.Span{ + Key: []byte(*row[0].(*tree.DBytes)), + EndKey: []byte(*row[1].(*tree.DBytes)), + } + var conf roachpb.SpanConfig + if err := protoutil.Unmarshal(([]byte)(*row[2].(*tree.DBytes)), &conf); err != nil { + return nil, err + } + + resp = append(resp, roachpb.SpanConfigEntry{ + Span: span, + Config: conf, + }) + } + if err != nil { + return nil, err + } + return resp, nil +} + +// UpdateSpanConfigEntries is part of the KVAccessor interface. +func (k *KVAccessor) UpdateSpanConfigEntries( + ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry, +) error { + if err := validateUpdateArgs(toDelete, toUpsert); err != nil { + return err + } + + var deleteStmt string + var deleteQueryArgs []interface{} + if len(toDelete) > 0 { + deleteStmt, deleteQueryArgs = k.constructDeleteStmtAndArgs(toDelete) + } + + var upsertStmt, validationStmt string + var upsertQueryArgs, validationQueryArgs []interface{} + if len(toUpsert) > 0 { + var err error + upsertStmt, upsertQueryArgs, err = k.constructUpsertStmtAndArgs(toUpsert) + if err != nil { + return err + } + + validationStmt, validationQueryArgs = k.constructValidationStmtAndArgs(toUpsert) + } + + if err := k.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if len(toDelete) > 0 { + n, err := k.ie.ExecEx(ctx, "delete-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + deleteStmt, deleteQueryArgs..., + ) + if err != nil { + return err + } + if n != len(toDelete) { + return errors.AssertionFailedf("expected to delete %d row(s), deleted %d", len(toDelete), n) + } + } + + if len(toUpsert) == 0 { + // Nothing left to do + return nil + } + + if n, err := k.ie.ExecEx(ctx, "upsert-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + upsertStmt, upsertQueryArgs..., + ); err != nil { + return err + } else if n != len(toUpsert) { + return errors.AssertionFailedf("expected to upsert %d row(s), upserted %d", len(toUpsert), n) + } + + if datums, err := k.ie.QueryRowEx(ctx, "validate-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + validationStmt, validationQueryArgs..., + ); err != nil { + return err + } else if valid := bool(tree.MustBeDBool(datums[0])); !valid { + return errors.AssertionFailedf("expected to find single row containing upserted spans") + } + + return nil + }); err != nil { + return err + } + return nil +} + +// constructGetStmtAndArgs constructs the statement and query arguments needed +// to fetch span configs for the given spans. +func (k *KVAccessor) constructGetStmtAndArgs(spans []roachpb.Span) (string, []interface{}) { + // We want to fetch the overlapping span configs for each requested span in + // a single round trip and using only constrained index scans. For a single + // requested span, we effectively want to query the following: + // + // -- to find spans overlapping with [$start, $end) + // SELECT * FROM system.span_configurations + // WHERE start_key < $end AND end_key > $start + // + // With the naive form above that translates to an unbounded index scan on + // followed by a filter. We can do better by observing that start_key < + // end_key, and that spans are non-overlapping. + // + // SELECT * FROM span_configurations + // WHERE start_key >= $start AND start_key < $end + // UNION ALL + // SELECT * FROM ( + // SELECT * FROM span_configurations + // WHERE start_key < $start ORDER BY start_key DESC LIMIT 1 + // ) WHERE end_key > $start; + // + // The idea is to first find all spans that start within the query span, and + // then to include the span with the start key immediately preceding the + // query start if it also overlaps with the query span (achieved by + // the outer filter). We're intentional about not pushing the filter down into + // the query -- we want to select using only the start_key index. Doing so + // results in an unbounded index scan [ - $start) with the filter and limit + // applied after. + // + // To batch multiple query spans into the same statement, we make use of + // UNION ALL. + // + // ( ... statement above for 1st query span ...) + // UNION ALL + // ( ... statement above for 2nd query span ...) + // UNION ALL + // ... + // + var getStmtBuilder strings.Builder + queryArgs := make([]interface{}, len(spans)*2) + for i, sp := range spans { + if i > 0 { + getStmtBuilder.WriteString(`UNION ALL`) + } + + startKeyIdx, endKeyIdx := i*2, (i*2)+1 + queryArgs[startKeyIdx] = sp.Key + queryArgs[endKeyIdx] = sp.EndKey + + fmt.Fprintf(&getStmtBuilder, ` +SELECT start_key, end_key, config FROM %[1]s + WHERE start_key >= $%[2]d AND start_key < $%[3]d +UNION ALL +SELECT start_key, end_key, config FROM ( + SELECT start_key, end_key, config FROM %[1]s + WHERE start_key < $%[2]d ORDER BY start_key DESC LIMIT 1 +) WHERE end_key > $%[2]d +`, + k.tableName, // [1] + startKeyIdx+1, // [2] -- prepared statement placeholder (1-indexed) + endKeyIdx+1, // [3] -- prepared statement placeholder (1-indexed) + ) + } + return getStmtBuilder.String(), queryArgs +} + +// constructDeleteStmtAndArgs constructs the statement and query arguments +// needed to delete span configs for the given spans. +func (k *KVAccessor) constructDeleteStmtAndArgs(toDelete []roachpb.Span) (string, []interface{}) { + // We're constructing a single delete statement to delete all requested + // spans. It's of the form: + // + // DELETE FROM span_configurations WHERE (start_key, end_key) IN + // (VALUES ( ... 1st span ... ), ( ... 2nd span ...), ... ); + // + values := make([]string, len(toDelete)) + deleteQueryArgs := make([]interface{}, len(toDelete)*2) + for i, sp := range toDelete { + startKeyIdx, endKeyIdx := i*2, (i*2)+1 + deleteQueryArgs[startKeyIdx] = sp.Key + deleteQueryArgs[endKeyIdx] = sp.EndKey + values[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES)", + startKeyIdx+1, endKeyIdx+1) // prepared statement placeholders (1-indexed) + } + deleteStmt := fmt.Sprintf(`DELETE FROM %[1]s WHERE (start_key, end_key) IN (VALUES %[2]s)`, + k.tableName, strings.Join(values, ", ")) + return deleteStmt, deleteQueryArgs +} + +// constructUpsertStmtAndArgs constructs the statement and query arguments +// needed to upsert the given span config entries. +func (k *KVAccessor) constructUpsertStmtAndArgs( + toUpsert []roachpb.SpanConfigEntry, +) (string, []interface{}, error) { + // We're constructing a single upsert statement to upsert all requested + // spans. It's of the form: + // + // UPSERT INTO span_configurations (start_key, end_key, config) + // VALUES ( ... 1st span ... ), ( ... 2nd span ...), ... ; + // + upsertValues := make([]string, len(toUpsert)) + upsertQueryArgs := make([]interface{}, len(toUpsert)*3) + for i, entry := range toUpsert { + marshaled, err := protoutil.Marshal(&entry.Config) + if err != nil { + return "", nil, err + } + + startKeyIdx, endKeyIdx, configIdx := i*3, (i*3)+1, (i*3)+2 + upsertQueryArgs[startKeyIdx] = entry.Span.Key + upsertQueryArgs[endKeyIdx] = entry.Span.EndKey + upsertQueryArgs[configIdx] = marshaled + upsertValues[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES, $%d::BYTES)", + startKeyIdx+1, endKeyIdx+1, configIdx+1) // prepared statement placeholders (1-indexed) + } + upsertStmt := fmt.Sprintf(`UPSERT INTO %[1]s (start_key, end_key, config) VALUES %[2]s`, + k.tableName, strings.Join(upsertValues, ", ")) + return upsertStmt, upsertQueryArgs, nil +} + +// constructValidationStmtAndArgs constructs the statement and query arguments +// needed to validate that the spans being upserted don't violate table +// invariants (spans are non overlapping). +func (k *KVAccessor) constructValidationStmtAndArgs( + toUpsert []roachpb.SpanConfigEntry, +) (string, []interface{}) { + // We want to validate that upserting spans does not break the invariant + // that spans in the table are non-overlapping. We only need to validate + // the spans that are being upserted, and can use a query similar to + // what we do in GetSpanConfigEntriesFor. For a single upserted span, we + // want effectively validate using: + // + // SELECT count(*) = 1 FROM system.span_configurations + // WHERE start_key < $end AND end_key > $start + // + // Applying the GetSpanConfigEntriesFor treatment, we can arrive at: + // + // SELECT count(*) = 1 FROM ( + // SELECT * FROM span_configurations + // WHERE start_key >= 100 AND start_key < 105 + // UNION ALL + // SELECT * FROM ( + // SELECT * FROM span_configurations + // WHERE start_key < 100 ORDER BY start_key DESC LIMIT 1 + // ) WHERE end_key > 100 + // ) + // + // To batch multiple query spans into the same statement, we make use of + // ALL and UNION ALL. + // + // SELECT true = ALL( + // ( ... validation statement for 1st query span ...), + // UNION ALL + // ( ... validation statement for 2nd query span ...), + // ... + // ) + // + var validationInnerStmtBuilder strings.Builder + validationQueryArgs := make([]interface{}, len(toUpsert)*2) + for i, entry := range toUpsert { + if i > 0 { + validationInnerStmtBuilder.WriteString(`UNION ALL`) + } + + startKeyIdx, endKeyIdx := i*2, (i*2)+1 + validationQueryArgs[startKeyIdx] = entry.Span.Key + validationQueryArgs[endKeyIdx] = entry.Span.EndKey + + fmt.Fprintf(&validationInnerStmtBuilder, ` +SELECT count(*) = 1 FROM ( + SELECT start_key, end_key, config FROM %[1]s + WHERE start_key >= $%[2]d AND start_key < $%[3]d + UNION ALL + SELECT start_key, end_key, config FROM ( + SELECT start_key, end_key, config FROM %[1]s + WHERE start_key < $%[2]d ORDER BY start_key DESC LIMIT 1 + ) WHERE end_key > $%[2]d +) +`, + k.tableName, // [1] + startKeyIdx+1, // [2] -- prepared statement placeholder (1-indexed) + endKeyIdx+1, // [3] -- prepared statement placeholder (1-indexed) + ) + } + validationStmt := fmt.Sprintf("SELECT true = ALL(%s)", validationInnerStmtBuilder.String()) + return validationStmt, validationQueryArgs +} + +// validateUpdateArgs returns an error the arguments to UpdateSpanConfigEntries +// are malformed. All spans included in the toDelete and toUpsert list are +// expected to be valid and to have non-empty end keys. Spans are also expected +// to be non-overlapping with other spans in the same list. +func validateUpdateArgs(toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry) error { + spansToUpdate := func(ents []roachpb.SpanConfigEntry) []roachpb.Span { + spans := make([]roachpb.Span, len(ents)) + for i, ent := range ents { + spans[i] = ent.Span + } + return spans + }(toUpsert) + + for _, list := range [][]roachpb.Span{toDelete, spansToUpdate} { + if err := validateSpans(list); err != nil { + return err + } + + spans := make([]roachpb.Span, len(list)) + copy(spans, list) + sort.Sort(roachpb.Spans(spans)) + for i := range spans { + if i == 0 { + continue + } + + if spans[i].Overlaps(spans[i-1]) { + return errors.AssertionFailedf("overlapping spans %s and %s in same list", + spans[i-1], spans[i]) + } + } + } + + return nil +} + +// validateSpans returns an error if any of the spans are invalid or have an +// empty end key. +func validateSpans(spans []roachpb.Span) error { + for _, span := range spans { + if !span.Valid() || len(span.EndKey) == 0 { + return errors.AssertionFailedf("invalid span: %s", span) + } + } + return nil +} diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go new file mode 100644 index 000000000000..9b35890d564f --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go @@ -0,0 +1,183 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigkvaccessor_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// TODO(irfansharif): This interface would benefit from a datadriven test. + +func TestKVAccessor(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + span := func(start, end string) roachpb.Span { + return roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)} + } + conf := func(maxBytes int64) roachpb.SpanConfig { + c := zonepb.DefaultZoneConfigRef().AsSpanConfig() + c.RangeMaxBytes = maxBytes + return c + } + toEntries := func(spans []roachpb.Span) []roachpb.SpanConfigEntry { + var entries []roachpb.SpanConfigEntry + for i, sp := range spans { + entries = append(entries, roachpb.SpanConfigEntry{ + Span: sp, + Config: conf(2000 + int64(i)), + }) + } + return entries + } + mergeLastTwo := func(spans []roachpb.Span) ([]roachpb.Span, roachpb.Span) { + prevLastSpan := spans[len(spans)-1] + spans = spans[:len(spans)-1] + spans[len(spans)-1].EndKey = prevLastSpan.EndKey + return spans, prevLastSpan + } + + testSpans := []roachpb.Span{ + span("a", "b"), + span("b", "c"), + span("c", "d"), + span("d", "e"), + } + everythingSpan := span("a", "e") + testEntries := toEntries(testSpans) + + const dummySpanConfigurationsFQN = "defaultdb.public.dummy_span_configurations" + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb.Exec(t, fmt.Sprintf("CREATE TABLE %s (LIKE system.span_configurations INCLUDING ALL)", dummySpanConfigurationsFQN)) + accessor := spanconfigkvaccessor.New( + tc.Server(0).DB(), + tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor), + tc.Server(0).ClusterSettings(), + dummySpanConfigurationsFQN, + ) + + { // With an empty slate. + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Empty(t, entries) + + err = accessor.UpdateSpanConfigEntries(ctx, []roachpb.Span{everythingSpan}, nil) + require.Truef(t, testutils.IsError(err, "expected to delete 1 row"), err.Error()) + } + + { // Verify that writing and reading a single entry behaves as expected. + testEntry := testEntries[0] + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, nil, []roachpb.SpanConfigEntry{testEntry})) + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, 1) + require.True(t, entries[0].Equal(testEntry)) + + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, []roachpb.Span{testEntry.Span}, nil)) + entries, err = accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, 0) + } + + { // Verify that adding all entries does in fact add all entries. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, nil, testEntries)) + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(testSpans)) + for i, got := range entries { + require.True(t, got.Equal(testEntries[i])) + } + } + + { // Verify that updating entries (including noops) show up as such. + for i := range testEntries { + if i%2 == 0 { + continue + } + testEntries[i].Config.RangeMaxBytes += 100 + } + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, nil, testEntries)) + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(testSpans)) + for i, got := range entries { + require.True(t, got.Equal(testEntries[i])) + } + } + + { // Verify that fetching entries for multiple spans behaves as expected. + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{testSpans[1], testSpans[2]}) + require.Nil(t, err) + require.Len(t, entries, 2) + require.True(t, entries[0].Equal(testEntries[1])) + require.True(t, entries[1].Equal(testEntries[2])) + } + + { // Verify that deleting entries actually removes them. + const toDelete = 2 + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, testSpans[:toDelete], nil)) + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(testSpans)-toDelete) + for i, got := range entries { + require.True(t, got.Equal(testEntries[toDelete+i])) + } + + // Attempts to delete non existent spans should error out. + require.NotNil(t, accessor.UpdateSpanConfigEntries(ctx, testSpans[:toDelete], nil)) + + // Attempts to re-write previously deleted spans should go through. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, nil, testEntries[:toDelete])) + entries, err = accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(testSpans)) + for i, got := range entries { + require.True(t, got.Equal(testEntries[i])) + } + } + + { // Verify that we're able to re-partition span configs correctly. + newTestSpans, prevLast := mergeLastTwo(testSpans) + newTestEntries := toEntries(newTestSpans) + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, []roachpb.Span{prevLast}, newTestEntries)) + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(newTestSpans)) + for i, got := range entries { + require.True(t, got.Equal(newTestEntries[i])) + } + + // We should also be able to "unmerge" correctly. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, newTestSpans, testEntries)) + entries, err = accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, len(testSpans)) + for i, got := range entries { + require.True(t, got.Equal(testEntries[i])) + } + } +} diff --git a/pkg/spanconfig/spanconfigkvaccessor/main_test.go b/pkg/spanconfig/spanconfigkvaccessor/main_test.go new file mode 100644 index 000000000000..971eaf460ca8 --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigkvaccessor_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/spanconfig/spanconfigkvaccessor/validation_test.go b/pkg/spanconfig/spanconfigkvaccessor/validation_test.go new file mode 100644 index 000000000000..196bf29cd924 --- /dev/null +++ b/pkg/spanconfig/spanconfigkvaccessor/validation_test.go @@ -0,0 +1,97 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigkvaccessor + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestValidation(t *testing.T) { + defer leaktest.AfterTest(t)() + + for _, tc := range []struct { + toDelete []roachpb.Span + toUpsert []roachpb.SpanConfigEntry + expErr string + }{ + { + toUpsert: nil, toDelete: nil, + expErr: "", + }, + { + toDelete: []roachpb.Span{ + {Key: roachpb.Key("a")}, // empty end key in delete list + }, + expErr: "invalid span: a", + }, + { + toUpsert: []roachpb.SpanConfigEntry{ + { + Span: roachpb.Span{Key: roachpb.Key("a")}, // empty end key in update list + }, + }, + expErr: "invalid span: a", + }, + { + toUpsert: []roachpb.SpanConfigEntry{ + { + Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + }, + }, + expErr: "invalid span: {b-a}", + }, + { + toDelete: []roachpb.Span{ + {Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + }, + expErr: "invalid span: {b-a}", + }, + { + toDelete: []roachpb.Span{ + {Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, // overlapping spans in the same list + {Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + }, + expErr: "overlapping spans {a-c} and {b-c} in same list", + }, + { + toUpsert: []roachpb.SpanConfigEntry{ // overlapping spans in the same list + { + Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, + }, + { + Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + }, + }, + expErr: "overlapping spans {a-c} and {b-c} in same list", + }, + { + toDelete: []roachpb.Span{ + {Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, + }, + toUpsert: []roachpb.SpanConfigEntry{ // overlapping spans in different lists + { + Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, + }, + { + Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + }, + }, + expErr: "", + }, + } { + require.True(t, testutils.IsError(validateUpdateArgs(tc.toDelete, tc.toUpsert), tc.expErr)) + } +} diff --git a/pkg/spanconfig/spanconfigmanager/BUILD.bazel b/pkg/spanconfig/spanconfigmanager/BUILD.bazel index dc721d23bc8c..8f5131b63899 100644 --- a/pkg/spanconfig/spanconfigmanager/BUILD.bazel +++ b/pkg/spanconfig/spanconfigmanager/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/kv", "//pkg/security", + "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/sql/sem/tree", diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index bb77b7cf2ab9..97306884a608 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -30,7 +31,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -var _ spanconfig.ReconciliationDependencies = &Manager{} +// checkAndStartReconciliationJobInterval is a cluster setting to control how +// often the existence of the automatic span config reconciliation job will be +// checked. If the check concludes that the job doesn't exist it will be started. +var checkAndStartReconciliationJobInterval = settings.RegisterDurationSetting( + "spanconfig.reconciliation_job.check_interval", + "the frequency at which to check if the span config reconciliation job exists (and to start it if not)", + 10*time.Minute, + settings.NonNegativeDuration, +) // Manager is the coordinator of the span config subsystem. It is responsible // for the following tasks: @@ -46,8 +55,12 @@ type Manager struct { stopper *stop.Stopper settings *cluster.Settings knobs *spanconfig.TestingKnobs + + spanconfig.KVAccessor } +var _ spanconfig.ReconciliationDependencies = &Manager{} + // New constructs a new Manager. func New( db *kv.DB, @@ -55,18 +68,20 @@ func New( ie sqlutil.InternalExecutor, stopper *stop.Stopper, settings *cluster.Settings, + kvAccessor spanconfig.KVAccessor, knobs *spanconfig.TestingKnobs, ) *Manager { if knobs == nil { knobs = &spanconfig.TestingKnobs{} } return &Manager{ - db: db, - jr: jr, - ie: ie, - stopper: stopper, - settings: settings, - knobs: knobs, + db: db, + jr: jr, + ie: ie, + stopper: stopper, + settings: settings, + knobs: knobs, + KVAccessor: kvAccessor, } } @@ -83,7 +98,7 @@ func (m *Manager) Start(ctx context.Context) error { func (m *Manager) run(ctx context.Context) { reconciliationIntervalChanged := make(chan struct{}, 1) - spanconfig.CheckAndStartReconciliationJobInterval.SetOnChange( + checkAndStartReconciliationJobInterval.SetOnChange( &m.settings.SV, func(ctx context.Context) { select { case reconciliationIntervalChanged <- struct{}{}: @@ -98,7 +113,7 @@ func (m *Manager) run(ctx context.Context) { // if for some reason it does not. for { timer.Reset(timeutil.Until( - lastChecked.Add(spanconfig.CheckAndStartReconciliationJobInterval.Get(&m.settings.SV)), + lastChecked.Add(checkAndStartReconciliationJobInterval.Get(&m.settings.SV)), )) select { case <-timer.C: diff --git a/pkg/spanconfig/spanconfigmanager/manager_test.go b/pkg/spanconfig/spanconfigmanager/manager_test.go index d24b43c85d29..2a365e4ebc61 100644 --- a/pkg/spanconfig/spanconfigmanager/manager_test.go +++ b/pkg/spanconfig/spanconfigmanager/manager_test.go @@ -66,6 +66,7 @@ func TestManagerConcurrentJobCreation(t *testing.T) { ts.InternalExecutor().(*sql.InternalExecutor), ts.Stopper(), ts.ClusterSettings(), + ts.SpanConfigAccessor().(spanconfig.KVAccessor), &spanconfig.TestingKnobs{ ManagerCreatedJobInterceptor: func(jobI interface{}) { job := jobI.(*jobs.Job) @@ -151,6 +152,7 @@ func TestManagerStartsJobIfFailed(t *testing.T) { ts.InternalExecutor().(*sql.InternalExecutor), ts.Stopper(), ts.ClusterSettings(), + ts.SpanConfigAccessor().(spanconfig.KVAccessor), &spanconfig.TestingKnobs{ ManagerAfterCheckedReconciliationJobExistsInterceptor: func(exists bool) { require.False(t, exists) diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 60203b69e08c..9a0fc1523d08 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -268,6 +268,7 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { target.AddDescriptor(systemschema.DatabaseRoleSettingsTable) target.AddDescriptorForSystemTenant(systemschema.TenantUsageTable) target.AddDescriptor(systemschema.SQLInstancesTable) + target.AddDescriptorForSystemTenant(systemschema.SpanConfigurationsTable) // Adding a new system table? It should be added here to the metadata schema, // and also created as a migration for older clusters. The includedInBootstrap diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index 5ea4574428e5..e91673272ef1 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -78,6 +78,7 @@ const ( DatabaseRoleSettingsTableName SystemTableName = "database_role_settings" TenantUsageTableName SystemTableName = "tenant_usage" SQLInstancesTableName SystemTableName = "sql_instances" + SpanConfigurationsTableName SystemTableName = "span_configurations" ) // Oid for virtual database and table. diff --git a/pkg/sql/catalog/catprivilege/system.go b/pkg/sql/catalog/catprivilege/system.go index 6c86009cc0c7..8e7a2b4bb0b9 100644 --- a/pkg/sql/catalog/catprivilege/system.go +++ b/pkg/sql/catalog/catprivilege/system.go @@ -61,6 +61,7 @@ var ( catconstants.DatabaseRoleSettingsTableName, catconstants.TenantUsageTableName, catconstants.SQLInstancesTableName, + catconstants.SpanConfigurationsTableName, } systemSuperuserPrivileges = func() map[descpb.NameInfo]privilege.List { diff --git a/pkg/sql/catalog/systemschema/BUILD.bazel b/pkg/sql/catalog/systemschema/BUILD.bazel index 963a31b4717d..566bfd309558 100644 --- a/pkg/sql/catalog/systemschema/BUILD.bazel +++ b/pkg/sql/catalog/systemschema/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/privilege", + "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 5ea74523c861..b371d87a4849 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -610,6 +611,15 @@ CREATE TABLE system.sql_instances ( session_id BYTES, FAMILY "primary" (id, addr, session_id) )` + + SpanConfigurationsTableSchema = ` +CREATE TABLE system.span_configurations ( + start_key BYTES NOT NULL PRIMARY KEY, + end_key BYTES NOT NULL, + config BYTES NOT NULL, + CONSTRAINT check_bounds CHECK (start_key < end_key), + FAMILY "primary" (start_key, end_key, config) +)` ) func pk(name string) descpb.IndexDescriptor { @@ -2213,6 +2223,37 @@ var ( pk("id"), )) + // SpanConfigurationsTable is the descriptor for the system tenant's span + // configurations table. + SpanConfigurationsTable = registerSystemTable( + SpanConfigurationsTableSchema, + systemTable( + catconstants.SpanConfigurationsTableName, + keys.SpanConfigurationsTableID, + []descpb.ColumnDescriptor{ + {Name: "start_key", ID: 1, Type: types.Bytes}, + {Name: "end_key", ID: 2, Type: types.Bytes}, + {Name: "config", ID: 3, Type: types.Bytes}, + }, + []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{"start_key", "end_key", "config"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3}, + }, + }, + pk("start_key"), + ), + func(tbl *descpb.TableDescriptor) { + tbl.Checks = []*descpb.TableDescriptor_CheckConstraint{{ + Name: "check_bounds", + Expr: "start_key < end_key", + ColumnIDs: []descpb.ColumnID{1, 2}, + }} + }, + ) + // UnleasableSystemDescriptors contains the system descriptors which cannot // be leased. This includes the lease table itself, among others. UnleasableSystemDescriptors = func(s []catalog.Descriptor) map[descpb.ID]catalog.Descriptor { @@ -2229,3 +2270,6 @@ var ( RangeEventTable, }) ) + +// SpanConfigurationsTableName represents system.span_configurations. +var SpanConfigurationsTableName = tree.NewTableNameWithSchema("system", tree.PublicSchemaName, tree.Name(catconstants.SpanConfigurationsTableName)) diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 7589b8cbc704..2e9ea838bd0b 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -925,16 +925,6 @@ system public join_tokens root GRANT system public join_tokens root INSERT system public join_tokens root SELECT system public join_tokens root UPDATE -system public tenant_usage admin DELETE -system public tenant_usage admin GRANT -system public tenant_usage admin INSERT -system public tenant_usage admin SELECT -system public tenant_usage admin UPDATE -system public tenant_usage root DELETE -system public tenant_usage root GRANT -system public tenant_usage root INSERT -system public tenant_usage root SELECT -system public tenant_usage root UPDATE system public statement_statistics admin GRANT system public statement_statistics admin SELECT system public statement_statistics root GRANT @@ -953,6 +943,16 @@ system public database_role_settings root GRANT system public database_role_settings root INSERT system public database_role_settings root SELECT system public database_role_settings root UPDATE +system public tenant_usage admin DELETE +system public tenant_usage admin GRANT +system public tenant_usage admin INSERT +system public tenant_usage admin SELECT +system public tenant_usage admin UPDATE +system public tenant_usage root DELETE +system public tenant_usage root GRANT +system public tenant_usage root INSERT +system public tenant_usage root SELECT +system public tenant_usage root UPDATE system public sql_instances admin DELETE system public sql_instances admin GRANT system public sql_instances admin INSERT @@ -963,6 +963,16 @@ system public sql_instances root GRANT system public sql_instances root INSERT system public sql_instances root SELECT system public sql_instances root UPDATE +system public span_configurations admin DELETE +system public span_configurations admin GRANT +system public span_configurations admin INSERT +system public span_configurations admin SELECT +system public span_configurations admin UPDATE +system public span_configurations root DELETE +system public span_configurations root GRANT +system public span_configurations root INSERT +system public span_configurations root SELECT +system public span_configurations root UPDATE a pg_extension NULL admin ALL a pg_extension NULL readwrite ALL a pg_extension NULL root ALL @@ -1425,6 +1435,11 @@ system public settings root GRA system public settings root INSERT system public settings root SELECT system public settings root UPDATE +system public span_configurations root DELETE +system public span_configurations root GRANT +system public span_configurations root INSERT +system public span_configurations root SELECT +system public span_configurations root UPDATE system public sql_instances root DELETE system public sql_instances root GRANT system public sql_instances root INSERT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 020202815666..cedcc23912bf 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1365,6 +1365,7 @@ system public transaction_statistics BASE T system public database_role_settings BASE TABLE YES 1 system public tenant_usage BASE TABLE YES 1 system public sql_instances BASE TABLE YES 1 +system public span_configurations BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -1530,6 +1531,11 @@ system public 630200280_6_1_not_null system public 630200280_6_2_not_null system public settings CHECK NO NO system public 630200280_6_3_not_null system public settings CHECK NO NO system public primary system public settings PRIMARY KEY NO NO +system public 630200280_47_1_not_null system public span_configurations CHECK NO NO +system public 630200280_47_2_not_null system public span_configurations CHECK NO NO +system public 630200280_47_3_not_null system public span_configurations CHECK NO NO +system public check_bounds system public span_configurations CHECK NO NO +system public primary system public span_configurations PRIMARY KEY NO NO system public 630200280_46_1_not_null system public sql_instances CHECK NO NO system public primary system public sql_instances PRIMARY KEY NO NO system public 630200280_39_1_not_null system public sqlliveness CHECK NO NO @@ -1750,6 +1756,9 @@ system public 630200280_45_2_not_null system public 630200280_45_3_not_null next_instance_id IS NOT NULL system public 630200280_45_4_not_null last_update IS NOT NULL system public 630200280_46_1_not_null id IS NOT NULL +system public 630200280_47_1_not_null start_key IS NOT NULL +system public 630200280_47_2_not_null end_key IS NOT NULL +system public 630200280_47_3_not_null config IS NOT NULL system public 630200280_4_1_not_null username IS NOT NULL system public 630200280_4_3_not_null isRole IS NOT NULL system public 630200280_5_1_not_null id IS NOT NULL @@ -1758,6 +1767,7 @@ system public 630200280_6_2_not_null system public 630200280_6_3_not_null lastUpdated IS NOT NULL system public 630200280_8_1_not_null id IS NOT NULL system public 630200280_8_2_not_null active IS NOT NULL +system public check_bounds ((start_key < end_key)) system public check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_shard_8 ((crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8))) system public check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 ((crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8))) system public check_singleton ((singleton)) @@ -1812,6 +1822,9 @@ system public role_options option system public role_options username system public primary system public scheduled_jobs schedule_id system public primary system public settings name system public primary +system public span_configurations end_key system public check_bounds +system public span_configurations start_key system public check_bounds +system public span_configurations start_key system public primary system public sql_instances id system public primary system public sqlliveness session_id system public primary system public statement_bundle_chunks id system public primary @@ -2043,6 +2056,9 @@ system public settings lastUpdated system public settings name 1 system public settings value 2 system public settings valueType 4 +system public span_configurations config 3 +system public span_configurations end_key 2 +system public span_configurations start_key 1 system pg_extension spatial_ref_sys auth_name 2 system pg_extension spatial_ref_sys auth_srid 3 system pg_extension spatial_ref_sys proj4text 5 @@ -3047,6 +3063,16 @@ NULL root system public settings NULL root system public settings INSERT NULL NO NULL root system public settings SELECT NULL YES NULL root system public settings UPDATE NULL NO +NULL admin system public span_configurations DELETE NULL NO +NULL admin system public span_configurations GRANT NULL NO +NULL admin system public span_configurations INSERT NULL NO +NULL admin system public span_configurations SELECT NULL YES +NULL admin system public span_configurations UPDATE NULL NO +NULL root system public span_configurations DELETE NULL NO +NULL root system public span_configurations GRANT NULL NO +NULL root system public span_configurations INSERT NULL NO +NULL root system public span_configurations SELECT NULL YES +NULL root system public span_configurations UPDATE NULL NO NULL admin system public sql_instances DELETE NULL NO NULL admin system public sql_instances GRANT NULL NO NULL admin system public sql_instances INSERT NULL NO @@ -3759,6 +3785,16 @@ NULL root system public sql_instances NULL root system public sql_instances INSERT NULL NO NULL root system public sql_instances SELECT NULL YES NULL root system public sql_instances UPDATE NULL NO +NULL admin system public span_configurations DELETE NULL NO +NULL admin system public span_configurations GRANT NULL NO +NULL admin system public span_configurations INSERT NULL NO +NULL admin system public span_configurations SELECT NULL YES +NULL admin system public span_configurations UPDATE NULL NO +NULL root system public span_configurations DELETE NULL NO +NULL root system public span_configurations GRANT NULL NO +NULL root system public span_configurations INSERT NULL NO +NULL root system public span_configurations SELECT NULL YES +NULL root system public span_configurations UPDATE NULL NO statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 3e286b18ab48..f7474c23cf56 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1037,6 +1037,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3 +2528390115 47 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL 2 2621181441 15 3 false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2 2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 @@ -1118,6 +1119,7 @@ indexrelid operator_argument_type_oid operator_argument_position 2407840836 0 1 2407840836 0 2 2407840836 0 3 +2528390115 0 1 2621181440 0 1 2621181440 0 2 2621181441 0 1 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 38adea43fbb4f8926a80b6e4f51eaae1c4f52b4b..4463d92fbdf3c48d7c1a7cdb72b44ec2d7397529 100644 GIT binary patch delta 131 zcmX^5gmM0J#trv$CSQ~nXEd3tC@DTUL5F{`jIJ$QL~`n|<8?#;YP1uV diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index b200e2dfb9d2..997900fd8c1c 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -209,11 +209,12 @@ SELECT * FROM [SHOW TABLES FROM system] ---- schema_name table_name type owner estimated_row_count locality public descriptor table NULL 0 NULL +public span_configurations table NULL 0 NULL +public sql_instances table NULL 0 NULL public tenant_usage table NULL 0 NULL public database_role_settings table NULL 0 NULL public transaction_statistics table NULL 0 NULL public statement_statistics table NULL 0 NULL -public sql_instances table NULL 0 NULL public join_tokens table NULL 0 NULL public migrations table NULL 0 NULL public sqlliveness table NULL 0 NULL @@ -249,9 +250,10 @@ SELECT * FROM [SHOW TABLES FROM system WITH COMMENT] ---- schema_name table_name type owner estimated_row_count locality comment public descriptor table NULL 0 NULL · +public span_configurations table NULL 0 NULL · +public sql_instances table NULL 0 NULL · public tenant_usage table NULL 0 NULL · public database_role_settings table NULL 0 NULL · -public sql_instances table NULL 0 NULL · public transaction_statistics table NULL 0 NULL · public statement_statistics table NULL 0 NULL · public join_tokens table NULL 0 NULL · diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 27a300066560..0b0972973480 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -31,6 +31,7 @@ public role_members table NULL 0 NULL public role_options table NULL 0 NULL public scheduled_jobs table NULL 0 NULL public settings table NULL 0 NULL +public span_configurations table NULL 0 NULL public sql_instances table NULL 0 NULL public sqlliveness table NULL 0 NULL public statement_bundle_chunks table NULL 0 NULL @@ -85,6 +86,7 @@ SELECT id FROM system.descriptor 44 45 46 +47 50 51 52 @@ -392,6 +394,16 @@ system public settings root GRANT system public settings root INSERT system public settings root SELECT system public settings root UPDATE +system public span_configurations admin DELETE +system public span_configurations admin GRANT +system public span_configurations admin INSERT +system public span_configurations admin SELECT +system public span_configurations admin UPDATE +system public span_configurations root DELETE +system public span_configurations root GRANT +system public span_configurations root INSERT +system public span_configurations root SELECT +system public span_configurations root UPDATE system public sql_instances admin DELETE system public sql_instances admin GRANT system public sql_instances admin INSERT diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace index 5af46dc6eee7..e28219456509 100644 --- a/pkg/sql/logictest/testdata/logic_test/system_namespace +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -33,6 +33,7 @@ SELECT * FROM system.namespace 1 29 role_options 33 1 29 scheduled_jobs 37 1 29 settings 6 +1 29 span_configurations 47 1 29 sql_instances 46 1 29 sqlliveness 39 1 29 statement_bundle_chunks 34 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic index 94c5ac3bb07c..88bd629d7221 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic @@ -41,7 +41,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -62,7 +62,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -86,7 +86,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -110,7 +110,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -132,8 +132,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -156,8 +156,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -192,7 +192,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -213,7 +213,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -237,7 +237,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -261,7 +261,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -283,8 +283,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 Put to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -307,8 +307,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 Put to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -343,8 +343,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -368,8 +368,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -393,8 +393,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -416,9 +416,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Put to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -441,9 +441,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Put to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Put to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -478,7 +478,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -499,7 +499,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -523,7 +523,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 DelRng to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -547,8 +547,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -570,9 +570,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Del to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Del to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -595,9 +595,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 2 Del to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 2 Del to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -644,9 +644,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 2 Get to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 2 Get to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -667,10 +667,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 1 Put to (n1,s1):1 -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 1 Put to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -692,10 +692,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 Get to (n1,s1):1 -dist sender send r42: sending batch 1 Del to (n1,s1):1 -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 Get to (n1,s1):1 +dist sender send r43: sending batch 1 Del to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. statement ok @@ -719,9 +719,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 1 DelRng to (n1,s1):1 -dist sender send r42: sending batch 1 Scan to (n1,s1):1 -dist sender send r42: sending batch 1 Del, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng to (n1,s1):1 +dist sender send r43: sending batch 1 Scan to (n1,s1):1 +dist sender send r43: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -749,9 +749,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -774,6 +774,6 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 2 CPut to (n1,s1):1 -dist sender send r42: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 2 CPut to (n1,s1):1 +dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index 1d859a01979f..38e752efb342 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -231,9 +231,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%' ---- batch flow coordinator DelRange /Table/57/1 - /Table/57/2 -dist sender send r42: sending batch 1 DelRng to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng to (n1,s1):1 batch flow coordinator DelRange /Table/57/1/601/0 - /Table/57/2 -dist sender send r42: sending batch 1 DelRng to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -249,4 +249,4 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- batch flow coordinator DelRange /Table/57/1/5 - /Table/57/1/5/# -dist sender send r42: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r43: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index e9ef58ac97bf..c2e32af26fe9 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -238,7 +238,7 @@ SET tracing = on; INSERT INTO t.kv3 (k, v) VALUES (1,1); SET tracing = off query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%' ---- -r43: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +r44: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 node received request: 1 CPut, 1 EndTxn # Check that we can run set tracing regardless of the current tracing state. @@ -314,9 +314,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r43: sending batch 1 CPut to (n1,s1):1 -dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 -dist sender send r43: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r44: sending batch 1 CPut to (n1,s1):1 +dist sender send r44: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r44: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # Make another session trace. statement ok @@ -345,9 +345,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r43: sending batch 4 CPut, 1 EndTxn to (n1,s1):1 -dist sender send r43: sending batch 5 CPut to (n1,s1):1 -dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r44: sending batch 4 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r44: sending batch 5 CPut to (n1,s1):1 +dist sender send r44: sending batch 1 EndTxn to (n1,s1):1 # make a table with some big strings in it. statement ok @@ -366,8 +366,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r43: sending batch 6 CPut to (n1,s1):1 -dist sender send r43: sending batch 6 CPut to (n1,s1):1 -dist sender send r43: sending batch 6 CPut to (n1,s1):1 -dist sender send r43: sending batch 6 CPut to (n1,s1):1 -dist sender send r43: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r44: sending batch 6 CPut to (n1,s1):1 +dist sender send r44: sending batch 6 CPut to (n1,s1):1 +dist sender send r44: sending batch 6 CPut to (n1,s1):1 +dist sender send r44: sending batch 6 CPut to (n1,s1):1 +dist sender send r44: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 28cc2d4efa76..e6c3d8f82e1f 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -561,7 +561,7 @@ func TestPGPreparedQuery(t *testing.T) { Results("users", "primary", false, 3, "isRole", "N/A", true, false), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(34), + baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(35), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal", gosql.NullString{}).Others(4), diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 1086f41e8669..8d345333b5be 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4857,6 +4857,32 @@ value if you rely on the HLC for accuracy.`, }, ), + // Return a pretty string for a given span, skipping the specified number of + // fields. + "crdb_internal.pretty_span": makeBuiltin( + tree.FunctionProperties{ + Category: categorySystemInfo, + }, + tree.Overload{ + Types: tree.ArgTypes{ + {"raw_key_start", types.Bytes}, + {"raw_key_end", types.Bytes}, + {"skip_fields", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + span := roachpb.Span{ + Key: roachpb.Key(tree.MustBeDBytes(args[0])), + EndKey: roachpb.Key(tree.MustBeDBytes(args[1])), + } + skip := int(tree.MustBeDInt(args[2])) + return tree.NewDString(catalogkeys.PrettySpan(nil /* valDirs */, span, skip)), nil + }, + Info: "This function is used only by CockroachDB's developers for testing purposes.", + Volatility: tree.VolatilityImmutable, + }, + ), + // Return statistics about a range. "crdb_internal.range_stats": makeBuiltin( tree.FunctionProperties{ diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 347cf72acbdd..588484a61981 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -146,8 +146,9 @@ func TestInitialKeysAndSplits(t *testing.T) { // statement strings that describe each system table with the TableDescriptor // literals that are actually used at runtime. This ensures we can use the hand- // written literals instead of having to evaluate the `CREATE TABLE` statements -// before initialization and with limited SQL machinery bootstraped, while still -// confident that the result is the same as if `CREATE TABLE` had been run. +// before initialization and with limited SQL machinery bootstrapped, while +// still confident that the result is the same as if `CREATE TABLE` had been +// run. // // This test may also be useful when writing a new system table: // adding the new schema along with a trivial, empty TableDescriptor literal @@ -172,7 +173,7 @@ func TestSystemTableLiterals(t *testing.T) { } } - const expectedNumberOfSystemTables = 36 + const expectedNumberOfSystemTables = 37 require.Equal(t, expectedNumberOfSystemTables, len(testcases)) for name, test := range testcases { diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 21f144b32269..d327f190ee9f 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -1,6 +1,6 @@ initial-keys tenant=system ---- -81 keys: +83 keys: /System/"desc-idgen" /Table/3/1/1/2/1 /Table/3/1/3/2/1 @@ -38,6 +38,7 @@ initial-keys tenant=system /Table/3/1/44/2/1 /Table/3/1/45/2/1 /Table/3/1/46/2/1 + /Table/3/1/47/2/1 /Table/5/1/0/2/1 /Table/5/1/1/2/1 /Table/5/1/16/2/1 @@ -68,6 +69,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"role_options"/4/1 /NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 /NamespaceTable/30/1/1/29/"settings"/4/1 + /NamespaceTable/30/1/1/29/"span_configurations"/4/1 /NamespaceTable/30/1/1/29/"sql_instances"/4/1 /NamespaceTable/30/1/1/29/"sqlliveness"/4/1 /NamespaceTable/30/1/1/29/"statement_bundle_chunks"/4/1 @@ -82,7 +84,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"users"/4/1 /NamespaceTable/30/1/1/29/"web_sessions"/4/1 /NamespaceTable/30/1/1/29/"zones"/4/1 -36 splits: +37 splits: /Table/11 /Table/12 /Table/13 @@ -119,6 +121,7 @@ initial-keys tenant=system /Table/44 /Table/45 /Table/46 + /Table/47 initial-keys tenant=5 ---- diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index c630c60096f3..4c63037420fa 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -114,6 +114,10 @@ type TestServerInterface interface { // MigrationServer returns the internal *migrationServer as in interface{} MigrationServer() interface{} + // SpanConfigAccessor returns the underlying spanconfig.KVAccessor as an + // interface{}. + SpanConfigAccessor() interface{} + // SQLServer returns the *sql.Server as an interface{}. SQLServer() interface{}