Skip to content

Commit

Permalink
Merge #49816 #49837
Browse files Browse the repository at this point in the history
49816: sql: add support for ANALYZE <tablename> r=rytaft a=rytaft

This commit adds support for `ANALYZE <tablename>` by adding the
statement as syntactic sugar for the equivalent command
`CREATE STATISTICS "" FROM <tablename>`. This improves compatibility
with Postgres, and is needed to run the PostGIS tutorial as written.

Note that this commit does not add support for `ANALYZE` without a
table name. We can add support for that and other variants later if
needed, but it is not necessary for the PostGIS tutorial.

Fixes #49214

Release note (sql change): Added support for `ANALYZE <tablename>`,
which causes the database to collect statistics on the given table for
use by the optimizer. The functionality of this command is equivalent
to the existing command `CREATE STATISTICS "" FROM <tablename>`, but
it increases compatibility with Postgres by using the same syntax that
Postgres uses.

49837: docs: enhance the contributor guide r=otan a=knz

The contributor guide only contained a URL to the wiki, and was
missing searchable keywords.

This patch adds that, as well as a link to the community slack.

Release note: None

Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
3 people committed Jun 3, 2020
3 parents 59b7964 + 3105d70 + 64f02af commit d8d001b
Show file tree
Hide file tree
Showing 12 changed files with 168 additions and 7 deletions.
24 changes: 22 additions & 2 deletions CONTRIBUTING.md
Original file line number Diff line number Diff line change
@@ -1,5 +1,25 @@
# Contributing to Cockroach

