From 91bb5880bc26a1447ed8a2ed6d6ddd900a863048 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Wed, 21 Aug 2019 19:54:47 -0400 Subject: [PATCH] storage: introduce a couple of replication reports This patch implements the "Insights into Constraint Conformance" RFC (#38309). At the time of this patch, the RFC is still pending and also out of date. Developed together with Darin. The patch introduces the following tables in the system database, providing information about constraint conformance, replication status and critical localities (i.e. localities that, were they to become unavailable, would cause some ranges to lose quorum): CREATE TABLE replication_constraint_stats ( zone_id INT8 NOT NULL, subzone_id INT8 NOT NULL, type STRING NOT NULL, config STRING NOT NULL, report_id INT8 NOT NULL, violation_start TIMESTAMP NULL, violating_ranges INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (zone_id ASC, subzone_id ASC, type ASC, config ASC), FAMILY "primary" (zone_id, subzone_id, type, config, report_id, violation_start, violating_ranges) ); CREATE TABLE replication_stats ( zone_id INT8 NOT NULL, subzone_id INT8 NOT NULL, report_id INT8 NOT NULL, total_ranges INT8 NOT NULL, unavailable_ranges INT8 NOT NULL, under_replicated_ranges INT8 NOT NULL, over_replicated_ranges INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (zone_id ASC, subzone_id ASC), FAMILY "primary" (zone_id, subzone_id, report_id, total_ranges, unavailable_ranges, under_replicated_ranges, over_replicated_ranges) ); CREATE TABLE replication_critical_localities ( zone_id INT8 NOT NULL, subzone_id INT8 NOT NULL, locality STRING NOT NULL, report_id INT8 NOT NULL, at_risk_ranges INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (zone_id ASC, subzone_id ASC, locality ASC), FAMILY "primary" (zone_id, subzone_id, locality, report_id, at_risk_ranges) ) And also a system.report_meta table with metadata for all these reports (their creation time). The reports are generated periodically (once a minute by default, subject to the cluster setting kv.replication_reports.interval) by a job running on the leaseholder of range 1. The data is produced by joing range descriptor data from meta2 with zone config information from the gossiped SystemConfig. Release note (sql change): The following system tables containing report about replication status, constraint conformance and critical localities are introduced: replication_constraint_stats, replication_stats, replication_critical_localities. --- docs/generated/settings/settings.html | 1 + pkg/base/test_server_args.go | 3 +- pkg/ccl/logictestccl/testdata/logic_test/zone | 40 +- pkg/cli/cli_test.go | 8 + pkg/cmd/roachtest/gossip.go | 2 + pkg/config/system.go | 12 +- pkg/config/zone.go | 37 +- pkg/config/zone_test.go | 3 + pkg/internal/client/db.go | 7 + pkg/internal/client/txn.go | 2 + pkg/keys/constants.go | 32 +- pkg/server/node.go | 31 +- pkg/sql/crdb_internal.go | 7 +- .../testdata/logic_test/crdb_internal | 7 +- .../logictest/testdata/logic_test/event_log | 2 +- .../logictest/testdata/logic_test/grant_table | 562 ++++---- .../testdata/logic_test/information_schema | 361 +++-- .../testdata/logic_test/json_builtins | 1 - .../logictest/testdata/logic_test/pg_catalog | 56 +- pkg/sql/logictest/testdata/logic_test/ranges | 164 +-- .../logictest/testdata/logic_test/show_source | 60 +- pkg/sql/logictest/testdata/logic_test/system | 376 ++--- .../opt/exec/execbuilder/testdata/show_trace | 12 +- pkg/sql/opt/exec/execbuilder/testdata/upsert | 6 +- pkg/sql/pgwire/pgwire_test.go | 2 +- pkg/sql/set_zone_config.go | 10 +- pkg/sql/show_zone_config.go | 18 +- pkg/sql/sqlbase/system.go | 217 ++- pkg/sql/tests/system_table_test.go | 2 +- pkg/sql/zone_config.go | 2 +- pkg/sql/zone_config_test.go | 2 +- pkg/sqlmigrations/migrations.go | 56 +- pkg/storage/allocator.go | 9 +- pkg/storage/allocator_scorer.go | 121 +- pkg/storage/allocator_scorer_test.go | 8 +- pkg/storage/allocator_test.go | 7 +- pkg/storage/constraint/analyzer.go | 90 ++ pkg/storage/reports/constraint_report.go | 447 ++++++ pkg/storage/reports/constraint_report_test.go | 167 +++ pkg/storage/reports/locality_report.go | 375 +++++ pkg/storage/reports/locality_report_test.go | 154 +++ pkg/storage/reports/main_test.go | 30 + pkg/storage/reports/range_report.go | 347 +++++ pkg/storage/reports/range_report_test.go | 134 ++ pkg/storage/reports/reporter.go | 644 +++++++++ pkg/storage/reports/reporter_test.go | 1215 +++++++++++++++++ pkg/storage/reports/zone_key.go | 64 + pkg/storage/store_pool.go | 15 + 48 files changed, 5076 insertions(+), 852 deletions(-) create mode 100644 pkg/storage/constraint/analyzer.go create mode 100644 pkg/storage/reports/constraint_report.go create mode 100644 pkg/storage/reports/constraint_report_test.go create mode 100644 pkg/storage/reports/locality_report.go create mode 100644 pkg/storage/reports/locality_report_test.go create mode 100644 pkg/storage/reports/main_test.go create mode 100644 pkg/storage/reports/range_report.go create mode 100644 pkg/storage/reports/range_report_test.go create mode 100644 pkg/storage/reports/reporter.go create mode 100644 pkg/storage/reports/reporter_test.go create mode 100644 pkg/storage/reports/zone_key.go diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 4eff757180ce..fbfb46f2a23b 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -54,6 +54,7 @@ kv.range_split.load_qps_thresholdinteger2500the QPS over which, the range becomes a candidate for load based splitting kv.rangefeed.concurrent_catchup_iteratorsinteger64number of rangefeeds catchup iterators a store will allow concurrently before queueing kv.rangefeed.enabledbooleanfalseif set, rangefeed registration is enabled +kv.replication_reports.intervalduration1m0sthe frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable) kv.snapshot_rebalance.max_ratebyte size8.0 MiBthe rate limit (bytes/sec) to use for rebalance and upreplication snapshots kv.snapshot_recovery.max_ratebyte size8.0 MiBthe rate limit (bytes/sec) to use for recovery snapshots kv.snapshot_sst.sync_sizebyte size2.0 MiBthreshold after which snapshot SST writes must fsync diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 543161aa4e77..49c8cc22e928 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -174,6 +174,7 @@ const ( // ReplicationManual means that the split and replication queues of all // servers are stopped, and the test must manually control splitting and // replication through the TestServer. - // Note that the server starts with a number of system ranges. + // Note that the server starts with a number of system ranges, + // all with a single replica on node 1. ReplicationManual ) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/zone b/pkg/ccl/logictestccl/testdata/logic_test/zone index 2c8a76ae5e74..398c12770ef2 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/zone +++ b/pkg/ccl/logictestccl/testdata/logic_test/zone @@ -518,25 +518,27 @@ query TTTTTT SELECT target, range_name, database_name, table_name, index_name, partition_name FROM crdb_internal.zones ---- -RANGE default default NULL NULL NULL NULL -DATABASE system NULL system NULL NULL NULL -TABLE system.public.jobs NULL system jobs NULL NULL -RANGE meta meta NULL NULL NULL NULL -RANGE system system NULL NULL NULL NULL -RANGE liveness liveness NULL NULL NULL NULL -TABLE test.public.t NULL test t NULL NULL -INDEX test.public.t@secondary NULL test t secondary NULL -INDEX test.public.t@tertiary NULL test t tertiary NULL -INDEX test.public.t36642@secondary NULL test t36642 secondary NULL -INDEX test.public.t36642@tertiary NULL test t36642 tertiary NULL -INDEX test.public.t36644@secondary NULL test t36644 secondary NULL -INDEX test.public.t36644@tertiary NULL test t36644 tertiary NULL -PARTITION x1_idx OF INDEX test.public.t38391@foo NULL test t38391 foo x1_idx -DATABASE "my database" NULL my database NULL NULL NULL -TABLE "my database".public."my table" NULL my database my table NULL NULL -INDEX "my database".public."my table"@"my index" NULL my database my table my index NULL -PARTITION "my partition" OF INDEX "my database".public."my table"@primary NULL my database my table primary my partition -PARTITION "my partition" OF INDEX "my database".public."my table"@"my index" NULL my database my table my index my partition +RANGE default default NULL NULL NULL NULL +DATABASE system NULL system NULL NULL NULL +TABLE system.public.jobs NULL system jobs NULL NULL +RANGE meta meta NULL NULL NULL NULL +RANGE system system NULL NULL NULL NULL +RANGE liveness liveness NULL NULL NULL NULL +TABLE system.public.replication_constraint_stats NULL system replication_constraint_stats NULL NULL +TABLE system.public.replication_stats NULL system replication_stats NULL NULL +TABLE test.public.t NULL test t NULL NULL +INDEX test.public.t@secondary NULL test t secondary NULL +INDEX test.public.t@tertiary NULL test t tertiary NULL +INDEX test.public.t36642@secondary NULL test t36642 secondary NULL +INDEX test.public.t36642@tertiary NULL test t36642 tertiary NULL +INDEX test.public.t36644@secondary NULL test t36644 secondary NULL +INDEX test.public.t36644@tertiary NULL test t36644 tertiary NULL +PARTITION x1_idx OF INDEX test.public.t38391@foo NULL test t38391 foo x1_idx +DATABASE "my database" NULL my database NULL NULL NULL +TABLE "my database".public."my table" NULL my database my table NULL NULL +INDEX "my database".public."my table"@"my index" NULL my database my table my index NULL +PARTITION "my partition" OF INDEX "my database".public."my table"@primary NULL my database my table primary my partition +PARTITION "my partition" OF INDEX "my database".public."my table"@"my index" NULL my database my table my index my partition # Test the zone information being displayed in SHOW CREATE statement ok diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index 9a159437d032..2ae33d25958c 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -2176,6 +2176,10 @@ writing ` + os.DevNull + ` debug/nodes/1/ranges/18.json debug/nodes/1/ranges/19.json debug/nodes/1/ranges/20.json + debug/nodes/1/ranges/21.json + debug/nodes/1/ranges/22.json + debug/nodes/1/ranges/23.json + debug/nodes/1/ranges/24.json debug/schema/defaultdb@details.json debug/schema/postgres@details.json debug/schema/system@details.json @@ -2187,6 +2191,10 @@ writing ` + os.DevNull + ` debug/schema/system/locations.json debug/schema/system/namespace.json debug/schema/system/rangelog.json + debug/schema/system/replication_constraint_stats.json + debug/schema/system/replication_critical_localities.json + debug/schema/system/replication_stats.json + debug/schema/system/reports_meta.json debug/schema/system/role_members.json debug/schema/system/settings.json debug/schema/system/table_statistics.json diff --git a/pkg/cmd/roachtest/gossip.go b/pkg/cmd/roachtest/gossip.go index ac21e7c3e168..1a2445327da8 100644 --- a/pkg/cmd/roachtest/gossip.go +++ b/pkg/cmd/roachtest/gossip.go @@ -351,6 +351,8 @@ func runGossipRestartNodeOne(ctx context.Context, t *test, c *cluster) { run(`ALTER RANGE meta %[1]s CONFIGURE ZONE %[2]s 'constraints: {"-rack=0"}'`) run(`ALTER RANGE liveness %[1]s CONFIGURE ZONE %[2]s 'constraints: {"-rack=0"}'`) run(`ALTER TABLE system.jobs %[1]s CONFIGURE ZONE %[2]s 'constraints: {"-rack=0"}'`) + run(`ALTER TABLE system.replication_stats %[1]s CONFIGURE ZONE %[2]s 'constraints: {"-rack=0"}'`) + run(`ALTER TABLE system.replication_constraint_stats %[1]s CONFIGURE ZONE %[2]s 'constraints: {"-rack=0"}'`) var lastReplCount int if err := retry.ForDuration(2*time.Minute, func() error { diff --git a/pkg/config/system.go b/pkg/config/system.go index 301be2ff0dbb..cf6b51dd1824 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -251,6 +251,11 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { // or database, specified by key.id). It is the caller's // responsibility to ensure that the range does not need to be split. func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*ZoneConfig, error) { + return s.getZoneConfigForKey(DecodeKeyIntoZoneIDAndSuffix(key)) +} + +// DecodeKeyIntoZoneIDAndSuffix figures out the zone that the key belongs to. +func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id uint32, keySuffix []byte) { objectID, keySuffix, ok := DecodeObjectID(key) if !ok { // Not in the structured data namespace. @@ -278,8 +283,7 @@ func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*ZoneConfig, error objectID = keys.SystemRangesID } } - - return s.getZoneConfigForKey(objectID, keySuffix) + return objectID, keySuffix } // isPseudoTableID returns true if id is in keys.PseudoTableIDs. @@ -350,14 +354,14 @@ func (s *SystemConfig) getZoneConfigForKey(id uint32, keySuffix []byte) (*ZoneCo } if entry.zone != nil { if entry.placeholder != nil { - if subzone := entry.placeholder.GetSubzoneForKeySuffix(keySuffix); subzone != nil { + if subzone, _ := entry.placeholder.GetSubzoneForKeySuffix(keySuffix); subzone != nil { if indexSubzone := entry.placeholder.GetSubzone(subzone.IndexID, ""); indexSubzone != nil { subzone.Config.InheritFromParent(&indexSubzone.Config) } subzone.Config.InheritFromParent(entry.zone) return &subzone.Config, nil } - } else if subzone := entry.zone.GetSubzoneForKeySuffix(keySuffix); subzone != nil { + } else if subzone, _ := entry.zone.GetSubzoneForKeySuffix(keySuffix); subzone != nil { if indexSubzone := entry.zone.GetSubzone(subzone.IndexID, ""); indexSubzone != nil { subzone.Config.InheritFromParent(&indexSubzone.Config) } diff --git a/pkg/config/zone.go b/pkg/config/zone.go index 2a6bc505a334..03bc9a3205a7 100644 --- a/pkg/config/zone.go +++ b/pkg/config/zone.go @@ -438,9 +438,11 @@ func (z *ZoneConfig) CopyFromZone(other ZoneConfig, fieldList []tree.Name) { } } -// StoreMatchesConstraint returns whether a store matches the given constraint. -func StoreMatchesConstraint(store roachpb.StoreDescriptor, constraint Constraint) bool { - hasConstraint := storeHasConstraint(store, constraint) +// StoreSatisfiesConstraint checks whether a store satisfies the given constraint. +// If the constraint is of the PROHIBITED type, satisfying it means the store +// not matching the constraint's spec. +func StoreSatisfiesConstraint(store roachpb.StoreDescriptor, constraint Constraint) bool { + hasConstraint := StoreMatchesConstraint(store, constraint) if (constraint.Type == Constraint_REQUIRED && !hasConstraint) || (constraint.Type == Constraint_PROHIBITED && hasConstraint) { return false @@ -448,10 +450,11 @@ func StoreMatchesConstraint(store roachpb.StoreDescriptor, constraint Constraint return true } -// storeHasConstraint returns whether a store's attributes or node's locality -// matches the key value pair in the constraint. It notably ignores whether -// the constraint is required, prohibited, positive, or otherwise. -func storeHasConstraint(store roachpb.StoreDescriptor, c Constraint) bool { +// StoreMatchesConstraint returns whether a store's attributes or node's +// locality match the constraint's spec. It notably ignores whether the +// constraint is required, prohibited, positive, or otherwise. +// Also see StoreSatisfiesConstraint(). +func StoreMatchesConstraint(store roachpb.StoreDescriptor, c Constraint) bool { if c.Key == "" { for _, attrs := range []roachpb.Attributes{store.Attrs, store.Node.Attrs} { for _, attr := range attrs.Attrs { @@ -460,11 +463,11 @@ func storeHasConstraint(store roachpb.StoreDescriptor, c Constraint) bool { } } } - } else { - for _, tier := range store.Node.Locality.Tiers { - if c.Key == tier.Key && c.Value == tier.Value { - return true - } + return false + } + for _, tier := range store.Node.Locality.Tiers { + if c.Key == tier.Key && c.Value == tier.Value { + return true } } return false @@ -472,7 +475,7 @@ func storeHasConstraint(store roachpb.StoreDescriptor, c Constraint) bool { // DeleteTableConfig removes any configuration that applies to the table // targeted by this ZoneConfig, leaving only its subzone configs, if any. After -// calling DeleteTableConfig, IsZubzonePlaceholder will return true. +// calling DeleteTableConfig, IsSubzonePlaceholder will return true. // // Only table zones can have subzones, so it does not make sense to call this // method on non-table ZoneConfigs. @@ -514,8 +517,8 @@ func (z *ZoneConfig) GetSubzone(indexID uint32, partition string) *Subzone { } // GetSubzoneForKeySuffix returns the ZoneConfig for the subzone that contains -// keySuffix, if it exists. -func (z ZoneConfig) GetSubzoneForKeySuffix(keySuffix []byte) *Subzone { +// keySuffix, if it exists and its position in the subzones slice. +func (z ZoneConfig) GetSubzoneForKeySuffix(keySuffix []byte) (*Subzone, int32) { // TODO(benesch): Use binary search instead. for _, s := range z.SubzoneSpans { // The span's Key is stored with the prefix removed, so we can compare @@ -523,10 +526,10 @@ func (z ZoneConfig) GetSubzoneForKeySuffix(keySuffix []byte) *Subzone { if (s.Key.Compare(keySuffix) <= 0) && ((s.EndKey == nil && bytes.HasPrefix(keySuffix, s.Key)) || s.EndKey.Compare(keySuffix) > 0) { copySubzone := z.Subzones[s.SubzoneIndex] - return ©Subzone + return ©Subzone, s.SubzoneIndex } } - return nil + return nil, -1 } // SetSubzone installs subzone into the ZoneConfig, overwriting any existing diff --git a/pkg/config/zone_test.go b/pkg/config/zone_test.go index 96b9e032edc0..6899ac50b3c4 100644 --- a/pkg/config/zone_test.go +++ b/pkg/config/zone_test.go @@ -752,6 +752,9 @@ func TestConstraintsListYAML(t *testing.T) { {input: "{+a: 1, '+a=1,+b,+c=d': b}", expectErr: true}, {input: "[+a: 1]", expectErr: true}, {input: "[+a: 1, '+a=1,+b,+c=d': 2]", expectErr: true}, + {input: "{\"+a=1,+b=2\": 1}"}, // this will work in SQL: constraints='{"+a=1,+b=2": 1}' + {input: "{\"+a=1,+b=2,+c\": 1}"}, // won't work in SQL: constraints='{"+a=1,+b=2,+c": 1}' + {input: "{'+a=1,+b=2,+c': 1}"}, // this will work in SQL: constraints=e'{\'+a=1,+b=2,+c\': 1}' } for _, tc := range testCases { diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go index 150ae325c930..304058859a4c 100644 --- a/pkg/internal/client/db.go +++ b/pkg/internal/client/db.go @@ -651,6 +651,13 @@ func (db *DB) Run(ctx context.Context, b *Batch) error { return sendAndFill(ctx, db.send, b) } +// NewTxn creates a new RootTxn. +func (db *DB) NewTxn(ctx context.Context, debugName string) *Txn { + txn := NewTxn(ctx, db, db.ctx.NodeID.Get(), RootTxn) + txn.SetDebugName(debugName) + return txn +} + // Txn executes retryable in the context of a distributed transaction. The // transaction is automatically aborted if retryable returns any error aside // from recoverable internal errors, and is automatically committed diff --git a/pkg/internal/client/txn.go b/pkg/internal/client/txn.go index d72c29c46879..7c35b3cf91b4 100644 --- a/pkg/internal/client/txn.go +++ b/pkg/internal/client/txn.go @@ -94,6 +94,8 @@ type Txn struct { // in the batch with the current node's ID. // If the gatewayNodeID is set and this is a root transaction, we optimize // away any clock uncertainty for our own node, as our clock is accessible. +// +// See also db.NewTxn(). func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID, typ TxnType) *Txn { now := db.clock.Now() txn := roachpb.MakeTransaction( diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 86001467db96..728c64849926 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -312,20 +312,24 @@ const ( // to "Ranges" instead of a Table - these IDs are needed to store custom // configuration for non-table ranges (e.g. Zone Configs). // NOTE: IDs must be <= MaxReservedDescID. - LeaseTableID = 11 - EventLogTableID = 12 - RangeEventTableID = 13 - UITableID = 14 - JobsTableID = 15 - MetaRangesID = 16 - SystemRangesID = 17 - TimeseriesRangesID = 18 - WebSessionsTableID = 19 - TableStatisticsTableID = 20 - LocationsTableID = 21 - LivenessRangesID = 22 - RoleMembersTableID = 23 - CommentsTableID = 24 + LeaseTableID = 11 + EventLogTableID = 12 + RangeEventTableID = 13 + UITableID = 14 + JobsTableID = 15 + MetaRangesID = 16 + SystemRangesID = 17 + TimeseriesRangesID = 18 + WebSessionsTableID = 19 + TableStatisticsTableID = 20 + LocationsTableID = 21 + LivenessRangesID = 22 + RoleMembersTableID = 23 + CommentsTableID = 24 + ReplicationConstraintStatsTableID = 25 + ReplicationCriticalLocalitiesTableID = 26 + ReplicationStatsTableID = 27 + ReportsMetaTableID = 28 // CommentType is type for system.comments DatabaseCommentType = 0 diff --git a/pkg/server/node.go b/pkg/server/node.go index 07f232ae1198..a4c2e699d056 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/storage/reports" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/growstack" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" @@ -44,7 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/logtags" - opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go" "github.com/pkg/errors" ) @@ -142,18 +143,19 @@ func (nm nodeMetrics) callComplete(d time.Duration, pErr *roachpb.Error) { // IDs for bootstrapping the node itself or new stores as they're added // on subsequent instantiations. type Node struct { - stopper *stop.Stopper - clusterID *base.ClusterIDContainer // UUID for Cockroach cluster - Descriptor roachpb.NodeDescriptor // Node ID, network/physical topology - storeCfg storage.StoreConfig // Config to use and pass to stores - eventLogger sql.EventLogger - stores *storage.Stores // Access to node-local stores - metrics nodeMetrics - recorder *status.MetricsRecorder - startedAt int64 - lastUp int64 - initialBoot bool // True if this is the first time this node has started. - txnMetrics kv.TxnMetrics + stopper *stop.Stopper + clusterID *base.ClusterIDContainer // UUID for Cockroach cluster + Descriptor roachpb.NodeDescriptor // Node ID, network/physical topology + storeCfg storage.StoreConfig // Config to use and pass to stores + eventLogger sql.EventLogger + stores *storage.Stores // Access to node-local stores + metrics nodeMetrics + recorder *status.MetricsRecorder + constraintStatsCollector *reports.Reporter + startedAt int64 + lastUp int64 + initialBoot bool // True if this is the first time this node has started. + txnMetrics kv.TxnMetrics perReplicaServer storage.Server } @@ -271,6 +273,7 @@ func NewNode( eventLogger: eventLogger, clusterID: clusterID, } + n.constraintStatsCollector = reports.NewReporter(n.stores, &n.storeCfg) n.perReplicaServer = storage.MakeServer(&n.Descriptor, n.stores) return n } @@ -498,6 +501,8 @@ func (n *Node) start( // bumped immediately, which would be possible if gossip got started earlier). n.startGossip(ctx, n.stopper) + n.constraintStatsCollector.Start(ctx, n.stopper) + allEngines := append([]engine.Engine(nil), initializedEngines...) allEngines = append(allEngines, emptyEngines...) log.Infof(ctx, "%s: started with %v engine(s) and attributes %v", n, allEngines, attrs.Attrs) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3306f0419163..fbedf0cbc5fa 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2092,6 +2092,7 @@ var crdbInternalZonesTable = virtualSchemaTable{ schema: ` CREATE TABLE crdb_internal.zones ( zone_id INT NOT NULL, + subzone_id INT NOT NULL, target STRING, range_name STRING, database_name STRING, @@ -2150,7 +2151,7 @@ CREATE TABLE crdb_internal.zones ( configProto.Subzones = nil configProto.SubzoneSpans = nil - if err := generateZoneConfigIntrospectionValues(values, r[0], zoneSpecifier, &configProto); err != nil { + if err := generateZoneConfigIntrospectionValues(values, r[0], tree.NewDInt(tree.DInt(0)), zoneSpecifier, &configProto); err != nil { return err } if err := addRow(values...); err != nil { @@ -2163,7 +2164,7 @@ CREATE TABLE crdb_internal.zones ( if err != nil { return err } - for _, s := range subzones { + for i, s := range subzones { index, err := table.FindIndexByID(sqlbase.IndexID(s.IndexID)) if err != nil { if err == sqlbase.ErrIndexGCMutationsList { @@ -2178,7 +2179,7 @@ CREATE TABLE crdb_internal.zones ( zoneSpecifier = &zs } - if err := generateZoneConfigIntrospectionValues(values, r[0], zoneSpecifier, &s.Config); err != nil { + if err := generateZoneConfigIntrospectionValues(values, r[0], tree.NewDInt(tree.DInt(i+1)), zoneSpecifier, &s.Config); err != nil { return err } if err := addRow(values...); err != nil { diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 10bab0ac4bd5..a2d80b452460 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -208,10 +208,11 @@ SELECT * FROM crdb_internal.forward_dependencies WHERE descriptor_name = '' ---- descriptor_id descriptor_name index_id dependedonby_id dependedonby_type dependedonby_index_id dependedonby_name dependedonby_details -query ITTTTTTTTT colnames +query IITTTTTTTTT colnames SELECT * FROM crdb_internal.zones WHERE false ---- -zone_id target range_name database_name table_name index_name partition_name config_yaml config_sql config_protobuf +zone_id subzone_id target range_name database_name table_name index_name partition_name +config_yaml config_sql config_protobuf query ITTTTTTTTTTTTI colnames SELECT * FROM crdb_internal.ranges WHERE range_id < 0 @@ -239,6 +240,8 @@ SELECT zone_id, target FROM crdb_internal.zones ORDER BY 1 17 RANGE system 18 RANGE timeseries 22 RANGE liveness +25 TABLE system.public.replication_constraint_stats +27 TABLE system.public.replication_stats 53 DATABASE testdb 54 TABLE testdb.public.foo diff --git a/pkg/sql/logictest/testdata/logic_test/event_log b/pkg/sql/logictest/testdata/logic_test/event_log index e6bdbb56af73..73d98cf4ac5f 100644 --- a/pkg/sql/logictest/testdata/logic_test/event_log +++ b/pkg/sql/logictest/testdata/logic_test/event_log @@ -390,7 +390,7 @@ ALTER TABLE a CONFIGURE ZONE DISCARD query IT SELECT "reportingID", "info" FROM system.eventlog -WHERE "eventType" = 'set_zone_config' +WHERE "eventType" = 'set_zone_config' and info not like '%system.public%' ORDER BY "timestamp" ---- 1 {"Target":"TABLE test.public.a","Options":"range_max_bytes = 67108865, range_min_bytes = 16777216","User":"root"} diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index d8e3fe6cff71..448791c32ff0 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -159,261 +159,321 @@ query TTTTT colnames,rowsort SELECT * FROM [SHOW GRANTS] WHERE schema_name NOT IN ('crdb_internal', 'pg_catalog', 'information_schema') ---- -database_name schema_name table_name grantee privilege_type -a public NULL admin ALL -a public NULL readwrite ALL -a public NULL root ALL -defaultdb public NULL admin ALL -defaultdb public NULL root ALL -postgres public NULL admin ALL -postgres public NULL root ALL -system public NULL admin GRANT -system public NULL admin SELECT -system public NULL root GRANT -system public NULL root SELECT -system public comments admin DELETE -system public comments admin GRANT -system public comments admin INSERT -system public comments admin SELECT -system public comments admin UPDATE -system public comments public DELETE -system public comments public GRANT -system public comments public INSERT -system public comments public SELECT -system public comments public UPDATE -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public descriptor admin GRANT -system public descriptor admin SELECT -system public descriptor root GRANT -system public descriptor root SELECT -system public eventlog admin DELETE -system public eventlog admin GRANT -system public eventlog admin INSERT -system public eventlog admin SELECT -system public eventlog admin UPDATE -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public jobs admin DELETE -system public jobs admin GRANT -system public jobs admin INSERT -system public jobs admin SELECT -system public jobs admin UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public lease admin DELETE -system public lease admin GRANT -system public lease admin INSERT -system public lease admin SELECT -system public lease admin UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public locations admin DELETE -system public locations admin GRANT -system public locations admin INSERT -system public locations admin SELECT -system public locations admin UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public namespace admin GRANT -system public namespace admin SELECT -system public namespace root GRANT -system public namespace root SELECT -system public rangelog admin DELETE -system public rangelog admin GRANT -system public rangelog admin INSERT -system public rangelog admin SELECT -system public rangelog admin UPDATE -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public role_members admin DELETE -system public role_members admin GRANT -system public role_members admin INSERT -system public role_members admin SELECT -system public role_members admin UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public settings admin DELETE -system public settings admin GRANT -system public settings admin INSERT -system public settings admin SELECT -system public settings admin UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public table_statistics admin DELETE -system public table_statistics admin GRANT -system public table_statistics admin INSERT -system public table_statistics admin SELECT -system public table_statistics admin UPDATE -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public ui admin DELETE -system public ui admin GRANT -system public ui admin INSERT -system public ui admin SELECT -system public ui admin UPDATE -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public users admin DELETE -system public users admin GRANT -system public users admin INSERT -system public users admin SELECT -system public users admin UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public web_sessions admin DELETE -system public web_sessions admin GRANT -system public web_sessions admin INSERT -system public web_sessions admin SELECT -system public web_sessions admin UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public zones admin DELETE -system public zones admin GRANT -system public zones admin INSERT -system public zones admin SELECT -system public zones admin UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE -test public NULL admin ALL -test public NULL root ALL +database_name schema_name table_name grantee privilege_type +system public NULL admin GRANT +system public NULL admin SELECT +system public NULL root GRANT +system public NULL root SELECT +defaultdb public NULL admin ALL +defaultdb public NULL root ALL +postgres public NULL admin ALL +postgres public NULL root ALL +test public NULL admin ALL +test public NULL root ALL +a public NULL admin ALL +a public NULL readwrite ALL +a public NULL root ALL +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT +system public descriptor admin GRANT +system public descriptor admin SELECT +system public descriptor root GRANT +system public descriptor root SELECT +system public users admin DELETE +system public users admin GRANT +system public users admin INSERT +system public users admin SELECT +system public users admin UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public zones admin DELETE +system public zones admin GRANT +system public zones admin INSERT +system public zones admin SELECT +system public zones admin UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE +system public settings admin DELETE +system public settings admin GRANT +system public settings admin INSERT +system public settings admin SELECT +system public settings admin UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public lease admin DELETE +system public lease admin GRANT +system public lease admin INSERT +system public lease admin SELECT +system public lease admin UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public eventlog admin DELETE +system public eventlog admin GRANT +system public eventlog admin INSERT +system public eventlog admin SELECT +system public eventlog admin UPDATE +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public rangelog admin DELETE +system public rangelog admin GRANT +system public rangelog admin INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public ui admin DELETE +system public ui admin GRANT +system public ui admin INSERT +system public ui admin SELECT +system public ui admin UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public jobs admin DELETE +system public jobs admin GRANT +system public jobs admin INSERT +system public jobs admin SELECT +system public jobs admin UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public web_sessions admin DELETE +system public web_sessions admin GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public table_statistics admin DELETE +system public table_statistics admin GRANT +system public table_statistics admin INSERT +system public table_statistics admin SELECT +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public locations admin DELETE +system public locations admin GRANT +system public locations admin INSERT +system public locations admin SELECT +system public locations admin UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public role_members admin DELETE +system public role_members admin GRANT +system public role_members admin INSERT +system public role_members admin SELECT +system public role_members admin UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public comments admin DELETE +system public comments admin GRANT +system public comments admin INSERT +system public comments admin SELECT +system public comments admin UPDATE +system public comments public DELETE +system public comments public GRANT +system public comments public INSERT +system public comments public SELECT +system public comments public UPDATE +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public replication_constraint_stats admin DELETE +system public replication_constraint_stats admin GRANT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities admin DELETE +system public replication_critical_localities admin GRANT +system public replication_critical_localities admin INSERT +system public replication_critical_localities admin SELECT +system public replication_critical_localities admin UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats admin DELETE +system public replication_stats admin GRANT +system public replication_stats admin INSERT +system public replication_stats admin SELECT +system public replication_stats admin UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta admin DELETE +system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta admin SELECT +system public reports_meta admin UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE query TTTTT colnames SHOW GRANTS FOR root ---- -database_name schema_name table_name grantee privilege_type -a crdb_internal NULL root ALL -a information_schema NULL root ALL -a pg_catalog NULL root ALL -a public NULL root ALL -defaultdb crdb_internal NULL root ALL -defaultdb information_schema NULL root ALL -defaultdb pg_catalog NULL root ALL -defaultdb public NULL root ALL -postgres crdb_internal NULL root ALL -postgres information_schema NULL root ALL -postgres pg_catalog NULL root ALL -postgres public NULL root ALL -system crdb_internal NULL root GRANT -system crdb_internal NULL root SELECT -system information_schema NULL root GRANT -system information_schema NULL root SELECT -system pg_catalog NULL root GRANT -system pg_catalog NULL root SELECT -system public NULL root GRANT -system public NULL root SELECT -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public descriptor root GRANT -system public descriptor root SELECT -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public namespace root GRANT -system public namespace root SELECT -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE -test crdb_internal NULL root ALL -test information_schema NULL root ALL -test pg_catalog NULL root ALL -test public NULL root ALL +database_name schema_name table_name grantee privilege_type +a crdb_internal NULL root ALL +a information_schema NULL root ALL +a pg_catalog NULL root ALL +a public NULL root ALL +defaultdb crdb_internal NULL root ALL +defaultdb information_schema NULL root ALL +defaultdb pg_catalog NULL root ALL +defaultdb public NULL root ALL +postgres crdb_internal NULL root ALL +postgres information_schema NULL root ALL +postgres pg_catalog NULL root ALL +postgres public NULL root ALL +system crdb_internal NULL root GRANT +system crdb_internal NULL root SELECT +system information_schema NULL root GRANT +system information_schema NULL root SELECT +system pg_catalog NULL root GRANT +system pg_catalog NULL root SELECT +system public NULL root GRANT +system public NULL root SELECT +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public descriptor root GRANT +system public descriptor root SELECT +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public namespace root GRANT +system public namespace root SELECT +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE +test crdb_internal NULL root ALL +test information_schema NULL root ALL +test pg_catalog NULL root ALL +test public NULL root ALL statement error pgcode 42P01 relation "a.t" does not exist SHOW GRANTS ON a.t diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 3fd4c418858d..32618905e7b8 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -611,6 +611,10 @@ system public table_statistics BASE TABLE system public locations BASE TABLE YES 1 system public role_members BASE TABLE YES 1 system public comments BASE TABLE YES 1 +system public replication_constraint_stats BASE TABLE YES 1 +system public replication_critical_localities BASE TABLE YES 1 +system public replication_stats BASE TABLE YES 1 +system public reports_meta BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -664,22 +668,26 @@ SELECT * FROM system.information_schema.table_constraints ORDER BY TABLE_NAME, CONSTRAINT_TYPE, CONSTRAINT_NAME ---- -constraint_catalog constraint_schema constraint_name table_catalog table_schema table_name constraint_type is_deferrable initially_deferred -system public primary system public comments PRIMARY KEY NO NO -system public primary system public descriptor PRIMARY KEY NO NO -system public primary system public eventlog PRIMARY KEY NO NO -system public primary system public jobs PRIMARY KEY NO NO -system public primary system public lease PRIMARY KEY NO NO -system public primary system public locations PRIMARY KEY NO NO -system public primary system public namespace PRIMARY KEY NO NO -system public primary system public rangelog PRIMARY KEY NO NO -system public primary system public role_members PRIMARY KEY NO NO -system public primary system public settings PRIMARY KEY NO NO -system public primary system public table_statistics PRIMARY KEY NO NO -system public primary system public ui PRIMARY KEY NO NO -system public primary system public users PRIMARY KEY NO NO -system public primary system public web_sessions PRIMARY KEY NO NO -system public primary system public zones PRIMARY KEY NO NO +constraint_catalog constraint_schema constraint_name table_catalog table_schema table_name constraint_type is_deferrable initially_deferred +system public primary system public comments PRIMARY KEY NO NO +system public primary system public descriptor PRIMARY KEY NO NO +system public primary system public eventlog PRIMARY KEY NO NO +system public primary system public jobs PRIMARY KEY NO NO +system public primary system public lease PRIMARY KEY NO NO +system public primary system public locations PRIMARY KEY NO NO +system public primary system public namespace PRIMARY KEY NO NO +system public primary system public rangelog PRIMARY KEY NO NO +system public primary system public replication_constraint_stats PRIMARY KEY NO NO +system public primary system public replication_critical_localities PRIMARY KEY NO NO +system public primary system public replication_stats PRIMARY KEY NO NO +system public primary system public reports_meta PRIMARY KEY NO NO +system public primary system public role_members PRIMARY KEY NO NO +system public primary system public settings PRIMARY KEY NO NO +system public primary system public table_statistics PRIMARY KEY NO NO +system public primary system public ui PRIMARY KEY NO NO +system public primary system public users PRIMARY KEY NO NO +system public primary system public web_sessions PRIMARY KEY NO NO +system public primary system public zones PRIMARY KEY NO NO query TTTT colnames SELECT * @@ -731,6 +739,26 @@ system public 630200280_24_1_not_null type IS NOT NULL system public 630200280_24_2_not_null object_id IS NOT NULL system public 630200280_24_3_not_null sub_id IS NOT NULL system public 630200280_24_4_not_null comment IS NOT NULL +system public 630200280_25_1_not_null zone_id IS NOT NULL +system public 630200280_25_2_not_null subzone_id IS NOT NULL +system public 630200280_25_3_not_null type IS NOT NULL +system public 630200280_25_4_not_null config IS NOT NULL +system public 630200280_25_5_not_null report_id IS NOT NULL +system public 630200280_25_7_not_null violating_ranges IS NOT NULL +system public 630200280_26_1_not_null zone_id IS NOT NULL +system public 630200280_26_2_not_null subzone_id IS NOT NULL +system public 630200280_26_3_not_null locality IS NOT NULL +system public 630200280_26_4_not_null report_id IS NOT NULL +system public 630200280_26_5_not_null at_risk_ranges IS NOT NULL +system public 630200280_27_1_not_null zone_id IS NOT NULL +system public 630200280_27_2_not_null subzone_id IS NOT NULL +system public 630200280_27_3_not_null report_id IS NOT NULL +system public 630200280_27_4_not_null total_ranges IS NOT NULL +system public 630200280_27_5_not_null unavailable_ranges IS NOT NULL +system public 630200280_27_6_not_null under_replicated_ranges IS NOT NULL +system public 630200280_27_7_not_null over_replicated_ranges IS NOT NULL +system public 630200280_28_1_not_null id IS NOT NULL +system public 630200280_28_2_not_null generated IS NOT NULL system public 630200280_2_1_not_null parentID IS NOT NULL system public 630200280_2_2_not_null name IS NOT NULL system public 630200280_3_1_not_null id IS NOT NULL @@ -746,33 +774,43 @@ SELECT * FROM system.information_schema.constraint_column_usage ORDER BY TABLE_NAME, COLUMN_NAME, CONSTRAINT_NAME ---- -table_catalog table_schema table_name column_name constraint_catalog constraint_schema constraint_name -system public comments object_id system public primary -system public comments sub_id system public primary -system public comments type system public primary -system public descriptor id system public primary -system public eventlog timestamp system public primary -system public eventlog uniqueID system public primary -system public jobs id system public primary -system public lease descID system public primary -system public lease expiration system public primary -system public lease nodeID system public primary -system public lease version system public primary -system public locations localityKey system public primary -system public locations localityValue system public primary -system public namespace name system public primary -system public namespace parentID system public primary -system public rangelog timestamp system public primary -system public rangelog uniqueID system public primary -system public role_members member system public primary -system public role_members role system public primary -system public settings name system public primary -system public table_statistics statisticID system public primary -system public table_statistics tableID system public primary -system public ui key system public primary -system public users username system public primary -system public web_sessions id system public primary -system public zones id system public primary +table_catalog table_schema table_name column_name constraint_catalog constraint_schema constraint_name +system public comments object_id system public primary +system public comments sub_id system public primary +system public comments type system public primary +system public descriptor id system public primary +system public eventlog timestamp system public primary +system public eventlog uniqueID system public primary +system public jobs id system public primary +system public lease descID system public primary +system public lease expiration system public primary +system public lease nodeID system public primary +system public lease version system public primary +system public locations localityKey system public primary +system public locations localityValue system public primary +system public namespace name system public primary +system public namespace parentID system public primary +system public rangelog timestamp system public primary +system public rangelog uniqueID system public primary +system public replication_constraint_stats config system public primary +system public replication_constraint_stats subzone_id system public primary +system public replication_constraint_stats type system public primary +system public replication_constraint_stats zone_id system public primary +system public replication_critical_localities locality system public primary +system public replication_critical_localities subzone_id system public primary +system public replication_critical_localities zone_id system public primary +system public replication_stats subzone_id system public primary +system public replication_stats zone_id system public primary +system public reports_meta id system public primary +system public role_members member system public primary +system public role_members role system public primary +system public settings name system public primary +system public table_statistics statisticID system public primary +system public table_statistics tableID system public primary +system public ui key system public primary +system public users username system public primary +system public web_sessions id system public primary +system public zones id system public primary statement ok CREATE DATABASE constraint_db @@ -853,74 +891,95 @@ FROM system.information_schema.columns WHERE table_schema != 'information_schema' AND table_schema != 'pg_catalog' AND table_schema != 'crdb_internal' ORDER BY 3,4 ---- -table_catalog table_schema table_name column_name ordinal_position -system public comments comment 4 -system public comments object_id 2 -system public comments sub_id 3 -system public comments type 1 -system public descriptor descriptor 2 -system public descriptor id 1 -system public eventlog eventType 2 -system public eventlog info 5 -system public eventlog reportingID 4 -system public eventlog targetID 3 -system public eventlog timestamp 1 -system public eventlog uniqueID 6 -system public jobs created 3 -system public jobs id 1 -system public jobs payload 4 -system public jobs progress 5 -system public jobs status 2 -system public lease descID 1 -system public lease expiration 4 -system public lease nodeID 3 -system public lease version 2 -system public locations latitude 3 -system public locations localityKey 1 -system public locations localityValue 2 -system public locations longitude 4 -system public namespace id 3 -system public namespace name 2 -system public namespace parentID 1 -system public rangelog eventType 4 -system public rangelog info 6 -system public rangelog otherRangeID 5 -system public rangelog rangeID 2 -system public rangelog storeID 3 -system public rangelog timestamp 1 -system public rangelog uniqueID 7 -system public role_members isAdmin 3 -system public role_members member 2 -system public role_members role 1 -system public settings lastUpdated 3 -system public settings name 1 -system public settings value 2 -system public settings valueType 4 -system public table_statistics columnIDs 4 -system public table_statistics createdAt 5 -system public table_statistics distinctCount 7 -system public table_statistics histogram 9 -system public table_statistics name 3 -system public table_statistics nullCount 8 -system public table_statistics rowCount 6 -system public table_statistics statisticID 2 -system public table_statistics tableID 1 -system public ui key 1 -system public ui lastUpdated 3 -system public ui value 2 -system public users hashedPassword 2 -system public users isRole 3 -system public users username 1 -system public web_sessions auditInfo 8 -system public web_sessions createdAt 4 -system public web_sessions expiresAt 5 -system public web_sessions hashedSecret 2 -system public web_sessions id 1 -system public web_sessions lastUsedAt 7 -system public web_sessions revokedAt 6 -system public web_sessions username 3 -system public zones config 2 -system public zones id 1 +table_catalog table_schema table_name column_name ordinal_position +system public comments comment 4 +system public comments object_id 2 +system public comments sub_id 3 +system public comments type 1 +system public descriptor descriptor 2 +system public descriptor id 1 +system public eventlog eventType 2 +system public eventlog info 5 +system public eventlog reportingID 4 +system public eventlog targetID 3 +system public eventlog timestamp 1 +system public eventlog uniqueID 6 +system public jobs created 3 +system public jobs id 1 +system public jobs payload 4 +system public jobs progress 5 +system public jobs status 2 +system public lease descID 1 +system public lease expiration 4 +system public lease nodeID 3 +system public lease version 2 +system public locations latitude 3 +system public locations localityKey 1 +system public locations localityValue 2 +system public locations longitude 4 +system public namespace id 3 +system public namespace name 2 +system public namespace parentID 1 +system public rangelog eventType 4 +system public rangelog info 6 +system public rangelog otherRangeID 5 +system public rangelog rangeID 2 +system public rangelog storeID 3 +system public rangelog timestamp 1 +system public rangelog uniqueID 7 +system public replication_constraint_stats config 4 +system public replication_constraint_stats report_id 5 +system public replication_constraint_stats subzone_id 2 +system public replication_constraint_stats type 3 +system public replication_constraint_stats violating_ranges 7 +system public replication_constraint_stats violation_start 6 +system public replication_constraint_stats zone_id 1 +system public replication_critical_localities at_risk_ranges 5 +system public replication_critical_localities locality 3 +system public replication_critical_localities report_id 4 +system public replication_critical_localities subzone_id 2 +system public replication_critical_localities zone_id 1 +system public replication_stats over_replicated_ranges 7 +system public replication_stats report_id 3 +system public replication_stats subzone_id 2 +system public replication_stats total_ranges 4 +system public replication_stats unavailable_ranges 5 +system public replication_stats under_replicated_ranges 6 +system public replication_stats zone_id 1 +system public reports_meta generated 2 +system public reports_meta id 1 +system public role_members isAdmin 3 +system public role_members member 2 +system public role_members role 1 +system public settings lastUpdated 3 +system public settings name 1 +system public settings value 2 +system public settings valueType 4 +system public table_statistics columnIDs 4 +system public table_statistics createdAt 5 +system public table_statistics distinctCount 7 +system public table_statistics histogram 9 +system public table_statistics name 3 +system public table_statistics nullCount 8 +system public table_statistics rowCount 6 +system public table_statistics statisticID 2 +system public table_statistics tableID 1 +system public ui key 1 +system public ui lastUpdated 3 +system public ui value 2 +system public users hashedPassword 2 +system public users isRole 3 +system public users username 1 +system public web_sessions auditInfo 8 +system public web_sessions createdAt 4 +system public web_sessions expiresAt 5 +system public web_sessions hashedSecret 2 +system public web_sessions id 1 +system public web_sessions lastUsedAt 7 +system public web_sessions revokedAt 6 +system public web_sessions username 3 +system public zones config 2 +system public zones id 1 statement ok SET DATABASE = test @@ -1410,6 +1469,46 @@ NULL root system public rangelog NULL root system public rangelog INSERT NULL NO NULL root system public rangelog SELECT NULL YES NULL root system public rangelog UPDATE NULL NO +NULL admin system public replication_constraint_stats DELETE NULL NO +NULL admin system public replication_constraint_stats GRANT NULL NO +NULL admin system public replication_constraint_stats INSERT NULL NO +NULL admin system public replication_constraint_stats SELECT NULL YES +NULL admin system public replication_constraint_stats UPDATE NULL NO +NULL root system public replication_constraint_stats DELETE NULL NO +NULL root system public replication_constraint_stats GRANT NULL NO +NULL root system public replication_constraint_stats INSERT NULL NO +NULL root system public replication_constraint_stats SELECT NULL YES +NULL root system public replication_constraint_stats UPDATE NULL NO +NULL admin system public replication_critical_localities DELETE NULL NO +NULL admin system public replication_critical_localities GRANT NULL NO +NULL admin system public replication_critical_localities INSERT NULL NO +NULL admin system public replication_critical_localities SELECT NULL YES +NULL admin system public replication_critical_localities UPDATE NULL NO +NULL root system public replication_critical_localities DELETE NULL NO +NULL root system public replication_critical_localities GRANT NULL NO +NULL root system public replication_critical_localities INSERT NULL NO +NULL root system public replication_critical_localities SELECT NULL YES +NULL root system public replication_critical_localities UPDATE NULL NO +NULL admin system public replication_stats DELETE NULL NO +NULL admin system public replication_stats GRANT NULL NO +NULL admin system public replication_stats INSERT NULL NO +NULL admin system public replication_stats SELECT NULL YES +NULL admin system public replication_stats UPDATE NULL NO +NULL root system public replication_stats DELETE NULL NO +NULL root system public replication_stats GRANT NULL NO +NULL root system public replication_stats INSERT NULL NO +NULL root system public replication_stats SELECT NULL YES +NULL root system public replication_stats UPDATE NULL NO +NULL admin system public reports_meta DELETE NULL NO +NULL admin system public reports_meta GRANT NULL NO +NULL admin system public reports_meta INSERT NULL NO +NULL admin system public reports_meta SELECT NULL YES +NULL admin system public reports_meta UPDATE NULL NO +NULL root system public reports_meta DELETE NULL NO +NULL root system public reports_meta GRANT NULL NO +NULL root system public reports_meta INSERT NULL NO +NULL root system public reports_meta SELECT NULL YES +NULL root system public reports_meta UPDATE NULL NO NULL admin system public role_members DELETE NULL NO NULL admin system public role_members GRANT NULL NO NULL admin system public role_members INSERT NULL NO @@ -1730,6 +1829,46 @@ NULL root system public comments NULL root system public comments INSERT NULL NO NULL root system public comments SELECT NULL YES NULL root system public comments UPDATE NULL NO +NULL admin system public replication_constraint_stats DELETE NULL NO +NULL admin system public replication_constraint_stats GRANT NULL NO +NULL admin system public replication_constraint_stats INSERT NULL NO +NULL admin system public replication_constraint_stats SELECT NULL YES +NULL admin system public replication_constraint_stats UPDATE NULL NO +NULL root system public replication_constraint_stats DELETE NULL NO +NULL root system public replication_constraint_stats GRANT NULL NO +NULL root system public replication_constraint_stats INSERT NULL NO +NULL root system public replication_constraint_stats SELECT NULL YES +NULL root system public replication_constraint_stats UPDATE NULL NO +NULL admin system public replication_critical_localities DELETE NULL NO +NULL admin system public replication_critical_localities GRANT NULL NO +NULL admin system public replication_critical_localities INSERT NULL NO +NULL admin system public replication_critical_localities SELECT NULL YES +NULL admin system public replication_critical_localities UPDATE NULL NO +NULL root system public replication_critical_localities DELETE NULL NO +NULL root system public replication_critical_localities GRANT NULL NO +NULL root system public replication_critical_localities INSERT NULL NO +NULL root system public replication_critical_localities SELECT NULL YES +NULL root system public replication_critical_localities UPDATE NULL NO +NULL admin system public replication_stats DELETE NULL NO +NULL admin system public replication_stats GRANT NULL NO +NULL admin system public replication_stats INSERT NULL NO +NULL admin system public replication_stats SELECT NULL YES +NULL admin system public replication_stats UPDATE NULL NO +NULL root system public replication_stats DELETE NULL NO +NULL root system public replication_stats GRANT NULL NO +NULL root system public replication_stats INSERT NULL NO +NULL root system public replication_stats SELECT NULL YES +NULL root system public replication_stats UPDATE NULL NO +NULL admin system public reports_meta DELETE NULL NO +NULL admin system public reports_meta GRANT NULL NO +NULL admin system public reports_meta INSERT NULL NO +NULL admin system public reports_meta SELECT NULL YES +NULL admin system public reports_meta UPDATE NULL NO +NULL root system public reports_meta DELETE NULL NO +NULL root system public reports_meta GRANT NULL NO +NULL root system public reports_meta INSERT NULL NO +NULL root system public reports_meta SELECT NULL YES +NULL root system public reports_meta UPDATE NULL NO statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/json_builtins b/pkg/sql/logictest/testdata/logic_test/json_builtins index efdffbdfd557..9fe45fb9930e 100644 --- a/pkg/sql/logictest/testdata/logic_test/json_builtins +++ b/pkg/sql/logictest/testdata/logic_test/json_builtins @@ -1031,4 +1031,3 @@ query T SELECT '["a", {"b":1}]'::jsonb #- '{1,b}' ---- ["a", {}] - diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 210b1971a178..56aff3882878 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -613,27 +613,31 @@ SELECT * FROM pg_catalog.pg_index ORDER BY indexrelid ---- -indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred -543291288 23 1 false false false false false true false false true false 1 3903121477 0 0 NULL NULL -543291289 23 1 false false false false false true false false true false 2 3903121477 0 0 NULL NULL -543291291 23 2 true true false true false true false false true false 1 2 3903121477 3903121477 0 0 0 0 NULL NULL -1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 0 0 NULL NULL -1582236367 3 1 true true false true false true false false true false 1 0 0 0 NULL NULL -1628632028 19 1 false false false false false true false false true false 5 0 0 0 NULL NULL -1628632029 19 1 false false false false false true false false true false 4 0 0 0 NULL NULL -1628632031 19 1 true true false true false true false false true false 1 0 0 0 NULL NULL -1841972634 6 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL -2101708905 5 1 true true false true false true false false true false 1 0 0 0 NULL NULL -2148104569 21 2 true true false true false true false false true false 1 2 3903121477 3903121477 0 0 0 0 NULL NULL -2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 0 0 0 NULL NULL -2621181440 15 2 false false false false false true false false true false 2 3 3903121477 0 0 0 0 0 NULL NULL -2621181443 15 1 true true false true false true false false true false 1 0 0 0 NULL NULL -2927313374 2 2 true true false true false true false false true false 1 2 0 3903121477 0 0 0 0 NULL NULL -3446785912 4 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL -3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 0 0 NULL NULL -3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 0 0 0 0 NULL NULL -3966258450 14 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL -4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 0 0 NULL NULL +indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred +543291288 23 1 false false false false false true false false true false 1 3903121477 0 0 NULL NULL +543291289 23 1 false false false false false true false false true false 2 3903121477 0 0 NULL NULL +543291291 23 2 true true false true false true false false true false 1 2 3903121477 3903121477 0 0 0 0 NULL NULL +803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3903121477 0 0 0 0 0 0 NULL NULL +1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3903121477 3903121477 0 0 0 0 0 0 0 0 NULL NULL +1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 0 0 NULL NULL +1322500096 28 1 true true false true false true false false true false 1 0 0 0 NULL NULL +1582236367 3 1 true true false true false true false false true false 1 0 0 0 NULL NULL +1628632028 19 1 false false false false false true false false true false 5 0 0 0 NULL NULL +1628632029 19 1 false false false false false true false false true false 4 0 0 0 NULL NULL +1628632031 19 1 true true false true false true false false true false 1 0 0 0 NULL NULL +1841972634 6 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL +2101708905 5 1 true true false true false true false false true false 1 0 0 0 NULL NULL +2148104569 21 2 true true false true false true false false true false 1 2 3903121477 3903121477 0 0 0 0 NULL NULL +2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 0 0 0 NULL NULL +2621181440 15 2 false false false false false true false false true false 2 3 3903121477 0 0 0 0 0 NULL NULL +2621181443 15 1 true true false true false true false false true false 1 0 0 0 NULL NULL +2927313374 2 2 true true false true false true false false true false 1 2 0 3903121477 0 0 0 0 NULL NULL +3446785912 4 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL +3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 0 0 NULL NULL +3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 0 0 0 0 NULL NULL +3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 0 0 NULL NULL +3966258450 14 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL +4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 0 0 NULL NULL # From #26504 query OOI colnames @@ -648,8 +652,16 @@ indexrelid operator_argument_type_oid operator_argument_position 543291289 0 1 543291291 0 1 543291291 0 2 +803027558 0 1 +803027558 0 2 +803027558 0 3 +1062763829 0 1 +1062763829 0 2 +1062763829 0 3 +1062763829 0 4 1276104432 0 1 1276104432 0 2 +1322500096 0 1 1582236367 0 1 1628632028 0 1 1628632029 0 1 @@ -673,6 +685,8 @@ indexrelid operator_argument_type_oid operator_argument_position 3706522183 0 2 3706522183 0 3 3706522183 0 4 +3752917847 0 1 +3752917847 0 2 3966258450 0 1 4225994721 0 1 4225994721 0 2 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index d033f3c68f0d..8600b81bb544 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -274,85 +274,93 @@ ALTER INDEX d.c@c_i_idx SPLIT AT VALUES (0) query TTTTTTTTI colnames SELECT start_key, start_pretty, end_key, end_pretty, database_name, table_name, index_name, replicas, crdb_internal.lease_holder(start_key) FROM crdb_internal.ranges_no_leases; ---- -start_key start_pretty end_key end_pretty database_name table_name index_name replicas crdb_internal.lease_holder -· /Min liveness- /System/NodeLiveness · · · {1} 1 -liveness- /System/NodeLiveness liveness. /System/NodeLivenessMax · · · {1} 1 -liveness. /System/NodeLivenessMax tsd /System/tsd · · · {1} 1 -tsd /System/tsd tse /System/"tse" · · · {1} 1 -tse /System/"tse" [136] /Table/SystemConfigSpan/Start · · · {1} 1 -[136] /Table/SystemConfigSpan/Start [147] /Table/11 · · · {1} 1 -[147] /Table/11 [148] /Table/12 system lease · {1} 1 -[148] /Table/12 [149] /Table/13 system eventlog · {1} 1 -[149] /Table/13 [150] /Table/14 system rangelog · {1} 1 -[150] /Table/14 [151] /Table/15 system ui · {1} 1 -[151] /Table/15 [152] /Table/16 system jobs · {1} 1 -[152] /Table/16 [153] /Table/17 · · · {1} 1 -[153] /Table/17 [154] /Table/18 · · · {1} 1 -[154] /Table/18 [155] /Table/19 · · · {1} 1 -[155] /Table/19 [156] /Table/20 system web_sessions · {1} 1 -[156] /Table/20 [157] /Table/21 system table_statistics · {1} 1 -[157] /Table/21 [158] /Table/22 system locations · {1} 1 -[158] /Table/22 [159] /Table/23 · · · {1} 1 -[159] /Table/23 [160] /Table/24 system role_members · {1} 1 -[160] /Table/24 [189 137] /Table/53/1 system comments · {1} 1 -[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 -[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 -[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 -[189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {2,3,5} 5 -[189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {1,2,4} 4 -[189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {1,2,4} 4 -[189 137 146] /Table/53/1/10 [189 137 147] /Table/53/1/11 test t · {1} 1 -[189 137 147] /Table/53/1/11 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 test t · {1} 1 -[189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 [189 138] /Table/53/2 test t · {1} 1 -[189 138] /Table/53/2 [189 138 144] /Table/53/2/8 test t idx {1} 1 -[189 138 144] /Table/53/2/8 [189 138 145] /Table/53/2/9 test t idx {1} 1 -[189 138 145] /Table/53/2/9 [189 138 236 137] /Table/53/2/100/1 test t idx {1} 1 -[189 138 236 137] /Table/53/2/100/1 [189 138 236 186] /Table/53/2/100/50 test t idx {3} 3 -[189 138 236 186] /Table/53/2/100/50 [195 137 136] /Table/59/1/0 test t idx {1} 1 -[195 137 136] /Table/59/1/0 [196 137 246 123] /Table/60/1/123 · b · {1} 1 -[196 137 246 123] /Table/60/1/123 ÄŠ /Table/60/2 d c · {1} 1 -ÄŠ /Table/60/2 [196 138 136] /Table/60/2/0 d c c_i_idx {1} 1 -[196 138 136] /Table/60/2/0 [255 255] /Max d c c_i_idx {1} 1 +start_key start_pretty end_key end_pretty database_name table_name index_name replicas crdb_internal.lease_holder +· /Min liveness- /System/NodeLiveness · · · {1} 1 +liveness- /System/NodeLiveness liveness. /System/NodeLivenessMax · · · {1} 1 +liveness. /System/NodeLivenessMax tsd /System/tsd · · · {1} 1 +tsd /System/tsd tse /System/"tse" · · · {1} 1 +tse /System/"tse" [136] /Table/SystemConfigSpan/Start · · · {1} 1 +[136] /Table/SystemConfigSpan/Start [147] /Table/11 · · · {1} 1 +[147] /Table/11 [148] /Table/12 system lease · {1} 1 +[148] /Table/12 [149] /Table/13 system eventlog · {1} 1 +[149] /Table/13 [150] /Table/14 system rangelog · {1} 1 +[150] /Table/14 [151] /Table/15 system ui · {1} 1 +[151] /Table/15 [152] /Table/16 system jobs · {1} 1 +[152] /Table/16 [153] /Table/17 · · · {1} 1 +[153] /Table/17 [154] /Table/18 · · · {1} 1 +[154] /Table/18 [155] /Table/19 · · · {1} 1 +[155] /Table/19 [156] /Table/20 system web_sessions · {1} 1 +[156] /Table/20 [157] /Table/21 system table_statistics · {1} 1 +[157] /Table/21 [158] /Table/22 system locations · {1} 1 +[158] /Table/22 [159] /Table/23 · · · {1} 1 +[159] /Table/23 [160] /Table/24 system role_members · {1} 1 +[160] /Table/24 [161] /Table/25 system comments · {1} 1 +[161] /Table/25 [162] /Table/26 system replication_constraint_stats · {1} 1 +[162] /Table/26 [163] /Table/27 system replication_critical_localities · {1} 1 +[163] /Table/27 [164] /Table/28 system replication_stats · {1} 1 +[164] /Table/28 [189 137] /Table/53/1 system reports_meta · {1} 1 +[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 +[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 +[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 +[189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {2,3,5} 5 +[189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {1,2,4} 4 +[189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {1,2,4} 4 +[189 137 146] /Table/53/1/10 [189 137 147] /Table/53/1/11 test t · {1} 1 +[189 137 147] /Table/53/1/11 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 test t · {1} 1 +[189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 [189 138] /Table/53/2 test t · {1} 1 +[189 138] /Table/53/2 [189 138 144] /Table/53/2/8 test t idx {1} 1 +[189 138 144] /Table/53/2/8 [189 138 145] /Table/53/2/9 test t idx {1} 1 +[189 138 145] /Table/53/2/9 [189 138 236 137] /Table/53/2/100/1 test t idx {1} 1 +[189 138 236 137] /Table/53/2/100/1 [189 138 236 186] /Table/53/2/100/50 test t idx {3} 3 +[189 138 236 186] /Table/53/2/100/50 [195 137 136] /Table/59/1/0 test t idx {1} 1 +[195 137 136] /Table/59/1/0 [196 137 246 123] /Table/60/1/123 · b · {1} 1 +[196 137 246 123] /Table/60/1/123 ÄŠ /Table/60/2 d c · {1} 1 +ÄŠ /Table/60/2 [196 138 136] /Table/60/2/0 d c c_i_idx {1} 1 +[196 138 136] /Table/60/2/0 [255 255] /Max d c c_i_idx {1} 1 query TTTTTTTTI colnames SELECT start_key, start_pretty, end_key, end_pretty, database_name, table_name, index_name, replicas, lease_holder FROM crdb_internal.ranges ---- -start_key start_pretty end_key end_pretty database_name table_name index_name replicas lease_holder -· /Min liveness- /System/NodeLiveness · · · {1} 1 -liveness- /System/NodeLiveness liveness. /System/NodeLivenessMax · · · {1} 1 -liveness. /System/NodeLivenessMax tsd /System/tsd · · · {1} 1 -tsd /System/tsd tse /System/"tse" · · · {1} 1 -tse /System/"tse" [136] /Table/SystemConfigSpan/Start · · · {1} 1 -[136] /Table/SystemConfigSpan/Start [147] /Table/11 · · · {1} 1 -[147] /Table/11 [148] /Table/12 system lease · {1} 1 -[148] /Table/12 [149] /Table/13 system eventlog · {1} 1 -[149] /Table/13 [150] /Table/14 system rangelog · {1} 1 -[150] /Table/14 [151] /Table/15 system ui · {1} 1 -[151] /Table/15 [152] /Table/16 system jobs · {1} 1 -[152] /Table/16 [153] /Table/17 · · · {1} 1 -[153] /Table/17 [154] /Table/18 · · · {1} 1 -[154] /Table/18 [155] /Table/19 · · · {1} 1 -[155] /Table/19 [156] /Table/20 system web_sessions · {1} 1 -[156] /Table/20 [157] /Table/21 system table_statistics · {1} 1 -[157] /Table/21 [158] /Table/22 system locations · {1} 1 -[158] /Table/22 [159] /Table/23 · · · {1} 1 -[159] /Table/23 [160] /Table/24 system role_members · {1} 1 -[160] /Table/24 [189 137] /Table/53/1 system comments · {1} 1 -[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 -[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 -[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 -[189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {2,3,5} 5 -[189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {1,2,4} 4 -[189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {1,2,4} 4 -[189 137 146] /Table/53/1/10 [189 137 147] /Table/53/1/11 test t · {1} 1 -[189 137 147] /Table/53/1/11 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 test t · {1} 1 -[189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 [189 138] /Table/53/2 test t · {1} 1 -[189 138] /Table/53/2 [189 138 144] /Table/53/2/8 test t idx {1} 1 -[189 138 144] /Table/53/2/8 [189 138 145] /Table/53/2/9 test t idx {1} 1 -[189 138 145] /Table/53/2/9 [189 138 236 137] /Table/53/2/100/1 test t idx {1} 1 -[189 138 236 137] /Table/53/2/100/1 [189 138 236 186] /Table/53/2/100/50 test t idx {3} 3 -[189 138 236 186] /Table/53/2/100/50 [195 137 136] /Table/59/1/0 test t idx {1} 1 -[195 137 136] /Table/59/1/0 [196 137 246 123] /Table/60/1/123 · b · {1} 1 -[196 137 246 123] /Table/60/1/123 ÄŠ /Table/60/2 d c · {1} 1 -ÄŠ /Table/60/2 [196 138 136] /Table/60/2/0 d c c_i_idx {1} 1 -[196 138 136] /Table/60/2/0 [255 255] /Max d c c_i_idx {1} 1 +start_key start_pretty end_key end_pretty database_name table_name index_name replicas lease_holder +· /Min liveness- /System/NodeLiveness · · · {1} 1 +liveness- /System/NodeLiveness liveness. /System/NodeLivenessMax · · · {1} 1 +liveness. /System/NodeLivenessMax tsd /System/tsd · · · {1} 1 +tsd /System/tsd tse /System/"tse" · · · {1} 1 +tse /System/"tse" [136] /Table/SystemConfigSpan/Start · · · {1} 1 +[136] /Table/SystemConfigSpan/Start [147] /Table/11 · · · {1} 1 +[147] /Table/11 [148] /Table/12 system lease · {1} 1 +[148] /Table/12 [149] /Table/13 system eventlog · {1} 1 +[149] /Table/13 [150] /Table/14 system rangelog · {1} 1 +[150] /Table/14 [151] /Table/15 system ui · {1} 1 +[151] /Table/15 [152] /Table/16 system jobs · {1} 1 +[152] /Table/16 [153] /Table/17 · · · {1} 1 +[153] /Table/17 [154] /Table/18 · · · {1} 1 +[154] /Table/18 [155] /Table/19 · · · {1} 1 +[155] /Table/19 [156] /Table/20 system web_sessions · {1} 1 +[156] /Table/20 [157] /Table/21 system table_statistics · {1} 1 +[157] /Table/21 [158] /Table/22 system locations · {1} 1 +[158] /Table/22 [159] /Table/23 · · · {1} 1 +[159] /Table/23 [160] /Table/24 system role_members · {1} 1 +[160] /Table/24 [161] /Table/25 system comments · {1} 1 +[161] /Table/25 [162] /Table/26 system replication_constraint_stats · {1} 1 +[162] /Table/26 [163] /Table/27 system replication_critical_localities · {1} 1 +[163] /Table/27 [164] /Table/28 system replication_stats · {1} 1 +[164] /Table/28 [189 137] /Table/53/1 system reports_meta · {1} 1 +[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 +[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 +[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 +[189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {2,3,5} 5 +[189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {1,2,4} 4 +[189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {1,2,4} 4 +[189 137 146] /Table/53/1/10 [189 137 147] /Table/53/1/11 test t · {1} 1 +[189 137 147] /Table/53/1/11 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 test t · {1} 1 +[189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 [189 138] /Table/53/2 test t · {1} 1 +[189 138] /Table/53/2 [189 138 144] /Table/53/2/8 test t idx {1} 1 +[189 138 144] /Table/53/2/8 [189 138 145] /Table/53/2/9 test t idx {1} 1 +[189 138 145] /Table/53/2/9 [189 138 236 137] /Table/53/2/100/1 test t idx {1} 1 +[189 138 236 137] /Table/53/2/100/1 [189 138 236 186] /Table/53/2/100/50 test t idx {3} 3 +[189 138 236 186] /Table/53/2/100/50 [195 137 136] /Table/59/1/0 test t idx {1} 1 +[195 137 136] /Table/59/1/0 [196 137 246 123] /Table/60/1/123 · b · {1} 1 +[196 137 246 123] /Table/60/1/123 ÄŠ /Table/60/2 d c · {1} 1 +ÄŠ /Table/60/2 [196 138 136] /Table/60/2/0 d c c_i_idx {1} 1 +[196 138 136] /Table/60/2/0 [255 255] /Max d c c_i_idx {1} 1 diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index fecbed1f2c3b..8ef6165987b6 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -159,41 +159,49 @@ query T colnames,rowsort SELECT * FROM [SHOW TABLES FROM system] ---- table_name -comments +namespace descriptor -eventlog -jobs +users +zones +settings lease -locations -namespace +eventlog rangelog -role_members -settings -table_statistics ui -users +jobs web_sessions -zones +table_statistics +locations +role_members +comments +replication_constraint_stats +replication_critical_localities +replication_stats +reports_meta query TT colnames,rowsort SELECT * FROM [SHOW TABLES FROM system WITH COMMENT] ---- -table_name comment -namespace · -descriptor · -users · -zones · -settings · -lease · -eventlog · -rangelog · -ui · -jobs · -web_sessions · -table_statistics · -locations · -role_members · -comments · +table_name comment +namespace · +descriptor · +users · +zones · +settings · +lease · +eventlog · +rangelog · +ui · +jobs · +web_sessions · +table_statistics · +locations · +role_members · +comments · +replication_constraint_stats · +replication_critical_localities · +replication_stats · +reports_meta · query ITTT colnames SELECT node_id, user_name, application_name, active_queries diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 15f66419634a..8d16c2070549 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -17,6 +17,10 @@ lease locations namespace rangelog +replication_constraint_stats +replication_critical_localities +replication_stats +reports_meta role_members settings table_statistics @@ -28,25 +32,29 @@ zones query ITI rowsort SELECT * FROM system.namespace ---- -0 defaultdb 50 -0 postgres 51 -0 system 1 -0 test 52 -1 comments 24 -1 descriptor 3 -1 eventlog 12 -1 jobs 15 -1 lease 11 -1 locations 21 -1 namespace 2 -1 rangelog 13 -1 role_members 23 -1 settings 6 -1 table_statistics 20 -1 ui 14 -1 users 4 -1 web_sessions 19 -1 zones 5 +0 defaultdb 50 +0 postgres 51 +0 system 1 +0 test 52 +1 comments 24 +1 descriptor 3 +1 eventlog 12 +1 jobs 15 +1 lease 11 +1 locations 21 +1 namespace 2 +1 rangelog 13 +1 replication_constraint_stats 25 +1 replication_critical_localities 26 +1 replication_stats 27 +1 reports_meta 28 +1 role_members 23 +1 settings 6 +1 table_statistics 20 +1 ui 14 +1 users 4 +1 web_sessions 19 +1 zones 5 query I rowsort SELECT id FROM system.descriptor @@ -67,6 +75,10 @@ SELECT id FROM system.descriptor 21 23 24 +25 +26 +27 +28 50 51 52 @@ -189,149 +201,189 @@ system public root SELECT query TTTTT SHOW GRANTS ON system.* ---- -system public comments admin DELETE -system public comments admin GRANT -system public comments admin INSERT -system public comments admin SELECT -system public comments admin UPDATE -system public comments public DELETE -system public comments public GRANT -system public comments public INSERT -system public comments public SELECT -system public comments public UPDATE -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public descriptor admin GRANT -system public descriptor admin SELECT -system public descriptor root GRANT -system public descriptor root SELECT -system public eventlog admin DELETE -system public eventlog admin GRANT -system public eventlog admin INSERT -system public eventlog admin SELECT -system public eventlog admin UPDATE -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public jobs admin DELETE -system public jobs admin GRANT -system public jobs admin INSERT -system public jobs admin SELECT -system public jobs admin UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public lease admin DELETE -system public lease admin GRANT -system public lease admin INSERT -system public lease admin SELECT -system public lease admin UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public locations admin DELETE -system public locations admin GRANT -system public locations admin INSERT -system public locations admin SELECT -system public locations admin UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public namespace admin GRANT -system public namespace admin SELECT -system public namespace root GRANT -system public namespace root SELECT -system public rangelog admin DELETE -system public rangelog admin GRANT -system public rangelog admin INSERT -system public rangelog admin SELECT -system public rangelog admin UPDATE -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public role_members admin DELETE -system public role_members admin GRANT -system public role_members admin INSERT -system public role_members admin SELECT -system public role_members admin UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public settings admin DELETE -system public settings admin GRANT -system public settings admin INSERT -system public settings admin SELECT -system public settings admin UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public table_statistics admin DELETE -system public table_statistics admin GRANT -system public table_statistics admin INSERT -system public table_statistics admin SELECT -system public table_statistics admin UPDATE -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public ui admin DELETE -system public ui admin GRANT -system public ui admin INSERT -system public ui admin SELECT -system public ui admin UPDATE -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public users admin DELETE -system public users admin GRANT -system public users admin INSERT -system public users admin SELECT -system public users admin UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public web_sessions admin DELETE -system public web_sessions admin GRANT -system public web_sessions admin INSERT -system public web_sessions admin SELECT -system public web_sessions admin UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public zones admin DELETE -system public zones admin GRANT -system public zones admin INSERT -system public zones admin SELECT -system public zones admin UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE +system public comments admin DELETE +system public comments admin GRANT +system public comments admin INSERT +system public comments admin SELECT +system public comments admin UPDATE +system public comments public DELETE +system public comments public GRANT +system public comments public INSERT +system public comments public SELECT +system public comments public UPDATE +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public descriptor admin GRANT +system public descriptor admin SELECT +system public descriptor root GRANT +system public descriptor root SELECT +system public eventlog admin DELETE +system public eventlog admin GRANT +system public eventlog admin INSERT +system public eventlog admin SELECT +system public eventlog admin UPDATE +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public jobs admin DELETE +system public jobs admin GRANT +system public jobs admin INSERT +system public jobs admin SELECT +system public jobs admin UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public lease admin DELETE +system public lease admin GRANT +system public lease admin INSERT +system public lease admin SELECT +system public lease admin UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public locations admin DELETE +system public locations admin GRANT +system public locations admin INSERT +system public locations admin SELECT +system public locations admin UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT +system public rangelog admin DELETE +system public rangelog admin GRANT +system public rangelog admin INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public replication_constraint_stats admin DELETE +system public replication_constraint_stats admin GRANT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities admin DELETE +system public replication_critical_localities admin GRANT +system public replication_critical_localities admin INSERT +system public replication_critical_localities admin SELECT +system public replication_critical_localities admin UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats admin DELETE +system public replication_stats admin GRANT +system public replication_stats admin INSERT +system public replication_stats admin SELECT +system public replication_stats admin UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta admin DELETE +system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta admin SELECT +system public reports_meta admin UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public role_members admin DELETE +system public role_members admin GRANT +system public role_members admin INSERT +system public role_members admin SELECT +system public role_members admin UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public settings admin DELETE +system public settings admin GRANT +system public settings admin INSERT +system public settings admin SELECT +system public settings admin UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public table_statistics admin DELETE +system public table_statistics admin GRANT +system public table_statistics admin INSERT +system public table_statistics admin SELECT +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public ui admin DELETE +system public ui admin GRANT +system public ui admin INSERT +system public ui admin SELECT +system public ui admin UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public users admin DELETE +system public users admin GRANT +system public users admin INSERT +system public users admin SELECT +system public users admin UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public web_sessions admin DELETE +system public web_sessions admin GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public zones admin DELETE +system public zones admin GRANT +system public zones admin INSERT +system public zones admin SELECT +system public zones admin UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE statement error user root does not have DROP privilege on database system ALTER DATABASE system RENAME TO not_system diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 3c18bde72d0f..868790669b6e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -76,7 +76,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r20: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -91,7 +91,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r20: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" statement error duplicate key value @@ -104,7 +104,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r20: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" statement ok @@ -186,7 +186,7 @@ table reader Scan /Table/54/{1-2} table reader fetched: /kv/primary/1/v -> /2 flow Del /Table/54/2/2/0 flow Del /Table/54/1/1/0 -kv.DistSender: sending partial batch r20: sending batch 1 Del to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 Del to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -243,7 +243,7 @@ SET tracing = on; INSERT INTO t.kv3 (k, v) VALUES (1,1); SET tracing = off query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%' ---- -r21: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +r25: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 1 CPut, 1 EndTxn ## TODO(tschottdorf): re-enable @@ -298,7 +298,7 @@ SELECT DISTINCT node_id, store_id, replica_id node_id store_id replica_id 1 1 6 1 1 7 -1 1 21 +1 1 25 subtest system_table_lookup diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 756a9cf13b30..b56173ab77fe 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -231,7 +231,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/2{-/#} flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r20: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -245,7 +245,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/1{-/#} flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r20: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -262,7 +262,7 @@ table reader fetched: /kv/primary/2/v -> /3 flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 flow Del /Table/57/2/3/0 flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) -kv.DistSender: sending partial batch r20: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r24: sending batch 1 Put, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 964427f46aad..0f9fd96e1393 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -732,7 +732,7 @@ func TestPGPreparedQuery(t *testing.T) { baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("comments").Others(14), + baseTest.Results("comments").Others(18), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal").Others(3), diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 53d040e25cc3..3d937a48decd 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -652,11 +652,11 @@ func validateZoneAttrsAndLocalities( node: for _, node := range nodes.Nodes { for _, store := range node.StoreStatuses { - // We could alternatively use config.storeHasConstraint here to catch - // typos in prohibited constraints as well, but as noted in the - // function-level comment that could break very reasonable use cases - // for prohibited constraints. - if config.StoreMatchesConstraint(store.Desc, constraint) { + // We could alternatively use config.StoreMatchesConstraint here to + // catch typos in prohibited constraints as well, but as noted in the + // function-level comment that could break very reasonable use cases for + // prohibited constraints. + if config.StoreSatisfiesConstraint(store.Desc, constraint) { found = true break node } diff --git a/pkg/sql/show_zone_config.go b/pkg/sql/show_zone_config.go index 1fb5a262cb6d..c119b6571fa4 100644 --- a/pkg/sql/show_zone_config.go +++ b/pkg/sql/show_zone_config.go @@ -28,6 +28,7 @@ import ( // These must match crdb_internal.zones. var showZoneConfigColumns = sqlbase.ResultColumns{ {Name: "zone_id", Typ: types.Int, Hidden: true}, + {Name: "subzone_id", Typ: types.Int, Hidden: true}, {Name: "target", Typ: types.String}, {Name: "range_name", Typ: types.String, Hidden: true}, {Name: "database_name", Typ: types.String, Hidden: true}, @@ -42,6 +43,7 @@ var showZoneConfigColumns = sqlbase.ResultColumns{ // These must match showZoneConfigColumns. const ( zoneIDCol int = iota + subZoneIDCol targetCol rangeNameCol databaseNameCol @@ -111,6 +113,7 @@ func getShowZoneConfigRow( return nil, err } + subZoneIdx := uint32(0) zoneID, zone, subzone, err := GetZoneConfigInTxn(ctx, p.txn, uint32(targetID), index, partition, false /* getInheritedDefault */) if err == errNoZoneConfigApplies { @@ -122,6 +125,12 @@ func getShowZoneConfigRow( } else if err != nil { return nil, err } else if subzone != nil { + for i := range zone.Subzones { + subZoneIdx++ + if subzone == &zone.Subzones[i] { + break + } + } zone = &subzone.Config } @@ -135,7 +144,7 @@ func getShowZoneConfigRow( vals := make(tree.Datums, len(showZoneConfigColumns)) if err := generateZoneConfigIntrospectionValues( - vals, tree.NewDInt(tree.DInt(zoneID)), &zs, zone, + vals, tree.NewDInt(tree.DInt(zoneID)), tree.NewDInt(tree.DInt(subZoneIdx)), &zs, zone, ); err != nil { return nil, err } @@ -149,10 +158,15 @@ func getShowZoneConfigRow( // provide it as 2nd argument. The function will compute // the remaining values based on the zone specifier and configuration. func generateZoneConfigIntrospectionValues( - values tree.Datums, zoneID tree.Datum, zs *tree.ZoneSpecifier, zone *config.ZoneConfig, + values tree.Datums, + zoneID tree.Datum, + subZoneID tree.Datum, + zs *tree.ZoneSpecifier, + zone *config.ZoneConfig, ) error { // Populate the ID column. values[zoneIDCol] = zoneID + values[subZoneIDCol] = subZoneID // Populate the zone specifier columns. values[targetCol] = tree.DNull diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index 71c345c3da65..c765b2d58a9d 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -12,6 +12,7 @@ package sqlbase import ( "fmt" + "time" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" @@ -258,12 +259,16 @@ var SystemAllowedPrivileges = map[ID]privilege.List{ // users will be able to modify system tables' schemas at will. CREATE and // DROP privileges are allowed on the above system tables for backwards // compatibility reasons only! - keys.JobsTableID: privilege.ReadWriteData, - keys.WebSessionsTableID: privilege.ReadWriteData, - keys.TableStatisticsTableID: privilege.ReadWriteData, - keys.LocationsTableID: privilege.ReadWriteData, - keys.RoleMembersTableID: privilege.ReadWriteData, - keys.CommentsTableID: privilege.ReadWriteData, + keys.JobsTableID: privilege.ReadWriteData, + keys.WebSessionsTableID: privilege.ReadWriteData, + keys.TableStatisticsTableID: privilege.ReadWriteData, + keys.LocationsTableID: privilege.ReadWriteData, + keys.RoleMembersTableID: privilege.ReadWriteData, + keys.CommentsTableID: privilege.ReadWriteData, + keys.ReplicationConstraintStatsTableID: privilege.ReadWriteData, + keys.ReplicationCriticalLocalitiesTableID: privilege.ReadWriteData, + keys.ReplicationStatsTableID: privilege.ReadWriteData, + keys.ReportsMetaTableID: privilege.ReadWriteData, } // Helpers used to make some of the TableDescriptor literals below more concise. @@ -861,6 +866,190 @@ var ( FormatVersion: InterleavedFormatVersion, NextMutationID: 1, } + + ReportsMetaTable = TableDescriptor{ + Name: "reports_meta", + ID: keys.ReportsMetaTableID, + ParentID: keys.SystemDatabaseID, + Version: 1, + Columns: []ColumnDescriptor{ + {Name: "id", ID: 1, Type: *types.Int}, + {Name: "generated", ID: 2, Type: *types.Timestamp}, + }, + NextColumnID: 3, + Families: []ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{"id", "generated"}, + ColumnIDs: []ColumnID{1, 2}, + }, + }, + NextFamilyID: 1, + PrimaryIndex: IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + ColumnNames: []string{"id"}, + ColumnDirections: []IndexDescriptor_Direction{ + IndexDescriptor_ASC, + }, + ColumnIDs: []ColumnID{1}, + }, + NextIndexID: 2, + Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.ReportsMetaTableID]), + FormatVersion: InterleavedFormatVersion, + NextMutationID: 1, + } + + ReplicationConstraintStatsTableTTL = time.Minute * 10 + // TODO(andrei): In 20.1 we should add a foreign key reference to the + // reports_meta table. Until then, it would cost us having to create an index + // on report_id. + ReplicationConstraintStatsTable = TableDescriptor{ + Name: "replication_constraint_stats", + ID: keys.ReplicationConstraintStatsTableID, + ParentID: keys.SystemDatabaseID, + Version: 1, + Columns: []ColumnDescriptor{ + {Name: "zone_id", ID: 1, Type: *types.Int}, + {Name: "subzone_id", ID: 2, Type: *types.Int}, + {Name: "type", ID: 3, Type: *types.String}, + {Name: "config", ID: 4, Type: *types.String}, + {Name: "report_id", ID: 5, Type: *types.Int}, + {Name: "violation_start", ID: 6, Type: *types.Timestamp, Nullable: true}, + {Name: "violating_ranges", ID: 7, Type: *types.Int}, + }, + NextColumnID: 8, + Families: []ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{ + "zone_id", + "subzone_id", + "type", + "config", + "report_id", + "violation_start", + "violating_ranges", + }, + ColumnIDs: []ColumnID{1, 2, 3, 4, 5, 6, 7}, + }, + }, + NextFamilyID: 1, + PrimaryIndex: IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + ColumnNames: []string{"zone_id", "subzone_id", "type", "config"}, + ColumnDirections: []IndexDescriptor_Direction{ + IndexDescriptor_ASC, IndexDescriptor_ASC, IndexDescriptor_ASC, IndexDescriptor_ASC, + }, + ColumnIDs: []ColumnID{1, 2, 3, 4}, + }, + NextIndexID: 2, + Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.ReplicationConstraintStatsTableID]), + FormatVersion: InterleavedFormatVersion, + NextMutationID: 1, + } + + // TODO(andrei): In 20.1 we should add a foreign key reference to the + // reports_meta table. Until then, it would cost us having to create an index + // on report_id. + ReplicationCriticalLocalitiesTable = TableDescriptor{ + Name: "replication_critical_localities", + ID: keys.ReplicationCriticalLocalitiesTableID, + ParentID: keys.SystemDatabaseID, + Version: 1, + Columns: []ColumnDescriptor{ + {Name: "zone_id", ID: 1, Type: *types.Int}, + {Name: "subzone_id", ID: 2, Type: *types.Int}, + {Name: "locality", ID: 3, Type: *types.String}, + {Name: "report_id", ID: 4, Type: *types.Int}, + {Name: "at_risk_ranges", ID: 5, Type: *types.Int}, + }, + NextColumnID: 6, + Families: []ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{ + "zone_id", + "subzone_id", + "locality", + "report_id", + "at_risk_ranges", + }, + ColumnIDs: []ColumnID{1, 2, 3, 4, 5}, + }, + }, + NextFamilyID: 1, + PrimaryIndex: IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + ColumnNames: []string{"zone_id", "subzone_id", "locality"}, + ColumnDirections: []IndexDescriptor_Direction{ + IndexDescriptor_ASC, IndexDescriptor_ASC, IndexDescriptor_ASC, + }, + ColumnIDs: []ColumnID{1, 2, 3}, + }, + NextIndexID: 2, + Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.ReplicationCriticalLocalitiesTableID]), + FormatVersion: InterleavedFormatVersion, + NextMutationID: 1, + } + + ReplicationStatsTableTTL = time.Minute * 10 + // TODO(andrei): In 20.1 we should add a foreign key reference to the + // reports_meta table. Until then, it would cost us having to create an index + // on report_id. + ReplicationStatsTable = TableDescriptor{ + Name: "replication_stats", + ID: keys.ReplicationStatsTableID, + ParentID: keys.SystemDatabaseID, + Version: 1, + Columns: []ColumnDescriptor{ + {Name: "zone_id", ID: 1, Type: *types.Int}, + {Name: "subzone_id", ID: 2, Type: *types.Int}, + {Name: "report_id", ID: 3, Type: *types.Int}, + {Name: "total_ranges", ID: 4, Type: *types.Int}, + {Name: "unavailable_ranges", ID: 5, Type: *types.Int}, + {Name: "under_replicated_ranges", ID: 6, Type: *types.Int}, + {Name: "over_replicated_ranges", ID: 7, Type: *types.Int}, + }, + NextColumnID: 8, + Families: []ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{ + "zone_id", + "subzone_id", + "report_id", + "total_ranges", + "unavailable_ranges", + "under_replicated_ranges", + "over_replicated_ranges", + }, + ColumnIDs: []ColumnID{1, 2, 3, 4, 5, 6, 7}, + }, + }, + NextFamilyID: 2, + PrimaryIndex: IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + ColumnNames: []string{"zone_id", "subzone_id"}, + ColumnDirections: []IndexDescriptor_Direction{IndexDescriptor_ASC, IndexDescriptor_ASC}, + ColumnIDs: []ColumnID{1, 2}, + }, + NextIndexID: 2, + Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.ReplicationStatsTableID]), + FormatVersion: InterleavedFormatVersion, + NextMutationID: 1, + } ) // Create a kv pair for the zone config for the given key and config value. @@ -906,6 +1095,10 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { // The CommentsTable has been introduced in 2.2. It was added here since it // was introduced, but it's also created as a migration for older clusters. target.AddDescriptor(keys.SystemDatabaseID, &CommentsTable) + target.AddDescriptor(keys.SystemDatabaseID, &ReportsMetaTable) + target.AddDescriptor(keys.SystemDatabaseID, &ReplicationConstraintStatsTable) + target.AddDescriptor(keys.SystemDatabaseID, &ReplicationStatsTable) + target.AddDescriptor(keys.SystemDatabaseID, &ReplicationCriticalLocalitiesTable) } // addSystemDatabaseToSchema populates the supplied MetadataSchema with the @@ -938,11 +1131,23 @@ func addSystemDatabaseToSchema( jobsZoneConf.GC.TTLSeconds = 10 * 60 // 10m target.otherKV = append(target.otherKV, createZoneConfigKV(keys.JobsTableID, jobsZoneConf)) + // Some reporting tables have shorter GC times. + replicationConstraintStatsZoneConf := &config.ZoneConfig{ + GC: &config.GCPolicy{TTLSeconds: int32(ReplicationConstraintStatsTableTTL.Seconds())}, + } + replicationStatsZoneConf := &config.ZoneConfig{ + GC: &config.GCPolicy{TTLSeconds: int32(ReplicationStatsTableTTL.Seconds())}, + } + // Liveness zone config entry with a shorter GC time. livenessZoneConf.GC.TTLSeconds = 10 * 60 // 10m target.otherKV = append(target.otherKV, createZoneConfigKV(keys.LivenessRangesID, livenessZoneConf)) target.otherKV = append(target.otherKV, createZoneConfigKV(keys.SystemRangesID, systemZoneConf)) target.otherKV = append(target.otherKV, createZoneConfigKV(keys.SystemDatabaseID, systemZoneConf)) + target.otherKV = append(target.otherKV, + createZoneConfigKV(keys.ReplicationConstraintStatsTableID, replicationConstraintStatsZoneConf)) + target.otherKV = append(target.otherKV, + createZoneConfigKV(keys.ReplicationStatsTableID, replicationStatsZoneConf)) } // IsSystemConfigID returns whether this ID is for a system config object. diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 0ebc6a817977..0355dd257730 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -30,7 +30,7 @@ func TestInitialKeys(t *testing.T) { defer leaktest.AfterTest(t)() const keysPerDesc = 2 - const nonDescKeys = 7 + const nonDescKeys = 9 ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) kv, _ /* splits */ := ms.GetInitialValues() diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 6f57ae0a0ac1..439c18745a2b 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -35,7 +35,7 @@ var errNoZoneConfigApplies = errors.New("no zone config applies") // getZoneConfig recursively looks up entries in system.zones until an // entry that applies to the object with the specified id is -// found. Returns the ID of the matching ID, its zone config, and an +// found. Returns the ID of the matching zone, its zone config, and an // optional placeholder ID and config if the looked-for ID was a table // with a zone config specifying only indexes and/or partitions. // diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index 67a8bd483623..97344a0dcd26 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -82,7 +82,7 @@ func waitForConfigChange(t testing.TB, s *server.TestServer) *config.SystemConfi // TODO(benesch,ridwansharif): modernize these tests to avoid hardcoding // expectations about descriptor IDs and zone config encoding. -// TestGetZoneConfig exercises config.GetZoneConfig and the sql hook for it. +// TestGetZoneConfig exercises config.getZoneConfig and the sql hook for it. func TestGetZoneConfig(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index e24626d3d4c6..944d04402158 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -200,8 +200,32 @@ var backwardCompatibleMigrations = []migrationDescriptor{ newDescriptorIDs: staticIDs(keys.CommentsTableID), }, { - // Introduced in v2.2. - // TODO(knz): bake this migration into v2.3. + name: "create system.replication_constraint_stats table", + workFn: createReplicationConstraintStatsTable, + includedInBootstrap: true, + newDescriptorIDs: staticIDs(keys.ReplicationConstraintStatsTableID), + }, + { + name: "create system.replication_critical_localities table", + workFn: createReplicationCriticalLocalitiesTable, + includedInBootstrap: true, + newDescriptorIDs: staticIDs(keys.ReplicationCriticalLocalitiesTableID), + }, + { + name: "create system.reports_meta table", + workFn: createReportsMetaTable, + includedInBootstrap: true, + newDescriptorIDs: staticIDs(keys.ReportsMetaTableID), + }, + { + name: "create system.replication_stats table", + workFn: createReplicationStatsTable, + includedInBootstrap: true, + newDescriptorIDs: staticIDs(keys.ReplicationStatsTableID), + }, + { + // Introduced in v19.1. + // TODO(knz): bake this migration into v19.2. name: "propagate the ts purge interval to the new setting names", workFn: retireOldTsPurgeIntervalSettings, }, @@ -556,6 +580,34 @@ func createCommentTable(ctx context.Context, r runner) error { return createSystemTable(ctx, r, sqlbase.CommentsTable) } +func createReplicationConstraintStatsTable(ctx context.Context, r runner) error { + if err := createSystemTable(ctx, r, sqlbase.ReplicationConstraintStatsTable); err != nil { + return err + } + _, err := r.sqlExecutor.Exec(ctx, "add-constraints-ttl", nil, /* txn */ + fmt.Sprintf("ALTER TABLE %s CONFIGURE ZONE USING gc.ttlseconds = %d", + sqlbase.ReplicationConstraintStatsTable.Name, int(sqlbase.ReplicationConstraintStatsTableTTL.Seconds()))) + return errors.Wrapf(err, "failed to set TTL on %s", sqlbase.ReplicationConstraintStatsTable.Name) +} + +func createReplicationCriticalLocalitiesTable(ctx context.Context, r runner) error { + return createSystemTable(ctx, r, sqlbase.ReplicationCriticalLocalitiesTable) +} + +func createReplicationStatsTable(ctx context.Context, r runner) error { + if err := createSystemTable(ctx, r, sqlbase.ReplicationStatsTable); err != nil { + return err + } + _, err := r.sqlExecutor.Exec(ctx, "add-replication-status-ttl", nil, /* txn */ + fmt.Sprintf("ALTER TABLE %s CONFIGURE ZONE USING gc.ttlseconds = %d", + sqlbase.ReplicationStatsTable.Name, int(sqlbase.ReplicationStatsTableTTL.Seconds()))) + return errors.Wrapf(err, "failed to set TTL on %s", sqlbase.ReplicationStatsTable.Name) +} + +func createReportsMetaTable(ctx context.Context, r runner) error { + return createSystemTable(ctx, r, sqlbase.ReportsMetaTable) +} + func runStmtAsRootWithRetry( ctx context.Context, r runner, opName string, stmt string, qargs ...interface{}, ) error { diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 664150b966cf..b8a64b6650a3 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/constraint" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/pkg/errors" @@ -504,7 +505,7 @@ func (a *Allocator) allocateTargetFromList( candidateReplicas []roachpb.ReplicaDescriptor, options scorerOptions, ) (*roachpb.StoreDescriptor, string) { - analyzedConstraints := analyzeConstraints( + analyzedConstraints := constraint.AnalyzeConstraints( ctx, a.storePool.getStoreDescriptor, candidateReplicas, zone) candidates := allocateCandidates( sl, analyzedConstraints, candidateReplicas, a.storePool.getLocalities(candidateReplicas), @@ -567,7 +568,7 @@ func (a Allocator) RemoveTarget( } sl, _, _ := a.storePool.getStoreListFromIDs(existingStoreIDs, roachpb.RangeID(0), storeFilterNone) - analyzedConstraints := analyzeConstraints( + analyzedConstraints := constraint.AnalyzeConstraints( ctx, a.storePool.getStoreDescriptor, existingReplicas, zone) options := a.scorerOptions() rankedCandidates := removeCandidates( @@ -661,7 +662,7 @@ func (a Allocator) RebalanceTarget( } } - analyzedConstraints := analyzeConstraints( + analyzedConstraints := constraint.AnalyzeConstraints( ctx, a.storePool.getStoreDescriptor, existingReplicas, zone) options := a.scorerOptions() results := rebalanceCandidates( @@ -1241,7 +1242,7 @@ func (a Allocator) preferredLeaseholders( if !ok { continue } - if subConstraintsCheck(storeDesc, preference.Constraints) { + if constraint.SubConstraintsCheck(storeDesc, preference.Constraints) { preferred = append(preferred, repl) } } diff --git a/pkg/storage/allocator_scorer.go b/pkg/storage/allocator_scorer.go index 582ecfb18795..e618e9a26ba3 100644 --- a/pkg/storage/allocator_scorer.go +++ b/pkg/storage/allocator_scorer.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/storage/constraint" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -380,7 +381,7 @@ func (cl candidateList) removeCandidate(c candidate) candidateList { // stores that meet the criteria are included in the list. func allocateCandidates( sl StoreList, - constraints analyzedConstraints, + constraints constraint.AnalyzedConstraints, existing []roachpb.ReplicaDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, options scorerOptions, @@ -434,7 +435,7 @@ func allocateCandidates( // marked as not valid, are in violation of a required criteria. func removeCandidates( sl StoreList, - constraints analyzedConstraints, + constraints constraint.AnalyzedConstraints, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, options scorerOptions, ) candidateList { @@ -492,7 +493,7 @@ type rebalanceOptions struct { func rebalanceCandidates( ctx context.Context, allStores StoreList, - constraints analyzedConstraints, + constraints constraint.AnalyzedConstraints, existingReplicas []roachpb.ReplicaDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, localityLookupFn func(roachpb.NodeID) string, @@ -880,62 +881,6 @@ func sameLocalityAndAttrs(s1, s2 roachpb.StoreDescriptor) bool { return true } -type analyzedConstraints struct { - constraints []config.Constraints - // True if the per-replica constraints don't fully cover all the desired - // replicas in the range (sum(constraints.NumReplicas) < zone.NumReplicas). - // In such cases, we allow replicas that don't match any of the per-replica - // constraints, but never mark them as necessary. - unconstrainedReplicas bool - // For each set of constraints in the above slice, track which StoreIDs - // satisfy them. This field is unused if there are no constraints. - satisfiedBy [][]roachpb.StoreID - // Maps from StoreID to the indices in the constraints slice of which - // constraints the store satisfies. This field is unused if there are no - // constraints. - satisfies map[roachpb.StoreID][]int -} - -// analyzeConstraints processes the zone config constraints that apply to a -// range along with the current replicas for a range, spitting back out -// information about which constraints are satisfied by which replicas and -// which replicas satisfy which constraints, aiding in allocation decisions. -func analyzeConstraints( - ctx context.Context, - getStoreDescFn func(roachpb.StoreID) (roachpb.StoreDescriptor, bool), - existing []roachpb.ReplicaDescriptor, - zone *config.ZoneConfig, -) analyzedConstraints { - result := analyzedConstraints{ - constraints: zone.Constraints, - } - - if len(zone.Constraints) > 0 { - result.satisfiedBy = make([][]roachpb.StoreID, len(zone.Constraints)) - result.satisfies = make(map[roachpb.StoreID][]int) - } - - var constrainedReplicas int32 - for i, subConstraints := range zone.Constraints { - constrainedReplicas += subConstraints.NumReplicas - for _, repl := range existing { - // If for some reason we don't have the store descriptor (which shouldn't - // happen once a node is hooked into gossip), trust that it's valid. This - // is a much more stable failure state than frantically moving everything - // off such a node. - store, ok := getStoreDescFn(repl.StoreID) - if !ok || subConstraintsCheck(store, subConstraints.Constraints) { - result.satisfiedBy[i] = append(result.satisfiedBy[i], store.StoreID) - result.satisfies[store.StoreID] = append(result.satisfies[store.StoreID], i) - } - } - } - if constrainedReplicas > 0 && constrainedReplicas < *zone.NumReplicas { - result.unconstrainedReplicas = true - } - return result -} - // allocateConstraintsCheck checks the potential allocation target store // against all the constraints. If it matches a constraint at all, it's valid. // If it matches a constraint that is not already fully satisfied by existing @@ -946,24 +891,26 @@ func analyzeConstraints( // constraints with NumReplicas set to 0. This is meant to be enforced in the // config package. func allocateConstraintsCheck( - store roachpb.StoreDescriptor, analyzed analyzedConstraints, + store roachpb.StoreDescriptor, analyzed constraint.AnalyzedConstraints, ) (valid bool, necessary bool) { // All stores are valid when there are no constraints. - if len(analyzed.constraints) == 0 { + if len(analyzed.Constraints) == 0 { return true, false } - for i, constraints := range analyzed.constraints { - if constraintsOK := subConstraintsCheck(store, constraints.Constraints); constraintsOK { + for i, constraints := range analyzed.Constraints { + if constraintsOK := constraint.SubConstraintsCheck( + store, constraints.Constraints, + ); constraintsOK { valid = true - matchingStores := analyzed.satisfiedBy[i] + matchingStores := analyzed.SatisfiedBy[i] if len(matchingStores) < int(constraints.NumReplicas) { return true, true } } } - if analyzed.unconstrainedReplicas { + if analyzed.UnconstrainedReplicas { valid = true } @@ -976,24 +923,24 @@ func allocateConstraintsCheck( // The difference between this and allocateConstraintsCheck is that this is to // be used on an existing replica of the range, not a potential addition. func removeConstraintsCheck( - store roachpb.StoreDescriptor, analyzed analyzedConstraints, + store roachpb.StoreDescriptor, analyzed constraint.AnalyzedConstraints, ) (valid bool, necessary bool) { // All stores are valid when there are no constraints. - if len(analyzed.constraints) == 0 { + if len(analyzed.Constraints) == 0 { return true, false } // The store satisfies none of the constraints, and the zone is not configured // to desire more replicas than constraints have been specified for. - if len(analyzed.satisfies[store.StoreID]) == 0 && !analyzed.unconstrainedReplicas { + if len(analyzed.Satisfies[store.StoreID]) == 0 && !analyzed.UnconstrainedReplicas { return false, false } // Check if the store matches a constraint that isn't overly satisfied. // If so, then keeping it around is necessary to ensure that constraint stays // fully satisfied. - for _, constraintIdx := range analyzed.satisfies[store.StoreID] { - if len(analyzed.satisfiedBy[constraintIdx]) <= int(analyzed.constraints[constraintIdx].NumReplicas) { + for _, constraintIdx := range analyzed.Satisfies[store.StoreID] { + if len(analyzed.SatisfiedBy[constraintIdx]) <= int(analyzed.Constraints[constraintIdx].NumReplicas) { return true, true } } @@ -1010,10 +957,12 @@ func removeConstraintsCheck( // will be necessary if fromStoreID (an existing replica) is removed from the // range. func rebalanceFromConstraintsCheck( - store roachpb.StoreDescriptor, fromStoreID roachpb.StoreID, analyzed analyzedConstraints, + store roachpb.StoreDescriptor, + fromStoreID roachpb.StoreID, + analyzed constraint.AnalyzedConstraints, ) (valid bool, necessary bool) { // All stores are valid when there are no constraints. - if len(analyzed.constraints) == 0 { + if len(analyzed.Constraints) == 0 { return true, false } @@ -1026,19 +975,21 @@ func rebalanceFromConstraintsCheck( // configured number of replicas for the range, or that there's just one set // of constraints with NumReplicas set to 0. This is meant to be enforced in // the config package. - for i, constraints := range analyzed.constraints { - if constraintsOK := subConstraintsCheck(store, constraints.Constraints); constraintsOK { + for i, constraints := range analyzed.Constraints { + if constraintsOK := constraint.SubConstraintsCheck( + store, constraints.Constraints, + ); constraintsOK { valid = true - matchingStores := analyzed.satisfiedBy[i] + matchingStores := analyzed.SatisfiedBy[i] if len(matchingStores) < int(constraints.NumReplicas) || (len(matchingStores) == int(constraints.NumReplicas) && - containsStore(analyzed.satisfiedBy[i], fromStoreID)) { + containsStore(analyzed.SatisfiedBy[i], fromStoreID)) { return true, true } } } - if analyzed.unconstrainedReplicas { + if analyzed.UnconstrainedReplicas { valid = true } @@ -1063,25 +1014,15 @@ func constraintsCheck(store roachpb.StoreDescriptor, constraints []config.Constr } for _, subConstraints := range constraints { - if constraintsOK := subConstraintsCheck(store, subConstraints.Constraints); constraintsOK { + if constraintsOK := constraint.SubConstraintsCheck( + store, subConstraints.Constraints, + ); constraintsOK { return true } } return false } -// subConstraintsCheck checks a store against a single set of constraints (out -// of the possibly numerous sets that apply to a range), returning true iff the -// store matches the constraints. -func subConstraintsCheck(store roachpb.StoreDescriptor, constraints []config.Constraint) bool { - for _, constraint := range constraints { - if !config.StoreMatchesConstraint(store, constraint) { - return false - } - } - return true -} - // rangeDiversityScore returns a value between 0 and 1 based on how diverse the // given range is. A higher score means the range is more diverse. // All below diversity-scoring methods should in theory be implemented by diff --git a/pkg/storage/allocator_scorer_test.go b/pkg/storage/allocator_scorer_test.go index f3422eae18f1..eb71659b8012 100644 --- a/pkg/storage/allocator_scorer_test.go +++ b/pkg/storage/allocator_scorer_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/constraint" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/gogo/protobuf/proto" "github.com/kr/pretty" @@ -902,7 +903,7 @@ func TestAllocateConstraintsCheck(t *testing.T) { Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } - analyzed := analyzeConstraints( + analyzed := constraint.AnalyzeConstraints( context.Background(), getTestStoreDesc, testStoreReplicas(tc.existing), zone) for _, s := range testStores { valid, necessary := allocateConstraintsCheck(s, analyzed) @@ -1035,7 +1036,8 @@ func TestRemoveConstraintsCheck(t *testing.T) { Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } - analyzed := analyzeConstraints(context.Background(), getTestStoreDesc, existing, zone) + analyzed := constraint.AnalyzeConstraints( + context.Background(), getTestStoreDesc, existing, zone) for storeID, expected := range tc.expected { valid, necessary := removeConstraintsCheck(testStores[storeID], analyzed) if e, a := expected.valid, valid; e != a { @@ -1191,7 +1193,7 @@ func TestShouldRebalanceDiversity(t *testing.T) { targets := rebalanceCandidates( context.Background(), filteredSL, - analyzedConstraints{}, + constraint.AnalyzedConstraints{}, replicas, existingNodeLocalities, func(nodeID roachpb.NodeID) string { diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index f2b1d9e8f9c4..58a326f359cc 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/constraint" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -2557,7 +2558,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { } } zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} - analyzed := analyzeConstraints( + analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, zone) candidates := allocateCandidates( sl, @@ -2780,7 +2781,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { } } zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} - analyzed := analyzeConstraints( + analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, zone) candidates := removeCandidates( sl, @@ -3574,7 +3575,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } - analyzed := analyzeConstraints( + analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, zone) results := rebalanceCandidates( context.Background(), diff --git a/pkg/storage/constraint/analyzer.go b/pkg/storage/constraint/analyzer.go new file mode 100644 index 000000000000..8a6445e26616 --- /dev/null +++ b/pkg/storage/constraint/analyzer.go @@ -0,0 +1,90 @@ +// Copyright 2019 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 constraint + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// AnalyzedConstraints comment !!! +type AnalyzedConstraints struct { + Constraints []config.Constraints + // True if the per-replica constraints don't fully cover all the desired + // replicas in the range (sum(constraints.NumReplicas) < zone.NumReplicas). + // In such cases, we allow replicas that don't match any of the per-replica + // constraints, but never mark them as necessary. + UnconstrainedReplicas bool + // For each set of constraints in the above slice, track which StoreIDs + // satisfy them. This field is unused if there are no constraints. + SatisfiedBy [][]roachpb.StoreID + // Maps from StoreID to the indices in the constraints slice of which + // constraints the store satisfies. This field is unused if there are no + // constraints. + Satisfies map[roachpb.StoreID][]int +} + +// AnalyzeConstraints processes the zone config constraints that apply to a +// range along with the current replicas for a range, spitting back out +// information about which constraints are satisfied by which replicas and +// which replicas satisfy which constraints, aiding in allocation decisions. +func AnalyzeConstraints( + ctx context.Context, + getStoreDescFn func(roachpb.StoreID) (roachpb.StoreDescriptor, bool), + existing []roachpb.ReplicaDescriptor, + zone *config.ZoneConfig, +) AnalyzedConstraints { + result := AnalyzedConstraints{ + Constraints: zone.Constraints, + } + + if len(zone.Constraints) > 0 { + result.SatisfiedBy = make([][]roachpb.StoreID, len(zone.Constraints)) + result.Satisfies = make(map[roachpb.StoreID][]int) + } + + var constrainedReplicas int32 + for i, subConstraints := range zone.Constraints { + constrainedReplicas += subConstraints.NumReplicas + for _, repl := range existing { + // If for some reason we don't have the store descriptor (which shouldn't + // happen once a node is hooked into gossip), trust that it's valid. This + // is a much more stable failure state than frantically moving everything + // off such a node. + store, ok := getStoreDescFn(repl.StoreID) + if !ok || SubConstraintsCheck(store, subConstraints.Constraints) { + result.SatisfiedBy[i] = append(result.SatisfiedBy[i], store.StoreID) + result.Satisfies[store.StoreID] = append(result.Satisfies[store.StoreID], i) + } + } + } + if constrainedReplicas > 0 && constrainedReplicas < *zone.NumReplicas { + result.UnconstrainedReplicas = true + } + return result +} + +// SubConstraintsCheck checks a store against a single set of constraints (out +// of the possibly numerous sets that apply to a range), returning true iff the +// store matches the constraints. +func SubConstraintsCheck(store roachpb.StoreDescriptor, constraints []config.Constraint) bool { + for _, constraint := range constraints { + // StoreSatisfiesConstraint returns whether a store matches the given constraint. + hasConstraint := config.StoreMatchesConstraint(store, constraint) + if (constraint.Type == config.Constraint_REQUIRED && !hasConstraint) || + (constraint.Type == config.Constraint_PROHIBITED && hasConstraint) { + return false + } + } + return true +} diff --git a/pkg/storage/reports/constraint_report.go b/pkg/storage/reports/constraint_report.go new file mode 100644 index 000000000000..93edc77d9767 --- /dev/null +++ b/pkg/storage/reports/constraint_report.go @@ -0,0 +1,447 @@ +// Copyright 2019 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 reports + +import ( + "context" + "fmt" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/pkg/errors" +) + +// replicationConstraintsReportID is the id of the row in the system. +// reports_meta table corresponding to the constraints conformance report (i.e. +// the system.replicationConstraintsReportID table). +const replicationConstraintsReportID = 1 + +// ConstraintReport contains information about the constraint conformance for +// the cluster's data. +type ConstraintReport map[ConstraintStatusKey]ConstraintStatus + +// ReplicationConstraintStatsReportSaver manages the content and the saving of the report. +type replicationConstraintStatsReportSaver struct { + previousVersion ConstraintReport + lastGenerated time.Time + lastUpdatedRowCount int + + constraints ConstraintReport +} + +// makeReplicationConstraintStatusReportSaver creates a new report saver. +func makeReplicationConstraintStatusReportSaver() replicationConstraintStatsReportSaver { + return replicationConstraintStatsReportSaver{ + constraints: ConstraintReport{}, + } +} + +// resetReport resets the report to an empty state. +func (r *replicationConstraintStatsReportSaver) resetReport() { + r.constraints = ConstraintReport{} +} + +// LastUpdatedRowCount is the count of the rows that were touched during the last save. +func (r *replicationConstraintStatsReportSaver) LastUpdatedRowCount() int { + return r.lastUpdatedRowCount +} + +// ConstraintStatus is the leaf in the constraintReport. +type ConstraintStatus struct { + FailRangeCount int +} + +// ConstraintType indicates what type of constraint is an entry in the +// constraint conformance report talking about. +type ConstraintType string + +const ( + // Constraint means that the entry refers to a constraint (i.e. a member of + // the constraints field in a zone config). + Constraint ConstraintType = "constraint" + // TODO(andrei): add leaseholder preference +) + +// Less compares two ConstraintTypes. +func (t ConstraintType) Less(other ConstraintType) bool { + return -1 == strings.Compare(string(t), string(other)) +} + +// ConstraintRepr is a string representation of a constraint. +type ConstraintRepr string + +// Less compares two ConstraintReprs. +func (c ConstraintRepr) Less(other ConstraintRepr) bool { + return -1 == strings.Compare(string(c), string(other)) +} + +// ConstraintStatusKey represents the key in the ConstraintReport. +type ConstraintStatusKey struct { + ZoneKey + ViolationType ConstraintType + Constraint ConstraintRepr +} + +func (k ConstraintStatusKey) String() string { + return fmt.Sprintf("zone:%s type:%s constraint:%s", k.ZoneKey, k.ViolationType, k.Constraint) +} + +// Less compares two ConstraintStatusKeys. +func (k ConstraintStatusKey) Less(other ConstraintStatusKey) bool { + if k.ZoneKey.Less(other.ZoneKey) { + return true + } + if other.ZoneKey.Less(k.ZoneKey) { + return false + } + if k.ViolationType.Less(other.ViolationType) { + return true + } + if other.ViolationType.Less(k.ViolationType) { + return true + } + return k.Constraint.Less(other.Constraint) +} + +// MakeConstraintRepr creates a canonical string representation for a +// constraint. The constraint is identified by the group it belongs to and the +// index within the group. +func MakeConstraintRepr(constraintGroup config.Constraints, constraintIdx int) ConstraintRepr { + cstr := constraintGroup.Constraints[constraintIdx].String() + if constraintGroup.NumReplicas == 0 { + return ConstraintRepr(cstr) + } + return ConstraintRepr(fmt.Sprintf("%q:%d", cstr, constraintGroup.NumReplicas)) +} + +// AddViolation add a constraint that is being violated for a given range. Each call +// will increase the number of ranges that failed. +func (r *replicationConstraintStatsReportSaver) AddViolation( + z ZoneKey, t ConstraintType, c ConstraintRepr, +) { + k := ConstraintStatusKey{ + ZoneKey: z, + ViolationType: t, + Constraint: c, + } + if _, ok := r.constraints[k]; !ok { + r.constraints[k] = ConstraintStatus{} + } + cRep := r.constraints[k] + cRep.FailRangeCount++ + r.constraints[k] = cRep +} + +// EnsureEntry us used to add an entry to the report even if there is no violation. +func (r *replicationConstraintStatsReportSaver) EnsureEntry( + z ZoneKey, t ConstraintType, c ConstraintRepr, +) { + k := ConstraintStatusKey{ + ZoneKey: z, + ViolationType: t, + Constraint: c, + } + if _, ok := r.constraints[k]; !ok { + r.constraints[k] = ConstraintStatus{} + } +} + +func (r *replicationConstraintStatsReportSaver) ensureEntries( + key ZoneKey, zone *config.ZoneConfig, +) { + for _, group := range zone.Constraints { + for i := range group.Constraints { + r.EnsureEntry(key, Constraint, MakeConstraintRepr(group, i)) + } + } + for i, sz := range zone.Subzones { + szKey := ZoneKey{ZoneID: key.ZoneID, SubzoneID: SubzoneIDFromIndex(i)} + r.ensureEntries(szKey, &sz.Config) + } +} + +func (r *replicationConstraintStatsReportSaver) loadPreviousVersion( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, +) error { + // The data for the previous save needs to be loaded if: + // - this is the first time that we call this method and lastUpdatedAt has never been set + // - in case that the lastUpdatedAt is set but is different than the timestamp in reports_meta + // this indicates that some other worker wrote after we did the write. + if !r.lastGenerated.IsZero() { + // check to see if the last timestamp for the update matches the local one. + row, err := ex.QueryRow( + ctx, + "get-previous-timestamp", + txn, + "select generated from system.reports_meta where id = $1", + replicationConstraintsReportID, + ) + if err != nil { + return err + } + + // if the row is nil then this is the first time we are running and the reload is needed. + if row != nil { + generated, ok := row[0].(*tree.DTimestamp) + if !ok { + return errors.Errorf("Expected to get time from system.reports_meta but got %+v", row) + } + if generated.Time == r.lastGenerated { + // No need to reload. + return nil + } + } + } + const prevViolations = "select zone_id, subzone_id, type, config, " + + "violating_ranges from system.replication_constraint_stats" + rows, err := ex.Query( + ctx, "get-previous-replication-constraint-stats", txn, prevViolations, + ) + if err != nil { + return err + } + + r.previousVersion = make(ConstraintReport, len(rows)) + for _, row := range rows { + key := ConstraintStatusKey{} + key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.SubzoneID = SubzoneID((*row[1].(*tree.DInt))) + key.ViolationType = (ConstraintType)(*row[2].(*tree.DString)) + key.Constraint = (ConstraintRepr)(*row[3].(*tree.DString)) + r.previousVersion[key] = ConstraintStatus{(int)(*row[4].(*tree.DInt))} + } + + return nil +} + +func (r *replicationConstraintStatsReportSaver) updatePreviousVersion() { + r.previousVersion = r.constraints + r.constraints = make(ConstraintReport, len(r.previousVersion)) +} + +func (r *replicationConstraintStatsReportSaver) updateTimestamp( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, reportTS time.Time, +) error { + if !r.lastGenerated.IsZero() && reportTS == r.lastGenerated { + return errors.Errorf( + "The new time %s is the same as the time of the last update %s", + reportTS.String(), + r.lastGenerated.String(), + ) + } + + _, err := ex.Exec( + ctx, + "timestamp-upsert-replication-constraint-stats", + txn, + "upsert into system.reports_meta(id, generated) values($1, $2)", + replicationConstraintsReportID, + reportTS, + ) + return err +} + +// Save the report. +// +// reportTS is the time that will be set in the updated_at column for every row. +func (r *replicationConstraintStatsReportSaver) Save( + ctx context.Context, reportTS time.Time, db *client.DB, ex sqlutil.InternalExecutor, +) error { + r.lastUpdatedRowCount = 0 + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + err := r.loadPreviousVersion(ctx, ex, txn) + if err != nil { + return err + } + + err = r.updateTimestamp(ctx, ex, txn, reportTS) + if err != nil { + return err + } + + for k, zoneCons := range r.constraints { + if err := r.upsertConstraintStatus( + ctx, reportTS, txn, k, zoneCons.FailRangeCount, db, ex, + ); err != nil { + return err + } + } + + for key := range r.previousVersion { + if _, ok := r.constraints[key]; !ok { + _, err := ex.Exec( + ctx, + "delete-old-replication-constraint-stats", + txn, + "delete from system.replication_constraint_stats "+ + "where zone_id = $1 and subzone_id = $2 and type = $3 and config = $4", + key.ZoneID, + key.SubzoneID, + key.ViolationType, + key.Constraint, + ) + + if err != nil { + return err + } + r.lastUpdatedRowCount++ + } + } + + return nil + }); err != nil { + return err + } + + r.lastGenerated = reportTS + r.updatePreviousVersion() + + return nil +} + +// upsertConstraintStatus upserts a row into system.replication_constraint_stats. +// +// existing is used to decide is this is a new violation. +func (r *replicationConstraintStatsReportSaver) upsertConstraintStatus( + ctx context.Context, + reportTS time.Time, + txn *client.Txn, + key ConstraintStatusKey, + violationCount int, + db *client.DB, + ex sqlutil.InternalExecutor, +) error { + var err error + previousStatus, hasOldVersion := r.previousVersion[key] + if hasOldVersion && previousStatus.FailRangeCount == violationCount { + // No change in the status so no update. + return nil + } else if violationCount != 0 { + if previousStatus.FailRangeCount != 0 { + // Updating an old violation. No need to update the start timestamp. + _, err = ex.Exec( + ctx, "upsert-replication-constraint-stat", txn, + "upsert into system.replication_constraint_stats(report_id, zone_id, subzone_id, type, "+ + "config, violating_ranges) values($1, $2, $3, $4, $5, $6)", + replicationConstraintsReportID, + key.ZoneID, key.SubzoneID, key.ViolationType, key.Constraint, violationCount, + ) + } else if previousStatus.FailRangeCount == 0 { + // New violation detected. Need to update the start timestamp. + _, err = ex.Exec( + ctx, "upsert-replication-constraint-stat", txn, + "upsert into system.replication_constraint_stats(report_id, zone_id, subzone_id, type, "+ + "config, violating_ranges, violation_start) values($1, $2, $3, $4, $5, $6, $7)", + replicationConstraintsReportID, + key.ZoneID, key.SubzoneID, key.ViolationType, key.Constraint, violationCount, reportTS, + ) + } + } else { + // Need to set the violation start to null as there was an violation that doesn't exist anymore. + _, err = ex.Exec( + ctx, "upsert-replication-constraint-stat", txn, + "upsert into system.replication_constraint_stats(report_id, zone_id, subzone_id, type, config, "+ + "violating_ranges, violation_start) values($1, $2, $3, $4, $5, $6, null)", + replicationConstraintsReportID, + key.ZoneID, key.SubzoneID, key.ViolationType, key.Constraint, violationCount, + ) + } + + if err != nil { + return err + } + + r.lastUpdatedRowCount++ + return nil +} + +// constraintConformanceVisitor is a visitor that, when passed to visitRanges(), +// computes the constraint conformance report (i.e. the +// system.replication_constraint_stats table). +type constraintConformanceVisitor struct { + cfg *config.SystemConfig + storeResolver StoreResolver + + report *replicationConstraintStatsReportSaver +} + +var _ rangeVisitor = &constraintConformanceVisitor{} + +func makeConstraintConformanceVisitor( + ctx context.Context, + cfg *config.SystemConfig, + storeResolver StoreResolver, + saver *replicationConstraintStatsReportSaver, +) constraintConformanceVisitor { + v := constraintConformanceVisitor{ + cfg: cfg, + storeResolver: storeResolver, + report: saver, + } + v.reset(ctx) + return v +} + +// reset is part of the rangeVisitor interface. +func (v *constraintConformanceVisitor) reset(ctx context.Context) { + v.report.resetReport() + + // Iterate through all the zone configs to create report entries for all the + // zones that have constraints. Otherwise, just iterating through the ranges + // wouldn't create entries for constraints that aren't violated, and + // definitely not for zones that don't apply to any ranges. + maxObjectID, err := v.cfg.GetLargestObjectID(0 /* maxID - return the largest ID in the config */) + if err != nil { + log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) + } + for i := uint32(1); i <= maxObjectID; i++ { + zone, err := getZoneByID(i, v.cfg) + if err != nil { + log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) + } + if zone == nil { + continue + } + v.report.ensureEntries(MakeZoneKey(i, NoSubzone), zone) + } +} + +// constraintConformanceVisitor is part of the rangeVisitor interface. +func (v *constraintConformanceVisitor) visit(ctx context.Context, r roachpb.RangeDescriptor) { + storeDescs := v.storeResolver(r) + + // Find the applicable constraints, which may be inherited. + var constraints []config.Constraints + var zKey ZoneKey + err := visitZones(ctx, r, v.cfg, + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { + if zone.Constraints == nil { + return false + } + constraints = zone.Constraints + zKey = key + return true + }) + if err != nil { + log.Fatalf(ctx, "unexpected error visiting zones: %s", err) + } + + violated := processRange(ctx, storeDescs, constraints) + for _, c := range violated { + v.report.AddViolation(zKey, Constraint, c) + } +} diff --git a/pkg/storage/reports/constraint_report_test.go b/pkg/storage/reports/constraint_report_test.go new file mode 100644 index 000000000000..e1866189f1c1 --- /dev/null +++ b/pkg/storage/reports/constraint_report_test.go @@ -0,0 +1,167 @@ +// Copyright 2019 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 reports + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestConstraintReport(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + // This test uses the cluster as a recipient for a report saved from outside + // the cluster. We disable the cluster's own production of reports so that it + // doesn't interfere with the test. + ReporterInterval.Override(&st.SV, 0) + s, _, db := serverutils.StartServer(t, base.TestServerArgs{Settings: st}) + con := s.InternalExecutor().(sqlutil.InternalExecutor) + defer s.Stopper().Stop(ctx) + + // Verify that tables are empty. + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{}) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{}) + + // Add several replication constraint statuses. + r := makeReplicationConstraintStatusReportSaver() + r.EnsureEntry(MakeZoneKey(1, 3), "constraint", "+country=CH") + r.AddViolation(MakeZoneKey(2, 3), "constraint", "+country=CH") + r.EnsureEntry(MakeZoneKey(2, 3), "constraint", "+country=CH") + r.AddViolation(MakeZoneKey(5, 6), "constraint", "+ssd") + r.AddViolation(MakeZoneKey(5, 6), "constraint", "+ssd") + r.AddViolation(MakeZoneKey(7, 8), "constraint", "+dc=west") + r.EnsureEntry(MakeZoneKey(7, 8), "constraint", "+dc=east") + r.EnsureEntry(MakeZoneKey(7, 8), "constraint", "+dc=east") + r.AddViolation(MakeZoneKey(8, 9), "constraint", "+dc=west") + r.EnsureEntry(MakeZoneKey(8, 9), "constraint", "+dc=east") + + time1 := time.Date(2001, 1, 1, 10, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time1, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{ + {"1", "3", "'constraint'", "'+country=CH'", "1", "NULL", "0"}, + {"2", "3", "'constraint'", "'+country=CH'", "1", "'2001-01-01 10:00:00+00:00'", "1"}, + {"5", "6", "'constraint'", "'+ssd'", "1", "'2001-01-01 10:00:00+00:00'", "2"}, + {"7", "8", "'constraint'", "'+dc=west'", "1", "'2001-01-01 10:00:00+00:00'", "1"}, + {"7", "8", "'constraint'", "'+dc=east'", "1", "NULL", "0"}, + {"8", "9", "'constraint'", "'+dc=west'", "1", "'2001-01-01 10:00:00+00:00'", "1"}, + {"8", "9", "'constraint'", "'+dc=east'", "1", "NULL", "0"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"1", "'2001-01-01 10:00:00+00:00'"}, + }) + require.Equal(t, 7, r.LastUpdatedRowCount()) + + // Add new set of replication constraint statuses to the existing report and verify the old ones are deleted. + r.AddViolation(MakeZoneKey(1, 3), "constraint", "+country=CH") + r.EnsureEntry(MakeZoneKey(5, 6), "constraint", "+ssd") + r.AddViolation(MakeZoneKey(6, 8), "constraint", "+dc=east") + r.EnsureEntry(MakeZoneKey(6, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(7, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(7, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(8, 9), "constraint", "+dc=west") + r.EnsureEntry(MakeZoneKey(8, 9), "constraint", "+dc=east") + + time2 := time.Date(2001, 1, 1, 11, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time2, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{ + // Wasn't violated before - is violated now. + {"1", "3", "'constraint'", "'+country=CH'", "1", "'2001-01-01 11:00:00+00:00'", "1"}, + // Was violated before - isn't violated now. + {"5", "6", "'constraint'", "'+ssd'", "1", "NULL", "0"}, + // Didn't exist before - new for this run and violated. + {"6", "8", "'constraint'", "'+dc=east'", "1", "'2001-01-01 11:00:00+00:00'", "1"}, + // Didn't exist before - new for this run and not violated. + {"6", "8", "'constraint'", "'+dc=west'", "1", "NULL", "0"}, + // Was violated before - and it still is but the range count changed. + {"7", "8", "'constraint'", "'+dc=west'", "1", "'2001-01-01 10:00:00+00:00'", "2"}, + // Was violated before - and it still is but the range count didn't change. + {"8", "9", "'constraint'", "'+dc=west'", "1", "'2001-01-01 10:00:00+00:00'", "1"}, + // Wasn't violated before - and is still not violated. + {"8", "9", "'constraint'", "'+dc=east'", "1", "NULL", "0"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"1", "'2001-01-01 11:00:00+00:00'"}, + }) + require.Equal(t, 7, r.LastUpdatedRowCount()) + + time3 := time.Date(2001, 1, 1, 11, 30, 0, 0, time.UTC) + // If some other server takes over and does an update. + rows, err := con.Exec(ctx, "another-updater", nil, "update system.reports_meta set generated=$1 where id=1", time3) + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "update system.replication_constraint_stats "+ + "set violation_start=null, violating_ranges=0 where zone_id=1 and subzone_id=3") + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "update system.replication_constraint_stats "+ + "set violation_start=$1, violating_ranges=5 where zone_id=5 and subzone_id=6", time3) + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "delete from system.replication_constraint_stats "+ + "where zone_id=7 and subzone_id=8") + require.NoError(t, err) + require.Equal(t, 1, rows) + + // Add new set of replication constraint statuses to the existing report and verify the everything is good. + r.AddViolation(MakeZoneKey(1, 3), "constraint", "+country=CH") + r.EnsureEntry(MakeZoneKey(5, 6), "constraint", "+ssd") + r.AddViolation(MakeZoneKey(6, 8), "constraint", "+dc=east") + r.EnsureEntry(MakeZoneKey(6, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(7, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(7, 8), "constraint", "+dc=west") + r.AddViolation(MakeZoneKey(8, 9), "constraint", "+dc=west") + r.EnsureEntry(MakeZoneKey(8, 9), "constraint", "+dc=east") + + time4 := time.Date(2001, 1, 1, 12, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time4, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{ + {"1", "3", "'constraint'", "'+country=CH'", "1", "'2001-01-01 12:00:00+00:00'", "1"}, + {"5", "6", "'constraint'", "'+ssd'", "1", "NULL", "0"}, + {"6", "8", "'constraint'", "'+dc=east'", "1", "'2001-01-01 11:00:00+00:00'", "1"}, + {"6", "8", "'constraint'", "'+dc=west'", "1", "NULL", "0"}, + {"7", "8", "'constraint'", "'+dc=west'", "1", "'2001-01-01 12:00:00+00:00'", "2"}, + {"8", "9", "'constraint'", "'+dc=west'", "1", "'2001-01-01 10:00:00+00:00'", "1"}, + {"8", "9", "'constraint'", "'+dc=east'", "1", "NULL", "0"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"1", "'2001-01-01 12:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + // A brand new report (after restart for example) - still works. + // Add several replication constraint statuses. + r = makeReplicationConstraintStatusReportSaver() + r.AddViolation(MakeZoneKey(1, 3), "constraint", "+country=CH") + + time5 := time.Date(2001, 1, 1, 12, 30, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time5, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{ + {"1", "3", "'constraint'", "'+country=CH'", "1", "'2001-01-01 12:00:00+00:00'", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"1", "'2001-01-01 12:30:00+00:00'"}, + }) + require.Equal(t, 6, r.LastUpdatedRowCount()) +} diff --git a/pkg/storage/reports/locality_report.go b/pkg/storage/reports/locality_report.go new file mode 100644 index 000000000000..e2e5458e392a --- /dev/null +++ b/pkg/storage/reports/locality_report.go @@ -0,0 +1,375 @@ +// Copyright 2019 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 reports + +import ( + "context" + "fmt" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// criticalLocalitiesReportID is the id of the row in the system. reports_meta +// table corresponding to the critical localities report (i.e. the +// system.replication_critical_localities table). +const criticalLocalitiesReportID = 2 + +type localityKey struct { + ZoneKey + locality LocalityRepr +} + +// LocalityRepr is a representation of a locality. +type LocalityRepr string + +type localityStatus struct { + atRiskRanges int32 +} + +// LocalityReport stores the range status information for each locality and +// applicable zone. +type LocalityReport map[localityKey]localityStatus + +// ReplicationCriticalLocalitiesReportSaver manages the content and the saving +// of the report. +type replicationCriticalLocalitiesReportSaver struct { + localities LocalityReport + previousVersion LocalityReport + lastGenerated time.Time + lastUpdatedRowCount int +} + +// makeReplicationCriticalLocalitiesReportSaver creates a new report saver. +func makeReplicationCriticalLocalitiesReportSaver() replicationCriticalLocalitiesReportSaver { + return replicationCriticalLocalitiesReportSaver{ + localities: LocalityReport{}, + } +} + +// resetReport resets the report to an empty state. +func (r *replicationCriticalLocalitiesReportSaver) resetReport() { + r.localities = LocalityReport{} +} + +// LastUpdatedRowCount is the count of the rows that were touched during the last save. +func (r *replicationCriticalLocalitiesReportSaver) LastUpdatedRowCount() int { + return r.lastUpdatedRowCount +} + +// AddCriticalLocality will add locality to the list of the critical localities. +func (r *replicationCriticalLocalitiesReportSaver) AddCriticalLocality( + zKey ZoneKey, loc LocalityRepr, +) { + lKey := localityKey{ + ZoneKey: zKey, + locality: loc, + } + if _, ok := r.localities[lKey]; !ok { + r.localities[lKey] = localityStatus{} + } + lStat := r.localities[lKey] + lStat.atRiskRanges++ + r.localities[lKey] = lStat +} + +func (r *replicationCriticalLocalitiesReportSaver) loadPreviousVersion( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, +) error { + // The data for the previous save needs to be loaded if: + // - this is the first time that we call this method and lastUpdatedAt has never been set + // - in case that the lastUpdatedAt is set but is different than the timestamp in reports_meta + // this indicates that some other worker wrote after we did the write. + if !r.lastGenerated.IsZero() { + // check to see if the last timestamp for the update matches the local one. + row, err := ex.QueryRow( + ctx, + "get-previous-timestamp", + txn, + "select generated from system.reports_meta where id = $1", + criticalLocalitiesReportID, + ) + if err != nil { + return err + } + + // if the row is nil then this is the first time we are running and the reload is needed. + if row != nil { + generated, ok := row[0].(*tree.DTimestamp) + if !ok { + return errors.Errorf("Expected to get time from system.reports_meta but got %+v", row) + } + if generated.Time == r.lastGenerated { + // No need to reload. + return nil + } + } + } + const prevViolations = "select zone_id, subzone_id, locality, at_risk_ranges " + + "from system.replication_critical_localities" + rows, err := ex.Query( + ctx, "get-previous-replication-critical-localities", txn, prevViolations, + ) + if err != nil { + return err + } + + r.previousVersion = make(LocalityReport, len(rows)) + for _, row := range rows { + key := localityKey{} + key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.SubzoneID = SubzoneID(*row[1].(*tree.DInt)) + key.locality = (LocalityRepr)(*row[2].(*tree.DString)) + r.previousVersion[key] = localityStatus{(int32)(*row[3].(*tree.DInt))} + } + + return nil +} + +func (r *replicationCriticalLocalitiesReportSaver) updatePreviousVersion() { + r.previousVersion = r.localities + r.localities = make(LocalityReport, len(r.previousVersion)) +} + +func (r *replicationCriticalLocalitiesReportSaver) updateTimestamp( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, reportTS time.Time, +) error { + if !r.lastGenerated.IsZero() && reportTS == r.lastGenerated { + return errors.Errorf( + "The new time %s is the same as the time of the last update %s", + reportTS.String(), + r.lastGenerated.String(), + ) + } + + _, err := ex.Exec( + ctx, + "timestamp-upsert-replication-critical-localities", + txn, + "upsert into system.reports_meta(id, generated) values($1, $2)", + criticalLocalitiesReportID, + reportTS, + ) + return err +} + +// Save the report. +// +// reportTS is the time that will be set in the updated_at column for every row. +func (r *replicationCriticalLocalitiesReportSaver) Save( + ctx context.Context, reportTS time.Time, db *client.DB, ex sqlutil.InternalExecutor, +) error { + r.lastUpdatedRowCount = 0 + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + err := r.loadPreviousVersion(ctx, ex, txn) + if err != nil { + return err + } + + err = r.updateTimestamp(ctx, ex, txn, reportTS) + if err != nil { + return err + } + + for key, status := range r.localities { + if err := r.upsertLocality( + ctx, reportTS, txn, key, status, db, ex, + ); err != nil { + return err + } + } + + for key := range r.previousVersion { + if _, ok := r.localities[key]; !ok { + _, err := ex.Exec( + ctx, + "delete-old-replication-critical-localities", + txn, + "delete from system.replication_critical_localities "+ + "where zone_id = $1 and subzone_id = $2 and locality = $3", + key.ZoneID, + key.SubzoneID, + key.locality, + ) + + if err != nil { + return err + } + r.lastUpdatedRowCount++ + } + } + + return nil + }); err != nil { + return err + } + + r.lastGenerated = reportTS + r.updatePreviousVersion() + + return nil +} + +// upsertLocality upserts a row into system.replication_critical_localities. +// +// existing is used to decide is this is a new violation. +func (r *replicationCriticalLocalitiesReportSaver) upsertLocality( + ctx context.Context, + reportTS time.Time, + txn *client.Txn, + key localityKey, + status localityStatus, + db *client.DB, + ex sqlutil.InternalExecutor, +) error { + var err error + previousStatus, hasOldVersion := r.previousVersion[key] + if hasOldVersion && previousStatus.atRiskRanges == status.atRiskRanges { + // No change in the status so no update. + return nil + } + + // Updating an old row. + _, err = ex.Exec( + ctx, "upsert-replication-critical-localities", txn, + "upsert into system.replication_critical_localities(report_id, zone_id, subzone_id, "+ + "locality, at_risk_ranges) values($1, $2, $3, $4, $5)", + criticalLocalitiesReportID, + key.ZoneID, key.SubzoneID, key.locality, status.atRiskRanges, + ) + + if err != nil { + return err + } + + r.lastUpdatedRowCount++ + return nil +} + +// criticalLocalitiesVisitor is a visitor that, when passed to visitRanges(), builds +// a LocalityReport. +type criticalLocalitiesVisitor struct { + localityConstraints []config.Constraints + cfg *config.SystemConfig + storeResolver StoreResolver + nodeChecker nodeChecker + + report *replicationCriticalLocalitiesReportSaver +} + +var _ rangeVisitor = &criticalLocalitiesVisitor{} + +func makeLocalityStatsVisitor( + ctx context.Context, + localityConstraints []config.Constraints, + cfg *config.SystemConfig, + storeResolver StoreResolver, + nodeChecker nodeChecker, + saver *replicationCriticalLocalitiesReportSaver, +) criticalLocalitiesVisitor { + v := criticalLocalitiesVisitor{ + localityConstraints: localityConstraints, + cfg: cfg, + storeResolver: storeResolver, + nodeChecker: nodeChecker, + report: saver, + } + return v +} + +// reset is part of the rangeVisitor interface. +func (v *criticalLocalitiesVisitor) reset(ctx context.Context) { + v.report.resetReport() +} + +// visit is part of the rangeVisitor interface. +func (v *criticalLocalitiesVisitor) visit(ctx context.Context, r roachpb.RangeDescriptor) { + stores := v.storeResolver(r) + for _, c := range v.localityConstraints { + processLocalityForRange(ctx, r, v.report, &c, v.cfg, v.nodeChecker, stores) + } +} + +// processLocalityForRange checks a single locality constraint against a +// range with replicas in each of the stores given, contributing to rep. +func processLocalityForRange( + ctx context.Context, + r roachpb.RangeDescriptor, + rep *replicationCriticalLocalitiesReportSaver, + c *config.Constraints, + cfg *config.SystemConfig, + nodeChecker nodeChecker, + storeDescs []roachpb.StoreDescriptor, +) { + // Get the zone. + var zKey ZoneKey + if err := visitZones(ctx, r, cfg, + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { + if zone.IsSubzonePlaceholder() { + return false + } + zKey = key + return true + }); err != nil { + log.Fatalf(ctx, "unexpected error visiting zones: %s", err) + } + + // Compute the required quorum and the number of live nodes. If the number of live nodes gets lower + // than the required quorum then the range is already unavailable. + quorumCount := len(r.Replicas().Voters())/2 + 1 + liveNodeCount := len(storeDescs) + for _, storeDesc := range storeDescs { + isStoreLive := nodeChecker(storeDesc.Node.NodeID) + if !isStoreLive { + if liveNodeCount >= quorumCount { + liveNodeCount-- + if liveNodeCount < quorumCount { + break + } + } + } + } + + cstrs := make([]string, 0, len(c.Constraints)) + for _, con := range c.Constraints { + cstrs = append(cstrs, fmt.Sprintf("%s=%s", con.Key, con.Value)) + } + loc := LocalityRepr(strings.Join(cstrs, ",")) + + passCount := 0 + for _, storeDesc := range storeDescs { + storeHasConstraint := true + for _, constraint := range c.Constraints { + // For required constraints - consider unavailable nodes as not matching. + if !config.StoreMatchesConstraint(storeDesc, constraint) { + storeHasConstraint = false + break + } + } + + if storeHasConstraint && nodeChecker(storeDesc.Node.NodeID) { + passCount++ + } + } + + // If the live nodes outside of the given locality are not enough to + // form quorum then this locality is critical. + if quorumCount > liveNodeCount-passCount { + rep.AddCriticalLocality(zKey, loc) + } +} diff --git a/pkg/storage/reports/locality_report_test.go b/pkg/storage/reports/locality_report_test.go new file mode 100644 index 000000000000..8b43660fdb08 --- /dev/null +++ b/pkg/storage/reports/locality_report_test.go @@ -0,0 +1,154 @@ +// Copyright 2019 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 reports + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestLocalityReport(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + // This test uses the cluster as a recipient for a report saved from outside + // the cluster. We disable the cluster's own production of reports so that it + // doesn't interfere with the test. + ReporterInterval.Override(&st.SV, 0) + s, _, db := serverutils.StartServer(t, base.TestServerArgs{Settings: st}) + con := s.InternalExecutor().(sqlutil.InternalExecutor) + defer s.Stopper().Stop(ctx) + + // Verify that tables are empty. + require.ElementsMatch(t, TableData(ctx, "system.replication_constraint_stats", con), [][]string{}) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{}) + + // Add several localities and verify the result + r := makeReplicationCriticalLocalitiesReportSaver() + r.AddCriticalLocality(MakeZoneKey(1, 3), "region=US") + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + + time1 := time.Date(2001, 1, 1, 10, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time1, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_critical_localities", con), [][]string{ + {"1", "3", "'region=US'", "2", "1"}, + {"5", "6", "'dc=A'", "2", "1"}, + {"7", "8", "'dc=B'", "2", "2"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"2", "'2001-01-01 10:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + // Add new set of localities and verify the old ones are deleted + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + r.AddCriticalLocality(MakeZoneKey(15, 6), "dc=A") + + time2 := time.Date(2001, 1, 1, 11, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time2, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_critical_localities", con), [][]string{ + {"5", "6", "'dc=A'", "2", "2"}, + {"7", "8", "'dc=B'", "2", "2"}, + {"15", "6", "'dc=A'", "2", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"2", "'2001-01-01 11:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + time3 := time.Date(2001, 1, 1, 11, 30, 0, 0, time.UTC) + // If some other server takes over and does an update. + rows, err := con.Exec(ctx, "another-updater", nil, "update system.reports_meta set generated=$1 where id=2", time3) + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "update system.replication_critical_localities "+ + "set at_risk_ranges=3 where zone_id=5 and subzone_id=6 and locality='dc=A'") + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "delete from system.replication_critical_localities "+ + "where zone_id=7 and subzone_id=8") + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "insert into system.replication_critical_localities("+ + "zone_id, subzone_id, locality, report_id, at_risk_ranges) values(16,16,'region=EU',2,6)") + require.NoError(t, err) + require.Equal(t, 1, rows) + + // Add new set of localities and verify the old ones are deleted + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + r.AddCriticalLocality(MakeZoneKey(7, 8), "dc=B") + r.AddCriticalLocality(MakeZoneKey(15, 6), "dc=A") + + time4 := time.Date(2001, 1, 1, 12, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time4, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_critical_localities", con), [][]string{ + {"5", "6", "'dc=A'", "2", "3"}, + {"7", "8", "'dc=B'", "2", "2"}, + {"15", "6", "'dc=A'", "2", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"2", "'2001-01-01 12:00:00+00:00'"}, + }) + require.Equal(t, 2, r.LastUpdatedRowCount()) + + // A brand new report (after restart for example) - still works. + r = makeReplicationCriticalLocalitiesReportSaver() + r.AddCriticalLocality(MakeZoneKey(5, 6), "dc=A") + + time5 := time.Date(2001, 1, 1, 12, 30, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time5, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_critical_localities", con), [][]string{ + {"5", "6", "'dc=A'", "2", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"2", "'2001-01-01 12:30:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) +} + +func TableData( + ctx context.Context, tableName string, executor sqlutil.InternalExecutor, +) [][]string { + if rows, err := executor.Query( + ctx, "test-select-"+tableName, nil /* txn */, "select * from "+tableName); err == nil { + result := make([][]string, 0, len(rows)) + for _, row := range rows { + stringRow := make([]string, 0, row.Len()) + for _, item := range row { + stringRow = append(stringRow, item.String()) + } + result = append(result, stringRow) + } + return result + } + return nil +} diff --git a/pkg/storage/reports/main_test.go b/pkg/storage/reports/main_test.go new file mode 100644 index 000000000000..6ae7e0986f0e --- /dev/null +++ b/pkg/storage/reports/main_test.go @@ -0,0 +1,30 @@ +// Copyright 2017 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +package reports_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/storage/reports/range_report.go b/pkg/storage/reports/range_report.go new file mode 100644 index 000000000000..8a4daca73d04 --- /dev/null +++ b/pkg/storage/reports/range_report.go @@ -0,0 +1,347 @@ +// Copyright 2019 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 reports + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// replicationStatsReportID is the id of the row in the system. reports_meta +// table corresponding to the replication stats report (i.e. the +// system.replication_stats table). +const replicationStatsReportID = 3 + +// RangeReport represents the system.zone_range_status report. +type RangeReport map[ZoneKey]zoneRangeStatus + +// zoneRangeStatus is the leaf of the RangeReport. +type zoneRangeStatus struct { + numRanges int32 + unavailable int32 + underReplicated int32 + overReplicated int32 +} + +// replicationStatsReportSaver manages the content and the saving of the report. +type replicationStatsReportSaver struct { + stats RangeReport + previousVersion RangeReport + lastGenerated time.Time + lastUpdatedRowCount int +} + +// makeReplicationStatsReportSaver creates a new report saver. +func makeReplicationStatsReportSaver() replicationStatsReportSaver { + return replicationStatsReportSaver{ + stats: RangeReport{}, + } +} + +// resetReport resets the report to an empty state. +func (r *replicationStatsReportSaver) resetReport() { + r.stats = RangeReport{} +} + +// LastUpdatedRowCount is the count of the rows that were touched during the last save. +func (r *replicationStatsReportSaver) LastUpdatedRowCount() int { + return r.lastUpdatedRowCount +} + +// AddZoneRangeStatus adds a row to the report. +func (r *replicationStatsReportSaver) AddZoneRangeStatus( + zKey ZoneKey, unavailable bool, underReplicated bool, overReplicated bool, +) { + if _, ok := r.stats[zKey]; !ok { + r.stats[zKey] = zoneRangeStatus{} + } + rStat := r.stats[zKey] + rStat.numRanges++ + if unavailable { + rStat.unavailable++ + } + if underReplicated { + rStat.underReplicated++ + } + if overReplicated { + rStat.overReplicated++ + } + r.stats[zKey] = rStat +} + +func (r *replicationStatsReportSaver) loadPreviousVersion( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, +) error { + // The data for the previous save needs to be loaded if: + // - this is the first time that we call this method and lastUpdatedAt has never been set + // - in case that the lastUpdatedAt is set but is different than the timestamp in reports_meta + // this indicates that some other worker wrote after we did the write. + if !r.lastGenerated.IsZero() { + // check to see if the last timestamp for the update matches the local one. + row, err := ex.QueryRow( + ctx, + "get-previous-timestamp", + txn, + "select generated from system.reports_meta where id = $1", + replicationStatsReportID, + ) + if err != nil { + return err + } + + // if the row is nil then this is the first time we are running and the reload is needed. + if row != nil { + generated, ok := row[0].(*tree.DTimestamp) + if !ok { + return errors.Errorf("Expected to get time from system.reports_meta but got %+v", row) + } + if generated.Time == r.lastGenerated { + // No need to reload. + return nil + } + } + } + const prevViolations = "select zone_id, subzone_id, total_ranges, " + + "unavailable_ranges, under_replicated_ranges, over_replicated_ranges " + + "from system.replication_stats" + rows, err := ex.Query( + ctx, "get-previous-replication-stats", txn, prevViolations, + ) + if err != nil { + return err + } + + r.previousVersion = make(RangeReport, len(rows)) + for _, row := range rows { + key := ZoneKey{} + key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.SubzoneID = SubzoneID(*row[1].(*tree.DInt)) + r.previousVersion[key] = zoneRangeStatus{ + (int32)(*row[2].(*tree.DInt)), + (int32)(*row[3].(*tree.DInt)), + (int32)(*row[4].(*tree.DInt)), + (int32)(*row[5].(*tree.DInt)), + } + } + + return nil +} + +func (r *replicationStatsReportSaver) updatePreviousVersion() { + r.previousVersion = r.stats + r.stats = make(RangeReport, len(r.previousVersion)) +} + +func (r *replicationStatsReportSaver) updateTimestamp( + ctx context.Context, ex sqlutil.InternalExecutor, txn *client.Txn, reportTS time.Time, +) error { + if !r.lastGenerated.IsZero() && reportTS == r.lastGenerated { + return errors.Errorf( + "The new time %s is the same as the time of the last update %s", + reportTS.String(), + r.lastGenerated.String(), + ) + } + + _, err := ex.Exec( + ctx, + "timestamp-upsert-replication-stats", + txn, + "upsert into system.reports_meta(id, generated) values($1, $2)", + replicationStatsReportID, + reportTS, + ) + return err +} + +// Save the report. +// +// reportTS is the time that will be set in the updated_at column for every row. +func (r *replicationStatsReportSaver) Save( + ctx context.Context, reportTS time.Time, db *client.DB, ex sqlutil.InternalExecutor, +) error { + r.lastUpdatedRowCount = 0 + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + err := r.loadPreviousVersion(ctx, ex, txn) + if err != nil { + return err + } + + err = r.updateTimestamp(ctx, ex, txn, reportTS) + if err != nil { + return err + } + + for key, status := range r.stats { + if err := r.upsertStats( + ctx, reportTS, txn, key, status, db, ex, + ); err != nil { + return err + } + } + + for key := range r.previousVersion { + if _, ok := r.stats[key]; !ok { + _, err := ex.Exec( + ctx, + "delete-old-replication-stats", + txn, + "delete from system.replication_stats "+ + "where zone_id = $1 and subzone_id = $2", + key.ZoneID, + key.SubzoneID, + ) + + if err != nil { + return err + } + r.lastUpdatedRowCount++ + } + } + + return nil + }); err != nil { + return err + } + + r.lastGenerated = reportTS + r.updatePreviousVersion() + + return nil +} + +// upsertStat upserts a row into system.replication_stats. +// +// existing is used to decide is this is a new data. +func (r *replicationStatsReportSaver) upsertStats( + ctx context.Context, + reportTS time.Time, + txn *client.Txn, + key ZoneKey, + stats zoneRangeStatus, + db *client.DB, + ex sqlutil.InternalExecutor, +) error { + var err error + previousStats, hasOldVersion := r.previousVersion[key] + if hasOldVersion && previousStats == stats { + // No change in the stats so no update. + return nil + } + + // Updating an old row. + _, err = ex.Exec( + ctx, "upsert-replication-stats", txn, + "upsert into system.replication_stats(report_id, zone_id, subzone_id, "+ + "total_ranges, unavailable_ranges, under_replicated_ranges, "+ + "over_replicated_ranges) values($1, $2, $3, $4, $5, $6, $7)", + replicationStatsReportID, + key.ZoneID, key.SubzoneID, stats.numRanges, stats.unavailable, + stats.underReplicated, stats.overReplicated, + ) + + if err != nil { + return err + } + + r.lastUpdatedRowCount++ + return nil +} + +// replicationStatsVisitor is a visitor that builds a RangeReport. +type replicationStatsVisitor struct { + cfg *config.SystemConfig + nodeChecker nodeChecker + + report *replicationStatsReportSaver +} + +var _ rangeVisitor = &replicationStatsVisitor{} + +func makeReplicationStatsVisitor( + ctx context.Context, + cfg *config.SystemConfig, + nodeChecker nodeChecker, + saver *replicationStatsReportSaver, +) replicationStatsVisitor { + v := replicationStatsVisitor{ + cfg: cfg, + nodeChecker: nodeChecker, + report: saver, + } + v.report.resetReport() + return v +} + +// reset is part of the rangeVisitor interface. +func (v *replicationStatsVisitor) reset(ctx context.Context) { + v.report.resetReport() + + // Iterate through all the zone configs to create report entries for all the + // zones that have constraints. Otherwise, just iterating through the ranges + // wouldn't create entries for zones that don't apply to any ranges. + maxObjectID, err := v.cfg.GetLargestObjectID(0 /* maxID - return the largest ID in the config */) + if err != nil { + log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) + } + for i := uint32(1); i <= maxObjectID; i++ { + zone, err := getZoneByID(i, v.cfg) + if err != nil { + log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) + } + if zone == nil { + continue + } + v.report.AddZoneRangeStatus( + MakeZoneKey(i, NoSubzone), + false, /* unavailable */ + false, /* underReplicated */ + false /* overReplicated */) + } +} + +// visit is part of the rangeVisitor interface. +func (v *replicationStatsVisitor) visit(ctx context.Context, r roachpb.RangeDescriptor) { + // Get the zone + var zKey ZoneKey + var zConfig *config.ZoneConfig + if err := visitZones(ctx, r, v.cfg, + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { + if zone.NumReplicas == nil || *zone.NumReplicas == 0 { + return false + } + zKey = key + zConfig = zone + return true + }); err != nil { + log.Fatalf(ctx, "unexpected error visiting zones: %s", err) + } + + underReplicated := *zConfig.NumReplicas > int32(len(r.Replicas().Voters())) + overReplicated := *zConfig.NumReplicas < int32(len(r.Replicas().Voters())) + var liveNodeCount int + for _, rep := range r.Replicas().Voters() { + if v.nodeChecker(rep.NodeID) { + liveNodeCount++ + } + } + unavailable := liveNodeCount < (len(r.Replicas().Voters())/2 + 1) + + v.report.AddZoneRangeStatus(zKey, unavailable, underReplicated, overReplicated) +} diff --git a/pkg/storage/reports/range_report_test.go b/pkg/storage/reports/range_report_test.go new file mode 100644 index 000000000000..799c35907fd4 --- /dev/null +++ b/pkg/storage/reports/range_report_test.go @@ -0,0 +1,134 @@ +// Copyright 2019 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 reports + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestRangeReport(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + // This test uses the cluster as a recipient for a report saved from outside + // the cluster. We disable the cluster's own production of reports so that it + // doesn't interfere with the test. + ReporterInterval.Override(&st.SV, 0) + s, _, db := serverutils.StartServer(t, base.TestServerArgs{Settings: st}) + con := s.InternalExecutor().(sqlutil.InternalExecutor) + defer s.Stopper().Stop(ctx) + + // Verify that tables are empty. + require.ElementsMatch(t, TableData(ctx, "system.replication_stats", con), [][]string{}) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{}) + + // Add several localities and verify the result + r := makeReplicationStatsReportSaver() + r.AddZoneRangeStatus(MakeZoneKey(1, 3), true, true, true) + r.AddZoneRangeStatus(MakeZoneKey(1, 3), false, true, true) + r.AddZoneRangeStatus(MakeZoneKey(1, 3), false, false, true) + r.AddZoneRangeStatus(MakeZoneKey(1, 3), true, true, false) + r.AddZoneRangeStatus(MakeZoneKey(2, 3), false, false, false) + r.AddZoneRangeStatus(MakeZoneKey(2, 4), false, true, false) + + time1 := time.Date(2001, 1, 1, 10, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time1, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_stats", con), [][]string{ + {"1", "3", "3", "4", "2", "3", "3"}, + {"2", "3", "3", "1", "0", "0", "0"}, + {"2", "4", "3", "1", "0", "1", "0"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"3", "'2001-01-01 10:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + // Add new set of localities and verify the old ones are deleted + r.AddZoneRangeStatus(MakeZoneKey(1, 3), false, true, true) + r.AddZoneRangeStatus(MakeZoneKey(2, 3), false, false, false) + r.AddZoneRangeStatus(MakeZoneKey(4, 4), false, true, true) + + time2 := time.Date(2001, 1, 1, 11, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time2, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_stats", con), [][]string{ + {"1", "3", "3", "1", "0", "1", "1"}, + {"2", "3", "3", "1", "0", "0", "0"}, + {"4", "4", "3", "1", "0", "1", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"3", "'2001-01-01 11:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + time3 := time.Date(2001, 1, 1, 11, 30, 0, 0, time.UTC) + // If some other server takes over and does an update. + rows, err := con.Exec(ctx, "another-updater", nil, "update system.reports_meta set generated=$1 where id=3", time3) + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "update system.replication_stats "+ + "set total_ranges=3 where zone_id=1 and subzone_id=3") + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "delete from system.replication_stats "+ + "where zone_id=2 and subzone_id=3") + require.NoError(t, err) + require.Equal(t, 1, rows) + rows, err = con.Exec(ctx, "another-updater", nil, "insert into system.replication_stats("+ + "zone_id, subzone_id, report_id, total_ranges, unavailable_ranges, under_replicated_ranges, "+ + "over_replicated_ranges) values(16, 16, 3, 6, 0, 1, 2)") + require.NoError(t, err) + require.Equal(t, 1, rows) + + // Add new set of localities and verify the old ones are deleted + r.AddZoneRangeStatus(MakeZoneKey(1, 3), false, true, true) + r.AddZoneRangeStatus(MakeZoneKey(2, 3), false, false, false) + r.AddZoneRangeStatus(MakeZoneKey(4, 4), false, true, true) + + time4 := time.Date(2001, 1, 1, 12, 0, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time4, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_stats", con), [][]string{ + {"1", "3", "3", "1", "0", "1", "1"}, + {"2", "3", "3", "1", "0", "0", "0"}, + {"4", "4", "3", "1", "0", "1", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"3", "'2001-01-01 12:00:00+00:00'"}, + }) + require.Equal(t, 3, r.LastUpdatedRowCount()) + + // A brand new report (after restart for example) - still works. + r = makeReplicationStatsReportSaver() + r.AddZoneRangeStatus(MakeZoneKey(1, 3), false, true, true) + + time5 := time.Date(2001, 1, 1, 12, 30, 0, 0, time.UTC) + require.NoError(t, r.Save(ctx, time5, db, con)) + + require.ElementsMatch(t, TableData(ctx, "system.replication_stats", con), [][]string{ + {"1", "3", "3", "1", "0", "1", "1"}, + }) + require.ElementsMatch(t, TableData(ctx, "system.reports_meta", con), [][]string{ + {"3", "'2001-01-01 12:30:00+00:00'"}, + }) + require.Equal(t, 2, r.LastUpdatedRowCount()) +} diff --git a/pkg/storage/reports/reporter.go b/pkg/storage/reports/reporter.go new file mode 100644 index 000000000000..225b5cf5defc --- /dev/null +++ b/pkg/storage/reports/reporter.go @@ -0,0 +1,644 @@ +// Copyright 2019 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 reports + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/storagepb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" +) + +// ReporterInterval is the interval between two generations of the reports. +// When set to zero - disables the report generation. +var ReporterInterval = settings.RegisterNonNegativeDurationSetting( + "kv.replication_reports.interval", + "the frequency for generating the replication_constraint_stats, replication_stats_report and "+ + "replication_critical_localities reports (set to 0 to disable)", + time.Minute, +) + +// Reporter periodically produces a couple of reports on the cluster's data +// distribution: the system tables: replication_constraint_stats, +// replication_stats_report and replication_critical_localities. +type Reporter struct { + // Contains the list of the stores of the current node + localStores *storage.Stores + // Constraints constructed from the locality information + localityConstraints []config.Constraints + // List of all stores known to exist on all nodes. + storeCfg *storage.StoreConfig + // The store that is the current meta 1 leaseholder + meta1LeaseHolder *storage.Store + // Latest zone config + latestConfig *config.SystemConfig + // Node liveness by store id + nodeLiveStatus map[roachpb.NodeID]storagepb.NodeLivenessStatus + + db *client.DB + + frequencyMu struct { + syncutil.Mutex + interval time.Duration + changeCh chan struct{} + } +} + +// NewReporter creates a Reporter. +func NewReporter(localStores *storage.Stores, storeCfg *storage.StoreConfig) *Reporter { + r := Reporter{ + db: storeCfg.DB, + localStores: localStores, + storeCfg: storeCfg, + } + r.frequencyMu.changeCh = make(chan struct{}) + return &r +} + +// reportInterval returns the current value of the frequency setting and a +// channel that will get closed when the value is not current any more. +func (stats *Reporter) reportInterval() (time.Duration, <-chan struct{}) { + stats.frequencyMu.Lock() + defer stats.frequencyMu.Unlock() + return ReporterInterval.Get(&stats.storeCfg.Settings.SV), stats.frequencyMu.changeCh +} + +// Start the periodic calls to Update(). +func (stats *Reporter) Start(ctx context.Context, stopper *stop.Stopper) { + ReporterInterval.SetOnChange(&stats.storeCfg.Settings.SV, func() { + stats.frequencyMu.Lock() + defer stats.frequencyMu.Unlock() + // Signal the current waiter (if any), and prepare the channel for future + // ones. + ch := stats.frequencyMu.changeCh + close(ch) + stats.frequencyMu.changeCh = make(chan struct{}) + stats.frequencyMu.interval = ReporterInterval.Get(&stats.storeCfg.Settings.SV) + }) + stopper.RunWorker(ctx, func(ctx context.Context) { + var timer timeutil.Timer + defer timer.Stop() + ctx = logtags.AddTag(ctx, "replication-reporter", nil /* value */) + + replStatsSaver := makeReplicationStatsReportSaver() + constraintsSaver := makeReplicationConstraintStatusReportSaver() + criticalLocSaver := makeReplicationCriticalLocalitiesReportSaver() + + for { + // Read the interval setting. We'll generate a report and then sleep for + // that long. We'll also wake up if the setting changes; that's useful for + // tests which want to lower the setting drastically and expect the report + // to be regenerated quickly, and also for users increasing the frequency. + interval, changeCh := stats.reportInterval() + + var timerCh <-chan time.Time + if interval != 0 { + // If (some store on) this node is the leaseholder for range 1, do the + // work. + stats.meta1LeaseHolder = stats.meta1LeaseHolderStore() + if stats.meta1LeaseHolder != nil { + if err := stats.update( + ctx, &constraintsSaver, &replStatsSaver, &criticalLocSaver, + ); err != nil { + log.Errorf(ctx, "failed to generate replication reports: %s", err) + } + } + timer.Reset(interval) + timerCh = timer.C + } + + // Wait until the timer expires (if there's a timer) or until there's an + // update to the frequency setting. + select { + case <-timerCh: + timer.Read = true + case <-changeCh: + case <-stopper.ShouldQuiesce(): + return + } + } + }) +} + +// update regenerates all the reports and saves them using the provided savers. +func (stats *Reporter) update( + ctx context.Context, + constraintsSaver *replicationConstraintStatsReportSaver, + replStatsSaver *replicationStatsReportSaver, + locSaver *replicationCriticalLocalitiesReportSaver, +) error { + stats.updateLatestConfig() + if stats.latestConfig == nil { + return nil + } + + stats.updateNodeLiveness() + + if err := stats.updateLocalityConstraints(); err != nil { + log.Errorf(ctx, "unable to update the locality constraints: %s", err) + } + + allStores := stats.storeCfg.StorePool.GetStores() + var getStoresFromGossip StoreResolver = func(r roachpb.RangeDescriptor) []roachpb.StoreDescriptor { + storeDescs := make([]roachpb.StoreDescriptor, len(r.Replicas().Voters())) + // We'll return empty descriptors for stores that gossip doesn't have a + // descriptor for. These stores will be considered to satisfy all + // constraints. + // TODO(andrei): note down that some descriptors were missing from gossip + // somewhere in the report. + for i, repl := range r.Replicas().Voters() { + storeDescs[i] = allStores[repl.StoreID] + } + return storeDescs + } + + // Create the visitors that we're going to pass to visitRanges() below. + constraintConfVisitor := makeConstraintConformanceVisitor( + ctx, stats.latestConfig, getStoresFromGossip, constraintsSaver) + localityStatsVisitor := makeLocalityStatsVisitor( + ctx, stats.localityConstraints, stats.latestConfig, + getStoresFromGossip, stats.isNodeLive, locSaver) + statusVisitor := makeReplicationStatsVisitor( + ctx, stats.latestConfig, stats.isNodeLive, replStatsSaver) + + // Iterate through all the ranges. + const descriptorReadBatchSize = 10000 + rangeIter := makeMeta2RangeIter(stats.db, descriptorReadBatchSize) + if err := visitRanges( + ctx, &rangeIter, stats.latestConfig, + &constraintConfVisitor, &localityStatsVisitor, &statusVisitor, + ); err != nil { + return errors.Wrap(err, "failed to compute constraint conformance report") + } + + if err := constraintConfVisitor.report.Save( + ctx, timeutil.Now() /* reportTS */, stats.db, stats.executor(), + ); err != nil { + return errors.Wrap(err, "failed to save constraint report") + } + if err := localityStatsVisitor.report.Save( + ctx, timeutil.Now() /* reportTS */, stats.db, stats.executor(), + ); err != nil { + return errors.Wrap(err, "failed to save locality report") + } + if err := statusVisitor.report.Save( + ctx, timeutil.Now() /* reportTS */, stats.db, stats.executor(), + ); err != nil { + return errors.Wrap(err, "failed to save range status report") + } + return nil +} + +func (stats *Reporter) executor() sqlutil.InternalExecutor { + return stats.storeCfg.SQLExecutor +} + +// meta1LeaseHolderStore returns the node store that is the leaseholder of Meta1 +// range or nil if none of the node's stores are holding the Meta1 lease. +func (stats *Reporter) meta1LeaseHolderStore() *storage.Store { + meta1RangeID := roachpb.RangeID(1) + var meta1LeaseHolder *storage.Store + err := stats.localStores.VisitStores(func(s *storage.Store) error { + if repl, _ := s.GetReplica(meta1RangeID); repl != nil { + if repl.OwnsValidLease(s.Clock().Now()) { + meta1LeaseHolder = s + return nil + } + } + return nil + }) + if err != nil { + log.Fatalf(context.TODO(), "unexpected error when visiting stores: %s", err) + } + return meta1LeaseHolder +} + +func (stats *Reporter) updateLatestConfig() { + stats.latestConfig = stats.meta1LeaseHolder.Gossip().GetSystemConfig() +} + +func (stats *Reporter) updateLocalityConstraints() error { + localityConstraintsByName := make(map[string]config.Constraints, 16) + for _, sd := range stats.storeCfg.StorePool.GetStores() { + c := config.Constraints{ + Constraints: make([]config.Constraint, 0), + } + for _, t := range sd.Node.Locality.Tiers { + c.Constraints = append( + c.Constraints, + config.Constraint{Type: config.Constraint_REQUIRED, Key: t.Key, Value: t.Value}) + localityConstraintsByName[c.String()] = c + } + } + stats.localityConstraints = make([]config.Constraints, 0, len(localityConstraintsByName)) + for _, c := range localityConstraintsByName { + stats.localityConstraints = append(stats.localityConstraints, c) + } + return nil +} + +func (stats *Reporter) updateNodeLiveness() { + stats.nodeLiveStatus = stats.storeCfg.NodeLiveness.GetLivenessStatusMap() +} + +// nodeChecker checks whether a node is to be considered alive or not. +type nodeChecker func(nodeID roachpb.NodeID) bool + +func (stats *Reporter) isNodeLive(nodeID roachpb.NodeID) bool { + l, ok := stats.nodeLiveStatus[nodeID] + if !ok { + return false + } + switch l { + // Decommissioning nodes are considered live nodes. + case storagepb.NodeLivenessStatus_LIVE, storagepb.NodeLivenessStatus_DECOMMISSIONING: + return true + default: + return false + } +} + +// visitZones applies a visitor to the hierarchy of zone zonfigs that apply to +// the given range, starting from the most specific to the default zone config. +// +// visitor is called for each zone config until it returns true, or until the +// default zone config is reached. It's passed zone configs and the +// corresponding zoneKeys. +func visitZones( + ctx context.Context, + r roachpb.RangeDescriptor, + cfg *config.SystemConfig, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, +) error { + id, keySuffix := config.DecodeKeyIntoZoneIDAndSuffix(r.StartKey) + zone, err := getZoneByID(id, cfg) + if err != nil { + return err + } + + // We've got the zone config (without considering for inheritance) for the + // "object" indicated by out key. Now we need to find where the constraints + // come from. We'll first look downwards - in subzones (if any). If there's no + // constraints there, we'll look in the zone config that we got. If not, + // we'll look upwards (e.g. database zone config, default zone config). + + if zone != nil { + // Try subzones. + subzone, subzoneIdx := zone.GetSubzoneForKeySuffix(keySuffix) + if subzone != nil { + if visitor(ctx, &subzone.Config, MakeZoneKey(id, SubzoneIDFromIndex(int(subzoneIdx)))) { + return nil + } + } + // Try the zone for our object. + if visitor(ctx, zone, MakeZoneKey(id, 0)) { + return nil + } + } + + // Go upwards. + return visitAncestors(ctx, id, cfg, visitor) +} + +// visitAncestors invokes the visitor of all the ancestors of the zone +// corresponding to id. The zone corresponding to id itself is not visited. +func visitAncestors( + ctx context.Context, + id uint32, + cfg *config.SystemConfig, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, +) error { + // Check to see if it's a table. If so, inherit from the database. + // For all other cases, inherit from the default. + descVal := cfg.GetValue(sqlbase.MakeDescMetadataKey(sqlbase.ID(id))) + if descVal == nil { + // Couldn't find a descriptor. This is not expected to happen. + // Let's just look at the default zone config. + visitDefaultZone(ctx, cfg, visitor) + return nil + } + + var desc sqlbase.Descriptor + if err := descVal.GetProto(&desc); err != nil { + return err + } + tableDesc := desc.GetTable() + // If it's a database, the parent is the default zone. + if tableDesc == nil { + visitDefaultZone(ctx, cfg, visitor) + return nil + } + + // If it's a table, the parent is a database. + zone, err := getZoneByID(uint32(tableDesc.ParentID), cfg) + if err != nil { + return err + } + if zone != nil { + visitor(ctx, zone, MakeZoneKey(uint32(tableDesc.ParentID), 0)) + return nil + } + // The parent database did not have constraints. Its parent is the default zone. + visitDefaultZone(ctx, cfg, visitor) + return nil +} + +func visitDefaultZone( + ctx context.Context, + cfg *config.SystemConfig, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, +) { + zone, err := getZoneByID(keys.RootNamespaceID, cfg) + if err != nil { + log.Fatalf(ctx, "failed to get default zone config: %s", err) + } + if zone == nil { + log.Fatal(ctx, "default zone config missing unexpectedly") + } + visitor(ctx, zone, MakeZoneKey(keys.RootNamespaceID, 0)) +} + +// getZoneByID returns a zone given its id. Inheritance does not apply. +func getZoneByID(id uint32, cfg *config.SystemConfig) (*config.ZoneConfig, error) { + zoneVal := cfg.GetValue(config.MakeZoneKey(id)) + if zoneVal == nil { + return nil, nil + } + zone := new(config.ZoneConfig) + if err := zoneVal.GetProto(zone); err != nil { + return nil, err + } + return zone, nil +} + +// processRange returns the list of constraints violated by a range. The range +// is represented by the descriptors of the replicas' stores. +func processRange( + ctx context.Context, storeDescs []roachpb.StoreDescriptor, constraintGroups []config.Constraints, +) []ConstraintRepr { + var res []ConstraintRepr + // Evaluate all zone constraints for the stores (i.e. replicas) of the given range. + for _, constraintGroup := range constraintGroups { + for i, c := range constraintGroup.Constraints { + replicasRequiredToMatch := int(constraintGroup.NumReplicas) + if replicasRequiredToMatch == 0 { + replicasRequiredToMatch = len(storeDescs) + } + if !constraintSatisfied(c, replicasRequiredToMatch, storeDescs) { + res = append(res, MakeConstraintRepr(constraintGroup, i)) + } + } + } + return res +} + +// constraintSatisfied checks that a range (represented by its replicas' stores) +// satisfies a constraint. +func constraintSatisfied( + c config.Constraint, replicasRequiredToMatch int, storeDescs []roachpb.StoreDescriptor, +) bool { + passCount := 0 + for _, storeDesc := range storeDescs { + // Consider stores for which we have no information to pass everything. + if storeDesc.StoreID == 0 { + passCount++ + continue + } + + storeMatches := true + match := config.StoreMatchesConstraint(storeDesc, c) + if c.Type == config.Constraint_REQUIRED && !match { + storeMatches = false + } + if c.Type == config.Constraint_PROHIBITED && match { + storeMatches = false + } + + if storeMatches { + passCount++ + } + } + return replicasRequiredToMatch <= passCount +} + +// StoreResolver is a function resolving a range to a store descriptor for each +// of the replicas. Empty store descriptors are to be returned when there's no +// information available for the store. +type StoreResolver func(roachpb.RangeDescriptor) []roachpb.StoreDescriptor + +type rangeVisitor interface { + visit(context.Context, roachpb.RangeDescriptor) + reset(ctx context.Context) +} + +// visitRanges iterates through all the range descriptors in Meta2 and calls the +// supplied visitors. +// +// An error is returned if some descriptors could not be read. +func visitRanges( + ctx context.Context, rangeStore RangeIterator, cfg *config.SystemConfig, visitor ...rangeVisitor, +) error { + // Iterate over all the ranges. + for { + rd, err := rangeStore.Next(ctx) + if err != nil { + if errIsRetriable(err) { + for _, v := range visitor { + v.reset(ctx) + } + // The iterator has been positioned to the beginning. + continue + } else { + return err + } + } + if rd.RangeID == 0 { + // We're done. + return nil + } + for _, v := range visitor { + v.visit(ctx, rd) + } + } +} + +// RangeIterator abstracts the interface for reading range descriptors. +type RangeIterator interface { + // Next returns the next range descriptors (in key order). + // Returns an empty RangeDescriptor when all the ranges have been exhausted. In that case, + // the iterator is not to be used any more (except for calling Close(), which will be a no-op). + // + // The returned error can be a retriable one (i.e. + // *roachpb.TransactionRetryWithProtoRefreshError, possibly wrapped). In that case, the iterator + // is reset automatically; the next Next() call ( should there be one) will + // return the first descriptor. + // In case of any other error, the iterator is automatically closed. + // It can't be used any more (except for calling Close(), which will be a noop). + Next(context.Context) (roachpb.RangeDescriptor, error) + + // Close destroys the iterator, releasing resources. It does not need to be + // called after Next() indicates exhaustion by returning an empty descriptor, + // or after Next() returns non-retriable errors. + Close(context.Context) +} + +// meta2RangeIter is an implementation of RangeIterator that scans meta2 in a +// paginated way. +type meta2RangeIter struct { + db *client.DB + // The size of the batches that descriptors will be read in. 0 for no limit. + batchSize int + + txn *client.Txn + // buffer contains descriptors read in the first batch, but not yet returned + // to the client. + buffer []client.KeyValue + // resumeSpan maintains the point where the meta2 scan stopped. + resumeSpan *roachpb.Span + // readingDone is set once we've scanned all of meta2. buffer may still + // contain descriptors. + readingDone bool +} + +func makeMeta2RangeIter(db *client.DB, batchSize int) meta2RangeIter { + return meta2RangeIter{db: db, batchSize: batchSize} +} + +var _ RangeIterator = &meta2RangeIter{} + +// Next is part of the rangeIterator interface. +func (r *meta2RangeIter) Next(ctx context.Context) (_ roachpb.RangeDescriptor, retErr error) { + defer func() { r.handleErr(ctx, retErr) }() + + rd, err := r.consumerBuffer() + if err != nil || rd.RangeID != 0 { + return rd, err + } + + if r.readingDone { + // No more batches to read. + return roachpb.RangeDescriptor{}, nil + } + + // Read a batch and consume the first row (if any). + if err := r.readBatch(ctx); err != nil { + return roachpb.RangeDescriptor{}, err + } + return r.consumerBuffer() +} + +func (r *meta2RangeIter) consumerBuffer() (roachpb.RangeDescriptor, error) { + if len(r.buffer) == 0 { + return roachpb.RangeDescriptor{}, nil + } + first := r.buffer[0] + var desc roachpb.RangeDescriptor + if err := first.ValueProto(&desc); err != nil { + return roachpb.RangeDescriptor{}, errors.NewAssertionErrorWithWrappedErrf(err, + "%s: unable to unmarshal range descriptor", first.Key) + } + r.buffer = r.buffer[1:] + return desc, nil +} + +// Close is part of the RangeIterator interface. +func (r *meta2RangeIter) Close(ctx context.Context) { + if r.readingDone { + return + } + _ = r.txn.Rollback(ctx) + r.txn = nil + r.readingDone = true +} + +func (r *meta2RangeIter) readBatch(ctx context.Context) (retErr error) { + defer func() { r.handleErr(ctx, retErr) }() + + if len(r.buffer) > 0 { + log.Fatalf(ctx, "buffer not exhausted: %d keys remaining", len(r.buffer)) + } + if r.txn == nil { + r.txn = r.db.NewTxn(ctx, "rangeStoreImpl") + } + + b := r.txn.NewBatch() + start := keys.Meta2Prefix + if r.resumeSpan != nil { + start = r.resumeSpan.Key + } + b.Scan(start, keys.MetaMax) + b.Header.MaxSpanRequestKeys = int64(r.batchSize) + err := r.txn.Run(ctx, b) + if err != nil { + return err + } + r.buffer = b.Results[0].Rows + r.resumeSpan = b.Results[0].ResumeSpan + if r.resumeSpan == nil { + if err := r.txn.Commit(ctx); err != nil { + return err + } + r.txn = nil + r.readingDone = true + } + return nil +} + +func errIsRetriable(err error) bool { + err = errors.UnwrapAll(err) + _, retriable := err.(*roachpb.TransactionRetryWithProtoRefreshError) + return retriable +} + +// handleErr manipulates the iterator's state in response to an error. +// In case of retriable error, the iterator is reset such that the next Next() +// call returns the first range. In case of any other error, resources are +// released and the iterator shouldn't be used any more. +// A nil error may be passed, in which case handleErr is a no-op. +// +// handleErr is idempotent. +func (r *meta2RangeIter) handleErr(ctx context.Context, err error) { + if err == nil { + return + } + if !errIsRetriable(err) { + if r.txn != nil { + // On any non-retriable error, rollback. + r.txn.CleanupOnError(ctx, err) + r.txn = nil + } + r.reset() + r.readingDone = true + } else { + r.reset() + } +} + +// reset the iterator. The next Next() call will return the first range. +func (r *meta2RangeIter) reset() { + r.buffer = nil + r.resumeSpan = nil + r.readingDone = false +} diff --git a/pkg/storage/reports/reporter_test.go b/pkg/storage/reports/reporter_test.go new file mode 100644 index 000000000000..4825f4977889 --- /dev/null +++ b/pkg/storage/reports/reporter_test.go @@ -0,0 +1,1215 @@ +// Copyright 2019 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 reports + +import ( + "bytes" + "context" + gosql "database/sql" + "fmt" + "sort" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/keysutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" + "github.com/gogo/protobuf/proto" + "github.com/stretchr/testify/require" + yaml "gopkg.in/yaml.v2" +) + +type zone struct { + // 0 means unset. + replicas int32 + // "" means unset. "[]" means empty. + constraints string +} + +func (z zone) toZoneConfig() config.ZoneConfig { + cfg := config.NewZoneConfig() + if z.replicas != 0 { + cfg.NumReplicas = proto.Int32(z.replicas) + } + if z.constraints != "" { + var constraintsList config.ConstraintsList + if err := yaml.UnmarshalStrict([]byte(z.constraints), &constraintsList); err != nil { + panic(err) + } + cfg.Constraints = constraintsList.Constraints + cfg.InheritedConstraints = false + } + return *cfg +} + +// Note that we support multi-column partitions but we don't support +// sub-partitions. +type partition struct { + name string + // start defines the partition's start key as a sequence of int column values. + start []int + // end defines the partition's end key as a sequence of int column values. + end []int + zone *zone +} + +func (p partition) toPartitionDescriptor() sqlbase.PartitioningDescriptor_Range { + var startKey roachpb.Key + for _, val := range p.start { + startKey = encoding.EncodeIntValue(startKey, encoding.NoColumnID, int64(val)) + } + var endKey roachpb.Key + for _, val := range p.end { + endKey = encoding.EncodeIntValue(endKey, encoding.NoColumnID, int64(val)) + } + return sqlbase.PartitioningDescriptor_Range{ + Name: p.name, + FromInclusive: startKey, + ToExclusive: endKey, + } +} + +type partitioning []partition + +func (p partitioning) numCols() int { + if len(p) == 0 { + return 0 + } + return len(p[0].start) +} + +func (p partitioning) validate() error { + // Validate that all partitions (if any) have the same number of columns. + if len(p) == 0 { + return nil + } + numCols := len(p[0].start) + for _, pp := range p { + if len(pp.start) != numCols { + return errors.Errorf("partition start doesn't have expected number of columns: %v", pp.start) + } + if len(pp.end) != numCols { + return errors.Errorf("partition end doesn't have expected number of columns: %v", pp.end) + } + } + return nil +} + +type index struct { + name string + zone *zone + partitions partitioning +} + +func (idx index) toIndexDescriptor(id int) sqlbase.IndexDescriptor { + var idxDesc sqlbase.IndexDescriptor + idxDesc.ID = sqlbase.IndexID(id) + idxDesc.Name = idx.name + if len(idx.partitions) > 0 { + neededCols := idx.partitions.numCols() + for i := 0; i < neededCols; i++ { + idxDesc.ColumnIDs = append(idxDesc.ColumnIDs, sqlbase.ColumnID(i)) + idxDesc.ColumnNames = append(idxDesc.ColumnNames, fmt.Sprintf("col%d", i)) + idxDesc.ColumnDirections = append(idxDesc.ColumnDirections, sqlbase.IndexDescriptor_ASC) + } + idxDesc.Partitioning.NumColumns = uint32(len(idx.partitions[0].start)) + for _, p := range idx.partitions { + idxDesc.Partitioning.Range = append(idxDesc.Partitioning.Range, p.toPartitionDescriptor()) + } + } + return idxDesc +} + +type table struct { + name string + zone *zone + indexes []index + partitions partitioning +} + +func (t table) validate() error { + return t.partitions.validate() +} + +type database struct { + name string + tables []table + zone *zone +} + +// constraintEntry represents an expected entry in the constraints conformance +// report. +type constraintEntry struct { + // object is the name of the table/index/partition that this entry refers to. + // The format is "" or "[.[.]]". A + // partition on the primary key is "
.". + object string + constraint string + constraintType ConstraintType + numRanges int +} + +// toReportEntry transforms the entry into the key/value format of the generated +// report. +func (c constraintEntry) toReportEntry( + objects map[string]ZoneKey, +) (ConstraintStatusKey, ConstraintStatus, error) { + zk, ok := objects[c.object] + if !ok { + return ConstraintStatusKey{}, ConstraintStatus{}, + errors.AssertionFailedf("missing object: " + c.object) + } + k := ConstraintStatusKey{ + ZoneKey: zk, + ViolationType: c.constraintType, + Constraint: ConstraintRepr(c.constraint), + } + v := ConstraintStatus{FailRangeCount: c.numRanges} + return k, v, nil +} + +type split struct { + key string + stores []int +} + +type store struct { + id int + attrs string // comma separated +} + +// toStoreDescriptor transforms the store into a StoreDescriptor. +// +// nodeDesc is the descriptor of the parent node. +func (s store) toStoreDesc(nodeDesc roachpb.NodeDescriptor) roachpb.StoreDescriptor { + desc := roachpb.StoreDescriptor{ + StoreID: roachpb.StoreID(s.id), + Node: nodeDesc, + } + desc.Attrs.Attrs = append(desc.Attrs.Attrs, strings.Split(s.attrs, ",")...) + return desc +} + +type node struct { + id int + // locality is the node's locality, in the same format that the --locality + // flag. "" for not setting a locality. + locality string + stores []store +} + +func (n node) toDescriptors() (roachpb.NodeDescriptor, []roachpb.StoreDescriptor, error) { + nodeDesc := roachpb.NodeDescriptor{ + NodeID: roachpb.NodeID(n.id), + } + if n.locality != "" { + if err := nodeDesc.Locality.Set(n.locality); err != nil { + return roachpb.NodeDescriptor{}, nil, err + } + } + storeDescs := make([]roachpb.StoreDescriptor, len(n.stores)) + for i, s := range n.stores { + storeDescs[i] = s.toStoreDesc(nodeDesc) + } + return nodeDesc, storeDescs, nil +} + +type conformanceConstraintTestCase struct { + name string + schema []database + splits []split + nodes []node + deadStores []int + defaultZone zone + + exp []constraintEntry +} + +func TestConformanceReport(t *testing.T) { + defer leaktest.AfterTest(t)() + tests := []conformanceConstraintTestCase{ + { + name: "simple no violations", + defaultZone: zone{replicas: 3}, + schema: []database{ + { + name: "db1", + tables: []table{{name: "t1"}, {name: "t2"}}, + // The database has a zone requesting everything to be on SSDs. + zone: &zone{ + replicas: 3, + constraints: "[+ssd]", + }, + }, + { + name: "db2", + tables: []table{{name: "sentinel"}}, + }, + }, + splits: []split{ + {key: "/Table/t1", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/1", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/2", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/3", stores: []int{1, 2, 3}}, + {key: "/Table/t2", stores: []int{1, 2, 3}}, + {key: "/Table/t2/pk", stores: []int{1, 2, 3}}, + { + // This range is not covered by the db1's zone config and so it won't + // be counted. + key: "/Table/sentinel", stores: []int{1, 2, 3}, + }, + }, + nodes: []node{ + {id: 1, locality: "", stores: []store{{id: 1, attrs: "ssd"}}}, + {id: 2, locality: "", stores: []store{{id: 2, attrs: "ssd"}}}, + {id: 3, locality: "", stores: []store{{id: 3, attrs: "ssd"}}}, + }, + deadStores: nil, + exp: []constraintEntry{{ + object: "db1", + constraint: "+ssd", + constraintType: Constraint, + numRanges: 0, + }}, + }, + { + // Test zone constraints inheritance at all levels. + name: "violations at multiple levels", + defaultZone: zone{replicas: 3, constraints: "[+default]"}, + schema: []database{ + { + name: "db1", + // All the objects will have zones asking for different tags. + zone: &zone{ + replicas: 3, + constraints: "[+db1]", + }, + tables: []table{ + { + name: "t1", + zone: &zone{ + replicas: 3, + constraints: "[+t1]", + }, + }, + { + name: "t2", + zone: &zone{ + replicas: 3, + constraints: "[+t2]", + }, + }, + // Violations for this one will count towards db1. + {name: "sentinel"}, + }, + }, { + name: "db2", + zone: &zone{constraints: "[+db2]"}, + // Violations for this one will count towards db2, except for the + // partition part. + tables: []table{{ + name: "t3", + partitions: []partition{{ + name: "p1", + start: []int{100}, + end: []int{200}, + zone: &zone{constraints: "[+p1]"}, + }}, + }}, + }, { + name: "db3", + // Violations for this one will count towards the default zone. + tables: []table{{name: "t4"}}, + }, + }, + splits: []split{ + {key: "/Table/t1", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/1", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/2", stores: []int{1, 2, 3}}, + {key: "/Table/t1/pk/3", stores: []int{1, 2, 3}}, + {key: "/Table/t2", stores: []int{1, 2, 3}}, + {key: "/Table/t2/pk", stores: []int{1, 2, 3}}, + {key: "/Table/sentinel", stores: []int{1, 2, 3}}, + {key: "/Table/t3", stores: []int{1, 2, 3}}, + {key: "/Table/t3/pk/100", stores: []int{1, 2, 3}}, + {key: "/Table/t3/pk/101", stores: []int{1, 2, 3}}, + {key: "/Table/t3/pk/199", stores: []int{1, 2, 3}}, + {key: "/Table/t3/pk/200", stores: []int{1, 2, 3}}, + {key: "/Table/t4", stores: []int{1, 2, 3}}, + }, + // None of the stores have any attributes. + nodes: []node{ + {id: 1, locality: "", stores: []store{{id: 1}}}, + {id: 2, locality: "", stores: []store{{id: 2}}}, + {id: 3, locality: "", stores: []store{{id: 3}}}, + }, + deadStores: nil, + exp: []constraintEntry{ + { + object: "default", + constraint: "+default", + constraintType: Constraint, + numRanges: 1, + }, + { + object: "db1", + constraint: "+db1", + constraintType: Constraint, + numRanges: 1, + }, + { + object: "t1", + constraint: "+t1", + constraintType: Constraint, + numRanges: 5, + }, + { + object: "t2", + constraint: "+t2", + constraintType: Constraint, + numRanges: 2, + }, + { + object: "db2", + constraint: "+db2", + constraintType: Constraint, + numRanges: 2, + }, + { + object: "t3.p1", + constraint: "+p1", + constraintType: Constraint, + numRanges: 3, + }, + }, + }, + } + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + runConformanceReportTest(t, tc) + }) + } +} + +type row struct { + ConstraintStatusKey + ConstraintStatus +} + +func (r row) String() string { + return fmt.Sprintf("%s failed:%d", r.ConstraintStatusKey, r.ConstraintStatus.FailRangeCount) +} + +type rows []row + +func (r rows) String() string { + var sb strings.Builder + for _, rr := range r { + sb.WriteString(rr.String()) + sb.WriteRune('\n') + } + return sb.String() +} + +// runConformanceReportTest runs one test case. It processes the input schema, +// runs the reports, and verifies that the report looks as expected. +func runConformanceReportTest(t *testing.T, tc conformanceConstraintTestCase) { + rangeStore, sysCfg, storeResolver, objects, err := processTestCase(tc) + if err != nil { + t.Fatal(err) + } + rep, err := computeConstraintConformanceReport( + context.Background(), &rangeStore, sysCfg, storeResolver) + if err != nil { + t.Fatal(err) + } + + // Sort the report's keys. + gotRows := make(rows, len(rep.constraints)) + i := 0 + for k, v := range rep.constraints { + gotRows[i] = row{ConstraintStatusKey: k, ConstraintStatus: v} + i++ + } + sort.Slice(gotRows, func(i, j int) bool { + return gotRows[i].ConstraintStatusKey.Less(gotRows[j].ConstraintStatusKey) + }) + + expRows := make(rows, len(tc.exp)) + for i, exp := range tc.exp { + k, v, err := exp.toReportEntry(objects) + if err != nil { + t.Fatalf("failed to process expected entry %d: %s", i, err) + } + expRows[i] = row{ + ConstraintStatusKey: k, + ConstraintStatus: v, + } + } + require.Equal(t, expRows, gotRows) +} + +type testRangeIter struct { + ranges []roachpb.RangeDescriptor +} + +var _ RangeIterator = &testRangeIter{} + +// Next is part of the RangeIterator interface. +func (t *testRangeIter) Next(ctx context.Context) (roachpb.RangeDescriptor, error) { + if len(t.ranges) == 0 { + return roachpb.RangeDescriptor{}, nil + } + first := t.ranges[0] + t.ranges = t.ranges[1:] + return first, nil +} + +// Close is part of the RangeIterator interface. +func (t *testRangeIter) Close(context.Context) { + t.ranges = nil +} + +// processTestCase takes the input schema and turns it into: +// - a collection of ranges represented as an implementation of RangeStore +// - a SystemConfig populated with descriptors and zone configs. +// - a collection of stores represented as an implementation of StoreResolver. +// - a map from "object names" to ZoneKeys; each table/index/partition with a zone is mapped to +// the id that the report will use for it. See constraintEntry.object for the key format. +func processTestCase( + tc conformanceConstraintTestCase, +) (testRangeIter, *config.SystemConfig, StoreResolver, map[string]ZoneKey, error) { + tableToID := make(map[string]int) + idxToID := make(map[string]int) + objects := make(map[string]ZoneKey) + // Databases and tables share the id space, so we'll use a common counter for them. + // And we're going to use keys in user space, otherwise there's special cases + // in the zone config lookup that we bump into. + objectCounter := keys.MinUserDescID + var sysCfgBuilder systemConfigBuilder + sysCfgBuilder.setDefaultZoneConfig(tc.defaultZone.toZoneConfig()) + objects["default"] = MakeZoneKey(keys.RootNamespaceID, NoSubzone) + // Assign ids to databases, table, indexes; create descriptors and populate + // the SystemConfig. + for _, db := range tc.schema { + dbID := objectCounter + objectCounter++ + objects[db.name] = MakeZoneKey(uint32(dbID), NoSubzone) + if db.zone != nil { + sysCfgBuilder.addZone(dbID, db.zone.toZoneConfig()) + } + sysCfgBuilder.addDBDesc(dbID, sqlbase.DatabaseDescriptor{ + Name: db.name, + ID: sqlbase.ID(dbID), + }) + + for _, table := range db.tables { + tableID := objectCounter + objectCounter++ + tableToID[table.name] = tableID + objects[table.name] = MakeZoneKey(uint32(tableID), NoSubzone) + // Create a table descriptor to be used for creating the zone config. + tableDesc, err := makeTableDesc(table, tableID, dbID) + if err != nil { + return testRangeIter{}, nil, nil, nil, errors.Wrap(err, "error creating table descriptor") + } + sysCfgBuilder.addTableDesc(tableID, tableDesc) + + // Create the table's zone config. + var tableZone *config.ZoneConfig + if table.zone != nil { + tableZone = new(config.ZoneConfig) + *tableZone = table.zone.toZoneConfig() + } + // Add subzones for the PK partitions. + if len(table.partitions) > 0 { + pk := index{ + name: "PK", + zone: nil, // PK's never have zones; the table might have a zone. + partitions: table.partitions, + } + tableZone = addIndexSubzones(pk, tableZone, tableDesc, 1 /* id of PK */, objects) + } + // Add subzones for all the indexes. + for i, idx := range table.indexes { + idxID := i + 2 // index 1 is the PK + idxToID[fmt.Sprintf("%s.%s", table.name, idx.name)] = idxID + tableZone = addIndexSubzones(idx, tableZone, tableDesc, idxID, objects) + } + // Fill in the SubzoneSpans. + if tableZone != nil { + var err error + tableZone.SubzoneSpans, err = sql.GenerateSubzoneSpans( + nil, uuid.UUID{} /* clusterID */, &tableDesc, tableZone.Subzones, + false /* hasNewSubzones */) + if err != nil { + return testRangeIter{}, nil, nil, nil, errors.Wrap(err, "error generating subzone spans") + } + sysCfgBuilder.addZone(tableID, *tableZone) + } + } + } + + keyScanner := keysutils.MakePrettyScannerForNamedTables(tableToID, idxToID) + ranges := make([]roachpb.RangeDescriptor, len(tc.splits)) + for i, split := range tc.splits { + prettyKey := tc.splits[i].key + startKey, err := keyScanner.Scan(split.key) + if err != nil { + return testRangeIter{}, nil, nil, nil, errors.Wrapf(err, "failed to parse key: %s", prettyKey) + } + var endKey roachpb.Key + if i < len(tc.splits)-1 { + prettyKey := tc.splits[i+1].key + endKey, err = keyScanner.Scan(prettyKey) + if err != nil { + return testRangeIter{}, nil, nil, nil, errors.Wrapf(err, "failed to parse key: %s", prettyKey) + } + } else { + endKey = roachpb.KeyMax + } + + rd := roachpb.RangeDescriptor{ + RangeID: roachpb.RangeID(i + 1), // IDs start at 1 + StartKey: keys.MustAddr(startKey), + EndKey: keys.MustAddr(endKey), + } + for _, storeID := range split.stores { + rd.AddReplica(roachpb.NodeID(storeID), roachpb.StoreID(storeID), roachpb.VOTER_FULL) + } + ranges[i] = rd + } + + var storeDescs []roachpb.StoreDescriptor + for _, n := range tc.nodes { + _ /* nodeDesc */, sds, err := n.toDescriptors() + if err != nil { + return testRangeIter{}, nil, nil, nil, err + } + storeDescs = append(storeDescs, sds...) + } + storeResolver := func(r roachpb.RangeDescriptor) []roachpb.StoreDescriptor { + stores := make([]roachpb.StoreDescriptor, len(r.Replicas().Voters())) + for i, rep := range r.Replicas().Voters() { + for _, desc := range storeDescs { + if rep.StoreID == desc.StoreID { + stores[i] = desc + break + } + } + } + return stores + } + return testRangeIter{ranges: ranges}, sysCfgBuilder.build(), storeResolver, objects, nil +} + +func makeTableDesc(t table, tableID int, dbID int) (sqlbase.TableDescriptor, error) { + if err := t.validate(); err != nil { + return sqlbase.TableDescriptor{}, err + } + + desc := sqlbase.TableDescriptor{ + ID: sqlbase.ID(tableID), + Name: t.name, + ParentID: sqlbase.ID(dbID), + } + pkIdx := index{ + name: "PK", + zone: nil, + partitions: t.partitions, + } + desc.Indexes = append(desc.Indexes, pkIdx.toIndexDescriptor(1)) + + for i, idx := range t.indexes { + idxID := i + 2 // index 1 is the PK + desc.Indexes = append(desc.Indexes, idx.toIndexDescriptor(idxID)) + } + numCols := 0 + for _, idx := range desc.Indexes { + c := len(idx.ColumnIDs) + if c > numCols { + numCols = c + } + } + for i := 0; i < numCols; i++ { + desc.Columns = append(desc.Columns, sqlbase.ColumnDescriptor{ + Name: fmt.Sprintf("col%d", i), + ID: sqlbase.ColumnID(i), + Type: *types.Int, + }) + } + + return desc, nil +} + +// addIndexSubzones creates subzones for an index and all of its partitions and +// appends them to a parent table zone, returning the amended parent. +// If the index and its partitions have no zones, the parent is returned unchanged ( +// and possibly nil). +// +// parent: Can be nil if the parent table doesn't have a zone of its own. In that +// case, if any subzones are created, a placeholder zone will also be created and returned. +// subzone: The index's ZoneConfig. Can be nil if the index doesn't have a zone config. +// objects: A mapping from object name to ZoneKey that gets populated with all the new subzones. +func addIndexSubzones( + idx index, + parent *config.ZoneConfig, + tableDesc sql.TableDescriptor, + idxID int, + objects map[string]ZoneKey, +) *config.ZoneConfig { + res := parent + + ensureParent := func() { + if res != nil { + return + } + // Create a placeholder zone config. + res = config.NewZoneConfig() + res.DeleteTableConfig() + } + + if idx.zone != nil { + ensureParent() + res.SetSubzone(config.Subzone{ + IndexID: uint32(idxID), + PartitionName: "", + Config: idx.zone.toZoneConfig(), + }) + objects[fmt.Sprintf("%s.%s", tableDesc.Name, idx.name)] = + MakeZoneKey(uint32(tableDesc.ID), SubzoneID(len(res.Subzones))) + } + + for _, p := range idx.partitions { + if p.zone != nil { + ensureParent() + res.SetSubzone(config.Subzone{ + IndexID: uint32(idxID), + PartitionName: p.name, + Config: p.zone.toZoneConfig(), + }) + var objectName string + if idxID == 1 { + objectName = fmt.Sprintf("%s.%s", tableDesc.Name, p.name) + } else { + objectName = fmt.Sprintf("%s.%s.%s", tableDesc.Name, idx.name, p.name) + } + objects[objectName] = MakeZoneKey(uint32(tableDesc.ID), SubzoneID(len(res.Subzones))) + } + } + return res +} + +// systemConfigBuilder build a system config. Clients will call some setters and then call build(). +type systemConfigBuilder struct { + kv []roachpb.KeyValue + defaultZoneConfig *config.ZoneConfig +} + +func (b *systemConfigBuilder) setDefaultZoneConfig(cfg config.ZoneConfig) { + b.defaultZoneConfig = &cfg + b.addZone(keys.RootNamespaceID, cfg) +} + +func (b *systemConfigBuilder) addZone(id int, cfg config.ZoneConfig) { + k := config.MakeZoneKey(uint32(id)) + var v roachpb.Value + if err := v.SetProto(&cfg); err != nil { + panic(err) + } + b.kv = append(b.kv, roachpb.KeyValue{Key: k, Value: v}) +} + +// build constructs a SystemConfig containing all the information accumulated in the builder. +func (b *systemConfigBuilder) build() *config.SystemConfig { + if b.defaultZoneConfig == nil { + panic("default zone config not set") + } + + sort.Slice(b.kv, func(i, j int) bool { + return bytes.Compare(b.kv[i].Key, b.kv[j].Key) < 0 + }) + + cfg := config.NewSystemConfig(b.defaultZoneConfig) + cfg.SystemConfigEntries.Values = b.kv + return cfg +} + +// addTableDesc adds a table descriptor to the SystemConfig. +func (b *systemConfigBuilder) addTableDesc(id int, desc sqlbase.TableDescriptor) { + if desc.ParentID == 0 { + panic(fmt.Sprintf("parent not set for table %q", desc.Name)) + } + // Write the table to the SystemConfig, in the descriptors table. + k := sqlbase.MakeDescMetadataKey(sqlbase.ID(id)) + dbDesc := &sqlbase.Descriptor{ + Union: &sqlbase.Descriptor_Table{ + Table: &desc, + }, + } + var v roachpb.Value + if err := v.SetProto(dbDesc); err != nil { + panic(err) + } + b.kv = append(b.kv, roachpb.KeyValue{Key: k, Value: v}) +} + +// addTableDesc adds a database descriptor to the SystemConfig. +func (b *systemConfigBuilder) addDBDesc(id int, desc sqlbase.DatabaseDescriptor) { + // Write the table to the SystemConfig, in the descriptors table. + k := sqlbase.MakeDescMetadataKey(sqlbase.ID(id)) + dbDesc := &sqlbase.Descriptor{ + Union: &sqlbase.Descriptor_Database{ + Database: &desc, + }, + } + var v roachpb.Value + if err := v.SetProto(dbDesc); err != nil { + panic(err) + } + b.kv = append(b.kv, roachpb.KeyValue{Key: k, Value: v}) +} + +// Test the constraint conformance report in a real cluster. +func TestConstraintConformanceReportIntegration(t *testing.T) { + defer leaktest.AfterTest(t)() + t.Skip("TODO(andrei): flaky under stress") + + ctx := context.Background() + tc := serverutils.StartTestCluster(t, 5, base.TestClusterArgs{ + ServerArgsPerNode: map[int]base.TestServerArgs{ + 0: {Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "r1"}}}}, + 1: {Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "r1"}}}}, + 2: {Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "r2"}}}}, + 3: {Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "r2"}}}}, + 4: {Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "r2"}}}}, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + // Speed up the generation of the + _, err := db.Exec("set cluster setting kv.replication_reports.interval = '1ms'") + require.NoError(t, err) + + // Create a table and a zone config for it. + // The zone will be configured with a constraints that can't be satisfied + // because there are not enough nodes in the requested region. + _, err = db.Exec("create table t(x int primary key); " + + "alter table t configure zone using constraints='[+region=r1]'") + require.NoError(t, err) + + // Get the id of the newly created zone. + r := db.QueryRow("select zone_id from crdb_internal.zones where table_name = 't'") + var zoneID int + require.NoError(t, r.Scan(&zoneID)) + + // Wait for the violation to be detected. + testutils.SucceedsSoon(t, func() error { + r := db.QueryRow( + "select violating_ranges from system.replication_constraint_stats where zone_id = $1", + zoneID) + var numViolations int + if err := r.Scan(&numViolations); err != nil { + return err + } + if numViolations == 0 { + return fmt.Errorf("violation not detected yet") + } + return nil + }) + + // Now change the constraint asking for t to be placed in r2. This time it can be satisfied. + _, err = db.Exec("alter table t configure zone using constraints='[+region=r2]'") + require.NoError(t, err) + + // Wait for the violation to clear. + testutils.SucceedsSoon(t, func() error { + r := db.QueryRow( + "select violating_ranges from system.replication_constraint_stats where zone_id = $1", + zoneID) + var numViolations int + if err := r.Scan(&numViolations); err != nil { + return err + } + if numViolations > 0 { + return fmt.Errorf("still reporting violations") + } + return nil + }) +} + +// Test the critical localities report in a real cluster. +func TestCriticalLocalitiesReportIntegration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + // 2 regions, 3 dcs per region. + tc := serverutils.StartTestCluster(t, 6, base.TestClusterArgs{ + // We're going to do our own replication. + // All the system ranges will start with a single replica on node 1. + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: map[int]base.TestServerArgs{ + 0: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r1"}, {Key: "dc", Value: "dc1"}}, + }, + }, + 1: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r1"}, {Key: "dc", Value: "dc2"}}, + }, + }, + 2: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r1"}, {Key: "dc", Value: "dc3"}}, + }, + }, + 3: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r2"}, {Key: "dc", Value: "dc4"}}, + }, + }, + 4: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r2"}, {Key: "dc", Value: "dc5"}}, + }, + }, + 5: { + Locality: roachpb.Locality{Tiers: []roachpb.Tier{ + {Key: "region", Value: "r2"}, {Key: "dc", Value: "dc6"}}, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + // Speed up the generation of the + _, err := db.Exec("set cluster setting kv.replication_reports.interval = '1ms'") + require.NoError(t, err) + + // Since we're using ReplicationManual, all the ranges will start with a + // single replica on node 1. So, the node's dc and the node's region are + // critical. Let's verify that. + + // Collect all the zones that exist at cluster bootstrap. + systemZoneIDs := make([]int, 0, 10) + { + rows, err := db.Query("select id from system.zones") + require.NoError(t, err) + for rows.Next() { + var zoneID int + require.NoError(t, rows.Scan(&zoneID)) + systemZoneIDs = append(systemZoneIDs, zoneID) + } + require.NoError(t, rows.Err()) + } + require.True(t, len(systemZoneIDs) > 0, "expected some system zones, got none") + + expCritLoc := []string{"region=r1", "region=r1,dc=dc1"} + + // Wait for the report to be generated. + { + var rowCount int + testutils.SucceedsSoon(t, func() error { + r := db.QueryRow("select count(1) from system.replication_critical_localities") + require.NoError(t, r.Scan(&rowCount)) + if rowCount == 0 { + return fmt.Errorf("no report yet") + } + return nil + }) + require.Equal(t, 2*len(systemZoneIDs), rowCount) + } + + // Check that we have all the expected rows. + for _, zid := range systemZoneIDs { + for _, s := range expCritLoc { + r := db.QueryRow( + "select at_risk_ranges from system.replication_critical_localities "+ + "where zone_id=$1 and locality=$2", + zid, s) + var numRanges int + require.NoError(t, r.Scan(&numRanges)) + require.NotEqual(t, 0, numRanges) + } + } + + // Now create a table and a zone for it. At first n1 should be critical for it. + // Then we'll upreplicate it in different ways. + + // Create a table with a dummy zone config. Configuring the zone is useful + // only for creating the zone; we don't actually care about the configuration. + // Also do a split by hand. With manual replication, we're not getting the + // split for the table automatically. + _, err = db.Exec("create table t(x int primary key); " + + "alter table t configure zone using num_replicas=3; " + + "alter table t split at values (0);") + require.NoError(t, err) + // Get the id of the newly created zone. + r := db.QueryRow("select zone_id from crdb_internal.zones where table_name = 't'") + var zoneID int + require.NoError(t, r.Scan(&zoneID)) + + // Check initial conditions. + require.NoError(t, checkCritical(db, zoneID, "region=r1", "region=r1,dc=dc1")) + + // Upreplicate to 2 dcs. Now they're both critical. + _, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2], 1)") + require.NoError(t, err) + require.NoError(t, checkCritical(db, zoneID, "region=r1", "region=r1,dc=dc1", "region=r1,dc=dc2")) + + // Upreplicate to one more dc. Now no dc is critical, only the region. + _, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 1)") + require.NoError(t, err) + require.NoError(t, checkCritical(db, zoneID, "region=r1")) + + // Move two replicas to the other region. Now that region is critical. + _, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,4,5], 1)") + require.NoError(t, err) + require.NoError(t, checkCritical(db, zoneID, "region=r2")) +} + +func checkCritical(db *gosql.DB, zoneID int, locs ...string) error { + return testutils.SucceedsSoonError(func() error { + rows, err := db.Query( + "select locality, at_risk_ranges from system.replication_critical_localities "+ + "where zone_id=$1", zoneID) + if err != nil { + return err + } + critical := make(map[string]struct{}) + for rows.Next() { + var numRanges int + var loc string + err := rows.Scan(&loc, &numRanges) + if err != nil { + return err + } + if numRanges == 0 { + return fmt.Errorf("expected ranges_at_risk for %s", loc) + } + critical[loc] = struct{}{} + } + if err := rows.Err(); err != nil { + return err + } + if len(locs) != len(critical) { + return fmt.Errorf("expected critical: %s, got: %s", locs, critical) + } + for _, l := range locs { + if _, ok := critical[l]; !ok { + return fmt.Errorf("missing critical locality: %s", l) + } + } + return nil + }) +} + +// Test the replication status report in a real cluster. +func TestReplicationStatusReportIntegration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + tc := serverutils.StartTestCluster(t, 4, base.TestClusterArgs{ + // We're going to do our own replication. + // All the system ranges will start with a single replica on node 1. + ReplicationMode: base.ReplicationManual, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + // Speed up the generation of the + _, err := db.Exec("set cluster setting kv.replication_reports.interval = '1ms'") + require.NoError(t, err) + + // Create a table with a dummy zone config. Configuring the zone is useful + // only for creating the zone; we don't actually care about the configuration. + // Also do a split by hand. With manual replication, we're not getting the + // split for the table automatically. + _, err = db.Exec("create table t(x int primary key); " + + "alter table t configure zone using num_replicas=3; " + + "alter table t split at values (0);") + require.NoError(t, err) + // Get the id of the newly created zone. + r := db.QueryRow("select zone_id from crdb_internal.zones where table_name = 't'") + var zoneID int + require.NoError(t, r.Scan(&zoneID)) + + // Upreplicate the range. + _, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 1)") + require.NoError(t, err) + require.NoError(t, checkZoneReplication(db, zoneID, 1, 0, 0, 0)) + + // Over-replicate. + _, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3,4], 1)") + require.NoError(t, err) + require.NoError(t, checkZoneReplication(db, zoneID, 1, 0, 1, 0)) + + // TODO(andrei): I'd like to downreplicate to one replica and then stop that + // node and check that the range is counter us "unavailable", but stopping a + // node makes the report generation simply block sometimes trying to scan + // Meta2. I believe I believe it's due to #40529. + // Once stopping a node works, next thing is to start it up again. + // Take inspiration from replica_learner_test.go. + + //// Down-replicate to one node and then kill the node. Check that the range becomes unavailable. + //_, err = db.Exec("ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[4], 1)") + //require.NoError(t, err) + //tc.StopServer(3) + //require.NoError(t, checkZoneReplication(db, zoneID, 1, 1, 0, 1)) +} + +func checkZoneReplication(db *gosql.DB, zoneID, total, under, over, unavailable int) error { + return testutils.SucceedsSoonError(func() error { + r := db.QueryRow( + "select total_ranges, under_replicated_ranges, over_replicated_ranges, "+ + "unavailable_ranges from system.replication_stats where zone_id=$1", + zoneID) + var gotTotal, gotUnder, gotOver, gotUnavailable int + if err := r.Scan(&gotTotal, &gotUnder, &gotOver, &gotUnavailable); err != nil { + return err + } + if total != gotTotal { + return fmt.Errorf("expected total: %d, got: %d", total, gotTotal) + } + if under != gotUnder { + return fmt.Errorf("expected under: %d, got: %d", total, gotUnder) + } + if over != gotOver { + return fmt.Errorf("expected over: %d, got: %d", over, gotOver) + } + if unavailable != gotUnavailable { + return fmt.Errorf("expected unavailable: %d, got: %d", unavailable, gotUnavailable) + } + return nil + }) +} + +func TestMeta2RangeIter(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + s, _, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + // First make an interator with a large page size and use it to determine the numner of ranges. + iter := makeMeta2RangeIter(db, 10000 /* batchSize */) + numRanges := 0 + for { + rd, err := iter.Next(ctx) + require.NoError(t, err) + if rd.RangeID == 0 { + break + } + numRanges++ + } + require.True(t, numRanges > 20, "expected over 20 ranges, got: %d", numRanges) + + // Now make an interator with a small page size and check that we get just as many ranges. + iter = makeMeta2RangeIter(db, 2 /* batch size */) + numRangesPaginated := 0 + for { + rd, err := iter.Next(ctx) + require.NoError(t, err) + if rd.RangeID == 0 { + break + } + numRangesPaginated++ + } + require.Equal(t, numRanges, numRangesPaginated) +} + +// Test that a retriable error returned from the range iterator is properly +// handled by resetting the report. +func TestRetriableErrorWhenGenerationReport(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + s, _, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + cfg := s.GossipI().(*gossip.Gossip).GetSystemConfig() + dummyNodeChecker := func(id roachpb.NodeID) bool { return true } + + saver := makeReplicationStatsReportSaver() + v := makeReplicationStatsVisitor(ctx, cfg, dummyNodeChecker, &saver) + realIter := makeMeta2RangeIter(db, 10000 /* batchSize */) + require.NoError(t, visitRanges(ctx, &realIter, cfg, &v)) + expReport := v.report + require.True(t, len(expReport.stats) > 0, "unexpected empty report") + + realIter = makeMeta2RangeIter(db, 10000 /* batchSize */) + errorIter := erroryRangeIterator{ + iter: realIter, + injectErrAfter: 3, + } + v = makeReplicationStatsVisitor(ctx, cfg, func(id roachpb.NodeID) bool { return true }, &saver) + require.NoError(t, visitRanges(ctx, &errorIter, cfg, &v)) + require.True(t, len(v.report.stats) > 0, "unexpected empty report") + require.Equal(t, expReport, v.report) +} + +type erroryRangeIterator struct { + iter meta2RangeIter + rangesReturned int + injectErrAfter int +} + +var _ RangeIterator = &erroryRangeIterator{} + +func (it *erroryRangeIterator) Next(ctx context.Context) (roachpb.RangeDescriptor, error) { + if it.rangesReturned == it.injectErrAfter { + // Don't inject any more errors. + it.injectErrAfter = -1 + + var err error + err = roachpb.NewTransactionRetryWithProtoRefreshError( + "injected err", uuid.Nil, roachpb.Transaction{}) + // Let's wrap the error to check the unwrapping. + err = errors.Wrap(err, "dummy wrapper") + // Feed the error to the underlying iterator to reset it. + it.iter.handleErr(ctx, err) + return roachpb.RangeDescriptor{}, err + } + it.rangesReturned++ + rd, err := it.iter.Next(ctx) + return rd, err +} + +func (it *erroryRangeIterator) Close(ctx context.Context) { + it.iter.Close(ctx) +} + +// computeConstraintConformanceReport iterates through all the ranges and +// generates the constraint conformance report. +func computeConstraintConformanceReport( + ctx context.Context, + rangeStore RangeIterator, + cfg *config.SystemConfig, + storeResolver StoreResolver, +) (*replicationConstraintStatsReportSaver, error) { + saver := makeReplicationConstraintStatusReportSaver() + v := makeConstraintConformanceVisitor(ctx, cfg, storeResolver, &saver) + err := visitRanges(ctx, rangeStore, cfg, &v) + return v.report, err +} diff --git a/pkg/storage/reports/zone_key.go b/pkg/storage/reports/zone_key.go new file mode 100644 index 000000000000..7fdd8ce384d6 --- /dev/null +++ b/pkg/storage/reports/zone_key.go @@ -0,0 +1,64 @@ +// Copyright 2019 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 reports + +import "fmt" + +// ZoneKey is the index of the first level in the constraint conformance report. +type ZoneKey struct { + // ZoneID is the id of the zone this key is referencing. + ZoneID uint32 + // SubzoneID identifies what subzone, if any, this key is referencing. The + // zero value (also named NoSubzone) indicates that the key is referring to a + // zone, not a subzone. + SubzoneID SubzoneID +} + +// NoSubzone is used inside a zoneKey to indicate that the key represents a +// zone, not a subzone. +const NoSubzone SubzoneID = 0 + +// SubzoneID represents a subzone within a zone. It's the subzone's index within +// the parent zone + 1; there's no subzone 0 so that 0 can be used as a +// sentinel. +type SubzoneID uint32 + +// SubzoneIDFromIndex turns a subzone's index within its parent zone into its +// SubzoneID. +func SubzoneIDFromIndex(idx int) SubzoneID { + return SubzoneID(idx + 1) +} + +// MakeZoneKey creates a zoneKey. +// +// Use NoSubzone for subzoneID to indicate that the key references a zone, not a +// subzone. +func MakeZoneKey(zoneID uint32, subzoneID SubzoneID) ZoneKey { + return ZoneKey{ + ZoneID: zoneID, + SubzoneID: subzoneID, + } +} + +func (k ZoneKey) String() string { + return fmt.Sprintf("%d,%d", k.ZoneID, k.SubzoneID) +} + +// Less compares two ZoneKeys. +func (k ZoneKey) Less(other ZoneKey) bool { + if k.ZoneID < other.ZoneID { + return true + } + if k.ZoneID > other.ZoneID { + return false + } + return k.SubzoneID < other.SubzoneID +} diff --git a/pkg/storage/store_pool.go b/pkg/storage/store_pool.go index 3e92901b0391..a081f17b8215 100644 --- a/pkg/storage/store_pool.go +++ b/pkg/storage/store_pool.go @@ -374,6 +374,21 @@ func newStoreDetail() *storeDetail { return &storeDetail{} } +// GetStores returns information on all the stores with descriptor in the pool. +// Stores without descriptor (a node that didn't come up yet after a cluster +// restart) will not be part of the returned set. +func (sp *StorePool) GetStores() map[roachpb.StoreID]roachpb.StoreDescriptor { + sp.detailsMu.RLock() + defer sp.detailsMu.RUnlock() + stores := make(map[roachpb.StoreID]roachpb.StoreDescriptor, len(sp.detailsMu.storeDetails)) + for _, s := range sp.detailsMu.storeDetails { + if s.desc != nil { + stores[s.desc.StoreID] = *s.desc + } + } + return stores +} + // getStoreDetailLocked returns the store detail for the given storeID. // The lock must be held *in write mode* even though this looks like a // read-only method.