Skip to content

Commit

Permalink
statistics: do not ignore DDL event handing errors (pingcap#58980) (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot authored Feb 11, 2025
1 parent 6f761e2 commit d90faf1
Show file tree
Hide file tree
Showing 5 changed files with 51 additions and 16 deletions.
7 changes: 7 additions & 0 deletions pkg/ddl/notifier/subscribe.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"context"
goerr "errors"
"fmt"
"strings"
"time"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -160,6 +161,12 @@ func (n *DDLNotifier) start() {
return
case <-ticker.C:
if err := n.processEvents(ctx); err != nil {
intest.Assert(
errors.ErrorEqual(err, context.Canceled) ||
strings.Contains(err.Error(), "mock handleTaskOnce error") ||
strings.Contains(err.Error(), "session pool closed"),
fmt.Sprintf("error processing events: %v", err),
)
logutil.Logger(ctx).Error("Error processing events", zap.Error(err))
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/statistics/handle/ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ go_test(
timeout = "short",
srcs = ["ddl_test.go"],
flaky = True,
shard_count = 21,
shard_count = 22,
deps = [
":ddl",
"//pkg/ddl/notifier",
Expand Down
22 changes: 12 additions & 10 deletions pkg/statistics/handle/ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,17 @@ package ddl

import (
"context"
"fmt"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/ddl/notifier"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/statistics/handle/lockstats"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/statistics/handle/storage"
"github.com/pingcap/tidb/pkg/statistics/handle/types"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"go.uber.org/zap"
"github.com/pingcap/tidb/pkg/util/intest"
)

type ddlHandlerImpl struct {
Expand All @@ -50,16 +51,17 @@ func NewDDLHandler(

// HandleDDLEvent begins to process a ddl task.
func (h *ddlHandlerImpl) HandleDDLEvent(ctx context.Context, sctx sessionctx.Context, s *notifier.SchemaChangeEvent) error {
// Ideally, we shouldn't allow any errors to be ignored, but for now, some queries can fail.
// Temporarily ignore the error and we need to check all queries to ensure they are correct.
if err := h.sub.handle(ctx, sctx, s); err != nil {
statslogutil.StatsLogger().Warn(
"failed to handle DDL event",
zap.String("event", s.String()),
zap.Error(err),
err := h.sub.handle(ctx, sctx, s)
if err != nil {
intest.Assert(
errors.ErrorEqual(err, context.Canceled) ||
strings.Contains(err.Error(), "mock handleTaskOnce error") ||
strings.Contains(err.Error(), "session pool closed"),
fmt.Sprintf("handle ddl event failed, err: %v", err),
)
}
return nil

return err
}

// DDLEventCh returns ddl events channel in handle.
Expand Down
28 changes: 27 additions & 1 deletion pkg/statistics/handle/ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1390,8 +1390,34 @@ func TestDumpStatsDeltaBeforeHandleDDLEvent(t *testing.T) {
tk.MustExec("insert into t values (1), (2), (3)")
h := dom.StatsHandle()
require.NoError(t, h.DumpStatsDeltaToKV(true))
// Also manually insert a histogram record.
is := dom.InfoSchema()
tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
require.NoError(t, err)
_, err = tk.Exec("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version) values (?, 0, ?, 0, ?)", tbl.Meta().ID, 1, 1)
require.NoError(t, err)
// Find the DDL event.
event := findEvent(h.DDLEventCh(), model.ActionCreateTable)
err := statstestutil.HandleDDLEventWithTxn(h, event)
err = statstestutil.HandleDDLEventWithTxn(h, event)
require.NoError(t, err)
}

func TestDumpStatsDeltaBeforeHandleAddColumnEvent(t *testing.T) {
store, do := testkit.CreateMockStoreAndDomain(t)
testKit := testkit.NewTestKit(t, store)
testKit.MustExec("use test")
testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))")
// Insert some data.
testKit.MustExec("insert into t values (1, 2), (2, 3), (3, 4)")
testKit.MustExec("analyze table t")
// Add column.
testKit.MustExec("alter table t add column c10 int")
// Insert some data.
testKit.MustExec("insert into t values (4, 5, 6)")
// Analyze table to force create the histogram meta record.
testKit.MustExec("analyze table t")
// Find the add column event.
event := findEvent(do.StatsHandle().DDLEventCh(), model.ActionAddColumn)
err := statstestutil.HandleDDLEventWithTxn(do.StatsHandle(), event)
require.NoError(t, err)
}
8 changes: 4 additions & 4 deletions pkg/statistics/handle/storage/save.go
Original file line number Diff line number Diff line change
Expand Up @@ -478,7 +478,7 @@ func InsertColStats2KV(
continue
}

// If this stats exists, we insert histogram meta first, the distinct_count will always be one.
// If this stats doest not exist, we insert histogram meta first, the distinct_count will always be one.
if _, err = util.ExecWithCtx(
ctx, sctx,
`insert into mysql.stats_histograms
Expand Down Expand Up @@ -521,15 +521,15 @@ func InsertTableStats2KV(
}
if _, err = util.ExecWithCtx(
ctx, sctx,
"insert into mysql.stats_meta (version, table_id) values(%?, %?)",
"insert ignore into mysql.stats_meta (version, table_id) values(%?, %?)",
startTS, physicalID,
); err != nil {
return 0, errors.Trace(err)
}
for _, col := range info.Columns {
if _, err = util.ExecWithCtx(
ctx, sctx,
`insert into mysql.stats_histograms
`insert ignore into mysql.stats_histograms
(table_id, is_index, hist_id, distinct_count, version)
values (%?, 0, %?, 0, %?)`,
physicalID, col.ID, startTS,
Expand All @@ -540,7 +540,7 @@ func InsertTableStats2KV(
for _, idx := range info.Indices {
if _, err = util.ExecWithCtx(
ctx, sctx,
`insert into mysql.stats_histograms
`insert ignore into mysql.stats_histograms
(table_id, is_index, hist_id, distinct_count, version)
values(%?, 1, %?, 0, %?)`,
physicalID, idx.ID, startTS,
Expand Down

0 comments on commit d90faf1

Please sign in to comment.