From f1fbca8372a593f6a001f2c724adca38d2923e28 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 21 Nov 2022 23:15:57 +0800 Subject: [PATCH 1/6] chunk: fix nil pointer when SortedRowContainer meet null type (#39254) close pingcap/tidb#39252 --- executor/aggregate_test.go | 8 ++++++++ util/chunk/row_container.go | 18 ++++++++++-------- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 1868802d65d8d..bd6366d6d115d 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1088,6 +1088,14 @@ func TestIssue10608(t *testing.T) { tk.MustExec("insert into t values(508931), (508932)") tk.MustQuery("select (select /*+ stream_agg() */ group_concat(concat(123,'-')) from t where t.a = s.b group by t.a) as t from s;").Check(testkit.Rows("123-", "123-")) tk.MustQuery("select (select /*+ hash_agg() */ group_concat(concat(123,'-')) from t where t.a = s.b group by t.a) as t from s;").Check(testkit.Rows("123-", "123-")) + + tk.MustExec("CREATE TABLE `t49`(`c0` char(1) DEFAULT '1', `c2` char(1) DEFAULT NULL, UNIQUE KEY `c2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("INSERT INTO `t49` VALUES ('0','0'),('0','1');") + tk.MustExec("CREATE TABLE `t0` (`c0` blob DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("INSERT INTO `t0` VALUES (_binary ']'),(_binary '777926278'),(_binary '0.2136404982804636'),(_binary '1901362489'),(_binary '1558203848'),(''),(_binary '1830406335'),(''),(_binary '0'),(NULL),(_binary '601930250'),(_binary '1558203848'),(_binary '-122008948'),(_binary '-2053608489'),(_binary 'hb/vt <7'),(_binary 'RC&2*'),(_binary '1'),(_binary '-1722334316'),(_binary '1830406335'),(_binary '1372126029'),(_binary '882291196'),(NULL),(_binary '-399693596');") + tk.MustExec("CREATE ALGORITHM=TEMPTABLE DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `v0` (`c0`, `c1`, `c2`) AS SELECT NULL AS `NULL`,`t49`.`c2` AS `c2`,(((CASE _UTF8MB4'I되EkfIO퀶' WHEN NULL THEN `t49`.`c0` WHEN `t49`.`c2` THEN `t0`.`c0` ELSE (CASE `t49`.`c0` WHEN _UTF8MB4'%' THEN 1035293362 ELSE _UTF8MB4',' END) END))<<(`t49`.`c0`)) AS `(((CASE 'I되EkfIO퀶' WHEN NULL THEN t49.c0 WHEN t49.c2 THEN t0.c0 ELSE (CASE t49.c0 WHEN '%' THEN 1035293362 ELSE ',' END ) END ))<<(t49.c0))` FROM (`t0`) JOIN `t49` WHERE TRUE;") + tk.MustQuery("SELECT /*+ STREAM_AGG()*/v0.c0 FROM t49, v0 LEFT OUTER JOIN t0 ON ('Iw') GROUP BY true;"). + Check(testkit.Rows("")) } func TestIssue12759HashAggCalledByApply(t *testing.T) { diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index d40a9846c47ab..7ae1a67879b03 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -472,14 +472,16 @@ func (c *SortedRowContainer) Close() error { func (c *SortedRowContainer) lessRow(rowI, rowJ Row) bool { for i, colIdx := range c.keyColumns { cmpFunc := c.keyCmpFuncs[i] - cmp := cmpFunc(rowI, colIdx, rowJ, colIdx) - if c.ByItemsDesc[i] { - cmp = -cmp - } - if cmp < 0 { - return true - } else if cmp > 0 { - return false + if cmpFunc != nil { + cmp := cmpFunc(rowI, colIdx, rowJ, colIdx) + if c.ByItemsDesc[i] { + cmp = -cmp + } + if cmp < 0 { + return true + } else if cmp > 0 { + return false + } } } return false From c59377b4d4e0b050a4375310f90fd45f40b3d065 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Mon, 21 Nov 2022 23:31:57 +0800 Subject: [PATCH 2/6] extension: fix unstable test `TestExtensionConnEvent` (#39294) close pingcap/tidb#39281 --- server/tidb_test.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index fc176613768ba..1b87b8a976cb8 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -2781,11 +2781,11 @@ func (l *connEventLogs) check(fn func()) { fn() } -func (l *connEventLogs) waitConnDisconnected() error { +func (l *connEventLogs) waitEvent(tp extension.ConnEventTp) error { totalSleep := 0 for { l.Lock() - if l.types[len(l.types)-1] == extension.ConnDisconnected { + if l.types[len(l.types)-1] == tp { l.Unlock() return nil } @@ -2812,6 +2812,8 @@ func TestExtensionConnEvent(t *testing.T) { require.NoError(t, extension.Setup()) ts := createTidbTestSuite(t) + // createTidbTestSuite create an inner connection, so wait the previous connection closed + require.NoError(t, logs.waitEvent(extension.ConnDisconnected)) // test for login success logs.reset() @@ -2828,6 +2830,7 @@ func TestExtensionConnEvent(t *testing.T) { }() var expectedConn2 variable.ConnectionInfo + require.NoError(t, logs.waitEvent(extension.ConnHandshakeAccepted)) logs.check(func() { require.Equal(t, []extension.ConnEventTp{ extension.ConnConnected, @@ -2861,7 +2864,7 @@ func TestExtensionConnEvent(t *testing.T) { require.NoError(t, conn.Close()) require.NoError(t, db.Close()) - require.NoError(t, logs.waitConnDisconnected()) + require.NoError(t, logs.waitEvent(extension.ConnDisconnected)) logs.check(func() { require.Equal(t, 3, len(logs.infos)) require.Equal(t, 1, len(logs.infos[2].ActiveRoles)) @@ -2889,6 +2892,7 @@ func TestExtensionConnEvent(t *testing.T) { _, err = db.Conn(context.Background()) require.Error(t, err) + require.NoError(t, logs.waitEvent(extension.ConnDisconnected)) logs.check(func() { require.Equal(t, []extension.ConnEventTp{ extension.ConnConnected, From d15ffca07177ec0695a75a11e6f2649826f6c6f4 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Tue, 22 Nov 2022 00:29:58 +0800 Subject: [PATCH 3/6] server: fix `ComStmtPrepare` end with `\0` (#39295) close pingcap/tidb#39132 --- server/conn.go | 5 +++++ server/conn_test.go | 27 +++++++++++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/server/conn.go b/server/conn.go index b319679fdbf14..f5ae429b8f18b 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1391,6 +1391,11 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { return cc.handleChangeUser(ctx, data) // ComBinlogDump, ComTableDump, ComConnectOut, ComRegisterSlave case mysql.ComStmtPrepare: + // For issue 39132, same as ComQuery + if len(data) > 0 && data[len(data)-1] == 0 { + data = data[:len(data)-1] + dataStr = string(hack.String(data)) + } return cc.handleStmtPrepare(ctx, dataStr) case mysql.ComStmtExecute: return cc.handleStmtExecute(ctx, data) diff --git a/server/conn_test.go b/server/conn_test.go index 84db98f9e1827..fcfdc2c9b5a42 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -584,6 +584,33 @@ func TestDispatchClientProtocol41(t *testing.T) { testDispatch(t, inputs, mysql.ClientProtocol41) } +func TestQueryEndWithZero(t *testing.T) { + inputs := []dispatchInput{ + { + com: mysql.ComStmtPrepare, + in: append([]byte("select 1"), 0x0), + err: nil, + out: []byte{ + 0xc, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x18, + 0x0, 0x0, 0x1, 0x3, 0x64, 0x65, 0x66, 0x0, 0x0, 0x0, 0x1, 0x31, 0x1, 0x31, 0xc, 0x3f, + 0x0, 0x1, 0x0, 0x0, 0x0, 0x8, 0x81, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x2, 0xfe, + }, + }, + { + com: mysql.ComQuery, + in: append([]byte("select 1"), 0x0), + err: nil, + out: []byte{ + 0x1, 0x0, 0x0, 0x3, 0x1, 0x18, 0x0, 0x0, 0x4, 0x3, 0x64, 0x65, 0x66, 0x0, 0x0, 0x0, + 0x1, 0x31, 0x1, 0x31, 0xc, 0x3f, 0x0, 0x1, 0x0, 0x0, 0x0, 0x8, 0x81, 0x0, 0x0, 0x0, + 0x0, 0x1, 0x0, 0x0, 0x5, 0xfe, 0x2, 0x0, 0x0, 0x6, 0x1, 0x31, 0x1, 0x0, 0x0, 0x7, 0xfe, + }, + }, + } + + testDispatch(t, inputs, 0) +} + func testDispatch(t *testing.T, inputs []dispatchInput, capability uint32) { store := testkit.CreateMockStore(t) From 8b39e6647d8bce1b5b6b906eafd11ca678a4a1e3 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 22 Nov 2022 08:31:57 +0800 Subject: [PATCH 4/6] ddl: refine sql to get ddl jobs (#39245) close pingcap/tidb#39244 --- .gitignore | 1 + ddl/db_test.go | 25 +++++++++++++++++++++++++ ddl/job_table.go | 2 +- 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index e2cdcd078a0f1..35af372bcccad 100644 --- a/.gitignore +++ b/.gitignore @@ -35,3 +35,4 @@ bazel-out bazel-testlogs bazel-tidb .ijwb/ +/oom_record/ diff --git a/ddl/db_test.go b/ddl/db_test.go index 207aad4e0f116..4a739520dfb81 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1736,3 +1736,28 @@ func TestTiDBDownBeforeUpdateGlobalVersion(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/mockDownBeforeUpdateGlobalVersion")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/checkDownBeforeUpdateGlobalVersion")) } + +func TestDDLBlockedCreateView(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int)") + + hook := &ddl.TestDDLCallback{Do: dom} + first := true + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.SchemaState != model.StateWriteOnly { + return + } + if !first { + return + } + first = false + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("create view v as select * from t") + } + dom.DDL().SetHook(hook) + tk.MustExec("alter table t modify column a char(10)") +} diff --git a/ddl/job_table.go b/ddl/job_table.go index 62bba65cf6678..ef006727fceb8 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -67,7 +67,7 @@ func (dc *ddlCtx) excludeJobIDs() string { } const ( - getJobSQL = "select job_meta, processing from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids, processing order by processing desc limit 1) and %s reorg %s order by processing desc, job_id" + getJobSQL = "select job_meta, processing from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids, processing) and %s reorg %s order by processing desc, job_id" ) type jobType int From bd97130f8eadbe85e34191a26a1990bf255762b6 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 22 Nov 2022 09:03:57 +0800 Subject: [PATCH 5/6] Revert "planner: enable cost model2 by default" (#39265) --- sessionctx/variable/tidb_vars.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d7c05c0a48e88..3511775de08f1 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -1079,7 +1079,7 @@ const ( DefTiDBForeignKeyChecks = false DefTiDBAnalyzePartitionConcurrency = 1 DefTiDBOptRangeMaxSize = 64 * int64(size.MB) // 64 MB - DefTiDBCostModelVer = 2 + DefTiDBCostModelVer = 1 DefTiDBServerMemoryLimitSessMinSize = 128 << 20 DefTiDBMergePartitionStatsConcurrency = 1 DefTiDBServerMemoryLimitGCTrigger = 0.7 From f894d9b78f68a478c279316959dd93d8b00279f5 Mon Sep 17 00:00:00 2001 From: Yifan Xu <30385241+xuyifangreeneyes@users.noreply.github.com> Date: Tue, 22 Nov 2022 09:33:57 +0800 Subject: [PATCH 6/6] domain, executor: refine error log and error handling for statistics and analyze (#39284) --- domain/domain.go | 4 ++-- executor/analyze_global_stats.go | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index b900cf3eb8d3a..8c63484730107 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1777,7 +1777,7 @@ func (do *Domain) loadStatsWorker() { t := time.Now() err := statsHandle.InitStats(do.InfoSchema()) if err != nil { - logutil.BgLogger().Debug("init stats info failed", zap.Error(err)) + logutil.BgLogger().Error("init stats info failed", zap.Duration("take time", time.Since(t)), zap.Error(err)) } else { logutil.BgLogger().Info("init stats info time", zap.Duration("take time", time.Since(t))) } @@ -1863,7 +1863,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) case t := <-statsHandle.DDLEventCh(): err := statsHandle.HandleDDLEvent(t) if err != nil { - logutil.BgLogger().Debug("handle ddl event failed", zap.Error(err)) + logutil.BgLogger().Error("handle ddl event failed", zap.String("event", t.String()), zap.Error(err)) } case <-deltaUpdateTicker.C: err := statsHandle.DumpStatsDeltaToKV(handle.DumpDelta) diff --git a/executor/analyze_global_stats.go b/executor/analyze_global_stats.go index 82c2678953285..961d41dea059d 100644 --- a/executor/analyze_global_stats.go +++ b/executor/analyze_global_stats.go @@ -73,10 +73,10 @@ func (e *AnalyzeExec) handleGlobalStats(ctx context.Context, needGlobalStats boo globalStatsID.tableID, info.isIndex, info.histIDs, tableAllPartitionStats) if err != nil { + logutil.BgLogger().Error("merge global stats failed", zap.String("info", job.JobInfo), zap.Error(err)) if types.ErrPartitionStatsMissing.Equal(err) || types.ErrPartitionColumnStatsMissing.Equal(err) { // When we find some partition-level stats are missing, we need to report warning. e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) - return nil } return err } @@ -95,14 +95,14 @@ func (e *AnalyzeExec) handleGlobalStats(ctx context.Context, needGlobalStats boo true, ) if err != nil { - logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.Error(err)) + logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.String("info", job.JobInfo), zap.Int64("histID", hg.ID), zap.Error(err)) } // Dump stats to historical storage. - if err := recordHistoricalStats(e.ctx, globalStatsID.tableID); err != nil { - logutil.BgLogger().Error("record historical stats failed", zap.Error(err)) + if err1 := recordHistoricalStats(e.ctx, globalStatsID.tableID); err1 != nil { + logutil.BgLogger().Error("record historical stats failed", zap.String("info", job.JobInfo), zap.Int64("histID", hg.ID), zap.Error(err1)) } } - return nil + return err }() FinishAnalyzeMergeJob(e.ctx, job, mergeStatsErr) }