Skip to content

Commit

Permalink
sql/stats: fix panic catching in the stats cache, show histogram
Browse files Browse the repository at this point in the history
This commit removes a top-level panic catcher from GetTableStats in the stats
cache, and instead adds a couple of lower level panic catchers. The problems
with the top-level catcher are:
- There are other entry points into the stats cache besides GetTableStats, so
  it missed some cases.
- Because the stats cache contains shared state, condition variables, mutexes,
  etc, it was possible to get into a scenario where a panic would bypass
  important logic (such as broadcasting when a resource is available), but the
  panic would be caught at the top level and leave the cache in a bad state.
The new panic catchers are below the level of this shared state manipulation,
but also ensure that all paths into the stats cache that might panic are
covered.

This commit also adds panic catching to SHOW HISTOGRAM so that decoding errors
do not crash the process when running SHOW HISTOGRAM.

Fixes #135940

Release note (bug fix): Fixed an issue where corrupted table statistics could
cause the cockroach process to crash.
  • Loading branch information
rytaft committed Nov 22, 2024
1 parent ba4f7ad commit 2484e05
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 16 deletions.
17 changes: 17 additions & 0 deletions pkg/sql/show_histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -62,6 +63,22 @@ func (p *planner) ShowHistogram(ctx context.Context, n *tree.ShowHistogram) (pla
return nil, fmt.Errorf("histogram %d not found", n.HistogramID)
}

// Guard against crashes in the code below .
defer func() {
if r := recover(); r != nil {
// Avoid crashing the process in case of a "safe" panic. This is only
// possible because the code does not update shared state and does not
// manipulate locks.
if ok, e := errorutil.ShouldCatch(r); ok {
err = e
} else {
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}
}()

histogram := &stats.HistogramData{}
histData := *row[0].(*tree.DBytes)
if err := protoutil.Unmarshal([]byte(histData), histogram); err != nil {
Expand Down
51 changes: 35 additions & 16 deletions pkg/sql/stats/stats_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -225,19 +225,6 @@ func (sc *TableStatisticsCache) GetTableStats(
if !statsUsageAllowed(table, sc.settings) {
return nil, nil
}
defer func() {
if r := recover(); r != nil {
// In the event of a "safe" panic, we only want to log the error and
// continue executing the query without stats for this table.
if ok, e := errorutil.ShouldCatch(r); ok {
err = e
} else {
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}
}()
forecast := forecastAllowed(table, sc.settings)
return sc.getTableStatsFromCache(
ctx, table.GetID(), &forecast, table.UserDefinedTypeColumns(),
Expand Down Expand Up @@ -639,9 +626,24 @@ func NewTableStatisticProto(datums tree.Datums) (*TableStatisticProto, error) {
// need to run a query to get user defined type metadata.
func (sc *TableStatisticsCache) parseStats(
ctx context.Context, datums tree.Datums,
) (*TableStatistic, *types.T, error) {
) (_ *TableStatistic, _ *types.T, err error) {
defer func() {
if r := recover(); r != nil {
// In the event of a "safe" panic, we only want to log the error and
// continue executing the query without stats for this table. This is only
// possible because the code does not update shared state and does not
// manipulate locks.
if ok, e := errorutil.ShouldCatch(r); ok {
err = e
} else {
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}
}()

var tsp *TableStatisticProto
var err error
tsp, err = NewTableStatisticProto(datums)
if err != nil {
return nil, nil, err
Expand Down Expand Up @@ -789,7 +791,7 @@ func (tsp *TableStatisticProto) IsAuto() bool {
// type that doesn't exist) and returns the rest (with no error).
func (sc *TableStatisticsCache) getTableStatsFromDB(
ctx context.Context, tableID descpb.ID, forecast bool, st *cluster.Settings,
) ([]*TableStatistic, map[descpb.ColumnID]*types.T, error) {
) (_ []*TableStatistic, _ map[descpb.ColumnID]*types.T, err error) {
getTableStatisticsStmt := `
SELECT
"tableID",
Expand Down Expand Up @@ -818,6 +820,23 @@ ORDER BY "createdAt" DESC, "columnIDs" DESC, "statisticID" DESC
return nil, nil, err
}

// Guard against crashes in the code below.
defer func() {
if r := recover(); r != nil {
// In the event of a "safe" panic, we only want to log the error and
// continue executing the query without stats for this table. This is only
// possible because the code does not update shared state and does not
// manipulate locks.
if ok, e := errorutil.ShouldCatch(r); ok {
err = e
} else {
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}
}()

var statsList []*TableStatistic
var udts map[descpb.ColumnID]*types.T
var ok bool
Expand Down

0 comments on commit 2484e05

Please sign in to comment.