Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
51621: descs: push descriptor type hydration into the desc.Collection r=rohany a=rohany

Fixes #49484.
Preparation for #51385.

Up until now, the `planner` was responsible for installing user defined
type metadata in tables that contained user defined types. This was
slightly messy and caused leakage of responsibility regarding when
descriptors had user defined types vs when they didn't. This commit
pushes that responsibility into the `descs.Collection`. It also paves
the way for work to avoid copying `ImmutableTableDescriptor`s that
contain user defined types every time that they need hydration.

Release note: None

51828: sql,stats: fix flakes in TestCacheWait and TestQueryCache r=jordanlewis a=rytaft

**stats: fix flake in TestCacheWait**

This commit fixes a flake in `TestCacheWait`, which was introduced
by #51616. That PR changed the call to `InvalidateTableStats` in
`TestCacheWait` to `RefreshTableStats`, but that change should not have
been made. The purpose of the test is to test that the cache invalidation
and waiting mechanisms work correctly, and therefore it must call
`InvalidateTableStats`, not `RefreshTableStats`.

Fixes #51712

**sql: fix flake in TestQueryCache/group/statschange**

This commit fixes a flake in TestQueryCache/group/statschange,
which was introduced by #51616. That PR made updates to the stats
cache asynchronous, so we can no longer expect the query cache to
be invalidated immediately after a stats update. This commit fixes
the problem by introducing a retry mechanism into the test.

Fixes #51693

51837: sql: don't show the "hidden" column flag in EXPLAIN r=RaduBerinde a=RaduBerinde

The ResultColumns.Hidden flag is used internally; it does not carry
useful information for users and should not be visible in EXPLAIN.

Release note (sql change): EXPLAIN no longer shows the "hidden"
annotation for columns.

Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
4 people committed Jul 23, 2020
4 parents 81cc169 + eaeb7c1 + 1bb2a55 + 42e24c4 commit fb77854
Show file tree
Hide file tree
Showing 18 changed files with 225 additions and 207 deletions.
16 changes: 0 additions & 16 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -655,21 +654,6 @@ func (sc *SchemaChanger) truncateIndexes(
if err != nil {
return err
}

// Hydrate types used in the retrieved table.
// TODO (rohany): This can be removed once table access from the
// desc.Collection returns tables with hydrated types.
typLookup := func(ctx context.Context, id sqlbase.ID) (*tree.TypeName, sqlbase.TypeDescriptorInterface, error) {
return resolver.ResolveTypeDescByID(ctx, txn, sc.execCfg.Codec, id, tree.ObjectLookupFlags{})
}
if err := sqlbase.HydrateTypesInTableDescriptor(
ctx,
tableDesc.TableDesc(),
sqlbase.TypeLookupFunc(typLookup),
); err != nil {
return err
}

rd, err := row.MakeDeleter(
ctx,
txn,
Expand Down
66 changes: 62 additions & 4 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -162,7 +163,11 @@ func (tc *Collection) GetMutableTableDescriptor(
if !ok {
return nil, nil
}
return mutDesc, nil
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, mutDesc)
if err != nil {
return nil, err
}
return hydrated.(*sqlbase.MutableTableDescriptor), nil
}

func (tc *Collection) getMutableObjectDescriptor(
Expand Down Expand Up @@ -288,7 +293,11 @@ func (tc *Collection) GetTableVersion(
}
return nil, nil
}
return table, nil
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, table)
if err != nil {
return nil, err
}
return hydrated.(*sqlbase.ImmutableTableDescriptor), nil
}

func (tc *Collection) getObjectVersion(
Expand Down Expand Up @@ -434,7 +443,11 @@ func (tc *Collection) GetTableVersionByID(
return nil, sqlbase.NewUndefinedRelationError(
&tree.TableRef{TableID: int64(tableID)})
}
return table, nil
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, table)
if err != nil {
return nil, err
}
return hydrated.(*sqlbase.ImmutableTableDescriptor), nil
}

func (tc *Collection) getDescriptorVersionByID(
Expand Down Expand Up @@ -506,7 +519,12 @@ func (tc *Collection) GetMutableTableVersionByID(
if err != nil {
return nil, err
}
return desc.(*sqlbase.MutableTableDescriptor), nil
table := desc.(*sqlbase.MutableTableDescriptor)
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, table)
if err != nil {
return nil, err
}
return hydrated.(*sqlbase.MutableTableDescriptor), nil
}

