Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
55248: sql: add issue link to ALTER TABLE DROP COLUMN with sql_safe_updates r=lucy-zhang a=ajwerner

Explicit transactions with drop column are very dangerous. Inform the user.

Touches #51863.

Release note (sql change): Added a note to inform users with an issue link
when attempting potentially hazardous DROP COLUMN operations when
sql_safe_updates is enabled.

55655: xform: organize custom functions into general_funcs.go and scan_funcs.go r=RaduBerinde a=mgartner

#### xform: rename custom_funcs.go to general_funcs.go

This is in preparation for breaking up the custom functions into
multiple files, similar to the organization of normalization custom
functions.

Release note: None

#### xform: move scan-related custom functions to scan_funcs.go

This is one step in breaking up the exploration custom functions into
files based on the rules they facilitate, similar to the normalization
custom functions.

Release note: None


55721: roachtest: bump estimated max warehouses of tpccbench/nodes=3/cpu=16 r=nvanbenschoten a=nvanbenschoten

Now that we're importing instead of restoring and picking up a number
of optimizations along the way, this estimated max warehouse count is
too low. This is why we have been flatlining on:
https://roachperf.crdb.dev/?filter=&view=tpccbench%2Fnodes%3D3%2Fcpu%3D16&tab=gce

55722: sql: ban renaming table to a new database unless both schemas are public r=lucy-zhang a=lucy-zhang

Previously we would seemingly allow renaming tables to a new database
with arbitrary source and target schemas without ever actually
reassigning the schema ID. This could lead to a corrupted descriptor
with an invalid schema ID for the database. This PR fixes the
implementation to return an error when renaming a table to a different
database unless both the source and target schemas are `public`.

Fixes #55710.

Release note (bug fix): Previously, changing the parent database and
schema of a table using `RENAME` was seemingly permitted but would lead
to corruption of the table metadata. Now an error is returned when
attempting to rename a table to a different database except in the case
where both the source and target schemas are the `public` schema in each
database, which continues to be supported.

Co-authored-by: Andrew Werner <ajwerner@cockroachlabs.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Lucy Zhang <lucy@cockroachlabs.com>
  • Loading branch information
