Skip to content

Commit

Permalink
sql: make SHOW RANGES FOR TABLE include all indexes
Browse files Browse the repository at this point in the history
Release note (backward-incompatible change): `SHOW RANGES FOR TABLE`
now includes rows for all indexes that support the table. Prior to
this change, `SHOW RANGES FOR TABLE foo` was an alias for `SHOW RANGES
FOR INDEX foo@primary`. This was causing confusion, as it would miss
data for secondary indexes. It is still possible to filter to just the
primary index using `SHOW RANGES FOR INDEX foo@primary`.

The statement output now also includes the index name.
  • Loading branch information
knz committed Dec 14, 2022
1 parent dc9cfae commit 5604fea
Show file tree
Hide file tree
Showing 10 changed files with 71 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -392,7 +392,7 @@ ALTER TABLE regional_by_row_table SPLIT AT VALUES ('ca-central-1', 0), ('us-east
ALTER TABLE regional_by_row_table EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'ap-southeast-2', 0), (ARRAY[4], 'ca-central-1', 0), (ARRAY[7], 'us-east-1', 0);

query TTTI colnames,rowsort
SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE regional_by_row_table]
SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX regional_by_row_table@primary]
----
start_key end_key replicas lease_holder
NULL /"\x80"/0 {1} 1
Expand Down Expand Up @@ -640,7 +640,7 @@ ALTER TABLE child SPLIT AT VALUES ('ca-central-1', 0), ('us-east-1', 0);
ALTER TABLE child EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'ap-southeast-2', 0), (ARRAY[4], 'ca-central-1', 0), (ARRAY[7], 'us-east-1', 0);

query TTTI colnames,rowsort
SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE child]
SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX child@primary]
----
start_key end_key replicas lease_holder
NULL /"\x80"/0 {1} 1
Expand Down
23 changes: 19 additions & 4 deletions pkg/sql/delegate/show_ranges.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,9 @@ func (d *delegator) delegateShowRanges(n *tree.ShowRanges) (tree.Statement, erro
return parse(fmt.Sprintf(dbQuery, n.DatabaseName.String(), lexbase.EscapeSQLString(string(n.DatabaseName))))
}

// Remember the original syntax: Resolve below modifies the TableOrIndex struct in-place.
noIndexSpecified := n.TableOrIndex.Index == ""

idx, resName, err := cat.ResolveTableIndex(
d.ctx, d.catalog, cat.Flags{AvoidDescriptorCaches: true}, &n.TableOrIndex,
)
Expand All @@ -94,13 +97,25 @@ func (d *delegator) delegateShowRanges(n *tree.ShowRanges) (tree.Statement, erro
return nil, errors.New("SHOW RANGES may not be called on a virtual table")
}

span := idx.Span()
startKey := hex.EncodeToString(span.Key)
endKey := hex.EncodeToString(span.EndKey)
var startKey, endKey string
if noIndexSpecified {
// All indexes.
tableID := idx.Table().ID()
prefix := d.evalCtx.Codec.TablePrefix(uint32(tableID))
startKey = hex.EncodeToString(prefix)
endKey = hex.EncodeToString(prefix.PrefixEnd())
} else {
// Just one index.
span := idx.Span()
startKey = hex.EncodeToString(span.Key)
endKey = hex.EncodeToString(span.EndKey)
}

return parse(fmt.Sprintf(`
SELECT
CASE WHEN r.start_key <= x'%[1]s' THEN NULL ELSE crdb_internal.pretty_key(r.start_key, 2) END AS start_key,
CASE WHEN r.end_key >= x'%[2]s' THEN NULL ELSE crdb_internal.pretty_key(r.end_key, 2) END AS end_key,
index_name,
range_id,
range_size / 1000000 as range_size_mb,
lease_holder,
Expand All @@ -109,7 +124,7 @@ SELECT
replica_localities
FROM %[3]s.crdb_internal.ranges AS r
WHERE (r.start_key < x'%[2]s')
AND (r.end_key > x'%[1]s') ORDER BY r.start_key
AND (r.end_key > x'%[1]s') ORDER BY index_name, r.start_key
`,
startKey, endKey, resName.CatalogName.String(), // note: CatalogName.String() != Catalog()
))
Expand Down
Binary file modified pkg/sql/logictest/testdata/logic_test/ranges
Binary file not shown.
2 changes: 1 addition & 1 deletion pkg/sql/multitenant_admin_function_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -367,7 +367,7 @@ func TestTruncateTable(t *testing.T) {
require.NoErrorf(t, err, "tenant=%s", tenant)
_, err = db.ExecContext(ctx, "TRUNCATE TABLE t;")
require.NoErrorf(t, err, "tenant=%s", tenant)
rows, err := db.QueryContext(ctx, "SELECT start_key, end_key from [SHOW RANGES FROM TABLE t];")
rows, err := db.QueryContext(ctx, "SELECT start_key, end_key from [SHOW RANGES FROM INDEX t@primary];")
require.NoErrorf(t, err, "tenant=%s", tenant)
verifyResults(t, tenant, rows, expectedResults)
}
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -29,23 +29,23 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1,
query TTTI rowsort
SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv]
----
NULL /1 {1} 1
/1 /2 {1} 1
/2 /3 {2} 2
/3 /4 {3} 3
/4 /5 {4} 4
/5 NULL {5} 5
NULL /1 {1} 1
/1 /2 {1} 1
/2 /3 {2} 2
/3 /4 {3} 3
/4 /5 {4} 4
/5 NULL {5} 5

