Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
40436: sql: more test fixes for opt-driven foreign keys r=RaduBerinde a=RaduBerinde

Fixing up some expected errors in tests and making sure we don't
buffer the mutation input if we fall back to the legacy path (the
bufferNode is unnecessary and interferes with an interleaved delete
fast path).

Release note: None

40451: testcluster: don't overwrite localities indiscriminately r=andreimatei a=andreimatei

Before this patch, a TestCluster would set localities for all nodes to a
static values. This was overwriting any values set throught the
TestClusterArgs. This patch makes it so that, if any localities are set
in the args, we don't overwrite them.

Release note: None

40485: sql: Fix a bug with ordinal_position in information_schema.columns r=arulajmani a=arulajmani

When a column other than the last is dropped, ordinal_position in
information_schema.columns virtual table no longer matches attnum from
the pg_attribute table. This PR fixes this issue.

Fixes #39787

Release note (bug fix): ordinal_position in information_schema.columns
matches pg_attribute.attnum after a column is dropped.

40511: exec: fix explain(vec) for queries with subqueries r=jordanlewis a=jordanlewis

Also add logic tests that show the explain(vec) plans for all of the
tpch queries.

Closes #40484.

Release note: None

Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
Co-authored-by: Arul Ajmani <arula@cockroachlabs.com>
Co-authored-by: Jordan Lewis <jordanthelewis@gmail.com>
  • Loading branch information
5 people committed Sep 5, 2019
5 parents 01c7cf5 + 0b4795f + c114113 + ce00e81 + 3ed2af0 commit 7f8bda1
Show file tree
Hide file tree
Showing 14 changed files with 691 additions and 40 deletions.
24 changes: 12 additions & 12 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1369,25 +1369,25 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {

// FK validation on customers from receipts is preserved.
db.ExpectErr(
t, "foreign key violation.* referenced in table \"receipts\"",
t, "foreign key violation.* referenced in table \"receipts\"|update.*violates foreign key constraint on table \"receipts\"",
`UPDATE store.customers SET email = concat(id::string, 'nope')`,
)

// FK validation on customers from orders is preserved.
db.ExpectErr(
t, "foreign key violation.* referenced in table \"orders\"",
t, "foreign key violation.* referenced in table \"orders\"|update.*violates foreign key constraint on table \"orders\"",
`UPDATE store.customers SET id = id * 1000`,
)

// FK validation of customer id is preserved.
db.ExpectErr(
t, "foreign key violation.* in customers@primary",
t, "foreign key violation.* in customers@primary|insert.*violates foreign key constraint \"fk_customerid_ref_customers\"",
`INSERT INTO store.orders VALUES (999, NULL, 999)`,
)

// FK validation of self-FK is preserved.
db.ExpectErr(
t, "foreign key violation: value .999. not found in receipts@primary",
t, "foreign key violation: value .999. not found in receipts@primary|insert.*violates foreign key constraint \"fk_reissue_ref_receipts\"",
`INSERT INTO store.receipts VALUES (1, 999, NULL, NULL)`,
)
})
Expand All @@ -1402,7 +1402,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {

// FK validation on customers from orders is preserved.
db.ExpectErr(
t, "foreign key violation.* referenced in table \"orders\"",
t, "foreign key violation.* referenced in table \"orders\"|update.*violates foreign key constraint on table \"orders\"",
`UPDATE store.customers SET id = id*100`,
)

Expand Down Expand Up @@ -1443,8 +1443,8 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {

// FK validation of self-FK is preserved.
db.ExpectErr(
t, "foreign key violation: value .999. not found in receipts@primary",
`INSERT INTO store.receipts VALUES (1, 999, NULL, NULL)`,
t, "foreign key violation: value .999. not found in receipts@primary|insert.*violates foreign key constraint \"fk_reissue_ref_receipts\"",
`INSERT INTO store.receipts VALUES (-1, 999, NULL, NULL)`,
)
})

