From 573f863a68a3dc15c10f936065cd2b7b08053fc6 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Mon, 22 Jul 2019 15:29:58 -0400 Subject: [PATCH] sql: Add support for a show partitions command. SHOW PARTITIONS FROM TABLE SHOW PARTITIONS FROM DATABASE SHOW PARTITIONS FROM INDEX Returns a table containing the following columns. * database_name * table_name * partition_name * parent_partition * column_names * index_name * partition_value * zone_constraints (NULL if no constraints were specified) To do this, changes were made to the crdb_internal.partitions table. Release note (sql change): Add support for a SHOW PARTITIONS command. --- docs/generated/sql/bnf/show_var.bnf | 1 + docs/generated/sql/bnf/stmt_block.bnf | 15 +- .../testdata/logic_test/crdb_internal | 26 ++-- .../testdata/logic_test/distsql_partitioning | 139 ++++++++++++++++++ pkg/sql/crdb_internal.go | 58 +++++++- pkg/sql/delegate/delegate.go | 3 + pkg/sql/delegate/show_partitions.go | 127 ++++++++++++++++ pkg/sql/parser/help_test.go | 2 + pkg/sql/parser/sql.y | 30 +++- pkg/sql/sem/tree/show.go | 24 +++ pkg/sql/sem/tree/stmt.go | 7 + 11 files changed, 410 insertions(+), 22 deletions(-) create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/distsql_partitioning create mode 100644 pkg/sql/delegate/show_partitions.go diff --git a/docs/generated/sql/bnf/show_var.bnf b/docs/generated/sql/bnf/show_var.bnf index 969e0a54ac77..4a009c606745 100644 --- a/docs/generated/sql/bnf/show_var.bnf +++ b/docs/generated/sql/bnf/show_var.bnf @@ -7,6 +7,7 @@ show_stmt ::= | show_databases_stmt | show_grants_stmt | show_indexes_stmt + | show_partitions_stmt | show_jobs_stmt | show_queries_stmt | show_ranges_stmt diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 8857b4b24ca0..0a2a202fc34d 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -174,6 +174,7 @@ show_stmt ::= | show_databases_stmt | show_grants_stmt | show_indexes_stmt + | show_partitions_stmt | show_jobs_stmt | show_queries_stmt | show_ranges_stmt @@ -479,6 +480,11 @@ show_indexes_stmt ::= | 'SHOW' 'KEYS' 'FROM' table_name | 'SHOW' 'KEYS' 'FROM' 'DATABASE' database_name +show_partitions_stmt ::= + 'SHOW' 'PARTITIONS' 'FROM' 'TABLE' table_name + | 'SHOW' 'PARTITIONS' 'FROM' 'DATABASE' database_name + | 'SHOW' 'PARTITIONS' 'FROM' 'INDEX' table_index_name + show_jobs_stmt ::= 'SHOW' opt_automatic 'JOBS' @@ -702,6 +708,7 @@ unreserved_keyword ::= | 'PARENT' | 'PARTIAL' | 'PARTITION' + | 'PARTITIONS' | 'PASSWORD' | 'PAUSE' | 'PHYSICAL' @@ -1142,6 +1149,10 @@ for_grantee_clause ::= 'FOR' name_list | +table_index_name ::= + table_name '@' index_name + | standalone_index_name + opt_automatic ::= 'AUTOMATIC' | @@ -1150,10 +1161,6 @@ opt_cluster ::= 'CLUSTER' | 'LOCAL' -table_index_name ::= - table_name '@' index_name - | standalone_index_name - opt_compact ::= 'COMPACT' | diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal index bbc02cf53d94..4b2b0dafee77 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal @@ -1,9 +1,9 @@ # LogicTest: local -query IITTI colnames +query IITTITTT colnames SELECT * FROM crdb_internal.partitions ---- -table_id index_id parent_name name columns +table_id index_id parent_name name columns column_names list_value range_value statement ok CREATE TABLE t1 ( @@ -33,16 +33,16 @@ CREATE table t2 (a STRING PRIMARY KEY) PARTITION BY LIST (a) ( PARTITION pfoo VALUES IN ('foo') ) -query IITTI +query IITTITTT SELECT * FROM crdb_internal.partitions ORDER BY table_id, index_id, name ---- -53 1 NULL p12 1 -53 1 p12 p12p3 1 -53 1 p12p3 p12p3p8 1 -53 1 NULL p6 1 -53 1 p6 p6p7 1 -53 1 p6 p6p8 1 -53 1 p6 p6px 1 -53 1 p12 pd 1 -53 2 NULL p00 2 -54 1 NULL pfoo 1 +53 1 NULL p12 1 a (1), (2) NULL +53 1 p12 p12p3 1 b (3) NULL +53 1 p12p3 p12p3p8 1 c (8) NULL +53 1 NULL p6 1 a (6) NULL +53 1 p6 p6p7 1 b NULL (MINVALUE) TO (7) +53 1 p6 p6p8 1 b NULL (7) TO (8) +53 1 p6 p6px 1 b NULL (8) TO (MAXVALUE) +53 1 p12 pd 1 b (DEFAULT) NULL +53 2 NULL p00 2 a, b (0, 0) NULL +54 1 NULL pfoo 1 a ('foo') NULL diff --git a/pkg/ccl/logictestccl/testdata/logic_test/distsql_partitioning b/pkg/ccl/logictestccl/testdata/logic_test/distsql_partitioning new file mode 100644 index 000000000000..1225a584813f --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/distsql_partitioning @@ -0,0 +1,139 @@ +# LogicTest: 5node-dist + +# Tests for the show partitions command. + +statement ok +CREATE TABLE t1 (x INT PRIMARY KEY) + +statement ok +ALTER TABLE t1 PARTITION BY LIST (x) ( + PARTITION p1 VALUES IN (1), + PARTITION p2 VALUES IN (2), + PARTITION p3 VALUES IN (3) +) + +statement ok +ALTER PARTITION p1 OF TABLE t1 CONFIGURE ZONE USING constraints='[+dc=dc1]'; +ALTER PARTITION p2 OF TABLE t1 CONFIGURE ZONE USING constraints='[+dc=dc2]'; +ALTER PARTITION p3 OF TABLE t1 CONFIGURE ZONE USING constraints='[+dc=dc3]' + +query TTTTTTTT colnames +SHOW PARTITIONS FROM DATABASE test +---- +database_name table_name partition_name parent_partition column_names index_name partition_value zone_constraints +test t1 p1 NULL x t1@primary (1) [+dc=dc1] +test t1 p2 NULL x t1@primary (2) [+dc=dc2] +test t1 p3 NULL x t1@primary (3) [+dc=dc3] + +query TTTTTTTT +SHOW PARTITIONS FROM TABLE t1 +---- +test t1 p1 NULL x t1@primary (1) [+dc=dc1] +test t1 p2 NULL x t1@primary (2) [+dc=dc2] +test t1 p3 NULL x t1@primary (3) [+dc=dc3] + +query TTTTTTTT +SHOW PARTITIONS FROM INDEX t1@primary +---- +test t1 p1 NULL x t1@primary (1) [+dc=dc1] +test t1 p2 NULL x t1@primary (2) [+dc=dc2] +test t1 p3 NULL x t1@primary (3) [+dc=dc3] + +statement ok +CREATE TABLE t2 (x INT PRIMARY KEY) + +statement ok +ALTER TABLE t2 PARTITION BY RANGE (x) ( + PARTITION p1 VALUES FROM (1) TO (2), + PARTITION p2 VALUES FROM (2) TO (3) +) + +statement ok +ALTER PARTITION p1 OF TABLE t2 CONFIGURE ZONE USING constraints='[+dc=dc1]'; +ALTER PARTITION p2 OF TABLE t2 CONFIGURE ZONE USING constraints='[+dc=dc2]' + +query TTTTTTTT +SHOW PARTITIONS FROM DATABASE test +---- +test t1 p1 NULL x t1@primary (1) [+dc=dc1] +test t1 p2 NULL x t1@primary (2) [+dc=dc2] +test t1 p3 NULL x t1@primary (3) [+dc=dc3] +test t2 p1 NULL x t2@primary (1) TO (2) [+dc=dc1] +test t2 p2 NULL x t2@primary (2) TO (3) [+dc=dc2] + +query TTTTTTTT +SHOW PARTITIONS FROM TABLE t2 +---- +test t2 p1 NULL x t2@primary (1) TO (2) [+dc=dc1] +test t2 p2 NULL x t2@primary (2) TO (3) [+dc=dc2] + +query TTTTTTTT +SHOW PARTITIONS FROM INDEX t2@primary +---- +test t2 p1 NULL x t2@primary (1) TO (2) [+dc=dc1] +test t2 p2 NULL x t2@primary (2) TO (3) [+dc=dc2] + +statement ok +CREATE TABLE t3 (x INT PRIMARY KEY, y INT, INDEX sec (y)) + +statement ok +ALTER TABLE t3 PARTITION BY LIST (x) ( + PARTITION p1 VALUES IN (1), + PARTITION p2 VALUES IN (2) +) + +statement ok +ALTER INDEX sec PARTITION BY LIST (y) ( + PARTITION p3 VALUES IN (3), + PARTITION p4 VALUES IN (4) +) + +statement ok +ALTER PARTITION p1 OF TABLE t3 CONFIGURE ZONE USING constraints='[+dc=dc1]'; +ALTER PARTITION p2 OF TABLE t3 CONFIGURE ZONE USING constraints='[+dc=dc2]'; +ALTER PARTITION p3 OF TABLE t3 CONFIGURE ZONE USING constraints='[+dc=dc3]'; +ALTER PARTITION p4 OF TABLE t3 CONFIGURE ZONE USING constraints='[+dc=dc4]' + +query TTTTTTTT +SHOW PARTITIONS FROM TABLE t3 +---- +test t3 p1 NULL x t3@primary (1) [+dc=dc1] +test t3 p2 NULL x t3@primary (2) [+dc=dc2] +test t3 p3 NULL y t3@sec (3) [+dc=dc3] +test t3 p4 NULL y t3@sec (4) [+dc=dc4] + +query TTTTTTTT +SHOW PARTITIONS FROM INDEX t3@sec +---- +test t3 p3 NULL y t3@sec (3) [+dc=dc3] +test t3 p4 NULL y t3@sec (4) [+dc=dc4] + +statement ok +CREATE TABLE t4 (x INT, y INT, PRIMARY KEY (x, y)) + +statement ok +ALTER TABLE t4 PARTITION BY LIST (x) ( + PARTITION p1 VALUES IN (1) PARTITION BY LIST (y) ( + PARTITION p1_a VALUES in (2), + PARTITION p1_b VALUES IN (3) + ), + PARTITION p2 VALUES IN (4) PARTITION BY LIST (y) ( + PARTITION p2_a VALUES IN (5) + ) +) + +statement ok +ALTER PARTITION p1 OF TABLE t4 CONFIGURE ZONE USING constraints='[+dc=dc1]'; +ALTER PARTITION p1_a OF TABLE t4 CONFIGURE ZONE USING constraints='[+dc=dc2]'; +ALTER PARTITION p1_b OF TABLE t4 CONFIGURE ZONE USING constraints='[+dc=dc3]'; +ALTER PARTITION p2 OF TABLE t4 CONFIGURE ZONE USING constraints='[+dc=dc4]'; +ALTER PARTITION p2_a OF TABLE t4 CONFIGURE ZONE USING constraints='[+dc=dc5]' + +query TTTTTTTT +SHOW PARTITIONS FROM TABLE t4 +---- +test t4 p1 NULL x t4@primary (1) [+dc=dc1] +test t4 p1_a p1 y t4@primary (2) [+dc=dc2] +test t4 p1_b p1 y t4@primary (3) [+dc=dc3] +test t4 p2 NULL x t4@primary (4) [+dc=dc4] +test t4 p2_a p2 y t4@primary (5) [+dc=dc5] diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 0338417df85f..defbe2d6f376 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2265,7 +2265,38 @@ func addPartitioningRows( indexID := tree.NewDInt(tree.DInt(index.ID)) numColumns := tree.NewDInt(tree.DInt(partitioning.NumColumns)) + var buf bytes.Buffer + for i := uint32(colOffset); i < uint32(colOffset)+partitioning.NumColumns; i++ { + if i != uint32(colOffset) { + buf.WriteString(`, `) + } + buf.WriteString(index.ColumnNames[i]) + } + colNames := tree.NewDString(buf.String()) + + var a sqlbase.DatumAlloc + + // We don't need real prefixes in the DecodePartitionTuple calls because we + // only use the tree.Datums part of the output. + fakePrefixDatums := make([]tree.Datum, colOffset) + for i := range fakePrefixDatums { + fakePrefixDatums[i] = tree.DNull + } + for _, l := range partitioning.List { + var buf bytes.Buffer + for j, values := range l.Values { + if j != 0 { + buf.WriteString(`, `) + } + tuple, _, err := sqlbase.DecodePartitionTuple( + &a, table, index, partitioning, values, fakePrefixDatums, + ) + if err != nil { + return err + } + buf.WriteString(tuple.String()) + } name := tree.NewDString(l.Name) if err := addRow( tableID, @@ -2273,6 +2304,9 @@ func addPartitioningRows( parentName, name, numColumns, + colNames, + tree.NewDString(buf.String()), + tree.DNull, ); err != nil { return err } @@ -2284,12 +2318,31 @@ func addPartitioningRows( } for _, r := range partitioning.Range { + var buf bytes.Buffer + fromTuple, _, err := sqlbase.DecodePartitionTuple( + &a, table, index, partitioning, r.FromInclusive, fakePrefixDatums, + ) + if err != nil { + return err + } + buf.WriteString(fromTuple.String()) + buf.WriteString(" TO ") + toTuple, _, err := sqlbase.DecodePartitionTuple( + &a, table, index, partitioning, r.ToExclusive, fakePrefixDatums, + ) + if err != nil { + return err + } + buf.WriteString(toTuple.String()) if err := addRow( tableID, indexID, parentName, tree.NewDString(r.Name), numColumns, + colNames, + tree.DNull, + tree.NewDString(buf.String()), ); err != nil { return err } @@ -2310,7 +2363,10 @@ CREATE TABLE crdb_internal.partitions ( index_id INT NOT NULL, parent_name STRING, name STRING NOT NULL, - columns INT NOT NULL + columns INT NOT NULL, + column_names STRING, + list_value STRING, + range_value STRING ) `, populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { diff --git a/pkg/sql/delegate/delegate.go b/pkg/sql/delegate/delegate.go index f14edec30426..7864aa75a19d 100644 --- a/pkg/sql/delegate/delegate.go +++ b/pkg/sql/delegate/delegate.go @@ -57,6 +57,9 @@ func TryDelegate( case *tree.ShowConstraints: return d.delegateShowConstraints(t) + case *tree.ShowPartitions: + return d.delegateShowPartitions(t) + case *tree.ShowGrants: return d.delegateShowGrants(t) diff --git a/pkg/sql/delegate/show_partitions.go b/pkg/sql/delegate/show_partitions.go new file mode 100644 index 000000000000..3be9e35ae71d --- /dev/null +++ b/pkg/sql/delegate/show_partitions.go @@ -0,0 +1,127 @@ +// 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 delegate + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/lex" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +func (d *delegator) delegateShowPartitions(n *tree.ShowPartitions) (tree.Statement, error) { + if n.IsTable { + flags := cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true} + tn := n.Table.ToTableName() + + dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, flags, &tn) + if err != nil { + return nil, err + } + if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil { + return nil, err + } + + const showTablePartitionsQuery = ` + SELECT + database_name, + tables.name AS table_name, + partitions.name AS partition_name, + partitions.parent_name AS parent_partition, + partitions.column_names, + concat(tables.name, '@', table_indexes.index_name) AS index_name, + coalesce(partitions.list_value, partitions.range_value) as partition_value, + regexp_extract(config_yaml, e'constraints: (\\[.*\\])') AS zone_constraints + FROM + crdb_internal.partitions + JOIN crdb_internal.tables ON partitions.table_id = tables.table_id + JOIN crdb_internal.table_indexes ON + table_indexes.descriptor_id = tables.table_id + AND table_indexes.index_id = partitions.index_id + LEFT JOIN crdb_internal.zones ON + zones.zone_name + = concat(database_name, '.', tables.name, '.', partitions.name) + WHERE + tables.name = %[1]s AND database_name = %[2]s; + ` + return parse(fmt.Sprintf(showTablePartitionsQuery, lex.EscapeSQLString(resName.Table()), lex.EscapeSQLString(resName.Catalog()))) + } else if n.IsDB { + const showDatabasePartitionsQuery = ` + SELECT + database_name, + tables.name AS table_name, + partitions.name AS partition_name, + partitions.parent_name AS parent_partition, + partitions.column_names, + concat(tables.name, '@', table_indexes.index_name) AS index_name, + coalesce(partitions.list_value, partitions.range_value) as partition_value, + regexp_extract(config_yaml, e'constraints: (\\[.*\\])') AS zone_constraints + FROM + %[1]s.crdb_internal.partitions + JOIN %[1]s.crdb_internal.tables ON partitions.table_id = tables.table_id + JOIN %[1]s.crdb_internal.table_indexes ON + table_indexes.descriptor_id = tables.table_id + AND table_indexes.index_id = partitions.index_id + LEFT JOIN %[1]s.crdb_internal.zones ON + zones.zone_name + = concat(database_name, '.', tables.name, '.', partitions.name) + WHERE + database_name = %[2]s + ORDER BY + tables.name, partitions.name; + ` + return parse(fmt.Sprintf(showDatabasePartitionsQuery, n.Object, lex.EscapeSQLString(n.Object))) + } + + flags := cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true} + tn := n.Index.Table + + dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, flags, &tn) + if err != nil { + return nil, err + } + if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil { + return nil, err + } + + // TODO (rohany): The dummy query to force resolution of the index + // is a dirty hack that needs to be fixed. + const showIndexPartitionsQuery = ` + WITH + dummy AS (SELECT * FROM %[3]s@%[4]s LIMIT 0) + SELECT + database_name, + tables.name AS table_name, + partitions.name AS partition_name, + partitions.parent_name AS parent_partition, + partitions.column_names, + concat(tables.name, '@', table_indexes.index_name) AS index_name, + coalesce(partitions.list_value, partitions.range_value) as partition_value, + regexp_extract(config_yaml, e'constraints: (\\[.*\\])') AS zone_constraints + FROM + crdb_internal.partitions + JOIN crdb_internal.table_indexes ON + partitions.index_id = table_indexes.index_id + AND partitions.table_id = table_indexes.descriptor_id + JOIN crdb_internal.tables ON table_indexes.descriptor_id = tables.table_id + LEFT JOIN crdb_internal.zones ON + zones.zone_name + = concat(database_name, '.', tables.name, '.', partitions.name) + WHERE + index_name = %[1]s AND tables.name = %[2]s; + ` + return parse(fmt.Sprintf(showIndexPartitionsQuery, + lex.EscapeSQLString(n.Index.Index.String()), + lex.EscapeSQLString(resName.Table()), + resName.Table(), + n.Index.Index.String())) +} diff --git a/pkg/sql/parser/help_test.go b/pkg/sql/parser/help_test.go index 5d00b0fb2638..b8379612ecf7 100644 --- a/pkg/sql/parser/help_test.go +++ b/pkg/sql/parser/help_test.go @@ -275,6 +275,8 @@ func TestContextualHelp(t *testing.T) { {`SHOW INDEXES FROM ??`, `SHOW INDEXES`}, {`SHOW INDEXES FROM blah ??`, `SHOW INDEXES`}, + {`SHOW PARTITIONS FROM ??`, `SHOW PARTITIONS`}, + {`SHOW ROLES ??`, `SHOW ROLES`}, {`SHOW SCHEMAS FROM ??`, `SHOW SCHEMAS`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 3b78a50c6694..ce00e0d4d782 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -530,7 +530,7 @@ func newNameFromStr(s string) *tree.Name { %token OF OFF OFFSET OID OIDS OIDVECTOR ON ONLY OPT OPTION OPTIONS OR %token ORDER ORDINALITY OUT OUTER OVER OVERLAPS OVERLAY OWNED OPERATOR -%token PARENT PARTIAL PARTITION PASSWORD PAUSE PHYSICAL PLACING +%token PARENT PARTIAL PARTITION PARTITIONS PASSWORD PAUSE PHYSICAL PLACING %token PLAN PLANS POSITION PRECEDING PRECISION PREPARE PRIMARY PRIORITY %token PROCEDURAL PUBLICATION @@ -725,6 +725,7 @@ func newNameFromStr(s string) *tree.Name { %type show_grants_stmt %type show_histogram_stmt %type show_indexes_stmt +%type show_partitions_stmt %type show_jobs_stmt %type show_queries_stmt %type show_ranges_stmt @@ -3190,9 +3191,9 @@ zone_value: // %Text: // SHOW BACKUP, SHOW CLUSTER SETTING, SHOW COLUMNS, SHOW CONSTRAINTS, // SHOW CREATE, SHOW DATABASES, SHOW HISTOGRAM, SHOW INDEXES, SHOW -// JOBS, SHOW QUERIES, SHOW ROLES, SHOW SCHEMAS, SHOW SEQUENCES, SHOW -// SESSION, SHOW SESSIONS, SHOW STATISTICS, SHOW SYNTAX, SHOW TABLES, -// SHOW TRACE SHOW TRANSACTION, SHOW USERS +// PARTITIONS, SHOW JOBS, SHOW QUERIES, SHOW ROLES, SHOW SCHEMAS, +// SHOW SEQUENCES, SHOW SESSION, SHOW SESSIONS, SHOW STATISTICS, +// SHOW SYNTAX, SHOW TABLES, SHOW TRACE SHOW TRANSACTION, SHOW USERS show_stmt: show_backup_stmt // EXTEND WITH HELP: SHOW BACKUP | show_columns_stmt // EXTEND WITH HELP: SHOW COLUMNS @@ -3204,6 +3205,7 @@ show_stmt: | show_grants_stmt // EXTEND WITH HELP: SHOW GRANTS | show_histogram_stmt // EXTEND WITH HELP: SHOW HISTOGRAM | show_indexes_stmt // EXTEND WITH HELP: SHOW INDEXES +| show_partitions_stmt // EXTEND WITH HELP: SHOW PARTITIONS | show_jobs_stmt // EXTEND WITH HELP: SHOW JOBS | show_queries_stmt // EXTEND WITH HELP: SHOW QUERIES | show_ranges_stmt // EXTEND WITH HELP: SHOW RANGES @@ -3349,6 +3351,25 @@ show_columns_stmt: } | SHOW COLUMNS error // SHOW HELP: SHOW COLUMNS +// %Help: SHOW PARTITIONS - list partition information +// %Category: DDL +// %Text: SHOW PARTITIONS FROM { TABLE
| INDEX | DATABASE } +// %SeeAlso: WEBDOCS/show-partitions.html +show_partitions_stmt: + SHOW PARTITIONS FROM TABLE table_name + { + $$.val = &tree.ShowPartitions{Object: $5.unresolvedObjectName().String(), IsTable: true, Table: $5.unresolvedObjectName()} + } +| SHOW PARTITIONS FROM DATABASE database_name + { + $$.val = &tree.ShowPartitions{Object: $5, IsDB: true} + } +| SHOW PARTITIONS FROM INDEX table_index_name + { + $$.val = &tree.ShowPartitions{Object: $5.newTableIndexName().String(), IsIndex: true, Index: $5.tableIndexName()} + } +| SHOW PARTITIONS error // SHOW HELP: SHOW PARTITIONS + // %Help: SHOW DATABASES - list databases // %Category: DDL // %Text: SHOW DATABASES @@ -9129,6 +9150,7 @@ unreserved_keyword: | PARENT | PARTIAL | PARTITION +| PARTITIONS | PASSWORD | PAUSE | PHYSICAL diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 2598e5e1d53b..83002013f21f 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -425,3 +425,27 @@ type ShowHistogram struct { func (node *ShowHistogram) Format(ctx *FmtCtx) { ctx.Printf("SHOW HISTOGRAM %d", node.HistogramID) } + +// ShowPartitions represents a SHOW PARTITIONS statement. +type ShowPartitions struct { + Object string + + IsDB bool + + IsIndex bool + Index TableIndexName + + IsTable bool + Table *UnresolvedObjectName +} + +// Format implements the NodeFormatter interface. +func (node *ShowPartitions) Format(ctx *FmtCtx) { + if node.IsDB { + ctx.Printf("SHOW PARTITIONS FROM DATABASE %s", node.Object) + } else if node.IsIndex { + ctx.Printf("SHOW PARTITIONS FROM INDEX %s", node.Object) + } else { + ctx.Printf("SHOW PARTITIONS FROM TABLE %s", node.Object) + } +} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 84ffdf42078e..b684b45b244d 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -687,6 +687,12 @@ func (*ShowIndexes) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowIndexes) StatementTag() string { return "SHOW INDEXES FROM TABLE" } +// StatementType implements the Statement interface. +func (*ShowPartitions) StatementType() StatementType { return Rows } + +// StatementTag returns a short string identifying the type of the statement. +func (*ShowPartitions) StatementTag() string { return "SHOW PARTITIONS" } + // StatementType implements the Statement interface. func (*ShowQueries) StatementType() StatementType { return Rows } @@ -917,6 +923,7 @@ func (n *ShowDatabaseIndexes) String() string { return AsString(n) } func (n *ShowGrants) String() string { return AsString(n) } func (n *ShowHistogram) String() string { return AsString(n) } func (n *ShowIndexes) String() string { return AsString(n) } +func (n *ShowPartitions) String() string { return AsString(n) } func (n *ShowJobs) String() string { return AsString(n) } func (n *ShowQueries) String() string { return AsString(n) } func (n *ShowRanges) String() string { return AsString(n) }