Skip to content

Commit

Permalink
Merge 2484e05 into blathers/backport-release-24.2-135944
Browse files Browse the repository at this point in the history
  • Loading branch information
blathers-crl[bot] authored Nov 22, 2024
2 parents 905be94 + 2484e05 commit cc6daf1
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 @@ -218,19 +218,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 @@ -632,9 +619,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 @@ -782,7 +784,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 @@ -811,6 +813,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 cc6daf1

Please sign in to comment.