Skip to content

Commit

Permalink
sql: fix CREATE TABLE AS sourcing SHOW <show_subcmd> <table> job …
Browse files Browse the repository at this point in the history
…failures

Fixes #106260

Previously `CREATE TABLE AS`/`CREATE MATERIALIZED VIEW AS` sourcing from
`SHOW <show_subcmd> <table>` generated a failing schema change job with a
`relation "tbl" does not exist error` because the SHOW source table was not
fully qualified.

This PR fixes this by respecting the `Builder.qualifyDataSourceNamesInAST` flag
in `delegate.TryDelegate()` which implements the failing SHOW commands.

Release note (bug fix): Fix failing schema change job when CREATE TABLE AS or
CREATE MATERAILIZED VIEW AS sources from a SHOW command:
1. CREATE TABLE t AS SELECT * FROM [SHOW CREATE TABLE tbl];
2. CREATE TABLE t AS SELECT * FROM [SHOW INDEXES FROM tbl];
3. CREATE TABLE t AS SELECT * FROM [SHOW COLUMNS FROM tbl];
4. CREATE TABLE t AS SELECT * FROM [SHOW CONSTRAINTS FROM tbl];
5. CREATE TABLE t AS SELECT * FROM [SHOW PARTITIONS FROM TABLE tbl];
6. CREATE TABLE t AS SELECT * FROM [SHOW PARTITIONS FROM INDEX tbl@tbl_pkey];
  • Loading branch information
