Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
45831: sql: add a comment that JSON behaviour is similar to JSONB r=jordanlewis a=giorgosp

Closes #44465
Release note : None

50369: sql: fix NewUniquenessConstraintViolationError reporting r=spaskob a=spaskob

Release note (bug fix):
If NewUniquenessConstraintViolationError cannot initialize a row fetcher
it will only report this error to the client without wrapping it with
information about the actual constraint violation. This is confusing.

See #46276.

Old error:
`ERROR: column-id "2" does not exist`

New Error:
```
ERROR: duplicate key value (b)=(couldn't fetch value: column-id "2"
does not exist) violates unique constraint "t_secondary"
```

50744: sql: fix drop database - sequence ownership bug r=solongordon a=arulajmani

Previously, `DROP DATABASE CASCADE` would not work if the database
contained a sequence owned by a table in the database. This would
happen because of two separate reasons:
- If the sequence was dropped before the table, the table would try to
"double drop" the sequence as it owned it; this would result in an
error.
- If the table was dropped before the sequence, the sequence would try
to remove the ownership dependency from the table descriptor, which had
already been dropped; this would also result in an error.

This PR addresses both these issues separately. Sequences are no longer
double dropped when dropping tables. Additionally, no attempt is made
to remove the ownership dependency from the table descriptor if the
table descriptor has already been dropped.

Fixes #50712

Release note (bug fix): `DROP DATABASE CASCADE` now works as expected
even when the database has a sequence with an owner in it.

50961: cmd/roachtest: fix --zones flag to work on AWS r=petermattis a=petermattis

Previously the `--zones` flag only worked on GCE and Azure and was
silently ignored on AWS. Noticed in passing while trying to run a
roachtest on a different AWS zone.

Release note: None

51000: acceptance: skip TestDockerCLI/test_demo_partitioning r=knz a=tbg

See: #50970

Release note: None

Co-authored-by: George Papadrosou <[email protected]>
Co-authored-by: Spas Bojanov <[email protected]>
Co-authored-by: arulajmani <[email protected]>
Co-authored-by: Peter Mattis <[email protected]>
Co-authored-by: Tobias Schottdorf <[email protected]>
  • Loading branch information
