Skip to content

Commit

Permalink
simple fixes for the staticcheck linter (#18923)
Browse files Browse the repository at this point in the history
Note that this found 2 bugs where errors were being silently ignored.

These are really simple fixes that didn't require much thought.
There are still more staticcheck warnings that should be fixed.

Co-authored-by: ti-srebot <66930949+ti-srebot@users.noreply.github.com>
  • Loading branch information
gregwebs and ti-srebot authored Aug 6, 2020
1 parent e49b4c0 commit 931ff98
Show file tree
Hide file tree
Showing 18 changed files with 81 additions and 73 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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:
#
Expand Down
4 changes: 2 additions & 2 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()")
Expand All @@ -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)")
Expand Down
6 changes: 6 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down
11 changes: 9 additions & 2 deletions ddl/util/syncer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}()

Expand All @@ -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)
Expand Down
1 change: 0 additions & 1 deletion executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion executor/set_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand Down
6 changes: 3 additions & 3 deletions expression/aggregation/agg_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_arithmetic.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()))
}
}
Expand Down
8 changes: 4 additions & 4 deletions expression/builtin_arithmetic_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()))
}

Expand All @@ -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()))
}

Expand All @@ -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()))
}

Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 0 additions & 2 deletions expression/builtin_vectorized_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down
8 changes: 4 additions & 4 deletions expression/chunk_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions planner/core/memtable_predicate_extractor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 1 addition & 3 deletions session/schema_amender.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
4 changes: 2 additions & 2 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
Expand Down
4 changes: 2 additions & 2 deletions table/tables/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
}
78 changes: 39 additions & 39 deletions util/plancodec/id.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion util/testutil/testutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down

0 comments on commit 931ff98

Please sign in to comment.