diff --git a/Makefile b/Makefile index dd0e5f521acab..d6681083dc284 100644 --- a/Makefile +++ b/Makefile @@ -57,6 +57,7 @@ check-static: tools/bin/golangci-lint --enable=unused \ --enable=structcheck \ --enable=deadcode \ + --enable=gosimple \ $$($(PACKAGE_DIRECTORIES)) check-slow:tools/bin/gometalinter tools/bin/gosec diff --git a/ddl/db_test.go b/ddl/db_test.go index eddad6d0d635d..ecef0d0144215 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2280,10 +2280,8 @@ func (s *testDBSuite6) TestDropColumn(c *C) { testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", []string{"insert into t2 set c1 = 1, c2 = 1, c3 = 1, c4 = 1"}, dmlDone) } for i := 0; i < num; i++ { - select { - case err := <-ddlDone: - c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) - } + err := <-ddlDone + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) } // Test for drop partition table column. @@ -6575,7 +6573,7 @@ func (s *testSerialDBSuite) TestModifyColumnTypeWhenInterception(c *C) { count := defaultBatchSize * 4 // Add some rows. - dml := fmt.Sprintf("insert into t values") + dml := "insert into t values" for i := 1; i <= count; i++ { dml += fmt.Sprintf("(%d, %f)", i, 11.22) if i != count { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e6d77c9e674e9..528a6087638fc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4040,7 +4040,7 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O) return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) } - break // Increasing auto_random shard bits is allowed. + // increasing auto_random shard bits is allowed. case oldRandBits > newRandBits: if newRandBits == 0 { return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index e532bfc2352af..a8d245007f39c 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -503,15 +503,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { // for failure table cases tk.MustExec("use ctwl_db") - failSQL := fmt.Sprintf("create table t1 like test_not_exist.t") + failSQL := "create table t1 like test_not_exist.t" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 like test.t_not_exist") + failSQL = "create table t1 like test.t_not_exist" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 (like test_not_exist.t)") + failSQL = "create table t1 (like test_not_exist.t)" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table test_not_exis.t1 like ctwl_db.t") + failSQL = "create table test_not_exis.t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrBadDB) - failSQL = fmt.Sprintf("create table t1 like ctwl_db.t") + failSQL = "create table t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrTableExists) // test failure for wrong object cases diff --git a/executor/aggfuncs/func_percentile.go b/executor/aggfuncs/func_percentile.go index 31855f791fb0f..ff13392276c61 100644 --- a/executor/aggfuncs/func_percentile.go +++ b/executor/aggfuncs/func_percentile.go @@ -53,9 +53,7 @@ func (e *basePercentile) AllocPartialResult() (pr PartialResult, memDelta int64) return } -func (e *basePercentile) ResetPartialResult(pr PartialResult) { - return -} +func (e *basePercentile) ResetPartialResult(pr PartialResult) {} func (e *basePercentile) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (memDelta int64, err error) { return diff --git a/executor/concurrent_map.go b/executor/concurrent_map.go index 27f13a4f21dcb..3d6ef1082f605 100644 --- a/executor/concurrent_map.go +++ b/executor/concurrent_map.go @@ -56,7 +56,6 @@ func (m concurrentMap) Insert(key uint64, value *entry) { shard.items[key] = value } shard.Unlock() - return } // UpsertCb : Callback to return new element to be inserted into the map diff --git a/executor/delete.go b/executor/delete.go index 1fe9c26b8ac82..16f0e9c421b19 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -188,10 +188,7 @@ func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { var err error rowMap.Range(func(h kv.Handle, val interface{}) bool { err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) - if err != nil { - return false - } - return true + return err == nil }) if err != nil { return err diff --git a/executor/executor_test.go b/executor/executor_test.go index 3b168636606ed..7c3a76a9cc08a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5809,7 +5809,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // Test for recover one table multiple time. tk.MustExec("drop table t_recover") tk.MustExec("flashback table t_recover to t_recover_tmp") - _, err = tk.Exec(fmt.Sprintf("recover table t_recover")) + _, err = tk.Exec("recover table t_recover") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) gcEnable, err := gcutil.CheckGCEnable(tk.Se) @@ -5876,7 +5876,7 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a,_tidb_rowid from t_flashback2;").Check(testkit.Rows("1 1", "2 2", "3 3", "4 5001", "5 5002", "6 5003", "7 10001", "8 10002", "9 10003")) // Test for flashback one table multiple time. - _, err = tk.Exec(fmt.Sprintf("flashback table t_flashback to t_flashback4")) + _, err = tk.Exec("flashback table t_flashback to t_flashback4") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) // Test for flashback truncated table to new table. diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index e29a7a3e24cee..46df545b1ff47 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -181,11 +181,11 @@ func (s *testSuite) TestExplainMemTablePredicate(c *C) { func (s *testSuite) TestExplainClusterTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb","tikv"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG instances:["192.168.1.7:2379"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb"], instances:["192.168.1.7:2379"]`)) } @@ -203,11 +203,11 @@ func (s *testSuite) TestInspectionResultTable(c *C) { func (s *testSuite) TestInspectionRuleTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection","summary"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`)) } @@ -355,12 +355,12 @@ func (s *testPrepareSerialSuite) TestExplainDotForQuery(c *C) { func (s *testSuite) TestExplainTableStorage(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]")) } func (s *testSuite) TestInspectionSummaryTable(c *C) { diff --git a/executor/grant_test.go b/executor/grant_test.go index 13686494feddc..aa02e9cb2480c 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -68,7 +68,7 @@ func (s *testSuite3) TestGrantDBScope(c *C) { createUserSQL := `CREATE USER 'testDB'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testDB\" and host=\"localhost\"") + sql := `"SELECT * FROM mysql.db WHERE User="testDB" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant each priv to the user. @@ -101,7 +101,7 @@ func (s *testSuite3) TestWithGrantOption(c *C) { createUserSQL := `CREATE USER 'testWithGrant'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testWithGrant\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testWithGrant" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant select priv to the user, with grant option. diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ae338bdd644d2..4d56cc55accac 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1856,7 +1856,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s } } e.rows = rows - return } func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error { diff --git a/executor/insert_test.go b/executor/insert_test.go index ffcfdc214bdb9..351b337bd84ae 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -426,7 +426,7 @@ func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (ts timestamp)") tk.MustExec("insert into t values ('2020-10-22T12:00:00Z'), ('2020-10-22T13:00:00Z'), ('2020-10-22T14:00:00Z')") - tk.MustQuery(fmt.Sprintf("select count(*) from t where ts > '2020-10-22T12:00:00Z'")).Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from t where ts > '2020-10-22T12:00:00Z'").Check(testkit.Rows("2")) // test for datetime with fsp fspCases := []struct { diff --git a/executor/join.go b/executor/join.go index c1a8045aba9a3..1a3f62de47ac1 100644 --- a/executor/join.go +++ b/executor/join.go @@ -1078,7 +1078,7 @@ func (e *joinRuntimeStats) String() string { if e.cache.useCache { buf.WriteString(fmt.Sprintf(", cache:ON, cacheHitRatio:%.3f%%", e.cache.hitRatio*100)) } else { - buf.WriteString(fmt.Sprintf(", cache:OFF")) + buf.WriteString(", cache:OFF") } } if e.hasHashStat { diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 24e2001131580..76ea478d91e71 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -230,9 +230,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String continue } var str string - switch val.(type) { + switch val := val.(type) { case string: // remove quotes - str = val.(string) + str = val default: tmp, err := json.Marshal(val) if err != nil { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 94b9c33a5a9d7..fb2c714f8c79d 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -11,6 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned package executor_test import ( @@ -277,7 +278,6 @@ func (s *testSerialSuite1) TestShuffleMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { defer config.RestoreFunc()() @@ -313,7 +313,6 @@ func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSuite2) TestMergeJoin(c *C) { diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index b849d3d961043..c0ecb19783273 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -43,7 +43,6 @@ func checkApplyPlan(c *C, tk *testkit.TestKit, sql string, parallel int) { } } c.Assert(containApply, IsTrue) - return } func (s *testSuite) TestParallelApply(c *C) { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5be39c3a04d54..ab5a19ca823e9 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -263,7 +263,7 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { } // test empty PointGet - queryHash := fmt.Sprintf("select a from thash where a=200") + queryHash := "select a from thash where a=200" c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used tk.MustQuery(queryHash).Check(testkit.Rows()) diff --git a/executor/show.go b/executor/show.go index c5df3bb290268..2bd9b786fcffb 100644 --- a/executor/show.go +++ b/executor/show.go @@ -862,10 +862,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if pkCol != nil { - // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. + // If PKIsHandle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") fmt.Fprintf(buf, " PRIMARY KEY (%s)", stringutil.Escape(pkCol.Name.O, sqlMode)) - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } publicIndices := make([]*model.IndexInfo, 0, len(tableInfo.Indices)) @@ -906,9 +906,9 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if idxInfo.Primary { if tableInfo.PKIsHandle || tableInfo.IsCommonHandle { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } else { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] NONCLUSTERED */")) + buf.WriteString(" /*T![clustered_index] NONCLUSTERED */") } } if i != len(publicIndices)-1 { diff --git a/expression/integration_test.go b/expression/integration_test.go index a3d983069cce9..8e0f70823f327 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5833,7 +5833,7 @@ func (s *testIntegrationSuite) TestDecodetoChunkReuse(c *C) { tk.MustExec("create table chk (a int,b varchar(20))") for i := 0; i < 200; i++ { if i%5 == 0 { - tk.MustExec(fmt.Sprintf("insert chk values (NULL,NULL)")) + tk.MustExec("insert chk values (NULL,NULL)") continue } tk.MustExec(fmt.Sprintf("insert chk values (%d,'%s')", i, strconv.Itoa(i))) diff --git a/infoschema/metrics_schema.go b/infoschema/metrics_schema.go index 49a57e4ac9eeb..3b4654f90f7f2 100644 --- a/infoschema/metrics_schema.go +++ b/infoschema/metrics_schema.go @@ -100,17 +100,9 @@ func (def *MetricTableDef) genColumnInfos() []columnInfo { // GenPromQL generates the promQL. func (def *MetricTableDef) GenPromQL(sctx sessionctx.Context, labels map[string]set.StringSet, quantile float64) string { promQL := def.PromQL - if strings.Contains(promQL, promQLQuantileKey) { - promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) - } - - if strings.Contains(promQL, promQLLabelConditionKey) { - promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) - } - - if strings.Contains(promQL, promQRangeDurationKey) { - promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) - } + promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) + promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) + promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) return promQL } diff --git a/planner/cascades/implementation_rules.go b/planner/cascades/implementation_rules.go index d7a08b4fabaab..56c2141cfb213 100644 --- a/planner/cascades/implementation_rules.go +++ b/planner/cascades/implementation_rules.go @@ -95,10 +95,7 @@ type ImplTableDual struct { // Match implements ImplementationRule Match interface. func (r *ImplTableDual) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. @@ -116,10 +113,7 @@ type ImplMemTableScan struct { // Match implements ImplementationRule Match interface. func (r *ImplMemTableScan) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 6d23e063f5877..9961509299a52 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -1503,10 +1503,7 @@ func NewRuleMergeAggregationProjection() Transformation { // Match implements Transformation interface. func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(proj.Exprs) { - return false - } - return true + return !plannercore.ExprsHasSideEffects(proj.Exprs) } // OnTransform implements Transformation interface. diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index dd614239de55d..e151efdecce03 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -327,7 +327,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk1.MustExec(insert) // Test query without condition - query := fmt.Sprintf("select * from t1 order by id,a,b") + query := "select * from t1 order by id,a,b" tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) } @@ -467,9 +467,9 @@ func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin partition by range(a) ( - partition p0 values less than (1), - partition p1 values less than (2), - partition p2 values less than (3), + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), partition p_max values less than (maxvalue));`) var input []string diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go index 3cfaf6708affe..a453596a0f288 100644 --- a/planner/core/pb_to_plan.go +++ b/planner/core/pb_to_plan.go @@ -255,15 +255,15 @@ func (b *PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { return &PhysicalSimpleWrapper{Inner: simple}, nil } -func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { - if p == nil { - return predicates, p +func (b *PBPlanBuilder) predicatePushDown(physicalPlan PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { + if physicalPlan == nil { + return predicates, physicalPlan } - switch p.(type) { + switch plan := physicalPlan.(type) { case *PhysicalMemTable: - memTable := p.(*PhysicalMemTable) + memTable := plan if memTable.Extractor == nil { - return predicates, p + return predicates, plan } names := make([]*types.FieldName, 0, len(memTable.Columns)) for _, col := range memTable.Columns { @@ -284,8 +284,8 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) return predicates, memTable case *PhysicalSelection: - selection := p.(*PhysicalSelection) - conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) + selection := plan + conditions, child := b.predicatePushDown(plan.Children()[0], selection.Conditions) if len(conditions) > 0 { selection.Conditions = conditions selection.SetChildren(child) @@ -293,10 +293,10 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio } return predicates, child default: - if children := p.Children(); len(children) > 0 { + if children := plan.Children(); len(children) > 0 { _, child := b.predicatePushDown(children[0], nil) - p.SetChildren(child) + plan.SetChildren(child) } - return predicates, p + return predicates, plan } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 43304971b4680..f4e6769ee8d70 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -911,9 +911,7 @@ func (p *PhysicalMergeJoin) Clone() (PhysicalPlan, error) { return nil, err } cloned.basePhysicalJoin = *base - for _, cf := range p.CompareFuncs { - cloned.CompareFuncs = append(cloned.CompareFuncs, cf) - } + cloned.CompareFuncs = append(cloned.CompareFuncs, p.CompareFuncs...) cloned.Desc = p.Desc return cloned, nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57858679b7795..b8f545e0c68c2 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -297,7 +297,7 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. }) continue } - return nil, errors.New(fmt.Sprintf("information of column %v is not found", colExpr.String())) + return nil, fmt.Errorf("information of column %v is not found", colExpr.String()) } return names, nil } @@ -1344,10 +1344,10 @@ func checkTableHintsApplicableForPartition(partitions []model.CIStr, partitionSe func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unknownPartitions []string) { if len(unknownPartitions) == 0 { return + } else { + warning := fmt.Errorf("Unknown partitions (%s) in optimizer hint %s", strings.Join(unknownPartitions, ","), hintName) + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } - ctx.GetSessionVars().StmtCtx.AppendWarning( - errors.New(fmt.Sprintf("Unknown partitions (%s) in optimizer hint %s", - strings.Join(unknownPartitions, ","), hintName))) } func (s *partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) { diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 4c63f6ff244b9..346b5b50e5742 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -270,31 +270,31 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *PhysicalShuffleReceiverStub: str = fmt.Sprintf("PartitionReceiverStub(%s)", x.ExplainInfo()) case *PointGetPlan: - str = fmt.Sprintf("PointGet(") + str = "PointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handle) } case *BatchPointGetPlan: - str = fmt.Sprintf("BatchPointGet(") + str = "BatchPointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handles) } case *PhysicalExchangeReceiver: - str = fmt.Sprintf("Recv(") + str = "Recv(" for _, task := range x.Tasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" case *PhysicalExchangeSender: - str = fmt.Sprintf("Send(") + str = "Send(" for _, task := range x.TargetTasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" default: str = fmt.Sprintf("%T", in) } diff --git a/planner/core/util.go b/planner/core/util.go index 753445f07ee42..19ce0a47673a2 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -291,7 +291,7 @@ func extractStringFromStringSet(set set.StringSet) string { l = append(l, fmt.Sprintf(`"%s"`, k)) } sort.Strings(l) - return fmt.Sprintf("%s", strings.Join(l, ",")) + return strings.Join(l, ",") } func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index bae1b3ff37497..24d0bf04b0309 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -62,19 +62,14 @@ func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugi switch event { case plugin.Log: fmt.Println("---- event: Log") - break case plugin.Error: fmt.Println("---- event: Error") - break case plugin.Result: fmt.Println("---- event: Result") - break case plugin.Status: fmt.Println("---- event: Status") - break default: fmt.Println("---- event: unrecognized") - break } fmt.Printf("---- cmd: %s\n", cmd) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index a7495b987f24a..7cdd7137138bd 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -578,7 +578,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV @@ -621,7 +621,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { c.Assert(err, IsNil) c.Assert(data2, DeepEquals, data) - resp, err = ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1?decode=true")) + resp, err = ts.fetchStatus("/mvcc/key/tidb/test/1?decode=true") c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) var data3 map[string]interface{} @@ -667,7 +667,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1234")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1234") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV diff --git a/server/server_test.go b/server/server_test.go index 20c0c2b508213..a342dab77d79a 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -499,14 +499,14 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { } // Test for record slow log for load data statement. - rows := dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query like 'load data local infile %% into table t_slow;' order by time desc limit 1")) + rows := dbt.mustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) // Test for record statements_summary for load data statement. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %%' limit 1")) + rows = dbt.mustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) // Test log normal statement after executing load date. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1")) + rows = dbt.mustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) }) diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 58313505e1c8e..9a80ede490c0d 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -155,17 +155,15 @@ func WaitBinlogRecover(timeout time.Duration) error { defer ticker.Stop() start := time.Now() for { - select { - case <-ticker.C: - if atomic.LoadInt32(&skippedCommitterCounter) == 0 { - logutil.BgLogger().Warn("[binloginfo] binlog recovered") - return nil - } - if time.Since(start) > timeout { - logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", - zap.Duration("duration", timeout)) - return errors.New("timeout") - } + <-ticker.C + if atomic.LoadInt32(&skippedCommitterCounter) == 0 { + logutil.BgLogger().Warn("[binloginfo] binlog recovered") + return nil + } + if time.Since(start) > timeout { + logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", + zap.Duration("duration", timeout)) + return errors.New("timeout") } } } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index f682a1507a4bf..68791a7a0787f 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -818,14 +818,11 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) { } func checkEmptyTopNs(topNs []*TopN) bool { - totCnt := uint64(0) + count := uint64(0) for _, topN := range topNs { - totCnt += topN.TotalCount() + count += topN.TotalCount() } - if totCnt == 0 { - return true - } - return false + return count == 0 } func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { diff --git a/statistics/feedback.go b/statistics/feedback.go index 89aeab32152b3..5b69163e21b2b 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -107,7 +107,6 @@ func (m *QueryFeedbackMap) Append(q *QueryFeedback) { Tp: q.Tp, } m.append(k, []*QueryFeedback{q}) - return } // MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. @@ -136,7 +135,6 @@ func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { break } } - return } var ( diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a8f51f8924e81..fe3644739c1b8 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -554,7 +554,6 @@ func (sc statsCache) initMemoryUsage() { sum += tb.MemoryUsage() } sc.memUsage = sum - return } // update updates the statistics table cache using copy on write. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 2bf1d93ff4246..6d39e42b076e0 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2377,7 +2377,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { // test NDV checkNDV := func(rows, ndv int) { tk.MustExec("analyze table t") - rs := tk.MustQuery(fmt.Sprintf("select value from mysql.stats_fm_sketch")).Rows() + rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() c.Assert(len(rs), Equals, rows) for i := range rs { fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index b105738098f4b..a1de28e78eeef 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -577,8 +577,8 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { // test if it will be limited by the time range c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='00:00 +0000'")) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='23:59 +0000'")) + tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") + tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 377e439a9392c..5c5c5f3219d8d 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -143,14 +143,10 @@ func (m *mppIterator) run(ctx context.Context) { break } m.mu.Lock() - switch task.State { - case kv.MppTaskReady: + if task.State == kv.MppTaskReady { task.State = kv.MppTaskRunning - m.mu.Unlock() - default: - m.mu.Unlock() - break } + m.mu.Unlock() m.wg.Add(1) bo := newBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index 152362c13d5e3..696a1497b068e 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -124,9 +124,7 @@ func buildClosureExecutorFromExecutorList(dagCtx *dagContext, executors []*tipb. outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes[idx]) } } else { - for _, tp := range originalOutputFieldTypes { - outputFieldTypes = append(outputFieldTypes, tp) - } + outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes...) } if len(executors) == 1 { ce.resultFieldType = outputFieldTypes diff --git a/store/mockstore/unistore/tikv/dbreader/db_reader.go b/store/mockstore/unistore/tikv/dbreader/db_reader.go index 158f295cc1bdf..6e3909ada740f 100644 --- a/store/mockstore/unistore/tikv/dbreader/db_reader.go +++ b/store/mockstore/unistore/tikv/dbreader/db_reader.go @@ -165,7 +165,6 @@ func (r *DBReader) BatchGet(keys [][]byte, startTS uint64, f BatchGetFunc) { } f(key, val, err) } - return } // ErrScanBreak is returned by ScanFunc to break the scan loop. diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index 0273bed5fe6a8..08344d3953567 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -57,10 +57,7 @@ type txnKeyHashPair struct { } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { - if p.registerTime.Add(ttl).Before(nowTime) { - return true - } - return false + return p.registerTime.Add(ttl).Before(nowTime) } // NewDetector creates a new Detector. diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index 1768cc377ec7c..22251e47a9e6b 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -26,7 +26,6 @@ package tikv import ( - "fmt" "testing" "time" @@ -54,7 +53,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(2)) err = detector.Detect(3, 1, 300) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(err.Error(), Equals,"deadlock") c.Assert(detector.totalSize, Equals, uint64(2)) detector.CleanUp(2) list2 := detector.waitForMap[2] diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 74ecdd7ce72b8..c701b895f8ba6 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -477,10 +477,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, if val.(bool) { ctx1, cancel := context.WithCancel(context.Background()) cancel() - select { - case <-ctx1.Done(): - } - + <-ctx1.Done() ctx = ctx1 err = ctx.Err() resp = nil diff --git a/store/tikv/txn.go b/store/tikv/txn.go index a8c0f70f8da8d..bfacec9796129 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -625,7 +625,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { - val, _ := lockCtx.Values[string(key)] + val := lockCtx.Values[string(key)] if len(val.Value) == 0 { valExists = tikv.SetKeyLockedValueNotExists } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index f67cbb36b76da..0ef0b573e1bb2 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -90,7 +90,6 @@ func (c *RowContainer) SpillToDisk() { } } c.m.records.Clear() - return } // Reset resets RowContainer. @@ -447,7 +446,6 @@ func (c *SortedRowContainer) Sort() { func (c *SortedRowContainer) sortAndSpillToDisk() { c.Sort() c.RowContainer.SpillToDisk() - return } // Add appends a chunk into the SortedRowContainer. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index ca045352dbd33..676f0a241d489 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -703,9 +703,7 @@ func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*Concurren e.Lock() defer e.Unlock() e.concurrency = e.concurrency[:0] - for _, info := range infos { - e.concurrency = append(e.concurrency, info) - } + e.concurrency = append(e.concurrency, infos...) } // Clone implements the RuntimeStats interface. diff --git a/util/expensivequery/memory_usage_alarm.go b/util/expensivequery/memory_usage_alarm.go index fbe9b6c5ff438..03d53c12342c9 100644 --- a/util/expensivequery/memory_usage_alarm.go +++ b/util/expensivequery/memory_usage_alarm.go @@ -83,7 +83,6 @@ func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() { } } record.initialized = true - return } // If Performance.ServerMemoryQuota is set, use `ServerMemoryQuota * MemoryUsageAlarmRatio` to check oom risk. diff --git a/util/profile/trackerRecorder.go b/util/profile/trackerRecorder.go index c5e5390a51bff..7c449ffa9113c 100644 --- a/util/profile/trackerRecorder.go +++ b/util/profile/trackerRecorder.go @@ -29,12 +29,10 @@ func HeapProfileForGlobalMemTracker(d time.Duration) { t := time.NewTicker(d) defer t.Stop() for { - select { - case <-t.C: - err := heapProfileForGlobalMemTracker() - if err != nil { - log.Warn("profile memory into tracker failed", zap.Error(err)) - } + <-t.C + err := heapProfileForGlobalMemTracker() + if err != nil { + log.Warn("profile memory into tracker failed", zap.Error(err)) } } } diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 69a78d1de7d43..0efd50ecaf27c 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -260,10 +260,7 @@ func (decoder *ChunkDecoder) tryAppendHandleColumn(colIdx int, col *ColInfo, han } coder := codec.NewDecoder(chk, decoder.loc) _, err := coder.DecodeOne(handle.EncodedCol(i), colIdx, col.Ft) - if err != nil { - return false - } - return true + return err == nil } } return false