6 people committed Jul 6, 2020
6 parents d6ab02f + a036e48 + fe967ee + 0ba9148 + 6c96546 + a9e8c33 commit 0b6e118
Show file tree
Hide file tree
Showing 11 changed files with 151 additions and 46 deletions.
42 changes: 23 additions & 19 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -853,21 +853,32 @@ func (s *clusterSpec) args() []string {
machineTypeArg := machineTypeFlag(machineType) + "=" + machineType
args = append(args, machineTypeArg)
}
if s.Zones != "" {
switch cloud {
case gce:
if s.Geo {
args = append(args, "--gce-zones="+s.Zones)
} else {
args = append(args, "--gce-zones="+firstZone(s.Zones))

if !local {
zones := s.Zones
if zones == "" {
zones = zonesF
}
if zones != "" {
if !s.Geo {
zones = firstZone(zones)
}
case azure:
args = append(args, "--azure-locations="+s.Zones)
default:
fmt.Fprintf(os.Stderr, "specifying zones is not yet supported on %s", cloud)
os.Exit(1)
var arg string
switch cloud {
case aws:
arg = "--aws-zones=" + zones
case gce:
arg = "--gce-zones=" + zones
case azure:
arg = "--azure-locations=" + zones
default:
fmt.Fprintf(os.Stderr, "specifying zones is not yet supported on %s", cloud)
os.Exit(1)
}
args = append(args, arg)
}
}

if s.Geo {
args = append(args, "--geo")
}
Expand Down Expand Up @@ -1175,13 +1186,6 @@ func (f *clusterFactory) newCluster(

sargs := []string{roachprod, "create", c.name, "-n", fmt.Sprint(c.spec.NodeCount)}
sargs = append(sargs, cfg.spec.args()...)
if !local && zonesF != "" && cfg.spec.Zones == "" {
if cfg.spec.Geo {
sargs = append(sargs, "--gce-zones="+zonesF)
} else {
sargs = append(sargs, "--gce-zones="+firstZone(zonesF))
}
}
if !cfg.useIOBarrier {
sargs = append(sargs, "--local-ssd-no-ext4-barrier")
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -334,7 +334,7 @@ func (n *alterTableNode) startExec(params runParams) error {
return err
}

if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop); err != nil {
if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop, true /* queueJob */); err != nil {
return err
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/drop_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -291,7 +291,7 @@ func (p *planner) dropTableImpl(

// Drop sequences that the columns of the table own
for _, col := range tableDesc.Columns {
if err := p.dropSequencesOwnedByCol(ctx, &col); err != nil {
if err := p.dropSequencesOwnedByCol(ctx, &col, queueJob); err != nil {
return droppedViews, err
}
}
Expand Down Expand Up @@ -338,7 +338,7 @@ func (p *planner) initiateDropTable(
drainName bool,
) error {
if tableDesc.Dropped() {
return fmt.Errorf("table %q is being dropped", tableDesc.Name)
return errors.Errorf("table %q is already being dropped", tableDesc.Name)
}

// If the table is not interleaved , use the delayed GC mechanism to
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/drop_index
Original file line number Diff line number Diff line change
Expand Up @@ -297,3 +297,19 @@ CREATE TABLE drop_index_test(a int); CREATE INDEX drop_index_test_index ON drop_
----
NOTICE: the data for dropped indexes is reclaimed asynchronously
HINT: The reclamation delay can be customized in the zone configuration for the table.

# test correct error reporting from NewUniquenessConstraintViolationError; see #46376
subtest new_uniqueness_constraint_error

statement ok
CREATE TABLE t (a INT PRIMARY KEY, b DECIMAL(10,1) NOT NULL DEFAULT(0), UNIQUE INDEX t_secondary (b), FAMILY (a, b));
INSERT INTO t VALUES (100, 500.5);

statement ok
BEGIN;
DROP INDEX t_secondary CASCADE;
ALTER TABLE t DROP COLUMN b;
INSERT INTO t SELECT a + 1 FROM t;

statement error pq: duplicate key value
UPSERT INTO t SELECT a + 1 FROM t;
18 changes: 0 additions & 18 deletions pkg/sql/logictest/testdata/logic_test/json_builtins
Original file line number Diff line number Diff line change
Expand Up @@ -324,29 +324,11 @@ SELECT to_json(x.*) FROM (VALUES (1,2)) AS x(a);
----
{"a": 1, "column2": 2}

# TODO(#44465): Implement the test cases below to be compatible with Postgres
# and delete this one
query T
SELECT to_json(x.*) FROM (VALUES (1,2)) AS x(column2);
----
{"column2": 2}

# Odd, but postgres-compatible
# query T
# SELECT to_json(x.*) FROM (VALUES (1,2)) AS x(a,a);
# ----
# {"a": 1, "a": 2}

# query T
# SELECT to_json(x.*) FROM (VALUES (1,2)) AS x(column1);
# ----
# {"column1": 1, "column2": 2}

# query T
# SELECT to_json(x.*) FROM (VALUES (1,2)) AS x(column2);
# ----
# {"column2": 1, "column2": 2}

# Regression test for #39502.
statement ok
SELECT json_agg((3808362714,))
Expand Down
87 changes: 87 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/sequences
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
# LogicTest: !3node-tenant(50840)
# see also files `drop_sequence`, `alter_sequence`, `rename_sequence`

# USING THE `lastval` FUNCTION
Expand Down Expand Up @@ -1095,3 +1096,89 @@ DROP SEQUENCE seq_50649

statement ok
DROP TABLE t_50649

subtest regression_50712

statement ok
CREATE DATABASE db_50712

statement ok
CREATE TABLE db_50712.t_50712(a INT PRIMARY KEY)

statement ok
CREATE SEQUENCE db_50712.seq_50712 OWNED BY db_50712.t_50712.a

statement ok
DROP DATABASE db_50712 CASCADE

# Same test like above, except the table is lexicographically less than the
# sequence, which results in drop database dropping the table before the
# sequence.
statement ok
CREATE DATABASE db_50712

statement ok
CREATE TABLE db_50712.a_50712(a INT PRIMARY KEY)

statement ok
CREATE SEQUENCE db_50712.seq_50712 OWNED BY db_50712.a_50712.a

statement ok
DROP DATABASE db_50712 CASCADE

# Same test like above, except the db is switched as the current db
statement ok
CREATE DATABASE db_50712

statement ok
SET DATABASE = db_50712

statement ok
CREATE TABLE a_50712(a INT PRIMARY KEY)

statement ok
CREATE SEQUENCE seq_50712 OWNED BY a_50712.a

statement ok
DROP DATABASE db_50712

statement ok
SET DATABASE = test

# Tests db drop.
# Sequence: outside db.
# Owner: inside db.
# The sequence should be automatically dropped.
statement ok
CREATE DATABASE db_50712

statement ok
CREATE TABLE db_50712.t_50712(a INT PRIMARY KEY)

statement ok
CREATE SEQUENCE seq_50712 OWNED BY db_50712.t_50712.a

statement ok
DROP DATABASE db_50712 CASCADE

statement error pq: relation "seq_50712" does not exist
SELECT * FROM seq_50712

# Tests db drop.
# Sequence: inside db
# Owner: outside db
# It should be possible to drop the table later.
statement ok
CREATE DATABASE db_50712

statement ok
CREATE TABLE t_50712(a INT PRIMARY KEY)

statement ok
CREATE SEQUENCE db_50712.seq_50712 OWNED BY t_50712.a

statement ok
DROP DATABASE db_50712 CASCADE

statement ok
DROP TABLE t_50712
6 changes: 5 additions & 1 deletion pkg/sql/row/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,11 @@ func NewUniquenessConstraintViolationError(
&sqlbase.DatumAlloc{},
tableArgs,
); err != nil {
return err
return pgerror.Newf(pgcode.UniqueViolation,
"duplicate key value (%s)=(%v) violates unique constraint %q",
strings.Join(index.ColumnNames, ","),
errors.Wrapf(err, "couldn't fetch value"),
index.Name)
}
f := singleKVFetcher{kvs: [1]roachpb.KeyValue{{Key: key}}}
if value != nil {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -2774,6 +2774,8 @@ may increase either contention or retry errors, or both.`,
})),

// JSON functions.
// The behavior of both the JSON and JSONB data types in CockroachDB is
// similar to the behavior of the JSONB data type in Postgres.

"json_to_recordset": makeBuiltin(tree.FunctionProperties{UnsupportedWithIssue: 33285, Category: categoryJSON}),
"jsonb_to_recordset": makeBuiltin(tree.FunctionProperties{UnsupportedWithIssue: 33285, Category: categoryJSON}),
Expand Down
13 changes: 11 additions & 2 deletions pkg/sql/sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -335,6 +335,11 @@ func removeSequenceOwnerIfExists(
if err != nil {
return err
}
// If the table descriptor has already been dropped, there is no need to
// remove the reference.
if tableDesc.Dropped() {
return nil
}
col, err := tableDesc.FindColumnByID(opts.SequenceOwner.OwnerColumnID)
if err != nil {
return err
Expand Down Expand Up @@ -469,17 +474,21 @@ func maybeAddSequenceDependencies(
// dropSequencesOwnedByCol drops all the sequences from col.OwnsSequenceIDs.
// Called when the respective column (or the whole table) is being dropped.
func (p *planner) dropSequencesOwnedByCol(
ctx context.Context, col *sqlbase.ColumnDescriptor,
ctx context.Context, col *sqlbase.ColumnDescriptor, queueJob bool,
) error {
for _, sequenceID := range col.OwnsSequenceIds {
seqDesc, err := p.Tables().GetMutableTableVersionByID(ctx, sequenceID, p.txn)
if err != nil {
return err
}
// This sequence is already getting dropped. Don't do it twice.
if seqDesc.Dropped() {
continue
}
jobDesc := fmt.Sprintf("removing sequence %q dependent on column %q which is being dropped",
seqDesc.Name, col.ColName())
if err := p.dropSequenceImpl(
ctx, seqDesc, true /* queueJob */, jobDesc, tree.DropRestrict,
ctx, seqDesc, queueJob, jobDesc, tree.DropRestrict,
); err != nil {
return err
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package sql

import (
"context"
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
Expand Down Expand Up @@ -198,7 +197,8 @@ func (p *planner) writeSchemaChange(
}
if tableDesc.Dropped() {
// We don't allow schema changes on a dropped table.
return fmt.Errorf("table %q is being dropped", tableDesc.Name)
return errors.Errorf("no schema changes allowed on table %q as it is being dropped",
tableDesc.Name)
}
if err := p.createOrUpdateSchemaChangeJob(ctx, tableDesc, jobDesc, mutationID); err != nil {
return err
Expand All @@ -214,7 +214,8 @@ func (p *planner) writeSchemaChangeToBatch(
}
if tableDesc.Dropped() {
// We don't allow schema changes on a dropped table.
return fmt.Errorf("table %q is being dropped", tableDesc.Name)
return errors.Errorf("no schema changes allowed on table %q as it is being dropped",
tableDesc.Name)
}
return p.writeTableDescToBatch(ctx, tableDesc, b)
}
Expand Down

0 comments on commit 0b6e118

Please sign in to comment.