Expand All @@ -1461,20 +1461,20 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {

// FK validation of customer email is preserved.
db.ExpectErr(
t, "foreign key violation.* in customers@customers_email_key",
`INSERT INTO store.receipts VALUES (1, NULL, '999', 999)`,
t, "foreign key violation.* in customers@customers_email_key|insert.*violates foreign key constraint \"fk_dest_ref_customers\"",
`INSERT INTO store.receipts VALUES (-1, NULL, '999', 999)`,
)

// FK validation on customers from receipts is preserved.
db.ExpectErr(
t, "foreign key violation.* referenced in table \"receipts\"",
t, "foreign key violation.* referenced in table \"receipts\"|delete.*violates foreign key constraint on table \"receipts\"",
`DELETE FROM store.customers`,
)

// FK validation of self-FK is preserved.
db.ExpectErr(
t, "foreign key violation: value .999. not found in receipts@primary",
`INSERT INTO store.receipts VALUES (1, 999, NULL, NULL)`,
t, "foreign key violation: value .999. not found in receipts@primary|insert.*violates foreign key constraint \"fk_reissue_ref_receipts\"",
`INSERT INTO store.receipts VALUES (-1, 999, NULL, NULL)`,
)
})

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/descriptor_mutation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -993,8 +993,8 @@ CREATE TABLE t.test (a STRING PRIMARY KEY, b STRING, c STRING, INDEX foo (c));
"SHOW COLUMNS FROM t.test",
[][]string{
{"a", "STRING", "false", "NULL", "", "{primary,ufo}", "false"},
{"d", "STRING", "true", "NULL", "", "{ufo}", "false"},
{"e", "STRING", "true", "NULL", "", "{}", "false"},
{"d", "STRING", "true", "NULL", "", "{ufo}", "false"},
},
)

Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/distsqlrun/column_exec_setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -774,12 +774,13 @@ func planProjectionOperators(
case *tree.BinaryExpr:
return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input)
case *tree.CastExpr:
op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, t.Expr.(tree.TypedExpr), columnTypes, input)
expr := t.Expr.(tree.TypedExpr)
op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, expr, columnTypes, input)
if err != nil {
return nil, 0, nil, 0, err
}
outputIdx := len(ct)
op, err = exec.GetCastOperator(op, resultIdx, outputIdx, t.Expr.(tree.TypedExpr).ResolvedType(), t.Type)
op, err = exec.GetCastOperator(op, resultIdx, outputIdx, expr.ResolvedType(), t.Type)
ct = append(ct, *t.Type)
if sMem, ok := op.(exec.StaticMemoryOperator); ok {
memUsed += sMem.EstimateStaticMemoryUsage()
Expand Down
60 changes: 60 additions & 0 deletions pkg/sql/exec/cast_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ package exec

import (
"context"
"fmt"
"math"

"github.com/cockroachdb/apd"
Expand Down Expand Up @@ -73,6 +74,14 @@ var _ interface{} = execgen.UNSAFEGET
func GetCastOperator(
input Operator, colIdx int, resultIdx int, fromType *semtypes.T, toType *semtypes.T,
) (Operator, error) {
if fromType.Family() == semtypes.UnknownFamily {
return &castOpNullAny{
OneInputNode: NewOneInputNode(input),
colIdx: colIdx,
outputIdx: resultIdx,
toType: typeconv.FromColumnType(toType),
}, nil
}
switch from := typeconv.FromColumnType(fromType); from {
// {{ range $typ, $overloads := . }}
case coltypes._ALLTYPES:
Expand All @@ -98,6 +107,57 @@ func GetCastOperator(
}
}

type castOpNullAny struct {
OneInputNode
colIdx int
outputIdx int
toType coltypes.T
}

var _ StaticMemoryOperator = &castOpNullAny{}

func (c *castOpNullAny) EstimateStaticMemoryUsage() int {
return EstimateBatchSizeBytes([]coltypes.T{c.toType}, coldata.BatchSize)
}

func (c *castOpNullAny) Init() {
c.input.Init()
}

func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch {
batch := c.input.Next(ctx)
n := batch.Length()
if n == 0 {
return batch
}
if c.outputIdx == batch.Width() {
batch.AppendCol(c.toType)
}
vec := batch.ColVec(c.colIdx)
projVec := batch.ColVec(c.outputIdx)
vecNulls := vec.Nulls()
projNulls := projVec.Nulls()
if sel := batch.Selection(); sel != nil {
sel = sel[:n]
for _, i := range sel {
if vecNulls.NullAt(i) {
projNulls.SetNull(i)
} else {
execerror.VectorizedInternalPanic(errors.Errorf("unexpected non-null at index %d", i))
}
}
} else {
for i := uint16(0); i < n; i++ {
if vecNulls.NullAt(uint16(i)) {
projNulls.SetNull(uint16(i))
} else {
execerror.VectorizedInternalPanic(fmt.Errorf("unexpected non-null at index %d", i))
}
}
}
return batch
}

// {{ range $typ, $overloads := . }}
// {{ range $overloads }}
// {{ if isCastFuncSet . }}
Expand Down
25 changes: 23 additions & 2 deletions pkg/sql/explain_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/exec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/treeprinter"
)

Expand All @@ -37,6 +39,7 @@ type explainVecNode struct {
// The current row returned by the node.
values tree.Datums
}
subqueryPlans []subquery
}

type flowWithNode struct {
Expand All @@ -62,7 +65,17 @@ func (n *explainVecNode) startExec(params runParams) error {
planCtx.ignoreClose = true
planCtx.planner = params.p
planCtx.stmtType = n.stmtType
planCtx.noEvalSubqueries = true
outerSubqueries := planCtx.planner.curPlan.subqueryPlans
defer func() {
planCtx.planner.curPlan.subqueryPlans = outerSubqueries
}()
planCtx.planner.curPlan.subqueryPlans = n.subqueryPlans
for i := range planCtx.planner.curPlan.subqueryPlans {
p := &planCtx.planner.curPlan.subqueryPlans[i]
// Fake subquery results - they're not important for our explain output.
p.started = true
p.result = tree.DNull
}

var plan PhysicalPlan
var err error
Expand All @@ -88,8 +101,16 @@ func (n *explainVecNode) startExec(params runParams) error {
flowCtx := &distsqlrun.FlowCtx{
NodeID: planCtx.EvalContext().NodeID,
EvalCtx: planCtx.EvalContext(),
Cfg: &distsqlrun.ServerConfig{},
Cfg: &distsqlrun.ServerConfig{
Settings: params.p.execCfg.Settings,
DiskMonitor: &mon.BytesMonitor{},
},
}
// Temporarily set vectorize to on so that we can get the whole plan back even
// if we wouldn't support it due to lack of streaming.
origMode := flowCtx.EvalCtx.SessionData.VectorizeMode
flowCtx.EvalCtx.SessionData.VectorizeMode = sessiondata.VectorizeExperimentalOn
defer func() { flowCtx.EvalCtx.SessionData.VectorizeMode = origMode }()

sortedFlows := make([]flowWithNode, 0, len(flows))
for nodeID, flow := range flows {
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/information_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -376,10 +376,7 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`,
return forEachTableDesc(ctx, p, dbContext, virtualMany, func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error {
dbNameStr := tree.NewDString(db.Name)
scNameStr := tree.NewDString(scName)
// Table descriptors already holds columns in-order.
visible := 0
return forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error {
visible++
collationCatalog := tree.DNull
collationSchema := tree.DNull
collationName := tree.DNull
Expand All @@ -393,7 +390,7 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`,
scNameStr, // table_schema
tree.NewDString(table.Name), // table_name
tree.NewDString(column.Name), // column_name
tree.NewDInt(tree.DInt(visible)), // ordinal_position, 1-indexed
tree.NewDInt(tree.DInt(column.ID)), // ordinal_position
dStringPtrOrNull(column.DefaultExpr), // column_default
yesOrNoDatum(column.Nullable), // is_nullable
tree.NewDString(column.Type.InformationSchemaName()), // data_type
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1043,8 +1043,6 @@ func (t *logicTest) setup(cfg testClusterConfig) {
},
ClusterName: "testclustername",
UseDatabase: "test",
// Set Locality so we can use it in zone config tests.
Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "test"}}},
},
// For distributed SQL tests, we use the fake span resolver; it doesn't
// matter where the data really is.
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -2198,3 +2198,30 @@ SET database = test

statement ok
DROP DATABASE dfk CASCADE

# Regression test for #39787. Verify information_schema.ordinal_position
# matches pg_attribute.attnum when a leading column is dropped.

statement ok
CREATE TABLE ab(a INT, b INT)

statement ok
ALTER TABLE ab DROP COLUMN a

let $attnum
SELECT attnum FROM pg_attribute WHERE attrelid = 'ab'::regclass AND attname = 'b'

query I
SELECT
ordinal_position
FROM
information_schema.columns
WHERE
table_name = 'ab'
AND column_name = 'b'
AND ordinal_position = $attnum;
----
2

statement ok
DROP TABLE ab
Loading

0 comments on commit 7f8bda1

Please sign in to comment.