Our contributor guidelines are available on the public wiki here:
https://wiki.crdb.io/wiki/spaces/CRDB/pages/73204033/Contributing+to+CockroachDB
Our contributor guidelines are available on [the public wiki at
**wiki.crdb.io**](https://wiki.crdb.io/wiki/spaces/CRDB/pages/73204033/Contributing+to+CockroachDB).

At this location, we share our team guidelines and knowledge base
regarding:

- repository layout
- how to build from source
- how to organize your code change
- commenting guidelines
- commit message guidelines
- code style guidelines
- how to write and run tests
- how to write release notes
- how to submit a change for review
- how to use continuous integration (CI)
- how to troubleshoot certain issues

as well as many other practical topics.

If you have any questions, hop into our [CockroachDB Community
Slack](https://cockroa.ch/slack), in particular the **#contributors**
channel where you can ask questions to the CockroachDB team directly.
12 changes: 9 additions & 3 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -75,14 +75,20 @@ CockroachDB supports the PostgreSQL wire protocol, so you can use any available
- For filing bugs, suggesting improvements, or requesting new features, help us out by
[opening an issue](https://github.com/cockroachdb/cockroach/issues/new).

## Building from source

See [our wiki](https://wiki.crdb.io/wiki/spaces/CRDB/pages/181338446/Getting+and+building+from+source) for more details.

## Contributing

We welcome your contributions! If you're looking for issues to work on, try
looking at the [good first issue](https://github.com/cockroachdb/cockroach/issues?q=is%3Aopen+is%3Aissue+label%3A%22good+first+issue%22)
list. We do our best to tag issues suitable for new external contributors with
that label, so it's a great way to find something you can help with! See our
[Wiki](https://wiki.crdb.io/wiki/spaces/CRDB/pages/73204033/Contributing+to+CockroachDB)
for more details.
that label, so it's a great way to find something you can help with!

See [our
wiki](https://wiki.crdb.io/wiki/spaces/CRDB/pages/73204033/Contributing+to+CockroachDB)
for more details.

Engineering discussion takes place on our public mailing list,
[cockroach-db@googlegroups.com](https://groups.google.com/forum/#!forum/cockroach-db),
Expand Down
8 changes: 8 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ stmt_block ::=
stmt ::=
'HELPTOKEN'
| preparable_stmt
| analyze_stmt
| copy_from_stmt
| comment_stmt
| execute_stmt
Expand Down Expand Up @@ -42,6 +43,10 @@ preparable_stmt ::=
| update_stmt
| upsert_stmt

analyze_stmt ::=
'ANALYZE' analyze_target
| 'ANALYSE' analyze_target

copy_from_stmt ::=
'COPY' table_name opt_column_list 'FROM' 'STDIN' opt_with_options

Expand Down Expand Up @@ -206,6 +211,9 @@ update_stmt ::=
upsert_stmt ::=
opt_with_clause 'UPSERT' 'INTO' insert_target insert_rest returning_clause

analyze_target ::=
table_name

table_name ::=
db_object_name

Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,14 @@ func (p *planner) CreateStatistics(ctx context.Context, n *tree.CreateStats) (pl
}, nil
}

// Analyze is syntactic sugar for CreateStatistics.
func (p *planner) Analyze(ctx context.Context, n *tree.Analyze) (planNode, error) {
return &createStatsNode{
CreateStats: tree.CreateStats{Table: n.Table},
p: p,
}, nil
}

// createStatsNode is a planNode implemented in terms of a function. The
// startJob function starts a Job during Start, and the remainder of the
// CREATE STATISTICS planning and execution is performed within the jobs
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -107,9 +107,36 @@ NULL {b} 256 4 0 true
let $json_stats
SHOW STATISTICS USING JSON FOR TABLE data

# ANALYZE is syntactic sugar for CREATE STATISTICS with default columns.
statement ok
ANALYZE data

query TTIIIB colnames
SELECT
statistics_name,
column_names,
row_count,
distinct_count,
null_count,
histogram_id IS NOT NULL AS has_histogram
FROM
[SHOW STATISTICS FOR TABLE data];
----
statistics_name column_names row_count distinct_count null_count has_histogram
NULL {a} 256 4 0 true
NULL {a,b} 256 16 0 false
NULL {a,b,c} 256 64 0 false
NULL {a,b,c,d} 256 256 0 false
NULL {c} 256 4 0 true
NULL {c,d} 256 16 0 false
NULL {b} 256 4 0 false
NULL {d} 256 4 0 false
NULL {e} 256 2 0 true

statement ok
DELETE FROM system.table_statistics

# Restore the old stats.
statement ok
ALTER TABLE data INJECT STATISTICS '$json_stats'

Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ func buildOpaque(
plan, err = p.AlterRole(ctx, n)
case *tree.AlterSequence:
plan, err = p.AlterSequence(ctx, n)
case *tree.Analyze:
plan, err = p.Analyze(ctx, n)
case *tree.CommentOnColumn:
plan, err = p.CommentOnColumn(ctx, n)
case *tree.CommentOnDatabase:
Expand Down Expand Up @@ -169,6 +171,7 @@ func init() {
&tree.AlterType{},
&tree.AlterSequence{},
&tree.AlterRole{},
&tree.Analyze{},
&tree.CommentOnColumn{},
&tree.CommentOnDatabase{},
&tree.CommentOnIndex{},
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/parser/help_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,11 @@ func TestContextualHelp(t *testing.T) {
{`ALTER PARTITION ??`, `ALTER PARTITION`},
{`ALTER PARTITION p OF INDEX tbl@idx ??`, `ALTER PARTITION`},

{`ANALYZE ??`, `ANALYZE`},
{`ANALYZE blah ??`, `ANALYZE`},
{`ANALYSE ??`, `ANALYZE`},
{`ANALYSE blah ??`, `ANALYZE`},

{`CANCEL ??`, `CANCEL`},
{`CANCEL JOB ??`, `CANCEL JOBS`},
{`CANCEL JOBS ??`, `CANCEL JOBS`},
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,13 +337,18 @@ func TestParse(t *testing.T) {

{`CREATE STATISTICS a ON col1 FROM t`},
{`EXPLAIN CREATE STATISTICS a ON col1 FROM t`},
{`CREATE STATISTICS a FROM t`},
{`CREATE STATISTICS a FROM [53]`},
{`CREATE STATISTICS a ON col1, col2 FROM t`},
{`CREATE STATISTICS a ON col1 FROM d.t`},
{`CREATE STATISTICS a ON col1 FROM t`},
{`CREATE STATISTICS a ON col1 FROM t WITH OPTIONS THROTTLING 0.9`},
{`CREATE STATISTICS a ON col1 FROM t WITH OPTIONS AS OF SYSTEM TIME '2016-01-01'`},
{`CREATE STATISTICS a ON col1 FROM t WITH OPTIONS THROTTLING 0.1 AS OF SYSTEM TIME '2016-01-01'`},

{`ANALYZE t`},
{`ANALYZE db.sc.t`},

{`CREATE TYPE a AS ENUM ()`},
{`CREATE TYPE a AS ENUM ('a')`},
{`CREATE TYPE a AS ENUM ('a', 'b', 'c')`},
Expand Down Expand Up @@ -516,6 +521,7 @@ func TestParse(t *testing.T) {
{`SHOW EXPERIMENTAL_REPLICA TRACE FOR SESSION`},
{`EXPLAIN SHOW EXPERIMENTAL_REPLICA TRACE FOR SESSION`},
{`SHOW STATISTICS FOR TABLE t`},
{`SHOW STATISTICS USING JSON FOR TABLE t`},
{`EXPLAIN SHOW STATISTICS FOR TABLE t`},
{`SHOW STATISTICS FOR TABLE d.t`},
{`SHOW HISTOGRAM 123`},
Expand Down Expand Up @@ -1582,6 +1588,8 @@ func TestParse2(t *testing.T) {
{`CREATE STATISTICS a ON col1 FROM t AS OF SYSTEM TIME '2016-01-01'`,
`CREATE STATISTICS a ON col1 FROM t WITH OPTIONS AS OF SYSTEM TIME '2016-01-01'`},

{`ANALYSE t`, `ANALYZE t`},

{`SELECT TIMESTAMP WITHOUT TIME ZONE 'foo'`, `SELECT TIMESTAMP 'foo'`},
{`SELECT CAST('foo' AS TIMESTAMP WITHOUT TIME ZONE)`, `SELECT CAST('foo' AS TIMESTAMP)`},
{`SELECT CAST(1 AS "timestamp")`, `SELECT CAST(1 AS TIMESTAMP)`},
Expand Down Expand Up @@ -2931,6 +2939,22 @@ DETAIL: source SQL:
CREATE STATISTICS a ON col1 FROM t WITH OPTIONS AS OF SYSTEM TIME '-1s' THROTTLING 0.1 AS OF SYSTEM TIME '-2s'
^`,
},
{
`ANALYZE`,
`at or near "EOF": syntax error
DETAIL: source SQL:
ANALYZE
^
HINT: try \h ANALYZE`,
},
{
`ANALYSE`,
`at or near "EOF": syntax error
DETAIL: source SQL:
ANALYSE
^
HINT: try \h ANALYZE`,
},
{
`ALTER PARTITION p OF TABLE tbl@idx CONFIGURE ZONE USING num_replicas = 1`,
`at or near "idx": syntax error: index name should not be specified in ALTER PARTITION ... OF TABLE
Expand Down
34 changes: 32 additions & 2 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -762,6 +762,7 @@ func (u *sqlSymUnion) alterTypeAddValuePlacement() *tree.AlterTypeAddValuePlacem
%type <tree.Statement> drop_view_stmt
%type <tree.Statement> drop_sequence_stmt

%type <tree.Statement> analyze_stmt
%type <tree.Statement> explain_stmt
%type <tree.Statement> prepare_stmt
%type <tree.Statement> preparable_stmt
Expand Down Expand Up @@ -887,7 +888,7 @@ func (u *sqlSymUnion) alterTypeAddValuePlacement() *tree.AlterTypeAddValuePlacem
%type <str> schema_name
%type <*tree.UnresolvedName> table_pattern complex_table_pattern
%type <*tree.UnresolvedName> column_path prefixed_column_path column_path_with_star
%type <tree.TableExpr> insert_target create_stats_target
%type <tree.TableExpr> insert_target create_stats_target analyze_target

%type <*tree.TableIndexName> table_index_name
%type <tree.TableIndexNames> table_index_name_list
Expand Down Expand Up @@ -1169,7 +1170,8 @@ stmt_block:

stmt:
HELPTOKEN { return helpWith(sqllex, "") }
| preparable_stmt // help texts in sub-rule
| preparable_stmt // help texts in sub-rule
| analyze_stmt // EXTEND WITH HELP: ANALYZE
| copy_from_stmt
| comment_stmt
| execute_stmt // EXTEND WITH HELP: EXECUTE
Expand Down Expand Up @@ -2844,6 +2846,34 @@ table_name_list:
$$.val = append($1.tableNames(), name)
}

// %Help: ANALYZE - collect table statistics
// %Category: Misc
// %Text:
// ANALYZE <tablename>
//
// %SeeAlso: CREATE STATISTICS
analyze_stmt:
ANALYZE analyze_target
{
$$.val = &tree.Analyze{
Table: $2.tblExpr(),
}
}
| ANALYZE error // SHOW HELP: ANALYZE
| ANALYSE analyze_target
{
$$.val = &tree.Analyze{
Table: $2.tblExpr(),
}
}
| ANALYSE error // SHOW HELP: ANALYZE

analyze_target:
table_name
{
$$.val = $1.unresolvedObjectName()
}

// %Help: EXPLAIN - show the logical plan of a query
// %Category: Misc
// %Text:
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)

switch stmt.AST.(type) {
case *tree.AlterIndex, *tree.AlterTable, *tree.AlterSequence,
*tree.Analyze,
*tree.BeginTransaction,
*tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable,
*tree.CommitTransaction,
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/sem/tree/analyze.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
// Copyright 2020 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 tree

// Analyze represents an ANALYZE statement.
type Analyze struct {
Table TableExpr
}

// Format implements the NodeFormatter interface.
func (node *Analyze) Format(ctx *FmtCtx) {
ctx.WriteString("ANALYZE ")
ctx.FormatNode(node.Table)
}
7 changes: 7 additions & 0 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,12 @@ func (*AlterRole) cclOnlyStatement() {}

func (*AlterRole) hiddenFromShowQueries() {}

// StatementType implements the Statement interface.
func (*Analyze) StatementType() StatementType { return DDL }

// StatementTag returns a short string identifying the type of statement.
func (*Analyze) StatementTag() string { return "ANALYZE" }

// StatementType implements the Statement interface.
func (*Backup) StatementType() StatementType { return Rows }

Expand Down Expand Up @@ -919,6 +925,7 @@ func (n *AlterTableSetNotNull) String() string { return AsString(n) }
func (n *AlterType) String() string { return AsString(n) }
func (n *AlterRole) String() string { return AsString(n) }
func (n *AlterSequence) String() string { return AsString(n) }
func (n *Analyze) String() string { return AsString(n) }
func (n *Backup) String() string { return AsString(n) }
func (n *BeginTransaction) String() string { return AsString(n) }
func (n *ControlJobs) String() string { return AsString(n) }
Expand Down

0 comments on commit d8d001b

Please sign in to comment.