func (tc *Collection) getMutableDescriptorByID(
Expand All @@ -528,6 +546,46 @@ func (tc *Collection) getMutableDescriptorByID(
return desc, nil
}

// hydrateTypesInTableDesc installs user defined type metadata in all types.T
// present in the input TableDescriptor. It always returns the same type of
// TableDescriptor that was passed in. It ensures that ImmutableTableDescriptors
// are not modified during the process of metadata installation.
func (tc *Collection) hydrateTypesInTableDesc(
ctx context.Context, txn *kv.Txn, desc sqlbase.TableDescriptorInterface,
) (sqlbase.TableDescriptorInterface, error) {
getType := func(ctx context.Context, id sqlbase.ID) (*tree.TypeName, sqlbase.TypeDescriptorInterface, error) {
// TODO (rohany): Use the collection API's here.
return resolver.ResolveTypeDescByID(ctx, txn, tc.codec(), id, tree.ObjectLookupFlags{})
}
switch t := desc.(type) {
case *sqlbase.MutableTableDescriptor:
// It is safe to hydrate directly into MutableTableDescriptor since it is
// not shared.
return desc, sqlbase.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), sqlbase.TypeLookupFunc(getType))
case *sqlbase.ImmutableTableDescriptor:
// ImmutableTableDescriptors need to be copied before hydration, because
// they are potentially read by multiple threads. If there aren't any user
// defined types in the descriptor, then return early.
if !t.ContainsUserDefinedTypes() {
return desc, nil
}

// TODO (rohany, ajwerner): Here we would look into the cached set of
// hydrated table descriptors and potentially return without having to
// make a copy. However, we could avoid hitting the cache if any of the
// user defined types have been modified in this transaction.

// Make a copy of the underlying descriptor before hydration.
descBase := protoutil.Clone(t.TableDesc()).(*sqlbase.TableDescriptor)
if err := sqlbase.HydrateTypesInTableDescriptor(ctx, descBase, sqlbase.TypeLookupFunc(getType)); err != nil {
return nil, err
}
return sqlbase.NewImmutableTableDescriptor(*descBase), nil
default:
return desc, nil
}
}

