diff --git a/Makefile b/Makefile index 599da2835012d..442bbda9a4de2 100644 --- a/Makefile +++ b/Makefile @@ -305,7 +305,7 @@ tools/bin/failpoint-ctl: go.mod $(GO) build -o $@ github.com/pingcap/failpoint/failpoint-ctl tools/bin/golangci-lint: - curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s -- -b ./tools/bin v1.21.0 + curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s -- -b ./tools/bin v1.29.0 # Usage: # diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 54781373a63f8..cf4bfffe9c667 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2207,7 +2207,7 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(a int) auto_id_cache 100") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + _, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tk.MustExec("insert into t values()") @@ -2223,7 +2223,7 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(a int null, b int auto_increment unique) auto_id_cache 100") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + _, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tk.MustExec("insert into t(b) values(NULL)") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ab1c8726d5d6f..e0f3fc6a7a46f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3468,6 +3468,9 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, ) + if err != nil { + return nil, errors.Trace(err) + } } if err = setCharsetCollationFlenDecimal(&newCol.FieldType, chs, coll); err != nil { @@ -4013,6 +4016,9 @@ func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCh ast.CharsetOpt{Chs: origCharset, Col: origCollate}, ast.CharsetOpt{Chs: dbInfo.Charset, Col: dbInfo.Collate}, ) + if err != nil { + return doNothing, err + } if err = checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origCollate, false); err != nil { return doNothing, err diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index 5a48912aa3280..9df93c7f1628b 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -116,16 +116,19 @@ func TestSyncerSimple(t *testing.T) { wg := sync.WaitGroup{} wg.Add(1) currentVer := int64(123) + var checkErr string go func() { defer wg.Done() select { case resp := <-d.SchemaSyncer().GlobalVersionCh(): if len(resp.Events) < 1 { - t.Fatalf("get chan events count less than 1") + checkErr = "get chan events count less than 1" + return } checkRespKV(t, 1, DDLGlobalSchemaVersion, fmt.Sprintf("%v", currentVer), resp.Events[0].Kv) case <-time.After(3 * time.Second): - t.Fatalf("get udpate version failed") + checkErr = "get udpate version failed" + return } }() @@ -137,6 +140,10 @@ func TestSyncerSimple(t *testing.T) { wg.Wait() + if checkErr != "" { + t.Fatalf(checkErr) + } + // for CheckAllVersions childCtx, cancel := goctx.WithTimeout(ctx, 200*time.Millisecond) err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer) diff --git a/executor/insert_common.go b/executor/insert_common.go index 7bf45c8668b75..5bd299102b9d5 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -258,7 +258,6 @@ func insertRows(ctx context.Context, base insertCommon) (err error) { if err != nil { return err } - rows = rows[:0] memTracker.Consume(-memUsageOfRows) return nil } diff --git a/executor/set_config.go b/executor/set_config.go index 53910b51e995a..05064b614d7a6 100644 --- a/executor/set_config.go +++ b/executor/set_config.go @@ -201,7 +201,7 @@ func ConvertConfigItem2JSON(ctx sessionctx.Context, key string, val expression.E str = fmt.Sprintf("%v", f) } case types.ETDecimal: - d := new(types.MyDecimal) + var d *types.MyDecimal d, isNull, err = val.EvalDecimal(ctx, chunk.Row{}) if err == nil && !isNull { str = string(d.ToString()) diff --git a/expression/aggregation/agg_to_pb.go b/expression/aggregation/agg_to_pb.go index 98bb1feded596..08c8ce7f069f9 100644 --- a/expression/aggregation/agg_to_pb.go +++ b/expression/aggregation/agg_to_pb.go @@ -26,9 +26,9 @@ import ( // AggFuncToPBExpr converts aggregate function to pb. func AggFuncToPBExpr(sc *stmtctx.StatementContext, client kv.Client, aggFunc *AggFuncDesc) *tipb.Expr { - if aggFunc.HasDistinct { - // do nothing and ignore aggFunc.HasDistinct - } + // if aggFunc.HasDistinct { + // do nothing and ignore aggFunc.HasDistinct + // } if len(aggFunc.OrderByItems) > 0 { return nil } diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index f29741c6d4261..e1f964bcf8f94 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -420,12 +420,12 @@ func (s *builtinArithmeticMinusIntSig) evalInt(row chunk.Row) (val int64, isNull isRHSUnsigned := !forceToSigned && mysql.HasUnsignedFlag(s.args[1].GetType().Flag) if forceToSigned && mysql.HasUnsignedFlag(s.args[0].GetType().Flag) { - if a < 0 || (a > math.MaxInt64) { + if a < 0 { return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String())) } } if forceToSigned && mysql.HasUnsignedFlag(s.args[1].GetType().Flag) { - if b < 0 || (b > math.MaxInt64) { + if b < 0 { return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String())) } } diff --git a/expression/builtin_arithmetic_vec.go b/expression/builtin_arithmetic_vec.go index dc1e727160db7..c713265af292d 100644 --- a/expression/builtin_arithmetic_vec.go +++ b/expression/builtin_arithmetic_vec.go @@ -376,11 +376,11 @@ func (b *builtinArithmeticMinusIntSig) minusFUU(result *chunk.Column, lhi64s, rh } lh, rh := lhi64s[i], rhi64s[i] - if lh < 0 || (lh > math.MaxInt64) { + if lh < 0 { return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) } - if rh < 0 || (rh > math.MaxInt64) { + if rh < 0 { return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) } @@ -400,7 +400,7 @@ func (b *builtinArithmeticMinusIntSig) minusFUS(result *chunk.Column, lhi64s, rh } lh, rh := lhi64s[i], rhi64s[i] - if lh < 0 || (lh > math.MaxInt64) { + if lh < 0 { return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) } @@ -420,7 +420,7 @@ func (b *builtinArithmeticMinusIntSig) minusFSU(result *chunk.Column, lhi64s, rh } lh, rh := lhi64s[i], rhi64s[i] - if rh < 0 || (rh > math.MaxInt64) { + if rh < 0 { return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) } diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 0b89d349b0cab..4c05906bb8b51 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -2101,14 +2101,14 @@ func (b *builtinNullEQIntSig) evalInt(row chunk.Row) (val int64, isNull bool, er case !isUnsigned0 && !isUnsigned1 && types.CompareInt64(arg0, arg1) == 0: res = 1 case isUnsigned0 && !isUnsigned1: - if arg1 < 0 || arg0 > math.MaxInt64 { + if arg1 < 0 { break } if types.CompareInt64(arg0, arg1) == 0 { res = 1 } case !isUnsigned0 && isUnsigned1: - if arg0 < 0 || arg1 > math.MaxInt64 { + if arg0 < 0 { break } if types.CompareInt64(arg0, arg1) == 0 { diff --git a/expression/builtin_vectorized_test.go b/expression/builtin_vectorized_test.go index 4a9d5d10b8cfc..ec3f4a3e7ed8f 100644 --- a/expression/builtin_vectorized_test.go +++ b/expression/builtin_vectorized_test.go @@ -784,8 +784,6 @@ func (s *testEvaluatorSuite) TestFloat32ColVec(c *C) { i++ } - // set an empty Sel - sel = sel[:0] c.Assert(col.VecEvalReal(ctx, chk, result), IsNil) } diff --git a/expression/chunk_executor.go b/expression/chunk_executor.go index 53772eb2aceb3..f962deb90329a 100644 --- a/expression/chunk_executor.go +++ b/expression/chunk_executor.go @@ -137,10 +137,10 @@ func evalOneVec(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, out } // TODO: recycle all old Columns returned here. output.SetCol(colIdx, buf) - } else if mysql.HasUnsignedFlag(ft.Flag) { - // the underlying memory formats of int64 and uint64 are the same in Golang, - // so we can do a no-op here. - } + } // else if mysql.HasUnsignedFlag(ft.Flag) { + // the underlying memory formats of int64 and uint64 are the same in Golang, + // so we can do a no-op here. + // } case types.ETReal: if err := expr.VecEvalReal(ctx, input, result); err != nil { return err diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index ed5080cd21a8b..6dece6b36f9a7 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -854,9 +854,9 @@ func (e *InspectionSummaryTableExtractor) Extract( predicates []expression.Expression, ) (remained []expression.Expression) { // Extract the `rule` columns - remained, ruleSkip, rules := e.extractCol(schema, names, predicates, "rule", true) + _, ruleSkip, rules := e.extractCol(schema, names, predicates, "rule", true) // Extract the `metric_name` columns - remained, metricNameSkip, metricNames := e.extractCol(schema, names, predicates, "metrics_name", true) + _, metricNameSkip, metricNames := e.extractCol(schema, names, predicates, "metrics_name", true) // Extract the `quantile` columns remained, quantileSkip, quantileSet := e.extractCol(schema, names, predicates, "quantile", false) e.SkipInspection = ruleSkip || quantileSkip || metricNameSkip diff --git a/session/schema_amender.go b/session/schema_amender.go index 56fabc9fe649d..0641367b1c415 100644 --- a/session/schema_amender.go +++ b/session/schema_amender.go @@ -475,9 +475,7 @@ func (s *SchemaAmender) getAmendableKeys(commitMutations tikv.CommitterMutations addKeys = append(addKeys, byteKey) } else if pb.Op_Del == keyOp { removeKeys = append(removeKeys, byteKey) - } else { - // Do nothing. - } + } // else Do nothing. } return addKeys, removeKeys } diff --git a/session/session_test.go b/session/session_test.go index fa11977ead911..35a443a33bdc4 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -542,9 +542,9 @@ func testTxnLazyInitialize(s *testSessionSuite, c *C, isPessimistic bool) { } tk.MustExec("set @@autocommit = 0") - txn, err := tk.Se.Txn(true) + _, err := tk.Se.Txn(true) c.Assert(kv.ErrInvalidTxn.Equal(err), IsTrue) - txn, err = tk.Se.Txn(false) + txn, err := tk.Se.Txn(false) c.Assert(err, IsNil) c.Assert(txn.Valid(), IsFalse) tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 981441236f81d..b9fb958e7ad11 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -587,7 +587,7 @@ func (ts *testSuite) TestAddRecordWithCtx(c *C) { defer ts.se.Execute(context.Background(), "DROP TABLE test.tRecord") c.Assert(ts.se.NewTxn(context.Background()), IsNil) - txn, err := ts.se.Txn(true) + _, err = ts.se.Txn(true) c.Assert(err, IsNil) recordCtx := tables.NewCommonAddRecordCtx(len(tb.Cols())) tables.SetAddRecordCtx(ts.se, recordCtx) @@ -612,7 +612,7 @@ func (ts *testSuite) TestAddRecordWithCtx(c *C) { c.Assert(i, Equals, len(records)) ts.se.StmtCommit() - txn, err = ts.se.Txn(true) + txn, err := ts.se.Txn(true) c.Assert(err, IsNil) c.Assert(txn.Commit(context.Background()), IsNil) } diff --git a/util/plancodec/id.go b/util/plancodec/id.go index a21b0286cfc02..219f2cfabedd0 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -110,45 +110,45 @@ const ( // Attention: for compatibility of encode/decode plan, The plan id shouldn't be changed. const ( typeSelID int = 1 - typeSetID = 2 - typeProjID = 3 - typeAggID = 4 - typeStreamAggID = 5 - typeHashAggID = 6 - typeShowID = 7 - typeJoinID = 8 - typeUnionID = 9 - typeTableScanID = 10 - typeMemTableScanID = 11 - typeUnionScanID = 12 - typeIdxScanID = 13 - typeSortID = 14 - typeTopNID = 15 - typeLimitID = 16 - typeHashJoinID = 17 - typeMergeJoinID = 18 - typeIndexJoinID = 19 - typeIndexMergeJoinID = 20 - typeIndexHashJoinID = 21 - typeApplyID = 22 - typeMaxOneRowID = 23 - typeExistsID = 24 - typeDualID = 25 - typeLockID = 26 - typeInsertID = 27 - typeUpdateID = 28 - typeDeleteID = 29 - typeIndexLookUpID = 30 - typeTableReaderID = 31 - typeIndexReaderID = 32 - typeWindowID = 33 - typeTiKVSingleGatherID = 34 - typeIndexMergeID = 35 - typePointGet = 36 - typeShowDDLJobs = 37 - typeBatchPointGet = 38 - typeClusterMemTableReader = 39 - typeDataSourceID = 40 + typeSetID int = 2 + typeProjID int = 3 + typeAggID int = 4 + typeStreamAggID int = 5 + typeHashAggID int = 6 + typeShowID int = 7 + typeJoinID int = 8 + typeUnionID int = 9 + typeTableScanID int = 10 + typeMemTableScanID int = 11 + typeUnionScanID int = 12 + typeIdxScanID int = 13 + typeSortID int = 14 + typeTopNID int = 15 + typeLimitID int = 16 + typeHashJoinID int = 17 + typeMergeJoinID int = 18 + typeIndexJoinID int = 19 + typeIndexMergeJoinID int = 20 + typeIndexHashJoinID int = 21 + typeApplyID int = 22 + typeMaxOneRowID int = 23 + typeExistsID int = 24 + typeDualID int = 25 + typeLockID int = 26 + typeInsertID int = 27 + typeUpdateID int = 28 + typeDeleteID int = 29 + typeIndexLookUpID int = 30 + typeTableReaderID int = 31 + typeIndexReaderID int = 32 + typeWindowID int = 33 + typeTiKVSingleGatherID int = 34 + typeIndexMergeID int = 35 + typePointGet int = 36 + typeShowDDLJobs int = 37 + typeBatchPointGet int = 38 + typeClusterMemTableReader int = 39 + typeDataSourceID int = 40 ) // TypeStringToPhysicalID converts the plan type string to plan id. diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index 9c691f15315d2..12781b29b66fc 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -366,7 +366,7 @@ func (t *TestData) ConvertRowsToStrings(rows [][]interface{}) (rs []string) { // ConvertSQLWarnToStrings converts []SQLWarn to []string. func (t *TestData) ConvertSQLWarnToStrings(warns []stmtctx.SQLWarn) (rs []string) { for _, warn := range warns { - rs = append(rs, fmt.Sprintf(warn.Err.Error())) + rs = append(rs, fmt.Sprint(warn.Err.Error())) } return rs }