Skip to content

Commit

Permalink
Merge #59839 #61912
Browse files Browse the repository at this point in the history
59839: sql: include non-voters in crdb_internal.ranges{_no_leases} r=aayushshah15 a=aayushshah15

Before this commit, the `replicas` column of
`crdb_internal.ranges_no_leases` would only included voting replicas.
This commit makes it such that it includes voting as well as non-voting
replicas.

Additionally, this commit also adds two new columns: `voting_replicas`
and `non_voting_replicas`.

Release justification: observability improvements for new functionality.

Release note: the `replicas` column of
`crdb_internal.ranges{_no_leases}` now includes both voting and
non-voting replicas and `crdb_internal.ranges{_no_leases}` include two
new columns: `voting_replicas` and `non_voting_replicas` which work as
labelled.

61912: builtins: fix json_extract_path_text NULL bug r=RichardJCai a=rafiss

If a null JSON is returned, then the text representation is a nil
pointer, which was not handled correctly.

No release note since this bug only exists in non-released versions.

Release note: None

Co-authored-by: Aayush Shah <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
3 people committed Mar 12, 2021
3 parents 309cca6 + 05fa9fd + eed2c7a commit 1b6c7f9
Show file tree
Hide file tree
Showing 6 changed files with 70 additions and 18 deletions.
21 changes: 16 additions & 5 deletions pkg/kv/kvserver/replicate_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -291,14 +291,24 @@ func TestReplicateQueueDownReplicate(t *testing.T) {
}

func scanAndGetNumNonVoters(
t *testing.T, tc *testcluster.TestCluster, store *kvserver.Store, scratchKey roachpb.Key,
) int {
ctx context.Context,
t *testing.T,
tc *testcluster.TestCluster,
store *kvserver.Store,
scratchKey roachpb.Key,
) (numNonVoters int) {
// Nudge the replicateQueue to up/down-replicate our scratch range.
if err := store.ForceReplicationScanAndProcess(); err != nil {
t.Fatal(err)
}
scratchRange := tc.LookupRangeOrFatal(t, scratchKey)
return len(scratchRange.Replicas().NonVoterDescriptors())
row := tc.ServerConn(0).QueryRow(
`SELECT array_length(non_voting_replicas, 1) FROM crdb_internal.ranges_no_leases WHERE range_id=$1`,
scratchRange.GetRangeID())
err := row.Scan(&numNonVoters)
log.Warningf(ctx, "error while retrieving the number of non-voters: %s", err)

return numNonVoters
}

