From 0a95d402b5bbaeba488a33c6a0b7f3dc011847fc Mon Sep 17 00:00:00 2001 From: Azhng Date: Fri, 18 Jun 2021 17:06:37 -0400 Subject: [PATCH 1/2] sql,server: introduce RPC fanout for index usage stats Related issue: #64740 Release note: None --- docs/generated/http/full.md | 44 + pkg/base/testing_knobs.go | 1 + pkg/server/BUILD.bazel | 4 + pkg/server/index_usage_stats.go | 112 ++ pkg/server/index_usage_stats_test.go | 292 ++++ pkg/server/server_sql.go | 4 + pkg/server/serverpb/status.go | 1 + pkg/server/serverpb/status.pb.go | 1239 +++++++++++------ pkg/server/serverpb/status.pb.gw.go | 83 ++ pkg/server/serverpb/status.proto | 19 + pkg/server/tenant_status.go | 11 + pkg/sql/conn_executor.go | 45 +- pkg/sql/exec_util.go | 2 + pkg/sql/idxusage/BUILD.bazel | 2 +- pkg/sql/idxusage/index_usage_stats.go | 53 - pkg/sql/idxusage/local_idx_usage_stats.go | 24 +- .../idxusage/local_index_usage_stats_test.go | 2 +- pkg/sql/idxusage/test_utils.go | 12 +- pkg/sql/opt_exec_factory.go | 8 +- pkg/sql/planner.go | 22 +- 20 files changed, 1451 insertions(+), 529 deletions(-) create mode 100644 pkg/server/index_usage_stats.go create mode 100644 pkg/server/index_usage_stats_test.go delete mode 100644 pkg/sql/idxusage/index_usage_stats.go diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index f62b2dfc45a0..bcdf79bcdf52 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -3195,6 +3195,50 @@ Response object returned by ResetSQLStats. +## IndexUsageStatistics + +`GET /_status/indexusagestatistics` + + + +Support status: [reserved](#support-status) + +#### Request Parameters + + + + +Request object for issuing IndexUsageStatistics request. + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| node_id | [string](#cockroach.server.serverpb.IndexUsageStatisticsRequest-string) | | node_id is the ID of the node where the stats data shall be retrieved from. If this is left empty, the cluster-wide aggregated result will be returned. | [reserved](#support-status) | + + + + + + + +#### Response Parameters + + + + +Response object returned by IndexUsageStatistics. + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| statistics | [cockroach.sql.CollectedIndexUsageStatistics](#cockroach.server.serverpb.IndexUsageStatisticsResponse-cockroach.sql.CollectedIndexUsageStatistics) | repeated | | [reserved](#support-status) | + + + + + + + ## RequestCA `GET /_join/v1/ca` diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index b760e6f4d6e3..b903e8f0ebb1 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -38,4 +38,5 @@ type TestingKnobs struct { JobsTestingKnobs ModuleTestingKnobs BackupRestore ModuleTestingKnobs MigrationManager ModuleTestingKnobs + IndexUsageStatsKnobs ModuleTestingKnobs } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 03e4d9487abe..2dd149acb1b9 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "drain.go", "grpc_server.go", "idle_monitor.go", + "index_usage_stats.go", "init.go", "init_handshake.go", "loopback.go", @@ -123,6 +124,7 @@ go_library( "//pkg/sql/flowinfra", "//pkg/sql/gcjob", "//pkg/sql/gcjob/gcjobnotifier", + "//pkg/sql/idxusage", "//pkg/sql/optionalnodeliveness", "//pkg/sql/parser", "//pkg/sql/pgwire", @@ -259,6 +261,7 @@ go_test( "drain_test.go", "graphite_test.go", "idle_monitor_test.go", + "index_usage_stats_test.go", "init_handshake_test.go", "intent_test.go", "main_test.go", @@ -319,6 +322,7 @@ go_test( "//pkg/sql/catalog/dbdesc", "//pkg/sql/catalog/descpb", "//pkg/sql/execinfrapb", + "//pkg/sql/idxusage", "//pkg/sql/sem/tree", "//pkg/sql/sqlstats", "//pkg/sql/tests", diff --git a/pkg/server/index_usage_stats.go b/pkg/server/index_usage_stats.go new file mode 100644 index 000000000000..8e990bc54f91 --- /dev/null +++ b/pkg/server/index_usage_stats.go @@ -0,0 +1,112 @@ +// 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 server + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" + "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// IndexUsageStatistics is the GRPC handler for serving index usage statistics. +// If the NodeID in the request payload is left empty, the handler will issue +// a cluster-wide RPC fanout to aggregate all index usage statistics from all +// the nodes. If the NodeID is specified, then the handler will handle the +// request either locally (if the NodeID matches the current node's NodeID) or +// forward it to the correct node. +func (s *statusServer) IndexUsageStatistics( + ctx context.Context, req *serverpb.IndexUsageStatisticsRequest, +) (*serverpb.IndexUsageStatisticsResponse, error) { + ctx = propagateGatewayMetadata(ctx) + ctx = s.AnnotateCtx(ctx) + + if _, err := s.privilegeChecker.requireViewActivityPermission(ctx); err != nil { + return nil, err + } + + localReq := &serverpb.IndexUsageStatisticsRequest{ + NodeID: "local", + } + + if len(req.NodeID) > 0 { + requestedNodeID, local, err := s.parseNodeID(req.NodeID) + if err != nil { + return nil, status.Error(codes.InvalidArgument, err.Error()) + } + if local { + statsReader := s.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics() + return indexUsageStatsLocal(statsReader) + } + + statusClient, err := s.dialNode(ctx, requestedNodeID) + if err != nil { + return nil, err + } + + // We issue a localReq instead of the incoming req to other nodes. This is + // to instruct other nodes to only return us their node-local stats and + // do not further propagates the RPC call. + return statusClient.IndexUsageStatistics(ctx, localReq) + } + + dialFn := func(ctx context.Context, nodeID roachpb.NodeID) (interface{}, error) { + client, err := s.dialNode(ctx, nodeID) + return client, err + } + + fetchIndexUsageStats := func(ctx context.Context, client interface{}, _ roachpb.NodeID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + return statusClient.IndexUsageStatistics(ctx, localReq) + } + + resp := &serverpb.IndexUsageStatisticsResponse{} + aggFn := func(_ roachpb.NodeID, nodeResp interface{}) { + stats := nodeResp.(*serverpb.IndexUsageStatisticsResponse) + resp.Statistics = append(resp.Statistics, stats.Statistics...) + } + + var combinedError error + errFn := func(_ roachpb.NodeID, nodeFnError error) { + combinedError = errors.CombineErrors(combinedError, nodeFnError) + } + + // It's unfortunate that we cannot use paginatedIterateNodes here because we + // need to aggregate all stats before returning. Returning a partial result + // yields an incorrect result. + if err := s.iterateNodes(ctx, + fmt.Sprintf("requesting index usage stats for node %s", req.NodeID), + dialFn, fetchIndexUsageStats, aggFn, errFn); err != nil { + return nil, err + } + + return resp, nil +} + +func indexUsageStatsLocal( + idxUsageStats *idxusage.LocalIndexUsageStats, +) (*serverpb.IndexUsageStatisticsResponse, error) { + resp := &serverpb.IndexUsageStatisticsResponse{} + if err := idxUsageStats.ForEach(idxusage.IteratorOptions{}, func(key *roachpb.IndexUsageKey, value *roachpb.IndexUsageStatistics) error { + resp.Statistics = append(resp.Statistics, roachpb.CollectedIndexUsageStatistics{Key: *key, + Stats: *value, + }) + return nil + }); err != nil { + return nil, err + } + return resp, nil +} diff --git a/pkg/server/index_usage_stats_test.go b/pkg/server/index_usage_stats_test.go new file mode 100644 index 000000000000..7502cb1fefaf --- /dev/null +++ b/pkg/server/index_usage_stats_test.go @@ -0,0 +1,292 @@ +// 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 server + +import ( + "context" + gosql "database/sql" + "net/url" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/stretchr/testify/require" +) + +func compareTimeHelper(t *testing.T, expected, actual time.Time, delta time.Duration) { + diff := actual.Sub(expected) + require.True(t, diff < delta, "expected delta %s, but found %s", delta, diff) +} + +func compareStatsHelper( + t *testing.T, expected, actual roachpb.IndexUsageStatistics, delta time.Duration, +) { + compareTimeHelper(t, expected.LastRead, actual.LastRead, delta) + compareTimeHelper(t, expected.LastWrite, actual.LastWrite, delta) + + // We don't perform deep comparison of time.Time. So we set them to a dummy + // value before deep equal. + dummyTime := timeutil.Now() + + expected.LastRead = dummyTime + expected.LastWrite = dummyTime + actual.LastRead = dummyTime + actual.LastWrite = dummyTime + + require.Equal(t, expected, actual) +} + +func createIndexStatsIngestedCallback() ( + func(key roachpb.IndexUsageKey), + chan roachpb.IndexUsageKey, +) { + // Create a buffered channel so the callback is non-blocking. + notify := make(chan roachpb.IndexUsageKey, 100) + + cb := func(key roachpb.IndexUsageKey) { + notify <- key + } + + return cb, notify +} + +func waitForStatsIngestion( + t *testing.T, + notify chan roachpb.IndexUsageKey, + expectedKeys map[roachpb.IndexUsageKey]struct{}, + expectedEventCnt int, + timeout time.Duration, +) { + var timer timeutil.Timer + eventCnt := 0 + + timer.Reset(timeout) + + for eventCnt < expectedEventCnt { + select { + case key := <-notify: + if _, ok := expectedKeys[key]; ok { + eventCnt++ + } + continue + case <-timer.C: + timer.Read = true + t.Fatalf("expected stats ingestion to complete within %s, but it timed out", timeout) + } + } +} + +func TestStatusAPIIndexUsage(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + statsIngestionCb, statsIngestionNotifier := createIndexStatsIngestedCallback() + + testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + IndexUsageStatsKnobs: &idxusage.TestingKnobs{ + OnIndexUsageStatsProcessedCallback: statsIngestionCb, + }, + }, + }, + }) + + ctx := context.Background() + defer testCluster.Stopper().Stop(ctx) + + firstServer := testCluster.Server(0 /* idx */) + firstLocalStatsReader := firstServer.SQLServer().(*sql.Server).GetLocalIndexStatistics() + + expectedStatsIndexA := roachpb.IndexUsageStatistics{ + TotalReadCount: 2, + LastRead: timeutil.Now(), + } + + expectedStatsIndexB := roachpb.IndexUsageStatistics{ + TotalReadCount: 2, + LastRead: timeutil.Now(), + } + + firstPgURL, firstServerConnCleanup := sqlutils.PGUrl( + t, firstServer.ServingSQLAddr(), "CreateConnections" /* prefix */, url.User(security.RootUser)) + defer firstServerConnCleanup() + + firstServerSQLConn, err := gosql.Open("postgres", firstPgURL.String()) + require.NoError(t, err) + + defer func() { + err := firstServerSQLConn.Close() + require.NoError(t, err) + }() + + // Create table on the first node. + _, err = firstServerSQLConn.Exec("CREATE TABLE t (k INT PRIMARY KEY, a INT, b INT, c INT, INDEX(a), INDEX(b))") + require.NoError(t, err) + + _, err = firstServerSQLConn.Exec("INSERT INTO t VALUES (1, 10, 100, 0), (2, 20, 200, 0), (3, 30, 300, 0)") + require.NoError(t, err) + + // We fetch the table ID of the testing table. + rows, err := firstServerSQLConn.Query("SELECT table_id FROM crdb_internal.tables WHERE name = 't'") + require.NoError(t, err) + require.NotNil(t, rows) + + defer func() { + err := rows.Close() + require.NoError(t, err) + }() + + var tableID int + require.True(t, rows.Next()) + err = rows.Scan(&tableID) + require.NoError(t, err) + require.False(t, rows.Next()) + + indexKeyA := roachpb.IndexUsageKey{ + TableID: roachpb.TableID(tableID), + IndexID: 2, // t@t_a_idx + } + + indexKeyB := roachpb.IndexUsageKey{ + TableID: roachpb.TableID(tableID), + IndexID: 3, // t@t_b_idx + } + + err = firstServerSQLConn.Close() + require.NoError(t, err) + + firstServerConnCleanup() + + // Run some queries on the second node. + secondServer := testCluster.Server(1 /* idx */) + secondLocalStatsReader := secondServer.SQLServer().(*sql.Server).GetLocalIndexStatistics() + + secondPgURL, secondServerConnCleanup := sqlutils.PGUrl( + t, secondServer.ServingSQLAddr(), "CreateConnections" /* prefix */, url.User(security.RootUser)) + defer secondServerConnCleanup() + + secondServerSQLConn, err := gosql.Open("postgres", secondPgURL.String()) + require.NoError(t, err) + + defer func() { + err := secondServerSQLConn.Close() + require.NoError(t, err) + }() + + // Records a non-full scan over t_a_idx. + _, err = secondServerSQLConn.Exec("SELECT k, a FROM t WHERE a = 0") + require.NoError(t, err) + + // Records an zigzag join that scans both t_a_idx and t_b_idx. + _, err = secondServerSQLConn.Exec("SELECT k FROM t WHERE a = 10 AND b = 200") + require.NoError(t, err) + + // Record a full scan over t_b_idx. + _, err = secondServerSQLConn.Exec("SELECT * FROM t@t_b_idx") + require.NoError(t, err) + + // Execute a explain query to ensure no index stats is collected. + _, err = secondServerSQLConn.Exec("EXPLAIN SELECT k, a FROM t WHERE a = 0") + require.NoError(t, err) + + // Check local node stats. + // Fetch stats reader from each individual + thirdServer := testCluster.Server(2 /* idx */) + thirdLocalStatsReader := thirdServer.SQLServer().(*sql.Server).GetLocalIndexStatistics() + + // Wait for the stats to be ingested. + waitForStatsIngestion(t, statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{ + indexKeyA: {}, + indexKeyB: {}, + }, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */) + + // First node should have nothing. + stats := firstLocalStatsReader.Get(indexKeyA) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) + + stats = firstLocalStatsReader.Get(indexKeyB) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) + + // Third node should have nothing. + stats = thirdLocalStatsReader.Get(indexKeyA) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 3, but found %v", stats) + + stats = thirdLocalStatsReader.Get(indexKeyB) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) + + // Second server should have nonempty local storage. + stats = secondLocalStatsReader.Get(indexKeyA) + compareStatsHelper(t, expectedStatsIndexA, stats, time.Minute) + + stats = secondLocalStatsReader.Get(indexKeyB) + compareStatsHelper(t, expectedStatsIndexB, stats, time.Minute) + + // Test cluster-wide RPC. + var resp serverpb.IndexUsageStatisticsResponse + err = getStatusJSONProto(thirdServer, "indexusagestatistics", &resp) + require.NoError(t, err) + + statsEntries := 0 + for _, stats := range resp.Statistics { + // Skip if the table is not what we expected. + if stats.Key.TableID != roachpb.TableID(tableID) { + continue + } + statsEntries++ + switch stats.Key.IndexID { + case indexKeyA.IndexID: // t@t_a_idx + compareStatsHelper(t, expectedStatsIndexA, stats.Stats, time.Minute) + case indexKeyB.IndexID: // t@t_b_idx + compareStatsHelper(t, expectedStatsIndexB, stats.Stats, time.Minute) + } + } + + require.True(t, statsEntries == 2, "expect to find two stats entries in RPC response, but found %d", statsEntries) + + // Test disabling subsystem. + _, err = secondServerSQLConn.Exec("SET CLUSTER SETTING sql.metrics.index_usage_stats.enabled = false") + require.NoError(t, err) + + // Records a non-full scan, but it shouldn't change the stats since we have + // stats collection disabled. + _, err = secondServerSQLConn.Exec("SELECT k, a FROM t WHERE a = 0") + require.NoError(t, err) + + err = getStatusJSONProto(thirdServer, "indexusagestatistics", &resp) + require.NoError(t, err) + + statsEntries = 0 + for _, stats := range resp.Statistics { + // Skip if the table is not what we expected. + if stats.Key.TableID != roachpb.TableID(tableID) { + continue + } + statsEntries++ + switch stats.Key.IndexID { + case 2: // t@t_a_idx + compareStatsHelper(t, expectedStatsIndexA, stats.Stats, time.Minute) + case 3: // t@t_b_idx + compareStatsHelper(t, expectedStatsIndexB, stats.Stats, time.Minute) + } + } + require.True(t, statsEntries == 2, "expect to find two stats entries in RPC response, but found %d", statsEntries) +} diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 4ba1a0dc569d..c58b72de0e14 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -59,6 +59,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/querycache" @@ -641,6 +642,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { if backupRestoreKnobs := cfg.TestingKnobs.BackupRestore; backupRestoreKnobs != nil { execCfg.BackupRestoreTestingKnobs = backupRestoreKnobs.(*sql.BackupRestoreTestingKnobs) } + if indexUsageStatsKnobs := cfg.TestingKnobs.IndexUsageStatsKnobs; indexUsageStatsKnobs != nil { + execCfg.IndexUsageStatsTestingKnobs = indexUsageStatsKnobs.(*idxusage.TestingKnobs) + } statsRefresher := stats.MakeRefresher( cfg.Settings, diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index 8ad77f7e3775..f16a12e9a6c2 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -29,6 +29,7 @@ type SQLStatusServer interface { Statements(context.Context, *StatementsRequest) (*StatementsResponse, error) ListDistSQLFlows(context.Context, *ListDistSQLFlowsRequest) (*ListDistSQLFlowsResponse, error) ListLocalDistSQLFlows(context.Context, *ListDistSQLFlowsRequest) (*ListDistSQLFlowsResponse, error) + IndexUsageStatistics(context.Context, *IndexUsageStatisticsRequest) (*IndexUsageStatisticsResponse, error) } // OptionalNodesStatusServer is a StatusServer that is only optionally present diff --git a/pkg/server/serverpb/status.pb.go b/pkg/server/serverpb/status.pb.go index 51e0bb379e86..5f0eb749807f 100644 --- a/pkg/server/serverpb/status.pb.go +++ b/pkg/server/serverpb/status.pb.go @@ -4080,6 +4080,76 @@ func (m *ResetSQLStatsResponse) XXX_DiscardUnknown() { var xxx_messageInfo_ResetSQLStatsResponse proto.InternalMessageInfo +// Request object for issuing IndexUsageStatistics request. +type IndexUsageStatisticsRequest struct { + // node_id is the ID of the node where the stats data shall be retrieved from. + // If this is left empty, the cluster-wide aggregated result will be returned. + NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` +} + +func (m *IndexUsageStatisticsRequest) Reset() { *m = IndexUsageStatisticsRequest{} } +func (m *IndexUsageStatisticsRequest) String() string { return proto.CompactTextString(m) } +func (*IndexUsageStatisticsRequest) ProtoMessage() {} +func (*IndexUsageStatisticsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_07f09f8b91174efc, []int{90} +} +func (m *IndexUsageStatisticsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IndexUsageStatisticsRequest) 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 *IndexUsageStatisticsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_IndexUsageStatisticsRequest.Merge(m, src) +} +func (m *IndexUsageStatisticsRequest) XXX_Size() int { + return m.Size() +} +func (m *IndexUsageStatisticsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_IndexUsageStatisticsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_IndexUsageStatisticsRequest proto.InternalMessageInfo + +// Response object returned by IndexUsageStatistics. +type IndexUsageStatisticsResponse struct { + Statistics []roachpb.CollectedIndexUsageStatistics `protobuf:"bytes,1,rep,name=statistics,proto3" json:"statistics"` +} + +func (m *IndexUsageStatisticsResponse) Reset() { *m = IndexUsageStatisticsResponse{} } +func (m *IndexUsageStatisticsResponse) String() string { return proto.CompactTextString(m) } +func (*IndexUsageStatisticsResponse) ProtoMessage() {} +func (*IndexUsageStatisticsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_07f09f8b91174efc, []int{91} +} +func (m *IndexUsageStatisticsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IndexUsageStatisticsResponse) 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 *IndexUsageStatisticsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_IndexUsageStatisticsResponse.Merge(m, src) +} +func (m *IndexUsageStatisticsResponse) XXX_Size() int { + return m.Size() +} +func (m *IndexUsageStatisticsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_IndexUsageStatisticsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_IndexUsageStatisticsResponse proto.InternalMessageInfo + func init() { proto.RegisterEnum("cockroach.server.serverpb.StacksType", StacksType_name, StacksType_value) proto.RegisterEnum("cockroach.server.serverpb.FileType", FileType_name, FileType_value) @@ -4198,443 +4268,451 @@ func init() { proto.RegisterType((*JobStatusResponse)(nil), "cockroach.server.serverpb.JobStatusResponse") proto.RegisterType((*ResetSQLStatsRequest)(nil), "cockroach.server.serverpb.ResetSQLStatsRequest") proto.RegisterType((*ResetSQLStatsResponse)(nil), "cockroach.server.serverpb.ResetSQLStatsResponse") + proto.RegisterType((*IndexUsageStatisticsRequest)(nil), "cockroach.server.serverpb.IndexUsageStatisticsRequest") + proto.RegisterType((*IndexUsageStatisticsResponse)(nil), "cockroach.server.serverpb.IndexUsageStatisticsResponse") } func init() { proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_07f09f8b91174efc) } var fileDescriptor_07f09f8b91174efc = []byte{ - // 6892 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7c, 0x6b, 0x6c, 0x1c, 0xc9, - 0x71, 0xb0, 0x86, 0xbb, 0x24, 0x77, 0x6b, 0xf9, 0x58, 0x35, 0x1f, 0x5a, 0xad, 0x74, 0xa4, 0x34, - 0xba, 0xd3, 0x51, 0x3a, 0xdd, 0xee, 0x9d, 0x74, 0x2f, 0x9f, 0xed, 0xf3, 0xf1, 0x25, 0x89, 0x12, - 0x8f, 0x92, 0x86, 0xe4, 0x77, 0xc6, 0xf9, 0xfb, 0xbc, 0xdf, 0xec, 0x4e, 0x73, 0x39, 0xe2, 0xee, - 0xcc, 0x72, 0x66, 0x96, 0xc7, 0xbd, 0xfb, 0xce, 0x9f, 0x73, 0x79, 0x39, 0x8e, 0xe3, 0x57, 0x62, - 0xc0, 0x41, 0x82, 0xc4, 0xb8, 0x1f, 0x76, 0x90, 0x20, 0x81, 0x8d, 0x00, 0x41, 0xf2, 0x23, 0x0f, - 0x04, 0x41, 0xec, 0x20, 0x40, 0x60, 0x20, 0xf9, 0x61, 0x24, 0x00, 0x9d, 0xd0, 0x41, 0x10, 0x20, - 0x3f, 0x82, 0x20, 0xf9, 0x75, 0x40, 0x90, 0xa0, 0xab, 0x7b, 0x66, 0x7b, 0x66, 0x97, 0xb3, 0x4b, - 0xe9, 0x74, 0xce, 0x0f, 0x89, 0xd3, 0xdd, 0xd5, 0xd5, 0xd5, 0xd5, 0xd5, 0x55, 0xd5, 0xd5, 0xd5, - 0x0b, 0x67, 0x5d, 0xea, 0xec, 0x51, 0xa7, 0xc8, 0xff, 0x34, 0xca, 0x45, 0xd7, 0xd3, 0xbd, 0xa6, - 0x5b, 0x68, 0x38, 0xb6, 0x67, 0x93, 0xd3, 0x15, 0xbb, 0xb2, 0xe3, 0xd8, 0x7a, 0x65, 0xbb, 0xc0, - 0x01, 0x0a, 0x3e, 0x5c, 0x3e, 0x5b, 0x6e, 0x9a, 0x35, 0xa3, 0x68, 0x5a, 0x5b, 0x36, 0x07, 0xce, - 0x4f, 0x54, 0x6d, 0xd7, 0x35, 0x1b, 0x45, 0xfe, 0x47, 0x54, 0x4e, 0xdf, 0xb7, 0xcb, 0x6e, 0x91, - 0xfd, 0xd7, 0x28, 0xe3, 0x1f, 0x51, 0x7f, 0x0a, 0xb1, 0x36, 0xca, 0x45, 0xbd, 0xd1, 0x28, 0xb1, - 0x31, 0xfd, 0x06, 0xe2, 0x37, 0x18, 0xba, 0xa7, 0xfb, 0x48, 0xfc, 0xba, 0x3a, 0xf5, 0x74, 0xa9, - 0xfe, 0xaa, 0x20, 0xde, 0x30, 0xf5, 0xaa, 0x65, 0xbb, 0x9e, 0x59, 0x71, 0xe5, 0x6f, 0x86, 0xa4, - 0x5d, 0x12, 0x7d, 0x2e, 0xf8, 0x13, 0xc6, 0x79, 0x8a, 0x3f, 0x91, 0x79, 0xe7, 0xcf, 0xbb, 0xbb, - 0xb5, 0x62, 0xc5, 0xb6, 0x3c, 0x6a, 0x79, 0xa6, 0x6d, 0x35, 0xca, 0x52, 0x41, 0x80, 0x9c, 0x66, - 0x20, 0x74, 0x9f, 0x56, 0x4c, 0x6b, 0xcb, 0xd1, 0x71, 0x22, 0xa6, 0x68, 0x7a, 0xcc, 0xf5, 0x6c, - 0x47, 0xaf, 0xd2, 0x22, 0xb5, 0xaa, 0xa6, 0x45, 0x1b, 0x65, 0xf1, 0x21, 0x9a, 0xcf, 0x74, 0x34, - 0xd7, 0xf7, 0x2a, 0x15, 0xd1, 0x38, 0xd3, 0xd1, 0xe8, 0xd8, 0x95, 0x1d, 0xd7, 0x28, 0x8b, 0xf6, - 0x4b, 0x3b, 0x7b, 0xc5, 0x9d, 0x3d, 0x31, 0x07, 0xff, 0xa3, 0x51, 0x2e, 0xd6, 0xa8, 0xee, 0xd2, - 0x52, 0x68, 0x12, 0xea, 0x11, 0xa0, 0x0c, 0xc8, 0xa7, 0xa5, 0x20, 0xc3, 0xd4, 0xcc, 0x3d, 0x6a, - 0x51, 0xd7, 0x0d, 0x3e, 0x18, 0x5e, 0xf1, 0x29, 0xe0, 0x73, 0x4d, 0xcf, 0xac, 0x15, 0x6b, 0x76, - 0x95, 0xfd, 0x63, 0xcd, 0x76, 0x55, 0xb4, 0xe4, 0xb1, 0xa5, 0x69, 0x39, 0xd4, 0xb5, 0x6b, 0x7b, - 0xd4, 0x28, 0xe9, 0x86, 0xe1, 0x84, 0x7a, 0x6d, 0xd7, 0x2a, 0x45, 0xcf, 0xac, 0x53, 0xd7, 0xd3, - 0xeb, 0xbe, 0x78, 0xcc, 0x50, 0xaf, 0x62, 0x14, 0x1d, 0x7d, 0xcb, 0x2b, 0xee, 0x5d, 0xc3, 0xbf, - 0x6c, 0xc6, 0xfa, 0x96, 0x27, 0xda, 0x27, 0xab, 0x76, 0xd5, 0xc6, 0xcf, 0x22, 0xfb, 0x12, 0xb5, - 0x67, 0xab, 0xb6, 0x5d, 0xad, 0x51, 0xc6, 0xf2, 0xa2, 0x6e, 0x59, 0xb6, 0xa7, 0xb3, 0x85, 0xf1, - 0x69, 0x9c, 0x15, 0xad, 0x58, 0x2a, 0x37, 0xb7, 0xa2, 0x83, 0xaa, 0x05, 0x98, 0x58, 0xa4, 0x8e, - 0x67, 0x6e, 0x99, 0x15, 0xdd, 0xa3, 0xae, 0x46, 0x77, 0x9b, 0xd4, 0xf5, 0xc8, 0x29, 0x18, 0xb6, - 0x6c, 0x83, 0x96, 0x4c, 0x23, 0xa7, 0x9c, 0x53, 0xe6, 0xd2, 0xda, 0x10, 0x2b, 0xae, 0x18, 0xea, - 0x7f, 0x26, 0x81, 0x48, 0x1d, 0x96, 0xa8, 0xa7, 0x9b, 0x35, 0x97, 0xdc, 0x83, 0xa4, 0xd7, 0x6a, - 0x50, 0x04, 0x1e, 0xbb, 0xfa, 0xf1, 0xc2, 0x91, 0x7b, 0xa5, 0xd0, 0xd9, 0x59, 0xae, 0xda, 0x68, - 0x35, 0xa8, 0x86, 0xa8, 0xc8, 0x05, 0x18, 0xa5, 0x8e, 0x63, 0x3b, 0xa5, 0x3a, 0x75, 0x5d, 0xbd, - 0x4a, 0x73, 0x03, 0x48, 0xc8, 0x08, 0x56, 0xbe, 0xc6, 0xeb, 0x08, 0x81, 0x24, 0xdb, 0x03, 0xb9, - 0xc4, 0x39, 0x65, 0x6e, 0x44, 0xc3, 0x6f, 0xa2, 0xc1, 0xd0, 0x96, 0x49, 0x6b, 0x86, 0x9b, 0x4b, - 0x9e, 0x4b, 0xcc, 0x65, 0xae, 0x3e, 0x77, 0x3c, 0x6a, 0xae, 0x63, 0xdf, 0x85, 0xe4, 0xf7, 0x0e, - 0x66, 0x4f, 0x68, 0x02, 0x53, 0xfe, 0x77, 0x07, 0x60, 0x88, 0x37, 0x90, 0x69, 0x18, 0x32, 0x5d, - 0xb7, 0x49, 0x1d, 0x9f, 0x33, 0xbc, 0x44, 0x72, 0x30, 0xec, 0x36, 0xcb, 0xf7, 0x69, 0xc5, 0x13, - 0x94, 0xfa, 0x45, 0xf2, 0x18, 0xc0, 0x9e, 0x5e, 0x33, 0x8d, 0xd2, 0x96, 0x63, 0xd7, 0x91, 0xd4, - 0x84, 0x96, 0xc6, 0x9a, 0xeb, 0x8e, 0x5d, 0x27, 0xb3, 0x90, 0xe1, 0xcd, 0x4d, 0xcb, 0x33, 0x6b, - 0xb9, 0x24, 0xb6, 0xf3, 0x1e, 0x9b, 0xac, 0x86, 0x9c, 0x85, 0x34, 0x13, 0x20, 0xea, 0xba, 0xd4, - 0xcd, 0x0d, 0x9e, 0x4b, 0xcc, 0xa5, 0xb5, 0x76, 0x05, 0x29, 0xc2, 0x84, 0x6b, 0x56, 0x2d, 0xdd, - 0x6b, 0x3a, 0xb4, 0xa4, 0xd7, 0xaa, 0xb6, 0x63, 0x7a, 0xdb, 0xf5, 0xdc, 0x10, 0xd2, 0x40, 0x82, - 0xa6, 0x79, 0xbf, 0x85, 0x91, 0xd3, 0x68, 0x96, 0x6b, 0x66, 0xa5, 0xb4, 0x43, 0x5b, 0xb9, 0x61, - 0x84, 0x4b, 0xf3, 0x9a, 0xdb, 0xb4, 0x45, 0xce, 0x40, 0x7a, 0x87, 0xb6, 0x4a, 0x4d, 0xe4, 0x79, - 0x0a, 0x47, 0x4b, 0xed, 0xd0, 0xd6, 0x26, 0xf2, 0xfb, 0x0a, 0x10, 0xba, 0xef, 0x51, 0xcb, 0xa0, - 0x46, 0xa9, 0x0d, 0x95, 0x46, 0xa8, 0xac, 0xdf, 0x72, 0x5b, 0x40, 0xab, 0xf7, 0x60, 0x3c, 0xb2, - 0xb6, 0x64, 0x08, 0x06, 0x16, 0xe7, 0xb3, 0x27, 0x48, 0x0a, 0x92, 0x6b, 0x77, 0x96, 0x96, 0xb3, - 0x0a, 0x19, 0x85, 0xf4, 0xe2, 0xea, 0xca, 0xf2, 0xda, 0x46, 0x69, 0x71, 0x3e, 0x3b, 0x40, 0x00, - 0x86, 0x78, 0x31, 0x9b, 0x20, 0x69, 0x18, 0xdc, 0x5c, 0x61, 0xd5, 0x49, 0xd6, 0x6f, 0x73, 0x25, - 0x3b, 0xa8, 0xda, 0x30, 0x19, 0x96, 0x57, 0xb7, 0x61, 0x5b, 0x2e, 0x25, 0xaf, 0xc3, 0x48, 0x45, - 0xaa, 0xcf, 0x29, 0xb8, 0xf4, 0x4f, 0x1f, 0x6b, 0xe9, 0xc5, 0x9a, 0x87, 0x10, 0xa9, 0x45, 0x18, - 0x13, 0xcd, 0xbd, 0xf6, 0xc6, 0xad, 0x64, 0x6a, 0x20, 0x9b, 0x50, 0xd7, 0x00, 0xd6, 0x5b, 0xae, - 0x47, 0xeb, 0x2b, 0xd6, 0x96, 0xcd, 0x16, 0xd7, 0xc5, 0x52, 0x89, 0x59, 0x07, 0xd1, 0x01, 0xdc, - 0x10, 0xc0, 0x0e, 0x75, 0x2c, 0x5a, 0xe3, 0x00, 0x5c, 0x74, 0x80, 0x57, 0x31, 0x00, 0xf5, 0x0b, - 0x09, 0x18, 0x0f, 0x28, 0x10, 0xb3, 0x7d, 0x23, 0x4c, 0xc2, 0xe0, 0xc2, 0xfc, 0xe1, 0xc1, 0xec, - 0xd0, 0x1a, 0x23, 0x63, 0xe9, 0xfd, 0x83, 0xd9, 0x6b, 0x55, 0xd3, 0xdb, 0x6e, 0x96, 0x0b, 0x15, - 0xbb, 0x5e, 0x0c, 0x18, 0x60, 0x94, 0xdb, 0xdf, 0xc5, 0xc6, 0x4e, 0xb5, 0x28, 0x0c, 0x49, 0x81, - 0x77, 0xf3, 0x67, 0x41, 0x5e, 0x81, 0x61, 0x21, 0x5c, 0x48, 0x4c, 0xe6, 0xea, 0x8c, 0xc4, 0x44, - 0xa6, 0xbc, 0x0a, 0x9b, 0x81, 0x62, 0x9b, 0x37, 0x0c, 0x47, 0x70, 0xcd, 0xef, 0x44, 0x5e, 0x06, - 0x40, 0x73, 0xc8, 0xe7, 0x93, 0x40, 0x14, 0x53, 0x12, 0x0a, 0x6c, 0x2c, 0xb0, 0xa9, 0x89, 0x9e, - 0x69, 0xac, 0x41, 0x66, 0xac, 0x86, 0xb9, 0x95, 0xc4, 0xce, 0x4f, 0xc4, 0x2c, 0x62, 0x9b, 0xd3, - 0x02, 0x99, 0xcc, 0xda, 0x75, 0xc8, 0xb8, 0xbb, 0xb5, 0x92, 0x3f, 0x9b, 0xc1, 0xbe, 0x66, 0x43, - 0x18, 0x9a, 0xc3, 0x83, 0x59, 0x58, 0xbf, 0xb7, 0x3a, 0xcf, 0x7b, 0x6a, 0xe0, 0xee, 0xd6, 0xc4, - 0xb7, 0x3a, 0x06, 0x23, 0x8c, 0x61, 0xbe, 0x34, 0xa8, 0xbf, 0x9e, 0x80, 0x51, 0x51, 0x21, 0x16, - 0xe7, 0x26, 0x0c, 0x32, 0x56, 0xfa, 0x32, 0x78, 0xa5, 0x0b, 0xf9, 0xdc, 0x34, 0xf9, 0xf6, 0x16, - 0x57, 0x60, 0x1d, 0x0b, 0x62, 0x16, 0x1c, 0x01, 0xf9, 0x23, 0x05, 0x26, 0x7c, 0xa3, 0x53, 0x2a, - 0xb7, 0x4a, 0xfe, 0x9a, 0x0f, 0x20, 0xe2, 0x57, 0x62, 0xf8, 0x12, 0xa2, 0xa8, 0xb0, 0x2a, 0x70, - 0x2c, 0xb4, 0x70, 0xad, 0x8d, 0x65, 0xcb, 0x73, 0x5a, 0x0b, 0x77, 0xc4, 0x4c, 0xb3, 0x91, 0xe6, - 0xa5, 0x77, 0x7f, 0xf8, 0x60, 0x12, 0x94, 0xad, 0x45, 0xc6, 0xc9, 0xbf, 0xab, 0xc0, 0x54, 0xd7, - 0xc1, 0x49, 0x16, 0x12, 0x4c, 0xfb, 0xa0, 0xf4, 0x6a, 0xec, 0x93, 0xac, 0xc3, 0xe0, 0x9e, 0x5e, - 0x6b, 0x72, 0x3d, 0x1f, 0xb6, 0x21, 0x3b, 0x7b, 0x05, 0xdf, 0x30, 0x17, 0x02, 0x23, 0xdc, 0x36, - 0xcc, 0x38, 0xbe, 0x3f, 0x0c, 0xe7, 0xa3, 0xc6, 0x71, 0xbd, 0x3c, 0xf0, 0x92, 0xa2, 0x66, 0x61, - 0x4c, 0xa3, 0x55, 0x66, 0x14, 0xfd, 0x35, 0xfb, 0x77, 0x05, 0xc6, 0x83, 0x2a, 0xb1, 0x6a, 0xf7, - 0x60, 0xd8, 0xe1, 0x55, 0x62, 0xdd, 0x5e, 0x8c, 0x61, 0x6f, 0xa4, 0xb3, 0x5f, 0xc6, 0xa9, 0x69, - 0x3e, 0x9e, 0xfc, 0x0c, 0x0c, 0xf1, 0x06, 0x32, 0x09, 0x83, 0x6f, 0xd9, 0x96, 0x10, 0x89, 0xb4, - 0xc6, 0x0b, 0xf9, 0x3a, 0x8c, 0xc8, 0x1d, 0x65, 0x9e, 0xa4, 0x39, 0x4f, 0x6e, 0xc8, 0x3c, 0xc9, - 0x5c, 0x7d, 0xf6, 0xd8, 0x24, 0xc9, 0x7c, 0xb8, 0x08, 0x19, 0xc6, 0xa8, 0x9e, 0x26, 0xfe, 0xdb, - 0x49, 0x48, 0x6b, 0xfa, 0x96, 0xc7, 0x38, 0xc9, 0x34, 0x3e, 0x38, 0xb4, 0x51, 0x33, 0x2b, 0xba, - 0x0f, 0x99, 0x5c, 0x18, 0x3d, 0x3c, 0x98, 0x4d, 0x6b, 0xbc, 0x76, 0x65, 0x49, 0x4b, 0x0b, 0x80, - 0x15, 0x83, 0xbc, 0x00, 0xb0, 0xad, 0x3b, 0x06, 0x3a, 0x5f, 0x3e, 0xd5, 0x27, 0x0b, 0xdc, 0x97, - 0x29, 0xdc, 0xd4, 0x1d, 0x03, 0x91, 0xfa, 0x1b, 0x7f, 0xdb, 0xaf, 0x60, 0x76, 0xbc, 0x46, 0x75, - 0x03, 0xd5, 0x45, 0x52, 0xc3, 0x6f, 0xc6, 0x34, 0x8e, 0x26, 0x89, 0xe4, 0xf1, 0x02, 0x33, 0xb3, - 0x7a, 0xa3, 0x51, 0x33, 0xa9, 0x81, 0x1b, 0x3a, 0xa9, 0xf9, 0x45, 0xb2, 0x01, 0xa9, 0x86, 0x63, - 0x57, 0x71, 0xaf, 0x0f, 0xe1, 0x12, 0x5e, 0x8d, 0xe3, 0x97, 0x3f, 0xc3, 0xc2, 0x5d, 0xd1, 0x89, - 0xef, 0x0a, 0x4e, 0x5a, 0x80, 0x89, 0x3c, 0x09, 0xe3, 0x8c, 0x9a, 0x92, 0xe7, 0xe8, 0x96, 0xbb, - 0x45, 0x1d, 0x4a, 0xd1, 0x64, 0x26, 0xb5, 0x31, 0x56, 0xbd, 0x11, 0xd4, 0xe6, 0x7f, 0x41, 0x81, - 0x94, 0x8f, 0x8a, 0xd1, 0x5e, 0xd7, 0xbd, 0xca, 0x36, 0x67, 0x98, 0xc6, 0x0b, 0x6c, 0x96, 0x16, - 0xdd, 0xe7, 0xfe, 0x41, 0x52, 0xc3, 0xef, 0xf6, 0x2c, 0x13, 0xf2, 0x2c, 0xa7, 0x61, 0xa8, 0xa1, - 0x37, 0x5d, 0x6a, 0xe0, 0xe4, 0x53, 0x9a, 0x28, 0x91, 0x4b, 0x90, 0x6d, 0x50, 0xcb, 0x30, 0xad, - 0x6a, 0xc9, 0xb5, 0xf4, 0x86, 0xbb, 0x6d, 0x7b, 0x82, 0x0d, 0xe3, 0xa2, 0x7e, 0x5d, 0x54, 0xe7, - 0xef, 0xc3, 0x68, 0x68, 0x66, 0xb2, 0x78, 0x25, 0xb9, 0x78, 0x2d, 0x86, 0xc5, 0xeb, 0xe9, 0x63, - 0xb1, 0x4b, 0x16, 0xad, 0xc3, 0x01, 0x18, 0xd5, 0x74, 0xab, 0x4a, 0xef, 0x3a, 0x76, 0xb9, 0x46, - 0xeb, 0x2e, 0x39, 0x07, 0x99, 0xa6, 0xa5, 0xef, 0xe9, 0x66, 0x4d, 0x2f, 0xd7, 0xb8, 0x5f, 0x98, - 0xd2, 0xe4, 0x2a, 0xf2, 0x3c, 0x9c, 0x62, 0x1c, 0xa4, 0x4e, 0xc9, 0xb2, 0xbd, 0x12, 0xf7, 0xd9, - 0xb7, 0xed, 0x9a, 0x41, 0x1d, 0x24, 0x27, 0xa5, 0x4d, 0xf2, 0xe6, 0x35, 0xdb, 0x5b, 0x65, 0x8d, - 0x37, 0xb1, 0x8d, 0x3c, 0x0e, 0x63, 0x96, 0x5d, 0x62, 0x12, 0x55, 0xe2, 0xed, 0xc8, 0xb8, 0x94, - 0x36, 0x62, 0xd9, 0x8c, 0xc6, 0x55, 0xac, 0x23, 0x73, 0x30, 0xde, 0xb4, 0x0c, 0xea, 0x08, 0xc9, - 0xf4, 0x02, 0x46, 0x46, 0xab, 0xc9, 0x45, 0x18, 0xb3, 0xf7, 0x42, 0x80, 0x29, 0x04, 0x8c, 0xd4, - 0x92, 0xd3, 0x90, 0xb2, 0x6c, 0x4e, 0x26, 0x72, 0x3c, 0xa5, 0x0d, 0x5b, 0x36, 0x12, 0x46, 0x5e, - 0x82, 0xdc, 0x6e, 0xd3, 0xa4, 0x6e, 0x85, 0x5a, 0x5e, 0x89, 0xee, 0x36, 0xf5, 0x9a, 0x5b, 0xf2, - 0xcc, 0xca, 0x8e, 0x69, 0x55, 0xd1, 0x0d, 0x4b, 0x69, 0xd3, 0x41, 0xfb, 0x32, 0x36, 0x6f, 0xf0, - 0x56, 0xf2, 0x14, 0x10, 0x3e, 0x13, 0xbb, 0x5a, 0xf2, 0x6c, 0xbb, 0x54, 0xd3, 0x9d, 0x2a, 0x97, - 0xaf, 0x94, 0x36, 0xce, 0x5a, 0x56, 0xed, 0xea, 0x86, 0x6d, 0xaf, 0xb2, 0x6a, 0x75, 0x07, 0xc6, - 0x91, 0xc7, 0x6c, 0x19, 0x4c, 0x3c, 0xc6, 0x31, 0x77, 0x6c, 0xb7, 0x49, 0x1d, 0x93, 0xba, 0xa5, - 0x06, 0x75, 0x4a, 0x2e, 0xad, 0xd8, 0x16, 0xdf, 0xa4, 0x8a, 0x96, 0x15, 0x2d, 0x77, 0xa9, 0xb3, - 0x8e, 0xf5, 0xe4, 0x32, 0x9c, 0x7c, 0xd3, 0x31, 0xbd, 0x30, 0xf0, 0x00, 0x02, 0x8f, 0xf3, 0x86, - 0x00, 0x56, 0xbd, 0x09, 0x70, 0xd7, 0xa1, 0x9e, 0xd7, 0x5a, 0x6f, 0xe8, 0x16, 0xf3, 0x09, 0x5d, - 0x4f, 0x77, 0xbc, 0x52, 0x5b, 0x3f, 0xa5, 0xb0, 0x82, 0x39, 0x8c, 0xa7, 0x60, 0x98, 0x5a, 0xe8, - 0x0e, 0x0a, 0xef, 0x65, 0x88, 0x5a, 0xcc, 0x07, 0x7c, 0x39, 0xf9, 0xcf, 0xdf, 0x98, 0x55, 0xd4, - 0xaf, 0x65, 0x98, 0x3a, 0xb1, 0xaa, 0x14, 0x6d, 0xf2, 0x27, 0x20, 0xe9, 0x36, 0x74, 0x0b, 0x91, - 0xc4, 0x9b, 0xf6, 0xf6, 0xf0, 0x62, 0x4f, 0x62, 0x47, 0xb2, 0x02, 0x80, 0x2c, 0x93, 0x35, 0xcc, - 0xe3, 0xfd, 0x08, 0xae, 0xaf, 0x74, 0x9c, 0x40, 0xb5, 0x5d, 0x97, 0x15, 0x4c, 0xe6, 0xea, 0xe5, - 0x23, 0x2c, 0x8e, 0x38, 0x85, 0x22, 0x2e, 0x31, 0x0d, 0xdf, 0x4c, 0xf3, 0xcd, 0x5a, 0x87, 0x31, - 0xd7, 0x6e, 0x3a, 0x15, 0x1a, 0x18, 0xe8, 0x41, 0x74, 0xca, 0x6e, 0x1c, 0x1e, 0xcc, 0x8e, 0xac, - 0x63, 0xcb, 0xc3, 0xb9, 0x66, 0x23, 0x6e, 0x1b, 0x89, 0x41, 0x76, 0x61, 0x5c, 0x0c, 0xc7, 0x28, - 0xc3, 0xf1, 0x86, 0x70, 0xbc, 0x95, 0xc3, 0x83, 0xd9, 0x51, 0x3e, 0xde, 0x3a, 0x6b, 0xc1, 0x01, - 0x9f, 0x3b, 0xd6, 0x80, 0xa2, 0x9f, 0x36, 0xea, 0x4a, 0x68, 0x8c, 0xce, 0xb3, 0xd8, 0x70, 0x97, - 0xb3, 0xd8, 0x22, 0x8c, 0x8a, 0x5d, 0x6c, 0x32, 0xc2, 0x5a, 0x78, 0x78, 0xc8, 0x5c, 0xcd, 0x49, - 0x6c, 0xf5, 0x87, 0xc1, 0x7d, 0xe3, 0xbb, 0xdb, 0xd8, 0xe9, 0x26, 0xef, 0x43, 0x6e, 0xa1, 0x12, - 0x47, 0x1d, 0x92, 0x4b, 0xe3, 0xb2, 0xcc, 0xc5, 0x2e, 0xae, 0xa4, 0x73, 0x24, 0xd5, 0xcd, 0x75, - 0x90, 0x58, 0x5f, 0x37, 0x07, 0x1d, 0xeb, 0xdb, 0x15, 0x51, 0x7b, 0x63, 0xc9, 0xeb, 0xeb, 0x92, - 0x4d, 0x18, 0x91, 0x43, 0x0a, 0xb9, 0x51, 0x44, 0x77, 0xa5, 0xa7, 0xb8, 0xe0, 0x2c, 0x43, 0x7e, - 0x5d, 0xa6, 0xd6, 0xae, 0x62, 0xc7, 0xba, 0x40, 0x2d, 0xe4, 0xc6, 0x70, 0xcf, 0xb7, 0x2b, 0x98, - 0x9d, 0xf3, 0x75, 0xc8, 0x38, 0x57, 0x37, 0xa2, 0x48, 0xce, 0xc3, 0x88, 0xc3, 0x2c, 0x52, 0x8d, - 0xd9, 0x14, 0xea, 0xe6, 0xb2, 0x78, 0x60, 0xcc, 0xb0, 0xba, 0x55, 0x5e, 0xc5, 0xd6, 0x0b, 0x37, - 0x74, 0x00, 0x73, 0x12, 0x61, 0x46, 0xb0, 0xd2, 0x07, 0x9a, 0x84, 0xc1, 0x9a, 0x5d, 0xd9, 0x71, - 0x73, 0x04, 0x1b, 0x79, 0x81, 0x5c, 0x83, 0x69, 0xfc, 0x28, 0xbd, 0x69, 0x7a, 0xdb, 0xa5, 0x37, - 0x75, 0xd3, 0x2b, 0xed, 0x36, 0x69, 0x93, 0xba, 0xb9, 0x09, 0x04, 0x9b, 0xc0, 0xd6, 0xd7, 0x4d, - 0x6f, 0xfb, 0x75, 0xdd, 0xf4, 0xee, 0x61, 0x13, 0xea, 0x72, 0xbb, 0xb2, 0x23, 0x81, 0xe3, 0x27, - 0x75, 0xdc, 0xdc, 0x24, 0xf6, 0x9a, 0x64, 0xcd, 0x41, 0x87, 0xd7, 0x79, 0x1b, 0x79, 0x1b, 0xce, - 0x7b, 0x76, 0xa3, 0xb4, 0x53, 0xe2, 0x23, 0x96, 0x5b, 0xdd, 0x10, 0x4c, 0xf5, 0x3c, 0xc9, 0x05, - 0xdb, 0xb2, 0xb0, 0x6a, 0x57, 0x76, 0xa4, 0xfd, 0x79, 0xc6, 0xb3, 0x1b, 0xb7, 0x59, 0x9d, 0xbb, - 0xd0, 0x8a, 0x0e, 0x9e, 0xff, 0x37, 0x05, 0x52, 0x3e, 0x3c, 0x9e, 0x89, 0x51, 0xdf, 0x48, 0x1a, - 0x2e, 0xcd, 0x6b, 0x98, 0x8a, 0xbb, 0xce, 0x4d, 0x27, 0xd3, 0x36, 0x23, 0x0b, 0xcf, 0xbd, 0x7f, - 0x30, 0xfb, 0xcc, 0xb1, 0x76, 0xd5, 0x6d, 0xda, 0xe2, 0x06, 0x97, 0x40, 0x72, 0x9b, 0xd6, 0x0c, - 0x61, 0xb2, 0xf0, 0x9b, 0x2d, 0xb4, 0x3f, 0x55, 0x7e, 0xf4, 0xf7, 0x8b, 0xcc, 0xf5, 0x60, 0x9f, - 0xcc, 0xd8, 0x3b, 0x68, 0xd6, 0xf8, 0x19, 0x26, 0xa1, 0x8d, 0x89, 0x6a, 0x8d, 0xd7, 0xca, 0x80, - 0xb8, 0xc2, 0x8e, 0x8b, 0x1a, 0xa1, 0x0d, 0xf8, 0x3a, 0xaf, 0xbd, 0x95, 0x4c, 0x65, 0xb2, 0x23, - 0xb7, 0x92, 0xa9, 0x91, 0xec, 0xa8, 0xfa, 0xf3, 0x8a, 0xb0, 0xd9, 0x3d, 0x0f, 0xb6, 0x44, 0x87, - 0xb4, 0xc3, 0x20, 0x4b, 0xa6, 0xe1, 0xe2, 0xe1, 0x23, 0xb1, 0xb0, 0x74, 0x78, 0x30, 0x9b, 0xe2, - 0x8c, 0x5f, 0x72, 0x8f, 0xad, 0x66, 0x44, 0x47, 0x2d, 0x85, 0x68, 0x57, 0x0c, 0x57, 0xdd, 0x80, - 0x31, 0x9f, 0x18, 0xe1, 0x90, 0x2f, 0xc0, 0x10, 0xb6, 0xfa, 0xfe, 0xf8, 0xe3, 0xfd, 0x48, 0x80, - 0x1f, 0xb6, 0xe1, 0x3d, 0xd5, 0x39, 0x18, 0xbd, 0x81, 0x11, 0xd8, 0x9e, 0x4e, 0xef, 0x7b, 0x03, - 0x30, 0xbe, 0x8c, 0x61, 0x46, 0xb6, 0x3b, 0x5d, 0x14, 0x8a, 0x4f, 0x43, 0x2a, 0xd0, 0xb0, 0xfc, - 0x98, 0xbd, 0x78, 0x78, 0x30, 0x3b, 0xfc, 0xb0, 0xba, 0x75, 0xd8, 0x15, 0x5a, 0x75, 0x0b, 0xa6, - 0xd9, 0x9e, 0xa6, 0x8e, 0x5b, 0xd2, 0x2d, 0x83, 0xab, 0xcd, 0xaa, 0xa3, 0xd7, 0xfd, 0x83, 0xf7, - 0x33, 0xf2, 0x8c, 0xb9, 0x56, 0x29, 0xf8, 0xa1, 0xd0, 0xc2, 0x06, 0xef, 0x39, 0x6f, 0x19, 0x37, - 0x83, 0x7e, 0xda, 0xa4, 0xd7, 0xa5, 0x96, 0xdc, 0x80, 0x0c, 0xef, 0x56, 0xc2, 0x18, 0x5d, 0x02, - 0xcf, 0x57, 0x17, 0xe3, 0x90, 0x73, 0x4e, 0x60, 0x30, 0x0e, 0x68, 0xf0, 0xad, 0x3e, 0x0d, 0x44, - 0xe2, 0x51, 0x4f, 0x9e, 0xfe, 0x1f, 0x98, 0x08, 0x81, 0x8b, 0x85, 0x0d, 0xd4, 0x32, 0x5f, 0xd7, - 0x38, 0xb5, 0x1c, 0x59, 0x91, 0x90, 0x5a, 0x56, 0xff, 0x2f, 0xc0, 0x86, 0xa3, 0x57, 0xe8, 0xf2, - 0x1e, 0xd3, 0x97, 0x2f, 0x41, 0xd2, 0x33, 0xeb, 0x54, 0x38, 0x16, 0xf9, 0x02, 0x0f, 0x7c, 0x16, - 0xfc, 0xc0, 0x67, 0x61, 0xc3, 0x0f, 0x7c, 0x2e, 0xa4, 0x18, 0x92, 0x2f, 0xff, 0x70, 0x56, 0xd1, - 0xb0, 0x07, 0xdb, 0x80, 0xe1, 0x10, 0xa3, 0x5f, 0x54, 0xbf, 0xad, 0xc0, 0xf8, 0x7c, 0xad, 0x66, - 0x57, 0x74, 0xcf, 0x76, 0x96, 0x9c, 0x96, 0xd6, 0xb4, 0x98, 0x50, 0xf8, 0x7b, 0x01, 0xc7, 0x4a, - 0x70, 0xa1, 0x10, 0x12, 0xfd, 0xc0, 0x3b, 0x61, 0x58, 0xec, 0x04, 0xf2, 0x71, 0x18, 0xa2, 0x6c, - 0x42, 0xae, 0x38, 0xe5, 0xc7, 0xb9, 0x48, 0xed, 0xe9, 0x6b, 0xa2, 0x93, 0x7a, 0x15, 0xa6, 0x02, - 0x8a, 0x11, 0xb7, 0xbf, 0x4a, 0xa7, 0xa3, 0x74, 0x07, 0x43, 0xaa, 0x7f, 0xa0, 0xc0, 0x74, 0xb4, - 0x53, 0xf7, 0x40, 0x53, 0xe2, 0x83, 0x0c, 0x34, 0x2d, 0xc2, 0xb0, 0xe1, 0xb4, 0x4a, 0x4e, 0xd3, - 0x12, 0xf2, 0x1e, 0x27, 0x09, 0x91, 0x65, 0xd0, 0x86, 0x0c, 0xfc, 0xab, 0x7e, 0x51, 0x81, 0x6c, - 0x9b, 0xf6, 0xff, 0x01, 0x8a, 0xec, 0x0d, 0x38, 0x29, 0xd1, 0x23, 0xd8, 0xb8, 0x0c, 0x29, 0x31, - 0xd5, 0x7e, 0xa4, 0x3e, 0x3a, 0xd7, 0x61, 0x3e, 0x57, 0x57, 0x55, 0x61, 0xe4, 0xd6, 0xfa, 0x9d, - 0xb5, 0x00, 0xad, 0x1f, 0xfd, 0x56, 0xda, 0xd1, 0x6f, 0xb5, 0x0e, 0xa3, 0x7e, 0xfb, 0x32, 0xf3, - 0xce, 0x98, 0x99, 0x47, 0x37, 0x4d, 0xb0, 0x82, 0x17, 0x58, 0xd7, 0x8a, 0x6d, 0x70, 0x89, 0x1f, - 0xd4, 0xf0, 0x5b, 0xde, 0x08, 0x89, 0xd0, 0x46, 0x60, 0x2d, 0x06, 0x0f, 0x41, 0x62, 0x4c, 0x3d, - 0xad, 0xf9, 0x45, 0xf5, 0x4f, 0x14, 0xc8, 0xac, 0xda, 0xd5, 0xde, 0x36, 0x84, 0x79, 0x1b, 0x74, - 0x8f, 0xd6, 0xc4, 0x1e, 0xe3, 0x05, 0x66, 0x77, 0xf9, 0xc1, 0x02, 0xf7, 0x2e, 0x1f, 0x95, 0x1f, - 0x35, 0xd8, 0x7e, 0x65, 0x42, 0xcb, 0x8e, 0x16, 0xd8, 0xc8, 0xa3, 0x00, 0xec, 0xa8, 0x81, 0x4d, - 0x59, 0x48, 0xd4, 0xf5, 0x7d, 0x34, 0x88, 0x69, 0x8d, 0x7d, 0x32, 0x22, 0x1b, 0xba, 0xe7, 0x51, - 0xc7, 0x12, 0xc1, 0x6f, 0xbf, 0xc8, 0x4e, 0xd3, 0x0e, 0x35, 0xf4, 0x8a, 0x27, 0x8e, 0x56, 0xa2, - 0x74, 0x2b, 0x99, 0x4a, 0x65, 0xd3, 0xea, 0x1d, 0x20, 0xab, 0x76, 0x95, 0x9d, 0x91, 0x4d, 0xc9, - 0xfc, 0x7c, 0x84, 0x9d, 0x6a, 0xb0, 0x4a, 0xac, 0xd8, 0xe9, 0x68, 0xe0, 0xb0, 0x66, 0x57, 0x0b, - 0x72, 0xcc, 0xc0, 0x87, 0x57, 0x0b, 0x30, 0xb1, 0x6a, 0x57, 0xaf, 0x9b, 0x35, 0xea, 0xae, 0x9a, - 0xae, 0xd7, 0x53, 0x4f, 0xde, 0x85, 0xc9, 0x30, 0xbc, 0x20, 0xe1, 0x25, 0x18, 0xdc, 0x62, 0x95, - 0x82, 0x80, 0xb3, 0xdd, 0x08, 0x60, 0xbd, 0x64, 0xd5, 0x88, 0x1d, 0xd4, 0x4f, 0xc1, 0x98, 0xc0, - 0xd8, 0x73, 0x5d, 0x08, 0x24, 0x59, 0x1f, 0xb1, 0x2c, 0xf8, 0x2d, 0xf1, 0x2b, 0x11, 0xe1, 0x57, - 0x32, 0x3b, 0xa8, 0x56, 0x60, 0x74, 0xdd, 0xd3, 0x2b, 0x3b, 0xbd, 0xd7, 0xfc, 0x23, 0xe2, 0x56, - 0x88, 0x47, 0xf4, 0x62, 0xe3, 0xb8, 0x88, 0xb0, 0x7d, 0xfb, 0xa3, 0xae, 0x43, 0x92, 0x91, 0x8f, - 0x21, 0x13, 0x5d, 0xa8, 0xf5, 0xb4, 0x86, 0xdf, 0xec, 0x34, 0xca, 0xc8, 0x2c, 0xb9, 0xe6, 0x5b, - 0x1c, 0x77, 0x42, 0x4b, 0xb1, 0x8a, 0x75, 0xf3, 0x2d, 0x4a, 0xf2, 0x90, 0x12, 0xf7, 0x93, 0xae, - 0xb8, 0x15, 0x0a, 0xca, 0xea, 0xaf, 0x2a, 0x30, 0x7e, 0x83, 0x7a, 0xc8, 0xe9, 0x9e, 0xc4, 0x9f, - 0x81, 0x74, 0xcd, 0x74, 0xbd, 0x92, 0x6d, 0xd5, 0x5a, 0x22, 0x22, 0x91, 0x62, 0x15, 0x77, 0xac, - 0x5a, 0x8b, 0xbc, 0x28, 0x66, 0x36, 0x88, 0x33, 0xbb, 0x10, 0x33, 0x33, 0x36, 0x98, 0x74, 0xab, - 0x95, 0x87, 0x94, 0x90, 0x4a, 0x1e, 0xa4, 0x4a, 0x6b, 0x41, 0x59, 0x5d, 0x81, 0x6c, 0x9b, 0x3a, - 0x21, 0x03, 0xcf, 0x87, 0x65, 0x60, 0xb6, 0xc7, 0x48, 0xbe, 0x00, 0xbc, 0xa7, 0xc0, 0xd8, 0x5d, - 0xc7, 0xde, 0xea, 0x47, 0x02, 0x16, 0x42, 0x73, 0x29, 0xc4, 0x1e, 0xc9, 0x65, 0x8c, 0x05, 0x69, - 0x5a, 0x39, 0x18, 0xe6, 0xf1, 0x04, 0xee, 0x79, 0x0e, 0x6a, 0x7e, 0x51, 0x3d, 0x0d, 0x49, 0xbc, - 0xf8, 0x49, 0x41, 0xf2, 0xe6, 0xf2, 0xfc, 0xdd, 0xec, 0x09, 0x32, 0x0c, 0x89, 0xc5, 0xbb, 0x9b, - 0x59, 0x45, 0xbd, 0x04, 0x63, 0xaf, 0x51, 0xcf, 0x31, 0x2b, 0xbd, 0x5d, 0x89, 0xdf, 0x46, 0x67, - 0x75, 0xcb, 0x43, 0x7d, 0xcb, 0xec, 0xc8, 0x23, 0xbd, 0x02, 0x79, 0x15, 0x06, 0x51, 0x9f, 0xf7, - 0x15, 0x5e, 0x88, 0x84, 0x04, 0xb0, 0xa3, 0x7a, 0x99, 0xb9, 0xb3, 0x82, 0x5c, 0xae, 0x86, 0x25, - 0xe5, 0xaa, 0x84, 0xbd, 0x8c, 0xcf, 0x0e, 0xc0, 0x78, 0x00, 0x2c, 0xce, 0x86, 0x8f, 0xda, 0xcb, - 0xb8, 0x01, 0x43, 0x68, 0x07, 0x7c, 0x2f, 0xe3, 0x52, 0x8f, 0x08, 0x4a, 0x7b, 0x22, 0xbe, 0x87, - 0xcd, 0xbb, 0x93, 0x25, 0xff, 0xb2, 0x23, 0x81, 0x78, 0xe6, 0xfa, 0xc1, 0xc3, 0xb8, 0x1d, 0xba, - 0xe8, 0x50, 0x9b, 0x90, 0x65, 0xad, 0x4b, 0xb4, 0xdc, 0xac, 0xfa, 0xb2, 0x10, 0xb2, 0xd5, 0xca, - 0x23, 0xb1, 0xd5, 0x7f, 0x33, 0x00, 0x27, 0xa5, 0x71, 0xc5, 0x96, 0xfb, 0xa2, 0x12, 0x39, 0x79, - 0xbc, 0xd4, 0x63, 0x52, 0xa1, 0xee, 0x7c, 0x18, 0x11, 0x4c, 0xfe, 0x18, 0x9b, 0xe4, 0xbb, 0x3f, - 0x7c, 0x40, 0x42, 0x05, 0x15, 0x1f, 0xd8, 0x62, 0xe5, 0x29, 0x64, 0x24, 0xea, 0xe4, 0x80, 0x70, - 0x82, 0x9f, 0x4f, 0x5f, 0x0d, 0x07, 0x84, 0x2f, 0xf7, 0x33, 0x50, 0xe7, 0x85, 0xcb, 0x5f, 0x24, - 0x61, 0x78, 0x63, 0xdf, 0xc2, 0x33, 0xd4, 0x3d, 0x18, 0x10, 0x22, 0x3c, 0xb2, 0x30, 0xcf, 0x88, - 0xf9, 0xdb, 0x7e, 0xf7, 0x26, 0xcf, 0xa1, 0x68, 0x9a, 0x46, 0x61, 0x73, 0x73, 0x85, 0xad, 0xfc, - 0xc0, 0xca, 0x92, 0x36, 0x60, 0x1a, 0xe4, 0x65, 0x3c, 0x3f, 0x38, 0x9e, 0x20, 0xb2, 0x3f, 0x57, - 0x9f, 0x77, 0x61, 0x27, 0x65, 0x6f, 0xdf, 0x2a, 0x19, 0xd4, 0xad, 0x38, 0x66, 0xc3, 0x33, 0x6d, - 0x4b, 0x38, 0x1d, 0x63, 0xde, 0xbe, 0xb5, 0xd4, 0xae, 0x25, 0x2f, 0xc0, 0x29, 0xab, 0x59, 0xe7, - 0x51, 0xc6, 0x3a, 0x33, 0x1e, 0x25, 0xba, 0x4f, 0x2b, 0x4d, 0x3f, 0x90, 0x3c, 0xa8, 0x4d, 0x59, - 0xcd, 0xfa, 0x7a, 0xd0, 0xba, 0x2c, 0x1a, 0xc9, 0x2c, 0x64, 0x58, 0x3f, 0x87, 0x72, 0xd7, 0x01, - 0x03, 0x81, 0x1a, 0x58, 0xcd, 0xba, 0xc6, 0x6b, 0xc8, 0x1c, 0x64, 0x19, 0x80, 0xde, 0xf4, 0xec, - 0x00, 0x8a, 0xab, 0xcc, 0x31, 0xab, 0x59, 0x9f, 0x6f, 0x7a, 0xb6, 0x0f, 0xf9, 0x2a, 0xa4, 0x0c, - 0xaa, 0x1b, 0x35, 0xd3, 0xe2, 0xf1, 0xb6, 0x7e, 0xa7, 0x1a, 0xf4, 0x62, 0xc6, 0xc6, 0xac, 0x37, - 0x6a, 0x66, 0xc5, 0xf4, 0x44, 0x54, 0x3b, 0x28, 0x33, 0x42, 0x75, 0xe6, 0x67, 0x96, 0xca, 0x2d, - 0x8f, 0xf2, 0x58, 0x5b, 0x42, 0x03, 0xac, 0x5a, 0x60, 0x35, 0xe4, 0x22, 0x8c, 0xd7, 0xf5, 0xfd, - 0x92, 0x0c, 0x04, 0x08, 0x34, 0x5a, 0xd7, 0xf7, 0xe7, 0xdb, 0x70, 0x67, 0x20, 0x8d, 0xe1, 0x28, - 0xb4, 0x93, 0x19, 0x3e, 0x0a, 0xab, 0x40, 0x3b, 0x79, 0x01, 0x46, 0x4d, 0x57, 0x04, 0x04, 0xcd, - 0x8a, 0x5e, 0xcb, 0x8d, 0xf0, 0x60, 0xbd, 0xe9, 0xde, 0x0c, 0xea, 0xd0, 0x26, 0x3a, 0xa6, 0xed, - 0x98, 0x5e, 0x0b, 0x63, 0x6b, 0xcc, 0x26, 0x8a, 0xb2, 0xfa, 0xcb, 0x09, 0xc8, 0xcc, 0x57, 0x3c, - 0x73, 0x8f, 0xde, 0x6b, 0x52, 0xa7, 0x45, 0xa6, 0x03, 0x79, 0x4a, 0x2f, 0x0c, 0x49, 0x42, 0xf1, - 0x29, 0x18, 0x62, 0x0b, 0x6b, 0x1a, 0xc8, 0xaa, 0x91, 0x85, 0xa5, 0x87, 0x93, 0xb5, 0x41, 0x26, - 0xbf, 0x4b, 0xda, 0xa0, 0xb7, 0x6f, 0xad, 0x18, 0x6c, 0xa3, 0xb8, 0xbb, 0xbe, 0xe7, 0xca, 0x3e, - 0xdb, 0x32, 0x98, 0x38, 0xbe, 0x0c, 0x3e, 0x01, 0x63, 0xa6, 0x5b, 0x32, 0x4c, 0xd7, 0x73, 0xcc, - 0x72, 0xb3, 0x7d, 0x35, 0x31, 0x6a, 0xba, 0x4b, 0xed, 0x4a, 0xb2, 0x00, 0x83, 0x8d, 0x6d, 0xff, - 0xb6, 0x61, 0xac, 0xeb, 0x35, 0x72, 0x70, 0x60, 0x68, 0x33, 0xa8, 0x70, 0x97, 0xf5, 0xd1, 0x78, - 0x57, 0xce, 0xd9, 0xe0, 0x4a, 0x4c, 0x99, 0x1b, 0x90, 0x2e, 0xb6, 0x4e, 0x43, 0x0a, 0x6f, 0xc7, - 0x2d, 0xdb, 0x42, 0xe1, 0x48, 0x6b, 0xc3, 0xee, 0x6e, 0x6d, 0xde, 0xb2, 0x2d, 0xf5, 0x09, 0x18, - 0x44, 0x34, 0x64, 0x14, 0xd2, 0x77, 0xb5, 0xe5, 0xbb, 0xf3, 0xda, 0xca, 0xda, 0x8d, 0xec, 0x09, - 0x56, 0x5c, 0xfe, 0xe4, 0xf2, 0xe2, 0xe6, 0x06, 0x2b, 0x2a, 0xea, 0xb3, 0x30, 0xc1, 0xfc, 0xd5, - 0x75, 0xea, 0xba, 0xd2, 0xed, 0x2a, 0x1b, 0xb4, 0xe9, 0x52, 0x47, 0x72, 0xdb, 0x82, 0xb2, 0xfa, - 0xaf, 0x49, 0x18, 0x16, 0xf0, 0x8f, 0xd4, 0x82, 0xcb, 0x34, 0x0c, 0x84, 0x69, 0x60, 0xfc, 0xaf, - 0xd4, 0x4c, 0x6a, 0x79, 0x41, 0x66, 0x00, 0x57, 0x01, 0xa3, 0xbc, 0x56, 0x5c, 0xf4, 0x93, 0x4b, - 0x90, 0xc5, 0x9b, 0xc5, 0x0a, 0x26, 0x54, 0x95, 0x10, 0x15, 0x3f, 0x83, 0x8c, 0x4b, 0xf5, 0x6b, - 0x0c, 0xe3, 0x3a, 0x8c, 0xe9, 0xb8, 0x04, 0x25, 0x71, 0xe7, 0x82, 0x59, 0x3a, 0x99, 0x70, 0x8c, - 0xe5, 0xe8, 0x35, 0x13, 0x5a, 0x7a, 0x54, 0x0f, 0xaa, 0xd8, 0xf6, 0x0f, 0x44, 0x6c, 0xe8, 0xf8, - 0x22, 0x76, 0x19, 0x4e, 0xd6, 0x74, 0xd7, 0x2b, 0x49, 0x54, 0xb5, 0xc4, 0x22, 0x8f, 0xb3, 0x86, - 0xce, 0x1d, 0x95, 0xc6, 0x5d, 0x23, 0xef, 0xa8, 0x88, 0x82, 0x80, 0x7e, 0x14, 0x44, 0xa6, 0x9b, - 0x82, 0x98, 0x07, 0x10, 0x74, 0x78, 0xfb, 0x16, 0x2a, 0x80, 0xcc, 0x55, 0x35, 0x2e, 0xaa, 0xc1, - 0x4d, 0x87, 0x96, 0xe6, 0xbd, 0x36, 0xf6, 0x2d, 0x0c, 0x4a, 0x47, 0xe7, 0xc3, 0x25, 0x97, 0xeb, - 0x8b, 0x89, 0xc8, 0xa4, 0x98, 0x14, 0xdf, 0x4a, 0xa6, 0x86, 0xb3, 0x29, 0xf5, 0xe7, 0x14, 0x38, - 0x29, 0x4b, 0x29, 0xf7, 0xc6, 0x1e, 0xa5, 0xec, 0x1d, 0x1d, 0x4f, 0xfa, 0x0d, 0x05, 0x26, 0xc3, - 0x3b, 0x46, 0xb8, 0x1c, 0x4b, 0x90, 0x72, 0x45, 0x9d, 0xf0, 0x39, 0xe2, 0x18, 0x24, 0xba, 0xfb, - 0xf7, 0x1d, 0x7e, 0x4f, 0x72, 0x2b, 0xe2, 0x27, 0xc4, 0xa9, 0x8c, 0x0e, 0x96, 0x84, 0x5d, 0x05, - 0x75, 0x17, 0xc8, 0xa2, 0x6e, 0x55, 0x68, 0x0d, 0xf9, 0xd9, 0xf3, 0x0c, 0x71, 0x11, 0x52, 0x7c, - 0x51, 0x4c, 0x7e, 0xa3, 0x98, 0x5e, 0xc8, 0x30, 0x7f, 0x15, 0x3b, 0x33, 0xbf, 0x13, 0x1b, 0x23, - 0xfb, 0x32, 0x11, 0xd1, 0x0d, 0x37, 0x60, 0x22, 0x34, 0xa4, 0xe0, 0x0d, 0x3b, 0xd0, 0x61, 0x35, - 0x35, 0xc4, 0x35, 0x72, 0x50, 0x6e, 0xc7, 0x36, 0x06, 0xa4, 0xd8, 0x86, 0xda, 0x82, 0x49, 0x8e, - 0x48, 0x4c, 0xb0, 0x27, 0xf5, 0x57, 0x00, 0x04, 0x13, 0x7d, 0xfa, 0x47, 0x78, 0x8e, 0x83, 0x40, - 0xb0, 0xb2, 0xa4, 0xa5, 0x05, 0x40, 0x8f, 0x39, 0xac, 0xc0, 0x54, 0x64, 0xe8, 0x07, 0x9e, 0xc5, - 0x63, 0x70, 0x86, 0x2d, 0xd2, 0x62, 0x90, 0x6c, 0x8b, 0x81, 0xbe, 0x20, 0x87, 0xc5, 0x97, 0x6b, - 0x94, 0x7a, 0xd3, 0x6b, 0xfd, 0x38, 0xe5, 0xfa, 0xf7, 0x15, 0x38, 0xdb, 0x9d, 0x56, 0x31, 0xfb, - 0xd5, 0x20, 0xa8, 0xc9, 0xc3, 0xb3, 0xa1, 0x43, 0xe6, 0x6e, 0xad, 0x20, 0xa7, 0x17, 0x17, 0xd6, - 0xa9, 0x63, 0xea, 0x35, 0xf3, 0x2d, 0x6a, 0x68, 0xb4, 0xca, 0x0c, 0x62, 0x2b, 0x90, 0x4d, 0xc4, - 0x71, 0x6c, 0x39, 0x0f, 0xb1, 0x28, 0x22, 0xe7, 0xa7, 0xe1, 0x14, 0x03, 0x61, 0x86, 0x77, 0xfd, - 0xde, 0xea, 0xf5, 0x9a, 0xfd, 0x66, 0xc0, 0xe1, 0x7f, 0x4a, 0x00, 0x11, 0xf5, 0x1a, 0xad, 0xdb, - 0x1e, 0xc5, 0x56, 0xb2, 0x05, 0xc3, 0x5b, 0x35, 0xfb, 0xcd, 0x52, 0xe0, 0xd6, 0xbe, 0x26, 0x5c, - 0x8d, 0x8f, 0xf6, 0xc5, 0xdc, 0x48, 0xaa, 0x74, 0x81, 0x21, 0x45, 0x97, 0x63, 0x88, 0x7f, 0x69, - 0x43, 0x0c, 0xfb, 0x8a, 0x41, 0xd6, 0x60, 0xd0, 0xb4, 0xb6, 0x6c, 0x7f, 0x92, 0x71, 0xb9, 0x2c, - 0x9d, 0x54, 0xca, 0xf9, 0x75, 0x1c, 0x4d, 0xfe, 0x3f, 0x14, 0x48, 0xa2, 0x4b, 0xfe, 0x28, 0x65, - 0x64, 0x01, 0xd2, 0x41, 0x86, 0xf1, 0xb1, 0xfc, 0xf3, 0x76, 0x37, 0x26, 0x2c, 0xe2, 0xa2, 0x95, - 0xdf, 0x54, 0x3c, 0x77, 0xbc, 0x99, 0x8b, 0xf3, 0x88, 0xc0, 0xa1, 0x9e, 0x87, 0x21, 0x71, 0xa6, - 0xce, 0xc0, 0xb0, 0xb6, 0xb9, 0xb6, 0xc6, 0x5d, 0x19, 0x80, 0xa1, 0x7b, 0x9b, 0xcb, 0x9b, 0xcb, - 0x4b, 0x59, 0x45, 0xfd, 0x8e, 0x02, 0xb9, 0x4e, 0x21, 0x10, 0xa2, 0xbb, 0x02, 0x83, 0x6c, 0x41, - 0xfa, 0xc9, 0x28, 0xed, 0x24, 0x26, 0x88, 0xca, 0xa1, 0xe4, 0x7c, 0x90, 0x72, 0xfb, 0x77, 0x0a, - 0x64, 0xd7, 0x1b, 0xba, 0x15, 0xba, 0x89, 0xb9, 0x10, 0x51, 0x70, 0x0b, 0xd0, 0x5e, 0xd9, 0x60, - 0x89, 0x34, 0x39, 0x97, 0x83, 0xeb, 0xba, 0xe7, 0xdf, 0x3f, 0x98, 0x7d, 0xf6, 0x78, 0x47, 0xd4, - 0xdb, 0xb4, 0x25, 0xa5, 0x80, 0xac, 0xb5, 0x53, 0x40, 0x12, 0x0f, 0x83, 0x51, 0x64, 0x8e, 0x30, - 0x85, 0x72, 0x52, 0x9a, 0x9d, 0x58, 0x8a, 0x59, 0xc8, 0xf0, 0x88, 0x40, 0xc5, 0x6e, 0x5a, 0x9e, - 0x08, 0x5d, 0x03, 0x56, 0x2d, 0xb2, 0x1a, 0xf2, 0x1c, 0x4c, 0xeb, 0x8d, 0x86, 0x63, 0xef, 0x9b, - 0x75, 0xdd, 0xa3, 0xcc, 0xc5, 0xde, 0x11, 0x8e, 0x09, 0xcf, 0x2b, 0x9b, 0x94, 0x5a, 0x97, 0x4c, - 0x77, 0x87, 0xfb, 0x27, 0xab, 0x90, 0xf1, 0x6c, 0x4f, 0xaf, 0x95, 0xfc, 0x5b, 0xa9, 0x8e, 0xcc, - 0x94, 0xe8, 0xf5, 0xd8, 0x6b, 0xff, 0x6b, 0x71, 0x11, 0x49, 0xf3, 0x93, 0x4e, 0xb1, 0x3f, 0xd6, - 0xa8, 0x1f, 0x85, 0x49, 0x91, 0x90, 0x10, 0xbe, 0x5c, 0xed, 0x67, 0x6d, 0xd4, 0xaf, 0x8f, 0xc2, - 0x54, 0xa4, 0x77, 0xe7, 0x45, 0x4c, 0xea, 0x83, 0xde, 0xb4, 0x7f, 0xae, 0xc0, 0x84, 0x9f, 0x34, - 0x21, 0xa7, 0x99, 0xa6, 0x51, 0x4a, 0xaf, 0xc7, 0x07, 0x04, 0x3b, 0x69, 0x2d, 0x04, 0x09, 0x19, - 0xdd, 0xd3, 0x4d, 0x23, 0xcd, 0x0f, 0x9e, 0x6e, 0xda, 0x88, 0x8c, 0x93, 0xff, 0xab, 0x34, 0x4f, - 0xce, 0x0d, 0xb2, 0xd0, 0x3a, 0xf2, 0x56, 0x94, 0x2e, 0x79, 0x2b, 0x3f, 0xa9, 0xc0, 0x94, 0x94, - 0x98, 0x56, 0x8a, 0xde, 0x10, 0xdd, 0x39, 0x3c, 0x98, 0x9d, 0xd8, 0x6c, 0x03, 0x3c, 0x74, 0x00, - 0x6a, 0xa2, 0x19, 0x45, 0x66, 0xb8, 0xe4, 0x77, 0x14, 0xb8, 0x28, 0x65, 0xb5, 0x75, 0x24, 0xc5, - 0x49, 0x64, 0x25, 0x90, 0xac, 0xff, 0x7d, 0x78, 0x30, 0x7b, 0xae, 0x9d, 0xf2, 0x16, 0x4e, 0x93, - 0x7b, 0x68, 0x1a, 0xcf, 0x39, 0xb1, 0x98, 0x0d, 0x97, 0x7c, 0x4e, 0x81, 0x5c, 0x38, 0x13, 0x4f, - 0x22, 0x31, 0x89, 0x24, 0xde, 0x3d, 0x3c, 0x98, 0x9d, 0x5c, 0x93, 0xf2, 0xf2, 0x1e, 0x9a, 0xac, - 0x49, 0xab, 0x03, 0x9b, 0xe1, 0x92, 0x7d, 0x20, 0x7e, 0x6e, 0x9e, 0x44, 0xc3, 0x20, 0xd2, 0x70, - 0xfb, 0xf0, 0x60, 0x76, 0x7c, 0x8d, 0x67, 0xea, 0x3d, 0xf4, 0xf0, 0xe3, 0x96, 0x8c, 0xc8, 0x70, - 0xc9, 0x97, 0x14, 0x38, 0x1d, 0xc9, 0x28, 0x94, 0x28, 0x18, 0x42, 0x0a, 0xd6, 0x0f, 0x0f, 0x66, - 0x4f, 0x6d, 0x86, 0x81, 0x1e, 0x9a, 0x92, 0x53, 0xcd, 0x6e, 0x08, 0x0d, 0x97, 0x7c, 0x41, 0x81, - 0x5c, 0x38, 0x75, 0x51, 0x22, 0x28, 0x8d, 0x04, 0x69, 0x87, 0x07, 0xb3, 0xd3, 0x77, 0xf6, 0x3e, - 0x50, 0x7a, 0xa6, 0xed, 0xbd, 0xae, 0xe4, 0x7c, 0x53, 0x01, 0xf5, 0xa8, 0xe4, 0x48, 0x89, 0xb0, - 0x61, 0x24, 0xec, 0x8d, 0xc3, 0x83, 0xd9, 0x99, 0x7b, 0x5d, 0x53, 0x25, 0x1f, 0x9a, 0xc0, 0x99, - 0xdd, 0x18, 0xbc, 0x86, 0x4b, 0xbe, 0xaa, 0xc0, 0xd9, 0xce, 0x5c, 0x4c, 0x89, 0xc4, 0x54, 0x7b, - 0x31, 0xb5, 0x70, 0x66, 0xe6, 0xc3, 0x2f, 0xa6, 0xd3, 0x0d, 0xa1, 0xe1, 0x62, 0xfe, 0x7c, 0x57, - 0x6d, 0xda, 0x2b, 0x7f, 0x3e, 0x13, 0xfb, 0x06, 0xab, 0xbb, 0xda, 0x96, 0x35, 0xa7, 0x14, 0xce, - 0xbd, 0x95, 0x4c, 0x29, 0xd9, 0x94, 0xfa, 0x22, 0x64, 0x6f, 0xda, 0xde, 0x03, 0xd8, 0xb4, 0xcf, - 0x0f, 0xc3, 0x49, 0xa9, 0xe7, 0x87, 0xf0, 0x82, 0xe5, 0xbb, 0x0a, 0x4c, 0x6d, 0xdb, 0x1e, 0x5f, - 0xbb, 0x2e, 0x0f, 0x27, 0x16, 0x63, 0x58, 0xd3, 0x41, 0x69, 0xbb, 0x26, 0x6c, 0xce, 0xee, 0x0a, - 0x73, 0x76, 0x32, 0xda, 0xfe, 0xc0, 0xf6, 0xec, 0xe4, 0x76, 0x74, 0xa4, 0xfc, 0x1e, 0xa4, 0x7c, - 0xf4, 0xe4, 0x63, 0x90, 0x34, 0xa8, 0x5b, 0x11, 0xfe, 0x89, 0xda, 0x25, 0xab, 0x12, 0xe1, 0xfc, - 0x20, 0x76, 0xe0, 0x31, 0x62, 0xaf, 0x23, 0x32, 0x85, 0x07, 0xba, 0x67, 0x0a, 0xe7, 0xff, 0x52, - 0x81, 0x51, 0x4c, 0x57, 0x0a, 0xd6, 0xeb, 0x51, 0xe7, 0x42, 0xbd, 0x01, 0xd0, 0x5e, 0x32, 0xb1, - 0x4e, 0xcf, 0x3f, 0xd0, 0x3a, 0x05, 0x8f, 0x0b, 0x7c, 0x88, 0xfc, 0xcf, 0x2a, 0xdc, 0x2d, 0x08, - 0x26, 0xd3, 0x97, 0x5b, 0xa0, 0xb1, 0x63, 0x88, 0xed, 0x04, 0xd4, 0xbc, 0x7c, 0x2c, 0x6a, 0x42, - 0xdc, 0xd3, 0x04, 0xa6, 0xfc, 0xff, 0x87, 0xe9, 0xee, 0xe2, 0xd4, 0x65, 0x3f, 0xdf, 0x09, 0xef, - 0xe7, 0x8f, 0x1c, 0x6b, 0x78, 0x79, 0xba, 0xf2, 0xd5, 0xcc, 0x25, 0x18, 0xe9, 0x37, 0x2b, 0xe8, - 0x5b, 0x83, 0x22, 0x3f, 0xf0, 0x43, 0xd9, 0xb3, 0xf2, 0x85, 0xe7, 0xc0, 0x23, 0xb8, 0xf0, 0xfc, - 0x53, 0x05, 0x26, 0x1d, 0x31, 0x91, 0x90, 0x4a, 0xe0, 0xf7, 0x96, 0x9f, 0xe8, 0x75, 0xc5, 0x2b, - 0xbd, 0xab, 0x11, 0x48, 0x8e, 0x50, 0x07, 0xd1, 0xf6, 0x07, 0x57, 0x07, 0x4e, 0x74, 0xa4, 0xfc, - 0x57, 0xa2, 0x82, 0x9c, 0x87, 0x94, 0x0f, 0xe5, 0x47, 0x95, 0x9c, 0x23, 0x85, 0xbc, 0xdb, 0xfb, - 0xd9, 0x57, 0xfd, 0x20, 0x43, 0xe2, 0xd8, 0x39, 0x96, 0x22, 0xac, 0xf0, 0x36, 0x4c, 0x77, 0x67, - 0x49, 0x17, 0x91, 0xbe, 0x1d, 0x16, 0xe9, 0xe7, 0xfb, 0x66, 0xfa, 0x11, 0xe2, 0x2c, 0x12, 0x52, - 0x9e, 0x06, 0xb2, 0xd4, 0x7e, 0xdb, 0xde, 0x33, 0x97, 0x60, 0x4e, 0xe8, 0xb6, 0xde, 0x90, 0xdf, - 0x19, 0x80, 0x11, 0x04, 0xf5, 0x9f, 0x39, 0x3f, 0x6a, 0x2d, 0xf8, 0x14, 0x9c, 0xa4, 0x56, 0xc5, - 0x69, 0xe1, 0x1d, 0xa4, 0x9f, 0x6e, 0x8e, 0x67, 0x74, 0x2d, 0xdb, 0x6e, 0x10, 0xf1, 0x8c, 0x59, - 0xff, 0xdc, 0xca, 0x13, 0x44, 0xf8, 0x11, 0x97, 0x1f, 0x45, 0x31, 0x87, 0xa4, 0x0d, 0xc0, 0xcf, - 0xc0, 0x49, 0x09, 0x80, 0x9f, 0x7c, 0xe7, 0x20, 0x2b, 0x22, 0xea, 0x3b, 0xb4, 0x25, 0xd0, 0xf0, - 0xd7, 0x44, 0xe2, 0x3e, 0xe3, 0x36, 0x6d, 0x71, 0x54, 0x61, 0x48, 0x8e, 0x6f, 0x28, 0x02, 0x89, - 0x38, 0xd5, 0xd7, 0x61, 0xcc, 0xe7, 0x6e, 0x90, 0xfc, 0xe6, 0x2b, 0x52, 0x1e, 0x42, 0x79, 0x32, - 0x36, 0x0f, 0xa8, 0xcd, 0x6d, 0x3f, 0xe2, 0xc1, 0x3b, 0xab, 0x2f, 0xc1, 0xc9, 0xf6, 0x7d, 0xeb, - 0xb1, 0x3c, 0x90, 0xef, 0xa6, 0x80, 0xc8, 0x5d, 0x05, 0x5d, 0x0d, 0xcc, 0x3d, 0x13, 0xb5, 0x82, - 0xb6, 0x5b, 0xf1, 0x39, 0x4a, 0x11, 0x14, 0x85, 0x45, 0xbb, 0x56, 0xa3, 0x15, 0x8f, 0x1a, 0x41, - 0x5b, 0xc7, 0x1b, 0x02, 0x69, 0x0c, 0xb2, 0x08, 0x80, 0xd7, 0x18, 0x0e, 0x75, 0xe9, 0xf1, 0xae, - 0x0e, 0xd3, 0xac, 0x9f, 0xc6, 0xba, 0x91, 0x17, 0x21, 0x67, 0x5a, 0x1e, 0x75, 0x2c, 0xbd, 0x56, - 0xd2, 0x1b, 0x0d, 0xbc, 0x98, 0x2a, 0x35, 0x1c, 0xba, 0x65, 0xee, 0x8b, 0xfb, 0xa9, 0x29, 0xbf, - 0x7d, 0xbe, 0xd1, 0x58, 0xd3, 0xeb, 0xf4, 0x2e, 0x36, 0x92, 0xcf, 0xc0, 0x08, 0x3e, 0x62, 0x63, - 0x0b, 0x66, 0x5b, 0xfe, 0x1d, 0xd5, 0xc6, 0xf1, 0x66, 0xbc, 0x2c, 0xde, 0x78, 0x07, 0x33, 0xdf, - 0x68, 0xa3, 0xec, 0x98, 0x7b, 0x68, 0xbc, 0xfc, 0x9f, 0x29, 0x30, 0xe3, 0x77, 0xef, 0xc2, 0x2f, - 0x9e, 0x67, 0x9f, 0x62, 0xf2, 0x15, 0x24, 0x35, 0x46, 0xe2, 0x30, 0xbb, 0xb5, 0x42, 0xf7, 0x8e, - 0x7e, 0x06, 0xde, 0x0e, 0x6d, 0x2d, 0xe9, 0x9e, 0x2e, 0x5b, 0xaa, 0x81, 0x0f, 0xd8, 0x52, 0xe5, - 0x7f, 0x6d, 0x00, 0xce, 0xc6, 0xad, 0x3b, 0x31, 0xda, 0x7a, 0x2f, 0x73, 0x75, 0xf5, 0xc1, 0xd8, - 0x1b, 0x3b, 0x4d, 0xd4, 0xa5, 0x9f, 0xc4, 0x6b, 0x3b, 0xdc, 0xf4, 0x0b, 0x37, 0xf9, 0xb5, 0xdd, - 0xfb, 0x07, 0xb3, 0xaf, 0x1c, 0x53, 0xff, 0xd4, 0xbd, 0xeb, 0xa6, 0x55, 0xa5, 0x4e, 0xc3, 0x31, - 0x2d, 0x4f, 0x5c, 0xfc, 0xbd, 0xe2, 0xe7, 0x67, 0x0f, 0x74, 0x5e, 0xd5, 0x75, 0x5f, 0x81, 0x50, - 0x5e, 0x76, 0xfe, 0x07, 0x0a, 0x5c, 0xec, 0x4f, 0x4c, 0x88, 0xc6, 0xb7, 0xa0, 0x2b, 0xaf, 0xf8, - 0xd3, 0x91, 0xf1, 0xfa, 0x92, 0xb4, 0x34, 0xa2, 0x79, 0xd4, 0x6b, 0xaf, 0x7e, 0x6e, 0x00, 0xf2, - 0xc1, 0xfc, 0x43, 0x36, 0xa7, 0x61, 0x3b, 0x1e, 0x19, 0x0b, 0x92, 0x13, 0x12, 0xc8, 0xc9, 0xb3, - 0x90, 0xae, 0xd8, 0xf5, 0x46, 0x8d, 0x7a, 0xd4, 0x10, 0x29, 0x84, 0xed, 0x0a, 0x72, 0x0d, 0xa6, - 0x02, 0xdd, 0x50, 0xda, 0x6a, 0x2f, 0x83, 0xb8, 0x54, 0x9a, 0x0c, 0x1a, 0xa5, 0x25, 0x22, 0xaf, - 0x40, 0xae, 0xdd, 0x49, 0xfa, 0x45, 0x17, 0x36, 0x5d, 0x7c, 0x3e, 0x82, 0x0c, 0x51, 0xb4, 0x69, - 0xb7, 0x0b, 0x99, 0x98, 0xf8, 0x35, 0xe2, 0x70, 0xdd, 0x49, 0x8d, 0x92, 0xee, 0x89, 0x47, 0xf1, - 0xfd, 0x29, 0xa1, 0x4c, 0xd0, 0x73, 0xde, 0x53, 0x3f, 0x0d, 0x4f, 0x2e, 0x3a, 0x54, 0xf7, 0xe8, - 0xd1, 0xfc, 0xf0, 0x95, 0xf4, 0x91, 0x13, 0x55, 0x8e, 0x9e, 0xa8, 0xda, 0x82, 0xb9, 0xde, 0xf8, - 0x85, 0x26, 0x7f, 0x0d, 0x86, 0x1c, 0xac, 0x11, 0x22, 0xf4, 0x7c, 0x3f, 0x9b, 0xae, 0x13, 0x9d, - 0x40, 0xa2, 0x3e, 0x0e, 0xea, 0xd1, 0x50, 0xc1, 0xf5, 0xd0, 0xff, 0x83, 0x0b, 0xb1, 0x50, 0x82, - 0xb6, 0x4d, 0x18, 0xe6, 0x68, 0x7d, 0x13, 0xf3, 0x60, 0xc4, 0xf9, 0x1a, 0x4e, 0xe0, 0x52, 0x7f, - 0x53, 0x81, 0xc9, 0x6e, 0xd0, 0x1d, 0x32, 0x78, 0x24, 0xf3, 0x07, 0x62, 0xa4, 0xec, 0x06, 0x8c, - 0x54, 0xfc, 0x6d, 0xc7, 0xa4, 0xe4, 0x38, 0xa6, 0x2a, 0x13, 0xf4, 0x9c, 0xf7, 0x33, 0x86, 0x5f, - 0x87, 0x33, 0xdd, 0x67, 0xc6, 0xe5, 0xe3, 0xa5, 0x18, 0x99, 0xe6, 0x13, 0x39, 0x42, 0x9a, 0xd5, - 0x5d, 0x38, 0xdb, 0x1d, 0x71, 0xf0, 0xa8, 0x3f, 0x23, 0xe1, 0x13, 0x0a, 0xad, 0x78, 0xdc, 0x05, - 0x90, 0x71, 0xa8, 0xd7, 0x20, 0x77, 0xcb, 0x2e, 0xfb, 0x37, 0x93, 0xe2, 0xb2, 0xa9, 0x97, 0x23, - 0xf9, 0x2f, 0x0a, 0x9c, 0xee, 0xd2, 0xeb, 0x43, 0x38, 0x57, 0x7d, 0x0a, 0x46, 0x9c, 0xa6, 0x65, - 0x99, 0x56, 0xb5, 0x74, 0xdf, 0x2e, 0xfb, 0x67, 0xd9, 0xb8, 0x8c, 0xc6, 0x23, 0xe9, 0xc4, 0x96, - 0x8c, 0xc0, 0x76, 0xcb, 0x2e, 0xbb, 0xf9, 0x29, 0x48, 0xdc, 0xb2, 0xcb, 0x51, 0x91, 0x53, 0x2f, - 0x41, 0xf6, 0x96, 0x5d, 0x0e, 0xb3, 0x66, 0x0a, 0x86, 0xee, 0xdb, 0xe5, 0xf6, 0x8a, 0x0e, 0xde, - 0xb7, 0xcb, 0x2b, 0x86, 0xba, 0x0c, 0x27, 0x25, 0x50, 0xc1, 0x8f, 0x67, 0x20, 0x71, 0xdf, 0x2e, - 0x8b, 0xbd, 0x3c, 0x13, 0x31, 0x07, 0xf8, 0xbb, 0x59, 0xfc, 0x37, 0xb4, 0x90, 0x20, 0x06, 0xaa, - 0x7e, 0x14, 0x26, 0xd1, 0x39, 0x5a, 0xbf, 0xb7, 0x7a, 0xec, 0x0b, 0x31, 0xf5, 0x14, 0x4c, 0x45, - 0x3a, 0x73, 0x3a, 0x2e, 0xcf, 0x01, 0xb4, 0xf3, 0xd2, 0xc9, 0x24, 0x64, 0x6f, 0xdc, 0xd1, 0xee, - 0x6c, 0x6e, 0xac, 0xac, 0x2d, 0x97, 0xd6, 0x37, 0xe6, 0x17, 0x6f, 0xaf, 0x67, 0x4f, 0xa8, 0xc9, - 0x94, 0x92, 0x55, 0x2e, 0x3f, 0x0e, 0x29, 0x3f, 0xcf, 0x5b, 0x4a, 0x74, 0x1e, 0x03, 0x08, 0x7a, - 0xac, 0x67, 0x95, 0xab, 0xff, 0x75, 0x25, 0xb8, 0x8b, 0xfc, 0xba, 0x02, 0x23, 0xf2, 0xef, 0xd8, - 0x90, 0x42, 0x7f, 0xbf, 0x54, 0xe3, 0xcf, 0x2c, 0x5f, 0xec, 0x1b, 0x9e, 0x4f, 0x46, 0x7d, 0xf2, - 0xdd, 0xbf, 0xfe, 0xc7, 0x5f, 0x1c, 0x38, 0x4f, 0x66, 0x8b, 0xe2, 0x5c, 0x51, 0x94, 0x7f, 0xe6, - 0xa6, 0xf8, 0xb6, 0xe0, 0xd3, 0x3b, 0xe4, 0xa7, 0x14, 0x18, 0xf6, 0xcf, 0x3b, 0x71, 0x99, 0xa6, - 0xe1, 0x5f, 0xc5, 0xc9, 0x5f, 0xee, 0x07, 0x54, 0xd0, 0xa2, 0x22, 0x2d, 0x67, 0x49, 0x3e, 0xa0, - 0x45, 0x3c, 0x27, 0x91, 0xc8, 0x28, 0xc3, 0xb0, 0xf8, 0x49, 0x8b, 0x58, 0x2a, 0xc2, 0xbf, 0xec, - 0x11, 0x4b, 0x45, 0xe4, 0x17, 0x32, 0xd4, 0x13, 0xc4, 0x81, 0x41, 0xfc, 0xa1, 0x14, 0xf2, 0x64, - 0xef, 0x9f, 0x52, 0xe1, 0xf8, 0xe7, 0xfa, 0xfd, 0xcd, 0x15, 0x75, 0x1a, 0xe7, 0x98, 0x25, 0x63, - 0xc1, 0x1c, 0xf9, 0x6f, 0xba, 0x7c, 0x06, 0x92, 0x98, 0xbf, 0x7e, 0xb1, 0x07, 0x26, 0x7f, 0xc4, - 0x63, 0xfd, 0x7c, 0x8c, 0x7a, 0x0e, 0x47, 0xcd, 0x93, 0x5c, 0x78, 0x54, 0x89, 0xaf, 0xef, 0xf0, - 0x1f, 0xf7, 0xc0, 0x9c, 0x65, 0xf2, 0x54, 0x7f, 0x99, 0xcd, 0x47, 0x53, 0x72, 0x64, 0x1a, 0xb4, - 0x3a, 0x85, 0x94, 0x8c, 0x93, 0xd1, 0x80, 0x12, 0x47, 0xdf, 0xf2, 0xc8, 0x67, 0x15, 0x18, 0xe2, - 0xb1, 0x2a, 0xd2, 0xf3, 0x61, 0x77, 0xc0, 0xf5, 0x4b, 0x7d, 0x40, 0x8a, 0x61, 0xcf, 0xe3, 0xb0, - 0x67, 0xc8, 0x69, 0x69, 0x58, 0x06, 0x20, 0x71, 0xc0, 0x85, 0x21, 0xfe, 0x28, 0x34, 0x96, 0x82, - 0xd0, 0xbb, 0xd1, 0xbc, 0xfc, 0xf6, 0x46, 0xfc, 0xa6, 0xdf, 0x8a, 0xb5, 0x65, 0x0b, 0xae, 0x77, - 0x0e, 0x2a, 0x7e, 0xfe, 0xaf, 0x3d, 0xe8, 0x2f, 0x29, 0x90, 0x91, 0x5e, 0x33, 0x92, 0xa7, 0xfb, - 0x7b, 0xf5, 0xe8, 0x8f, 0x5f, 0xe8, 0x17, 0x5c, 0xb0, 0xe1, 0x22, 0x52, 0x74, 0x8e, 0xcc, 0x04, - 0x14, 0xf1, 0xcb, 0x6b, 0x74, 0x93, 0x25, 0xb2, 0xbe, 0xa2, 0x40, 0x3a, 0x78, 0x6e, 0x16, 0x2b, - 0x0e, 0xd1, 0x47, 0x76, 0xb1, 0xe2, 0xd0, 0xf1, 0x02, 0x4e, 0xbd, 0x84, 0x04, 0x5d, 0x20, 0xe7, - 0x03, 0x82, 0x74, 0x1f, 0x06, 0x45, 0x54, 0xa2, 0xe9, 0x3d, 0x05, 0xc6, 0xc2, 0xcf, 0x11, 0xc9, - 0x33, 0x7d, 0x8d, 0x25, 0x05, 0x36, 0xf3, 0xcf, 0x1e, 0xa3, 0x87, 0x20, 0xf1, 0x29, 0x24, 0xf1, - 0x09, 0x72, 0xa1, 0x0b, 0x89, 0x28, 0x44, 0xc5, 0xb7, 0xfd, 0x10, 0xe5, 0x3b, 0xe4, 0xf3, 0x0a, - 0x8c, 0xc8, 0x39, 0x74, 0xb1, 0x0a, 0xbc, 0x4b, 0x96, 0x6c, 0xac, 0x02, 0xef, 0x96, 0x23, 0xa8, - 0x9e, 0x46, 0xf2, 0x26, 0xc8, 0xc9, 0x80, 0xbc, 0x20, 0xf1, 0xef, 0x6b, 0x22, 0x17, 0x6c, 0xd5, - 0xae, 0xe8, 0xb5, 0x0f, 0x8f, 0xa2, 0x59, 0xa4, 0xe8, 0x34, 0x39, 0x15, 0x50, 0xc4, 0xd8, 0x55, - 0x2b, 0x05, 0x74, 0x7d, 0x5d, 0x81, 0x8c, 0x94, 0xd2, 0x17, 0x2b, 0xf4, 0x9d, 0xd9, 0x86, 0xb1, - 0x42, 0xdf, 0x25, 0x53, 0x50, 0xbd, 0x8c, 0xf4, 0x3c, 0xae, 0x4a, 0x26, 0x0e, 0xa1, 0x78, 0xbe, - 0x68, 0x5b, 0xc2, 0x5e, 0x56, 0x2e, 0x93, 0xdf, 0x12, 0xa9, 0x98, 0xd1, 0x94, 0x35, 0xf2, 0x42, - 0x0f, 0x2e, 0x1c, 0x91, 0x8f, 0x97, 0x7f, 0xf1, 0xd8, 0xfd, 0x8e, 0x34, 0x86, 0xed, 0x6c, 0xb8, - 0x92, 0xc8, 0x78, 0xfb, 0x3d, 0x05, 0x4e, 0x07, 0x0b, 0xfc, 0xe3, 0x27, 0x79, 0x0e, 0x49, 0x56, - 0xc9, 0xb9, 0xc8, 0xc2, 0x77, 0x12, 0xfe, 0x2b, 0x0a, 0x64, 0xa3, 0xa9, 0x55, 0xe4, 0x6a, 0x8f, - 0x71, 0xbb, 0x24, 0xe3, 0xe5, 0xaf, 0x1d, 0xab, 0x8f, 0xa0, 0x73, 0x06, 0xe9, 0xcc, 0x91, 0xe9, - 0xb6, 0x9f, 0x61, 0xba, 0x9e, 0xbb, 0x5b, 0x2b, 0xf1, 0x84, 0xac, 0x6f, 0xe2, 0xcf, 0x93, 0x09, - 0xb6, 0xfe, 0x78, 0x48, 0x7c, 0x1c, 0x49, 0x9c, 0x21, 0x67, 0x23, 0xac, 0x0c, 0x13, 0xfa, 0x0d, - 0x05, 0x46, 0x43, 0x79, 0xa5, 0xa4, 0xd8, 0x73, 0x6f, 0x84, 0x93, 0x5f, 0xf3, 0xcf, 0xf4, 0xdf, - 0x41, 0x90, 0x76, 0x05, 0x49, 0xbb, 0xa8, 0x9e, 0x8f, 0x6e, 0x27, 0xb1, 0xbf, 0xc3, 0x1b, 0xea, - 0xb3, 0x0a, 0xa4, 0x83, 0x94, 0xad, 0x58, 0x4b, 0x12, 0x4d, 0x5b, 0x8b, 0xb5, 0x24, 0x1d, 0x59, - 0x60, 0x6a, 0x0e, 0xc9, 0x22, 0x6a, 0xdb, 0xb1, 0x70, 0x1b, 0xba, 0xc5, 0x48, 0xf8, 0x0c, 0xba, - 0xd7, 0x95, 0x9d, 0x78, 0xd7, 0x22, 0xf4, 0x76, 0x35, 0x1f, 0xe7, 0xf9, 0xc9, 0x6f, 0xad, 0xbb, - 0xd8, 0x78, 0x17, 0x11, 0x49, 0x86, 0xeb, 0x27, 0x14, 0x18, 0x16, 0x4f, 0x24, 0x63, 0x7d, 0xd6, - 0xf0, 0x33, 0xca, 0xfe, 0x49, 0xe8, 0xd4, 0x14, 0x0d, 0x8e, 0x29, 0x42, 0x83, 0x78, 0x54, 0x19, - 0x4b, 0x43, 0xf8, 0xe1, 0xe5, 0xc3, 0xd0, 0x50, 0xe7, 0x98, 0x24, 0x1a, 0x7e, 0x5a, 0x81, 0x94, - 0xff, 0x90, 0x95, 0xc4, 0x79, 0xe4, 0x91, 0xb7, 0xb8, 0xf9, 0xa7, 0xfa, 0x82, 0x15, 0x94, 0x74, - 0xba, 0xba, 0x78, 0x81, 0x11, 0xf6, 0xb9, 0x46, 0xe4, 0x87, 0xd5, 0xf1, 0x16, 0xb1, 0xf3, 0xc5, - 0x76, 0xbc, 0x45, 0xec, 0xf2, 0x62, 0x5b, 0xbd, 0x80, 0x34, 0x3d, 0x46, 0xce, 0x48, 0xbb, 0xb9, - 0x1a, 0x25, 0xeb, 0x4b, 0x0a, 0x0c, 0x8b, 0xde, 0xb1, 0x4b, 0x14, 0x7e, 0xc1, 0x9d, 0x7f, 0x3a, - 0x1e, 0x34, 0xf2, 0x7e, 0xdd, 0x37, 0x86, 0x44, 0x8d, 0x21, 0xa5, 0xf8, 0x36, 0xab, 0x78, 0x87, - 0x9d, 0x49, 0x56, 0xed, 0xaa, 0x1b, 0x7b, 0x26, 0x91, 0x1e, 0xf9, 0x1f, 0x97, 0x94, 0x6e, 0x7e, - 0x42, 0x55, 0xe6, 0xc8, 0x57, 0x15, 0xfc, 0xad, 0xba, 0x76, 0x4e, 0x4a, 0xac, 0x7a, 0xeb, 0x96, - 0x5e, 0x19, 0xab, 0xde, 0xba, 0xa6, 0xbb, 0x74, 0x31, 0x0e, 0x22, 0x9f, 0x50, 0xbc, 0xba, 0x7c, - 0x57, 0x81, 0x74, 0x70, 0xb1, 0x1e, 0xab, 0xd0, 0xa2, 0x79, 0x31, 0xb1, 0x0a, 0xad, 0xe3, 0xae, - 0x5e, 0xcd, 0x23, 0x21, 0x93, 0x84, 0x04, 0x84, 0x6c, 0xdb, 0x9e, 0x20, 0xe2, 0x1d, 0x18, 0xe4, - 0x1e, 0xf0, 0x93, 0xbd, 0xef, 0x4a, 0x7b, 0x9f, 0x50, 0xc3, 0xfe, 0xee, 0x11, 0x47, 0x25, 0xd9, - 0xcb, 0x7d, 0x4f, 0x81, 0x8c, 0x1c, 0x5c, 0x8c, 0xcf, 0x7e, 0x8e, 0x06, 0xf6, 0xf2, 0x1f, 0xeb, - 0x04, 0x97, 0x7f, 0x84, 0x3c, 0xf4, 0xf3, 0xe4, 0x52, 0x7f, 0x1e, 0xf1, 0xec, 0x72, 0x86, 0x91, - 0x7f, 0xdb, 0xbc, 0x2d, 0x3d, 0xec, 0x48, 0xc9, 0xaf, 0x1c, 0x7b, 0xe8, 0x7d, 0xe9, 0xce, 0x37, - 0xf6, 0x48, 0x19, 0xbe, 0xbf, 0xec, 0xaa, 0xf9, 0x19, 0x80, 0x44, 0xc2, 0xcf, 0x28, 0x18, 0x2a, - 0xf2, 0xef, 0xf9, 0xae, 0xf4, 0x79, 0xe9, 0xd3, 0x7b, 0x37, 0x75, 0x5e, 0x11, 0xa9, 0x67, 0x90, - 0x9c, 0x29, 0x32, 0x21, 0x1b, 0x22, 0x7f, 0xe4, 0xef, 0x2b, 0x70, 0xae, 0x57, 0xd8, 0x9c, 0x2c, - 0xc4, 0xb9, 0x02, 0xfd, 0xc5, 0xf4, 0xf3, 0x8b, 0x0f, 0x85, 0x23, 0xac, 0x2e, 0xd5, 0x9c, 0x34, - 0x95, 0xba, 0xc7, 0x56, 0x59, 0x84, 0xb9, 0x99, 0x55, 0xff, 0x63, 0xe5, 0xa8, 0x20, 0x2f, 0x52, - 0xe2, 0x92, 0x8f, 0x3f, 0x50, 0x40, 0x3d, 0x60, 0xff, 0x2b, 0x0f, 0xda, 0xfd, 0x48, 0x3b, 0x14, - 0x99, 0x04, 0xf9, 0xc3, 0xa3, 0x62, 0xf5, 0x2f, 0x1c, 0x7b, 0xe8, 0xde, 0x8e, 0x7b, 0x5c, 0x3c, - 0x5c, 0x7d, 0x0e, 0x69, 0x2d, 0x90, 0x2b, 0x1d, 0xb4, 0x16, 0xdf, 0x3e, 0x2a, 0x04, 0xff, 0x0e, - 0xf9, 0x96, 0x82, 0x51, 0xda, 0x70, 0x54, 0x98, 0x5c, 0x3b, 0x5e, 0x0c, 0x99, 0x53, 0xfe, 0xdc, - 0x83, 0x04, 0x9e, 0xbb, 0xc4, 0x2e, 0xef, 0xdb, 0xe5, 0x92, 0x23, 0x80, 0xc3, 0x9e, 0x47, 0x3a, - 0x88, 0x27, 0xc7, 0xea, 0xec, 0x68, 0x80, 0x3a, 0x56, 0x67, 0x77, 0x84, 0xa8, 0xd5, 0xc7, 0x90, - 0xa2, 0x53, 0x64, 0x4a, 0xa6, 0xa8, 0xf8, 0x36, 0x0f, 0x71, 0xbf, 0xc3, 0x0e, 0xe4, 0xa3, 0xa1, - 0x98, 0x72, 0xac, 0x41, 0xeb, 0x16, 0xba, 0x8e, 0x35, 0x68, 0x5d, 0xc3, 0xd5, 0xbe, 0x9e, 0x52, - 0xdb, 0x06, 0x0d, 0xf3, 0x0c, 0xdc, 0xdd, 0x1a, 0x46, 0x7d, 0x5e, 0x56, 0x2e, 0x2f, 0x5c, 0xfe, - 0xde, 0x3f, 0xcc, 0x9c, 0xf8, 0xde, 0xe1, 0x8c, 0xf2, 0xfd, 0xc3, 0x19, 0xe5, 0x07, 0x87, 0x33, - 0xca, 0xdf, 0x1f, 0xce, 0x28, 0x5f, 0xfe, 0xd1, 0xcc, 0x89, 0xef, 0xff, 0x68, 0xe6, 0xc4, 0x0f, - 0x7e, 0x34, 0x73, 0xe2, 0x8d, 0x94, 0x3f, 0x48, 0x79, 0x08, 0x6f, 0x79, 0xae, 0xfd, 0x77, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x2f, 0xc2, 0x29, 0x78, 0x16, 0x63, 0x00, 0x00, + // 6981 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7d, 0x6b, 0x6c, 0x1c, 0xd7, + 0x95, 0xa6, 0x8a, 0xdd, 0x24, 0xbb, 0x4f, 0xf3, 0xd1, 0xba, 0x7c, 0xa8, 0xd5, 0x92, 0x49, 0xa9, + 0x64, 0xcb, 0x94, 0x6c, 0x37, 0x6d, 0xc9, 0xaf, 0x38, 0x89, 0x63, 0xbe, 0x24, 0x51, 0xa2, 0x29, + 0xa9, 0x48, 0xae, 0x03, 0x67, 0x37, 0xbd, 0xd5, 0x5d, 0x97, 0xcd, 0x12, 0xab, 0xab, 0x9a, 0x55, + 0xd5, 0x34, 0xdb, 0x5e, 0x67, 0xb3, 0xde, 0x57, 0x36, 0x9b, 0xcd, 0x6b, 0x37, 0x40, 0x16, 0xbb, + 0xd8, 0x0d, 0xfc, 0x23, 0x59, 0xec, 0x62, 0x83, 0x04, 0x0b, 0x2c, 0x76, 0x7f, 0xcc, 0x03, 0x83, + 0xc1, 0x24, 0x83, 0x01, 0x06, 0x01, 0x66, 0x7e, 0x04, 0x33, 0x00, 0x33, 0xc3, 0x0c, 0x06, 0x03, + 0xcc, 0x8f, 0xc1, 0x60, 0xe6, 0x97, 0x81, 0x01, 0x06, 0xf7, 0xdc, 0x5b, 0xd5, 0xb7, 0xaa, 0x9b, + 0xd5, 0x4d, 0xc9, 0x72, 0xe6, 0x87, 0xcd, 0xae, 0xfb, 0x38, 0xf7, 0xbb, 0xe7, 0x9e, 0x7b, 0xce, + 0xb9, 0xe7, 0x9e, 0x2a, 0xc1, 0x79, 0x8f, 0xba, 0xfb, 0xd4, 0x9d, 0xe7, 0x7f, 0x1a, 0x95, 0x79, + 0xcf, 0xd7, 0xfd, 0xa6, 0x57, 0x6a, 0xb8, 0x8e, 0xef, 0x90, 0xb3, 0x55, 0xa7, 0xba, 0xeb, 0x3a, + 0x7a, 0x75, 0xa7, 0xc4, 0x1b, 0x94, 0x82, 0x76, 0xc5, 0x7c, 0xa5, 0x69, 0x5a, 0xc6, 0xbc, 0x69, + 0x6f, 0x3b, 0xbc, 0x71, 0x71, 0xa2, 0xe6, 0x78, 0x9e, 0xd9, 0x98, 0xe7, 0x7f, 0x44, 0xe1, 0xf4, + 0x03, 0xa7, 0xe2, 0xcd, 0xb3, 0xff, 0x35, 0x2a, 0xf8, 0x47, 0x94, 0x9f, 0x41, 0xaa, 0x8d, 0xca, + 0xbc, 0xde, 0x68, 0x94, 0xd9, 0x98, 0x41, 0x05, 0x09, 0x2a, 0x0c, 0xdd, 0xd7, 0x45, 0xd9, 0x6c, + 0x50, 0x66, 0xda, 0x06, 0x3d, 0x28, 0x37, 0x3d, 0xbd, 0x46, 0x23, 0x9d, 0xa6, 0x83, 0x06, 0x75, + 0xea, 0xeb, 0x52, 0xc7, 0x6b, 0x62, 0x76, 0x86, 0xa9, 0xd7, 0x6c, 0xc7, 0xf3, 0xcd, 0xaa, 0x27, + 0xff, 0x66, 0xa3, 0xb4, 0x9f, 0x44, 0x9f, 0x4b, 0x01, 0x47, 0x90, 0x11, 0xe2, 0x4f, 0x8c, 0x31, + 0xc5, 0x8b, 0xde, 0x9e, 0x35, 0x5f, 0x75, 0x6c, 0x9f, 0xda, 0xbe, 0xe9, 0xd8, 0x8d, 0x8a, 0xf4, + 0x20, 0x9a, 0x9c, 0x65, 0x4d, 0xe8, 0x01, 0xad, 0x9a, 0xf6, 0xb6, 0xab, 0xe3, 0x4c, 0x4d, 0x51, + 0xf5, 0x84, 0xe7, 0x3b, 0xae, 0x5e, 0xa3, 0xf3, 0xd4, 0xae, 0x99, 0x36, 0x6d, 0x54, 0xc4, 0x0f, + 0x51, 0x7d, 0xae, 0xa3, 0xba, 0xbe, 0x5f, 0xad, 0x8a, 0xca, 0x99, 0x8e, 0x4a, 0xd7, 0xa9, 0xee, + 0x7a, 0x46, 0x45, 0xd4, 0x5f, 0xd9, 0xdd, 0x9f, 0xdf, 0xdd, 0x17, 0x73, 0x08, 0x7e, 0x34, 0x2a, + 0xf3, 0x16, 0xd5, 0x3d, 0xce, 0xb4, 0x70, 0x12, 0xea, 0x31, 0x4d, 0x59, 0xa3, 0x00, 0x4b, 0x49, + 0x6e, 0x63, 0x99, 0xfb, 0xd4, 0xa6, 0x9e, 0x17, 0xfe, 0x60, 0x74, 0xc5, 0x4f, 0xd1, 0xbe, 0xd0, + 0xf4, 0x4d, 0x6b, 0xde, 0x72, 0x6a, 0xec, 0x3f, 0x56, 0xed, 0xd4, 0x44, 0x4d, 0x11, 0x6b, 0x9a, + 0xb6, 0x4b, 0x3d, 0xc7, 0xda, 0xa7, 0x46, 0x59, 0x37, 0x0c, 0x37, 0xd2, 0x6b, 0xc7, 0xaa, 0xce, + 0xfb, 0x66, 0x9d, 0x7a, 0xbe, 0x5e, 0x0f, 0xe4, 0x67, 0x86, 0xfa, 0x55, 0x63, 0xde, 0xd5, 0xb7, + 0xfd, 0xf9, 0xfd, 0xeb, 0xf8, 0x97, 0xcd, 0x58, 0xdf, 0xf6, 0x45, 0xfd, 0x64, 0xcd, 0xa9, 0x39, + 0xf8, 0x73, 0x9e, 0xfd, 0x12, 0xa5, 0xe7, 0x6b, 0x8e, 0x53, 0xb3, 0x28, 0x63, 0xf9, 0xbc, 0x6e, + 0xdb, 0x8e, 0xaf, 0xb3, 0x85, 0x09, 0x30, 0xce, 0x8a, 0x5a, 0x7c, 0xaa, 0x34, 0xb7, 0xe3, 0x83, + 0xaa, 0x25, 0x98, 0x58, 0xa2, 0xae, 0x6f, 0x6e, 0x9b, 0x55, 0xdd, 0xa7, 0x9e, 0x46, 0xf7, 0x9a, + 0xd4, 0xf3, 0xc9, 0x19, 0x18, 0xb6, 0x1d, 0x83, 0x96, 0x4d, 0xa3, 0xa0, 0x5c, 0x50, 0xe6, 0xb2, + 0xda, 0x10, 0x7b, 0x5c, 0x35, 0xd4, 0xbf, 0x4b, 0x03, 0x91, 0x3a, 0x2c, 0x53, 0x5f, 0x37, 0x2d, + 0x8f, 0xdc, 0x87, 0xb4, 0xdf, 0x6a, 0x50, 0x6c, 0x3c, 0x76, 0xed, 0xb3, 0xa5, 0x63, 0x37, 0x53, + 0xa9, 0xb3, 0xb3, 0x5c, 0xb4, 0xd9, 0x6a, 0x50, 0x0d, 0x49, 0x91, 0x4b, 0x30, 0x4a, 0x5d, 0xd7, + 0x71, 0xcb, 0x75, 0xea, 0xb1, 0x5d, 0x50, 0x18, 0x40, 0x20, 0x23, 0x58, 0xf8, 0x26, 0x2f, 0x23, + 0x04, 0xd2, 0x6c, 0x0f, 0x14, 0x52, 0x17, 0x94, 0xb9, 0x11, 0x0d, 0x7f, 0x13, 0x0d, 0x86, 0xb6, + 0x4d, 0x6a, 0x19, 0x5e, 0x21, 0x7d, 0x21, 0x35, 0x97, 0xbb, 0xf6, 0xe2, 0xc9, 0xd0, 0xdc, 0xc0, + 0xbe, 0x8b, 0xe9, 0x9f, 0x1e, 0xce, 0x9e, 0xd2, 0x04, 0xa5, 0xe2, 0xff, 0x19, 0x80, 0x21, 0x5e, + 0x41, 0xa6, 0x61, 0xc8, 0xf4, 0xbc, 0x26, 0x75, 0x03, 0xce, 0xf0, 0x27, 0x52, 0x80, 0x61, 0xaf, + 0x59, 0x79, 0x40, 0xab, 0xbe, 0x40, 0x1a, 0x3c, 0x92, 0x27, 0x00, 0xf6, 0x75, 0xcb, 0x34, 0xca, + 0xdb, 0xae, 0x53, 0x47, 0xa8, 0x29, 0x2d, 0x8b, 0x25, 0x37, 0x5c, 0xa7, 0x4e, 0x66, 0x21, 0xc7, + 0xab, 0x9b, 0xb6, 0x6f, 0x5a, 0x85, 0x34, 0xd6, 0xf3, 0x1e, 0x5b, 0xac, 0x84, 0x9c, 0x87, 0x2c, + 0x13, 0x20, 0xea, 0x79, 0xd4, 0x2b, 0x0c, 0x5e, 0x48, 0xcd, 0x65, 0xb5, 0x76, 0x01, 0x99, 0x87, + 0x09, 0xcf, 0xac, 0xd9, 0xba, 0xdf, 0x74, 0x69, 0x59, 0xb7, 0x6a, 0x8e, 0x6b, 0xfa, 0x3b, 0xf5, + 0xc2, 0x10, 0x62, 0x20, 0x61, 0xd5, 0x42, 0x50, 0xc3, 0xe0, 0x34, 0x9a, 0x15, 0xcb, 0xac, 0x96, + 0x77, 0x69, 0xab, 0x30, 0x8c, 0xed, 0xb2, 0xbc, 0xe4, 0x0e, 0x6d, 0x91, 0x73, 0x90, 0xdd, 0xa5, + 0x2d, 0xae, 0x79, 0x0a, 0x19, 0x1c, 0x2d, 0xb3, 0x4b, 0x5b, 0x5b, 0xc8, 0xef, 0x67, 0x81, 0xd0, + 0x03, 0x9f, 0xda, 0x06, 0x35, 0xca, 0xed, 0x56, 0x59, 0x6c, 0x95, 0x0f, 0x6a, 0xee, 0x88, 0xd6, + 0xea, 0x7d, 0x18, 0x8f, 0xad, 0x2d, 0x19, 0x82, 0x81, 0xa5, 0x85, 0xfc, 0x29, 0x92, 0x81, 0xf4, + 0xfa, 0xdd, 0xe5, 0x95, 0xbc, 0x42, 0x46, 0x21, 0xbb, 0xb4, 0xb6, 0xba, 0xb2, 0xbe, 0x59, 0x5e, + 0x5a, 0xc8, 0x0f, 0x10, 0x80, 0x21, 0xfe, 0x98, 0x4f, 0x91, 0x2c, 0x0c, 0x6e, 0xad, 0xb2, 0xe2, + 0x34, 0xeb, 0xb7, 0xb5, 0x9a, 0x1f, 0x54, 0x1d, 0x98, 0x8c, 0xca, 0xab, 0xd7, 0x70, 0x6c, 0x8f, + 0x92, 0xb7, 0x60, 0xa4, 0x2a, 0x95, 0x17, 0x14, 0x5c, 0xfa, 0xe7, 0x4e, 0xb4, 0xf4, 0x62, 0xcd, + 0x23, 0x84, 0xd4, 0x79, 0x18, 0x13, 0xd5, 0xbd, 0xf6, 0xc6, 0xed, 0x74, 0x66, 0x20, 0x9f, 0x52, + 0xd7, 0x01, 0x36, 0x5a, 0x9e, 0x4f, 0xeb, 0xab, 0xf6, 0xb6, 0xc3, 0x16, 0xd7, 0xc3, 0xa7, 0x32, + 0x33, 0x1f, 0xa2, 0x03, 0x78, 0x91, 0x06, 0xbb, 0xd4, 0xb5, 0xa9, 0xc5, 0x1b, 0x70, 0xd1, 0x01, + 0x5e, 0xc4, 0x1a, 0xa8, 0x5f, 0x4b, 0xc1, 0x78, 0x88, 0x40, 0xcc, 0xf6, 0xed, 0x28, 0x84, 0xc1, + 0xc5, 0x85, 0xa3, 0xc3, 0xd9, 0xa1, 0x75, 0x06, 0x63, 0xf9, 0xa3, 0xc3, 0xd9, 0xeb, 0x35, 0xd3, + 0xdf, 0x69, 0x56, 0x4a, 0x55, 0xa7, 0x3e, 0x1f, 0x32, 0xc0, 0xa8, 0xb4, 0x7f, 0xcf, 0x37, 0x76, + 0x6b, 0xf3, 0xc2, 0x90, 0x94, 0x78, 0xb7, 0x60, 0x16, 0xe4, 0x75, 0x18, 0x16, 0xc2, 0x85, 0x60, + 0x72, 0xd7, 0x66, 0x24, 0x26, 0x32, 0xe5, 0x55, 0xda, 0x0a, 0x15, 0xdb, 0x82, 0x61, 0xb8, 0x82, + 0x6b, 0x41, 0x27, 0xf2, 0x1a, 0x00, 0xda, 0x4b, 0x3e, 0x9f, 0x14, 0x92, 0x98, 0x92, 0x48, 0x60, + 0x65, 0x89, 0x4d, 0x4d, 0xf4, 0xcc, 0x62, 0x09, 0x32, 0x63, 0x2d, 0xca, 0xad, 0x34, 0x76, 0x7e, + 0x2a, 0x61, 0x11, 0xdb, 0x9c, 0x16, 0xc4, 0x64, 0xd6, 0x6e, 0x40, 0xce, 0xdb, 0xb3, 0xca, 0xc1, + 0x6c, 0x06, 0xfb, 0x9a, 0x0d, 0x61, 0x64, 0x8e, 0x0e, 0x67, 0x61, 0xe3, 0xfe, 0xda, 0x02, 0xef, + 0xa9, 0x81, 0xb7, 0x67, 0x89, 0xdf, 0xea, 0x18, 0x8c, 0x30, 0x86, 0x05, 0xd2, 0xa0, 0xfe, 0xf7, + 0x14, 0x8c, 0x8a, 0x02, 0xb1, 0x38, 0xb7, 0x60, 0x90, 0xb1, 0x32, 0x90, 0xc1, 0x67, 0xbb, 0xc0, + 0xe7, 0xa6, 0x29, 0xb0, 0xb7, 0xb8, 0x02, 0x1b, 0xf8, 0x20, 0x66, 0xc1, 0x09, 0x90, 0x5f, 0x57, + 0x60, 0x22, 0x30, 0x3a, 0xe5, 0x4a, 0xab, 0x1c, 0xac, 0xf9, 0x00, 0x12, 0x7e, 0x3d, 0x81, 0x2f, + 0x11, 0x44, 0xa5, 0x35, 0x41, 0x63, 0xb1, 0x85, 0x6b, 0x6d, 0xac, 0xd8, 0xbe, 0xdb, 0x5a, 0xbc, + 0x2b, 0x66, 0x9a, 0x8f, 0x55, 0x2f, 0x7f, 0xf0, 0x8b, 0x87, 0x93, 0xa0, 0xbc, 0x15, 0x1b, 0xa7, + 0xf8, 0x81, 0x02, 0x53, 0x5d, 0x07, 0x27, 0x79, 0x48, 0x31, 0xed, 0x83, 0xd2, 0xab, 0xb1, 0x9f, + 0x64, 0x03, 0x06, 0xf7, 0x75, 0xab, 0xc9, 0xf5, 0x7c, 0xd4, 0x86, 0xec, 0xee, 0x97, 0x02, 0xc3, + 0x5c, 0x0a, 0x8d, 0x70, 0xdb, 0x30, 0xe3, 0xf8, 0xc1, 0x30, 0x9c, 0x8f, 0x1a, 0xa7, 0xf5, 0xda, + 0xc0, 0xab, 0x8a, 0x9a, 0x87, 0x31, 0x8d, 0xd6, 0x98, 0x51, 0x0c, 0xd6, 0xec, 0x6f, 0x14, 0x18, + 0x0f, 0x8b, 0xc4, 0xaa, 0xdd, 0x87, 0x61, 0x97, 0x17, 0x89, 0x75, 0x7b, 0x25, 0x81, 0xbd, 0xb1, + 0xce, 0xc1, 0x33, 0x4e, 0x4d, 0x0b, 0xe8, 0x14, 0x67, 0x60, 0x88, 0x57, 0x90, 0x49, 0x18, 0x7c, + 0xd7, 0xb1, 0x85, 0x48, 0x64, 0x35, 0xfe, 0x50, 0xac, 0xc3, 0x88, 0xdc, 0x51, 0xe6, 0x49, 0x96, + 0xf3, 0xe4, 0xa6, 0xcc, 0x93, 0xdc, 0xb5, 0x17, 0x4e, 0x0c, 0x49, 0xe6, 0xc3, 0x65, 0xc8, 0x31, + 0x46, 0xf5, 0x34, 0xf1, 0x3f, 0x4a, 0x43, 0x56, 0xd3, 0xb7, 0x7d, 0xc6, 0x49, 0xa6, 0xf1, 0xc1, + 0xa5, 0x0d, 0xcb, 0xac, 0xea, 0x41, 0xcb, 0xf4, 0xe2, 0xe8, 0xd1, 0xe1, 0x6c, 0x56, 0xe3, 0xa5, + 0xab, 0xcb, 0x5a, 0x56, 0x34, 0x58, 0x35, 0xc8, 0xcb, 0x00, 0x3b, 0xba, 0x6b, 0xa0, 0xf3, 0x15, + 0xa0, 0x3e, 0x5d, 0xe2, 0xbe, 0x4c, 0xe9, 0x96, 0xee, 0x1a, 0x48, 0x34, 0xd8, 0xf8, 0x3b, 0x41, + 0x01, 0xb3, 0xe3, 0x16, 0xd5, 0x0d, 0x54, 0x17, 0x69, 0x0d, 0x7f, 0x33, 0xa6, 0x71, 0x32, 0x69, + 0x84, 0xc7, 0x1f, 0x98, 0x99, 0xd5, 0x1b, 0x0d, 0xcb, 0xa4, 0x06, 0x6e, 0xe8, 0xb4, 0x16, 0x3c, + 0x92, 0x4d, 0xc8, 0x34, 0x5c, 0xa7, 0x86, 0x7b, 0x7d, 0x08, 0x97, 0xf0, 0x5a, 0x12, 0xbf, 0x82, + 0x19, 0x96, 0xee, 0x89, 0x4e, 0x7c, 0x57, 0x70, 0x68, 0x21, 0x25, 0xf2, 0x34, 0x8c, 0x33, 0x34, + 0x65, 0xdf, 0xd5, 0x6d, 0x6f, 0x9b, 0xba, 0x94, 0xa2, 0xc9, 0x4c, 0x6b, 0x63, 0xac, 0x78, 0x33, + 0x2c, 0x2d, 0xfe, 0x07, 0x05, 0x32, 0x01, 0x29, 0x86, 0xbd, 0xae, 0xfb, 0xd5, 0x1d, 0xce, 0x30, + 0x8d, 0x3f, 0xb0, 0x59, 0xda, 0xf4, 0x80, 0xfb, 0x07, 0x69, 0x0d, 0x7f, 0xb7, 0x67, 0x99, 0x92, + 0x67, 0x39, 0x0d, 0x43, 0x0d, 0xbd, 0xe9, 0x51, 0x03, 0x27, 0x9f, 0xd1, 0xc4, 0x13, 0xb9, 0x02, + 0xf9, 0x06, 0xb5, 0x0d, 0xd3, 0xae, 0x95, 0x3d, 0x5b, 0x6f, 0x78, 0x3b, 0x8e, 0x2f, 0xd8, 0x30, + 0x2e, 0xca, 0x37, 0x44, 0x71, 0xf1, 0x01, 0x8c, 0x46, 0x66, 0x26, 0x8b, 0x57, 0x9a, 0x8b, 0xd7, + 0x52, 0x54, 0xbc, 0x9e, 0x3b, 0x11, 0xbb, 0x64, 0xd1, 0x3a, 0x1a, 0x80, 0x51, 0x4d, 0xb7, 0x6b, + 0xf4, 0x9e, 0xeb, 0x54, 0x2c, 0x5a, 0xf7, 0xc8, 0x05, 0xc8, 0x35, 0x6d, 0x7d, 0x5f, 0x37, 0x2d, + 0xbd, 0x62, 0x71, 0xbf, 0x30, 0xa3, 0xc9, 0x45, 0xe4, 0x25, 0x38, 0xc3, 0x38, 0x48, 0xdd, 0xb2, + 0xed, 0xf8, 0x65, 0xee, 0xb3, 0xef, 0x38, 0x96, 0x41, 0x5d, 0x84, 0x93, 0xd1, 0x26, 0x79, 0xf5, + 0xba, 0xe3, 0xaf, 0xb1, 0xca, 0x5b, 0x58, 0x47, 0x9e, 0x84, 0x31, 0xdb, 0x29, 0x33, 0x89, 0x2a, + 0xf3, 0x7a, 0x64, 0x5c, 0x46, 0x1b, 0xb1, 0x1d, 0x86, 0x71, 0x0d, 0xcb, 0xc8, 0x1c, 0x8c, 0x37, + 0x6d, 0x83, 0xba, 0x42, 0x32, 0xfd, 0x90, 0x91, 0xf1, 0x62, 0x72, 0x19, 0xc6, 0x9c, 0xfd, 0x48, + 0xc3, 0x0c, 0x36, 0x8c, 0x95, 0x92, 0xb3, 0x90, 0xb1, 0x1d, 0x0e, 0x13, 0x39, 0x9e, 0xd1, 0x86, + 0x6d, 0x07, 0x81, 0x91, 0x57, 0xa1, 0xb0, 0xd7, 0x34, 0xa9, 0x57, 0xa5, 0xb6, 0x5f, 0xa6, 0x7b, + 0x4d, 0xdd, 0xf2, 0xca, 0xbe, 0x59, 0xdd, 0x35, 0xed, 0x1a, 0xba, 0x61, 0x19, 0x6d, 0x3a, 0xac, + 0x5f, 0xc1, 0xea, 0x4d, 0x5e, 0x4b, 0x9e, 0x01, 0xc2, 0x67, 0xe2, 0xd4, 0xca, 0xbe, 0xe3, 0x94, + 0x2d, 0xdd, 0xad, 0x71, 0xf9, 0xca, 0x68, 0xe3, 0xac, 0x66, 0xcd, 0xa9, 0x6d, 0x3a, 0xce, 0x1a, + 0x2b, 0x56, 0x77, 0x61, 0x1c, 0x79, 0xcc, 0x96, 0xc1, 0xc4, 0x63, 0x1c, 0x73, 0xc7, 0xf6, 0x9a, + 0xd4, 0x35, 0xa9, 0x57, 0x6e, 0x50, 0xb7, 0xec, 0xd1, 0xaa, 0x63, 0xf3, 0x4d, 0xaa, 0x68, 0x79, + 0x51, 0x73, 0x8f, 0xba, 0x1b, 0x58, 0x4e, 0xae, 0xc2, 0xe9, 0x77, 0x5c, 0xd3, 0x8f, 0x36, 0x1e, + 0xc0, 0xc6, 0xe3, 0xbc, 0x22, 0x6c, 0xab, 0xde, 0x02, 0xb8, 0xe7, 0x52, 0xdf, 0x6f, 0x6d, 0x34, + 0x74, 0x9b, 0xf9, 0x84, 0x9e, 0xaf, 0xbb, 0x7e, 0xb9, 0xad, 0x9f, 0x32, 0x58, 0xc0, 0x1c, 0xc6, + 0x33, 0x30, 0x4c, 0x6d, 0x74, 0x07, 0x85, 0xf7, 0x32, 0x44, 0x6d, 0xe6, 0x03, 0xbe, 0x96, 0xfe, + 0x8b, 0xef, 0xcd, 0x2a, 0xea, 0x77, 0x72, 0x4c, 0x9d, 0xd8, 0x35, 0x8a, 0x36, 0xf9, 0x73, 0x90, + 0xf6, 0x1a, 0xba, 0x8d, 0x44, 0x92, 0x4d, 0x7b, 0x7b, 0x78, 0xb1, 0x27, 0xb1, 0x23, 0x59, 0x05, + 0x40, 0x96, 0xc9, 0x1a, 0xe6, 0xc9, 0x7e, 0x04, 0x37, 0x50, 0x3a, 0x6e, 0xa8, 0xda, 0x6e, 0xc8, + 0x0a, 0x26, 0x77, 0xed, 0xea, 0x31, 0x16, 0x47, 0x9c, 0x42, 0x91, 0x96, 0x98, 0x46, 0x60, 0xa6, + 0xf9, 0x66, 0xad, 0xc3, 0x98, 0xe7, 0x34, 0xdd, 0x2a, 0x0d, 0x0d, 0xf4, 0x20, 0x3a, 0x65, 0x37, + 0x8f, 0x0e, 0x67, 0x47, 0x36, 0xb0, 0xe6, 0xd1, 0x5c, 0xb3, 0x11, 0xaf, 0x4d, 0xc4, 0x20, 0x7b, + 0x30, 0x2e, 0x86, 0x63, 0xc8, 0x70, 0xbc, 0x21, 0x1c, 0x6f, 0xf5, 0xe8, 0x70, 0x76, 0x94, 0x8f, + 0xb7, 0xc1, 0x6a, 0x70, 0xc0, 0x17, 0x4f, 0x34, 0xa0, 0xe8, 0xa7, 0x8d, 0x7a, 0x12, 0x19, 0xa3, + 0xf3, 0x2c, 0x36, 0xdc, 0xe5, 0x2c, 0xb6, 0x04, 0xa3, 0x62, 0x17, 0x9b, 0x0c, 0x58, 0x0b, 0x0f, + 0x0f, 0xb9, 0x6b, 0x05, 0x89, 0xad, 0xc1, 0x30, 0xb8, 0x6f, 0x02, 0x77, 0x1b, 0x3b, 0xdd, 0xe2, + 0x7d, 0xc8, 0x6d, 0x54, 0xe2, 0xa8, 0x43, 0x0a, 0x59, 0x5c, 0x96, 0xb9, 0xc4, 0xc5, 0x95, 0x74, + 0x8e, 0xa4, 0xba, 0xb9, 0x0e, 0x12, 0xeb, 0xeb, 0x15, 0xa0, 0x63, 0x7d, 0xbb, 0x12, 0x6a, 0x6f, + 0x2c, 0x79, 0x7d, 0x3d, 0xb2, 0x05, 0x23, 0x72, 0x48, 0xa1, 0x30, 0x8a, 0xe4, 0x9e, 0xed, 0x29, + 0x2e, 0x38, 0xcb, 0x88, 0x5f, 0x97, 0xb3, 0xda, 0x45, 0xec, 0x58, 0x17, 0xaa, 0x85, 0xc2, 0x18, + 0xee, 0xf9, 0x76, 0x01, 0xb3, 0x73, 0x81, 0x0e, 0x19, 0xe7, 0xea, 0x46, 0x3c, 0x92, 0x8b, 0x30, + 0xe2, 0x32, 0x8b, 0x64, 0x31, 0x9b, 0x42, 0xbd, 0x42, 0x1e, 0x0f, 0x8c, 0x39, 0x56, 0xb6, 0xc6, + 0x8b, 0xd8, 0x7a, 0xe1, 0x86, 0x0e, 0xdb, 0x9c, 0xc6, 0x36, 0x23, 0x58, 0x18, 0x34, 0x9a, 0x84, + 0x41, 0xcb, 0xa9, 0xee, 0x7a, 0x05, 0x82, 0x95, 0xfc, 0x81, 0x5c, 0x87, 0x69, 0xfc, 0x51, 0x7e, + 0xc7, 0xf4, 0x77, 0xca, 0xef, 0xe8, 0xa6, 0x5f, 0xde, 0x6b, 0xd2, 0x26, 0xf5, 0x0a, 0x13, 0xd8, + 0x6c, 0x02, 0x6b, 0xdf, 0x32, 0xfd, 0x9d, 0xb7, 0x74, 0xd3, 0xbf, 0x8f, 0x55, 0xa8, 0xcb, 0x9d, + 0xea, 0xae, 0xd4, 0x1c, 0x7f, 0x52, 0xd7, 0x2b, 0x4c, 0x62, 0xaf, 0x49, 0x56, 0x1d, 0x76, 0x78, + 0x8b, 0xd7, 0x91, 0xf7, 0xe0, 0xa2, 0xef, 0x34, 0xca, 0xbb, 0x65, 0x3e, 0x62, 0xa5, 0xd5, 0x8d, + 0xc0, 0x54, 0xcf, 0x93, 0x5c, 0xb8, 0x2d, 0x4b, 0x6b, 0x4e, 0x75, 0x57, 0xda, 0x9f, 0xe7, 0x7c, + 0xa7, 0x71, 0x87, 0x95, 0x79, 0x8b, 0xad, 0xf8, 0xe0, 0xc5, 0xbf, 0x56, 0x20, 0x13, 0xb4, 0xc7, + 0x33, 0x31, 0xea, 0x1b, 0x49, 0xc3, 0x65, 0x79, 0x09, 0x53, 0x71, 0x37, 0xb8, 0xe9, 0x64, 0xda, + 0x66, 0x64, 0xf1, 0xc5, 0x8f, 0x0e, 0x67, 0x9f, 0x3f, 0xd1, 0xae, 0xba, 0x43, 0x5b, 0xdc, 0xe0, + 0x12, 0x48, 0xef, 0x50, 0xcb, 0x10, 0x26, 0x0b, 0x7f, 0xb3, 0x85, 0x0e, 0xa6, 0xca, 0x8f, 0xfe, + 0xc1, 0x23, 0x73, 0x3d, 0xd8, 0x4f, 0x66, 0xec, 0x5d, 0x34, 0x6b, 0xfc, 0x0c, 0x93, 0xd2, 0xc6, + 0x44, 0xb1, 0xc6, 0x4b, 0xe5, 0x86, 0xb8, 0xc2, 0xae, 0x87, 0x1a, 0xa1, 0xdd, 0xf0, 0x2d, 0x5e, + 0x7a, 0x3b, 0x9d, 0xc9, 0xe5, 0x47, 0x6e, 0xa7, 0x33, 0x23, 0xf9, 0x51, 0xf5, 0xdf, 0x2b, 0xc2, + 0x66, 0xf7, 0x3c, 0xd8, 0x12, 0x1d, 0xb2, 0x2e, 0x6b, 0x59, 0x36, 0x0d, 0x0f, 0x0f, 0x1f, 0xa9, + 0xc5, 0xe5, 0xa3, 0xc3, 0xd9, 0x0c, 0x67, 0xfc, 0xb2, 0x77, 0x62, 0x35, 0x23, 0x3a, 0x6a, 0x19, + 0x24, 0xbb, 0x6a, 0x78, 0xea, 0x26, 0x8c, 0x05, 0x60, 0x84, 0x43, 0xbe, 0x08, 0x43, 0x58, 0x1b, + 0xf8, 0xe3, 0x4f, 0xf6, 0x23, 0x01, 0x41, 0xd8, 0x86, 0xf7, 0x54, 0xe7, 0x60, 0xf4, 0x26, 0x86, + 0x68, 0x7b, 0x3a, 0xbd, 0x1f, 0x0e, 0xc0, 0xf8, 0x0a, 0x86, 0x19, 0xd9, 0xee, 0xf4, 0x50, 0x28, + 0xbe, 0x08, 0x99, 0x50, 0xc3, 0xf2, 0x63, 0xf6, 0xd2, 0xd1, 0xe1, 0xec, 0xf0, 0xa3, 0xea, 0xd6, + 0x61, 0x4f, 0x68, 0xd5, 0x6d, 0x98, 0x66, 0x7b, 0x9a, 0xba, 0x5e, 0x59, 0xb7, 0x0d, 0xae, 0x36, + 0x6b, 0xae, 0x5e, 0x0f, 0x0e, 0xde, 0xcf, 0xcb, 0x33, 0xe6, 0x5a, 0xa5, 0x14, 0x84, 0x42, 0x4b, + 0x9b, 0xbc, 0xe7, 0x82, 0x6d, 0xdc, 0x0a, 0xfb, 0x69, 0x93, 0x7e, 0x97, 0x52, 0x72, 0x13, 0x72, + 0xbc, 0x5b, 0x19, 0x63, 0x74, 0x29, 0x3c, 0x5f, 0x5d, 0x4e, 0x22, 0xce, 0x39, 0x81, 0xc1, 0x38, + 0xa0, 0xe1, 0x6f, 0xf5, 0x39, 0x20, 0x12, 0x8f, 0x7a, 0xf2, 0xf4, 0x9f, 0xc0, 0x44, 0xa4, 0xb9, + 0x58, 0xd8, 0x50, 0x2d, 0xf3, 0x75, 0x4d, 0x52, 0xcb, 0xb1, 0x15, 0x89, 0xa8, 0x65, 0xf5, 0x9f, + 0x02, 0x6c, 0xba, 0x7a, 0x95, 0xae, 0xec, 0x33, 0x7d, 0xf9, 0x2a, 0xa4, 0x7d, 0xb3, 0x4e, 0x85, + 0x63, 0x51, 0x2c, 0xf1, 0xc0, 0x67, 0x29, 0x08, 0x7c, 0x96, 0x36, 0x83, 0xc0, 0xe7, 0x62, 0x86, + 0x11, 0xf9, 0xe6, 0x2f, 0x66, 0x15, 0x0d, 0x7b, 0xb0, 0x0d, 0x18, 0x0d, 0x31, 0x06, 0x8f, 0xea, + 0x8f, 0x14, 0x18, 0x5f, 0xb0, 0x2c, 0xa7, 0xaa, 0xfb, 0x8e, 0xbb, 0xec, 0xb6, 0xb4, 0xa6, 0xcd, + 0x84, 0x22, 0xd8, 0x0b, 0x38, 0x56, 0x8a, 0x0b, 0x85, 0x90, 0xe8, 0x87, 0xde, 0x09, 0xc3, 0x62, + 0x27, 0x90, 0xcf, 0xc2, 0x10, 0x65, 0x13, 0xf2, 0xc4, 0x29, 0x3f, 0xc9, 0x45, 0x6a, 0x4f, 0x5f, + 0x13, 0x9d, 0xd4, 0x6b, 0x30, 0x15, 0x22, 0x46, 0xda, 0xc1, 0x2a, 0x9d, 0x8d, 0xe3, 0x0e, 0x87, + 0x54, 0xff, 0xbf, 0x02, 0xd3, 0xf1, 0x4e, 0xdd, 0x03, 0x4d, 0xa9, 0x8f, 0x33, 0xd0, 0xb4, 0x04, + 0xc3, 0x86, 0xdb, 0x2a, 0xbb, 0x4d, 0x5b, 0xc8, 0x7b, 0x92, 0x24, 0xc4, 0x96, 0x41, 0x1b, 0x32, + 0xf0, 0xaf, 0xfa, 0x75, 0x05, 0xf2, 0x6d, 0xec, 0xff, 0x00, 0x14, 0xd9, 0xdb, 0x70, 0x5a, 0xc2, + 0x23, 0xd8, 0xb8, 0x02, 0x19, 0x31, 0xd5, 0x7e, 0xa4, 0x3e, 0x3e, 0xd7, 0x61, 0x3e, 0x57, 0x4f, + 0x55, 0x61, 0xe4, 0xf6, 0xc6, 0xdd, 0xf5, 0x90, 0x6c, 0x10, 0xfd, 0x56, 0xda, 0xd1, 0x6f, 0xb5, + 0x0e, 0xa3, 0x41, 0xfd, 0x0a, 0xf3, 0xce, 0x98, 0x99, 0x47, 0x37, 0x4d, 0xb0, 0x82, 0x3f, 0xb0, + 0xae, 0x55, 0xc7, 0xe0, 0x12, 0x3f, 0xa8, 0xe1, 0x6f, 0x79, 0x23, 0xa4, 0x22, 0x1b, 0x81, 0xd5, + 0x18, 0x3c, 0x04, 0x89, 0x31, 0xf5, 0xac, 0x16, 0x3c, 0xaa, 0xbf, 0xa9, 0x40, 0x6e, 0xcd, 0xa9, + 0xf5, 0xb6, 0x21, 0xcc, 0xdb, 0xa0, 0xfb, 0xd4, 0x12, 0x7b, 0x8c, 0x3f, 0x30, 0xbb, 0xcb, 0x0f, + 0x16, 0xb8, 0x77, 0xf9, 0xa8, 0xfc, 0xa8, 0xc1, 0xf6, 0x2b, 0x13, 0x5a, 0x76, 0xb4, 0xc0, 0x4a, + 0x1e, 0x05, 0x60, 0x47, 0x0d, 0xac, 0xca, 0x43, 0xaa, 0xae, 0x1f, 0xa0, 0x41, 0xcc, 0x6a, 0xec, + 0x27, 0x03, 0xd9, 0xd0, 0x7d, 0x9f, 0xba, 0xb6, 0x08, 0x7e, 0x07, 0x8f, 0xec, 0x34, 0xed, 0x52, + 0x43, 0xaf, 0xfa, 0xe2, 0x68, 0x25, 0x9e, 0x6e, 0xa7, 0x33, 0x99, 0x7c, 0x56, 0xbd, 0x0b, 0x64, + 0xcd, 0xa9, 0xb1, 0x33, 0xb2, 0x29, 0x99, 0x9f, 0x4f, 0xb1, 0x53, 0x0d, 0x16, 0x89, 0x15, 0x3b, + 0x1b, 0x0f, 0x1c, 0x5a, 0x4e, 0xad, 0x24, 0xc7, 0x0c, 0x82, 0xf6, 0x6a, 0x09, 0x26, 0xd6, 0x9c, + 0xda, 0x0d, 0xd3, 0xa2, 0xde, 0x9a, 0xe9, 0xf9, 0x3d, 0xf5, 0xe4, 0x3d, 0x98, 0x8c, 0xb6, 0x17, + 0x10, 0x5e, 0x85, 0xc1, 0x6d, 0x56, 0x28, 0x00, 0x9c, 0xef, 0x06, 0x80, 0xf5, 0x92, 0x55, 0x23, + 0x76, 0x50, 0xbf, 0x00, 0x63, 0x82, 0x62, 0xcf, 0x75, 0x21, 0x90, 0x66, 0x7d, 0xc4, 0xb2, 0xe0, + 0x6f, 0x89, 0x5f, 0xa9, 0x18, 0xbf, 0xd2, 0xf9, 0x41, 0xb5, 0x0a, 0xa3, 0x1b, 0xbe, 0x5e, 0xdd, + 0xed, 0xbd, 0xe6, 0x9f, 0x12, 0xb7, 0x42, 0x3c, 0xa2, 0x97, 0x18, 0xc7, 0x45, 0x82, 0xed, 0xdb, + 0x1f, 0x75, 0x03, 0xd2, 0x0c, 0x3e, 0x86, 0x4c, 0x74, 0xa1, 0xd6, 0xb3, 0x1a, 0xfe, 0x66, 0xa7, + 0x51, 0x06, 0xb3, 0xec, 0x99, 0xef, 0x72, 0xda, 0x29, 0x2d, 0xc3, 0x0a, 0x36, 0xcc, 0x77, 0x29, + 0x29, 0x42, 0x46, 0xdc, 0x4f, 0x7a, 0xe2, 0x56, 0x28, 0x7c, 0x56, 0xff, 0xab, 0x02, 0xe3, 0x37, + 0xa9, 0x8f, 0x9c, 0xee, 0x09, 0xfe, 0x1c, 0x64, 0x2d, 0xd3, 0xf3, 0xcb, 0x8e, 0x6d, 0xb5, 0x44, + 0x44, 0x22, 0xc3, 0x0a, 0xee, 0xda, 0x56, 0x8b, 0xbc, 0x22, 0x66, 0x36, 0x88, 0x33, 0xbb, 0x94, + 0x30, 0x33, 0x36, 0x98, 0x74, 0xab, 0x55, 0x84, 0x8c, 0x90, 0x4a, 0x1e, 0xa4, 0xca, 0x6a, 0xe1, + 0xb3, 0xba, 0x0a, 0xf9, 0x36, 0x3a, 0x21, 0x03, 0x2f, 0x45, 0x65, 0x60, 0xb6, 0xc7, 0x48, 0x81, + 0x00, 0x7c, 0xa8, 0xc0, 0xd8, 0x3d, 0xd7, 0xd9, 0xee, 0x47, 0x02, 0x16, 0x23, 0x73, 0x29, 0x25, + 0x1e, 0xc9, 0x65, 0x8a, 0x25, 0x69, 0x5a, 0x05, 0x18, 0xe6, 0xf1, 0x04, 0xee, 0x79, 0x0e, 0x6a, + 0xc1, 0xa3, 0x7a, 0x16, 0xd2, 0x78, 0xf1, 0x93, 0x81, 0xf4, 0xad, 0x95, 0x85, 0x7b, 0xf9, 0x53, + 0x64, 0x18, 0x52, 0x4b, 0xf7, 0xb6, 0xf2, 0x8a, 0x7a, 0x05, 0xc6, 0xde, 0xa4, 0xbe, 0x6b, 0x56, + 0x7b, 0xbb, 0x12, 0xff, 0x1b, 0x9d, 0xd5, 0x6d, 0x1f, 0xf5, 0x2d, 0xb3, 0x23, 0x8f, 0xf5, 0x0a, + 0xe4, 0x0d, 0x18, 0x44, 0x7d, 0xde, 0x57, 0x78, 0x21, 0x16, 0x12, 0xc0, 0x8e, 0xea, 0x55, 0xe6, + 0xce, 0x0a, 0xb8, 0x5c, 0x0d, 0x4b, 0xca, 0x55, 0x89, 0x7a, 0x19, 0x5f, 0x1e, 0x80, 0xf1, 0xb0, + 0xb1, 0x38, 0x1b, 0x3e, 0x6e, 0x2f, 0xe3, 0x26, 0x0c, 0xa1, 0x1d, 0x08, 0xbc, 0x8c, 0x2b, 0x3d, + 0x22, 0x28, 0xed, 0x89, 0x04, 0x1e, 0x36, 0xef, 0x4e, 0x96, 0x83, 0xcb, 0x8e, 0x14, 0xd2, 0x99, + 0xeb, 0x87, 0x0e, 0xe3, 0x76, 0xe4, 0xa2, 0x43, 0x6d, 0x42, 0x9e, 0xd5, 0x2e, 0xd3, 0x4a, 0xb3, + 0x16, 0xc8, 0x42, 0xc4, 0x56, 0x2b, 0x8f, 0xc5, 0x56, 0xff, 0xe1, 0x00, 0x9c, 0x96, 0xc6, 0x15, + 0x5b, 0xee, 0xeb, 0x4a, 0xec, 0xe4, 0xf1, 0x6a, 0x8f, 0x49, 0x45, 0xba, 0xf3, 0x61, 0x44, 0x30, + 0xf9, 0x33, 0x6c, 0x92, 0x1f, 0xfc, 0xe2, 0x21, 0x81, 0x0a, 0x14, 0x1f, 0xdb, 0x62, 0x15, 0x29, + 0xe4, 0x24, 0x74, 0x72, 0x40, 0x38, 0xc5, 0xcf, 0xa7, 0x6f, 0x44, 0x03, 0xc2, 0x57, 0xfb, 0x19, + 0xa8, 0xf3, 0xc2, 0xe5, 0x77, 0xd3, 0x30, 0xbc, 0x79, 0x60, 0xe3, 0x19, 0xea, 0x3e, 0x0c, 0x08, + 0x11, 0x1e, 0x59, 0x5c, 0x60, 0x60, 0xfe, 0xa8, 0xdf, 0xbd, 0xc9, 0x73, 0x28, 0x9a, 0xa6, 0x51, + 0xda, 0xda, 0x5a, 0x65, 0x2b, 0x3f, 0xb0, 0xba, 0xac, 0x0d, 0x98, 0x06, 0x79, 0x0d, 0xcf, 0x0f, + 0xae, 0x2f, 0x40, 0xf6, 0xe7, 0xea, 0xf3, 0x2e, 0xec, 0xa4, 0xec, 0x1f, 0xd8, 0x65, 0x83, 0x7a, + 0x55, 0xd7, 0x6c, 0xf8, 0xa6, 0x63, 0x0b, 0xa7, 0x63, 0xcc, 0x3f, 0xb0, 0x97, 0xdb, 0xa5, 0xe4, + 0x65, 0x38, 0x63, 0x37, 0xeb, 0x3c, 0xca, 0x58, 0x67, 0xc6, 0xa3, 0x4c, 0x0f, 0x68, 0xb5, 0x19, + 0x04, 0x92, 0x07, 0xb5, 0x29, 0xbb, 0x59, 0xdf, 0x08, 0x6b, 0x57, 0x44, 0x25, 0x99, 0x85, 0x1c, + 0xeb, 0xe7, 0x52, 0xee, 0x3a, 0x60, 0x20, 0x50, 0x03, 0xbb, 0x59, 0xd7, 0x78, 0x09, 0x99, 0x83, + 0x3c, 0x6b, 0xa0, 0x37, 0x7d, 0x27, 0x6c, 0xc5, 0x55, 0xe6, 0x98, 0xdd, 0xac, 0x2f, 0x34, 0x7d, + 0x27, 0x68, 0xf9, 0x06, 0x64, 0x0c, 0xaa, 0x1b, 0x96, 0x69, 0xf3, 0x78, 0x5b, 0xbf, 0x53, 0x0d, + 0x7b, 0x31, 0x63, 0x63, 0xd6, 0x1b, 0x96, 0x59, 0x35, 0x7d, 0x11, 0xd5, 0x0e, 0x9f, 0x19, 0x50, + 0x9d, 0xf9, 0x99, 0xe5, 0x4a, 0xcb, 0xa7, 0x3c, 0xd6, 0x96, 0xd2, 0x00, 0x8b, 0x16, 0x59, 0x09, + 0xb9, 0x0c, 0xe3, 0x75, 0xfd, 0xa0, 0x2c, 0x37, 0x02, 0x6c, 0x34, 0x5a, 0xd7, 0x0f, 0x16, 0xda, + 0xed, 0xce, 0x41, 0x16, 0xc3, 0x51, 0x68, 0x27, 0x73, 0x7c, 0x14, 0x56, 0x80, 0x76, 0xf2, 0x12, + 0x8c, 0x9a, 0x9e, 0x08, 0x08, 0x9a, 0x55, 0xdd, 0x2a, 0x8c, 0xf0, 0x60, 0xbd, 0xe9, 0xdd, 0x0a, + 0xcb, 0xd0, 0x26, 0xba, 0xa6, 0xe3, 0x9a, 0x7e, 0x0b, 0x63, 0x6b, 0xcc, 0x26, 0x8a, 0x67, 0xf5, + 0x3f, 0xa7, 0x20, 0xb7, 0x50, 0xf5, 0xcd, 0x7d, 0x7a, 0xbf, 0x49, 0xdd, 0x16, 0x99, 0x0e, 0xe5, + 0x29, 0xbb, 0x38, 0x24, 0x09, 0xc5, 0x17, 0x60, 0x88, 0x2d, 0xac, 0x69, 0x20, 0xab, 0x46, 0x16, + 0x97, 0x1f, 0x4d, 0xd6, 0x06, 0x99, 0xfc, 0x2e, 0x6b, 0x83, 0xfe, 0x81, 0xbd, 0x6a, 0xb0, 0x8d, + 0xe2, 0xed, 0x05, 0x9e, 0x2b, 0xfb, 0xd9, 0x96, 0xc1, 0xd4, 0xc9, 0x65, 0xf0, 0x29, 0x18, 0x33, + 0xbd, 0xb2, 0x61, 0x7a, 0xbe, 0x6b, 0x56, 0x9a, 0xed, 0xab, 0x89, 0x51, 0xd3, 0x5b, 0x6e, 0x17, + 0x92, 0x45, 0x18, 0x6c, 0xec, 0x04, 0xb7, 0x0d, 0x63, 0x5d, 0xaf, 0x91, 0xc3, 0x03, 0x43, 0x9b, + 0x41, 0xa5, 0x7b, 0xac, 0x8f, 0xc6, 0xbb, 0x72, 0xce, 0x86, 0x57, 0x62, 0xca, 0xdc, 0x80, 0x74, + 0xb1, 0x75, 0x16, 0x32, 0x78, 0x3b, 0x6e, 0x3b, 0x36, 0x0a, 0x47, 0x56, 0x1b, 0xf6, 0xf6, 0xac, + 0x05, 0xdb, 0xb1, 0xd5, 0xa7, 0x60, 0x10, 0xc9, 0x90, 0x51, 0xc8, 0xde, 0xd3, 0x56, 0xee, 0x2d, + 0x68, 0xab, 0xeb, 0x37, 0xf3, 0xa7, 0xd8, 0xe3, 0xca, 0xe7, 0x57, 0x96, 0xb6, 0x36, 0xd9, 0xa3, + 0xa2, 0xbe, 0x00, 0x13, 0xcc, 0x5f, 0xdd, 0xa0, 0x9e, 0x27, 0xdd, 0xae, 0xb2, 0x41, 0x9b, 0x1e, + 0x75, 0x25, 0xb7, 0x2d, 0x7c, 0x56, 0xff, 0x2a, 0x0d, 0xc3, 0xa2, 0xfd, 0x63, 0xb5, 0xe0, 0x32, + 0x86, 0x81, 0x28, 0x06, 0xc6, 0xff, 0xaa, 0x65, 0x52, 0xdb, 0x0f, 0x33, 0x03, 0xb8, 0x0a, 0x18, + 0xe5, 0xa5, 0xe2, 0xa2, 0x9f, 0x5c, 0x81, 0x3c, 0xde, 0x2c, 0x56, 0x31, 0xa1, 0xaa, 0x8c, 0xa4, + 0xf8, 0x19, 0x64, 0x5c, 0x2a, 0x5f, 0x67, 0x14, 0x37, 0x60, 0x4c, 0xc7, 0x25, 0x28, 0x8b, 0x3b, + 0x17, 0xcc, 0xd2, 0xc9, 0x45, 0x63, 0x2c, 0xc7, 0xaf, 0x99, 0xd0, 0xd2, 0xa3, 0x7a, 0x58, 0xc4, + 0xb6, 0x7f, 0x28, 0x62, 0x43, 0x27, 0x17, 0xb1, 0xab, 0x70, 0xda, 0xd2, 0x3d, 0xbf, 0x2c, 0xa1, + 0x6a, 0x89, 0x45, 0x1e, 0x67, 0x15, 0x9d, 0x3b, 0x2a, 0x8b, 0xbb, 0x46, 0xde, 0x51, 0x31, 0x05, + 0x01, 0xfd, 0x28, 0x88, 0x5c, 0x37, 0x05, 0xb1, 0x00, 0x20, 0x70, 0xf8, 0x07, 0x36, 0x2a, 0x80, + 0xdc, 0x35, 0x35, 0x29, 0xaa, 0xc1, 0x4d, 0x87, 0x96, 0xe5, 0xbd, 0x36, 0x0f, 0x6c, 0x0c, 0x4a, + 0xc7, 0xe7, 0xc3, 0x25, 0x97, 0xeb, 0x8b, 0x89, 0xd8, 0xa4, 0x98, 0x14, 0xdf, 0x4e, 0x67, 0x86, + 0xf3, 0x19, 0xf5, 0xdf, 0x29, 0x70, 0x5a, 0x96, 0x52, 0xee, 0x8d, 0x3d, 0x4e, 0xd9, 0x3b, 0x3e, + 0x9e, 0xf4, 0x3f, 0x14, 0x98, 0x8c, 0xee, 0x18, 0xe1, 0x72, 0x2c, 0x43, 0xc6, 0x13, 0x65, 0xc2, + 0xe7, 0x48, 0x62, 0x90, 0xe8, 0x1e, 0xdc, 0x77, 0x04, 0x3d, 0xc9, 0xed, 0x98, 0x9f, 0x90, 0xa4, + 0x32, 0x3a, 0x58, 0x12, 0x75, 0x15, 0xd4, 0x3d, 0x20, 0x4b, 0xba, 0x5d, 0xa5, 0x16, 0xf2, 0xb3, + 0xe7, 0x19, 0xe2, 0x32, 0x64, 0xf8, 0xa2, 0x98, 0xfc, 0x46, 0x31, 0xbb, 0x98, 0x63, 0xfe, 0x2a, + 0x76, 0x66, 0x7e, 0x27, 0x56, 0xc6, 0xf6, 0x65, 0x2a, 0xa6, 0x1b, 0x6e, 0xc2, 0x44, 0x64, 0x48, + 0xc1, 0x1b, 0x76, 0xa0, 0xc3, 0x62, 0x6a, 0x88, 0x6b, 0xe4, 0xf0, 0xb9, 0x1d, 0xdb, 0x18, 0x90, + 0x62, 0x1b, 0x6a, 0x0b, 0x26, 0x39, 0x21, 0x31, 0xc1, 0x9e, 0xe8, 0x9f, 0x05, 0x10, 0x4c, 0x0c, + 0xf0, 0x8f, 0xf0, 0x1c, 0x07, 0x41, 0x60, 0x75, 0x59, 0xcb, 0x8a, 0x06, 0x3d, 0xe6, 0xb0, 0x0a, + 0x53, 0xb1, 0xa1, 0x1f, 0x7a, 0x16, 0x4f, 0xc0, 0x39, 0xb6, 0x48, 0x4b, 0x61, 0xb2, 0x2d, 0x06, + 0xfa, 0xc2, 0x1c, 0x96, 0x40, 0xae, 0x51, 0xea, 0x4d, 0xbf, 0xf5, 0xab, 0x94, 0xeb, 0xff, 0xa7, + 0xc0, 0xf9, 0xee, 0x58, 0xc5, 0xec, 0xd7, 0xc2, 0xa0, 0x26, 0x0f, 0xcf, 0x46, 0x0e, 0x99, 0x7b, + 0x56, 0x49, 0x4e, 0x2f, 0x2e, 0x6d, 0x50, 0xd7, 0xd4, 0x2d, 0xf3, 0x5d, 0x6a, 0x68, 0xb4, 0xc6, + 0x0c, 0x62, 0x2b, 0x94, 0x4d, 0xa4, 0x71, 0x62, 0x39, 0x8f, 0xb0, 0x28, 0x26, 0xe7, 0x67, 0xe1, + 0x0c, 0x6b, 0xc2, 0x0c, 0xef, 0xc6, 0xfd, 0xb5, 0x1b, 0x96, 0xf3, 0x4e, 0xc8, 0xe1, 0x3f, 0x4f, + 0x01, 0x11, 0xe5, 0x1a, 0xad, 0x3b, 0x3e, 0xc5, 0x5a, 0xb2, 0x0d, 0xc3, 0xdb, 0x96, 0xf3, 0x4e, + 0x39, 0x74, 0x6b, 0xdf, 0x14, 0xae, 0xc6, 0xa7, 0xfb, 0x62, 0x6e, 0x2c, 0x55, 0xba, 0xc4, 0x88, + 0xa2, 0xcb, 0x31, 0xc4, 0x7f, 0x69, 0x43, 0x8c, 0xfa, 0xaa, 0x41, 0xd6, 0x61, 0xd0, 0xb4, 0xb7, + 0x9d, 0x60, 0x92, 0x49, 0xb9, 0x2c, 0x9d, 0x28, 0xe5, 0xfc, 0x3a, 0x4e, 0xa6, 0xf8, 0xb7, 0x0a, + 0xa4, 0xd1, 0x25, 0x7f, 0x9c, 0x32, 0xb2, 0x08, 0xd9, 0x30, 0xc3, 0xf8, 0x44, 0xfe, 0x79, 0xbb, + 0x1b, 0x13, 0x16, 0x71, 0xd1, 0xca, 0x6f, 0x2a, 0x5e, 0x3c, 0xd9, 0xcc, 0xc5, 0x79, 0x44, 0xd0, + 0x50, 0x2f, 0xc2, 0x90, 0x38, 0x53, 0xe7, 0x60, 0x58, 0xdb, 0x5a, 0x5f, 0xe7, 0xae, 0x0c, 0xc0, + 0xd0, 0xfd, 0xad, 0x95, 0xad, 0x95, 0xe5, 0xbc, 0xa2, 0xfe, 0x58, 0x81, 0x42, 0xa7, 0x10, 0x08, + 0xd1, 0x5d, 0x85, 0x41, 0xb6, 0x20, 0xfd, 0x64, 0x94, 0x76, 0x82, 0x09, 0xa3, 0x72, 0x28, 0x39, + 0x1f, 0xa7, 0xdc, 0xfe, 0xb1, 0x02, 0xf9, 0x8d, 0x86, 0x6e, 0x47, 0x6e, 0x62, 0x2e, 0xc5, 0x14, + 0xdc, 0x22, 0xb4, 0x57, 0x36, 0x5c, 0x22, 0x4d, 0xce, 0xe5, 0xe0, 0xba, 0xee, 0xa5, 0x8f, 0x0e, + 0x67, 0x5f, 0x38, 0xd9, 0x11, 0xf5, 0x0e, 0x6d, 0x49, 0x29, 0x20, 0xeb, 0xed, 0x14, 0x90, 0xd4, + 0xa3, 0x50, 0x14, 0x99, 0x23, 0x4c, 0xa1, 0x9c, 0x96, 0x66, 0x27, 0x96, 0x62, 0x16, 0x72, 0x3c, + 0x22, 0x50, 0x75, 0x9a, 0xb6, 0x2f, 0x42, 0xd7, 0x80, 0x45, 0x4b, 0xac, 0x84, 0xbc, 0x08, 0xd3, + 0x7a, 0xa3, 0xe1, 0x3a, 0x07, 0x66, 0x5d, 0xf7, 0x29, 0x73, 0xb1, 0x77, 0x85, 0x63, 0xc2, 0xf3, + 0xca, 0x26, 0xa5, 0xda, 0x65, 0xd3, 0xdb, 0xe5, 0xfe, 0xc9, 0x1a, 0xe4, 0x7c, 0xc7, 0xd7, 0xad, + 0x72, 0x70, 0x2b, 0xd5, 0x91, 0x99, 0x12, 0xbf, 0x1e, 0x7b, 0xf3, 0x1f, 0x2d, 0x2d, 0x21, 0xb4, + 0x20, 0xe9, 0x14, 0xfb, 0x63, 0x89, 0xfa, 0x69, 0x98, 0x14, 0x09, 0x09, 0xd1, 0xcb, 0xd5, 0x7e, + 0xd6, 0x46, 0xfd, 0xee, 0x28, 0x4c, 0xc5, 0x7a, 0x77, 0x5e, 0xc4, 0x64, 0x3e, 0xee, 0x4d, 0xfb, + 0x3b, 0x0a, 0x4c, 0x04, 0x49, 0x13, 0x72, 0x9a, 0x69, 0x16, 0xa5, 0xf4, 0x46, 0x72, 0x40, 0xb0, + 0x13, 0x6b, 0x29, 0x4c, 0xc8, 0xe8, 0x9e, 0x6e, 0x1a, 0xab, 0x7e, 0xf8, 0x74, 0xd3, 0x46, 0x6c, + 0x9c, 0xe2, 0xef, 0x67, 0x79, 0x72, 0x6e, 0x98, 0x85, 0xd6, 0x91, 0xb7, 0xa2, 0x74, 0xc9, 0x5b, + 0xf9, 0x97, 0x0a, 0x4c, 0x49, 0x89, 0x69, 0xe5, 0xf8, 0x0d, 0xd1, 0xdd, 0xa3, 0xc3, 0xd9, 0x89, + 0xad, 0x76, 0x83, 0x47, 0x0e, 0x40, 0x4d, 0x34, 0xe3, 0xc4, 0x0c, 0x8f, 0xfc, 0x50, 0x81, 0xcb, + 0x52, 0x56, 0x5b, 0x47, 0x52, 0x9c, 0x04, 0x2b, 0x85, 0xb0, 0xfe, 0xf1, 0xd1, 0xe1, 0xec, 0x85, + 0x76, 0xca, 0x5b, 0x34, 0x4d, 0xee, 0x91, 0x31, 0x5e, 0x70, 0x13, 0x29, 0x1b, 0x1e, 0xf9, 0x8a, + 0x02, 0x85, 0x68, 0x26, 0x9e, 0x04, 0x31, 0x8d, 0x10, 0xef, 0x1d, 0x1d, 0xce, 0x4e, 0xae, 0x4b, + 0x79, 0x79, 0x8f, 0x0c, 0x6b, 0xd2, 0xee, 0xa0, 0x66, 0x78, 0xe4, 0x00, 0x48, 0x90, 0x9b, 0x27, + 0x61, 0x18, 0x44, 0x0c, 0x77, 0x8e, 0x0e, 0x67, 0xc7, 0xd7, 0x79, 0xa6, 0xde, 0x23, 0x0f, 0x3f, + 0x6e, 0xcb, 0x84, 0x0c, 0x8f, 0x7c, 0x43, 0x81, 0xb3, 0xb1, 0x8c, 0x42, 0x09, 0xc1, 0x10, 0x22, + 0xd8, 0x38, 0x3a, 0x9c, 0x3d, 0xb3, 0x15, 0x6d, 0xf4, 0xc8, 0x48, 0xce, 0x34, 0xbb, 0x11, 0x34, + 0x3c, 0xf2, 0x35, 0x05, 0x0a, 0xd1, 0xd4, 0x45, 0x09, 0x50, 0x16, 0x01, 0x69, 0x47, 0x87, 0xb3, + 0xd3, 0x77, 0xf7, 0x3f, 0x56, 0x3c, 0xd3, 0xce, 0x7e, 0x57, 0x38, 0xdf, 0x57, 0x40, 0x3d, 0x2e, + 0x39, 0x52, 0x02, 0x36, 0x8c, 0xc0, 0xde, 0x3e, 0x3a, 0x9c, 0x9d, 0xb9, 0xdf, 0x35, 0x55, 0xf2, + 0x91, 0x01, 0xce, 0xec, 0x25, 0xd0, 0x35, 0x3c, 0xf2, 0x6d, 0x05, 0xce, 0x77, 0xe6, 0x62, 0x4a, + 0x10, 0x33, 0xed, 0xc5, 0xd4, 0xa2, 0x99, 0x99, 0x8f, 0xbe, 0x98, 0x6e, 0x37, 0x82, 0x86, 0x87, + 0xf9, 0xf3, 0x5d, 0xb5, 0x69, 0xaf, 0xfc, 0xf9, 0x5c, 0xe2, 0x3b, 0x58, 0xdd, 0xd5, 0xb6, 0xac, + 0x39, 0xa5, 0x70, 0xee, 0xed, 0x74, 0x46, 0xc9, 0x67, 0xd4, 0x57, 0x20, 0x7f, 0xcb, 0xf1, 0x1f, + 0xc2, 0xa6, 0x7d, 0x75, 0x18, 0x4e, 0x4b, 0x3d, 0x3f, 0x81, 0x37, 0x58, 0x7e, 0xa2, 0xc0, 0xd4, + 0x8e, 0xe3, 0xf3, 0xb5, 0xeb, 0xf2, 0xe2, 0xc4, 0x52, 0x02, 0x6b, 0x3a, 0x90, 0xb6, 0x4b, 0xa2, + 0xe6, 0xec, 0x9e, 0x30, 0x67, 0xa7, 0xe3, 0xf5, 0x0f, 0x6d, 0xcf, 0x4e, 0xef, 0xc4, 0x47, 0x2a, + 0xee, 0x43, 0x26, 0x20, 0x4f, 0x3e, 0x03, 0x69, 0x83, 0x7a, 0x55, 0xe1, 0x9f, 0xa8, 0x5d, 0xb2, + 0x2a, 0xb1, 0x5d, 0x10, 0xc4, 0x0e, 0x3d, 0x46, 0xec, 0x75, 0x4c, 0xa6, 0xf0, 0x40, 0xf7, 0x4c, + 0xe1, 0xe2, 0xef, 0x29, 0x30, 0x8a, 0xe9, 0x4a, 0xe1, 0x7a, 0x3d, 0xee, 0x5c, 0xa8, 0xb7, 0x01, + 0xda, 0x4b, 0x26, 0xd6, 0xe9, 0xa5, 0x87, 0x5a, 0xa7, 0xf0, 0xe5, 0x82, 0xa0, 0x45, 0xf1, 0xdf, + 0x2a, 0xdc, 0x2d, 0x08, 0x27, 0xd3, 0x97, 0x5b, 0xa0, 0xb1, 0x63, 0x88, 0xe3, 0x86, 0x68, 0x5e, + 0x3b, 0x11, 0x9a, 0x08, 0xf7, 0x34, 0x41, 0xa9, 0xf8, 0xcf, 0x61, 0xba, 0xbb, 0x38, 0x75, 0xd9, + 0xcf, 0x77, 0xa3, 0xfb, 0xf9, 0x53, 0x27, 0x1a, 0x5e, 0x9e, 0xae, 0x7c, 0x35, 0x73, 0x05, 0x46, + 0xfa, 0xcd, 0x0a, 0xfa, 0xc1, 0xa0, 0xc8, 0x0f, 0xfc, 0x44, 0xf6, 0xac, 0x7c, 0xe1, 0x39, 0xf0, + 0x18, 0x2e, 0x3c, 0x7f, 0x4b, 0x81, 0x49, 0x57, 0x4c, 0x24, 0xa2, 0x12, 0xf8, 0xbd, 0xe5, 0xe7, + 0x7a, 0x5d, 0xf1, 0x4a, 0xef, 0xd5, 0x08, 0x22, 0xc7, 0xa8, 0x83, 0x78, 0xfd, 0xc3, 0xab, 0x03, + 0x37, 0x3e, 0x52, 0xf1, 0x5b, 0x71, 0x41, 0x2e, 0x42, 0x26, 0x68, 0x15, 0x44, 0x95, 0xdc, 0x63, + 0x85, 0xbc, 0xdb, 0xfb, 0xb3, 0x6f, 0x04, 0x41, 0x86, 0xd4, 0x89, 0x73, 0x2c, 0x45, 0x58, 0xe1, + 0x3d, 0x98, 0xee, 0xce, 0x92, 0x2e, 0x22, 0x7d, 0x27, 0x2a, 0xd2, 0x2f, 0xf5, 0xcd, 0xf4, 0x63, + 0xc4, 0x59, 0x24, 0xa4, 0x3c, 0x07, 0x64, 0xb9, 0xfd, 0x6e, 0x7b, 0xcf, 0x5c, 0x82, 0x39, 0xa1, + 0xdb, 0x7a, 0xb7, 0xfc, 0xf1, 0x00, 0x8c, 0x60, 0xd3, 0xe0, 0x35, 0xe7, 0xc7, 0xad, 0x05, 0x9f, + 0x81, 0xd3, 0xd4, 0xae, 0xba, 0x2d, 0xbc, 0x83, 0x0c, 0xd2, 0xcd, 0xf1, 0x8c, 0xae, 0xe5, 0xdb, + 0x15, 0x22, 0x9e, 0x31, 0x1b, 0x9c, 0x5b, 0x79, 0x82, 0x08, 0x3f, 0xe2, 0xf2, 0xa3, 0x28, 0xe6, + 0x90, 0xb4, 0x1b, 0xf0, 0x33, 0x70, 0x5a, 0x6a, 0xc0, 0x4f, 0xbe, 0x73, 0x90, 0x17, 0x11, 0xf5, + 0x5d, 0xda, 0x12, 0x64, 0xf8, 0xdb, 0x44, 0xe2, 0x3e, 0xe3, 0x0e, 0x6d, 0x71, 0x52, 0xd1, 0x96, + 0x9c, 0xde, 0x50, 0xac, 0x25, 0xd2, 0x54, 0xdf, 0x82, 0xb1, 0x80, 0xbb, 0x61, 0xf2, 0x5b, 0xa0, + 0x48, 0x79, 0x08, 0xe5, 0xe9, 0xc4, 0x3c, 0xa0, 0x36, 0xb7, 0x83, 0x88, 0x07, 0xef, 0xac, 0xbe, + 0x0a, 0xa7, 0xdb, 0xf7, 0xad, 0x27, 0xf2, 0x40, 0x7e, 0x92, 0x01, 0x22, 0x77, 0x15, 0xb8, 0x1a, + 0x98, 0x7b, 0x26, 0x4a, 0x05, 0xb6, 0xdb, 0xc9, 0x39, 0x4a, 0x31, 0x12, 0xa5, 0x25, 0xc7, 0xb2, + 0x68, 0xd5, 0xa7, 0x46, 0x58, 0xd7, 0xf1, 0x0e, 0x81, 0x34, 0x06, 0x59, 0x02, 0xc0, 0x6b, 0x0c, + 0x97, 0x7a, 0xf4, 0x64, 0x57, 0x87, 0x59, 0xd6, 0x4f, 0x63, 0xdd, 0xc8, 0x2b, 0x50, 0x30, 0x6d, + 0x9f, 0xba, 0xb6, 0x6e, 0x95, 0xf5, 0x46, 0x03, 0x2f, 0xa6, 0xca, 0x0d, 0x97, 0x6e, 0x9b, 0x07, + 0xe2, 0x7e, 0x6a, 0x2a, 0xa8, 0x5f, 0x68, 0x34, 0xd6, 0xf5, 0x3a, 0xbd, 0x87, 0x95, 0xe4, 0x4b, + 0x30, 0x82, 0x2f, 0xb1, 0xb1, 0x05, 0x73, 0xec, 0xe0, 0x8e, 0x6a, 0xf3, 0x64, 0x33, 0x5e, 0x11, + 0xef, 0x78, 0x87, 0x33, 0xdf, 0x6c, 0x93, 0xec, 0x98, 0x7b, 0x64, 0xbc, 0xe2, 0x6f, 0x2b, 0x30, + 0x13, 0x74, 0xef, 0xc2, 0x2f, 0x9e, 0x67, 0x9f, 0x61, 0xf2, 0x15, 0x26, 0x35, 0xc6, 0xe2, 0x30, + 0x7b, 0x56, 0xa9, 0x7b, 0xc7, 0x20, 0x03, 0x6f, 0x97, 0xb6, 0x96, 0x75, 0x5f, 0x97, 0x2d, 0xd5, + 0xc0, 0xc7, 0x6c, 0xa9, 0x8a, 0xff, 0x6d, 0x00, 0xce, 0x27, 0xad, 0x3b, 0x31, 0xda, 0x7a, 0x2f, + 0x77, 0x6d, 0xed, 0xe1, 0xd8, 0x9b, 0x38, 0x4d, 0xd4, 0xa5, 0x9f, 0xc7, 0x6b, 0x3b, 0xdc, 0xf4, + 0x8b, 0xb7, 0xf8, 0xb5, 0xdd, 0x47, 0x87, 0xb3, 0xaf, 0x9f, 0x50, 0xff, 0xd4, 0xfd, 0x1b, 0xa6, + 0x5d, 0xa3, 0x6e, 0xc3, 0x35, 0x6d, 0x5f, 0x5c, 0xfc, 0xbd, 0x1e, 0xe4, 0x67, 0x0f, 0x74, 0x5e, + 0xd5, 0x75, 0x5f, 0x81, 0x48, 0x5e, 0x76, 0xf1, 0xe7, 0x0a, 0x5c, 0xee, 0x4f, 0x4c, 0x88, 0xc6, + 0xb7, 0xa0, 0x27, 0xaf, 0xf8, 0x73, 0xb1, 0xf1, 0xfa, 0x92, 0xb4, 0x2c, 0x92, 0x79, 0xdc, 0x6b, + 0xaf, 0x7e, 0x65, 0x00, 0x8a, 0xe1, 0xfc, 0x23, 0x36, 0xa7, 0xe1, 0xb8, 0x3e, 0x19, 0x0b, 0x93, + 0x13, 0x52, 0xc8, 0xc9, 0xf3, 0x90, 0xad, 0x3a, 0xf5, 0x86, 0x45, 0x7d, 0x6a, 0x88, 0x14, 0xc2, + 0x76, 0x01, 0xb9, 0x0e, 0x53, 0xa1, 0x6e, 0x28, 0x6f, 0xb7, 0x97, 0x41, 0x5c, 0x2a, 0x4d, 0x86, + 0x95, 0xd2, 0x12, 0x91, 0xd7, 0xa1, 0xd0, 0xee, 0x24, 0x7d, 0xd1, 0x85, 0x4d, 0x17, 0x5f, 0x1f, + 0x41, 0x86, 0x28, 0xda, 0xb4, 0xd7, 0x05, 0x26, 0x26, 0x7e, 0x8d, 0xb8, 0x5c, 0x77, 0x52, 0xa3, + 0xac, 0xfb, 0xe2, 0xa5, 0xf8, 0xfe, 0x94, 0x50, 0x2e, 0xec, 0xb9, 0xe0, 0xab, 0x5f, 0x84, 0xa7, + 0x97, 0x5c, 0xaa, 0xfb, 0xf4, 0x78, 0x7e, 0x04, 0x4a, 0xfa, 0xd8, 0x89, 0x2a, 0xc7, 0x4f, 0x54, + 0x6d, 0xc1, 0x5c, 0x6f, 0xfa, 0x42, 0x93, 0xbf, 0x09, 0x43, 0x2e, 0x96, 0x08, 0x11, 0x7a, 0xa9, + 0x9f, 0x4d, 0xd7, 0x49, 0x4e, 0x10, 0x51, 0x9f, 0x04, 0xf5, 0xf8, 0x56, 0xe1, 0xf5, 0xd0, 0x3f, + 0x83, 0x4b, 0x89, 0xad, 0x04, 0xb6, 0x2d, 0x18, 0xe6, 0x64, 0x03, 0x13, 0xf3, 0x70, 0xe0, 0x02, + 0x0d, 0x27, 0x68, 0xa9, 0xff, 0x53, 0x81, 0xc9, 0x6e, 0xad, 0x3b, 0x64, 0xf0, 0x58, 0xe6, 0x0f, + 0x24, 0x48, 0xd9, 0x4d, 0x18, 0xa9, 0x06, 0xdb, 0x8e, 0x49, 0xc9, 0x49, 0x4c, 0x55, 0x2e, 0xec, + 0xb9, 0x10, 0x64, 0x0c, 0xbf, 0x05, 0xe7, 0xba, 0xcf, 0x8c, 0xcb, 0xc7, 0xab, 0x09, 0x32, 0xcd, + 0x27, 0x72, 0x8c, 0x34, 0xab, 0x7b, 0x70, 0xbe, 0x3b, 0xe1, 0xf0, 0xa5, 0xfe, 0x9c, 0x44, 0x4f, + 0x28, 0xb4, 0xf9, 0x93, 0x2e, 0x80, 0x4c, 0x43, 0xbd, 0x0e, 0x85, 0xdb, 0x4e, 0x25, 0xb8, 0x99, + 0x14, 0x97, 0x4d, 0xbd, 0x1c, 0xc9, 0xbf, 0x54, 0xe0, 0x6c, 0x97, 0x5e, 0x9f, 0xc0, 0xb9, 0xea, + 0x0b, 0x30, 0xe2, 0x36, 0x6d, 0xdb, 0xb4, 0x6b, 0xe5, 0x07, 0x4e, 0x25, 0x38, 0xcb, 0x26, 0x65, + 0x34, 0x1e, 0x8b, 0x13, 0x6b, 0x72, 0x82, 0xda, 0x6d, 0xa7, 0xe2, 0x15, 0xa7, 0x20, 0x75, 0xdb, + 0xa9, 0xc4, 0x45, 0x4e, 0xbd, 0x02, 0xf9, 0xdb, 0x4e, 0x25, 0xca, 0x9a, 0x29, 0x18, 0x7a, 0xe0, + 0x54, 0xda, 0x2b, 0x3a, 0xf8, 0xc0, 0xa9, 0xac, 0x1a, 0xea, 0x0a, 0x9c, 0x96, 0x9a, 0x0a, 0x7e, + 0x3c, 0x0f, 0xa9, 0x07, 0x4e, 0x45, 0xec, 0xe5, 0x99, 0x98, 0x39, 0xc0, 0x0f, 0x6b, 0xf1, 0x8f, + 0x6c, 0x21, 0x20, 0xd6, 0x54, 0xfd, 0x34, 0x4c, 0xa2, 0x73, 0xb4, 0x71, 0x7f, 0xed, 0xc4, 0x17, + 0x62, 0xea, 0x19, 0x98, 0x8a, 0x75, 0xe6, 0x38, 0xd4, 0x45, 0x38, 0xb7, 0x6a, 0x1b, 0xf4, 0x00, + 0x3f, 0x66, 0xd3, 0x36, 0x39, 0x27, 0x22, 0xee, 0xc2, 0xf9, 0xee, 0x34, 0xc4, 0x5c, 0x85, 0x05, + 0xe4, 0xa5, 0xdd, 0xbe, 0x18, 0x22, 0x5b, 0xc0, 0x6e, 0x94, 0x64, 0x37, 0x93, 0x97, 0x5c, 0x9d, + 0x03, 0x68, 0xe7, 0xd3, 0x93, 0x49, 0xc8, 0xdf, 0xbc, 0xab, 0xdd, 0xdd, 0xda, 0x5c, 0x5d, 0x5f, + 0x29, 0x6f, 0x6c, 0x2e, 0x2c, 0xdd, 0xd9, 0xc8, 0x9f, 0x52, 0xd3, 0x19, 0x25, 0xaf, 0x5c, 0x7d, + 0x12, 0x32, 0x41, 0x7e, 0xba, 0x94, 0xa0, 0x3d, 0x06, 0x10, 0xf6, 0xd8, 0xc8, 0x2b, 0xd7, 0x7e, + 0x54, 0x0a, 0xef, 0x50, 0xbf, 0xab, 0xc0, 0x88, 0xfc, 0xfd, 0x1d, 0x52, 0xea, 0xef, 0x0b, 0x3b, + 0x01, 0xd3, 0x8a, 0xf3, 0x7d, 0xb7, 0x17, 0x8b, 0xf0, 0xf4, 0x07, 0x7f, 0xf0, 0x67, 0xff, 0x71, + 0xe0, 0x22, 0x99, 0x9d, 0x17, 0xe7, 0xa1, 0x79, 0xf9, 0xf3, 0x3c, 0xf3, 0xef, 0x89, 0x25, 0x78, + 0x9f, 0xfc, 0x2b, 0x05, 0x86, 0x83, 0x73, 0x5a, 0x52, 0x86, 0x6c, 0xf4, 0x6b, 0x3e, 0xc5, 0xab, + 0xfd, 0x34, 0x15, 0x58, 0x54, 0xc4, 0x72, 0x9e, 0x14, 0x43, 0x2c, 0xe2, 0x35, 0x18, 0x09, 0x46, + 0x05, 0x86, 0xc5, 0xa7, 0x38, 0x12, 0x51, 0x44, 0xbf, 0x48, 0x92, 0x88, 0x22, 0xf6, 0x65, 0x0f, + 0xf5, 0x14, 0x71, 0x61, 0x10, 0x3f, 0xf0, 0x42, 0x9e, 0xee, 0xfd, 0x09, 0x18, 0x4e, 0x7f, 0xae, + 0xdf, 0x6f, 0xc5, 0xa8, 0xd3, 0x38, 0xc7, 0x3c, 0x19, 0x0b, 0xe7, 0xc8, 0xbf, 0x45, 0xf3, 0x25, + 0x48, 0x63, 0xde, 0xfd, 0xe5, 0x1e, 0x94, 0x82, 0x11, 0x4f, 0xf4, 0xd9, 0x1b, 0xf5, 0x02, 0x8e, + 0x5a, 0x24, 0x85, 0xe8, 0xa8, 0x12, 0x5f, 0xdf, 0xe7, 0x1f, 0x25, 0xc1, 0x5c, 0x6b, 0xf2, 0x4c, + 0x7f, 0x19, 0xd9, 0xc7, 0x23, 0x39, 0x36, 0x7d, 0x5b, 0x9d, 0x42, 0x24, 0xe3, 0x64, 0x34, 0x44, + 0xe2, 0xea, 0xdb, 0x3e, 0xf9, 0xb2, 0x02, 0x43, 0x3c, 0xc6, 0x46, 0x7a, 0xbe, 0x90, 0x1e, 0x72, + 0xfd, 0x4a, 0x1f, 0x2d, 0xc5, 0xb0, 0x17, 0x71, 0xd8, 0x73, 0xe4, 0xac, 0x34, 0x2c, 0x6b, 0x20, + 0x71, 0xc0, 0x83, 0x21, 0xfe, 0x32, 0x6b, 0x22, 0x82, 0xc8, 0xfb, 0xae, 0x45, 0xf9, 0x9d, 0x21, + 0xf1, 0xb1, 0xc2, 0x55, 0x7b, 0xdb, 0x11, 0x5c, 0xef, 0x1c, 0x54, 0x7c, 0xd7, 0xb0, 0x3d, 0xe8, + 0x7f, 0x52, 0x20, 0x27, 0xbd, 0x85, 0x49, 0x9e, 0xeb, 0xef, 0x6d, 0xcd, 0x60, 0xfc, 0x52, 0xbf, + 0xcd, 0x05, 0x1b, 0x2e, 0x23, 0xa2, 0x0b, 0x64, 0x26, 0x44, 0xc4, 0x2f, 0xdd, 0xd1, 0xbd, 0x97, + 0x60, 0x7d, 0x4b, 0x81, 0x6c, 0xf8, 0x9a, 0x5c, 0xa2, 0x38, 0xc4, 0x5f, 0x0e, 0x4c, 0x14, 0x87, + 0x8e, 0x37, 0xf7, 0xd4, 0x2b, 0x08, 0xe8, 0x12, 0xb9, 0x18, 0x02, 0xd2, 0x83, 0x36, 0x28, 0xa2, + 0x12, 0xa6, 0x0f, 0x15, 0x18, 0x8b, 0xbe, 0x46, 0x49, 0x9e, 0xef, 0x6b, 0x2c, 0x29, 0x20, 0x5b, + 0x7c, 0xe1, 0x04, 0x3d, 0x04, 0xc4, 0x67, 0x10, 0xe2, 0x53, 0xe4, 0x52, 0x17, 0x88, 0x28, 0x44, + 0xf3, 0xef, 0x05, 0xa1, 0xd5, 0xf7, 0xc9, 0x57, 0x15, 0x18, 0x91, 0x73, 0xff, 0x12, 0x15, 0x78, + 0x97, 0xec, 0xde, 0x44, 0x05, 0xde, 0x2d, 0xb7, 0x51, 0x3d, 0x8b, 0xf0, 0x26, 0xc8, 0xe9, 0x10, + 0x5e, 0x98, 0xb0, 0xf8, 0x1d, 0x91, 0xc3, 0xb6, 0xe6, 0x54, 0x75, 0xeb, 0x93, 0x43, 0x34, 0x8b, + 0x88, 0xce, 0x92, 0x33, 0x21, 0x22, 0xc6, 0x2e, 0xab, 0x1c, 0xe2, 0xfa, 0xae, 0x02, 0x39, 0x29, + 0x15, 0x31, 0x51, 0xe8, 0x3b, 0xb3, 0x24, 0x13, 0x85, 0xbe, 0x4b, 0x86, 0xa3, 0x7a, 0x15, 0xf1, + 0x3c, 0xa9, 0x4a, 0x26, 0x0e, 0x5b, 0xf1, 0x3c, 0xd7, 0xb6, 0x84, 0xbd, 0xa6, 0x5c, 0x25, 0xff, + 0x4b, 0xa4, 0x90, 0xc6, 0x53, 0xed, 0xc8, 0xcb, 0x3d, 0xb8, 0x70, 0x4c, 0x1e, 0x61, 0xf1, 0x95, + 0x13, 0xf7, 0x3b, 0xd6, 0x18, 0xb6, 0xb3, 0xf8, 0xca, 0x22, 0x53, 0xef, 0xff, 0x2a, 0x70, 0x36, + 0x5c, 0xe0, 0x5f, 0x3d, 0xe4, 0x39, 0x84, 0xac, 0x92, 0x0b, 0xb1, 0x85, 0xef, 0x04, 0xfe, 0x5f, + 0x14, 0xc8, 0xc7, 0x53, 0xc2, 0xc8, 0xb5, 0x1e, 0xe3, 0x76, 0x49, 0x22, 0x2c, 0x5e, 0x3f, 0x51, + 0x1f, 0x81, 0x73, 0x06, 0x71, 0x16, 0xc8, 0x74, 0xdb, 0xcf, 0x30, 0x3d, 0xdf, 0xdb, 0xb3, 0xca, + 0x3c, 0x91, 0xec, 0xfb, 0xf8, 0x59, 0x35, 0xc1, 0xd6, 0x5f, 0x0d, 0xc4, 0x27, 0x11, 0xe2, 0x0c, + 0x39, 0x1f, 0x63, 0x65, 0x14, 0xe8, 0xf7, 0x14, 0x18, 0x8d, 0xe4, 0xc3, 0x92, 0xf9, 0x9e, 0x7b, + 0x23, 0x9a, 0xb4, 0x5b, 0x7c, 0xbe, 0xff, 0x0e, 0x02, 0xda, 0xb3, 0x08, 0xed, 0xb2, 0x7a, 0x31, + 0xbe, 0x9d, 0xc4, 0xfe, 0x8e, 0x6e, 0xa8, 0x2f, 0x2b, 0x90, 0x0d, 0x53, 0xcd, 0x12, 0x2d, 0x49, + 0x3c, 0xdd, 0x2e, 0xd1, 0x92, 0x74, 0x64, 0xaf, 0xa9, 0x05, 0x84, 0x45, 0xd4, 0xb6, 0x63, 0xe1, + 0x35, 0x74, 0x9b, 0x41, 0xf8, 0x12, 0xba, 0xd7, 0xd5, 0xdd, 0x64, 0xd7, 0x22, 0xf2, 0xce, 0x6d, + 0x31, 0xc9, 0xf3, 0x93, 0xdf, 0x11, 0xef, 0x62, 0xe3, 0x3d, 0x24, 0x24, 0x19, 0xae, 0x7f, 0xa1, + 0xc0, 0xb0, 0x78, 0xb5, 0x33, 0xd1, 0x67, 0x8d, 0xbe, 0xfe, 0xd9, 0x3f, 0x84, 0x4e, 0x4d, 0xd1, + 0xe0, 0x94, 0x62, 0x18, 0xc4, 0xcb, 0xa0, 0x89, 0x18, 0xa2, 0x2f, 0x8c, 0x3e, 0x0a, 0x86, 0x3a, + 0xa7, 0x24, 0x61, 0xf8, 0xd7, 0x0a, 0x64, 0x82, 0x17, 0x70, 0x49, 0x92, 0x47, 0x1e, 0x7b, 0x87, + 0xb8, 0xf8, 0x4c, 0x5f, 0x6d, 0x05, 0x92, 0x4e, 0x57, 0x17, 0x2f, 0x5e, 0xa2, 0x3e, 0xd7, 0x88, + 0xfc, 0x42, 0x78, 0xb2, 0x45, 0xec, 0x7c, 0xd3, 0x3c, 0xd9, 0x22, 0x76, 0x79, 0xd3, 0x5c, 0xbd, + 0x84, 0x98, 0x9e, 0x20, 0xe7, 0xa4, 0xdd, 0x5c, 0x8b, 0xc3, 0xfa, 0x86, 0x02, 0xc3, 0xa2, 0x77, + 0xe2, 0x12, 0x45, 0xdf, 0x3c, 0x2f, 0x3e, 0x97, 0xdc, 0x34, 0xf6, 0xde, 0x7d, 0x60, 0x0c, 0x89, + 0x9a, 0x00, 0x65, 0xfe, 0x3d, 0x56, 0xf0, 0x3e, 0x3b, 0x93, 0xac, 0x39, 0x35, 0x2f, 0xf1, 0x4c, + 0x22, 0x7d, 0x9c, 0xe0, 0xa4, 0x50, 0xba, 0xf9, 0x09, 0x35, 0x99, 0x23, 0xdf, 0x56, 0xf0, 0x1b, + 0x7b, 0xed, 0x5c, 0x9a, 0x44, 0xf5, 0xd6, 0x2d, 0x2d, 0x34, 0x51, 0xbd, 0x75, 0x4d, 0xd3, 0xe9, + 0x62, 0x1c, 0x44, 0x1e, 0xa4, 0x78, 0x5b, 0xf4, 0x03, 0x05, 0xb2, 0x61, 0x42, 0x40, 0xa2, 0x42, + 0x8b, 0xe7, 0xf3, 0x24, 0x2a, 0xb4, 0x8e, 0x1c, 0x03, 0xb5, 0x88, 0x40, 0x26, 0x09, 0x09, 0x81, + 0xec, 0x38, 0xbe, 0x00, 0xf1, 0x3e, 0x0c, 0x72, 0x0f, 0xf8, 0xe9, 0xde, 0x77, 0xbc, 0xbd, 0x4f, + 0xa8, 0x51, 0x7f, 0xf7, 0x98, 0xa3, 0x92, 0xec, 0xe5, 0x7e, 0xa8, 0x40, 0x4e, 0x0e, 0x8a, 0x26, + 0x67, 0x6d, 0xc7, 0x03, 0x92, 0xc5, 0xcf, 0x74, 0x36, 0x97, 0x3f, 0x9e, 0x1e, 0xf9, 0xac, 0xba, + 0xd4, 0x9f, 0x47, 0x6a, 0xbb, 0x9c, 0x61, 0xe4, 0x6f, 0xb2, 0xb7, 0xa5, 0x87, 0x1d, 0x29, 0xf9, + 0x55, 0x69, 0x0f, 0xbd, 0x2f, 0xdd, 0x55, 0x27, 0x1e, 0x29, 0xa3, 0xf7, 0xae, 0x5d, 0x35, 0x3f, + 0x6b, 0x20, 0x41, 0xf8, 0x37, 0x0a, 0x86, 0x8a, 0x82, 0xfb, 0xc9, 0x67, 0xfb, 0xbc, 0xac, 0xea, + 0xbd, 0x9b, 0x3a, 0xaf, 0xb6, 0xd4, 0x73, 0x08, 0x67, 0x8a, 0x4c, 0xc8, 0x86, 0x28, 0x18, 0xf9, + 0x67, 0x0a, 0x5c, 0xe8, 0x15, 0xee, 0x27, 0x8b, 0x49, 0xae, 0x40, 0x7f, 0x77, 0x11, 0xc5, 0xa5, + 0x47, 0xa2, 0x11, 0x55, 0x97, 0x6a, 0x41, 0x9a, 0x4a, 0xdd, 0x67, 0xab, 0x2c, 0xc2, 0xf3, 0xcc, + 0xaa, 0xff, 0x86, 0x72, 0x5c, 0x70, 0x1a, 0x91, 0x78, 0xe4, 0xb3, 0x0f, 0x75, 0x11, 0x10, 0xb2, + 0xff, 0xf5, 0x87, 0xed, 0x7e, 0xac, 0x1d, 0x8a, 0x4d, 0x82, 0xfc, 0xda, 0x71, 0x77, 0x0c, 0x2f, + 0x9f, 0x78, 0xe8, 0xde, 0x8e, 0x7b, 0x52, 0x1c, 0x5f, 0x7d, 0x11, 0xb1, 0x96, 0xc8, 0xb3, 0x1d, + 0x58, 0xe7, 0xdf, 0x3b, 0xee, 0xea, 0xe0, 0x7d, 0xf2, 0x03, 0x05, 0xa3, 0xcb, 0xd1, 0x68, 0x36, + 0xb9, 0x7e, 0xb2, 0xd8, 0x37, 0x47, 0xfe, 0xe2, 0xc3, 0x04, 0xcc, 0xbb, 0xc4, 0x2e, 0x1f, 0x38, + 0x95, 0xb2, 0x2b, 0x1a, 0x47, 0x3d, 0x8f, 0x6c, 0x18, 0x07, 0x4f, 0xd4, 0xd9, 0xf1, 0xc0, 0x7a, + 0xa2, 0xce, 0xee, 0x08, 0xad, 0xab, 0x4f, 0x20, 0xa2, 0x33, 0x64, 0x4a, 0x46, 0x34, 0xff, 0x1e, + 0x0f, 0xcd, 0xbf, 0xcf, 0x0e, 0xe4, 0xa3, 0x91, 0x58, 0x78, 0xa2, 0x41, 0xeb, 0x16, 0x72, 0x4f, + 0x34, 0x68, 0xdd, 0xc3, 0xec, 0x42, 0x4f, 0xa9, 0x6d, 0x83, 0x86, 0xf9, 0x11, 0xde, 0x9e, 0x85, + 0x51, 0x1f, 0xb6, 0x97, 0x7e, 0xa8, 0xc0, 0x64, 0xb7, 0xe0, 0x77, 0xa2, 0x24, 0x26, 0xc4, 0xee, + 0x13, 0x25, 0x31, 0x29, 0x5e, 0xaf, 0x3e, 0x85, 0x60, 0x67, 0xc9, 0x13, 0x21, 0x58, 0xfc, 0x87, + 0x3a, 0xf0, 0x3b, 0xf8, 0xed, 0x10, 0xfc, 0xe2, 0xd5, 0x9f, 0xfe, 0xe9, 0xcc, 0xa9, 0x9f, 0x1e, + 0xcd, 0x28, 0x3f, 0x3b, 0x9a, 0x51, 0x7e, 0x7e, 0x34, 0xa3, 0xfc, 0xc9, 0xd1, 0x8c, 0xf2, 0xcd, + 0x5f, 0xce, 0x9c, 0xfa, 0xd9, 0x2f, 0x67, 0x4e, 0xfd, 0xfc, 0x97, 0x33, 0xa7, 0xde, 0xce, 0x04, + 0xe3, 0x55, 0x86, 0xf0, 0x3a, 0xed, 0xfa, 0xdf, 0x07, 0x00, 0x00, 0xff, 0xff, 0x90, 0x4c, 0x45, + 0xf0, 0xa0, 0x64, 0x00, 0x00, } func (this *PrettySpan) Equal(that interface{}) bool { @@ -4806,6 +4884,7 @@ type StatusClient interface { JobRegistryStatus(ctx context.Context, in *JobRegistryStatusRequest, opts ...grpc.CallOption) (*JobRegistryStatusResponse, error) JobStatus(ctx context.Context, in *JobStatusRequest, opts ...grpc.CallOption) (*JobStatusResponse, error) ResetSQLStats(ctx context.Context, in *ResetSQLStatsRequest, opts ...grpc.CallOption) (*ResetSQLStatsResponse, error) + IndexUsageStatistics(ctx context.Context, in *IndexUsageStatisticsRequest, opts ...grpc.CallOption) (*IndexUsageStatisticsResponse, error) } type statusClient struct { @@ -5167,6 +5246,15 @@ func (c *statusClient) ResetSQLStats(ctx context.Context, in *ResetSQLStatsReque return out, nil } +func (c *statusClient) IndexUsageStatistics(ctx context.Context, in *IndexUsageStatisticsRequest, opts ...grpc.CallOption) (*IndexUsageStatisticsResponse, error) { + out := new(IndexUsageStatisticsResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/IndexUsageStatistics", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // StatusServer is the server API for Status service. type StatusServer interface { // Certificates retrieves a copy of the TLS certificates. @@ -5298,6 +5386,7 @@ type StatusServer interface { JobRegistryStatus(context.Context, *JobRegistryStatusRequest) (*JobRegistryStatusResponse, error) JobStatus(context.Context, *JobStatusRequest) (*JobStatusResponse, error) ResetSQLStats(context.Context, *ResetSQLStatsRequest) (*ResetSQLStatsResponse, error) + IndexUsageStatistics(context.Context, *IndexUsageStatisticsRequest) (*IndexUsageStatisticsResponse, error) } // UnimplementedStatusServer can be embedded to have forward compatible implementations. @@ -5421,6 +5510,9 @@ func (*UnimplementedStatusServer) JobStatus(ctx context.Context, req *JobStatusR func (*UnimplementedStatusServer) ResetSQLStats(ctx context.Context, req *ResetSQLStatsRequest) (*ResetSQLStatsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResetSQLStats not implemented") } +func (*UnimplementedStatusServer) IndexUsageStatistics(ctx context.Context, req *IndexUsageStatisticsRequest) (*IndexUsageStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method IndexUsageStatistics not implemented") +} func RegisterStatusServer(s *grpc.Server, srv StatusServer) { s.RegisterService(&_Status_serviceDesc, srv) @@ -6128,6 +6220,24 @@ func _Status_ResetSQLStats_Handler(srv interface{}, ctx context.Context, dec fun return interceptor(ctx, in, info, handler) } +func _Status_IndexUsageStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(IndexUsageStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StatusServer).IndexUsageStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Status/IndexUsageStatistics", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StatusServer).IndexUsageStatistics(ctx, req.(*IndexUsageStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Status_serviceDesc = grpc.ServiceDesc{ ServiceName: "cockroach.server.serverpb.Status", HandlerType: (*StatusServer)(nil), @@ -6288,6 +6398,10 @@ var _Status_serviceDesc = grpc.ServiceDesc{ MethodName: "ResetSQLStats", Handler: _Status_ResetSQLStats_Handler, }, + { + MethodName: "IndexUsageStatistics", + Handler: _Status_IndexUsageStatistics_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "server/serverpb/status.proto", @@ -11108,6 +11222,73 @@ func (m *ResetSQLStatsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *IndexUsageStatisticsRequest) 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 *IndexUsageStatisticsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *IndexUsageStatisticsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.NodeID) > 0 { + i -= len(m.NodeID) + copy(dAtA[i:], m.NodeID) + i = encodeVarintStatus(dAtA, i, uint64(len(m.NodeID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *IndexUsageStatisticsResponse) 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 *IndexUsageStatisticsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *IndexUsageStatisticsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Statistics) > 0 { + for iNdEx := len(m.Statistics) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Statistics[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStatus(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func encodeVarintStatus(dAtA []byte, offset int, v uint64) int { offset -= sovStatus(v) base := offset @@ -13119,6 +13300,34 @@ func (m *ResetSQLStatsResponse) Size() (n int) { return n } +func (m *IndexUsageStatisticsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeID) + if l > 0 { + n += 1 + l + sovStatus(uint64(l)) + } + return n +} + +func (m *IndexUsageStatisticsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Statistics) > 0 { + for _, e := range m.Statistics { + l = e.Size() + n += 1 + l + sovStatus(uint64(l)) + } + } + return n +} + func sovStatus(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -26891,6 +27100,172 @@ func (m *ResetSQLStatsResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *IndexUsageStatisticsRequest) 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 ErrIntOverflowStatus + } + 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: IndexUsageStatisticsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IndexUsageStatisticsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + 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 ErrInvalidLengthStatus + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthStatus + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStatus(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthStatus + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *IndexUsageStatisticsResponse) 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 ErrIntOverflowStatus + } + 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: IndexUsageStatisticsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IndexUsageStatisticsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStatus + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStatus + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Statistics = append(m.Statistics, roachpb.CollectedIndexUsageStatistics{}) + if err := m.Statistics[len(m.Statistics)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStatus(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthStatus + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipStatus(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/server/serverpb/status.pb.gw.go b/pkg/server/serverpb/status.pb.gw.go index 621b676bad8a..fa7ce214894e 100644 --- a/pkg/server/serverpb/status.pb.gw.go +++ b/pkg/server/serverpb/status.pb.gw.go @@ -1881,6 +1881,42 @@ func local_request_Status_ResetSQLStats_0(ctx context.Context, marshaler runtime } +var ( + filter_Status_IndexUsageStatistics_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_Status_IndexUsageStatistics_0(ctx context.Context, marshaler runtime.Marshaler, client StatusClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq IndexUsageStatisticsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_Status_IndexUsageStatistics_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.IndexUsageStatistics(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_Status_IndexUsageStatistics_0(ctx context.Context, marshaler runtime.Marshaler, server StatusServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq IndexUsageStatisticsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_Status_IndexUsageStatistics_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.IndexUsageStatistics(ctx, &protoReq) + return msg, metadata, err + +} + // RegisterStatusHandlerServer registers the http handlers for service Status to "mux". // UnaryRPC :call StatusServer directly. // StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. @@ -2761,6 +2797,29 @@ func RegisterStatusHandlerServer(ctx context.Context, mux *runtime.ServeMux, ser }) + mux.Handle("GET", pattern_Status_IndexUsageStatistics_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_Status_IndexUsageStatistics_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Status_IndexUsageStatistics_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } @@ -3562,6 +3621,26 @@ func RegisterStatusHandlerClient(ctx context.Context, mux *runtime.ServeMux, cli }) + mux.Handle("GET", pattern_Status_IndexUsageStatistics_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_Status_IndexUsageStatistics_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Status_IndexUsageStatistics_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } @@ -3641,6 +3720,8 @@ var ( pattern_Status_JobStatus_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"_status", "job", "job_id"}, "", runtime.AssumeColonVerbOpt(true))) pattern_Status_ResetSQLStats_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"_status", "resetsqlstats"}, "", runtime.AssumeColonVerbOpt(true))) + + pattern_Status_IndexUsageStatistics_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"_status", "indexusagestatistics"}, "", runtime.AssumeColonVerbOpt(true))) ) var ( @@ -3719,4 +3800,6 @@ var ( forward_Status_JobStatus_0 = runtime.ForwardResponseMessage forward_Status_ResetSQLStats_0 = runtime.ForwardResponseMessage + + forward_Status_IndexUsageStatistics_0 = runtime.ForwardResponseMessage ) diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 408e75d7436d..0870ae3b0cd9 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -17,6 +17,7 @@ import "gossip/gossip.proto"; import "jobs/jobspb/jobs.proto"; import "roachpb/app_stats.proto"; import "roachpb/data.proto"; +import "roachpb/index_usage_stats.proto"; import "roachpb/metadata.proto"; import "server/diagnostics/diagnosticspb/diagnostics.proto"; import "server/status/statuspb/status.proto"; @@ -1105,6 +1106,18 @@ message ResetSQLStatsRequest { message ResetSQLStatsResponse { } +// Request object for issuing IndexUsageStatistics request. +message IndexUsageStatisticsRequest { + // node_id is the ID of the node where the stats data shall be retrieved from. + // If this is left empty, the cluster-wide aggregated result will be returned. + string node_id = 1 [(gogoproto.customname) = "NodeID"]; +} + +// Response object returned by IndexUsageStatistics. +message IndexUsageStatisticsResponse { + repeated cockroach.sql.CollectedIndexUsageStatistics statistics = 1 [(gogoproto.nullable) = false]; +} + service Status { // Certificates retrieves a copy of the TLS certificates. rpc Certificates(CertificatesRequest) returns (CertificatesResponse) { @@ -1423,4 +1436,10 @@ service Status { body: "*" }; } + + rpc IndexUsageStatistics(IndexUsageStatisticsRequest) returns (IndexUsageStatisticsResponse) { + option (google.api.http) = { + get: "/_status/indexusagestatistics" + }; + } } diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index d436474f9593..11c7fe491979 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -140,3 +140,14 @@ func (t *tenantStatusServer) ListDistSQLFlows( ) (*serverpb.ListDistSQLFlowsResponse, error) { return t.ListLocalDistSQLFlows(ctx, request) } + +func (t *tenantStatusServer) IndexUsageStatistics( + ctx context.Context, request *serverpb.IndexUsageStatisticsRequest, +) (*serverpb.IndexUsageStatisticsResponse, error) { + if _, err := t.privilegeChecker.requireViewActivityPermission(ctx); err != nil { + return nil, err + } + + idxUsageStats := t.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics() + return indexUsageStatsLocal(idxUsageStats) +} diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 9433e2e179f4..24198e502f74 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -269,7 +269,8 @@ type Server struct { // pool is the parent monitor for all session monitors. pool *mon.BytesMonitor - // indexUsageStats tracks the index usage statistics. + // indexUsageStats tracks the index usage statistics queries that use current + // node as gateway node. indexUsageStats *idxusage.LocalIndexUsageStats // Metrics is used to account normal queries. @@ -334,6 +335,7 @@ func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server { indexUsageStats: idxusage.NewLocalIndexUsageStats(&idxusage.Config{ ChannelSize: idxusage.DefaultChannelSize, Setting: cfg.Settings, + Knobs: cfg.IndexUsageStatsTestingKnobs, }), } @@ -626,6 +628,11 @@ func (s *Server) ServeConn( return h.ex.run(ctx, s.pool, reserved, cancel) } +// GetLocalIndexStatistics returns a idxusage.LocalIndexUsageStats. +func (s *Server) GetLocalIndexStatistics() *idxusage.LocalIndexUsageStats { + return s.indexUsageStats +} + // newSessionData a SessionData that can be passed to newConnExecutor. func (s *Server) newSessionData(args SessionArgs) *sessiondata.SessionData { sd := &sessiondata.SessionData{ @@ -745,7 +752,7 @@ func (s *Server) newConnExecutor( executorType: executorTypeExec, hasCreatedTemporarySchema: false, stmtDiagnosticsRecorder: s.cfg.StmtDiagnosticsRecorder, - indexUsageStatsWriter: s.indexUsageStats, + indexUsageStats: s.indexUsageStats, } ex.state.txnAbortCount = ex.metrics.EngineMetrics.TxnAbortCount @@ -1293,8 +1300,8 @@ type connExecutor struct { // information collected. stmtDiagnosticsRecorder *stmtdiagnostics.Registry - // indexUsageStatsWriter is used to track index usage stats. - indexUsageStatsWriter idxusage.Writer + // indexUsageStats is used to track index usage stats. + indexUsageStats *idxusage.LocalIndexUsageStats } // ctxHolder contains a connection's context and, while session tracing is @@ -2302,21 +2309,21 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo SQLStatsResetter: ex.server, CompactEngineSpan: ex.server.cfg.CompactEngineSpanFunc, }, - SessionMutator: ex.dataMutator, - VirtualSchemas: ex.server.cfg.VirtualSchemas, - Tracing: &ex.sessionTracing, - NodesStatusServer: ex.server.cfg.NodesStatusServer, - RegionsServer: ex.server.cfg.RegionsServer, - SQLStatusServer: ex.server.cfg.SQLStatusServer, - MemMetrics: &ex.memMetrics, - Descs: &ex.extraTxnState.descCollection, - ExecCfg: ex.server.cfg, - DistSQLPlanner: ex.server.cfg.DistSQLPlanner, - TxnModesSetter: ex, - Jobs: &ex.extraTxnState.jobs, - SchemaChangeJobCache: ex.extraTxnState.schemaChangeJobsCache, - statsStorage: ex.server.sqlStats, - indexUsageStatsWriter: ex.indexUsageStatsWriter, + SessionMutator: ex.dataMutator, + VirtualSchemas: ex.server.cfg.VirtualSchemas, + Tracing: &ex.sessionTracing, + NodesStatusServer: ex.server.cfg.NodesStatusServer, + RegionsServer: ex.server.cfg.RegionsServer, + SQLStatusServer: ex.server.cfg.SQLStatusServer, + MemMetrics: &ex.memMetrics, + Descs: &ex.extraTxnState.descCollection, + ExecCfg: ex.server.cfg, + DistSQLPlanner: ex.server.cfg.DistSQLPlanner, + TxnModesSetter: ex, + Jobs: &ex.extraTxnState.jobs, + SchemaChangeJobCache: ex.extraTxnState.schemaChangeJobsCache, + statsStorage: ex.server.sqlStats, + indexUsageStats: ex.indexUsageStats, } } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 389014ce3038..6776d62de72a 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -60,6 +60,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -904,6 +905,7 @@ type ExecutorConfig struct { EvalContextTestingKnobs tree.EvalContextTestingKnobs TenantTestingKnobs *TenantTestingKnobs BackupRestoreTestingKnobs *BackupRestoreTestingKnobs + IndexUsageStatsTestingKnobs *idxusage.TestingKnobs // HistogramWindowInterval is (server.Config).HistogramWindowInterval. HistogramWindowInterval time.Duration diff --git a/pkg/sql/idxusage/BUILD.bazel b/pkg/sql/idxusage/BUILD.bazel index 5054d4073860..23b5cfec295d 100644 --- a/pkg/sql/idxusage/BUILD.bazel +++ b/pkg/sql/idxusage/BUILD.bazel @@ -19,13 +19,13 @@ go_library( name = "idxusage", srcs = [ "cluster_settings.go", - "index_usage_stats.go", "local_idx_usage_stats.go", "test_utils.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/idxusage", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/roachpb:with-mocks", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/sql/idxusage/index_usage_stats.go b/pkg/sql/idxusage/index_usage_stats.go deleted file mode 100644 index 425be80a7782..000000000000 --- a/pkg/sql/idxusage/index_usage_stats.go +++ /dev/null @@ -1,53 +0,0 @@ -// 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. -// -// idxusage is a subsystem that is responsible for collecting index usage -// statistics. - -package idxusage - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/roachpb" -) - -// Writer provides interface to record index usage statistics. -type Writer interface { - // RecordRead records a read operation on the specified index. - RecordRead(ctx context.Context, key roachpb.IndexUsageKey) - - // TODO(azhng): as we introduce more plumbing throughout the codebase, - // we should introduce additional interfaces here to record other index usage - // type. -} - -// IteratorOptions provides knobs to change the iterating behavior when -// calling ForEach. -type IteratorOptions struct { - SortedTableID bool - SortedIndexID bool - Max *uint64 -} - -// StatsVisitor is the callback invoked when calling ForEach. -type StatsVisitor func(key *roachpb.IndexUsageKey, value *roachpb.IndexUsageStatistics) error - -// Reader provides interfaces to retrieve index usage statistics from the -// subsystem. -type Reader interface { - // Get returns the index usage statistics for a given key. - Get(key roachpb.IndexUsageKey) roachpb.IndexUsageStatistics - - // ForEach iterates through all stored index usage statistics - // based on the options specified in IteratorOptions. If an error is - // encountered when calling StatsVisitor, the iteration is aborted. - ForEach(options IteratorOptions, visitor StatsVisitor) error -} diff --git a/pkg/sql/idxusage/local_idx_usage_stats.go b/pkg/sql/idxusage/local_idx_usage_stats.go index f2d3c77c5679..723b13df206c 100644 --- a/pkg/sql/idxusage/local_idx_usage_stats.go +++ b/pkg/sql/idxusage/local_idx_usage_stats.go @@ -102,12 +102,20 @@ type Config struct { Knobs *TestingKnobs } +// IteratorOptions provides knobs to change the iterating behavior when +// calling ForEach. +type IteratorOptions struct { + SortedTableID bool + SortedIndexID bool + Max *uint64 +} + +// StatsVisitor is the callback invoked when calling ForEach. +type StatsVisitor func(key *roachpb.IndexUsageKey, value *roachpb.IndexUsageStatistics) error + // DefaultChannelSize is the default size of the statsChan. const DefaultChannelSize = uint64(128) -var _ Reader = &LocalIndexUsageStats{} -var _ Writer = &LocalIndexUsageStats{} - var emptyIndexUsageStats roachpb.IndexUsageStatistics // NewLocalIndexUsageStats returns a new instance of LocalIndexUsageStats. @@ -128,7 +136,7 @@ func (s *LocalIndexUsageStats) Start(ctx context.Context, stopper *stop.Stopper) s.startStatsIngestionLoop(ctx, stopper) } -// RecordRead implements the idxusage.Writer interface. +// RecordRead records a read operation on the specified index. func (s *LocalIndexUsageStats) RecordRead(ctx context.Context, key roachpb.IndexUsageKey) { s.record(ctx, indexUse{ key: key, @@ -150,7 +158,7 @@ func (s *LocalIndexUsageStats) record(ctx context.Context, payload indexUse) { } } -// Get implements the idxusage.Reader interface. +// Get returns the index usage statistics for a given key. func (s *LocalIndexUsageStats) Get(key roachpb.IndexUsageKey) roachpb.IndexUsageStatistics { s.mu.RLock() defer s.mu.RUnlock() @@ -177,7 +185,9 @@ func (s *LocalIndexUsageStats) Get(key roachpb.IndexUsageKey) roachpb.IndexUsage return indexStats.IndexUsageStatistics } -// ForEach the idxusage.Reader interface. +// ForEach iterates through all stored index usage statistics +// based on the options specified in IteratorOptions. If an error is +// encountered when calling StatsVisitor, the iteration is aborted. func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVisitor) error { maxIterationLimit := uint64(math.MaxUint64) if options.Max != nil { @@ -356,7 +366,7 @@ func (s *LocalIndexUsageStats) startStatsIngestionLoop(ctx context.Context, stop case payload := <-s.statsChan: s.insertIndexUsage(&payload) if s.testingKnobs != nil && s.testingKnobs.OnIndexUsageStatsProcessedCallback != nil { - s.testingKnobs.OnIndexUsageStatsProcessedCallback() + s.testingKnobs.OnIndexUsageStatsProcessedCallback(payload.key) } case <-stopper.ShouldQuiesce(): return diff --git a/pkg/sql/idxusage/local_index_usage_stats_test.go b/pkg/sql/idxusage/local_index_usage_stats_test.go index 6ef274420092..36d32e104f08 100644 --- a/pkg/sql/idxusage/local_index_usage_stats_test.go +++ b/pkg/sql/idxusage/local_index_usage_stats_test.go @@ -111,7 +111,7 @@ func TestIndexUsageStatisticsSubsystem(t *testing.T) { } statsProcessedSignal := make(chan struct{}) - onStatsIngested := func() { + onStatsIngested := func(_ roachpb.IndexUsageKey) { statsProcessedSignal <- struct{}{} } waitForStatsIngested := func() { diff --git a/pkg/sql/idxusage/test_utils.go b/pkg/sql/idxusage/test_utils.go index f03f825ffa30..1da46fc34826 100644 --- a/pkg/sql/idxusage/test_utils.go +++ b/pkg/sql/idxusage/test_utils.go @@ -10,10 +10,20 @@ package idxusage +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + // TestingKnobs is the testing knobs that provides callbacks that unit tests // can hook into. type TestingKnobs struct { // OnIndexUsageStatsProcessedCallback is invoked whenever a index usage event // is processed. - OnIndexUsageStatsProcessedCallback func() + OnIndexUsageStatsProcessedCallback func(key roachpb.IndexUsageKey) } + +var _ base.ModuleTestingKnobs = &TestingKnobs{} + +// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. +func (t *TestingKnobs) ModuleTestingKnobs() {} diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 2e387f3ef7be..95f6b21e98a7 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -134,7 +134,7 @@ func (ef *execFactory) ConstructScan( TableID: roachpb.TableID(tabDesc.GetID()), IndexID: roachpb.IndexID(idx.GetID()), } - ef.planner.extendedEvalCtx.indexUsageStatsWriter.RecordRead(ctx, idxUsageKey) + ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(ctx, idxUsageKey) } return scan, nil @@ -679,7 +679,7 @@ func (ef *execFactory) ConstructLookupJoin( TableID: roachpb.TableID(tabDesc.GetID()), IndexID: roachpb.IndexID(idx.GetID()), } - ef.planner.extendedEvalCtx.indexUsageStatsWriter.RecordRead(ctx, idxUsageKey) + ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(ctx, idxUsageKey) } n := &lookupJoinNode{ @@ -809,7 +809,7 @@ func (ef *execFactory) ConstructInvertedJoin( TableID: roachpb.TableID(tabDesc.GetID()), IndexID: roachpb.IndexID(idx.GetID()), } - ef.planner.extendedEvalCtx.indexUsageStatsWriter.RecordRead(ctx, idxUsageKey) + ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(ctx, idxUsageKey) } n := &invertedJoinNode{ @@ -873,7 +873,7 @@ func (ef *execFactory) constructScanForZigzag( TableID: roachpb.TableID(tableDesc.GetID()), IndexID: roachpb.IndexID(index.GetID()), } - ef.planner.extendedEvalCtx.indexUsageStatsWriter.RecordRead(ctx, idxUsageKey) + ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(ctx, idxUsageKey) } scan.index = index diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 05b58370ca14..4717326f634f 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -97,7 +97,7 @@ type extendedEvalContext struct { statsStorage sqlstats.Storage - indexUsageStatsWriter idxusage.Writer + indexUsageStats *idxusage.LocalIndexUsageStats SchemaChangerState *SchemaChangerState } @@ -416,7 +416,7 @@ func internalExtendedEvalCtx( ) extendedEvalContext { evalContextTestingKnobs := execCfg.EvalContextTestingKnobs - var indexUsageStats idxusage.Writer + var indexUsageStats *idxusage.LocalIndexUsageStats var sqlStatsResetter tree.SQLStatsResetter if execCfg.InternalExecutor != nil { sqlStatsResetter = execCfg.InternalExecutor.s @@ -448,15 +448,15 @@ func internalExtendedEvalCtx( InternalExecutor: execCfg.InternalExecutor, SQLStatsResetter: sqlStatsResetter, }, - SessionMutator: dataMutator, - VirtualSchemas: execCfg.VirtualSchemas, - Tracing: &SessionTracing{}, - NodesStatusServer: execCfg.NodesStatusServer, - RegionsServer: execCfg.RegionsServer, - Descs: tables, - ExecCfg: execCfg, - DistSQLPlanner: execCfg.DistSQLPlanner, - indexUsageStatsWriter: indexUsageStats, + SessionMutator: dataMutator, + VirtualSchemas: execCfg.VirtualSchemas, + Tracing: &SessionTracing{}, + NodesStatusServer: execCfg.NodesStatusServer, + RegionsServer: execCfg.RegionsServer, + Descs: tables, + ExecCfg: execCfg, + DistSQLPlanner: execCfg.DistSQLPlanner, + indexUsageStats: indexUsageStats, } } From 584fb971e3bbdaa750b54ad6425748e9673058b7 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 26 Jul 2021 01:40:33 -0400 Subject: [PATCH 2/2] kv: grab raftMu during no-op writes with local gossip triggers Fixes #68011. As of 9f8c019, it is now possible to have no-op writes that do not go through Raft but do set one of the gossip triggers. These gossip triggers require the raftMu to be held, so we were running into trouble when handling the local eval results above Raft. For instance, we see this case when a transaction sets the system config trigger and then performs a delete range over an empty span before committing. In this case, the transaction will have no intents to remove, so it can auto-GC its record during an EndTxn. If its record was never written in the first place, this is a no-op (as of 9f8c019). There appear to be three ways we could solve this: 1. we can avoid setting gossip triggers on transactions that don't perform any writes. 2. we can force EndTxn requests with gossip triggers to go through Raft even if they are otherwise no-ops. 3. we can properly handle gossip triggers on the above Raft local eval result path. This commit opts for the third option. --- .../replica_application_state_machine.go | 2 +- pkg/kv/kvserver/replica_proposal.go | 18 +++++++++++++++++- pkg/kv/kvserver/replica_raft.go | 2 +- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 3d4fc8d53fc7..80de1ca9c959 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -1206,7 +1206,7 @@ func (sm *replicaStateMachine) ApplySideEffects( if cmd.IsLocal() { // Handle the LocalResult. if cmd.localResult != nil { - sm.r.handleReadWriteLocalEvalResult(ctx, *cmd.localResult) + sm.r.handleReadWriteLocalEvalResult(ctx, *cmd.localResult, true /* raftMuHeld */) } rejected := cmd.Rejected() diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 5837fa2bd751..f8f3fbcc3238 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -641,7 +641,9 @@ func addSSTablePreApply( return copied } -func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult result.LocalResult) { +func (r *Replica) handleReadWriteLocalEvalResult( + ctx context.Context, lResult result.LocalResult, raftMuHeld bool, +) { // Fields for which no action is taken in this method are zeroed so that // they don't trigger an assertion at the end of the method (which checks // that all fields were handled). @@ -707,7 +709,19 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re lResult.MaybeAddToSplitQueue = false } + // The following three triggers require the raftMu to be held. If a + // trigger is present, acquire the mutex if it is not held already. + maybeAcquireRaftMu := func() func() { + if raftMuHeld { + return func() {} + } + raftMuHeld = true + r.raftMu.Lock() + return r.raftMu.Unlock + } + if lResult.MaybeGossipSystemConfig { + defer maybeAcquireRaftMu()() if err := r.MaybeGossipSystemConfigRaftMuLocked(ctx); err != nil { log.Errorf(ctx, "%v", err) } @@ -715,6 +729,7 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re } if lResult.MaybeGossipSystemConfigIfHaveFailure { + defer maybeAcquireRaftMu()() if err := r.MaybeGossipSystemConfigIfHaveFailureRaftMuLocked(ctx); err != nil { log.Errorf(ctx, "%v", err) } @@ -722,6 +737,7 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re } if lResult.MaybeGossipNodeLiveness != nil { + defer maybeAcquireRaftMu()() if err := r.MaybeGossipNodeLivenessRaftMuLocked(ctx, *lResult.MaybeGossipNodeLiveness); err != nil { log.Errorf(ctx, "%v", err) } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 67125e639c3a..05a7d0b04bcd 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -115,7 +115,7 @@ func (r *Replica) evalAndPropose( if proposal.command == nil { intents := proposal.Local.DetachEncounteredIntents() endTxns := proposal.Local.DetachEndTxns(pErr != nil /* alwaysOnly */) - r.handleReadWriteLocalEvalResult(ctx, *proposal.Local) + r.handleReadWriteLocalEvalResult(ctx, *proposal.Local, false /* raftMuHeld */) pr := proposalResult{ Reply: proposal.Local.Reply,