// ReleaseSpecifiedLeases releases the leases for the descriptors with ids in
// the passed slice. Errors are logged but ignored.
func (tc *Collection) ReleaseSpecifiedLeases(ctx context.Context, descs []lease.IDVersion) {
Expand Down
20 changes: 10 additions & 10 deletions pkg/sql/opt/exec/execbuilder/testdata/explain
Original file line number Diff line number Diff line change
Expand Up @@ -601,16 +601,16 @@ CREATE TABLE tc (a INT, b INT, INDEX c(a), FAMILY "primary" (a, b, rowid))
query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM tc WHERE a = 10 ORDER BY b
----
· distribution local · ·
· vectorized true · ·
sort · · (a, b) +b
│ order +b · ·
└── index-join · · (a, b) ·
│ table tc@primary · ·
│ key columns rowid · ·
└── scan · · (a, rowid[hidden]) ·
· table tc@c · ·
· spans /10-/11 · ·
· distribution local · ·
· vectorized true · ·
sort · · (a, b) +b
│ order +b · ·
└── index-join · · (a, b) ·
│ table tc@primary · ·
│ key columns rowid · ·
└── scan · · (a, rowid) ·
· table tc@c · ·
· spans /10-/11 · ·

query TTTTT colnames
EXPLAIN (TYPES) INSERT INTO t VALUES (1, 2)
Expand Down
32 changes: 16 additions & 16 deletions pkg/sql/opt/exec/execbuilder/testdata/insert
Original file line number Diff line number Diff line change
Expand Up @@ -428,8 +428,8 @@ INSERT INTO insert_t (SELECT length(k), 2 FROM kv ORDER BY k || v LIMIT 10) RETU
· vectorized false · ·
render · · ("?column?") ·
│ render 0 x + v · ·
└── run · · (x, v, rowid[hidden]) ·
└── insert · · (x, v, rowid[hidden]) ·
└── run · · (x, v, rowid) ·
└── insert · · (x, v, rowid) ·
│ into insert_t(x, v, rowid) · ·
│ strategy inserter · ·
└── render · · (length, "?column?", column11) ·
Expand Down Expand Up @@ -462,18 +462,18 @@ BEGIN; ALTER TABLE mutation DROP COLUMN y
query TTTTT
EXPLAIN (VERBOSE) INSERT INTO mutation(x) VALUES (2) RETURNING *
----
· distribution local · ·
· vectorized false · ·
render · · (x) ·
│ render 0 x · ·
└── run · · (x, rowid[hidden]) ·
└── insert-fast-path · · (x, rowid[hidden]) ·
· into mutation(x, rowid, y) · ·
· strategy inserter · ·
· size 3 columns, 1 row · ·
· row 0, expr 0 2 · ·
· row 0, expr 1 unique_rowid() · ·
· row 0, expr 2 10 · ·
· distribution local · ·
· vectorized false · ·
render · · (x) ·
│ render 0 x · ·
└── run · · (x, rowid) ·
└── insert-fast-path · · (x, rowid) ·
· into mutation(x, rowid, y) · ·
· strategy inserter · ·
· size 3 columns, 1 row · ·
· row 0, expr 0 2 · ·
· row 0, expr 1 unique_rowid() · ·
· row 0, expr 2 10 · ·

statement ok
ROLLBACK
Expand Down Expand Up @@ -527,8 +527,8 @@ root · ·
└── spool · · (z) ·
└── render · · (z) ·
│ render 0 z · ·
└── run · · (z, rowid[hidden]) ·
└── insert · · (z, rowid[hidden]) ·
└── run · · (z, rowid) ·
└── insert · · (z, rowid) ·
│ into xyz(x, y, z, rowid) · ·
│ strategy inserter · ·
└── render · · (a, b, c, column11) ·
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/opt/exec/execbuilder/testdata/orderby
Original file line number Diff line number Diff line change
Expand Up @@ -907,15 +907,15 @@ CREATE TABLE xyz (x INT, y INT, z INT, INDEX(z,y))
query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM xyz WHERE z=1 AND x=y ORDER BY x;
----
· distribution local · ·
· vectorized true · ·
sort · · (x, y, z) +x
│ order +x · ·
└── filter · · (x, y, z) ·
│ filter x = y · ·
└── index-join · · (x, y, z) ·
│ table xyz@primary · ·
│ key columns rowid · ·
└── scan · · (y, z, rowid[hidden]) ·
· table xyz@xyz_z_y_idx · ·
· spans /1/!NULL-/2 · ·
· distribution local · ·
· vectorized true · ·
sort · · (x, y, z) +x
│ order +x · ·
└── filter · · (x, y, z) ·
│ filter x = y · ·
└── index-join · · (x, y, z) ·
│ table xyz@primary · ·
│ key columns rowid · ·
└── scan · · (y, z, rowid) ·
· table xyz@xyz_z_y_idx · ·
· spans /1/!NULL-/2 · ·
20 changes: 10 additions & 10 deletions pkg/sql/opt/exec/execbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -238,11 +238,11 @@ scan · · () ·
query TTTTT
EXPLAIN (VERBOSE) SELECT rowid FROM [54(3) AS num_ref_alias]
----
· distribution local · ·
· vectorized true · ·
scan · · (rowid[hidden]) ·
· table num_ref_hidden@primary · ·
· spans FULL SCAN · ·
· distribution local · ·
· vectorized true · ·
scan · · (rowid) ·
· table num_ref_hidden@primary · ·
· spans FULL SCAN · ·

query error pq: \[666\(1\) AS num_ref_alias\]: relation \"\[666\]\" does not exist
EXPLAIN (VERBOSE) SELECT * FROM [666(1) AS num_ref_alias]
Expand Down Expand Up @@ -400,11 +400,11 @@ scan · · (x, y) ·
query TTTTT
EXPLAIN (VERBOSE) SELECT x, y, rowid FROM b WHERE rowid > 0
----
· distribution local · ·
· vectorized true · ·
scan · · (x, y, rowid[hidden]) ·
· table b@primary · ·
· spans /1- · ·
· distribution local · ·
· vectorized true · ·
scan · · (x, y, rowid) ·
· table b@primary · ·
· spans /1- · ·

statement ok
DROP TABLE b
Expand Down
52 changes: 26 additions & 26 deletions pkg/sql/opt/exec/execbuilder/testdata/select_index
Original file line number Diff line number Diff line change
Expand Up @@ -966,40 +966,40 @@ CREATE TABLE xy (x INT, y INT, INDEX (y))
query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM xy WHERE y IS NOT DISTINCT FROM NULL
----
· distribution local · ·
· vectorized true · ·
index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid[hidden]) ·
· table xy@xy_y_idx · ·
· spans /NULL-/!NULL · ·
· distribution local · ·
· vectorized true · ·
index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid) ·
· table xy@xy_y_idx · ·
· spans /NULL-/!NULL · ·

query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM xy WHERE y IS NOT DISTINCT FROM 4
----
· distribution local · ·
· vectorized true · ·
index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid[hidden]) ·
· table xy@xy_y_idx · ·
· spans /4-/5 · ·
· distribution local · ·
· vectorized true · ·
index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid) ·
· table xy@xy_y_idx · ·
· spans /4-/5 · ·

query TTTTT
EXPLAIN (VERBOSE) SELECT x FROM xy WHERE y > 0 AND y < 2 ORDER BY y
----
· distribution local · ·
· vectorized true · ·
render · · (x) ·
│ render 0 x · ·
└── index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid[hidden]) ·
· table xy@xy_y_idx · ·
· spans /1-/2 · ·
· distribution local · ·
· vectorized true · ·
render · · (x) ·
│ render 0 x · ·
└── index-join · · (x, y) ·
│ table xy@primary · ·
│ key columns rowid · ·
└── scan · · (y, rowid) ·
· table xy@xy_y_idx · ·
· spans /1-/2 · ·

query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM xy WHERE y IS DISTINCT FROM NULL
Expand Down
Loading

0 comments on commit fb77854

Please sign in to comment.