diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 352d033e0d8f..1aaeb054cfb6 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2856,6 +2856,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 46ee156323e9..a9084663c54f 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -161,8 +161,8 @@ ALL_TESTS = [ "//pkg/migration/migrationcluster:migrationcluster_test", "//pkg/migration/migrationmanager:migrationmanager_test", "//pkg/migration/migrations:migrations_test", + "//pkg/roachpb:external_test", "//pkg/roachpb:roachpb_test", - "//pkg/roachpb:string_test", "//pkg/rpc/nodedialer:nodedialer_test", "//pkg/rpc:rpc_test", "//pkg/security/certmgr:certmgr_test", @@ -183,6 +183,7 @@ ALL_TESTS = [ "//pkg/server:server_test", "//pkg/settings:settings_test", "//pkg/spanconfig/spanconfigmanager:spanconfigmanager_test", + "//pkg/spanconfig:spanconfig_test", "//pkg/sql/catalog/catalogkeys:catalogkeys_test", "//pkg/sql/catalog/catalogkv:catalogkv_test", "//pkg/sql/catalog/catformat:catformat_test", 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 fb6889aad3f8..571691b54205 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..f1de54b11420 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 { @@ -502,3 +506,42 @@ func (c *Connector) tryForgetClient(ctx context.Context, client roachpb.Internal c.mu.client = nil } } + +// GetSpanConfigEntriesFor implements the spanconfig.KVAccessor interface. +func (c *Connector) GetSpanConfigEntriesFor( + ctx context.Context, spans []roachpb.Span, +) ([][]roachpb.SpanConfigEntry, error) { + for ctx.Err() == nil { + client, err := c.getClient(ctx) + if err != nil { + continue + } + resp, err := client.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{ + Spans: spans, + }) + if err != nil { + return nil, err + } + return resp.Unnest(), nil + } + return nil, ctx.Err() +} + +// UpdateSpanConfigEntries implements the spanconfig.KVAccessor +// interface. +func (c *Connector) UpdateSpanConfigEntries( + ctx context.Context, update []roachpb.SpanConfigEntry, delete []roachpb.Span, +) error { + for ctx.Err() == nil { + client, err := c.getClient(ctx) + if err != nil { + continue + } + _, err = client.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{ + SpanConfigsToUpdate: update, + SpansToDelete: delete, + }) + return err + } + return ctx.Err() +} 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/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..bd34df025154 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,105 @@ 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 { + const unhydratedZoneConfigMessage = "expected hydrated zone config: %s unset" + if z.RangeMaxBytes == nil { + return errors.AssertionFailedf(unhydratedZoneConfigMessage, "RangeMaxBytes") + } + if z.RangeMinBytes == nil { + return errors.AssertionFailedf(unhydratedZoneConfigMessage, "RangeMinBytes") + } + if z.GC == nil { + return errors.AssertionFailedf(unhydratedZoneConfigMessage, "GCPolicy") + } + if z.NumReplicas == nil { + return errors.AssertionFailedf(unhydratedZoneConfigMessage, "NumReplicas") + } + 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..9df164c5c40c 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,295 @@ 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{ + 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.True(t, testutils.IsError(err, tc.errorStr)) + } + require.Equal(t, tc.expectSpanConfig, spanConfig) + } +} 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 35802ebc7b48..e9de5def2da2 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/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index d034ef7d8c41..b0948bc30923 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -1,4 +1,5 @@ # gazelle:exclude string_test.go +# gazelle:exclude client_test.go load("@bazel_gomock//:gomock.bzl", "gomock") load("@rules_proto//proto:defs.bzl", "proto_library") @@ -184,15 +185,19 @@ go_test( # keep go_test( - name = "string_test", + name = "external_test", size = "small", - srcs = ["string_test.go"], + srcs = [ + "client_test.go", + "string_test.go", + ], deps = [ ":with-mocks", "//pkg/cli/exit", "//pkg/keys", "//pkg/kv/kvserver/concurrency/lock", "//pkg/storage/enginepb", + "//pkg/testutils", "//pkg/util/hlc", "//pkg/util/uuid", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 06fe71c5d719..7c5c5fb07395 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1628,3 +1628,56 @@ func (c *TenantConsumption) Sub(other *TenantConsumption) { c.SQLPodsCPUSeconds -= other.SQLPodsCPUSeconds } } + +// Unnest is a convenience method to return the slice-of-slices form of +// GetSpanConfigsResponse. +func (r *GetSpanConfigsResponse) Unnest() [][]SpanConfigEntry { + var ret [][]SpanConfigEntry + for _, result := range r.Results { + ret = append(ret, result.SpanConfigs) + } + return ret +} + +// Equal compares two span config entries. +func (e SpanConfigEntry) Equal(other SpanConfigEntry) bool { + return e.Span.Equal(other.Span) && e.Config.Equal(other.Config) +} + +// Validate returns an error the request is malformed. Spans included in the +// request are expected to be valid, and have non-empty end keys. Additionally, +// spans in each list (update/delete) are expected to be non-overlapping. +func (r *UpdateSpanConfigsRequest) Validate() error { + spansToUpdate := func(ents []SpanConfigEntry) []Span { + var spans []Span + for _, ent := range ents { + spans = append(spans, ent.Span) + } + return spans + }(r.SpanConfigsToUpdate) + + for _, list := range [][]Span{ + r.SpansToDelete, spansToUpdate, + } { + for _, span := range list { + if !span.Valid() || len(span.EndKey) == 0 { + return errors.AssertionFailedf("invalid span: %s", span) + } + } + + for i := range list { + for j := range list { + if i == j { + continue + } + + if list[i].Overlaps(list[j]) { + return errors.AssertionFailedf("overlapping spans %s and %s in same list", + list[i], list[j]) + } + } + } + } + + return nil +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8aaa89efc58a..171579343eaf 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -434,6 +434,36 @@ func (AdmissionHeader_SourceLocation) EnumDescriptor() ([]byte, []int) { return fileDescriptor_e08772acc330f58b, []int{102, 1} } +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_e08772acc330f58b, []int{121, 0} +} + // RequestHeader is supplied with every storage node request. type RequestHeader struct { // The key for request. If the request operates on a range, this @@ -7392,7 +7422,6 @@ type GossipSubscriptionEvent struct { Content Value `protobuf:"bytes,2,opt,name=content,proto3" json:"content"` // Which pattern does this gossip information match? PatternMatched string `protobuf:"bytes,3,opt,name=pattern_matched,json=patternMatched,proto3" json:"pattern_matched,omitempty"` - // If non-nil, the other fields will be empty and this will be the final event // send on the stream before it is terminated. Error *Error `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` } @@ -7620,6 +7649,421 @@ func (m *JoinNodeResponse) XXX_DiscardUnknown() { var xxx_messageInfo_JoinNodeResponse proto.InternalMessageInfo +// 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_e08772acc330f58b, []int{121} +} +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_e08772acc330f58b, []int{122} +} +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_e08772acc330f58b, []int{123} +} +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_e08772acc330f58b, []int{124} +} +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_e08772acc330f58b, []int{125} +} +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. + 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_e08772acc330f58b, []int{126} +} +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 over the specified +// keyspans. +type GetSpanConfigsResponse struct { + // Results capture the span configurations over the specified spans. There's + // one result for Span in the matching GetSpanConfigsRequeust, in the + // same order. A result will be present for each Span even if there are no + // configs to return. + Results []GetSpanConfigsResponse_Result `protobuf:"bytes,2,rep,name=results,proto3" json:"results"` +} + +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_e08772acc330f58b, []int{127} +} +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 + +type GetSpanConfigsResponse_Result struct { + SpanConfigs []SpanConfigEntry `protobuf:"bytes,1,rep,name=span_configs,json=spanConfigs,proto3" json:"span_configs"` +} + +func (m *GetSpanConfigsResponse_Result) Reset() { *m = GetSpanConfigsResponse_Result{} } +func (m *GetSpanConfigsResponse_Result) String() string { return proto.CompactTextString(m) } +func (*GetSpanConfigsResponse_Result) ProtoMessage() {} +func (*GetSpanConfigsResponse_Result) Descriptor() ([]byte, []int) { + return fileDescriptor_e08772acc330f58b, []int{127, 0} +} +func (m *GetSpanConfigsResponse_Result) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetSpanConfigsResponse_Result) 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_Result) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSpanConfigsResponse_Result.Merge(m, src) +} +func (m *GetSpanConfigsResponse_Result) XXX_Size() int { + return m.Size() +} +func (m *GetSpanConfigsResponse_Result) XXX_DiscardUnknown() { + xxx_messageInfo_GetSpanConfigsResponse_Result.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSpanConfigsResponse_Result proto.InternalMessageInfo + +// UpdateSpanConfigsRequest is used to update the span configurations over the +// specified keyspans. +type UpdateSpanConfigsRequest struct { + // SpanConfigsToUpdate lists out the spans we want to update and the configs + // we want to update with. + SpanConfigsToUpdate []SpanConfigEntry `protobuf:"bytes,1,rep,name=span_configs_to_update,json=spanConfigsToUpdate,proto3" json:"span_configs_to_update"` + // SpansToDelete lists out the spans we want to delete span configs for. + SpansToDelete []Span `protobuf:"bytes,2,rep,name=spans_to_delete,json=spansToDelete,proto3" json:"spans_to_delete"` +} + +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_e08772acc330f58b, []int{128} +} +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_e08772acc330f58b, []int{129} +} +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 + // ContentionEvent is a message that will be attached to BatchResponses // indicating any conflicts with another transaction during replica evaluation. type ContentionEvent struct { @@ -7635,7 +8079,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_e08772acc330f58b, []int{121} + return fileDescriptor_e08772acc330f58b, []int{130} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7672,6 +8116,7 @@ func init() { proto.RegisterEnum("cockroach.roachpb.CheckConsistencyResponse_Status", CheckConsistencyResponse_Status_name, CheckConsistencyResponse_Status_value) proto.RegisterEnum("cockroach.roachpb.AdmissionHeader_Source", AdmissionHeader_Source_name, AdmissionHeader_Source_value) proto.RegisterEnum("cockroach.roachpb.AdmissionHeader_SourceLocation", AdmissionHeader_SourceLocation_name, AdmissionHeader_SourceLocation_value) + proto.RegisterEnum("cockroach.roachpb.Constraint_Type", Constraint_Type_name, Constraint_Type_value) proto.RegisterType((*RequestHeader)(nil), "cockroach.roachpb.RequestHeader") proto.RegisterType((*ResponseHeader)(nil), "cockroach.roachpb.ResponseHeader") proto.RegisterType((*GetRequest)(nil), "cockroach.roachpb.GetRequest") @@ -7807,577 +8252,774 @@ func init() { proto.RegisterType((*TokenBucketResponse)(nil), "cockroach.roachpb.TokenBucketResponse") proto.RegisterType((*JoinNodeRequest)(nil), "cockroach.roachpb.JoinNodeRequest") proto.RegisterType((*JoinNodeResponse)(nil), "cockroach.roachpb.JoinNodeResponse") + 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((*GetSpanConfigsResponse_Result)(nil), "cockroach.roachpb.GetSpanConfigsResponse.Result") + proto.RegisterType((*UpdateSpanConfigsRequest)(nil), "cockroach.roachpb.UpdateSpanConfigsRequest") + proto.RegisterType((*UpdateSpanConfigsResponse)(nil), "cockroach.roachpb.UpdateSpanConfigsResponse") proto.RegisterType((*ContentionEvent)(nil), "cockroach.roachpb.ContentionEvent") } func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 9018 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, - 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, + // 9522 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0x49, + 0x92, 0x98, 0x8a, 0xa4, 0x24, 0x32, 0x28, 0x92, 0xa5, 0x54, 0x3f, 0xd4, 0xea, 0x19, 0xa9, 0xbb, + 0x7a, 0xfa, 0x79, 0x3b, 0xd4, 0x74, 0xf7, 0xee, 0xed, 0xec, 0xcc, 0xec, 0xec, 0x89, 0x14, 0xbb, + 0x49, 0xbd, 0x5a, 0x5d, 0xa4, 0x7a, 0x30, 0x73, 0xbb, 0x57, 0x57, 0xaa, 0x4a, 0x49, 0xb5, 0x22, + 0xab, 0xd8, 0x55, 0x45, 0xb5, 0x34, 0x80, 0x3f, 0xce, 0x3e, 0xac, 0xe7, 0xcb, 0x58, 0xc3, 0x06, + 0x6e, 0x17, 0x67, 0x1c, 0xf6, 0xbc, 0x07, 0xfb, 0xc3, 0x1f, 0x36, 0x6c, 0xc3, 0xc6, 0x19, 0xf6, + 0x9d, 0xfd, 0xe5, 0x85, 0x71, 0xf0, 0xed, 0xfe, 0x1d, 0x0c, 0x58, 0x3e, 0x6b, 0xfd, 0x71, 0x07, + 0xc3, 0xf0, 0x13, 0x38, 0x63, 0x01, 0x3f, 0x90, 0x8f, 0x7a, 0x91, 0x45, 0x8a, 0xea, 0xa9, 0xb1, + 0x17, 0xb8, 0x1f, 0x89, 0x15, 0x99, 0x11, 0x99, 0x19, 0x99, 0x19, 0x19, 0x91, 0x19, 0x91, 0x09, + 0xb3, 0xb6, 0xa5, 0x6a, 0x07, 0xdd, 0xdd, 0x65, 0xb5, 0x6b, 0x94, 0xbb, 0xb6, 0xe5, 0x5a, 0x68, + 0x56, 0xb3, 0xb4, 0x43, 0x0a, 0x2e, 0xf3, 0xc4, 0x85, 0xcb, 0xd8, 0xb6, 0x2d, 0xdb, 0xe9, 0xee, + 0x2e, 0xb3, 0x1f, 0x2c, 0xe7, 0xc2, 0x83, 0xc3, 0xa3, 0xe5, 0xc3, 0x23, 0x07, 0xdb, 0x47, 0xd8, + 0x5e, 0xd6, 0x2c, 0x53, 0xeb, 0xd9, 0x36, 0x36, 0xb5, 0x93, 0xe5, 0xb6, 0xa5, 0x1d, 0xd2, 0x3f, + 0x86, 0xb9, 0x1f, 0x97, 0xd7, 0xc6, 0xaa, 0xee, 0xf4, 0x3a, 0x1d, 0xd5, 0x3e, 0x59, 0xa6, 0x64, + 0xf9, 0x07, 0xcf, 0x8b, 0xbc, 0x4a, 0xe9, 0xaa, 0xab, 0x72, 0xd8, 0x25, 0x0f, 0x16, 0xa9, 0xc1, + 0x15, 0x0f, 0xda, 0xc1, 0xae, 0x1a, 0xca, 0x7d, 0xdd, 0x71, 0x2d, 0x5b, 0xdd, 0xc7, 0xcb, 0xd8, + 0xdc, 0x37, 0x4c, 0x4c, 0x32, 0x1c, 0x69, 0x1a, 0x4f, 0x7c, 0x23, 0x36, 0xf1, 0x31, 0x4f, 0x9d, + 0xef, 0xb9, 0x46, 0x7b, 0xf9, 0xa0, 0xad, 0x2d, 0xbb, 0x46, 0x07, 0x3b, 0xae, 0xda, 0xe9, 0x7a, + 0x4d, 0xa0, 0x29, 0xae, 0xad, 0x6a, 0x86, 0xb9, 0xef, 0xfd, 0xef, 0xee, 0x2e, 0xdb, 0x58, 0xb3, + 0x6c, 0x1d, 0xeb, 0x8a, 0xd3, 0x55, 0x4d, 0xaf, 0xba, 0xfb, 0xd6, 0xbe, 0x45, 0x7f, 0x2e, 0x93, + 0x5f, 0x1c, 0xba, 0xb8, 0x6f, 0x59, 0xfb, 0x6d, 0xbc, 0x4c, 0xbf, 0x76, 0x7b, 0x7b, 0xcb, 0x7a, + 0xcf, 0x56, 0x5d, 0xc3, 0xe2, 0x58, 0xd2, 0x3f, 0x12, 0xa0, 0x20, 0xe3, 0x97, 0x3d, 0xec, 0xb8, + 0x75, 0xac, 0xea, 0xd8, 0x46, 0xd7, 0x20, 0x7d, 0x88, 0x4f, 0xe6, 0xd3, 0x37, 0x84, 0x7b, 0x33, + 0x95, 0xe9, 0x9f, 0x9d, 0x2e, 0xa5, 0xd7, 0xf1, 0x89, 0x4c, 0x60, 0xe8, 0x06, 0x4c, 0x63, 0x53, + 0x57, 0x48, 0x72, 0x26, 0x9a, 0x3c, 0x85, 0x4d, 0x7d, 0x1d, 0x9f, 0xa0, 0x6f, 0x42, 0xd6, 0x21, + 0xd4, 0x4c, 0x0d, 0xcf, 0x4f, 0xde, 0x10, 0xee, 0x4d, 0x56, 0x7e, 0xe9, 0x67, 0xa7, 0x4b, 0x1f, + 0xec, 0x1b, 0xee, 0x41, 0x6f, 0xb7, 0xac, 0x59, 0x9d, 0x65, 0xbf, 0xab, 0xf5, 0xdd, 0xe0, 0xf7, + 0x72, 0xf7, 0x70, 0x7f, 0xb9, 0x9f, 0x47, 0xe5, 0xd6, 0xb1, 0xd9, 0xc4, 0x2f, 0x65, 0x9f, 0xe2, + 0x5a, 0x26, 0x2b, 0x88, 0xa9, 0xb5, 0x4c, 0x36, 0x25, 0xa6, 0xa5, 0x3f, 0x4c, 0x41, 0x51, 0xc6, + 0x4e, 0xd7, 0x32, 0x1d, 0xcc, 0x6b, 0xfe, 0x0e, 0xa4, 0xdd, 0x63, 0x93, 0xd6, 0x3c, 0xff, 0x68, + 0xb1, 0x3c, 0x30, 0xa8, 0xca, 0x2d, 0x5b, 0x35, 0x1d, 0x55, 0x23, 0xcd, 0x97, 0x49, 0x56, 0xf4, + 0x2e, 0xe4, 0x6d, 0xec, 0xf4, 0x3a, 0x98, 0x32, 0x92, 0x36, 0x2a, 0xff, 0xe8, 0x6a, 0x0c, 0x66, + 0xb3, 0xab, 0x9a, 0x32, 0xb0, 0xbc, 0xe4, 0x37, 0x6a, 0x42, 0x81, 0x63, 0xda, 0x58, 0x75, 0x2c, + 0x73, 0x7e, 0xfa, 0x86, 0x70, 0xaf, 0xf8, 0xa8, 0x1c, 0x83, 0x1b, 0xad, 0x25, 0xf9, 0xec, 0x75, + 0xb0, 0x4c, 0xb1, 0xe4, 0x19, 0x3b, 0xf4, 0x85, 0xae, 0x41, 0xd6, 0xec, 0x75, 0x08, 0x7f, 0x1d, + 0xca, 0xbd, 0xb4, 0x3c, 0x6d, 0xf6, 0x3a, 0xeb, 0xf8, 0xc4, 0x41, 0xd7, 0x21, 0x47, 0x92, 0x76, + 0x4f, 0x5c, 0xec, 0xcc, 0x67, 0x69, 0x1a, 0xc9, 0x5b, 0x21, 0xdf, 0xd2, 0x87, 0x30, 0x13, 0xa6, + 0x8a, 0x10, 0x14, 0xe5, 0x5a, 0x73, 0x67, 0xb3, 0xa6, 0xec, 0x6c, 0xad, 0x6f, 0x3d, 0xfb, 0x68, + 0x4b, 0x9c, 0x40, 0x97, 0x40, 0xe4, 0xb0, 0xf5, 0xda, 0xc7, 0xca, 0x46, 0x63, 0xb3, 0xd1, 0x12, + 0x85, 0x85, 0xcc, 0x67, 0x3f, 0x5c, 0x9c, 0x58, 0xcb, 0x64, 0xa7, 0xc4, 0x69, 0xe9, 0x87, 0x02, + 0xc0, 0x53, 0xec, 0xf2, 0xd1, 0x80, 0x2a, 0x30, 0x75, 0x40, 0x6b, 0x3c, 0x2f, 0x50, 0xb6, 0xdc, + 0x88, 0x6d, 0x5a, 0x68, 0xe4, 0x54, 0xb2, 0x3f, 0x3a, 0x5d, 0x9a, 0xf8, 0xf1, 0xe9, 0x92, 0x20, + 0x73, 0x4c, 0xf4, 0x1c, 0xf2, 0x87, 0xf8, 0x44, 0xe1, 0xf3, 0x72, 0x3e, 0x45, 0x79, 0xf4, 0x4e, + 0x88, 0xd0, 0xe1, 0x51, 0xd9, 0x9b, 0xa2, 0xe5, 0xd0, 0x74, 0x2e, 0x13, 0x8c, 0x72, 0xd3, 0xb5, + 0xb1, 0xb9, 0xef, 0x1e, 0xc8, 0x70, 0x88, 0x4f, 0x36, 0x18, 0x0d, 0xe9, 0xf7, 0x05, 0xc8, 0xd3, + 0x5a, 0x32, 0xa6, 0xa2, 0x6a, 0x5f, 0x35, 0x6f, 0x9e, 0xdb, 0x03, 0x31, 0xf5, 0x2c, 0xc3, 0xe4, + 0x91, 0xda, 0xee, 0x61, 0x5a, 0xc3, 0xfc, 0xa3, 0xf9, 0x18, 0x1a, 0x2f, 0x48, 0xba, 0xcc, 0xb2, + 0xa1, 0xf7, 0x61, 0xc6, 0x30, 0x5d, 0x6c, 0xba, 0x0a, 0x43, 0x4b, 0x9f, 0x83, 0x96, 0x67, 0xb9, + 0xe9, 0x87, 0xf4, 0x0f, 0x05, 0x80, 0xed, 0x5e, 0xa2, 0x7c, 0xfe, 0xf2, 0x98, 0xf5, 0xaf, 0x64, + 0x08, 0xaa, 0xd7, 0x8a, 0x2b, 0x30, 0x65, 0x98, 0x6d, 0xc3, 0x64, 0xf5, 0xcf, 0xca, 0xfc, 0x0b, + 0x5d, 0x82, 0xc9, 0xdd, 0xb6, 0x61, 0xea, 0x74, 0x3e, 0x64, 0x65, 0xf6, 0x21, 0xc9, 0x90, 0xa7, + 0xb5, 0x4e, 0x90, 0xef, 0xd2, 0x69, 0x0a, 0x2e, 0x57, 0x2d, 0x53, 0x37, 0xc8, 0x94, 0x54, 0xdb, + 0x3f, 0x17, 0x5c, 0x59, 0x83, 0x4b, 0x3a, 0xee, 0xda, 0x58, 0x53, 0x5d, 0xac, 0x2b, 0xf8, 0xb8, + 0x3b, 0x66, 0x1f, 0xa3, 0x00, 0xab, 0x76, 0xdc, 0xa5, 0x30, 0x32, 0x6b, 0x09, 0x01, 0x36, 0x6b, + 0xa7, 0x88, 0xc8, 0x94, 0xb3, 0xf8, 0xb8, 0x4b, 0x67, 0x6d, 0x3c, 0x9b, 0xd1, 0x97, 0xe1, 0xaa, + 0xda, 0x6e, 0x5b, 0xaf, 0x14, 0x63, 0x4f, 0xd1, 0x2d, 0xec, 0x28, 0xa6, 0xe5, 0x2a, 0xf8, 0xd8, + 0x70, 0x5c, 0x2a, 0x12, 0xb2, 0xf2, 0x1c, 0x4d, 0x6e, 0xec, 0xad, 0x5a, 0xd8, 0xd9, 0xb2, 0xdc, + 0x1a, 0x49, 0x0a, 0x75, 0xe5, 0x74, 0xb8, 0x2b, 0xa5, 0x6f, 0xc1, 0x95, 0x7e, 0xfe, 0x26, 0xd9, + 0x7f, 0x7f, 0x20, 0x40, 0xb1, 0x61, 0x1a, 0xee, 0xcf, 0x45, 0xc7, 0xf9, 0xfc, 0x4c, 0x87, 0xf9, + 0xf9, 0x00, 0xc4, 0x3d, 0xd5, 0x68, 0x3f, 0x33, 0x5b, 0x56, 0x67, 0xd7, 0x71, 0x2d, 0x13, 0x3b, + 0x9c, 0xe1, 0x03, 0x70, 0xe9, 0x05, 0x94, 0xfc, 0xd6, 0x24, 0xc9, 0x26, 0x17, 0xc4, 0x86, 0xa9, + 0xd9, 0xb8, 0x83, 0xcd, 0x44, 0xf9, 0xf4, 0x06, 0xe4, 0x0c, 0x8f, 0x2e, 0xe5, 0x55, 0x5a, 0x0e, + 0x00, 0x52, 0x0f, 0x66, 0x43, 0xa5, 0x26, 0x29, 0x2e, 0xc9, 0x62, 0x84, 0x5f, 0x29, 0x41, 0x1f, + 0x91, 0xc5, 0x08, 0xbf, 0x62, 0xe2, 0xad, 0x09, 0x85, 0x55, 0xdc, 0xc6, 0x2e, 0x4e, 0xb0, 0xa5, + 0xd2, 0x0e, 0x14, 0x3d, 0xa2, 0x49, 0x76, 0xcc, 0x6f, 0x08, 0x80, 0x38, 0x5d, 0xd5, 0xdc, 0x4f, + 0xb2, 0xc6, 0x68, 0x89, 0xa8, 0x16, 0x6e, 0xcf, 0x36, 0xd9, 0x72, 0xce, 0xc6, 0x24, 0x30, 0x10, + 0x5d, 0xd1, 0x83, 0x29, 0x9b, 0x09, 0x4f, 0x59, 0xae, 0xde, 0xbc, 0x82, 0xb9, 0x48, 0xc5, 0x92, + 0xed, 0xbe, 0x0c, 0xad, 0x53, 0xea, 0x46, 0x3a, 0xac, 0xc3, 0x51, 0xa0, 0xf4, 0x3d, 0x01, 0x66, + 0xab, 0x6d, 0xac, 0xda, 0x89, 0x73, 0xe4, 0x1b, 0x90, 0xd5, 0xb1, 0xaa, 0xd3, 0x26, 0xb3, 0x89, + 0xfd, 0x66, 0x88, 0x0a, 0xd1, 0x74, 0xcb, 0x07, 0x6d, 0xad, 0xdc, 0xf2, 0x74, 0x60, 0x3e, 0xbb, + 0x7d, 0x24, 0xe9, 0x63, 0x40, 0xe1, 0x9a, 0x25, 0x39, 0x10, 0x7e, 0x27, 0x05, 0x48, 0xc6, 0x47, + 0xd8, 0x76, 0x13, 0x6f, 0xf6, 0x2a, 0xe4, 0x5d, 0xd5, 0xde, 0xc7, 0xae, 0x42, 0xb4, 0xfb, 0x8b, + 0xb4, 0x1c, 0x18, 0x1e, 0x01, 0xa3, 0x16, 0xdc, 0xc5, 0xa6, 0xba, 0xdb, 0xc6, 0x94, 0x8a, 0xb2, + 0x6b, 0xf5, 0x4c, 0x5d, 0x31, 0x5c, 0x6c, 0xab, 0xae, 0x65, 0x2b, 0x56, 0xd7, 0x35, 0x3a, 0xc6, + 0xa7, 0x54, 0xb1, 0xe7, 0x43, 0xed, 0x16, 0xcb, 0x4e, 0x90, 0x2b, 0x24, 0x73, 0x83, 0xe7, 0x7d, + 0x16, 0xca, 0x8a, 0xca, 0x30, 0x67, 0xec, 0x9b, 0x96, 0x8d, 0x95, 0x7d, 0x4d, 0x71, 0x0f, 0x6c, + 0xec, 0x1c, 0x58, 0x6d, 0x6f, 0x41, 0x9a, 0x65, 0x49, 0x4f, 0xb5, 0x96, 0x97, 0x20, 0x7d, 0x02, + 0x73, 0x11, 0x2e, 0x25, 0xd9, 0x05, 0xff, 0x45, 0x80, 0x7c, 0x53, 0x53, 0xcd, 0x24, 0x79, 0xff, + 0x21, 0xe4, 0x1d, 0x4d, 0x35, 0x95, 0x3d, 0xcb, 0xee, 0xa8, 0x2e, 0x6d, 0x57, 0x31, 0xc2, 0x7b, + 0x5f, 0xbf, 0xd7, 0x54, 0xf3, 0x09, 0xcd, 0x24, 0x83, 0xe3, 0xff, 0xee, 0xd7, 0x5f, 0x27, 0x3f, + 0xbf, 0xfe, 0xca, 0xa6, 0xf7, 0x5a, 0x26, 0x9b, 0x16, 0x33, 0xd2, 0x9f, 0x09, 0x30, 0xc3, 0x9a, + 0x9c, 0xe4, 0xf4, 0xfe, 0x0a, 0x64, 0x6c, 0xeb, 0x15, 0x9b, 0xde, 0xf9, 0x47, 0xd7, 0x63, 0x48, + 0xac, 0xe3, 0x93, 0xf0, 0xfa, 0x49, 0xb3, 0xa3, 0x0a, 0x70, 0x2d, 0x55, 0xa1, 0xd8, 0xe9, 0x71, + 0xb1, 0x81, 0x61, 0xc9, 0x84, 0xc6, 0x5d, 0x28, 0xed, 0xaa, 0xae, 0x76, 0xa0, 0xd8, 0xbc, 0x92, + 0x64, 0xad, 0x4d, 0xdf, 0x9b, 0x91, 0x8b, 0x14, 0xec, 0x55, 0xdd, 0x21, 0x2d, 0x67, 0xf3, 0xcd, + 0xc1, 0x7f, 0xce, 0xfa, 0xfc, 0x7f, 0x0b, 0x7c, 0x0e, 0x79, 0x2d, 0xff, 0xf3, 0xd6, 0xf5, 0xdf, + 0x4f, 0xc1, 0xd5, 0xea, 0x01, 0xd6, 0x0e, 0xab, 0x96, 0xe9, 0x18, 0x8e, 0x4b, 0x78, 0x97, 0x64, + 0xff, 0x5f, 0x87, 0xdc, 0x2b, 0xc3, 0x3d, 0x50, 0x74, 0x63, 0x6f, 0x8f, 0x4a, 0xdb, 0xac, 0x9c, + 0x25, 0x80, 0x55, 0x63, 0x6f, 0x0f, 0x3d, 0x86, 0x4c, 0xc7, 0xd2, 0x99, 0x32, 0x5f, 0x7c, 0xb4, + 0x14, 0x43, 0x9e, 0x56, 0xcd, 0xe9, 0x75, 0x36, 0x2d, 0x1d, 0xcb, 0x34, 0x33, 0x5a, 0x04, 0xd0, + 0x08, 0xb4, 0x6b, 0x19, 0xa6, 0xcb, 0x85, 0x63, 0x08, 0x82, 0xea, 0x90, 0x73, 0xb1, 0xdd, 0x31, + 0x4c, 0xd5, 0xc5, 0xf3, 0x93, 0x94, 0x79, 0x6f, 0xc5, 0x56, 0xbc, 0xdb, 0x36, 0x34, 0x75, 0x15, + 0x3b, 0x9a, 0x6d, 0x74, 0x5d, 0xcb, 0xe6, 0x5c, 0x0c, 0x90, 0xa5, 0xbf, 0x92, 0x81, 0xf9, 0x41, + 0xde, 0x24, 0x39, 0x42, 0xb6, 0x61, 0xca, 0xc6, 0x4e, 0xaf, 0xed, 0xf2, 0x31, 0xf2, 0x68, 0x18, + 0x0b, 0x62, 0x6a, 0x40, 0xb7, 0x2e, 0xda, 0x2e, 0xaf, 0x36, 0xa7, 0xb3, 0xf0, 0x4f, 0x05, 0x98, + 0x62, 0x09, 0xe8, 0x21, 0x64, 0x6d, 0xb2, 0x30, 0x28, 0x86, 0x4e, 0xeb, 0x98, 0xae, 0x5c, 0x39, + 0x3b, 0x5d, 0x9a, 0xa6, 0x8b, 0x45, 0x63, 0xf5, 0x67, 0xc1, 0x4f, 0x79, 0x9a, 0xe6, 0x6b, 0xe8, + 0xa4, 0xb7, 0x1c, 0x57, 0xb5, 0x5d, 0xba, 0xa9, 0x94, 0x62, 0x16, 0x12, 0x05, 0xac, 0xe3, 0x13, + 0xb4, 0x06, 0x53, 0x8e, 0xab, 0xba, 0x3d, 0x87, 0xf7, 0xd7, 0x85, 0x2a, 0xdb, 0xa4, 0x98, 0x32, + 0xa7, 0x40, 0xd4, 0x2d, 0x1d, 0xbb, 0xaa, 0xd1, 0xa6, 0x1d, 0x98, 0x93, 0xf9, 0x97, 0xf4, 0x9b, + 0x02, 0x4c, 0xb1, 0xac, 0xe8, 0x2a, 0xcc, 0xc9, 0x2b, 0x5b, 0x4f, 0x6b, 0x4a, 0x63, 0x6b, 0xb5, + 0xd6, 0xaa, 0xc9, 0x9b, 0x8d, 0xad, 0x95, 0x56, 0x4d, 0x9c, 0x40, 0x57, 0x00, 0x79, 0x09, 0xd5, + 0x67, 0x5b, 0xcd, 0x46, 0xb3, 0x55, 0xdb, 0x6a, 0x89, 0x02, 0xdd, 0x53, 0xa1, 0xf0, 0x10, 0x34, + 0x85, 0xde, 0x82, 0x1b, 0xfd, 0x50, 0xa5, 0xd9, 0x5a, 0x69, 0x35, 0x95, 0x5a, 0xb3, 0xd5, 0xd8, + 0x5c, 0x69, 0xd5, 0x56, 0xc5, 0xf4, 0x88, 0x5c, 0xa4, 0x10, 0x59, 0xae, 0x55, 0x5b, 0x62, 0x46, + 0x72, 0xe1, 0xb2, 0x8c, 0x35, 0xab, 0xd3, 0xed, 0xb9, 0x98, 0xd4, 0xd2, 0x49, 0x72, 0xa6, 0x5c, + 0x85, 0x69, 0xdd, 0x3e, 0x51, 0xec, 0x9e, 0xc9, 0xe7, 0xc9, 0x94, 0x6e, 0x9f, 0xc8, 0x3d, 0x53, + 0xfa, 0x7b, 0x02, 0x5c, 0xe9, 0x2f, 0x36, 0xc9, 0x41, 0xf8, 0x1c, 0xf2, 0xaa, 0xae, 0x63, 0x5d, + 0xd1, 0x71, 0xdb, 0x55, 0xb9, 0x4a, 0xf4, 0x20, 0x44, 0x89, 0x6f, 0x05, 0x96, 0xfd, 0xad, 0xc0, + 0xcd, 0x17, 0xd5, 0x2a, 0xad, 0xc8, 0x2a, 0xc1, 0xf0, 0xc4, 0x0f, 0x25, 0x42, 0x21, 0xd2, 0xf7, + 0x33, 0x50, 0xa8, 0x99, 0x7a, 0xeb, 0x38, 0xd1, 0xb5, 0xe4, 0x0a, 0x4c, 0x69, 0x56, 0xa7, 0x63, + 0xb8, 0x1e, 0x83, 0xd8, 0x17, 0xfa, 0x5a, 0x48, 0x95, 0x4d, 0x8f, 0xa1, 0xd0, 0x05, 0x4a, 0x2c, + 0xfa, 0x55, 0xb8, 0x4a, 0xa4, 0xa6, 0x6d, 0xaa, 0x6d, 0x85, 0x51, 0x53, 0x5c, 0xdb, 0xd8, 0xdf, + 0xc7, 0x36, 0xdf, 0x7e, 0xbc, 0x17, 0x53, 0xcf, 0x06, 0xc7, 0xa8, 0x52, 0x84, 0x16, 0xcb, 0x2f, + 0x5f, 0x36, 0xe2, 0xc0, 0xe8, 0x03, 0x00, 0xb2, 0x14, 0xd1, 0x2d, 0x4d, 0x87, 0xcb, 0xa3, 0x61, + 0x7b, 0x9a, 0x9e, 0x08, 0x22, 0x08, 0xe4, 0xdb, 0x41, 0xcf, 0x41, 0x34, 0x4c, 0x65, 0xaf, 0x6d, + 0xec, 0x1f, 0xb8, 0xca, 0x2b, 0xdb, 0x70, 0xb1, 0x33, 0x3f, 0x4b, 0x69, 0xc4, 0x75, 0x75, 0x93, + 0x6f, 0xcd, 0xea, 0x1f, 0x91, 0x9c, 0x9c, 0x5a, 0xd1, 0x30, 0x9f, 0x50, 0x7c, 0x0a, 0x74, 0xd0, + 0x32, 0x31, 0x85, 0x5e, 0xf6, 0x0c, 0x1b, 0x2b, 0x0f, 0xbb, 0x1a, 0xdd, 0x07, 0xc9, 0x56, 0x8a, + 0x67, 0xa7, 0x4b, 0x20, 0x33, 0xf0, 0xc3, 0xed, 0x2a, 0x31, 0x8d, 0xd8, 0xef, 0xae, 0x46, 0xd8, + 0xde, 0xb5, 0x0c, 0xc7, 0x32, 0xe7, 0x73, 0x8c, 0xed, 0xec, 0x0b, 0xdd, 0x07, 0xd1, 0x3d, 0x36, + 0x95, 0x03, 0xac, 0xda, 0xee, 0x2e, 0x56, 0x5d, 0xb2, 0x3e, 0x03, 0xcd, 0x51, 0x72, 0x8f, 0xcd, + 0x7a, 0x08, 0xbc, 0x96, 0xc9, 0x4e, 0x8b, 0xd9, 0xb5, 0x4c, 0x36, 0x2b, 0xe6, 0xa4, 0x7f, 0x2b, + 0x40, 0xd1, 0x1b, 0x1b, 0x49, 0x0e, 0xe3, 0x7b, 0x20, 0x5a, 0x26, 0x56, 0xba, 0x07, 0xaa, 0x83, + 0x79, 0x5f, 0xf2, 0xd5, 0xa1, 0x68, 0x99, 0x78, 0x9b, 0x80, 0x59, 0xcf, 0xa0, 0x6d, 0x98, 0x75, + 0x5c, 0x75, 0xdf, 0x30, 0xf7, 0x15, 0x7f, 0x8b, 0x9f, 0x6a, 0x16, 0x63, 0x5a, 0x02, 0x22, 0xc7, + 0xf6, 0xe1, 0x11, 0x95, 0xe2, 0x27, 0x02, 0xcc, 0xae, 0xe8, 0x1d, 0xc3, 0x6c, 0x76, 0xdb, 0x46, + 0xa2, 0x1b, 0x0c, 0x6f, 0x41, 0xce, 0x21, 0x34, 0x03, 0xe9, 0x1c, 0x98, 0x8b, 0x59, 0x9a, 0x42, + 0xc4, 0xf4, 0x06, 0x94, 0xf0, 0x71, 0xd7, 0x60, 0xe7, 0x0a, 0xcc, 0xca, 0xc9, 0x8c, 0xdf, 0xb6, + 0x62, 0x80, 0x4b, 0x92, 0x78, 0x9b, 0x3e, 0x06, 0x14, 0x6e, 0x52, 0x92, 0x86, 0xc6, 0xc7, 0x30, + 0x47, 0x49, 0xef, 0x98, 0x4e, 0xc2, 0xfc, 0x92, 0x7e, 0x19, 0x2e, 0x45, 0x49, 0x27, 0x59, 0xef, + 0x8f, 0x78, 0x2f, 0x6f, 0x62, 0x3b, 0x51, 0x0b, 0xd5, 0xe7, 0x35, 0x27, 0x9c, 0x64, 0x9d, 0x7f, + 0x5d, 0x80, 0x6b, 0x94, 0x36, 0x3d, 0x7a, 0xd9, 0xc3, 0xf6, 0x06, 0x56, 0x9d, 0x44, 0xcd, 0xeb, + 0x5b, 0x30, 0xc5, 0xcc, 0x64, 0x3a, 0x3e, 0x27, 0x2b, 0x79, 0xa2, 0x66, 0x34, 0x5d, 0xcb, 0x26, + 0x6a, 0x06, 0x4f, 0x92, 0x54, 0x58, 0x88, 0xab, 0x45, 0x92, 0x2d, 0xfd, 0x2d, 0x01, 0x66, 0xb9, + 0x86, 0x47, 0x86, 0x72, 0xf5, 0x80, 0x28, 0x38, 0xa8, 0x06, 0x79, 0x8d, 0xfe, 0x52, 0xdc, 0x93, + 0x2e, 0xa6, 0xf4, 0x8b, 0xa3, 0x94, 0x43, 0x86, 0xd6, 0x3a, 0xe9, 0x62, 0xa2, 0x61, 0x7a, 0xbf, + 0x09, 0xa3, 0x42, 0x8d, 0x1c, 0xa9, 0x5e, 0xd2, 0x79, 0x44, 0xf3, 0x7a, 0x7a, 0x1a, 0xe7, 0xc1, + 0x3f, 0x48, 0x73, 0x26, 0xb0, 0x32, 0x78, 0xf6, 0x44, 0x15, 0x8a, 0x4f, 0xe0, 0x4a, 0x68, 0xeb, + 0x3c, 0xdc, 0xf0, 0xd4, 0x05, 0x1a, 0x1e, 0xda, 0x7e, 0x0f, 0xa0, 0xe8, 0x63, 0x08, 0x6d, 0xb0, + 0x2b, 0xac, 0x4d, 0x9e, 0xa9, 0x72, 0x11, 0x76, 0xcc, 0x06, 0x54, 0x18, 0xdc, 0x41, 0x55, 0xc8, + 0xe2, 0xe3, 0xae, 0xa2, 0x63, 0x47, 0xe3, 0x82, 0x4b, 0x8a, 0x23, 0x48, 0xaa, 0x32, 0xa0, 0xbc, + 0x4f, 0xe3, 0xe3, 0x2e, 0x01, 0xa2, 0x1d, 0xb2, 0x6e, 0x7a, 0xeb, 0x3a, 0xad, 0xb6, 0x73, 0xbe, + 0x2d, 0x10, 0x8c, 0x14, 0x4e, 0xae, 0xe4, 0x2f, 0xe9, 0x8c, 0x84, 0xf4, 0x03, 0x01, 0xae, 0xc7, + 0xf6, 0x5a, 0x92, 0x0b, 0xd9, 0x07, 0x90, 0xa1, 0x8d, 0x4f, 0x5d, 0xb0, 0xf1, 0x14, 0x4b, 0xfa, + 0x2c, 0xc5, 0xe7, 0xb8, 0x8c, 0xdb, 0x16, 0x61, 0x6c, 0xe2, 0x5b, 0x68, 0xcf, 0xa0, 0x70, 0x64, + 0xb9, 0xd8, 0xf6, 0xbb, 0x3d, 0x75, 0xe1, 0x6e, 0x9f, 0xa1, 0x04, 0xbc, 0x1e, 0x7f, 0x01, 0xb3, + 0xa6, 0x65, 0x2a, 0x51, 0xa2, 0x17, 0x1f, 0x4b, 0x25, 0xd3, 0x32, 0x5f, 0x84, 0xe8, 0xfa, 0x72, + 0xa6, 0x8f, 0x13, 0x49, 0xca, 0x99, 0xef, 0x0a, 0x30, 0xe7, 0x6b, 0x3a, 0x09, 0xab, 0xbb, 0x5f, + 0x81, 0xb4, 0x69, 0xbd, 0xba, 0xc8, 0x16, 0x25, 0xc9, 0x4f, 0x56, 0xbd, 0x68, 0x8d, 0x92, 0x6c, + 0xef, 0x3f, 0x4b, 0x41, 0xee, 0x69, 0x35, 0xc9, 0x56, 0x7e, 0xc0, 0xb7, 0xbf, 0x59, 0x7f, 0xc7, + 0x8d, 0x76, 0xbf, 0xbc, 0xf2, 0xd3, 0xea, 0x3a, 0x3e, 0xf1, 0x46, 0x3b, 0xc1, 0x42, 0x2b, 0x90, + 0x8b, 0x6e, 0x94, 0x8e, 0xc9, 0xa9, 0x00, 0x6b, 0x01, 0xc3, 0x24, 0xa5, 0xeb, 0xb9, 0x5a, 0x08, + 0x31, 0xae, 0x16, 0xa4, 0x18, 0x5f, 0x53, 0x4c, 0x5d, 0xa4, 0x98, 0x90, 0x8a, 0x38, 0x29, 0x4e, + 0x49, 0xcf, 0x01, 0x48, 0x73, 0x92, 0xec, 0x92, 0xef, 0xa4, 0xa1, 0xb8, 0xdd, 0x73, 0x0e, 0x12, + 0x1e, 0x7d, 0x55, 0x80, 0x6e, 0xcf, 0x39, 0x20, 0x33, 0xf2, 0xd8, 0xe4, 0x6d, 0x3e, 0xc7, 0x8b, + 0xc3, 0x6b, 0x34, 0xc3, 0x6b, 0x1d, 0x9b, 0xa8, 0xce, 0x89, 0x60, 0x25, 0x70, 0x05, 0xb9, 0x35, + 0xca, 0xb2, 0x6c, 0x1d, 0x9b, 0x9b, 0xd8, 0x37, 0x29, 0x19, 0x25, 0x4c, 0x28, 0x7d, 0x00, 0xd3, + 0xe4, 0x43, 0x71, 0xad, 0x8b, 0x74, 0xf3, 0x14, 0xc1, 0x69, 0x59, 0xe8, 0x7d, 0xc8, 0x31, 0x6c, + 0xb2, 0xfa, 0x4d, 0xd1, 0xd5, 0x2f, 0xae, 0x2d, 0x9c, 0x8d, 0x74, 0xdd, 0xcb, 0x52, 0x54, 0xb2, + 0xd6, 0x5d, 0x82, 0xc9, 0x3d, 0xcb, 0xd6, 0xbc, 0xc3, 0x5c, 0xf6, 0xc1, 0xfa, 0x93, 0x99, 0x34, + 0x6b, 0x99, 0x6c, 0x4e, 0x04, 0xe9, 0x37, 0x05, 0x28, 0xf9, 0x1d, 0x91, 0xe4, 0x82, 0x50, 0x8d, + 0x70, 0xf1, 0xe2, 0x5d, 0x41, 0x18, 0x28, 0xfd, 0x4b, 0xaa, 0x11, 0x69, 0xd6, 0x11, 0xed, 0x99, + 0x24, 0x47, 0xca, 0xfb, 0xcc, 0xd1, 0x27, 0x75, 0xd1, 0xde, 0xa5, 0x3e, 0x3f, 0x0f, 0xe1, 0x92, + 0xd1, 0x21, 0xf2, 0xdc, 0x70, 0xdb, 0x27, 0xdc, 0x6c, 0x73, 0xb1, 0x77, 0x6a, 0x3c, 0x17, 0xa4, + 0x55, 0xbd, 0x24, 0xe9, 0x77, 0xe8, 0x6e, 0x75, 0xd0, 0x92, 0x24, 0x59, 0xdd, 0x80, 0x82, 0xcd, + 0x48, 0x13, 0xb5, 0xe6, 0x82, 0xdc, 0x9e, 0xf1, 0x51, 0x09, 0xc3, 0x7f, 0x3b, 0x05, 0xa5, 0xe7, + 0x3d, 0x6c, 0x9f, 0xfc, 0x3c, 0xb1, 0xfb, 0x0e, 0x94, 0x5e, 0xa9, 0x86, 0xab, 0xec, 0x59, 0xb6, + 0xd2, 0xeb, 0xea, 0xaa, 0xeb, 0x79, 0x9b, 0x14, 0x08, 0xf8, 0x89, 0x65, 0xef, 0x50, 0x20, 0xc2, + 0x80, 0x0e, 0x4d, 0xeb, 0x95, 0xa9, 0x10, 0x30, 0x35, 0x94, 0x8f, 0x4d, 0xbe, 0x85, 0x5c, 0xf9, + 0xea, 0xbf, 0x39, 0x5d, 0x7a, 0x3c, 0x96, 0x0f, 0x19, 0xf5, 0x97, 0xeb, 0xf5, 0x0c, 0xbd, 0xbc, + 0xb3, 0xd3, 0x58, 0x95, 0x45, 0x4a, 0xf2, 0x23, 0x46, 0xb1, 0x75, 0x6c, 0x3a, 0xd2, 0xdf, 0x4a, + 0x81, 0x18, 0xf0, 0x28, 0xc9, 0x8e, 0xac, 0x41, 0xfe, 0x65, 0x0f, 0xdb, 0xc6, 0x6b, 0x74, 0x23, + 0x70, 0x44, 0x22, 0x76, 0x1e, 0xc0, 0xac, 0x7b, 0x6c, 0x2a, 0xcc, 0xc3, 0x8f, 0x39, 0x7e, 0x78, + 0x0e, 0x0b, 0x25, 0x97, 0xd4, 0x99, 0xc0, 0xa9, 0xd3, 0x87, 0x83, 0x3e, 0x81, 0x99, 0x08, 0xb7, + 0xd2, 0x9f, 0x8f, 0x5b, 0xf9, 0x57, 0x21, 0x46, 0xfd, 0xbe, 0x00, 0x88, 0x32, 0xaa, 0xc1, 0xf6, + 0xf8, 0x7f, 0x5e, 0xc6, 0xd3, 0x3d, 0x10, 0xa9, 0x3f, 0xa6, 0x62, 0xec, 0x29, 0x1d, 0xc3, 0x71, + 0x0c, 0x73, 0x9f, 0x0f, 0xa8, 0x22, 0x85, 0x37, 0xf6, 0x36, 0x19, 0x54, 0xfa, 0x0b, 0x30, 0x17, + 0x69, 0x40, 0x92, 0x9d, 0x7d, 0x13, 0x66, 0xf6, 0xd8, 0x11, 0x2c, 0x25, 0xce, 0xb7, 0x07, 0xf3, + 0x14, 0xc6, 0xca, 0x93, 0xfe, 0x63, 0x0a, 0x2e, 0xc9, 0xd8, 0xb1, 0xda, 0x47, 0x38, 0x79, 0x16, + 0xd6, 0x81, 0x9f, 0xbd, 0x28, 0xaf, 0xc5, 0xc9, 0x1c, 0x43, 0x66, 0xcb, 0x5c, 0x74, 0x8f, 0xfd, + 0xad, 0xd1, 0x23, 0x76, 0x70, 0x57, 0x9d, 0xef, 0xd4, 0x65, 0x22, 0x3b, 0x75, 0x16, 0x94, 0xd8, + 0xe9, 0xb1, 0xae, 0x38, 0xf8, 0xa5, 0xd9, 0xeb, 0x78, 0xc6, 0x50, 0x79, 0x54, 0x25, 0x1b, 0x0c, + 0xa5, 0x89, 0x5f, 0x6e, 0xf5, 0x3a, 0x54, 0x77, 0xae, 0x5c, 0x21, 0xf5, 0x3d, 0x3b, 0x5d, 0x2a, + 0x46, 0xd2, 0x1c, 0xb9, 0x68, 0xf8, 0xdf, 0x84, 0xba, 0xf4, 0x4d, 0xb8, 0xdc, 0xc7, 0xec, 0x24, + 0x35, 0x9e, 0x7f, 0x92, 0x86, 0x6b, 0x51, 0xf2, 0x49, 0x9b, 0x38, 0x3f, 0xef, 0x1d, 0x5a, 0x87, + 0x42, 0xc7, 0x30, 0x5f, 0x6f, 0xf7, 0x72, 0xa6, 0x63, 0x98, 0x3e, 0x2c, 0x6e, 0x68, 0x4c, 0x7d, + 0xa1, 0x43, 0x43, 0x85, 0x85, 0xb8, 0xbe, 0x4b, 0x72, 0x7c, 0x7c, 0x26, 0xc0, 0x4c, 0xd2, 0xdb, + 0x72, 0xaf, 0xe7, 0x05, 0x27, 0xb5, 0xa0, 0xf0, 0x05, 0xec, 0xe3, 0xfd, 0xb6, 0x00, 0xa8, 0x65, + 0xf7, 0x4c, 0x62, 0xd4, 0x6e, 0x58, 0xfb, 0x49, 0x36, 0xf3, 0x12, 0x4c, 0x1a, 0xa6, 0x8e, 0x8f, + 0x69, 0x33, 0x33, 0x32, 0xfb, 0x88, 0x1c, 0x25, 0xa6, 0xc7, 0x3a, 0x4a, 0x94, 0x3e, 0x81, 0xb9, + 0x48, 0x15, 0x93, 0x6c, 0xff, 0x9f, 0xa6, 0x60, 0x8e, 0x37, 0x24, 0xf1, 0x1d, 0xcc, 0x2f, 0xc3, + 0x64, 0x9b, 0xd0, 0x1c, 0xd1, 0xcf, 0xb4, 0x4c, 0xaf, 0x9f, 0x69, 0x66, 0xf4, 0x75, 0x80, 0xae, + 0x8d, 0x8f, 0x14, 0x86, 0x9a, 0x1e, 0x0b, 0x35, 0x47, 0x30, 0x28, 0x00, 0x7d, 0x4f, 0x80, 0x12, + 0x99, 0xd0, 0x5d, 0xdb, 0xea, 0x5a, 0x0e, 0xd1, 0x59, 0x9c, 0xf1, 0xcc, 0x9c, 0xe7, 0x67, 0xa7, + 0x4b, 0x85, 0x4d, 0xc3, 0xdc, 0xe6, 0x88, 0xad, 0xe6, 0xd8, 0x0e, 0xfe, 0x5e, 0x98, 0x43, 0xb9, + 0xda, 0xb6, 0xb4, 0xc3, 0xe0, 0x70, 0x8c, 0x48, 0x16, 0x9f, 0x9c, 0x23, 0xfd, 0xa1, 0x00, 0x97, + 0xbe, 0xb0, 0xed, 0xe2, 0xff, 0x1f, 0xcc, 0x96, 0x5e, 0x80, 0x48, 0x7f, 0x34, 0xcc, 0x3d, 0x2b, + 0xc9, 0x8d, 0xfb, 0xff, 0x25, 0xc0, 0x6c, 0x88, 0x70, 0x92, 0x0a, 0xce, 0xeb, 0xf2, 0xa9, 0xc0, + 0xdc, 0x61, 0xdc, 0xf1, 0x58, 0x25, 0xcf, 0xf0, 0xec, 0x6c, 0x50, 0x96, 0x61, 0x06, 0x13, 0x29, + 0x46, 0xb7, 0x78, 0x77, 0x59, 0x90, 0x49, 0xdf, 0x8e, 0x7e, 0xde, 0xcf, 0x50, 0x39, 0x91, 0x7e, + 0x99, 0x68, 0x58, 0xe1, 0x49, 0x99, 0xe4, 0x94, 0xff, 0xc7, 0x29, 0xb8, 0x52, 0x65, 0x47, 0xe0, + 0x9e, 0x4f, 0x48, 0x92, 0x03, 0x71, 0x1e, 0xa6, 0x8f, 0xb0, 0xed, 0x18, 0x16, 0x5b, 0xed, 0x0b, + 0xb2, 0xf7, 0x89, 0x16, 0x20, 0xeb, 0x98, 0x6a, 0xd7, 0x39, 0xb0, 0xbc, 0xe3, 0x44, 0xff, 0xdb, + 0xf7, 0x5f, 0x99, 0x7c, 0x7d, 0xff, 0x95, 0xa9, 0xd1, 0xfe, 0x2b, 0xd3, 0x9f, 0xc3, 0x7f, 0x85, + 0x9f, 0xdd, 0xfd, 0x2b, 0x01, 0xae, 0x0e, 0x70, 0x2e, 0xc9, 0xc1, 0xf9, 0x6d, 0xc8, 0x6b, 0x9c, + 0x30, 0x59, 0x1f, 0xd8, 0xc1, 0x64, 0x83, 0x64, 0x7b, 0x4d, 0xd3, 0xe7, 0xec, 0x74, 0x09, 0xbc, + 0xaa, 0x36, 0x56, 0x39, 0x73, 0xc8, 0x6f, 0x5d, 0xfa, 0x4f, 0x00, 0xa5, 0xda, 0x31, 0xdb, 0x94, + 0x6f, 0x32, 0xad, 0x04, 0x3d, 0x81, 0x6c, 0xd7, 0xb6, 0x8e, 0x0c, 0xaf, 0x19, 0xc5, 0x88, 0xf3, + 0x82, 0xd7, 0x8c, 0x3e, 0xac, 0x6d, 0x8e, 0x21, 0xfb, 0xb8, 0xa8, 0x05, 0xb9, 0x0d, 0x4b, 0x53, + 0xdb, 0x4f, 0x8c, 0xb6, 0x37, 0xd1, 0xde, 0x39, 0x9f, 0x50, 0xd9, 0xc7, 0xd9, 0x56, 0xdd, 0x03, + 0xaf, 0x13, 0x7c, 0x20, 0x6a, 0x40, 0xb6, 0xee, 0xba, 0x5d, 0x92, 0xc8, 0xe7, 0xdf, 0xdd, 0x31, + 0x88, 0x12, 0x14, 0xcf, 0xe3, 0xd6, 0x43, 0x47, 0x2d, 0x98, 0x7d, 0x4a, 0xe3, 0xc7, 0xaa, 0x6d, + 0xab, 0xa7, 0x57, 0x2d, 0x73, 0xcf, 0xd8, 0xe7, 0xcb, 0xc4, 0x9d, 0x31, 0x68, 0x3e, 0xad, 0x36, + 0xe5, 0x41, 0x02, 0x68, 0x05, 0xb2, 0xcd, 0xc7, 0x9c, 0x18, 0x53, 0x23, 0x6f, 0x8f, 0x41, 0xac, + 0xf9, 0x58, 0xf6, 0xd1, 0xd0, 0x1a, 0xe4, 0x57, 0x3e, 0xed, 0xd9, 0x98, 0x53, 0x99, 0x1a, 0xea, + 0x39, 0xd1, 0x4f, 0x85, 0x62, 0xc9, 0x61, 0x64, 0xf4, 0x4d, 0x28, 0x11, 0xbe, 0xb5, 0xd4, 0xdd, + 0xb6, 0x47, 0x2f, 0x4b, 0xe9, 0x7d, 0x69, 0x0c, 0x7a, 0x3e, 0xa6, 0x77, 0x24, 0xd0, 0x47, 0x6a, + 0x41, 0x86, 0x42, 0xa4, 0xbf, 0x10, 0x82, 0x4c, 0x97, 0x74, 0x8d, 0x40, 0xdd, 0x90, 0xe8, 0x6f, + 0xf4, 0x36, 0x4c, 0x9b, 0x96, 0x8e, 0xbd, 0xc1, 0x5c, 0xa8, 0x5c, 0x3a, 0x3b, 0x5d, 0x9a, 0xda, + 0xb2, 0x74, 0xa6, 0xeb, 0xf0, 0x5f, 0xf2, 0x14, 0xc9, 0xd4, 0xd0, 0x17, 0x6e, 0x40, 0x86, 0x74, + 0x11, 0x91, 0x21, 0xbb, 0xaa, 0x83, 0x77, 0x6c, 0x83, 0x53, 0xf3, 0x3e, 0x17, 0xfe, 0x6e, 0x0a, + 0x52, 0xcd, 0xc7, 0x44, 0x9b, 0xdf, 0xed, 0x69, 0x87, 0xd8, 0xe5, 0xe9, 0xfc, 0x8b, 0x6a, 0xf9, + 0x36, 0xde, 0x33, 0x98, 0xd2, 0x95, 0x93, 0xf9, 0x17, 0x7a, 0x13, 0x40, 0xd5, 0x34, 0xec, 0x38, + 0x8a, 0x17, 0x02, 0x98, 0x93, 0x73, 0x0c, 0xb2, 0x8e, 0x4f, 0x08, 0x9a, 0x83, 0x35, 0x1b, 0xbb, + 0x9e, 0x0f, 0x15, 0xfb, 0x22, 0x68, 0x2e, 0xee, 0x74, 0x15, 0xd7, 0x3a, 0xc4, 0x26, 0xed, 0xd2, + 0x1c, 0x91, 0x0a, 0x9d, 0x6e, 0x8b, 0x00, 0x88, 0x40, 0xc3, 0xa6, 0x1e, 0x48, 0x9f, 0x9c, 0xec, + 0x7f, 0x13, 0x92, 0x36, 0xde, 0x37, 0x78, 0x00, 0x5d, 0x4e, 0xe6, 0x5f, 0x84, 0x4b, 0x6a, 0xcf, + 0x3d, 0xa0, 0x3d, 0x91, 0x93, 0xe9, 0x6f, 0x74, 0x07, 0x4a, 0xcc, 0xed, 0x52, 0xc1, 0xa6, 0xa6, + 0x50, 0x39, 0x98, 0xa3, 0xc9, 0x05, 0x06, 0xae, 0x99, 0x1a, 0x91, 0x7a, 0xe8, 0x31, 0x70, 0x80, + 0x72, 0xd8, 0x71, 0x08, 0x4f, 0x81, 0xe4, 0xaa, 0x94, 0xce, 0x4e, 0x97, 0xf2, 0x4d, 0x9a, 0xb0, + 0xbe, 0xd9, 0x24, 0x6b, 0x09, 0xcb, 0xb5, 0xde, 0x71, 0x1a, 0xfa, 0xc2, 0x5f, 0x17, 0x20, 0xfd, + 0xb4, 0xda, 0xbc, 0x30, 0xcb, 0xbc, 0x8a, 0xa6, 0x43, 0x15, 0xbd, 0x0b, 0xa5, 0x5d, 0xa3, 0xdd, + 0x36, 0xcc, 0x7d, 0xa2, 0x5f, 0x7d, 0x1b, 0x6b, 0x1e, 0xc3, 0x8a, 0x1c, 0xbc, 0xcd, 0xa0, 0xe8, + 0x06, 0xe4, 0x35, 0x1b, 0xeb, 0xd8, 0x74, 0x0d, 0xb5, 0xed, 0x70, 0xce, 0x85, 0x41, 0x0b, 0xbf, + 0x26, 0xc0, 0x24, 0x1d, 0xac, 0xe8, 0x0d, 0xc8, 0x69, 0x96, 0xe9, 0xaa, 0x86, 0xc9, 0xa5, 0x4e, + 0x4e, 0x0e, 0x00, 0x43, 0xab, 0x77, 0x13, 0x66, 0x54, 0x4d, 0xb3, 0x7a, 0xa6, 0xab, 0x98, 0x6a, + 0x07, 0xf3, 0x6a, 0xe6, 0x39, 0x6c, 0x4b, 0xed, 0x60, 0xb4, 0x04, 0xde, 0xa7, 0x1f, 0xd9, 0x99, + 0x93, 0x81, 0x83, 0xd6, 0xf1, 0xc9, 0x02, 0x86, 0x9c, 0x3f, 0xaa, 0x49, 0x7b, 0x7b, 0x8e, 0x5f, + 0x03, 0xfa, 0x1b, 0xbd, 0x03, 0x97, 0x5e, 0xf6, 0xd4, 0xb6, 0xb1, 0x47, 0x37, 0xbf, 0xa8, 0x97, + 0x3a, 0x2d, 0x8c, 0x55, 0x05, 0xf9, 0x69, 0x94, 0x02, 0x2d, 0xd3, 0x9b, 0x04, 0xe9, 0x60, 0x12, + 0x30, 0x97, 0x1d, 0xe9, 0x04, 0x66, 0x65, 0xec, 0xda, 0x27, 0x2d, 0x16, 0xec, 0x5a, 0x3b, 0xc2, + 0xa6, 0x4b, 0xda, 0x6e, 0x75, 0x31, 0x73, 0x12, 0xf1, 0xda, 0xee, 0x03, 0xd0, 0x6d, 0x28, 0xaa, + 0x2e, 0x19, 0x6e, 0xae, 0x62, 0xf6, 0x3a, 0xbb, 0xd8, 0x66, 0xae, 0x00, 0x72, 0x81, 0x43, 0xb7, + 0x28, 0x90, 0x47, 0x64, 0xd8, 0x27, 0x0a, 0xdd, 0x27, 0xe2, 0x45, 0x03, 0x05, 0xd5, 0x08, 0x44, + 0xba, 0x0f, 0x97, 0x49, 0x3b, 0x6b, 0xa6, 0x66, 0x9f, 0x74, 0x09, 0xe5, 0x67, 0xf4, 0xaf, 0x83, + 0xc4, 0xd0, 0x39, 0x0d, 0x3d, 0x9e, 0x91, 0x7e, 0x34, 0x0d, 0x85, 0xda, 0x71, 0xd7, 0xb2, 0x13, + 0xdd, 0xd5, 0xa9, 0xc0, 0x34, 0x37, 0x7c, 0x47, 0x1c, 0xc5, 0xf6, 0x49, 0x20, 0xef, 0x1c, 0x9a, + 0x23, 0xa2, 0x17, 0x7e, 0x60, 0xea, 0x21, 0x3e, 0x21, 0x5a, 0xfd, 0xcc, 0x38, 0x5a, 0xfd, 0x1c, + 0xb7, 0xa5, 0xf3, 0x2c, 0x8e, 0x74, 0x1d, 0x9f, 0xb4, 0x9a, 0x32, 0x8f, 0x8d, 0x25, 0x1f, 0x0e, + 0xda, 0x82, 0x7c, 0xe7, 0x48, 0xd3, 0x94, 0x3d, 0xa3, 0xed, 0x72, 0x5f, 0xb5, 0x78, 0xb7, 0xea, + 0xcd, 0x17, 0xd5, 0xea, 0x13, 0x9a, 0x89, 0xf9, 0x78, 0x05, 0xdf, 0x32, 0x10, 0x0a, 0xec, 0x37, + 0xaa, 0x00, 0x30, 0xc7, 0x4f, 0xea, 0x2f, 0x94, 0xbe, 0xc0, 0x09, 0x17, 0x45, 0xa3, 0x41, 0x11, + 0x12, 0x14, 0x98, 0x7b, 0x52, 0xc7, 0x60, 0x51, 0xc9, 0x05, 0xb6, 0x0d, 0x47, 0x81, 0x9b, 0x06, + 0x8d, 0x48, 0xfe, 0x12, 0xf0, 0xa0, 0x1b, 0xc5, 0xf1, 0x42, 0xe8, 0x2a, 0x85, 0xb3, 0xd3, 0xa5, + 0x9c, 0x4c, 0xa1, 0xcd, 0x66, 0x4b, 0xce, 0xb1, 0x0c, 0x4d, 0xc7, 0xbd, 0x48, 0x98, 0xc5, 0xf4, + 0xf8, 0x61, 0x16, 0x7f, 0x59, 0x80, 0x2b, 0xbc, 0x7f, 0x94, 0x5d, 0xea, 0x5a, 0xae, 0xb6, 0x0d, + 0xf7, 0x44, 0x39, 0x3c, 0x9a, 0xcf, 0x52, 0x75, 0xeb, 0x6b, 0xb1, 0xfd, 0x1c, 0x1a, 0x5e, 0x65, + 0xaf, 0xb7, 0x4f, 0x36, 0x38, 0xf2, 0xfa, 0x51, 0xcd, 0x74, 0xed, 0x93, 0xca, 0xd5, 0xb3, 0xd3, + 0xa5, 0xb9, 0xc1, 0xd4, 0x17, 0xf2, 0x9c, 0x33, 0x88, 0x82, 0xea, 0x00, 0xd8, 0x1f, 0xde, 0x54, + 0x7a, 0xc6, 0x2f, 0x9b, 0xb1, 0xf3, 0x40, 0x0e, 0xe1, 0xa2, 0x7b, 0x20, 0xf2, 0xb0, 0x96, 0x3d, + 0xa3, 0x8d, 0x15, 0xc7, 0xf8, 0x14, 0x53, 0x39, 0x9b, 0x96, 0x8b, 0x0c, 0x4e, 0x48, 0x34, 0x8d, + 0x4f, 0x31, 0x7a, 0x08, 0x97, 0x83, 0x1e, 0x50, 0x76, 0x71, 0xdb, 0x7a, 0xc5, 0xb2, 0xe7, 0x69, + 0x76, 0xe4, 0x73, 0xbf, 0x42, 0x92, 0x08, 0xca, 0xc2, 0xb7, 0x61, 0x7e, 0x58, 0x83, 0xc3, 0x93, + 0x31, 0xc7, 0xce, 0x4a, 0xdf, 0x8d, 0x6e, 0x94, 0x8c, 0x31, 0x69, 0xf8, 0x66, 0xc9, 0x7b, 0xa9, + 0x77, 0x05, 0xe9, 0xef, 0xa4, 0xa0, 0x50, 0xe9, 0xb5, 0x0f, 0x9f, 0x75, 0x9b, 0xec, 0x4a, 0x00, + 0x74, 0x1d, 0x72, 0xba, 0xea, 0xaa, 0xac, 0x92, 0x02, 0x0b, 0x6f, 0x23, 0x00, 0xda, 0x9a, 0xbb, + 0x50, 0x0a, 0xf9, 0xa1, 0x70, 0x6f, 0x7b, 0xda, 0xec, 0x00, 0x4c, 0x1d, 0xe2, 0xdf, 0x85, 0xf9, + 0x50, 0x46, 0xba, 0xab, 0xa1, 0x60, 0xd3, 0xb5, 0x0d, 0xcc, 0x76, 0xe6, 0xd2, 0x72, 0xc8, 0x59, + 0xa6, 0x41, 0x92, 0x6b, 0x2c, 0x15, 0xb5, 0x60, 0x86, 0x64, 0x3c, 0x51, 0xa8, 0x04, 0xf6, 0x76, + 0x4e, 0x1f, 0xc6, 0x34, 0x2b, 0x52, 0xef, 0x32, 0xe5, 0x4f, 0x95, 0xe2, 0xd0, 0x9f, 0x72, 0x1e, + 0x07, 0x90, 0x85, 0x0f, 0x41, 0xec, 0xcf, 0x10, 0xe6, 0x65, 0x86, 0xf1, 0xf2, 0x52, 0x98, 0x97, + 0xe9, 0x10, 0x9f, 0xd6, 0x32, 0xd9, 0x8c, 0x38, 0x29, 0xfd, 0x56, 0x06, 0x8a, 0xde, 0xc8, 0x4c, + 0x52, 0xa5, 0xaf, 0xc0, 0x24, 0x19, 0x47, 0x9e, 0x6b, 0xc7, 0x9d, 0x11, 0x13, 0x82, 0xfb, 0x77, + 0x93, 0xf1, 0xe5, 0x59, 0x9f, 0x14, 0x35, 0x09, 0x91, 0xb2, 0xf0, 0x7b, 0x29, 0xc8, 0x50, 0x2d, + 0xfa, 0x21, 0x64, 0xe8, 0x9d, 0x00, 0xc2, 0xc8, 0x3b, 0x01, 0x3c, 0xd7, 0x00, 0x92, 0x15, 0x6d, + 0x92, 0xc9, 0xa5, 0x7b, 0x72, 0x37, 0x37, 0x4e, 0xf9, 0x22, 0x97, 0xbb, 0xd9, 0x1a, 0xbd, 0x3f, + 0xa1, 0xd5, 0xa4, 0xaa, 0x11, 0x93, 0xb8, 0xde, 0x1a, 0x99, 0x0a, 0x29, 0x8a, 0x15, 0xea, 0xaa, + 0x64, 0xd9, 0x2e, 0xd6, 0xb9, 0xd2, 0x7b, 0xe3, 0xbc, 0x61, 0xe1, 0x29, 0xf5, 0x1e, 0x1e, 0xba, + 0x06, 0x69, 0x22, 0x0a, 0xa7, 0x99, 0xd7, 0xc1, 0xd9, 0xe9, 0x52, 0x9a, 0x08, 0x41, 0x02, 0x43, + 0xcb, 0x90, 0x8f, 0x0a, 0x27, 0xa2, 0x37, 0x51, 0x29, 0x1e, 0x12, 0x2c, 0xd0, 0xf6, 0x67, 0x24, + 0x33, 0xf8, 0xd8, 0xd0, 0xe0, 0xfe, 0x06, 0xbf, 0x2e, 0x70, 0xf7, 0xca, 0xa6, 0x46, 0x96, 0x5f, + 0x3b, 0xc9, 0xf5, 0xf1, 0x3e, 0x88, 0xb6, 0x6a, 0xea, 0x56, 0xc7, 0xf8, 0x14, 0xb3, 0x0d, 0x06, + 0x87, 0x9f, 0xbc, 0x94, 0x7c, 0x38, 0xdd, 0x09, 0x70, 0xa4, 0x3f, 0x11, 0xb8, 0x2b, 0xa6, 0x5f, + 0x8d, 0x64, 0xcf, 0xc7, 0xf3, 0x7c, 0x77, 0xd2, 0xdc, 0xb3, 0x3c, 0x4f, 0x92, 0x37, 0x86, 0xf9, + 0x4d, 0x35, 0xcc, 0x3d, 0xcb, 0x3b, 0xe9, 0xb3, 0x3d, 0x80, 0xb3, 0xf0, 0x4b, 0x30, 0x49, 0x93, + 0x5f, 0x63, 0xa8, 0xf9, 0xee, 0xbf, 0x29, 0x31, 0x2d, 0xfd, 0x71, 0x0a, 0xde, 0xa2, 0x4d, 0x7d, + 0x81, 0x6d, 0x63, 0xef, 0x64, 0xdb, 0xb6, 0x5c, 0xac, 0xb9, 0x58, 0x0f, 0x36, 0xd8, 0x12, 0xec, + 0x02, 0x1d, 0x72, 0xfc, 0x68, 0xd2, 0xd0, 0xf9, 0x25, 0x20, 0x4f, 0x3f, 0x9f, 0xe1, 0x9d, 0x65, + 0x47, 0x9a, 0x8d, 0x55, 0x39, 0xcb, 0x28, 0x37, 0x74, 0xb4, 0x02, 0xb9, 0xae, 0xd7, 0x8c, 0x0b, + 0x79, 0xbf, 0xf8, 0x58, 0x68, 0x1d, 0x4a, 0xbc, 0xa2, 0x6a, 0xdb, 0x38, 0xc2, 0x8a, 0xea, 0x5e, + 0x44, 0x22, 0x14, 0x18, 0xee, 0x0a, 0x41, 0x5d, 0x71, 0xa5, 0xbf, 0x9a, 0x81, 0xdb, 0xe7, 0xb0, + 0x38, 0xc9, 0xe1, 0xb5, 0x00, 0xd9, 0x23, 0x52, 0x90, 0xc1, 0x5b, 0x9f, 0x95, 0xfd, 0x6f, 0xb4, + 0x1b, 0x59, 0x56, 0xf6, 0x54, 0xa3, 0x4d, 0x96, 0x21, 0xe6, 0x6f, 0x38, 0xdc, 0xa3, 0x29, 0xde, + 0x7f, 0x2f, 0xb4, 0x00, 0x3d, 0xa1, 0x84, 0x68, 0x36, 0x07, 0x7d, 0x26, 0xc0, 0x02, 0x2b, 0x90, + 0x39, 0xbd, 0xf5, 0x15, 0x93, 0xa1, 0xc5, 0xac, 0xc6, 0x14, 0x33, 0x16, 0x8f, 0xca, 0xa1, 0xb2, + 0x78, 0x45, 0xe6, 0xc3, 0xa5, 0x85, 0xab, 0xb2, 0xf0, 0x1b, 0x02, 0xe4, 0x43, 0x00, 0x74, 0x67, + 0x20, 0xc4, 0x28, 0x7f, 0x16, 0x17, 0x57, 0x74, 0x7b, 0x20, 0xae, 0xa8, 0x92, 0xfd, 0xd9, 0xe9, + 0x52, 0x46, 0x66, 0xae, 0xeb, 0x5e, 0x84, 0xd1, 0xcd, 0xe0, 0x46, 0x9b, 0x74, 0x5f, 0x26, 0xef, + 0x4a, 0x1b, 0x6a, 0xa1, 0xaa, 0xde, 0x89, 0x18, 0xb5, 0x50, 0xc9, 0x97, 0xf4, 0xfd, 0x14, 0xcc, + 0xae, 0xe8, 0x7a, 0xb3, 0x49, 0xad, 0x9a, 0x24, 0xe7, 0x18, 0x82, 0x0c, 0x51, 0x37, 0x78, 0x38, + 0x14, 0xfd, 0x8d, 0xde, 0x06, 0xa4, 0x1b, 0x0e, 0xbb, 0x19, 0xc2, 0x39, 0x50, 0x75, 0xeb, 0x55, + 0x70, 0xf0, 0x3d, 0xeb, 0xa5, 0x34, 0xbd, 0x04, 0xd4, 0x04, 0xaa, 0x6b, 0x2b, 0x8e, 0xab, 0xfa, + 0x1b, 0xfb, 0xb7, 0xc7, 0x0a, 0xb0, 0x61, 0xca, 0xb1, 0xff, 0x29, 0xe7, 0x08, 0x1d, 0xfa, 0x93, + 0xa8, 0x7c, 0x06, 0xe9, 0x14, 0x57, 0x51, 0x1d, 0x2f, 0x34, 0x84, 0xdd, 0x49, 0x51, 0x64, 0xf0, + 0x15, 0x87, 0x45, 0x7c, 0x30, 0x8f, 0xf2, 0x80, 0x35, 0x49, 0x6e, 0xcb, 0xfe, 0x4d, 0x01, 0x8a, + 0x32, 0xde, 0xb3, 0xb1, 0x73, 0x90, 0x24, 0xcf, 0x9f, 0xc0, 0x8c, 0xcd, 0xa8, 0x2a, 0x7b, 0xb6, + 0xd5, 0xb9, 0x88, 0xac, 0xc8, 0x73, 0xc4, 0x27, 0xb6, 0xd5, 0xe1, 0x22, 0xf9, 0x05, 0x94, 0xfc, + 0x3a, 0x26, 0xd9, 0xf8, 0xbf, 0x4d, 0x83, 0x47, 0x19, 0xe1, 0xa4, 0x4f, 0xa0, 0x93, 0xe5, 0x00, + 0xdd, 0x9a, 0x0f, 0x57, 0x34, 0x49, 0x36, 0xfc, 0x07, 0x01, 0x8a, 0xcd, 0xde, 0x2e, 0xbb, 0xf1, + 0x28, 0x39, 0x0e, 0xd4, 0x20, 0xd7, 0xc6, 0x7b, 0xae, 0xf2, 0x5a, 0xbe, 0xd0, 0x59, 0x82, 0x4a, + 0x3d, 0xc1, 0x9f, 0x02, 0xd8, 0x34, 0x7a, 0x8a, 0xd2, 0x49, 0x5f, 0x90, 0x4e, 0x8e, 0xe2, 0x12, + 0x30, 0x59, 0x75, 0x4a, 0x7e, 0x33, 0x93, 0x5c, 0x5f, 0x3e, 0x8a, 0x48, 0x87, 0xf4, 0x45, 0xa4, + 0xc3, 0x2c, 0x57, 0x58, 0xe3, 0x25, 0x44, 0x19, 0xe6, 0xa8, 0x5a, 0xa6, 0xa8, 0xdd, 0x6e, 0xdb, + 0xf0, 0xcc, 0x1e, 0x2a, 0x7f, 0x32, 0xf2, 0x2c, 0x4d, 0x5a, 0x61, 0x29, 0xd4, 0xe0, 0x41, 0xdf, + 0x11, 0x60, 0x66, 0xcf, 0xc6, 0xf8, 0x53, 0xac, 0x50, 0x91, 0x3c, 0x9e, 0x57, 0xc1, 0x2a, 0xa9, + 0xc3, 0xe7, 0x3e, 0x75, 0xcc, 0xb3, 0x82, 0x9b, 0xa4, 0x5c, 0xb4, 0x05, 0xa2, 0xd6, 0x66, 0xe7, + 0xa0, 0xbe, 0x87, 0xc3, 0xd4, 0xf8, 0x13, 0xa0, 0xc4, 0x90, 0x03, 0x27, 0x87, 0xe7, 0x64, 0x32, + 0xa9, 0xba, 0xc2, 0x6f, 0x99, 0xa3, 0xca, 0x76, 0xd4, 0xc3, 0x21, 0x1c, 0x45, 0x1e, 0xba, 0x9c, + 0xae, 0x2c, 0x63, 0x55, 0xe7, 0x9a, 0x3b, 0x99, 0x57, 0xfe, 0x07, 0x9f, 0x57, 0x1f, 0xc1, 0x2c, + 0x1d, 0x37, 0x49, 0x07, 0x83, 0x4a, 0x7f, 0x3f, 0x0d, 0x28, 0x4c, 0xf9, 0x8b, 0x1b, 0x6f, 0xa9, + 0xe4, 0xc6, 0xdb, 0x1a, 0x48, 0x21, 0x65, 0xa8, 0xad, 0x3a, 0xae, 0xc2, 0x5c, 0xe9, 0x1c, 0xa5, + 0x8b, 0x6d, 0xc5, 0xc1, 0x9a, 0xc5, 0xef, 0x03, 0x12, 0xe4, 0xc5, 0x20, 0xe7, 0x86, 0xea, 0xb8, + 0xcf, 0x59, 0xbe, 0x6d, 0x6c, 0x37, 0x69, 0x2e, 0xf4, 0x18, 0xae, 0x74, 0xd4, 0xe3, 0x38, 0xfc, + 0x49, 0x8a, 0x3f, 0xd7, 0x51, 0x8f, 0x07, 0x90, 0xde, 0x83, 0x85, 0x78, 0x24, 0xc5, 0xc1, 0xde, + 0x51, 0xdb, 0x95, 0x18, 0xc4, 0x26, 0x76, 0xd1, 0x0a, 0x40, 0x60, 0x44, 0xf0, 0x35, 0x7a, 0x1c, + 0x1b, 0x22, 0xe7, 0xdb, 0x10, 0xd2, 0x77, 0x05, 0x28, 0x6e, 0x1a, 0xfb, 0xb6, 0x9a, 0xe8, 0x6d, + 0x3b, 0xe8, 0xbd, 0xe8, 0xd9, 0x64, 0xfe, 0xd1, 0x42, 0x9c, 0xef, 0x09, 0xcb, 0xe1, 0xed, 0x3f, + 0x72, 0x04, 0xb2, 0xf4, 0xf9, 0x35, 0x4a, 0x52, 0xe6, 0x6b, 0xf0, 0x26, 0xf5, 0xe6, 0xe3, 0xae, + 0x3c, 0x5f, 0x88, 0x75, 0x23, 0xfd, 0xae, 0x00, 0x8b, 0xc3, 0x4a, 0x49, 0x72, 0x42, 0xc8, 0xf4, + 0xde, 0x41, 0x5a, 0x82, 0xe2, 0xcf, 0x88, 0x73, 0x24, 0x0d, 0xe2, 0x33, 0x01, 0xfc, 0xba, 0x35, + 0xe9, 0x8d, 0x84, 0xec, 0xb7, 0x43, 0x18, 0xd4, 0xd4, 0x54, 0x93, 0x86, 0x0a, 0xb7, 0xb1, 0x7a, + 0x84, 0xb9, 0x23, 0x62, 0xa2, 0x52, 0xe2, 0x07, 0x02, 0x2c, 0x0e, 0x2b, 0x25, 0x49, 0x06, 0x7d, + 0x0d, 0xa6, 0x99, 0x47, 0x9b, 0x67, 0xd4, 0x5c, 0x1b, 0x12, 0x15, 0x6d, 0x7a, 0xb1, 0x38, 0x5e, + 0x7e, 0xa9, 0x05, 0xc5, 0x8a, 0x6a, 0xdb, 0x46, 0xa2, 0x5b, 0x0f, 0xd2, 0xf7, 0x05, 0x28, 0xf9, + 0x64, 0x93, 0x6c, 0xe9, 0xe7, 0x0f, 0xf4, 0x90, 0xfe, 0xf3, 0x75, 0x98, 0xe1, 0xf5, 0xdf, 0x31, + 0x0d, 0xcb, 0x44, 0x0f, 0x21, 0xbd, 0xcf, 0x0f, 0xb0, 0xf2, 0xb1, 0x7b, 0xf4, 0xc1, 0x35, 0x8f, + 0xf5, 0x09, 0x99, 0xe4, 0x25, 0x28, 0xdd, 0x9e, 0x1b, 0x53, 0x81, 0x20, 0x52, 0x21, 0x8c, 0xd2, + 0xed, 0xb9, 0xa8, 0x09, 0x25, 0x2d, 0xb8, 0x5b, 0x4e, 0x21, 0xe8, 0xe9, 0xa1, 0x1b, 0xca, 0xb1, + 0xb7, 0xfc, 0xd5, 0x27, 0xe4, 0xa2, 0x16, 0x49, 0x40, 0xd5, 0xf0, 0x95, 0x66, 0x99, 0x01, 0x37, + 0xc8, 0xa0, 0xeb, 0xa3, 0xd7, 0xa9, 0xd5, 0x27, 0x42, 0x37, 0x9f, 0xa1, 0xf7, 0x60, 0x4a, 0xa7, + 0x97, 0x67, 0x71, 0x7d, 0x22, 0xae, 0xc3, 0x23, 0x77, 0x94, 0xd5, 0x27, 0x64, 0x8e, 0x81, 0xd6, + 0x60, 0x86, 0xfd, 0x62, 0xd6, 0x2e, 0xd7, 0x02, 0x6e, 0x0f, 0xa7, 0x10, 0xd2, 0xc3, 0xeb, 0x13, + 0x72, 0x5e, 0x0f, 0xa0, 0xe8, 0x29, 0xe4, 0xb5, 0x36, 0x56, 0x6d, 0x4e, 0xea, 0xce, 0xd0, 0xd8, + 0xcd, 0x81, 0x0b, 0xb7, 0xea, 0x13, 0x32, 0x68, 0x3e, 0x90, 0x54, 0xca, 0xa6, 0xf7, 0x2e, 0x71, + 0x4a, 0xef, 0x0c, 0xad, 0xd4, 0xe0, 0x25, 0x56, 0x75, 0xaa, 0x9f, 0xfb, 0x50, 0xf4, 0x65, 0xc8, + 0x38, 0x9a, 0x6a, 0x72, 0x95, 0x64, 0x71, 0xc8, 0xc5, 0x38, 0x01, 0x32, 0xcd, 0x8d, 0xde, 0x67, + 0x86, 0xb2, 0x7b, 0xec, 0x9d, 0x1a, 0xc4, 0xf1, 0x34, 0x72, 0x01, 0x03, 0xe1, 0x29, 0xa6, 0x00, + 0xc2, 0x07, 0x55, 0xef, 0x18, 0xa6, 0x42, 0x0f, 0x66, 0xe8, 0x31, 0x41, 0x3c, 0x1f, 0x06, 0xa2, + 0xd8, 0xeb, 0xf4, 0x96, 0x07, 0x0f, 0x88, 0x36, 0xa1, 0xc0, 0x08, 0xf5, 0x58, 0x80, 0xf5, 0xfc, + 0xf2, 0x50, 0x5f, 0x84, 0x98, 0x10, 0xef, 0xfa, 0x84, 0x3c, 0xa3, 0x86, 0xc0, 0x41, 0xbd, 0x3a, + 0xd8, 0xde, 0x67, 0xe7, 0x11, 0x23, 0xea, 0x15, 0x76, 0xf0, 0xf4, 0xeb, 0x45, 0x81, 0xe8, 0x57, + 0xe1, 0x12, 0x23, 0xe4, 0x72, 0xbf, 0x35, 0xee, 0xfe, 0xf4, 0xe6, 0x50, 0x3f, 0x82, 0xa1, 0x41, + 0xd1, 0xf5, 0x09, 0x19, 0xa9, 0x03, 0x89, 0x48, 0x83, 0xcb, 0xac, 0x04, 0x1e, 0x55, 0x6b, 0xf3, + 0x40, 0xd0, 0xf9, 0x5b, 0xb4, 0x88, 0xb7, 0x87, 0x15, 0x11, 0x1b, 0xec, 0x5b, 0x9f, 0x90, 0xe7, + 0xd4, 0xc1, 0xd4, 0xa0, 0x19, 0x36, 0x8f, 0x5f, 0xe4, 0xc3, 0xed, 0xed, 0xd1, 0xcd, 0x88, 0x8b, + 0xfb, 0xf4, 0x9b, 0x11, 0x49, 0x24, 0x1d, 0xe8, 0xdf, 0xde, 0x40, 0x07, 0xd3, 0xcc, 0xd0, 0x0e, + 0x8c, 0x09, 0x72, 0x24, 0x1d, 0x78, 0x10, 0x02, 0xa3, 0x32, 0xa4, 0xf6, 0x35, 0x7a, 0xea, 0x17, + 0xaf, 0x3a, 0xf9, 0x81, 0x7c, 0xf5, 0x09, 0x39, 0xb5, 0xaf, 0xa1, 0x0f, 0x21, 0xcb, 0xa2, 0xb2, + 0x8e, 0xcd, 0xf9, 0xe2, 0x50, 0x99, 0x1d, 0x8d, 0x6d, 0xab, 0x4f, 0xc8, 0x34, 0x10, 0x8c, 0x0f, + 0x64, 0x1e, 0x71, 0x43, 0x49, 0x94, 0x47, 0x04, 0x63, 0xf7, 0xc5, 0x3d, 0x91, 0x01, 0x63, 0xfb, + 0x40, 0xb4, 0x0d, 0x45, 0xbe, 0x74, 0x7b, 0x11, 0x04, 0xe2, 0x50, 0x4f, 0x9d, 0xb8, 0x20, 0x82, + 0x3a, 0xdd, 0xa2, 0x0c, 0xc1, 0x49, 0xdf, 0x45, 0x29, 0xf2, 0xbe, 0x9b, 0x1d, 0xda, 0x77, 0x43, + 0x1d, 0xda, 0x49, 0xdf, 0xd9, 0x03, 0x89, 0xe8, 0xab, 0x30, 0xc9, 0xe6, 0x09, 0xa2, 0x24, 0xe3, + 0x9c, 0xcf, 0xfa, 0xa6, 0x08, 0xcb, 0x4f, 0xa4, 0x97, 0xcb, 0x1d, 0x73, 0x95, 0xb6, 0xb5, 0x3f, + 0x3f, 0x37, 0x54, 0x7a, 0x0d, 0xba, 0x18, 0x13, 0xe9, 0xe5, 0x06, 0x50, 0x32, 0x80, 0x6c, 0x96, + 0xc2, 0xa7, 0xd8, 0xa5, 0xa1, 0x03, 0x28, 0xc6, 0x5f, 0xb7, 0x4e, 0x43, 0xa6, 0x02, 0xb0, 0x2f, + 0x58, 0x1d, 0xac, 0x50, 0xa1, 0x78, 0x79, 0xb4, 0x60, 0x8d, 0xdc, 0x56, 0xe6, 0x0b, 0x56, 0x06, + 0x45, 0x2f, 0x40, 0xe4, 0x57, 0xe6, 0x28, 0x9e, 0xff, 0xd8, 0xfc, 0x15, 0x4a, 0xef, 0x7e, 0xec, + 0x82, 0x18, 0xe7, 0x5a, 0x58, 0x27, 0xb6, 0x64, 0x34, 0x05, 0x7d, 0x0c, 0xb3, 0x94, 0x9e, 0xa2, + 0x05, 0xb7, 0x1c, 0xcd, 0xcf, 0x0f, 0xdc, 0x99, 0x33, 0xfc, 0x42, 0x24, 0x8f, 0xb2, 0xa8, 0xf5, + 0x25, 0x91, 0xf9, 0x60, 0x98, 0x86, 0x4b, 0xd7, 0xee, 0x85, 0xa1, 0xf3, 0x21, 0x7a, 0xc3, 0x6b, + 0x9d, 0xea, 0x5a, 0x14, 0x42, 0x86, 0x71, 0x9f, 0xc4, 0x7b, 0x63, 0xe8, 0x30, 0x1e, 0x22, 0xec, + 0x0a, 0x6e, 0x44, 0xce, 0xad, 0x02, 0xb0, 0x1d, 0x04, 0x6a, 0x14, 0x2d, 0x0e, 0x55, 0x00, 0xfa, + 0xfd, 0x69, 0x89, 0x02, 0xd0, 0xf6, 0x60, 0x44, 0x01, 0x60, 0xc7, 0x5d, 0xf3, 0x37, 0x86, 0x2f, + 0x56, 0xe1, 0xf3, 0x75, 0xba, 0x58, 0x51, 0x00, 0x51, 0xc8, 0x88, 0x39, 0x77, 0x42, 0x67, 0xf8, + 0xcd, 0xa1, 0xbb, 0x37, 0x7d, 0x81, 0x76, 0xf5, 0x09, 0x39, 0xfb, 0x92, 0x83, 0xc8, 0xa8, 0x62, + 0x24, 0xf8, 0xdc, 0x7e, 0x30, 0x74, 0x54, 0x0d, 0x46, 0x58, 0x91, 0x51, 0xf5, 0x32, 0x80, 0x06, + 0x4b, 0x9e, 0xc3, 0x0e, 0xb2, 0xe6, 0xdf, 0x1a, 0xbd, 0xe4, 0x45, 0x8f, 0xdd, 0xfc, 0x25, 0x8f, + 0x83, 0xd9, 0x92, 0xa7, 0x2b, 0x8e, 0x43, 0xdd, 0x74, 0xe6, 0x6f, 0x8f, 0x58, 0xf2, 0xfa, 0xb6, + 0xb6, 0xd9, 0x92, 0xa7, 0x37, 0x19, 0x26, 0xd1, 0xfe, 0x6c, 0xef, 0x8a, 0x28, 0x6e, 0xd8, 0xdf, + 0x1d, 0xaa, 0xfd, 0xc5, 0xde, 0x61, 0x45, 0xb4, 0x3f, 0x3b, 0x92, 0x80, 0xbe, 0x0e, 0xd3, 0x7c, + 0x2b, 0x71, 0xfe, 0xde, 0x08, 0x95, 0x3a, 0xbc, 0xfb, 0x4b, 0x86, 0x23, 0xc7, 0x61, 0xc2, 0x81, + 0x6d, 0x61, 0x32, 0xe1, 0x77, 0x7f, 0x84, 0x70, 0x18, 0xd8, 0x45, 0x65, 0xc2, 0x21, 0x00, 0x93, + 0xda, 0x38, 0x6c, 0xfb, 0x6d, 0xfe, 0x17, 0x86, 0xd6, 0x26, 0xba, 0x0f, 0x49, 0x6a, 0xc3, 0x71, + 0xe8, 0x62, 0x41, 0xd7, 0x6a, 0xc6, 0x9d, 0x2f, 0x0d, 0x5f, 0x2c, 0xfa, 0x37, 0x74, 0xea, 0xde, + 0x41, 0x21, 0xe3, 0xca, 0x5f, 0x14, 0xe0, 0x06, 0x1b, 0x03, 0xf4, 0x98, 0xe4, 0x44, 0xf1, 0x4f, + 0xb9, 0x42, 0xbb, 0x55, 0x0f, 0x29, 0xf9, 0xaf, 0x5e, 0xfc, 0x50, 0xc6, 0x2b, 0xf1, 0x4d, 0x75, + 0x54, 0x3e, 0xc2, 0x8c, 0x0e, 0xb3, 0xeb, 0xe7, 0x1f, 0x0d, 0x65, 0x46, 0x74, 0x2f, 0x82, 0x30, + 0x83, 0xe3, 0xa0, 0x36, 0xcc, 0xb3, 0x29, 0x11, 0xd8, 0xbd, 0x7e, 0xd5, 0x1f, 0x0f, 0xf5, 0x7c, + 0x1d, 0x69, 0xf1, 0xd7, 0x27, 0xe4, 0x2b, 0x2f, 0x63, 0x33, 0x90, 0xd2, 0xe8, 0x1d, 0x90, 0x46, + 0x60, 0xa6, 0x2a, 0x9e, 0x3d, 0xf9, 0xe5, 0xa1, 0xa5, 0x8d, 0x34, 0x9f, 0x49, 0x69, 0x4e, 0x6c, + 0x06, 0xc2, 0x9a, 0x5d, 0x66, 0x1a, 0xce, 0x7f, 0x65, 0x28, 0x6b, 0xa2, 0x36, 0x29, 0x61, 0x0d, + 0xc7, 0xa9, 0x4c, 0x73, 0x97, 0x0b, 0x3f, 0xb4, 0xbb, 0x24, 0x8a, 0x6b, 0x99, 0xec, 0x55, 0x71, + 0x7e, 0x2d, 0x93, 0xbd, 0x26, 0x2e, 0xac, 0x65, 0xb2, 0xd7, 0xc5, 0x37, 0xd6, 0x32, 0xd9, 0x25, + 0xf1, 0xc6, 0x5a, 0x26, 0x2b, 0x89, 0xb7, 0xa4, 0x9f, 0x5c, 0x87, 0x82, 0x67, 0x59, 0x32, 0x93, + 0xef, 0x51, 0xd8, 0xe4, 0x5b, 0x1c, 0x66, 0xf2, 0x71, 0x5b, 0x94, 0xdb, 0x7c, 0x8f, 0xc2, 0x36, + 0xdf, 0xe2, 0x30, 0x9b, 0x2f, 0xc0, 0x21, 0x46, 0x5f, 0x6b, 0x98, 0xd1, 0x77, 0x7f, 0x0c, 0xa3, + 0xcf, 0x27, 0xd5, 0x6f, 0xf5, 0xad, 0x0e, 0x5a, 0x7d, 0x6f, 0x8d, 0xb6, 0xfa, 0x7c, 0x52, 0x21, + 0xb3, 0xef, 0xfd, 0x3e, 0xb3, 0xef, 0xe6, 0x08, 0xb3, 0xcf, 0xc7, 0xf7, 0xec, 0xbe, 0xf5, 0x58, + 0xbb, 0xef, 0xce, 0x79, 0x76, 0x9f, 0x4f, 0x27, 0x62, 0xf8, 0xd5, 0xe3, 0x0c, 0xbf, 0xdb, 0xe7, + 0x18, 0x7e, 0x3e, 0xa9, 0xb0, 0xe5, 0xb7, 0x1e, 0x6b, 0xf9, 0xdd, 0x39, 0xcf, 0xf2, 0x0b, 0xaa, + 0x15, 0x36, 0xfd, 0xbe, 0x12, 0x31, 0xfd, 0x96, 0x86, 0x9a, 0x7e, 0x3e, 0x36, 0xb3, 0xfd, 0x3e, + 0xe8, 0xb7, 0xfd, 0x6e, 0x8e, 0xb0, 0xfd, 0x02, 0xc6, 0x72, 0xe3, 0xaf, 0x1e, 0x67, 0xfc, 0xdd, + 0x3e, 0xc7, 0xf8, 0x0b, 0x78, 0x11, 0xb2, 0xfe, 0xb6, 0xe2, 0xad, 0xbf, 0xbb, 0xe7, 0x5a, 0x7f, + 0x3e, 0xb5, 0xa8, 0xf9, 0x57, 0x8f, 0x33, 0xff, 0x6e, 0x9f, 0x63, 0xfe, 0xf5, 0xd5, 0x8c, 0xd9, + 0x7f, 0xea, 0x48, 0xfb, 0xef, 0xed, 0x31, 0xed, 0x3f, 0x9f, 0x74, 0x9c, 0x01, 0xa8, 0x8f, 0x36, + 0x00, 0xcb, 0xe3, 0x1a, 0x80, 0x7e, 0x21, 0xb1, 0x16, 0xa0, 0x3a, 0xd2, 0x02, 0x7c, 0x7b, 0x4c, + 0x0b, 0xb0, 0xaf, 0x21, 0x51, 0x13, 0x70, 0x2b, 0xde, 0x04, 0xbc, 0x7b, 0xae, 0x09, 0x18, 0xf4, + 0x62, 0xc4, 0x06, 0x5c, 0x0e, 0xd9, 0x80, 0x6f, 0x0e, 0xb1, 0x01, 0x7d, 0x54, 0x62, 0x04, 0x7e, + 0x63, 0xc0, 0x08, 0x94, 0x46, 0x19, 0x81, 0x3e, 0xae, 0x6f, 0x05, 0xd6, 0xe3, 0xac, 0xc0, 0xdb, + 0xe7, 0x58, 0x81, 0xc1, 0xb8, 0x09, 0x99, 0x81, 0xcf, 0x87, 0x98, 0x81, 0xf7, 0xce, 0x37, 0x03, + 0x7d, 0x7a, 0x7d, 0x76, 0xa0, 0x3a, 0xd2, 0x0e, 0x7c, 0x7b, 0x4c, 0x3b, 0x30, 0xe8, 0xc1, 0x18, + 0x43, 0xf0, 0xdd, 0xa8, 0x21, 0x78, 0x63, 0xb8, 0x21, 0xe8, 0x93, 0xe1, 0x96, 0xe0, 0x7a, 0xac, + 0x25, 0x78, 0xe7, 0x3c, 0x4b, 0x30, 0x90, 0x66, 0x61, 0x53, 0x70, 0x2b, 0xde, 0x14, 0xbc, 0x7b, + 0xae, 0x29, 0x18, 0x0c, 0xa4, 0x88, 0x2d, 0xb8, 0x1e, 0x6b, 0x0b, 0xde, 0x39, 0xcf, 0x16, 0xec, + 0x13, 0xb5, 0xdc, 0x18, 0xfc, 0x68, 0xa8, 0x31, 0xf8, 0x60, 0x1c, 0x63, 0xd0, 0x27, 0x3a, 0x60, + 0x0d, 0x7e, 0x32, 0xdc, 0x1a, 0xfc, 0x85, 0x0b, 0x5c, 0x8f, 0x1b, 0x6b, 0x0e, 0x7e, 0x63, 0xc0, + 0x1c, 0x94, 0x46, 0x99, 0x83, 0xc1, 0xcc, 0xf0, 0xec, 0xc1, 0x5a, 0x8c, 0xf5, 0xf6, 0xd6, 0x68, + 0xeb, 0x2d, 0x58, 0xc8, 0x03, 0xf3, 0xed, 0xfd, 0x3e, 0xf3, 0xed, 0xe6, 0xb9, 0xde, 0xa0, 0x21, + 0xfb, 0xad, 0x32, 0x68, 0xbf, 0xdd, 0x1a, 0x69, 0xbf, 0xf9, 0x14, 0x02, 0x03, 0x6e, 0x3d, 0xd6, + 0x80, 0xbb, 0x73, 0x9e, 0x01, 0x17, 0x0c, 0x85, 0xb0, 0x05, 0xb7, 0x15, 0x6f, 0xc1, 0xdd, 0x3d, + 0xd7, 0x82, 0xeb, 0x5b, 0xb6, 0x3c, 0x13, 0xae, 0x1e, 0x67, 0xc2, 0xdd, 0x3e, 0xc7, 0x84, 0x0b, + 0x2f, 0x5b, 0xbe, 0x0d, 0xd7, 0x1a, 0x66, 0xc3, 0xdd, 0x1f, 0xc3, 0x86, 0x0b, 0x94, 0xb9, 0x3e, + 0x23, 0xee, 0xc3, 0x7e, 0x23, 0x4e, 0x1a, 0x65, 0xc4, 0x05, 0x83, 0xc8, 0xb3, 0xe2, 0xb6, 0xe2, + 0xad, 0xb8, 0xbb, 0xe7, 0x5a, 0x71, 0xe1, 0x79, 0x1d, 0x32, 0xe3, 0x3e, 0xec, 0x37, 0xe3, 0xa4, + 0x51, 0x66, 0x5c, 0x50, 0x1f, 0xcf, 0x8e, 0xab, 0xc7, 0xd9, 0x71, 0xb7, 0xcf, 0xb1, 0xe3, 0x42, + 0xe2, 0x3e, 0x30, 0xe4, 0xfe, 0xd2, 0xf8, 0x86, 0xdc, 0xbb, 0xaf, 0xeb, 0x5d, 0x77, 0xbe, 0x25, + 0xf7, 0x61, 0xbf, 0x25, 0x27, 0x8d, 0xb2, 0xe4, 0x02, 0x7e, 0x78, 0xa6, 0x5c, 0xe7, 0x5c, 0x53, + 0xee, 0xe1, 0x05, 0x4c, 0x39, 0x9f, 0xfe, 0x30, 0x5b, 0xae, 0x73, 0xae, 0x2d, 0xf7, 0xf0, 0x02, + 0xb6, 0x5c, 0x50, 0xdc, 0x10, 0x63, 0xee, 0xc3, 0x7e, 0x63, 0x4e, 0x1a, 0x65, 0xcc, 0x05, 0xdc, + 0xb9, 0xb0, 0x35, 0xf7, 0x86, 0xf8, 0x66, 0xc4, 0xa6, 0xfb, 0x6b, 0x39, 0x98, 0xaa, 0xc7, 0x9c, + 0x09, 0x0a, 0xaf, 0x73, 0x26, 0x88, 0xbe, 0x0e, 0xd7, 0xfd, 0x0f, 0xea, 0xcf, 0xa5, 0xf0, 0x90, + 0x38, 0xad, 0x6d, 0x69, 0x87, 0x74, 0xa9, 0xcc, 0xca, 0xf3, 0x7e, 0x96, 0x27, 0xb6, 0xd5, 0x61, + 0xa1, 0x71, 0xd4, 0x4b, 0x06, 0xad, 0x92, 0x39, 0x4c, 0x75, 0xc2, 0xf3, 0x6f, 0x1c, 0x1d, 0xbc, + 0x13, 0x93, 0xa3, 0xbe, 0xc6, 0x25, 0x0e, 0xe8, 0x2b, 0x50, 0xe8, 0x39, 0xd8, 0x56, 0xba, 0xb6, + 0x61, 0xd9, 0x86, 0xcb, 0xc2, 0xd1, 0x84, 0x8a, 0xf8, 0xb3, 0xd3, 0xa5, 0x99, 0x1d, 0x07, 0xdb, + 0xdb, 0x1c, 0x2e, 0xcf, 0xf4, 0x42, 0x5f, 0xde, 0xd3, 0x7f, 0x93, 0xe3, 0x3f, 0xfd, 0xf7, 0x1c, + 0x44, 0xea, 0xa1, 0x13, 0x5e, 0x46, 0xd9, 0x3d, 0x6d, 0xf1, 0x2b, 0xbe, 0xaa, 0x87, 0x56, 0x4a, + 0x7a, 0x5f, 0x5b, 0xc9, 0x8e, 0x02, 0xd1, 0x53, 0x28, 0xda, 0x56, 0x8f, 0x5e, 0xc7, 0xd4, 0xb5, + 0xda, 0x86, 0x76, 0x42, 0xf5, 0x9b, 0x62, 0xfc, 0x79, 0x33, 0xcb, 0xb8, 0x4d, 0xf3, 0xc9, 0x05, + 0x3b, 0xfc, 0x89, 0x9a, 0x40, 0xaf, 0x62, 0xf2, 0xa8, 0xa0, 0x81, 0xcb, 0xef, 0x47, 0x3e, 0x41, + 0xf1, 0x91, 0x6a, 0xb8, 0x9c, 0x2e, 0xbc, 0xf2, 0x7f, 0xa3, 0x27, 0x30, 0x43, 0xaf, 0x05, 0x27, + 0x7d, 0x6e, 0xf5, 0x5c, 0xae, 0xde, 0x5c, 0x2b, 0xb3, 0x07, 0x22, 0xcb, 0xde, 0x03, 0x91, 0xe5, + 0x55, 0xfe, 0x40, 0x24, 0x3b, 0xab, 0xfe, 0xde, 0xbf, 0x5b, 0x12, 0xe4, 0xbc, 0xe7, 0x39, 0x65, + 0xf5, 0x5c, 0xf4, 0x10, 0x2e, 0x77, 0xd4, 0x63, 0x7a, 0xbb, 0xb8, 0xe2, 0xe9, 0x5f, 0xf4, 0x3e, + 0x45, 0xf6, 0x2a, 0x21, 0xea, 0xa8, 0xc7, 0xf4, 0xb5, 0x44, 0x96, 0x44, 0x9f, 0x3a, 0xba, 0x09, + 0x33, 0x3c, 0x56, 0x88, 0xbd, 0x84, 0x56, 0xa2, 0x39, 0xf9, 0xb3, 0x38, 0xec, 0x31, 0xb4, 0xdb, + 0x50, 0xd4, 0x0d, 0xc7, 0x35, 0x4c, 0xcd, 0xe5, 0x17, 0x97, 0xb3, 0xab, 0xbf, 0x0b, 0x1e, 0x94, + 0xdd, 0x4e, 0xde, 0x82, 0x59, 0xad, 0x6d, 0xf8, 0x5a, 0x2d, 0xd3, 0x33, 0x66, 0x87, 0xce, 0xd3, + 0x2a, 0xcd, 0xdb, 0xef, 0x40, 0x53, 0xd2, 0xa2, 0x60, 0x54, 0x85, 0xd2, 0xbe, 0xea, 0xe2, 0x57, + 0xea, 0x89, 0xe2, 0x85, 0xe1, 0xe6, 0xe9, 0xd5, 0x03, 0xd7, 0xcf, 0x4e, 0x97, 0x0a, 0x4f, 0x59, + 0xd2, 0x40, 0x34, 0x6e, 0x61, 0x3f, 0x94, 0xa0, 0xa3, 0xbb, 0x50, 0x52, 0x9d, 0x13, 0x53, 0xa3, + 0x23, 0x0a, 0x9b, 0x4e, 0xcf, 0xe1, 0xe1, 0x68, 0x45, 0x0a, 0xae, 0x7a, 0x50, 0xf4, 0x3e, 0x2c, + 0xf0, 0xf7, 0x49, 0x5e, 0xa9, 0xb6, 0xae, 0xd0, 0x51, 0x18, 0x4c, 0x77, 0x91, 0xe2, 0x5c, 0x65, + 0xef, 0x91, 0x90, 0x0c, 0x64, 0xe8, 0x05, 0xd2, 0xf0, 0x05, 0xcc, 0xd2, 0xa0, 0x34, 0xac, 0x93, + 0xe5, 0xa8, 0x8d, 0x4d, 0xec, 0x38, 0x23, 0x4e, 0x19, 0x2a, 0x2c, 0x6f, 0xd3, 0xcb, 0xca, 0x04, + 0x8c, 0x2c, 0xee, 0xf6, 0xc1, 0xfd, 0xfb, 0xd2, 0x41, 0xcc, 0xaf, 0x65, 0xb2, 0x33, 0x62, 0x61, + 0x2d, 0x93, 0x2d, 0x8a, 0x25, 0xe9, 0xff, 0x08, 0x70, 0x25, 0x9e, 0x08, 0x6a, 0xc2, 0x5c, 0xe4, + 0x4a, 0x20, 0x16, 0x2d, 0x77, 0x11, 0x79, 0x35, 0x1b, 0xbe, 0x18, 0x88, 0x96, 0x42, 0x98, 0x13, + 0x43, 0x54, 0x71, 0x5c, 0xdb, 0xd0, 0xbc, 0x6b, 0xb6, 0xae, 0x0e, 0xa0, 0x35, 0x69, 0x32, 0xad, + 0x91, 0x7a, 0x3c, 0x50, 0xa3, 0xf4, 0x45, 0x6a, 0xa4, 0x1e, 0x47, 0x49, 0x4b, 0xdf, 0x49, 0x43, + 0x89, 0xac, 0xbc, 0x8e, 0x63, 0x58, 0x66, 0xdd, 0x77, 0xd0, 0xf7, 0x05, 0x94, 0x40, 0x63, 0x4d, + 0xfd, 0x6f, 0xb4, 0x44, 0x63, 0x7a, 0x89, 0x85, 0xe3, 0xbf, 0xf7, 0x94, 0x96, 0x81, 0x81, 0x68, + 0xd4, 0xe2, 0x0a, 0x4c, 0x39, 0x56, 0xcf, 0xd6, 0xbc, 0x57, 0x48, 0xee, 0x0f, 0x59, 0xea, 0x43, + 0x05, 0x96, 0x9b, 0x14, 0x41, 0xe6, 0x88, 0xe8, 0x13, 0x28, 0xb1, 0x5f, 0x34, 0x96, 0x90, 0xc6, + 0xf2, 0xb1, 0x80, 0xcc, 0x87, 0x63, 0xd3, 0xda, 0xe0, 0x88, 0x72, 0xd1, 0x89, 0x7c, 0xa3, 0x0f, + 0xe1, 0x0d, 0xd3, 0x52, 0x3a, 0xb8, 0x63, 0xb1, 0x25, 0x9e, 0x48, 0x19, 0x5d, 0x51, 0x5d, 0x85, + 0x57, 0x9a, 0x79, 0x7c, 0xcf, 0x9b, 0xd6, 0x26, 0xcd, 0x22, 0xf3, 0x1c, 0x2b, 0x2e, 0xa3, 0x2b, + 0x95, 0x61, 0x8a, 0xfd, 0x42, 0x39, 0x98, 0x7c, 0xd6, 0xaa, 0xd7, 0x64, 0x71, 0x02, 0xcd, 0x40, + 0xf6, 0x89, 0xfc, 0x6c, 0x53, 0x69, 0x3e, 0xdf, 0x10, 0x05, 0x94, 0x87, 0x69, 0xf9, 0xd9, 0xb3, + 0x96, 0xb2, 0xfe, 0x42, 0x4c, 0x49, 0x77, 0xa1, 0x18, 0xad, 0x11, 0x02, 0x98, 0x92, 0x6b, 0x9b, + 0xcf, 0xe8, 0xd3, 0x1b, 0x39, 0x98, 0xdc, 0x78, 0x56, 0x5d, 0xd9, 0x10, 0x05, 0xe9, 0x4f, 0x05, + 0x98, 0xa9, 0xb0, 0xb7, 0x64, 0x98, 0x5f, 0xcf, 0xfb, 0x7d, 0xfe, 0x37, 0xd7, 0xe2, 0xb7, 0x10, + 0x86, 0xf9, 0xdd, 0x64, 0xb9, 0xf4, 0xf2, 0x62, 0xce, 0x96, 0x86, 0x1b, 0x8e, 0x74, 0x8f, 0xd5, + 0x73, 0xf4, 0xf5, 0xd0, 0x50, 0x13, 0x44, 0xd5, 0xe3, 0xad, 0xc2, 0x6b, 0x32, 0xdc, 0xdd, 0xb7, + 0xaf, 0x1b, 0x3c, 0x59, 0xa4, 0x46, 0xc1, 0xef, 0x65, 0xbe, 0xf7, 0x83, 0xa5, 0x09, 0xe9, 0xcf, + 0x32, 0x50, 0xa8, 0x84, 0xdf, 0xcd, 0x41, 0x8d, 0xbe, 0xc6, 0xde, 0x8d, 0x55, 0x4b, 0x42, 0x18, + 0xe5, 0x11, 0x2f, 0x92, 0xe5, 0x82, 0x47, 0x7a, 0x58, 0xdb, 0x6f, 0x8c, 0x70, 0x5d, 0x0a, 0x37, + 0x3e, 0x40, 0x5c, 0xf8, 0xd7, 0x69, 0x5f, 0x5f, 0x29, 0xc3, 0x24, 0x8b, 0xa9, 0x16, 0x06, 0xae, + 0x7b, 0xa1, 0x2b, 0x15, 0x31, 0xd7, 0x48, 0xba, 0xcc, 0xb2, 0x11, 0xfd, 0xa6, 0xf5, 0x5a, 0x3e, + 0x4f, 0x81, 0x1c, 0xbc, 0xf8, 0x5b, 0xbf, 0x3d, 0x76, 0xb9, 0xf1, 0xff, 0x43, 0x0f, 0x63, 0x52, + 0x1e, 0xfa, 0x16, 0x94, 0x34, 0xab, 0xdd, 0x66, 0x5a, 0x3e, 0x5b, 0xd9, 0x06, 0xaf, 0x3b, 0xa3, + 0x55, 0xe0, 0xcf, 0x3b, 0x97, 0xfd, 0x67, 0x9e, 0xcb, 0x32, 0x7f, 0xe6, 0x39, 0x14, 0xfe, 0x55, + 0xf4, 0x89, 0xb1, 0x05, 0xb1, 0x2f, 0x12, 0x6d, 0xfa, 0x75, 0x22, 0xd1, 0x58, 0xfc, 0x1e, 0x1f, + 0x79, 0x3f, 0x11, 0xb8, 0x1f, 0xf0, 0x86, 0x65, 0x1d, 0xf6, 0x7c, 0xef, 0xca, 0x85, 0xf0, 0x55, + 0xc5, 0x41, 0x90, 0x0c, 0x8d, 0x19, 0x8d, 0x53, 0xa5, 0x52, 0x9f, 0x4f, 0x95, 0xba, 0x09, 0x33, + 0x5d, 0x1b, 0xef, 0x61, 0x57, 0x3b, 0x50, 0xcc, 0x5e, 0x87, 0x07, 0xcc, 0xe6, 0x3d, 0xd8, 0x56, + 0xaf, 0x83, 0xee, 0x83, 0xe8, 0x67, 0xe1, 0x3b, 0x2f, 0xde, 0x3d, 0x99, 0x1e, 0x9c, 0xef, 0xd3, + 0x48, 0xff, 0x4d, 0x80, 0xb9, 0x48, 0x9b, 0xf8, 0x9c, 0x5a, 0x83, 0xbc, 0xee, 0x2b, 0xaf, 0xce, + 0xbc, 0x70, 0xc1, 0xf0, 0xa9, 0x30, 0x32, 0x52, 0xe0, 0x8a, 0x57, 0x2c, 0x7d, 0xd8, 0x26, 0x20, + 0x9b, 0xba, 0x20, 0xd9, 0xcb, 0x01, 0x9d, 0xd5, 0x50, 0x01, 0xfe, 0x24, 0x4b, 0x8f, 0x35, 0xc9, + 0xa4, 0xff, 0x21, 0x80, 0x48, 0x0b, 0x78, 0x82, 0xb1, 0x9e, 0x88, 0xc8, 0xf4, 0xe2, 0x14, 0x53, + 0xe3, 0x87, 0xc4, 0x46, 0x1e, 0xe3, 0x4a, 0xf7, 0x3d, 0xc6, 0x15, 0x27, 0x3f, 0x33, 0x9f, 0x53, + 0x7e, 0x4a, 0x3f, 0x10, 0xa0, 0xe8, 0x37, 0x9b, 0xbd, 0xc2, 0x3b, 0xe2, 0x9a, 0xed, 0xd7, 0x7b, + 0x69, 0xd6, 0xbb, 0x0e, 0x6c, 0xac, 0x87, 0x81, 0xc3, 0xd7, 0x81, 0xb1, 0x17, 0x52, 0xff, 0x86, + 0x37, 0x1c, 0x49, 0x15, 0xab, 0xc1, 0x3d, 0x4c, 0xaf, 0x11, 0x72, 0xfc, 0x45, 0x38, 0x12, 0x3f, + 0x09, 0x31, 0x90, 0x8e, 0x28, 0xc2, 0xa5, 0xb1, 0xe4, 0xbb, 0xc7, 0x25, 0x36, 0x00, 0xff, 0x20, + 0xdc, 0x13, 0xec, 0x1e, 0x8f, 0xc7, 0x90, 0x3e, 0x52, 0xdb, 0xa3, 0xdc, 0x65, 0x23, 0x3d, 0x27, + 0x93, 0xdc, 0xe8, 0x49, 0xe4, 0xfa, 0xaa, 0xd4, 0xf0, 0x5d, 0xd9, 0x41, 0x96, 0x46, 0xae, 0xb9, + 0xfa, 0x6a, 0x74, 0x02, 0x8d, 0x2c, 0x3e, 0x3c, 0x93, 0xde, 0xcb, 0xfc, 0xe8, 0x07, 0x4b, 0x82, + 0xf4, 0x01, 0x20, 0xa2, 0xeb, 0xb8, 0xcf, 0x7b, 0x96, 0x1d, 0x5c, 0x05, 0xd6, 0x1f, 0x8f, 0x38, + 0x19, 0x1f, 0x8f, 0x28, 0x5d, 0x86, 0xb9, 0x08, 0x36, 0x93, 0x40, 0xd2, 0x57, 0xe1, 0xda, 0x53, + 0xcb, 0x71, 0x8c, 0x6e, 0xb3, 0xb7, 0xcb, 0xa6, 0x3a, 0x59, 0xaf, 0x7c, 0x99, 0x9b, 0xed, 0xd2, + 0x2d, 0x3f, 0x93, 0xc9, 0xa6, 0x9c, 0xec, 0x7f, 0x4b, 0xbf, 0x27, 0xc0, 0xd5, 0x41, 0x4c, 0xc6, + 0xe5, 0xb8, 0x1b, 0x12, 0xa6, 0x35, 0x2b, 0xb8, 0xa9, 0xf6, 0xfc, 0xd1, 0xea, 0x65, 0x27, 0x56, + 0x0d, 0x2f, 0x53, 0xe9, 0xa8, 0x54, 0x26, 0xf1, 0x8b, 0x53, 0x8a, 0x1c, 0xbc, 0xc9, 0xa0, 0x81, + 0x78, 0xca, 0x8c, 0x27, 0x9e, 0xfe, 0xa7, 0x00, 0xb3, 0x2d, 0x6c, 0xaa, 0xa6, 0x4b, 0xe4, 0x7e, + 0xaf, 0xc3, 0x6e, 0x95, 0x28, 0x41, 0xda, 0x56, 0x7a, 0xb4, 0xea, 0x82, 0x9c, 0xb2, 0x77, 0xd0, + 0x2d, 0x28, 0xd0, 0xb5, 0x25, 0xa4, 0xab, 0x09, 0xf7, 0x32, 0x32, 0x8d, 0xae, 0x91, 0x3d, 0x45, + 0xec, 0x4d, 0x00, 0x9a, 0x89, 0x59, 0x97, 0x69, 0x9a, 0x23, 0x47, 0x20, 0xbe, 0x6d, 0x49, 0xa3, + 0x15, 0x03, 0x22, 0x2c, 0x20, 0xa9, 0x40, 0xa1, 0x3e, 0x95, 0x25, 0xc8, 0xb3, 0x6c, 0x8c, 0xcc, + 0x24, 0xcd, 0x03, 0x14, 0xc4, 0xe8, 0x3c, 0x81, 0x4b, 0xce, 0xcb, 0xb6, 0xd2, 0xb5, 0x74, 0x47, + 0xd1, 0xba, 0x3d, 0x1e, 0xe9, 0xc1, 0x1e, 0xf6, 0x16, 0x2a, 0x97, 0xcf, 0x4e, 0x97, 0x66, 0x9b, + 0xcf, 0x37, 0xb6, 0x2d, 0xdd, 0xa9, 0x6e, 0xef, 0xb0, 0x38, 0x0f, 0x47, 0x9e, 0x75, 0x5e, 0xb6, + 0x29, 0xa8, 0xdb, 0xe3, 0x20, 0xe9, 0xb3, 0x14, 0x20, 0x7a, 0x33, 0x52, 0x85, 0x5e, 0x2d, 0xe4, + 0x75, 0xb7, 0x05, 0x8b, 0x5a, 0xc0, 0x0a, 0xc5, 0x31, 0x4c, 0xa2, 0xdf, 0xab, 0x8e, 0xeb, 0xd5, + 0x9b, 0xcf, 0x9b, 0xd8, 0x0b, 0x5d, 0xfb, 0x39, 0xc9, 0xfb, 0xf1, 0x7a, 0x88, 0x62, 0x93, 0x10, + 0xdc, 0x50, 0x1d, 0xbf, 0xc0, 0xfb, 0x90, 0x73, 0x29, 0x9e, 0x77, 0xef, 0x54, 0xa6, 0x32, 0x73, + 0x76, 0xba, 0x94, 0x65, 0xc4, 0x1a, 0xab, 0x72, 0x96, 0x25, 0x37, 0x74, 0xb4, 0x0c, 0x79, 0xc3, + 0x74, 0x5c, 0x95, 0x54, 0x89, 0x6f, 0xe6, 0x14, 0xd8, 0xc5, 0x00, 0x0d, 0x0e, 0x6e, 0xac, 0xca, + 0xe0, 0x65, 0xa1, 0xf1, 0xb7, 0x45, 0x1f, 0x81, 0x9d, 0xce, 0xd0, 0x60, 0x71, 0xb9, 0xe0, 0x41, + 0xd9, 0xcd, 0x88, 0x4d, 0x98, 0x8b, 0x70, 0x82, 0x2f, 0xcc, 0x1f, 0x44, 0x05, 0x4e, 0x58, 0x3b, + 0xa5, 0x70, 0x87, 0x1e, 0x38, 0x6b, 0x96, 0xce, 0x67, 0x6a, 0x54, 0xf0, 0xb4, 0xa0, 0xb4, 0x66, + 0x19, 0x26, 0xb1, 0xcb, 0xbd, 0xa6, 0xae, 0x40, 0x71, 0xd7, 0x30, 0x55, 0xfb, 0x44, 0xf1, 0x02, + 0x5b, 0x84, 0xf3, 0x02, 0x5b, 0xe4, 0x02, 0xc3, 0xe0, 0x9f, 0xd2, 0x8f, 0x05, 0x10, 0x03, 0xb2, + 0xbc, 0xa2, 0x5f, 0x02, 0xd0, 0xda, 0x3d, 0xc7, 0xc5, 0xb6, 0x27, 0x00, 0x66, 0x58, 0x00, 0x6d, + 0x95, 0x41, 0x1b, 0xab, 0x72, 0x8e, 0x67, 0x68, 0xe8, 0xe8, 0x56, 0xf4, 0x9a, 0xaf, 0xc9, 0x0a, + 0x9c, 0x0d, 0x5c, 0xee, 0x45, 0x24, 0x8a, 0xe3, 0x5a, 0xb6, 0xcf, 0x67, 0x2e, 0x51, 0xbc, 0x0b, + 0x10, 0xe9, 0x45, 0x3f, 0x98, 0xc6, 0xc8, 0x17, 0x89, 0x7a, 0x7b, 0x84, 0xfd, 0x26, 0x65, 0xce, + 0x6f, 0x12, 0xc3, 0xf0, 0x9a, 0xf4, 0x3d, 0x01, 0x80, 0x8c, 0x19, 0xd7, 0x56, 0x89, 0x80, 0xfc, + 0x45, 0xc8, 0x84, 0x5e, 0x29, 0x92, 0xe2, 0x1d, 0x3a, 0x78, 0xe6, 0x32, 0xd5, 0xdb, 0x68, 0x7e, + 0x4f, 0xde, 0xa4, 0x02, 0x79, 0xe3, 0xdf, 0x22, 0xc2, 0x64, 0x05, 0xbf, 0x9a, 0xf6, 0x2d, 0xc8, + 0xd0, 0xe7, 0x0d, 0x66, 0x20, 0x2b, 0xd7, 0x9e, 0xef, 0x34, 0xe4, 0xda, 0xaa, 0x38, 0x81, 0x8a, + 0x00, 0xdb, 0xf2, 0xb3, 0x7a, 0xa3, 0xd2, 0x68, 0xd5, 0x56, 0x45, 0xe1, 0xbd, 0xcc, 0x9f, 0x10, + 0x69, 0xfb, 0x99, 0x40, 0x5f, 0xae, 0xe7, 0xa5, 0x39, 0x55, 0xcb, 0xfc, 0x76, 0xcf, 0xa4, 0xda, + 0x3c, 0xd1, 0x0d, 0xcd, 0x5e, 0x27, 0x38, 0x00, 0x67, 0x06, 0x78, 0xde, 0x24, 0x62, 0x95, 0x1f, + 0x63, 0xd7, 0x20, 0xaf, 0x05, 0xc8, 0x5c, 0x03, 0x7b, 0x73, 0x64, 0x83, 0x3c, 0x9d, 0x2e, 0x84, + 0xc7, 0xab, 0xf2, 0x2b, 0x50, 0xa2, 0x83, 0x75, 0x9b, 0xa8, 0x65, 0x36, 0x36, 0x35, 0xdc, 0x4f, + 0x5f, 0xf8, 0x5c, 0xf4, 0xff, 0x6b, 0x1a, 0x80, 0x2c, 0xec, 0xfc, 0x3a, 0xba, 0x3b, 0x50, 0x62, + 0x2b, 0x4a, 0xc7, 0x30, 0xb9, 0x28, 0x62, 0x77, 0xd0, 0x14, 0x28, 0x78, 0xd3, 0x30, 0x99, 0x34, + 0x0a, 0xf2, 0xa9, 0xc7, 0x3c, 0x5f, 0x2a, 0x9c, 0x4f, 0x3d, 0x66, 0xf9, 0x6e, 0xc0, 0xd4, 0xbe, + 0xa6, 0xb8, 0x6e, 0x9b, 0x8f, 0xa6, 0xdc, 0xd9, 0xe9, 0xd2, 0xe4, 0xd3, 0x6a, 0xab, 0xb5, 0x21, + 0x4f, 0xee, 0x6b, 0x2d, 0xb7, 0x4d, 0x18, 0xba, 0xdf, 0xb6, 0x76, 0xd5, 0x36, 0xdd, 0x8b, 0xf2, + 0x6e, 0x9b, 0xcf, 0x33, 0x18, 0x51, 0xd7, 0x9d, 0x01, 0x9e, 0x4f, 0x0e, 0xf2, 0xfc, 0x4d, 0x00, + 0x92, 0x85, 0xbe, 0xa9, 0xc3, 0x64, 0xe2, 0xa4, 0x9c, 0x33, 0x7b, 0x1d, 0xfa, 0x40, 0x8e, 0x83, + 0x9e, 0x47, 0x59, 0xc6, 0x0c, 0x95, 0xfb, 0x23, 0x59, 0x16, 0xee, 0xf5, 0x18, 0xf6, 0xa1, 0x6f, + 0xc2, 0x2c, 0x7b, 0xc1, 0x27, 0x4c, 0x38, 0xfb, 0x7a, 0x84, 0x45, 0x4a, 0x29, 0x94, 0x05, 0xed, + 0x00, 0x0b, 0x58, 0x55, 0xba, 0x7e, 0xbf, 0x3b, 0xf3, 0xb9, 0xa1, 0xba, 0x7c, 0xdf, 0x10, 0xf1, + 0xc8, 0xb6, 0xa3, 0x60, 0xaf, 0xcf, 0x7f, 0x4d, 0x80, 0x52, 0xd0, 0xe7, 0xec, 0x32, 0x9e, 0xd7, + 0x50, 0xff, 0xde, 0x87, 0x29, 0x8d, 0xdd, 0x58, 0x38, 0x3c, 0x70, 0x27, 0x28, 0xc6, 0x7b, 0xa2, + 0x84, 0xa1, 0x48, 0x1b, 0x70, 0xf9, 0x29, 0x76, 0x83, 0x64, 0x3f, 0x50, 0xec, 0x31, 0x4c, 0x32, + 0x43, 0x55, 0x18, 0xe7, 0xed, 0x48, 0x96, 0x57, 0xfa, 0x17, 0x02, 0x5c, 0xe9, 0x27, 0xc7, 0x85, + 0xe4, 0x36, 0x4c, 0xb3, 0xb7, 0x62, 0xbd, 0x99, 0xf8, 0x4e, 0xbc, 0x7f, 0x5a, 0x0c, 0x6e, 0xf4, + 0xc9, 0x59, 0x8f, 0xcc, 0xc2, 0x8e, 0xff, 0xe4, 0xec, 0x3a, 0xcc, 0xd0, 0x9d, 0x68, 0xd6, 0xa6, + 0x51, 0x36, 0x5c, 0x1f, 0xbb, 0xbd, 0x01, 0xe5, 0x04, 0x85, 0x4a, 0xff, 0x5c, 0x80, 0x79, 0xf6, + 0x1c, 0x45, 0x0c, 0x57, 0xbe, 0x05, 0x57, 0xc2, 0x25, 0x29, 0xae, 0xe5, 0xbd, 0x67, 0x71, 0xd1, + 0x32, 0xe7, 0x42, 0x65, 0xb6, 0x2c, 0xfe, 0xfe, 0x45, 0x0d, 0x4a, 0x94, 0x91, 0x84, 0x2e, 0xf7, + 0x62, 0x4b, 0x8d, 0xc3, 0xfe, 0x02, 0xc5, 0x6a, 0x59, 0xcc, 0x2b, 0x4d, 0xba, 0x0e, 0xd7, 0x62, + 0x5a, 0xc0, 0xb5, 0xcd, 0xdf, 0x15, 0xa0, 0x54, 0x65, 0x8a, 0x9d, 0xaf, 0x2c, 0x8e, 0x30, 0x8e, + 0x56, 0x21, 0xeb, 0x1e, 0x9b, 0x4a, 0x07, 0xfb, 0x6f, 0xb4, 0x5e, 0xe0, 0x46, 0xfa, 0x69, 0x97, + 0x7d, 0xd2, 0x67, 0xff, 0xf9, 0x91, 0x02, 0xd7, 0xbc, 0xc7, 0x3a, 0x73, 0xf0, 0x91, 0xd8, 0xbe, + 0xc4, 0x83, 0x26, 0x51, 0xa0, 0x07, 0x76, 0x0e, 0xc8, 0xea, 0x11, 0x7a, 0x7c, 0x77, 0x82, 0x3e, + 0xc9, 0x5b, 0x5b, 0x59, 0x55, 0x76, 0xb6, 0xaa, 0xcf, 0x36, 0x37, 0x1b, 0x2d, 0xba, 0xa6, 0x20, + 0x11, 0x66, 0x22, 0x4f, 0xf7, 0xa6, 0x16, 0x32, 0x9f, 0xfd, 0x70, 0x71, 0xe2, 0xc1, 0xdb, 0x50, + 0x88, 0x1c, 0xc4, 0xa0, 0x12, 0xe4, 0x37, 0x6a, 0x2b, 0xcd, 0x5a, 0xfd, 0xd9, 0xc6, 0x2a, 0xdd, + 0xb8, 0xcc, 0xc3, 0xf4, 0x56, 0x6d, 0x45, 0xae, 0x35, 0x5b, 0xa2, 0xf0, 0xe0, 0x17, 0x01, 0x82, + 0x47, 0xc3, 0x49, 0xd1, 0xeb, 0xb5, 0x8f, 0x95, 0x17, 0x2b, 0x1b, 0x3b, 0xb5, 0xa6, 0x38, 0x81, + 0x10, 0x14, 0x2b, 0x2b, 0xad, 0x6a, 0x5d, 0x91, 0x6b, 0xcd, 0xed, 0x67, 0x5b, 0xcd, 0x9a, 0x28, + 0xf0, 0x62, 0x56, 0x61, 0x26, 0x7c, 0x2d, 0x2f, 0x9a, 0x83, 0x52, 0xb5, 0x5e, 0xab, 0xae, 0x2b, + 0x2f, 0x1a, 0x2b, 0xca, 0xf3, 0x9d, 0xda, 0x4e, 0x8d, 0xad, 0x83, 0x0c, 0xf8, 0x64, 0x67, 0x63, + 0x43, 0x14, 0x48, 0x55, 0xd8, 0x37, 0x7d, 0x15, 0x58, 0x4c, 0x3d, 0xd8, 0x84, 0x7c, 0xe8, 0xb9, + 0x20, 0x52, 0xdc, 0xf6, 0x4e, 0xb3, 0xae, 0xb4, 0x1a, 0x9b, 0xb5, 0x66, 0x6b, 0x65, 0x73, 0x9b, + 0xaf, 0xa5, 0x04, 0xb6, 0x52, 0x79, 0x26, 0xb7, 0x44, 0xc1, 0xff, 0x6e, 0x3d, 0xdb, 0xa9, 0xd6, + 0xbd, 0x56, 0x4b, 0x99, 0x6c, 0x5a, 0x4c, 0x3f, 0x38, 0x86, 0xab, 0x43, 0x6e, 0xa8, 0x25, 0x8d, + 0xde, 0x31, 0xe9, 0xd3, 0x29, 0xe2, 0x04, 0x2a, 0x40, 0x8e, 0x68, 0x26, 0xf4, 0xd2, 0x27, 0x51, + 0x40, 0x59, 0xc8, 0x1c, 0xb8, 0x6e, 0x57, 0x4c, 0xa1, 0x29, 0x48, 0x39, 0x8f, 0xc5, 0x34, 0xf9, + 0xbf, 0xef, 0x88, 0x19, 0x94, 0x83, 0x49, 0xf5, 0xd3, 0x9e, 0x8d, 0xc5, 0x49, 0xb2, 0xc2, 0xf7, + 0x1c, 0x6c, 0xef, 0x19, 0x6d, 0x2c, 0x4e, 0x13, 0x14, 0xb3, 0xd7, 0x6e, 0x8b, 0x59, 0x29, 0x93, + 0x9d, 0x12, 0xa7, 0x1e, 0xdc, 0x84, 0xd0, 0xa5, 0x80, 0x08, 0x60, 0x6a, 0x43, 0x75, 0xb1, 0xe3, + 0x8a, 0x13, 0x68, 0x1a, 0xd2, 0x2b, 0xed, 0xb6, 0x28, 0x3c, 0xfa, 0xef, 0x53, 0x90, 0xf5, 0x1e, + 0xbd, 0x45, 0x1b, 0x30, 0x49, 0x77, 0x36, 0xd1, 0xd2, 0xf0, 0x3d, 0x4f, 0x3a, 0x47, 0x17, 0x6e, + 0x9c, 0xb7, 0x29, 0x2a, 0x4d, 0xa0, 0x5f, 0x81, 0x7c, 0x68, 0x2f, 0x08, 0x0d, 0x3d, 0xc8, 0x8f, + 0xec, 0x7f, 0x2d, 0xdc, 0x39, 0x2f, 0x9b, 0x4f, 0xff, 0x23, 0xc8, 0xf9, 0x66, 0x24, 0xba, 0x35, + 0xca, 0xc8, 0xf4, 0x68, 0x8f, 0xb6, 0x44, 0xc9, 0x2c, 0x95, 0x26, 0xde, 0x11, 0x90, 0x0d, 0x68, + 0xd0, 0xe2, 0x43, 0x71, 0x91, 0x3d, 0x43, 0x4d, 0xca, 0x85, 0x07, 0x63, 0xe5, 0x0e, 0xca, 0x24, + 0xcc, 0x0a, 0xcc, 0xd6, 0x78, 0x66, 0x0d, 0x18, 0xc5, 0xf1, 0xcc, 0x8a, 0xb1, 0x7e, 0x69, 0x67, + 0x84, 0xf4, 0xff, 0x58, 0xfa, 0x83, 0x96, 0x52, 0x2c, 0xfd, 0x18, 0x33, 0x42, 0x9a, 0x40, 0xcf, + 0x21, 0x43, 0x74, 0x76, 0x14, 0x27, 0x9c, 0xfb, 0x6c, 0x84, 0x85, 0x5b, 0x23, 0xf3, 0xf8, 0x24, + 0xf7, 0xa1, 0x18, 0x5d, 0xab, 0xd0, 0xbd, 0x31, 0x96, 0x33, 0x56, 0xc4, 0xfd, 0xb1, 0x17, 0x3e, + 0x69, 0x02, 0x75, 0x61, 0x76, 0x40, 0x94, 0xa3, 0x38, 0x1f, 0xaf, 0x61, 0x4b, 0xd6, 0xc2, 0x97, + 0xc6, 0xcb, 0xec, 0x95, 0x58, 0xb9, 0xff, 0xa3, 0x7f, 0xbf, 0x38, 0xf1, 0xa3, 0xb3, 0x45, 0xe1, + 0xc7, 0x67, 0x8b, 0xc2, 0x1f, 0x9d, 0x2d, 0x0a, 0x7f, 0x7c, 0xb6, 0x28, 0x7c, 0xf7, 0xa7, 0x8b, + 0x13, 0x3f, 0xfe, 0xe9, 0xe2, 0xc4, 0x1f, 0xfd, 0x74, 0x71, 0xe2, 0x93, 0x69, 0x4e, 0x66, 0x77, + 0x8a, 0xca, 0xee, 0xc7, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xae, 0x07, 0xf8, 0xe7, 0xce, 0x91, + 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 } // Reference imports to suppress errors if they are not otherwise used. @@ -8403,6 +9045,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 +9170,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 +9201,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 +9234,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 +9377,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 +9437,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{ { @@ -17495,6 +18217,455 @@ func (m *JoinNodeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +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 = encodeVarintApi(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 = encodeVarintApi(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0x12 + } + if m.Type != 0 { + i = encodeVarintApi(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 = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.NumReplicas != 0 { + i = encodeVarintApi(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 = encodeVarintApi(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 = encodeVarintApi(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 = encodeVarintApi(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 = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if m.NumVoters != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.NumVoters)) + i-- + dAtA[i] = 0x30 + } + if m.NumReplicas != 0 { + i = encodeVarintApi(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 = encodeVarintApi(dAtA, i, uint64(m.GCTTL)) + i-- + dAtA[i] = 0x18 + } + if m.RangeMaxBytes != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.RangeMaxBytes)) + i-- + dAtA[i] = 0x10 + } + if m.RangeMinBytes != 0 { + i = encodeVarintApi(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 = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.Span.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(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 = encodeVarintApi(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.Results) > 0 { + for iNdEx := len(m.Results) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Results[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + return len(dAtA) - i, nil +} + +func (m *GetSpanConfigsResponse_Result) 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_Result) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetSpanConfigsResponse_Result) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.SpanConfigs) > 0 { + for iNdEx := len(m.SpanConfigs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SpanConfigs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(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.SpansToDelete) > 0 { + for iNdEx := len(m.SpansToDelete) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SpansToDelete[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.SpanConfigsToUpdate) > 0 { + for iNdEx := len(m.SpanConfigsToUpdate) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SpanConfigsToUpdate[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(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 (m *ContentionEvent) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -17515,12 +18686,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n289, err289 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err289 != nil { - return 0, err289 + n291, err291 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err291 != nil { + return 0, err291 } - i -= n289 - i = encodeVarintApi(dAtA, i, uint64(n289)) + i -= n291 + i = encodeVarintApi(dAtA, i, uint64(n291)) i-- dAtA[i] = 0x1a { @@ -21028,6 +22199,192 @@ func (m *JoinNodeResponse) Size() (n int) { return n } +func (m *Constraint) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovApi(uint64(m.Type)) + } + l = len(m.Key) + if l > 0 { + n += 1 + l + sovApi(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovApi(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 + sovApi(uint64(m.NumReplicas)) + } + if len(m.Constraints) > 0 { + for _, e := range m.Constraints { + l = e.Size() + n += 1 + l + sovApi(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 + sovApi(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 + sovApi(uint64(m.RangeMinBytes)) + } + if m.RangeMaxBytes != 0 { + n += 1 + sovApi(uint64(m.RangeMaxBytes)) + } + if m.GCTTL != 0 { + n += 1 + sovApi(uint64(m.GCTTL)) + } + if m.GlobalReads { + n += 2 + } + if m.NumReplicas != 0 { + n += 1 + sovApi(uint64(m.NumReplicas)) + } + if m.NumVoters != 0 { + n += 1 + sovApi(uint64(m.NumVoters)) + } + if len(m.Constraints) > 0 { + for _, e := range m.Constraints { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + if len(m.VoterConstraints) > 0 { + for _, e := range m.VoterConstraints { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + if len(m.LeasePreferences) > 0 { + for _, e := range m.LeasePreferences { + l = e.Size() + n += 1 + l + sovApi(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 + sovApi(uint64(l)) + l = m.Config.Size() + n += 1 + l + sovApi(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 + sovApi(uint64(l)) + } + } + return n +} + +func (m *GetSpanConfigsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Results) > 0 { + for _, e := range m.Results { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + return n +} + +func (m *GetSpanConfigsResponse_Result) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.SpanConfigs) > 0 { + for _, e := range m.SpanConfigs { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + return n +} + +func (m *UpdateSpanConfigsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.SpanConfigsToUpdate) > 0 { + for _, e := range m.SpanConfigsToUpdate { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + if len(m.SpansToDelete) > 0 { + for _, e := range m.SpansToDelete { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + return n +} + +func (m *UpdateSpanConfigsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *ContentionEvent) Size() (n int) { if m == nil { return 0 @@ -42459,6 +43816,1129 @@ func (m *JoinNodeResponse) Unmarshal(dAtA []byte) error { } return nil } +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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrInvalidLengthApi + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrInvalidLengthApi + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Config.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + 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 := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Results = append(m.Results, GetSpanConfigsResponse_Result{}) + if err := m.Results[len(m.Results)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetSpanConfigsResponse_Result) 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 ErrIntOverflowApi + } + 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: Result: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Result: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SpanConfigs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SpanConfigs = append(m.SpanConfigs, SpanConfigEntry{}) + if err := m.SpanConfigs[len(m.SpanConfigs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 SpanConfigsToUpdate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SpanConfigsToUpdate = append(m.SpanConfigsToUpdate, SpanConfigEntry{}) + if err := m.SpanConfigsToUpdate[len(m.SpanConfigsToUpdate)-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 SpansToDelete", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SpansToDelete = append(m.SpansToDelete, Span{}) + if err := m.SpansToDelete[len(m.SpansToDelete)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + 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 ErrIntOverflowApi + } + 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 := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ContentionEvent) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 2218207a5dcc..3ecfafbf7db6 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2449,8 +2449,7 @@ message GossipSubscriptionEvent { string key = 1; Value content = 2 [(gogoproto.nullable) = false]; // Which pattern does this gossip information match? - string pattern_matched = 3; - // If non-nil, the other fields will be empty and this will be the final event + string pattern_matched = 3; // If non-nil, the other fields will be empty and this will be the final event // send on the stream before it is terminated. kv.kvpb.Error error = 4; } @@ -2512,6 +2511,161 @@ message JoinNodeResponse { roachpb.Version active_version = 4; } +// 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. + repeated Span spans = 1 [(gogoproto.nullable) = false]; +}; + +// GetSpanConfigsResponse lists out the span configurations over the specified +// keyspans. +message GetSpanConfigsResponse { + message Result { + repeated SpanConfigEntry span_configs = 1 [(gogoproto.nullable) = false]; + } + + // Results capture the span configurations over the specified spans. There's + // one result for Span in the matching GetSpanConfigsRequeust, in the + // same order. A result will be present for each Span even if there are no + // configs to return. + repeated Result results = 2 [(gogoproto.nullable) = false]; +}; + +// UpdateSpanConfigsRequest is used to update the span configurations over the +// specified keyspans. +message UpdateSpanConfigsRequest { + // SpanConfigsToUpdate lists out the spans we want to update and the configs + // we want to update with. + repeated SpanConfigEntry span_configs_to_update = 1 [(gogoproto.nullable) = false]; + + // SpansToDelete lists out the spans we want to delete span configs for. + repeated Span spans_to_delete = 2 [(gogoproto.nullable) = false]; +}; + +message UpdateSpanConfigsResponse { }; + // Batch and RangeFeed service implemented by nodes for KV API requests. service Internal { rpc Batch (BatchRequest) returns (BatchResponse) {} @@ -2527,6 +2681,15 @@ 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/api_test.go b/pkg/roachpb/api_test.go index 735434af1d8e..15e26d4ff2ee 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -11,6 +11,7 @@ package roachpb import ( + "fmt" "reflect" "testing" @@ -303,3 +304,41 @@ func TestTenantConsumptionAddSub(t *testing.T) { t.Errorf("expected\n%#v\ngot\n%#v", exp, c) } } + +func TestUnnestGetSpanConfigsResponse(t *testing.T) { + entry := func(i, j int) SpanConfigEntry { + return SpanConfigEntry{ + Span: Span{ + Key: Key(fmt.Sprintf("start_%d", i)), + EndKey: Key(fmt.Sprintf("end_%d", j)), + }, + Config: SpanConfig{ + NumReplicas: int32(i), + NumVoters: int32(j), + }, + } + } + + r := GetSpanConfigsResponse{} + const rows, columns = 5, 3 + for i := 0; i < rows; i++ { + var entries []SpanConfigEntry + for j := 0; j < columns; j++ { + entries = append(entries, entry(i, j)) + } + r.Results = append(r.Results, GetSpanConfigsResponse_Result{ + SpanConfigs: entries, + }) + + } + result := r.Unnest() + require.Len(t, result, rows) + + for i := 0; i < rows; i++ { + require.Len(t, result[i], columns) + for j := 0; j < columns; j++ { + expected := entry(i, j) + require.True(t, result[i][j].Equal(expected)) + } + } +} diff --git a/pkg/roachpb/client_test.go b/pkg/roachpb/client_test.go new file mode 100644 index 000000000000..0622762d895c --- /dev/null +++ b/pkg/roachpb/client_test.go @@ -0,0 +1,91 @@ +// 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_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/stretchr/testify/require" +) + +func TestValidateUpdateSpanConfigsRequest(t *testing.T) { + for _, tc := range []struct { + req roachpb.UpdateSpanConfigsRequest + expErr string + }{ + { + req: roachpb.UpdateSpanConfigsRequest{}, + expErr: "", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpansToDelete: []roachpb.Span{ + {Key: roachpb.Key("a")}, // empty end key in delete list + }, + }, + expErr: "invalid span", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpanConfigsToUpdate: []roachpb.SpanConfigEntry{ + { + Span: roachpb.Span{Key: roachpb.Key("a")}, // empty end key in update list + }, + }, + }, + expErr: "invalid span", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpanConfigsToUpdate: []roachpb.SpanConfigEntry{ + { + Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + }, + }, + }, + expErr: "invalid span", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpansToDelete: []roachpb.Span{ + {Key: roachpb.Key("b"), EndKey: roachpb.Key("a")}, // invalid span; end < start + }, + }, + expErr: "invalid span", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpansToDelete: []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", + }, + { + req: roachpb.UpdateSpanConfigsRequest{ + SpanConfigsToUpdate: []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", + }, + } { + require.True(t, testutils.IsError(tc.req.Validate(), tc.expErr)) + } +} 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/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index 6d0fb4be38c3..ca5a5c5a8e82 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -68,6 +68,12 @@ func (a tenantAuthorizer) authorize( case "/cockroach.server.serverpb.Status/Statements": 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) } @@ -203,6 +209,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 { + for _, sp := range args.Spans { + rSpan, err := keys.SpanAddr(sp) + if err != nil { + return authError(err.Error()) + } + tenSpan := tenantPrefix(tenID) + 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.SpanConfigsToUpdate { + if err := validate(entry.Span); err != nil { + return err + } + } + for _, span := range args.SpansToDelete { + 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/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..e52ea8df4004 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -330,6 +330,7 @@ go_test( "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catconstants", diff --git a/pkg/server/node.go b/pkg/server/node.go index 56a89dfc3645..f813f04b87e4 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -33,11 +33,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/security" "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/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/admission" @@ -46,6 +50,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -114,6 +119,9 @@ var ( 10*time.Second, settings.NonNegativeDurationWithMaximum(maxGraphiteInterval), ).WithPublic() + enableSpanConfigs = settings.RegisterBoolSetting( + "server.experimental_span_config_rpcs.enabled", + "enable the use of the span configuration RPCs", false) ) type nodeMetrics struct { @@ -1426,3 +1434,160 @@ type emptyMetricStruct struct{} var _ metric.Struct = emptyMetricStruct{} func (emptyMetricStruct) MetricStruct() {} + +var _ spanconfig.KVAccessor = &Node{} + +// GetSpanConfigEntriesFor implements the spanconfig.KVAccessor interface. It's a +// convenience wrapper around the RPC implementation. +func (n *Node) GetSpanConfigEntriesFor( + ctx context.Context, spans []roachpb.Span, +) ([][]roachpb.SpanConfigEntry, error) { + resp, err := n.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{Spans: spans}) + if err != nil { + return nil, err + } + + return resp.Unnest(), nil +} + +// UpdateSpanConfigEntries implements the spanconfig.KVAccessor +// interface. It's a convenience wrapper around the RPC implementation. +func (n *Node) UpdateSpanConfigEntries( + ctx context.Context, update []roachpb.SpanConfigEntry, delete []roachpb.Span, +) error { + _, err := n.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{ + SpanConfigsToUpdate: update, + SpansToDelete: delete, + }) + return err +} + +// GetSpanConfigs implements the roachpb.InternalServer interface. +func (n *Node) GetSpanConfigs( + ctx context.Context, req *roachpb.GetSpanConfigsRequest, +) (resp *roachpb.GetSpanConfigsResponse, retErr error) { + if enableSpanConfigs.Get(&n.storeCfg.Settings.SV) { + return nil, errors.New("use of span configs disabled") + } + + for _, sp := range req.Spans { + if !sp.Valid() || len(sp.EndKey) == 0 { + return nil, errors.AssertionFailedf("invalid span: %s", sp) + } + } + + if err := n.storeCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + resp = &roachpb.GetSpanConfigsResponse{} + + var res []roachpb.SpanConfigEntry + for _, sp := range req.Spans { + it, err := n.sqlExec.QueryIteratorEx(ctx, "get-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + `SELECT start_key, end_key, config FROM system.span_configurations + WHERE $1 < end_key AND start_key < $2`, + sp.Key, sp.EndKey, + ) + if err != nil { + return 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 scfg roachpb.SpanConfig + if err := protoutil.Unmarshal(([]byte)(*row[2].(*tree.DBytes)), &scfg); err != nil { + return err + } + + res = append(res, roachpb.SpanConfigEntry{ + Span: span, + Config: scfg, + }) + } + if err != nil { + return err + } + + resp.Results = append(resp.Results, roachpb.GetSpanConfigsResponse_Result{ + SpanConfigs: res, + }) + } + + return nil + }); err != nil { + return nil, err + } + return resp, nil +} + +// UpdateSpanConfigs implements the roachpb.InternalServer interface. +func (n *Node) UpdateSpanConfigs( + ctx context.Context, req *roachpb.UpdateSpanConfigsRequest, +) (*roachpb.UpdateSpanConfigsResponse, error) { + if err := req.Validate(); err != nil { + return nil, err + } + + // TODO(irfansharif): Once we have the reconciliation job populating this + // table, we could write a datadriven test asserting on its contents. + if err := n.storeCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + for _, span := range req.SpansToDelete { + n, err := n.sqlExec.ExecEx(ctx, "delete-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "DELETE FROM system.span_configurations WHERE start_key = $1 AND end_key = $2", + span.Key, span.EndKey, + ) + if err != nil { + return err + } + if n != 1 { + return errors.AssertionFailedf("expected to delete single row, deleted %d (span=%s)", n, span) + } + } + + for _, scfg := range req.SpanConfigsToUpdate { + buf, err := protoutil.Marshal(&scfg.Config) + if err != nil { + return err + } + if _, err := n.sqlExec.ExecEx(ctx, "update-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "UPSERT INTO system.span_configurations (start_key, end_key, config) VALUES ($1, $2, $3)", + scfg.Span.Key, scfg.Span.EndKey, buf, + ); err != nil { + return err + } + + } + + for _, scfg := range req.SpanConfigsToUpdate { + // Ensure that there are no overlapping spans. + datums, err := n.sqlExec.QueryRowEx(ctx, "validate-span-cfgs", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + `SELECT count(*) FROM system.span_configurations WHERE $1 < end_key AND start_key < $2`, + scfg.Span.Key, scfg.Span.EndKey, + ) + if err != nil { + return err + } + if count := int64(tree.MustBeDInt(datums[0])); count != 1 { + return errors.AssertionFailedf("expected to find single row containing %s, found %d", scfg.Span, count) + } + } + + return nil + }); err != nil { + return nil, err + } + return &roachpb.UpdateSpanConfigsResponse{}, nil +} diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 1a40bd03badb..ec738103c3e9 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -649,3 +650,145 @@ func TestNodeSendUnknownBatchRequest(t *testing.T) { t.Fatalf("expected unsupported request, not %v", br.Error) } } + +func TestSpanConfigAccessorBasics(t *testing.T) { + defer leaktest.AfterTest(t) + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + // Disable the reconciliation process; this test wants sole + // access to the server's SpanConfigAccessor. + ManagerDisableJobCreation: true, + }, + }, + }) + defer s.Stopper().Stop(context.Background()) + + ts := s.(*TestServer) + accessor := ts.SpanConfigAccessor().(spanconfig.KVAccessor) + + 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 + } + + everythingSpan := roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey} + + { // With an empty slate. + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, 1) + require.Empty(t, entries[0]) + + require.True(t, testutils.IsError( + accessor.UpdateSpanConfigEntries(ctx, nil, []roachpb.Span{everythingSpan} /* delete */), + "expected to delete single row")) + } + + // Write and delete a batch of entries. + spans := []roachpb.Span{ + span("a", "b"), + span("b", "c"), + span("c", "d"), + span("d", "e"), + } + entries := toEntries(spans) + + { // Verify that writing and reading a single entry behaves as expected. + entry := entries[0] + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, []roachpb.SpanConfigEntry{entry}, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], 1) + got := entriesList[0][0] + require.True(t, got.Equal(entry)) + + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, nil, []roachpb.Span{entry.Span} /* delete */)) + entriesList, err = accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Empty(t, entriesList[0]) + } + + { // Verify that adding all entries does in fact add all entries. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } + + { // Verify that updating entries (including noops) show up as such. + for i := range entries { + if i%2 == 0 { + continue + } + entries[i].Config.RangeMaxBytes += 100 + } + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } + + { // Verify that deleting entries actually removes them. + const toDelete = 2 + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, nil, spans[:toDelete] /* delete */)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)-toDelete) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[toDelete+i])) + } + + // Attempts to delete non existent spans should error out. + require.NotNil(t, accessor.UpdateSpanConfigEntries(ctx, nil, spans[:toDelete] /* delete */)) + // Attempts to re-write previously deleted spans should go through. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries[:toDelete], nil)) + } + + { // Verify that we're able to re-partition span configs correctly. + spans, prevLast := mergeLastTwo(spans) + entries := toEntries(spans) + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, []roachpb.Span{prevLast})) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } +} diff --git a/pkg/server/server.go b/pkg/server/server.go index 63e19a8a94be..cbaa4e0200fb 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -749,6 +749,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { rpcContext: rpcContext, nodeDescs: g, systemConfigProvider: g, + spanConfigAccessor: node, nodeDialer: nodeDialer, distSender: distSender, db: db, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index ff44307e558d..399efdf06cea 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -219,6 +219,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 @@ -824,6 +827,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/status/recorder_test.go b/pkg/server/status/recorder_test.go index fe0e8ed96ad0..463d9ec867e6 100644 --- a/pkg/server/status/recorder_test.go +++ b/pkg/server/status/recorder_test.go @@ -159,7 +159,7 @@ func TestMetricsRecorder(t *testing.T) { // Generate Metrics Data & Expected Results // ======================================== - // Flatten the four registries into an array for ease of use. + // Unnest the four registries into an array for ease of use. regList := []struct { reg *metric.Registry prefix string diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 0293eb67d755..fc87c46b1ce3 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 1c2e9d009528..b76343f3eca1 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 +} + // SQLServer is part of TestServerInterface. func (ts *TestServer) SQLServer() interface{} { return ts.PGServer().SQLServer diff --git a/pkg/spanconfig/BUILD.bazel b/pkg/spanconfig/BUILD.bazel index be35186fa41d..66c7a9bc899d 100644 --- a/pkg/spanconfig/BUILD.bazel +++ b/pkg/spanconfig/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "spanconfig", @@ -10,6 +10,30 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/roachpb:with-mocks", "//pkg/settings", ], ) + +go_test( + name = "spanconfig_test", + srcs = [ + "main_test.go", + "spanconfig_test.go", + ], + deps = [ + ":spanconfig", + "//pkg/base", + "//pkg/config/zonepb", + "//pkg/keys", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/spanconfig/main_test.go b/pkg/spanconfig/main_test.go new file mode 100644 index 000000000000..61d93570826e --- /dev/null +++ b/pkg/spanconfig/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 spanconfig_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/spanconfig.go b/pkg/spanconfig/spanconfig.go index 864995b1f2da..d946e524c7c3 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -11,8 +11,10 @@ package spanconfig import ( + "context" "time" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" ) @@ -26,19 +28,33 @@ var CheckAndStartReconciliationJobInterval = settings.RegisterDurationSetting( settings.NonNegativeDuration, ) +// KVAccessor mediates access to KV span configurations pertaining to a given +// tenant. +type KVAccessor interface { + // GetSpanConfigEntriesFor retrieves the span configurations over the + // requested 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 updates for the + // new ones. + UpdateSpanConfigEntries(ctx context.Context, update []roachpb.SpanConfigEntry, delete []roachpb.Span) 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/spanconfig_test.go b/pkg/spanconfig/spanconfig_test.go new file mode 100644 index 000000000000..910b9d2cc2e0 --- /dev/null +++ b/pkg/spanconfig/spanconfig_test.go @@ -0,0 +1,167 @@ +// 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 spanconfig_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestKVAccessor(t *testing.T) { + defer leaktest.AfterTest(t) + + ctx := context.Background() + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + // Disable the reconciliation process; this test wants sole + // access to the server's SpanConfigAccessor. + ManagerDisableJobCreation: true, + }, + }, + }) + defer s.Stopper().Stop(context.Background()) + + 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 + } + + ts := s.(*server.TestServer) + everythingSpan := roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey} + accessor := ts.SpanConfigAccessor().(spanconfig.KVAccessor) + + { // With an empty slate. + entries, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entries, 1) + require.Empty(t, entries[0]) + + require.True(t, testutils.IsError( + accessor.UpdateSpanConfigEntries(ctx, nil, []roachpb.Span{everythingSpan} /* delete */), + "expected to delete single row")) + } + + // Write and delete a batch of entries. + spans := []roachpb.Span{ + span("a", "b"), + span("b", "c"), + span("c", "d"), + span("d", "e"), + } + entries := toEntries(spans) + + { // Verify that writing and reading a single entry behaves as expected. + entry := entries[0] + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, []roachpb.SpanConfigEntry{entry}, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], 1) + got := entriesList[0][0] + require.True(t, got.Equal(entry)) + + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, nil, []roachpb.Span{entry.Span} /* delete */)) + entriesList, err = accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Empty(t, entriesList[0]) + } + + { // Verify that adding all entries does in fact add all entries. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } + + { // Verify that updating entries (including noops) show up as such. + for i := range entries { + if i%2 == 0 { + continue + } + entries[i].Config.RangeMaxBytes += 100 + } + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, nil)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } + + { // Verify that deleting entries actually removes them. + const toDelete = 2 + require.Nil(t, accessor.UpdateSpanConfigEntries(ctx, nil, spans[:toDelete] /* delete */)) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)-toDelete) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[toDelete+i])) + } + + // Attempts to delete non existent spans should error out. + require.NotNil(t, accessor.UpdateSpanConfigEntries(ctx, nil, spans[:toDelete] /* delete */)) + // Attempts to re-write previously deleted spans should go through. + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries[:toDelete], nil)) + } + + { // Verify that we're able to re-partition span configs correctly. + spans, prevLast := mergeLastTwo(spans) + entries := toEntries(spans) + require.NoError(t, accessor.UpdateSpanConfigEntries(ctx, entries, []roachpb.Span{prevLast})) + entriesList, err := accessor.GetSpanConfigEntriesFor(ctx, []roachpb.Span{everythingSpan}) + require.Nil(t, err) + require.Len(t, entriesList, 1) + require.Len(t, entriesList[0], len(spans)) + for i, got := range entriesList[0] { + require.True(t, got.Equal(entries[i])) + } + } +} diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index bb77b7cf2ab9..32eaa3cabc82 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -46,6 +46,8 @@ type Manager struct { stopper *stop.Stopper settings *cluster.Settings knobs *spanconfig.TestingKnobs + + spanconfig.KVAccessor } // New constructs a new Manager. @@ -55,18 +57,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, } } 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/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 5054cf298683..0f1834eefbaf 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4822,6 +4822,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/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{}