5 people committed Oct 20, 2020
5 parents e25436f + c029cc4 + a82cb57 + 1bffca2 + a978b55 commit 65bf9e6
Show file tree
Hide file tree
Showing 6 changed files with 153 additions and 64 deletions.
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tpcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -369,8 +369,8 @@ func registerTPCC(r *testRegistry) {
Nodes: 3,
CPUs: 16,

LoadWarehouses: gceOrAws(cloud, 2000, 3000),
EstimatedMax: gceOrAws(cloud, 1600, 2500),
LoadWarehouses: gceOrAws(cloud, 2500, 3000),
EstimatedMax: gceOrAws(cloud, 2200, 2500),
})
registerTPCCBenchSpec(r, tpccBenchSpec{
Nodes: 12,
Expand Down
12 changes: 11 additions & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,17 @@ func (n *alterTableNode) startExec(params runParams) error {

case *tree.AlterTableDropColumn:
if params.SessionData().SafeUpdates {
return pgerror.DangerousStatementf("ALTER TABLE DROP COLUMN will remove all data in that column")
err := pgerror.DangerousStatementf("ALTER TABLE DROP COLUMN will " +
"remove all data in that column")
if !params.extendedEvalCtx.TxnImplicit {
err = errors.WithIssueLink(err, errors.IssueLink{
IssueURL: "https://github.com/cockroachdb/cockroach/issues/46541",
Detail: "when used in an explicit transaction combined with other " +
"schema changes to the same table, DROP COLUMN can result in data " +
"loss if one of the other schema change fails or is canceled",
})
}
return err
}

colToDrop, dropped, err := n.tableDesc.FindColumnByName(t.Column)
Expand Down
56 changes: 56 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/rename_table
Original file line number Diff line number Diff line change
Expand Up @@ -261,3 +261,59 @@ query TTTTI
SHOW TABLES
----
public kv table root 0

# Test that tables can't be renamed to a different database unless both the
# old and new schemas are in the public schema.
subtest rename_table_across_dbs

statement ok
CREATE DATABASE olddb

statement ok
CREATE DATABASE newdb

statement ok
USE olddb

statement ok
CREATE SCHEMA oldsc

statement ok
USE newdb

statement ok
CREATE SCHEMA newsc

statement ok
CREATE TABLE olddb.public.tbl1();

statement ok
CREATE TABLE olddb.oldsc.tbl2();

statement error pgcode 42602 cannot change database of table
ALTER TABLE olddb.public.tbl1 RENAME TO newdb.newsc.tbl1

statement error pgcode 42602 cannot change database of table
ALTER TABLE olddb.oldsc.tbl2 RENAME TO newdb.public.tbl2

statement error pgcode 42602 cannot change database of table
ALTER TABLE olddb.oldsc.tbl2 RENAME TO newdb.newsc.tbl2

# Try different but equivalent names.

statement error pgcode 42602 cannot change database of table
ALTER TABLE olddb.tbl1 RENAME TO newdb.newsc.tbl1

statement error pgcode 42602 cannot change database of table
ALTER TABLE olddb.oldsc.tbl2 RENAME TO newdb.tbl2

# Using the public schemas should still work.

statement ok
ALTER TABLE olddb.public.tbl1 RENAME TO newdb.public.tbl1

statement ok
DROP DATABASE olddb CASCADE

statement ok
DROP DATABASE newdb CASCADE
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,6 @@ func (c *CustomFuncs) Init(e *explorer) {
c.im.Init(e.f, e.mem.Metadata(), e.evalCtx)
}

// ----------------------------------------------------------------------
//
// Scan Rules
// Custom match and replace functions used with scan.opt rules.
//
// ----------------------------------------------------------------------

// IsCanonicalScan returns true if the given ScanPrivate is an original
// unaltered primary index Scan operator (i.e. unconstrained and not limited).
func (c *CustomFuncs) IsCanonicalScan(scan *memo.ScanPrivate) bool {
Expand All @@ -71,60 +64,6 @@ func (c *CustomFuncs) IsLocking(scan *memo.ScanPrivate) bool {
return scan.IsLocking()
}

// GenerateIndexScans enumerates all non-inverted secondary indexes on the given
// Scan operator's table and generates an alternate Scan operator for each index
// that includes the set of needed columns specified in the ScanOpDef.
//
// This transformation can only consider partial indexes that are guaranteed to
// index every row in the table. Therefore, only partial indexes with predicates
// that always evaluate to true are considered. Such an index is pseudo-partial
// in that it behaves the exactly the same as a non-partial secondary index.
//
// NOTE: This does not generate index joins for non-covering indexes (except in
// case of ForceIndex). Index joins are usually only introduced "one level
// up", when the Scan operator is wrapped by an operator that constrains
// or limits scan output in some way (e.g. Select, Limit, InnerJoin).
// Index joins are only lower cost when their input does not include all
// rows from the table. See ConstrainScans and LimitScans for cases where
// index joins are introduced into the memo.
func (c *CustomFuncs) GenerateIndexScans(grp memo.RelExpr, scanPrivate *memo.ScanPrivate) {
// Iterate over all non-inverted and non-partial secondary indexes.
var iter scanIndexIter
iter.init(c.e.mem, &c.im, scanPrivate, nil /* originalFilters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool) {
// If the secondary index includes the set of needed columns, then construct
// a new Scan operator using that index.
if isCovering {
scan := memo.ScanExpr{ScanPrivate: *scanPrivate}
scan.Index = index.Ordinal()
c.e.mem.AddScanToGroup(&scan, grp)
return
}

// Otherwise, if the index must be forced, then construct an IndexJoin
// operator that provides the columns missing from the index. Note that
// if ForceIndex=true, scanIndexIter only returns the one index that is
// being forced, so no need to check that here.
if !scanPrivate.Flags.ForceIndex {
return
}

var sb indexScanBuilder
sb.init(c, scanPrivate.Table)

// Scan whatever columns we need which are available from the index, plus
// the PK columns.
newScanPrivate := *scanPrivate
newScanPrivate.Index = index.Ordinal()
newScanPrivate.Cols = indexCols.Intersection(scanPrivate.Cols)
newScanPrivate.Cols.UnionWith(sb.primaryKeyCols())
sb.setScan(&newScanPrivate)

sb.addIndexJoin(scanPrivate.Cols)
sb.build(grp)
})
}

// ----------------------------------------------------------------------
//
// Select Rules
Expand Down
71 changes: 71 additions & 0 deletions pkg/sql/opt/xform/scan_funcs.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
// 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 xform

import (
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
)

// GenerateIndexScans enumerates all non-inverted secondary indexes on the given
// Scan operator's table and generates an alternate Scan operator for each index
// that includes the set of needed columns specified in the ScanOpDef.
//
// This transformation can only consider partial indexes that are guaranteed to
// index every row in the table. Therefore, only partial indexes with predicates
// that always evaluate to true are considered. Such an index is pseudo-partial
// in that it behaves the exactly the same as a non-partial secondary index.
//
// NOTE: This does not generate index joins for non-covering indexes (except in
// case of ForceIndex). Index joins are usually only introduced "one level
// up", when the Scan operator is wrapped by an operator that constrains
// or limits scan output in some way (e.g. Select, Limit, InnerJoin).
// Index joins are only lower cost when their input does not include all
// rows from the table. See ConstrainScans and LimitScans for cases where
// index joins are introduced into the memo.
func (c *CustomFuncs) GenerateIndexScans(grp memo.RelExpr, scanPrivate *memo.ScanPrivate) {
// Iterate over all non-inverted and non-partial secondary indexes.
var iter scanIndexIter
iter.init(c.e.mem, &c.im, scanPrivate, nil /* originalFilters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool) {
// If the secondary index includes the set of needed columns, then construct
// a new Scan operator using that index.
if isCovering {
scan := memo.ScanExpr{ScanPrivate: *scanPrivate}
scan.Index = index.Ordinal()
c.e.mem.AddScanToGroup(&scan, grp)
return
}

// Otherwise, if the index must be forced, then construct an IndexJoin
// operator that provides the columns missing from the index. Note that
// if ForceIndex=true, scanIndexIter only returns the one index that is
// being forced, so no need to check that here.
if !scanPrivate.Flags.ForceIndex {
return
}

var sb indexScanBuilder
sb.init(c, scanPrivate.Table)

// Scan whatever columns we need which are available from the index, plus
// the PK columns.
newScanPrivate := *scanPrivate
newScanPrivate.Index = index.Ordinal()
newScanPrivate.Cols = indexCols.Intersection(scanPrivate.Cols)
newScanPrivate.Cols.UnionWith(sb.primaryKeyCols())
sb.setScan(&newScanPrivate)

sb.addIndexJoin(scanPrivate.Cols)
sb.build(grp)
})
}
13 changes: 13 additions & 0 deletions pkg/sql/rename_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,15 @@ func (n *renameTableNode) startExec(params runParams) error {
)
}

// Disable renaming objects between databases unless both the source and
// target schemas are the public schema. This preserves backward compatibility
// for the behavior prior to user-defined schemas.
if oldTn.Catalog() != newTn.Catalog() &&
(oldTn.Schema() != string(tree.PublicSchemaName) || newTn.Schema() != string(tree.PublicSchemaName)) {
return pgerror.Newf(pgcode.InvalidName,
"cannot change database of table unless both the old and new schemas are the public schema in each database")
}

// oldTn and newTn are already normalized, so we can compare directly here.
if oldTn.Catalog() == newTn.Catalog() &&
oldTn.Schema() == newTn.Schema() &&
Expand All @@ -167,6 +176,10 @@ func (n *renameTableNode) startExec(params runParams) error {
return err
}

// The parent schema ID is never modified here because changing the schema of
// a table within the same database is disallowed, and changing the database
// of a table is only allowed if both the source and target schemas are the
// public schema.
tableDesc.SetName(newTn.Table())
tableDesc.ParentID = targetDbDesc.GetID()

Expand Down

0 comments on commit 65bf9e6

Please sign in to comment.