Skip to content

Commit

Permalink
Merge #34726
Browse files Browse the repository at this point in the history
34726: sql: Fix panic when auto stats are computed on views r=andy-kimball a=andy-kimball

At startup, all crdb_internal.tables are enqueued in order to have stats
computed over them, if needed. However, that vtable includes views as well
as tables, and views cause the CREATE STATISTICS code to panic.

This commit fixes the panic (returns an error instead). It also prevents
the auto stats code from continually re-enqueuing the refresh request for
views and vtables, since they will fail every time.

We need to come back and fix this better by not enqueuing views in the
first place. However, that turned out to be problematic because the
crdb_internal.tables vtable has no column that distinguishes view from
tables.

Release note: None

Co-authored-by: Andrew Kimball <andyk@cockroachlabs.com>
  • Loading branch information
craig[bot] and andy-kimball committed Feb 7, 2019
2 parents ee130c0 + 3567d2f commit 4bdf0ad
Show file tree
Hide file tree
Showing 3 changed files with 38 additions and 12 deletions.
7 changes: 6 additions & 1 deletion pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/rowcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -111,7 +112,11 @@ func (n *createStatsNode) startJob(ctx context.Context, resultsCh chan<- tree.Da
}

if tableDesc.IsVirtualTable() {
return errors.Errorf("cannot create statistics on virtual tables")
return pgerror.NewError(pgerror.CodeWrongObjectTypeError, "cannot create statistics on virtual tables")
}

if tableDesc.IsView() {
return pgerror.NewError(pgerror.CodeWrongObjectTypeError, "cannot create statistics on views")
}

if err := n.p.CheckPrivilege(ctx, tableDesc, privilege.SELECT); err != nil {
Expand Down
29 changes: 19 additions & 10 deletions pkg/sql/stats/automatic_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -330,18 +330,27 @@ func (r *Refresher) maybeRefreshStats(

if err := r.refreshStats(ctx, tableID, asOf); err != nil {
pgerr, ok := errors.Cause(err).(*pgerror.Error)
if ok && pgerr.Code == pgerror.CodeUndefinedTableError {
// Wait so that the latest changes will be reflected according to the
// AS OF time, then try again.
timer := time.NewTimer(asOf)
defer timer.Stop()
select {
case <-timer.C:
break
case <-stopper.ShouldQuiesce():
if ok {
switch pgerr.Code {
case pgerror.CodeUndefinedTableError:
// Wait so that the latest changes will be reflected according to the
// AS OF time, then try again.
timer := time.NewTimer(asOf)
defer timer.Stop()
select {
case <-timer.C:
break
case <-stopper.ShouldQuiesce():
return
}
err = r.refreshStats(ctx, tableID, asOf)

case pgerror.CodeWrongObjectTypeError:
// Don't reschedule the refresh for this error, as it gets produced
// if we're trying to run auto statistics on a view.
// TODO(rytaft): Change code to not enqueue views to begin with.
return
}
err = r.refreshStats(ctx, tableID, asOf)
}
if err != nil {
// It's likely that another stats job was already running. Attempt
Expand Down
14 changes: 13 additions & 1 deletion pkg/sql/stats/automatic_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,8 @@ func TestMaybeRefreshStats(t *testing.T) {
sqlRun.Exec(t,
`CREATE DATABASE t;
CREATE TABLE t.a (k INT PRIMARY KEY, v CHAR);
INSERT INTO t.a VALUES (1, 'a');`)
INSERT INTO t.a VALUES (1, 'a');
CREATE VIEW t.vw AS SELECT k, v FROM t.a;`)

executor := s.InternalExecutor().(sqlutil.InternalExecutor)
descA := sqlbase.GetTableDescriptor(s.DB(), "t", "a")
Expand Down Expand Up @@ -84,6 +85,17 @@ func TestMaybeRefreshStats(t *testing.T) {
if err := checkStatsCount(ctx, cache, descA.ID, 2 /* expected */); err != nil {
t.Fatal(err)
}

// Ensure that attempt to refresh stats on view does not result in re-
// enqueuing the attempt.
// TODO(rytaft): Should not enqueue views to begin with.
descVW := sqlbase.GetTableDescriptor(s.DB(), "t", "vw")
refresher.maybeRefreshStats(ctx, s.Stopper(), descVW.ID, 0 /* rowsAffected */, 0 /* asOf */)
select {
case <-refresher.mutations:
t.Fatal("refresher should not re-enqueue attempt to create stats over view")
default:
}
}

func TestAverageRefreshTime(t *testing.T) {
Expand Down

0 comments on commit 4bdf0ad

Please sign in to comment.