ecwall committed Jul 18, 2023
1 parent 25855ac commit 8e4578b
Show file tree
Hide file tree
Showing 5 changed files with 87 additions and 59 deletions.
18 changes: 0 additions & 18 deletions pkg/sql/create_as_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,9 +137,6 @@ func TestCreateAsShow(t *testing.T) {
{
sql: "SHOW CREATE TABLE show_create_tbl",
setup: "CREATE TABLE show_create_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_create_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW CREATE FUNCTION show_create_fn",
Expand All @@ -157,40 +154,25 @@ func TestCreateAsShow(t *testing.T) {
{
sql: "SHOW INDEXES FROM show_indexes_tbl",
setup: "CREATE TABLE show_indexes_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_indexes_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW COLUMNS FROM show_columns_tbl",
setup: "CREATE TABLE show_columns_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_columns_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW CONSTRAINTS FROM show_constraints_tbl",
setup: "CREATE TABLE show_constraints_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_constraints_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW PARTITIONS FROM DATABASE defaultdb",
},
{
sql: "SHOW PARTITIONS FROM TABLE show_partitions_tbl",
setup: "CREATE TABLE show_partitions_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_partitions_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW PARTITIONS FROM INDEX show_partitions_idx_tbl@show_partitions_idx_tbl_pkey",
setup: "CREATE TABLE show_partitions_idx_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_partitions_idx_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW GRANTS",
Expand Down
81 changes: 74 additions & 7 deletions pkg/sql/delegate/delegate.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,17 @@ import (
// can be rewritten as a lower level query. If it can, returns a new AST which
// is equivalent to the original statement. Otherwise, returns nil.
func TryDelegate(
ctx context.Context, catalog cat.Catalog, evalCtx *eval.Context, stmt tree.Statement,
ctx context.Context,
catalog cat.Catalog,
evalCtx *eval.Context,
stmt tree.Statement,
qualifyDataSourceNamesInAST bool,
) (tree.Statement, error) {
d := delegator{
ctx: ctx,
catalog: catalog,
evalCtx: evalCtx,
ctx: ctx,
catalog: catalog,
evalCtx: evalCtx,
qualifyDataSourceNamesInAST: qualifyDataSourceNamesInAST,
}
switch t := stmt.(type) {
case *tree.ShowClusterSettingList:
Expand Down Expand Up @@ -197,9 +202,10 @@ func TryDelegate(
}

type delegator struct {
ctx context.Context
catalog cat.Catalog
evalCtx *eval.Context
ctx context.Context
catalog cat.Catalog
evalCtx *eval.Context
qualifyDataSourceNamesInAST bool
}

func (d *delegator) parse(sql string) (tree.Statement, error) {
Expand All @@ -210,3 +216,64 @@ func (d *delegator) parse(sql string) (tree.Statement, error) {
d.evalCtx.Planner.MaybeReallocateAnnotations(s.NumAnnotations)
return s.AST, err
}

// We avoid the cache so that we can observe the details without
// taking a lease, like other SHOW commands.
var resolveFlags = cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true}

// resolveAndModifyUnresolvedObjectName may modify the name input
// if d.qualifyDataSourceNamesInAST == true
func (d *delegator) resolveAndModifyUnresolvedObjectName(
name *tree.UnresolvedObjectName,
) (cat.DataSource, cat.DataSourceName, error) {
tn := name.ToTableName()
dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, resolveFlags, &tn)
if err != nil {
return nil, cat.DataSourceName{}, err
}
if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil {
return nil, cat.DataSourceName{}, err
}
// Use qualifyDataSourceNamesInAST similarly to the Builder so that
// CREATE TABLE AS can source from a delegated expression.
// For example: CREATE TABLE t2 AS SELECT * FROM [SHOW CREATE t1];
if d.qualifyDataSourceNamesInAST {
resName.ExplicitSchema = true
resName.ExplicitCatalog = true
*name = *resName.ToUnresolvedObjectName()
}
return dataSource, resName, nil
}

// resolveAndModifyTableIndexName may modify the name input
// if d.qualifyDataSourceNamesInAST == true
func (d *delegator) resolveAndModifyTableIndexName(
name *tree.TableIndexName,
) (cat.DataSource, cat.DataSourceName, error) {

tn := name.Table
dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, resolveFlags, &tn)
if err != nil {
return nil, cat.DataSourceName{}, err
}

if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil {
return nil, cat.DataSourceName{}, err
}

// Force resolution of the index.
_, _, err = cat.ResolveTableIndex(d.ctx, d.catalog, resolveFlags, name)
if err != nil {
return nil, cat.DataSourceName{}, err
}

// Use qualifyDataSourceNamesInAST similarly to the Builder so that
// CREATE TABLE AS can source from a delegated expression.
// For example: CREATE TABLE t2 AS SELECT * FROM [SHOW PARTITIONS FROM INDEX t1@t1_pkey];
if d.qualifyDataSourceNamesInAST {
resName.ExplicitSchema = true
resName.ExplicitCatalog = true
(*name).Table = resName.ToUnresolvedObjectName().ToTableName()
}
return dataSource, resName, nil
}
27 changes: 4 additions & 23 deletions pkg/sql/delegate/show_partitions.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand All @@ -25,16 +24,10 @@ import (
func (d *delegator) delegateShowPartitions(n *tree.ShowPartitions) (tree.Statement, error) {
sqltelemetry.IncrementShowCounter(sqltelemetry.Partitions)
if n.IsTable {
flags := cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true}
tn := n.Table.ToTableName()

dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, flags, &tn)
_, resName, err := d.resolveAndModifyUnresolvedObjectName(n.Table)
if err != nil {
return nil, err
}
if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil {
return nil, err
}

// We use the raw_config_sql from the partition_lookup result to get the
// official zone config for the partition, and use the full_config_sql from the zones table
Expand Down Expand Up @@ -108,28 +101,16 @@ func (d *delegator) delegateShowPartitions(n *tree.ShowPartitions) (tree.Stateme
return d.parse(fmt.Sprintf(showDatabasePartitionsQuery, n.Database.String(), lexbase.EscapeSQLString(string(n.Database))))
}

flags := cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true}
tn := n.Index.Table

// Throw a more descriptive error if the user did not use the index hint syntax.
if tn.ObjectName == "" {
tableIndexName := &n.Index
if tableIndexName.Table.ObjectName == "" {
err := errors.New("no table specified")
err = pgerror.WithCandidateCode(err, pgcode.InvalidParameterValue)
err = errors.WithHint(err, "Specify a table using the hint syntax of table@index.")
return nil, err
}

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
}

// Force resolution of the index.
_, _, err = cat.ResolveTableIndex(d.ctx, d.catalog, flags, &n.Index)
_, resName, err := d.resolveAndModifyTableIndexName(tableIndexName)
if err != nil {
return nil, err
}
Expand Down
12 changes: 2 additions & 10 deletions pkg/sql/delegate/show_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -297,18 +296,11 @@ func (d *delegator) delegateShowCreateAllTables() (tree.Statement, error) {
func (d *delegator) showTableDetails(
name *tree.UnresolvedObjectName, query string,
) (tree.Statement, error) {
// We avoid the cache so that we can observe the details without
// taking a lease, like other SHOW commands.
flags := cat.Flags{AvoidDescriptorCaches: true, NoTableStats: true}
tn := name.ToTableName()
dataSource, resName, err := d.catalog.ResolveDataSource(d.ctx, flags, &tn)

dataSource, resName, err := d.resolveAndModifyUnresolvedObjectName(name)
if err != nil {
return nil, err
}
if err := d.catalog.CheckAnyPrivilege(d.ctx, dataSource); err != nil {
return nil, err
}

fullQuery := fmt.Sprintf(query,
lexbase.EscapeSQLString(resName.Catalog()),
lexbase.EscapeSQLString(resName.Table()),
Expand Down
8 changes: 7 additions & 1 deletion pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -423,7 +423,13 @@ func (b *Builder) buildStmt(
default:
// See if this statement can be rewritten to another statement using the
// delegate functionality.
newStmt, err := delegate.TryDelegate(b.ctx, b.catalog, b.evalCtx, stmt)
newStmt, err := delegate.TryDelegate(
b.ctx,
b.catalog,
b.evalCtx,
stmt,
b.qualifyDataSourceNamesInAST,
)
if err != nil {
panic(err)
}
Expand Down

0 comments on commit 8e4578b

Please sign in to comment.