// TestReplicateQueueUpAndDownReplicateNonVoters is an end-to-end test ensuring
Expand All @@ -309,6 +319,7 @@ func TestReplicateQueueUpAndDownReplicateNonVoters(t *testing.T) {
skip.UnderRace(t)
defer log.Scope(t).Close(t)

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{ReplicationMode: base.ReplicationAuto},
)
Expand Down Expand Up @@ -336,7 +347,7 @@ func TestReplicateQueueUpAndDownReplicateNonVoters(t *testing.T) {

var expectedNonVoterCount = 2
testutils.SucceedsSoon(t, func() error {
if found := scanAndGetNumNonVoters(t, tc, store, scratchKey); found != expectedNonVoterCount {
if found := scanAndGetNumNonVoters(ctx, t, tc, store, scratchKey); found != expectedNonVoterCount {
return errors.Errorf("expected upreplication to %d non-voters; found %d",
expectedNonVoterCount, found)
}
Expand All @@ -349,7 +360,7 @@ func TestReplicateQueueUpAndDownReplicateNonVoters(t *testing.T) {
require.NoError(t, err)
expectedNonVoterCount = 0
testutils.SucceedsSoon(t, func() error {
if found := scanAndGetNumNonVoters(t, tc, store, scratchKey); found != expectedNonVoterCount {
if found := scanAndGetNumNonVoters(ctx, t, tc, store, scratchKey); found != expectedNonVoterCount {
return errors.Errorf("expected downreplication to %d non-voters; found %d",
expectedNonVoterCount, found)
}
Expand Down
40 changes: 32 additions & 8 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -2619,6 +2619,8 @@ CREATE VIEW crdb_internal.ranges AS SELECT
index_name,
replicas,
replica_localities,
voting_replicas,
non_voting_replicas,
learner_replicas,
split_enforced_until,
crdb_internal.lease_holder(start_key) AS lease_holder,
Expand All @@ -2639,6 +2641,8 @@ FROM crdb_internal.ranges_no_leases
{Name: "index_name", Typ: types.String},
{Name: "replicas", Typ: types.Int2Vector},
{Name: "replica_localities", Typ: types.StringArray},
{Name: "voting_replicas", Typ: types.Int2Vector},
{Name: "non_voting_replicas", Typ: types.Int2Vector},
{Name: "learner_replicas", Typ: types.Int2Vector},
{Name: "split_enforced_until", Typ: types.Timestamp},
{Name: "lease_holder", Typ: types.Int},
Expand All @@ -2652,6 +2656,9 @@ FROM crdb_internal.ranges_no_leases
// TODO(tbg): prefix with kv_.
var crdbInternalRangesNoLeasesTable = virtualSchemaTable{
comment: `range metadata without leaseholder details (KV join; expensive!)`,
// NB 1: The `replicas` column is the union of `voting_replicas` and
// `non_voting_replicas` and does not include `learner_replicas`.
// NB 2: All the values in the `*replicas` columns correspond to store IDs.
schema: `
CREATE TABLE crdb_internal.ranges_no_leases (
range_id INT NOT NULL,
Expand All @@ -2666,8 +2673,10 @@ CREATE TABLE crdb_internal.ranges_no_leases (
index_name STRING NOT NULL,
replicas INT[] NOT NULL,
replica_localities STRING[] NOT NULL,
learner_replicas INT[] NOT NULL,
split_enforced_until TIMESTAMP
voting_replicas INT[] NOT NULL,
non_voting_replicas INT[] NOT NULL,
learner_replicas INT[] NOT NULL,
split_enforced_until TIMESTAMP
)
`,
generator: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
Expand Down Expand Up @@ -2736,21 +2745,34 @@ CREATE TABLE crdb_internal.ranges_no_leases (
return nil, err
}

voterReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().VoterDescriptors()...)
votersAndNonVoters := append([]roachpb.ReplicaDescriptor(nil),
desc.Replicas().VoterAndNonVoterDescriptors()...)
var learnerReplicaStoreIDs []int
for _, rd := range desc.Replicas().LearnerDescriptors() {
learnerReplicaStoreIDs = append(learnerReplicaStoreIDs, int(rd.StoreID))
}
sort.Slice(voterReplicas, func(i, j int) bool {
return voterReplicas[i].StoreID < voterReplicas[j].StoreID
sort.Slice(votersAndNonVoters, func(i, j int) bool {
return votersAndNonVoters[i].StoreID < votersAndNonVoters[j].StoreID
})
sort.Ints(learnerReplicaStoreIDs)
votersAndNonVotersArr := tree.NewDArray(types.Int)
for _, replica := range votersAndNonVoters {
if err := votersAndNonVotersArr.Append(tree.NewDInt(tree.DInt(replica.StoreID))); err != nil {
return nil, err
}
}
votersArr := tree.NewDArray(types.Int)
for _, replica := range voterReplicas {
for _, replica := range desc.Replicas().VoterDescriptors() {
if err := votersArr.Append(tree.NewDInt(tree.DInt(replica.StoreID))); err != nil {
return nil, err
}
}
nonVotersArr := tree.NewDArray(types.Int)
for _, replica := range desc.Replicas().NonVoterDescriptors() {
if err := nonVotersArr.Append(tree.NewDInt(tree.DInt(replica.StoreID))); err != nil {
return nil, err
}
}
learnersArr := tree.NewDArray(types.Int)
for _, replica := range learnerReplicaStoreIDs {
if err := learnersArr.Append(tree.NewDInt(tree.DInt(replica))); err != nil {
Expand All @@ -2759,7 +2781,7 @@ CREATE TABLE crdb_internal.ranges_no_leases (
}

replicaLocalityArr := tree.NewDArray(types.String)
for _, replica := range voterReplicas {
for _, replica := range votersAndNonVoters {
replicaLocality := nodeIDToLocality[replica.NodeID].String()
if err := replicaLocalityArr.Append(tree.NewDString(replicaLocality)); err != nil {
return nil, err
Expand Down Expand Up @@ -2806,8 +2828,10 @@ CREATE TABLE crdb_internal.ranges_no_leases (
tree.NewDString(schemaName),
tree.NewDString(tableName),
tree.NewDString(indexName),
votersArr,
votersAndNonVotersArr,
replicaLocalityArr,
votersArr,
nonVotersArr,
learnersArr,
splitEnforcedUntil,
}, nil
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -269,15 +269,15 @@ SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0
----
trace_id parent_span_id span_id goroutine_id finished start_time duration operation

query ITTTTITTTTTTTTTI colnames
query ITTTTITTTTTTTTTTTI colnames
SELECT * FROM crdb_internal.ranges WHERE range_id < 0
----
range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities learner_replicas split_enforced_until lease_holder range_size
range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until lease_holder range_size

query ITTTTITTTTTTTT colnames
query ITTTTITTTTTTTTTT colnames
SELECT * FROM crdb_internal.ranges_no_leases WHERE range_id < 0
----
range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities learner_replicas split_enforced_until
range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until

statement ok
INSERT INTO system.zones (id, config) VALUES
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/create_statements
Original file line number Diff line number Diff line change
Expand Up @@ -806,7 +806,7 @@ CREATE TABLE crdb_internal.predefined_comments (
sub_id INT8 NULL,
comment STRING NULL
) {} {}
CREATE VIEW crdb_internal.ranges (range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, learner_replicas, split_enforced_until, lease_holder, range_size) AS SELECT range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8 AS range_size FROM crdb_internal.ranges_no_leases CREATE VIEW crdb_internal.ranges (range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, learner_replicas, split_enforced_until, lease_holder, range_size) AS SELECT range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8 AS range_size FROM crdb_internal.ranges_no_leases {} {}
CREATE VIEW crdb_internal.ranges (range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, lease_holder, range_size) AS SELECT range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8 AS range_size FROM crdb_internal.ranges_no_leases CREATE VIEW crdb_internal.ranges (range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, lease_holder, range_size) AS SELECT range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8 AS range_size FROM crdb_internal.ranges_no_leases {} {}
CREATE TABLE crdb_internal.ranges_no_leases (
range_id INT8 NOT NULL,
start_key BYTES NOT NULL,
Expand All @@ -820,6 +820,8 @@ CREATE TABLE crdb_internal.ranges_no_leases (
index_name STRING NOT NULL,
replicas INT8[] NOT NULL,
replica_localities STRING[] NOT NULL,
voting_replicas INT8[] NOT NULL,
non_voting_replicas INT8[] NOT NULL,
learner_replicas INT8[] NOT NULL,
split_enforced_until TIMESTAMP NULL
) CREATE TABLE crdb_internal.ranges_no_leases (
Expand All @@ -835,6 +837,8 @@ CREATE TABLE crdb_internal.ranges_no_leases (
index_name STRING NOT NULL,
replicas INT8[] NOT NULL,
replica_localities STRING[] NOT NULL,
voting_replicas INT8[] NOT NULL,
non_voting_replicas INT8[] NOT NULL,
learner_replicas INT8[] NOT NULL,
split_enforced_until TIMESTAMP NULL
) {} {}
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/json_builtins
Original file line number Diff line number Diff line change
Expand Up @@ -614,6 +614,11 @@ SELECT json_extract_path('{"a": {"b": 2}}', 'a', 'b', 'c')
----
NULL

query T
SELECT json_extract_path('null')
----
null

query T
SELECT json_extract_path_text('{"a": 1}', 'a')
----
Expand Down Expand Up @@ -649,6 +654,11 @@ SELECT json_extract_path_text('{"a": {"b": 2}}', 'a', 'b', 'c')
----
NULL

query T
SELECT json_extract_path_text('null')
----
NULL

query T
SELECT jsonb_pretty('{"a": 1}')
----
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -5588,6 +5588,9 @@ var jsonExtractPathTextImpl = tree.Overload{
if err != nil {
return nil, err
}
if text == nil {
return tree.DNull, nil
}
return tree.NewDString(*text), nil
},
Info: "Returns the JSON value as text pointed to by the variadic arguments.",
Expand Down

0 comments on commit 1b6c7f9

Please sign in to comment.