# Verify data placement.
query TTTI rowsort
SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw]
----
NULL /1 {5} 5
/1 /2 {1} 1
/2 /3 {2} 2
/3 /4 {3} 3
/4 /5 {4} 4
/5 NULL {5} 5
NULL /1 {5} 5
/1 /2 {1} 1
/2 /3 {2} 2
/3 /4 {3} 3
/4 /5 {4} 4
/5 NULL {5} 5

# Verify that EXPLAIN ANALYZE (DISTSQL) works in a distributed setting.
query T
Expand Down
23 changes: 21 additions & 2 deletions pkg/sql/opt/exec/execbuilder/testdata/distsql_inverted_index
Original file line number Diff line number Diff line change
Expand Up @@ -63,13 +63,23 @@ ALTER TABLE json_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10),

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE json_tab] ORDER BY lease_holder
FROM [SHOW RANGES FROM INDEX json_tab@primary] ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /10 {1} 1
/10 /20 {2} 2
/20 NULL {3} 3

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW RANGES FROM TABLE json_tab] ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /10 {1} 1
· NULL {1} 1
/10 /20 {2} 2
/20 · {3} 3

# Filter with a fully-specified array. This should use a zigzag join.
query T
EXPLAIN (DISTSQL)
Expand Down Expand Up @@ -193,12 +203,21 @@ ALTER TABLE array_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 3),

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE array_tab] ORDER BY lease_holder
FROM [SHOW RANGES FROM INDEX array_tab@primary] ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /3 {1} 1
/3 NULL {3} 3

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW RANGES FROM TABLE array_tab] ORDER BY lease_holder
----
start_key end_key replicas lease_holder
NULL /3 {1} 1
· NULL {1} 1
/3 · {3} 3

# This should use a zigzag join.
query T
EXPLAIN (DISTSQL)
Expand Down
11 changes: 9 additions & 2 deletions pkg/sql/opt/exec/execbuilder/testdata/distsql_tighten_spans
Original file line number Diff line number Diff line change
Expand Up @@ -94,15 +94,19 @@ ALTER TABLE p2 EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1,3

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p1]
ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /2 {1} 1
/2 NULL {3} 3
/2 /0 {3} 3
/0 /2 {4} 4
/2 NULL {5} 5

# Indexes

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM INDEX b]
ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /0 {3} 3
Expand All @@ -111,11 +115,14 @@ NULL /0 {3} 3

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p2]
ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /0 {1} 1
· /2 {1} 1
/0 /2 {1} 1
/2 NULL {3} 3
/2 NULL {1} 1
/2 · {3} 3

###############
# Query tests #
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,8 @@ NULL /1152921574000000000 {2} 2
/1152921574000000000 NULL {2} 2

query ITTTI colnames,rowsort
SELECT range_id, start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE geo_table]
SELECT range_id, start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@primary]
ORDER BY lease_holder, start_key
----
range_id start_key end_key replicas lease_holder
54 NULL NULL {2} 2
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ ALTER TABLE json_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10),

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE json_tab] ORDER BY lease_holder
FROM [SHOW RANGES FROM INDEX json_tab@primary] ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /10 {1} 1
Expand Down Expand Up @@ -287,7 +287,7 @@ ALTER TABLE array_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 3),

query TTTI colnames
SELECT start_key, end_key, replicas, lease_holder
FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE array_tab] ORDER BY lease_holder
FROM [SHOW RANGES FROM INDEX array_tab@primary] ORDER BY lease_holder, start_key
----
start_key end_key replicas lease_holder
NULL /3 {1} 1
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/ttl/ttljob/ttljob_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -506,11 +506,11 @@ func TestRowLevelTTLJobMultipleNodes(t *testing.T) {

// Split table
ranges := sqlDB.QueryStr(t, fmt.Sprintf(
`SHOW RANGES FROM TABLE %s`,
`SELECT lease_holder FROM [SHOW RANGES FROM INDEX %s@primary]`,
tableName,
))
require.Equal(t, 1, len(ranges))
leaseHolderNodeIDInt, err := strconv.Atoi(ranges[0][4])
leaseHolderNodeIDInt, err := strconv.Atoi(ranges[0][0])
leaseHolderNodeID := roachpb.NodeID(leaseHolderNodeIDInt)
require.NoError(t, err)
leaseHolderServerIdx := -1
Expand Down Expand Up @@ -559,7 +559,7 @@ func TestRowLevelTTLJobMultipleNodes(t *testing.T) {
)
testCluster.SplitTable(t, tableDesc, splitPoints)
newRanges := sqlDB.QueryStr(t, fmt.Sprintf(
`SHOW RANGES FROM TABLE %s`,
`SHOW RANGES FROM INDEX %s@primary`,
tableName,
))
require.Equal(t, numRanges, len(newRanges))
Expand Down

0 comments on commit 5604fea

Please sign in to comment.