From 412f4efb49c347d8504a95824d1fc43cf5a15306 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 23 Jan 2025 17:27:01 +0800 Subject: [PATCH 1/7] ddl: fix memory state is wrongly updated before persistence Signed-off-by: lance6716 --- pkg/ddl/notifier/subscribe.go | 15 +++-- pkg/ddl/notifier/testkit_test.go | 98 ++++++++++++++++++++++++++++ pkg/ddl/session/session.go | 14 ++++ pkg/ddl/session/session_pool_test.go | 50 ++++++++++++++ 4 files changed, 171 insertions(+), 6 deletions(-) diff --git a/pkg/ddl/notifier/subscribe.go b/pkg/ddl/notifier/subscribe.go index 9975d29021495..a922dc97abc20 100644 --- a/pkg/ddl/notifier/subscribe.go +++ b/pkg/ddl/notifier/subscribe.go @@ -268,15 +268,20 @@ func (n *DDLNotifier) processEventForHandler( if (change.processedByFlag & (1 << handlerID)) != 0 { return nil } + newFlag := change.processedByFlag | (1 << handlerID) - if err = session.Begin(ctx); err != nil { + if err = session.BeginPessimistic(ctx); err != nil { return errors.Trace(err) } defer func() { - if err == nil { - err = errors.Trace(session.Commit(ctx)) - } else { + if err != nil { session.Rollback() + return + } + + err = errors.Trace(session.Commit(ctx)) + if err == nil { + change.processedByFlag = newFlag } }() @@ -293,7 +298,6 @@ func (n *DDLNotifier) processEventForHandler( zap.Duration("duration", time.Since(now))) } - newFlag := change.processedByFlag | (1 << handlerID) if err = n.store.UpdateProcessed( ctx, session, @@ -303,7 +307,6 @@ func (n *DDLNotifier) processEventForHandler( ); err != nil { return errors.Trace(err) } - change.processedByFlag = newFlag return nil } diff --git a/pkg/ddl/notifier/testkit_test.go b/pkg/ddl/notifier/testkit_test.go index 17e92a8c1f144..3cb6fc1d6d3e5 100644 --- a/pkg/ddl/notifier/testkit_test.go +++ b/pkg/ddl/notifier/testkit_test.go @@ -489,3 +489,101 @@ func TestBeginTwice(t *testing.T) { require.NoError(t, err) require.NotContains(t, string(content), "context provider not set") } + +func TestHandlersSeePessimisticTxnError(t *testing.T) { + // 1. One always fails + // 2. One always succeeds + // Make sure events don't get lost after the second handler succeeds. + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test") + tk.MustExec("DROP TABLE IF EXISTS " + ddl.NotifierTableName) + tk.MustExec(ddl.NotifierTableSQL) + ctx := context.Background() + s := notifier.OpenTableStore("test", ddl.NotifierTableName) + sessionPool := util.NewSessionPool( + 4, + func() (pools.Resource, error) { + return testkit.NewTestKit(t, store).Session(), nil + }, + nil, + nil, + ) + n := notifier.NewDDLNotifier(sessionPool, s, 50*time.Millisecond) + // Always fails + failHandler := func(_ context.Context, sctx sessionctx.Context, _ *notifier.SchemaChangeEvent) error { + // Mock a duplicate key error + _, err := sctx.GetSQLExecutor().Execute(ctx, "INSERT INTO test."+ddl.NotifierTableName+" VALUES(1, -1, 'some', 0)") + return err + } + // Always succeeds + successHandler := func(context.Context, sessionctx.Context, *notifier.SchemaChangeEvent) error { + return nil + } + n.RegisterHandler(2, successHandler) + n.RegisterHandler(1, failHandler) + n.OnBecomeOwner() + tk2 := testkit.NewTestKit(t, store) + se := sess.NewSession(tk2.Session()) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) + err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) + require.NoError(t, err) + require.Never(t, func() bool { + changes := make([]*notifier.SchemaChange, 8) + result, closeFn := s.List(ctx, se) + count, err2 := result.Read(changes) + require.NoError(t, err2) + closeFn() + return count == 0 + }, time.Second, 50*time.Millisecond) +} + +func TestCommitFailed(t *testing.T) { + // Make sure events don't get lost if internal txn commit failed. + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test") + tk.MustExec("set global tidb_enable_metadata_lock=0") + tk.MustExec("DROP TABLE IF EXISTS " + ddl.NotifierTableName) + tk.MustExec(ddl.NotifierTableSQL) + tk.MustExec("CREATE TABLE subscribe_table (id INT PRIMARY KEY, c INT)") + tk.MustExec("INSERT INTO subscribe_table VALUES (1, 1)") + + ctx := context.Background() + s := notifier.OpenTableStore("test", ddl.NotifierTableName) + sessionPool := util.NewSessionPool( + 4, + func() (pools.Resource, error) { + return testkit.NewTestKit(t, store).Session(), nil + }, + nil, + nil, + ) + n := notifier.NewDDLNotifier(sessionPool, s, 50*time.Millisecond) + handler := func(_ context.Context, sctx sessionctx.Context, _ *notifier.SchemaChangeEvent) error { + // pessimistic + DDL will cause an "infoschema is changed" error at commit time. + _, err := sctx.GetSQLExecutor().Execute( + ctx, "UPDATE test.subscribe_table SET c = c + 1 WHERE id = 1", + ) + require.NoError(t, err) + + tk.MustExec("TRUNCATE test.subscribe_table") + return nil + } + n.RegisterHandler(notifier.TestHandlerID, handler) + n.OnBecomeOwner() + tk2 := testkit.NewTestKit(t, store) + se := sess.NewSession(tk2.Session()) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) + err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) + require.NoError(t, err) + require.Never(t, func() bool { + changes := make([]*notifier.SchemaChange, 8) + result, closeFn := s.List(ctx, se) + count, err2 := result.Read(changes) + require.NoError(t, err2) + closeFn() + return count == 0 + }, time.Second, 50*time.Millisecond) + n.OnRetireOwner() +} diff --git a/pkg/ddl/session/session.go b/pkg/ddl/session/session.go index 4875168e7720d..0aeed9d7579a4 100644 --- a/pkg/ddl/session/session.go +++ b/pkg/ddl/session/session.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -49,6 +50,19 @@ func (s *Session) Begin(ctx context.Context) error { return nil } +// BeginPessimistic starts a pessimistic transaction. +func (s *Session) BeginPessimistic(ctx context.Context) error { + err := sessiontxn.GetTxnManager(s.Context).EnterNewTxn(ctx, &sessiontxn.EnterNewTxnRequest{ + Type: sessiontxn.EnterNewTxnDefault, + TxnMode: ast.Pessimistic, + }) + if err != nil { + return err + } + s.GetSessionVars().SetInTxn(true) + return nil +} + // Commit commits the transaction. func (s *Session) Commit(ctx context.Context) error { s.StmtCommit(ctx) diff --git a/pkg/ddl/session/session_pool_test.go b/pkg/ddl/session/session_pool_test.go index d3b8080c1b658..1c8ec934789b0 100644 --- a/pkg/ddl/session/session_pool_test.go +++ b/pkg/ddl/session/session_pool_test.go @@ -17,6 +17,7 @@ package session_test import ( "context" "testing" + "time" "github.com/ngaut/pools" "github.com/pingcap/tidb/pkg/ddl/session" @@ -66,3 +67,52 @@ func TestSessionPool(t *testing.T) { } require.Equal(t, uint64(0), targetTS) } + +func TestPessimisticTxn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int primary key, b int)") + tk.MustExec("insert into t values (1, 1)") + + resourcePool := pools.NewResourcePool(func() (pools.Resource, error) { + newTk := testkit.NewTestKit(t, store) + return newTk.Session(), nil + }, 4, 4, 0) + pool := session.NewSessionPool(resourcePool) + ctx := context.Background() + + sessCtx, err := pool.Get() + require.NoError(t, err) + se := session.NewSession(sessCtx) + sessCtx2, err := pool.Get() + require.NoError(t, err) + se2 := session.NewSession(sessCtx2) + + err = se.BeginPessimistic(ctx) + require.NoError(t, err) + err = se2.BeginPessimistic(ctx) + require.NoError(t, err) + _, err = se.Execute(ctx, "update test.t set b = b + 1 where a = 1", "ut") + require.NoError(t, err) + done := make(chan struct{}, 1) + go func() { + _, err := se2.Execute(ctx, "update test.t set b = b + 1 where a = 1", "ut") + require.NoError(t, err) + done <- struct{}{} + err = se2.Commit(ctx) + require.NoError(t, err) + close(done) + }() + + time.Sleep(100 * time.Millisecond) + // because this is a pessimistic transaction, the second transaction should be blocked + require.Len(t, done, 0) + err = se.Commit(ctx) + require.NoError(t, err) + <-done + _, ok := <-done + require.False(t, ok) + pool.Put(sessCtx) + pool.Put(sessCtx2) +} From d0839aa49e10273aff8c33d20dd2ca1536361a82 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 23 Jan 2025 17:38:10 +0800 Subject: [PATCH 2/7] fix bazel Signed-off-by: lance6716 --- pkg/ddl/notifier/BUILD.bazel | 2 +- pkg/ddl/session/BUILD.bazel | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/ddl/notifier/BUILD.bazel b/pkg/ddl/notifier/BUILD.bazel index 52e1b14ed10db..b5e5118264f0f 100644 --- a/pkg/ddl/notifier/BUILD.bazel +++ b/pkg/ddl/notifier/BUILD.bazel @@ -36,7 +36,7 @@ go_test( ], embed = [":notifier"], flaky = True, - shard_count = 10, + shard_count = 12, deps = [ "//pkg/ddl", "//pkg/ddl/session", diff --git a/pkg/ddl/session/BUILD.bazel b/pkg/ddl/session/BUILD.bazel index e982b11627cf2..70365492a8c10 100644 --- a/pkg/ddl/session/BUILD.bazel +++ b/pkg/ddl/session/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/domain/infosync", "//pkg/kv", "//pkg/metrics", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", From d89ecf460cbf8afb24593b197351b9268af3a775 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 23 Jan 2025 20:11:27 +0800 Subject: [PATCH 3/7] revert mdl change Signed-off-by: lance6716 --- pkg/ddl/notifier/testkit_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/ddl/notifier/testkit_test.go b/pkg/ddl/notifier/testkit_test.go index 3cb6fc1d6d3e5..30dd5f22778e9 100644 --- a/pkg/ddl/notifier/testkit_test.go +++ b/pkg/ddl/notifier/testkit_test.go @@ -544,6 +544,9 @@ func TestCommitFailed(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("USE test") tk.MustExec("set global tidb_enable_metadata_lock=0") + t.Cleanup(func() { + tk.MustExec("set global tidb_enable_metadata_lock=1") + }) tk.MustExec("DROP TABLE IF EXISTS " + ddl.NotifierTableName) tk.MustExec(ddl.NotifierTableSQL) tk.MustExec("CREATE TABLE subscribe_table (id INT PRIMARY KEY, c INT)") From bef13bf134d125a8db59f82a0f91e629052d7eda Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 24 Jan 2025 17:46:39 +0800 Subject: [PATCH 4/7] fix CI Signed-off-by: lance6716 --- pkg/ddl/notifier/store.go | 24 ++++++++++++++++++------ pkg/ddl/notifier/subscribe.go | 9 +++------ pkg/ddl/notifier/testkit_test.go | 6 +++--- 3 files changed, 24 insertions(+), 15 deletions(-) diff --git a/pkg/ddl/notifier/store.go b/pkg/ddl/notifier/store.go index cd168f3d50faa..b365da0fb51a9 100644 --- a/pkg/ddl/notifier/store.go +++ b/pkg/ddl/notifier/store.go @@ -35,7 +35,8 @@ type Store interface { se *sess.Session, ddlJobID int64, multiSchemaChangeID int64, - processedBy uint64, + oldProcessedBy uint64, + newProcessedBy uint64, ) error DeleteAndCommit(ctx context.Context, se *sess.Session, ddlJobID int64, multiSchemaChangeID int) error // List will start a transaction of given session and read all schema changes @@ -87,17 +88,28 @@ func (t *tableStore) UpdateProcessed( se *sess.Session, ddlJobID int64, multiSchemaChangeID int64, - processedBy uint64, + oldProcessedBy uint64, + newProcessedBy uint64, ) error { sql := fmt.Sprintf(` UPDATE %s.%s SET processed_by_flag = %d - WHERE ddl_job_id = %d AND sub_job_id = %d`, + WHERE ddl_job_id = %d AND sub_job_id = %d AND processed_by_flag = %d`, t.db, t.table, - processedBy, - ddlJobID, multiSchemaChangeID) + newProcessedBy, + ddlJobID, multiSchemaChangeID, oldProcessedBy, + ) _, err := se.Execute(ctx, sql, "ddl_notifier") - return err + if err != nil { + return errors.Trace(err) + } + if se.GetSessionVars().StmtCtx.AffectedRows() == 0 { + return errors.Errorf( + "failed to update processed_by_flag, maybe the row has been updated by other owner. ddl_job_id: %d, sub_job_id: %d", + ddlJobID, multiSchemaChangeID, + ) + } + return nil } // DeleteAndCommit implements Store interface. diff --git a/pkg/ddl/notifier/subscribe.go b/pkg/ddl/notifier/subscribe.go index a922dc97abc20..5039b19362cc2 100644 --- a/pkg/ddl/notifier/subscribe.go +++ b/pkg/ddl/notifier/subscribe.go @@ -298,17 +298,14 @@ func (n *DDLNotifier) processEventForHandler( zap.Duration("duration", time.Since(now))) } - if err = n.store.UpdateProcessed( + return errors.Trace(n.store.UpdateProcessed( ctx, session, change.ddlJobID, change.subJobID, + change.processedByFlag, newFlag, - ); err != nil { - return errors.Trace(err) - } - - return nil + )) } // Stop stops the background loop. diff --git a/pkg/ddl/notifier/testkit_test.go b/pkg/ddl/notifier/testkit_test.go index 30dd5f22778e9..cd5f03711eab0 100644 --- a/pkg/ddl/notifier/testkit_test.go +++ b/pkg/ddl/notifier/testkit_test.go @@ -318,9 +318,9 @@ func Test2OwnerForAShortTime(t *testing.T) { s := notifier.OpenTableStore("test", ddl.NotifierTableName) sessionPool := util.NewSessionPool( - 1, + 4, func() (pools.Resource, error) { - return tk.Session(), nil + return testkit.NewTestKit(t, store).Session(), nil }, nil, nil, @@ -360,7 +360,7 @@ func Test2OwnerForAShortTime(t *testing.T) { if !bytes.Contains(content, []byte("Error processing change")) { return false } - return bytes.Contains(content, []byte("Write conflict")) + return bytes.Contains(content, []byte("maybe the row has been updated by other owner")) }, time.Second, 25*time.Millisecond) // the handler should not commit tk2.MustQuery("SELECT * FROM test.result").Check(testkit.Rows()) From c600f88ebc536dda693795ac97bf2f6f15a595d6 Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 7 Feb 2025 16:02:37 +0800 Subject: [PATCH 5/7] executor: filter warnings to show only 'Warning' and 'Error' levels in analyze tests Signed-off-by: Rustin170506 --- tests/integrationtest/r/executor/analyze.result | 3 +-- tests/integrationtest/t/executor/analyze.test | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integrationtest/r/executor/analyze.result b/tests/integrationtest/r/executor/analyze.result index 641f7c5525351..65c8cfb1a1b33 100644 --- a/tests/integrationtest/r/executor/analyze.result +++ b/tests/integrationtest/r/executor/analyze.result @@ -739,11 +739,10 @@ create index idxc on t (c); analyze table t partition p0 index idxa; analyze table t partition p1 index idxb; analyze table t partition p2 index idxc; -show warnings; +SHOW WARNINGS WHERE Level IN ('Warning', 'Error'); Level Code Message Warning 1105 No predicate column has been collected yet for table executor__analyze.t, so only indexes and the columns composing the indexes will be analyzed Warning 1105 The version 2 would collect all statistics not only the selected indexes -Note 1105 Analyze use auto adjusted sample rate 1.000000 for table executor__analyze.t's partition p2, reason to use this rate is "use min(1, 110000/10000) as the sample-rate=1" analyze table t partition p0; show warnings; Level Code Message diff --git a/tests/integrationtest/t/executor/analyze.test b/tests/integrationtest/t/executor/analyze.test index f4b0aa789ee56..c6022e50a513c 100644 --- a/tests/integrationtest/t/executor/analyze.test +++ b/tests/integrationtest/t/executor/analyze.test @@ -808,7 +808,7 @@ create index idxc on t (c); analyze table t partition p0 index idxa; analyze table t partition p1 index idxb; analyze table t partition p2 index idxc; -show warnings; +SHOW WARNINGS WHERE Level IN ('Warning', 'Error'); analyze table t partition p0; show warnings; set tidb_partition_prune_mode=default; From c7a8985e19aead514bfc9f0e8d508112491dd46b Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 7 Feb 2025 16:09:43 +0800 Subject: [PATCH 6/7] executor: filter warnings to show only 'Warning' and 'Error' levels in analyze tests Signed-off-by: Rustin170506 --- tests/integrationtest/r/executor/analyze.result | 3 +-- tests/integrationtest/t/executor/analyze.test | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integrationtest/r/executor/analyze.result b/tests/integrationtest/r/executor/analyze.result index 65c8cfb1a1b33..60167cfbb5801 100644 --- a/tests/integrationtest/r/executor/analyze.result +++ b/tests/integrationtest/r/executor/analyze.result @@ -744,10 +744,9 @@ Level Code Message Warning 1105 No predicate column has been collected yet for table executor__analyze.t, so only indexes and the columns composing the indexes will be analyzed Warning 1105 The version 2 would collect all statistics not only the selected indexes analyze table t partition p0; -show warnings; +SHOW WARNINGS WHERE Level IN ('Warning', 'Error'); Level Code Message Warning 1105 No predicate column has been collected yet for table executor__analyze.t, so only indexes and the columns composing the indexes will be analyzed -Note 1105 Analyze use auto adjusted sample rate 1.000000 for table executor__analyze.t's partition p0, reason to use this rate is "use min(1, 110000/2) as the sample-rate=1" set tidb_partition_prune_mode=default; set @@session.tidb_enable_fast_analyze=1; show warnings; diff --git a/tests/integrationtest/t/executor/analyze.test b/tests/integrationtest/t/executor/analyze.test index c6022e50a513c..0673ba9d07edd 100644 --- a/tests/integrationtest/t/executor/analyze.test +++ b/tests/integrationtest/t/executor/analyze.test @@ -810,7 +810,7 @@ analyze table t partition p1 index idxb; analyze table t partition p2 index idxc; SHOW WARNINGS WHERE Level IN ('Warning', 'Error'); analyze table t partition p0; -show warnings; +SHOW WARNINGS WHERE Level IN ('Warning', 'Error'); set tidb_partition_prune_mode=default; From e2cd37f60c2845bc289beadb58218dfe52cfd256 Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 7 Feb 2025 16:45:35 +0800 Subject: [PATCH 7/7] executor: update analyze table command to include all columns Signed-off-by: Rustin170506 --- .../integrationtest/r/executor/issues.result | 32 +++++++++---------- tests/integrationtest/t/executor/issues.test | 4 +-- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/integrationtest/r/executor/issues.result b/tests/integrationtest/r/executor/issues.result index 601dd844ae771..679f1e3f42fe8 100644 --- a/tests/integrationtest/r/executor/issues.result +++ b/tests/integrationtest/r/executor/issues.result @@ -897,7 +897,7 @@ insert into pt (val) select (val) from pt; split table pt between (0) and (40960) regions 30; TOTAL_SPLIT_REGION SCATTER_FINISH_RATIO 203 1 -analyze table pt; +analyze table pt all columns; set @@tidb_distsql_scan_concurrency = default; explain analyze select * from t order by id; id estRows actRows task access object execution info operator info memory disk @@ -969,18 +969,18 @@ Limit_7 256.00 root NULL NULL offset:0, count:100000 └─TableFullScan_9 256.00 cop[tikv] table:pt NULL keep order:false explain analyze select * from pt where val = 125 limit 100; id estRows actRows task access object execution info operator info memory disk -Limit_8 100.00 root NULL NULL offset:0, count:100 -└─TableReader_13 100.00 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_12 100.00 cop[tikv] NULL NULL offset:0, count:100 - └─Selection_11 100.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) - └─TableFullScan_10 125.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:missing] +Limit_8 1.00 root NULL NULL offset:0, count:100 +└─TableReader_13 1.00 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_12 1.00 cop[tikv] NULL NULL offset:0, count:100 + └─Selection_11 1.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) + └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false explain analyze select * from pt where val = 125 limit 100000; id estRows actRows task access object execution info operator info memory disk -Limit_8 204.80 root NULL NULL offset:0, count:100000 -└─TableReader_13 204.80 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_12 204.80 cop[tikv] NULL NULL offset:0, count:100000 - └─Selection_11 204.80 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) - └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:missing] +Limit_8 1.00 root NULL NULL offset:0, count:100000 +└─TableReader_13 1.00 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_12 1.00 cop[tikv] NULL NULL offset:0, count:100000 + └─Selection_11 1.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) + └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false explain analyze select * from pt order by id limit 100; id estRows actRows task access object execution info operator info memory disk Limit_10 100.00 root NULL NULL offset:0, count:100 @@ -995,11 +995,11 @@ Limit_11 256.00 root NULL NULL offset:0, count:100000 └─TableFullScan_19 256.00 cop[tikv] table:pt NULL keep order:true explain analyze select * from pt where val = 126 order by id limit 100; id estRows actRows task access object execution info operator info memory disk -Limit_11 100.00 root NULL NULL offset:0, count:100 -└─TableReader_20 100.00 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_19 100.00 cop[tikv] NULL NULL offset:0, count:100 - └─Selection_18 100.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 126) - └─TableFullScan_17 125.00 cop[tikv] table:pt NULL keep order:true, stats:partial[val:missing] +Limit_11 1.00 root NULL NULL offset:0, count:100 +└─TableReader_20 1.00 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_19 1.00 cop[tikv] NULL NULL offset:0, count:100 + └─Selection_18 1.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 126) + └─TableFullScan_17 256.00 cop[tikv] table:pt NULL keep order:true CREATE TABLE test_55837 (col1 int(4) NOT NULL, col2 bigint(4) NOT NULL, KEY col2_index (col2)); insert into test_55837 values(0,1725292800),(0,1725292800); select from_unixtime( if(col2 >9999999999, col2/1000, col2), '%Y-%m-%d %H:%i:%s') as result from test_55837; diff --git a/tests/integrationtest/t/executor/issues.test b/tests/integrationtest/t/executor/issues.test index 87d8cf0a424dc..bffb90b0dc01d 100644 --- a/tests/integrationtest/t/executor/issues.test +++ b/tests/integrationtest/t/executor/issues.test @@ -695,7 +695,7 @@ insert into pt (val) select (val) from pt; insert into pt (val) select (val) from pt; insert into pt (val) select (val) from pt; split table pt between (0) and (40960) regions 30; -analyze table pt; +analyze table pt all columns; set @@tidb_distsql_scan_concurrency = default; -- replace_column 8 9 3 @@ -769,4 +769,4 @@ insert into test_55837 values(0,1725292800),(0,1725292800); select from_unixtime( if(col2 >9999999999, col2/1000, col2), '%Y-%m-%d %H:%i:%s') as result from test_55837; # TestIssue56641 -select ((exists (select 1)) * -5) as c1; \ No newline at end of file +select ((exists (select 1)) * -5) as c1;