Skip to content

Commit

Permalink
*: Modify the Makefile for go vet (#3900)
Browse files Browse the repository at this point in the history
  • Loading branch information
jackysp authored Jul 31, 2017
1 parent 0aa27c3 commit dff73cc
Show file tree
Hide file tree
Showing 45 changed files with 219 additions and 169 deletions.
10 changes: 5 additions & 5 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,9 @@ GOVERALLS := goveralls
ARCH := "`uname -s`"
LINUX := "Linux"
MAC := "Darwin"
PACKAGES := $$(go list ./...| grep -vE 'vendor')
FILES := $$(find . -name '*.go' | grep -vE 'vendor')
PACKAGES := $$(go list ./...| grep -vE "vendor")
FILES := $$(find . -name "*.go" | grep -vE "vendor")
TOPDIRS := $$(ls -d */ | grep -vE "vendor")

LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBBuildTS=$(shell date -u '+%Y-%m-%d %I:%M:%S')"
LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitHash=$(shell git rev-parse HEAD)"
Expand Down Expand Up @@ -74,9 +75,8 @@ check:
go get github.com/golang/lint/golint

@echo "vet"
@ go tool vet $(FILES) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'
@echo "vet --shadow"
@ go tool vet --shadow $(FILES) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'
@ go tool vet -all -shadow $(TOPDIRS) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'
@ go tool vet -all -shadow *.go 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'
@echo "golint"
@ golint ./... 2>&1 | grep -vE 'context\.Context|LastInsertId|NewLexer|\.pb\.go' | awk '{print} END{if(NR>0) {exit 1}}'
@echo "gofmt (simplify)"
Expand Down
7 changes: 4 additions & 3 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -451,13 +451,14 @@ func upgradeToVer12(s Session) {
user := row.Data[0].GetString()
host := row.Data[1].GetString()
pass := row.Data[2].GetString()
newpass, err := oldPasswordUpgrade(pass)
var newPass string
newPass, err = oldPasswordUpgrade(pass)
if err != nil {
log.Fatal(err)
return
}
sql := fmt.Sprintf(`UPDATE mysql.user set password = "%s" where user="%s" and host="%s"`, newpass, user, host)
sqls = append(sqls, sql)
updateSQL := fmt.Sprintf(`UPDATE mysql.user set password = "%s" where user="%s" and host="%s"`, newPass, user, host)
sqls = append(sqls, updateSQL)
row, err = r.Next()
}

Expand Down
6 changes: 3 additions & 3 deletions cmd/benchfilesort/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,9 +417,9 @@ func main() {
flag.Parse()

if len(os.Args) == 1 {
fmt.Println("Usage:\n")
fmt.Println("\tbenchfilesort command [arguments]\n")
fmt.Println("The commands are:\n")
fmt.Printf("Usage:\n\n")
fmt.Printf("\tbenchfilesort command [arguments]\n\n")
fmt.Printf("The commands are:\n\n")
fmt.Println("\tgen\t", "generate rows")
fmt.Println("\trun\t", "run tests")
fmt.Println("")
Expand Down
6 changes: 4 additions & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,8 @@ func (d *ddl) onAddColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
case model.StateWriteReorganization:
// reorganization -> public
// Get the current version for reorganization if we don't have it.
reorgInfo, err := d.getReorgInfo(t, job)
var reorgInfo *reorgInfo
reorgInfo, err = d.getReorgInfo(t, job)
if err != nil || reorgInfo.first {
// If we run reorg firstly, we should update the job snapshot version
// and then run the reorg next time.
Expand Down Expand Up @@ -233,7 +234,8 @@ func (d *ddl) onDropColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
ver, err = updateTableInfo(t, job, tblInfo, originalState)
case model.StateDeleteReorganization:
// reorganization -> absent
reorgInfo, err := d.getReorgInfo(t, job)
var reorgInfo *reorgInfo
reorgInfo, err = d.getReorgInfo(t, job)
if err != nil || reorgInfo.first {
// If we run reorg firstly, we should update the job snapshot version
// and then run the reorg next time.
Expand Down
8 changes: 4 additions & 4 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -889,7 +889,7 @@ func (d *ddl) AddColumn(ctx context.Context, ti ast.Ident, spec *ast.AlterTableS
referableColNames[col.Name.L] = struct{}{}
}
_, dependColNames := findDependedColumnNames(spec.NewColumn)
if err := columnNamesCover(referableColNames, dependColNames); err != nil {
if err = columnNamesCover(referableColNames, dependColNames); err != nil {
return errors.Trace(err)
}
}
Expand Down Expand Up @@ -1158,7 +1158,7 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, ident ast.Ident, origi
if err != nil {
return nil, errors.Trace(err)
}
if err := setDefaultAndComment(ctx, newCol, spec.NewColumn.Options); err != nil {
if err = setDefaultAndComment(ctx, newCol, spec.NewColumn.Options); err != nil {
return nil, errors.Trace(err)
}

Expand Down Expand Up @@ -1261,12 +1261,12 @@ func (d *ddl) AlterColumn(ctx context.Context, ident ast.Ident, spec *ast.AlterT
}

// Clean the NoDefaultValueFlag value.
col.Flag &= (^uint(mysql.NoDefaultValueFlag))
col.Flag &= ^uint(mysql.NoDefaultValueFlag)
if len(spec.NewColumn.Options) == 0 {
col.DefaultValue = nil
setNoDefaultValueFlag(col, false)
} else {
err := setDefaultValue(ctx, col, spec.NewColumn.Options[0])
err = setDefaultValue(ctx, col, spec.NewColumn.Options[0])
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -698,7 +698,7 @@ LOOP:
matchRows(c, rows, [][]interface{}{{count - int64(step)}})

for i := num; i < num+step; i++ {
rows := s.mustQuery(c, "select c4 from t2 where c4 = ?", i)
rows = s.mustQuery(c, "select c4 from t2 where c4 = ?", i)
matchRows(c, rows, [][]interface{}{{i}})
}

Expand Down
6 changes: 4 additions & 2 deletions ddl/foreign_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,8 @@ func (s *testForeighKeySuite) TestForeignKey(c *C) {
}
mu.Lock()
defer mu.Unlock()
t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID)
var t table.Table
t, err = testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID)
if err != nil {
hookErr = errors.Trace(err)
return
Expand Down Expand Up @@ -178,7 +179,8 @@ func (s *testForeighKeySuite) TestForeignKey(c *C) {
}
mu.Lock()
defer mu.Unlock()
t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID)
var t table.Table
t, err = testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID)
if err != nil {
hookErr = errors.Trace(err)
return
Expand Down
3 changes: 2 additions & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,8 @@ func (d *ddl) onCreateIndex(t *meta.Meta, job *model.Job) (ver int64, err error)
ver, err = updateTableInfo(t, job, tblInfo, originalState)
case model.StateWriteReorganization:
// reorganization -> public
reorgInfo, err := d.getReorgInfo(t, job)
var reorgInfo *reorgInfo
reorgInfo, err = d.getReorgInfo(t, job)
if err != nil || reorgInfo.first {
// If we run reorg firstly, we should update the job snapshot version
// and then run the reorg next time.
Expand Down
4 changes: 2 additions & 2 deletions ddl/owner_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,9 +184,9 @@ func (m *ownerManager) campaignLoop(ctx goctx.Context, etcdSession *concurrency.
case <-ctx.Done():
// Revoke the session lease.
// If revoke takes longer than the ttl, lease is expired anyway.
ctx, cancel := goctx.WithTimeout(goctx.Background(),
cancelCtx, cancel := goctx.WithTimeout(goctx.Background(),
time.Duration(ManagerSessionTTL)*time.Second)
_, err = m.etcdCli.Revoke(ctx, etcdSession.Lease())
_, err = m.etcdCli.Revoke(cancelCtx, etcdSession.Lease())
cancel()
log.Infof("[ddl] %s break campaign loop err %v", idInfo, err)
return
Expand Down
3 changes: 2 additions & 1 deletion ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,8 @@ func (d *ddl) onDropSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) {
err = t.UpdateDatabase(dbInfo)
case model.StateDeleteOnly:
dbInfo.State = model.StateNone
tables, err := t.ListTables(job.SchemaID)
var tables []*model.TableInfo
tables, err = t.ListTables(job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion distsql/xeval/eval_control_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,8 @@ func (e *Evaluator) evalIf(expr *tipb.Expr) (d types.Datum, err error) {
return d, errors.Trace(err)
}
if !child1.IsNull() {
x, err := child1.ToBool(e.StatementCtx)
var x int64
x, err = child1.ToBool(e.StatementCtx)
if err != nil {
return d, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion distsql/xeval/eval_data_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,8 @@ func (e *Evaluator) evalColumnRef(val []byte) (types.Datum, error) {

// TODO: Remove this case.
if e.ColVals == nil {
d, ok := e.Row[i]
var ok bool
d, ok = e.Row[i]
if !ok {
return d, ErrInvalid.Gen("column % x not found", val)
}
Expand Down
3 changes: 2 additions & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -399,7 +399,8 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio

if ebd, ok := store.(etcdBackend); ok {
if addrs := ebd.EtcdAddrs(); addrs != nil {
cli, err := clientv3.New(clientv3.Config{
var cli *clientv3.Client
cli, err = clientv3.New(clientv3.Config{
Endpoints: addrs,
DialTimeout: 5 * time.Second,
})
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ func CollectSamplesAndEstimateNDVs(ctx context.Context, e ast.RecordSet, numCols
return collectors, pkBuilder, nil
}
if pkInfo != nil {
err := pkBuilder.Iterate(row.Data)
err = pkBuilder.Iterate(row.Data)
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -839,7 +839,7 @@ func (b *executorBuilder) buildTableScanForAnalyze(tblInfo *model.TableInfo, pk
cols = append([]*model.ColumnInfo{pk}, cols...)
}
schema := expression.NewSchema(expression.ColumnInfos2Columns(tblInfo.Name, cols)...)
ranges := []types.IntColumnRange{{math.MinInt64, math.MaxInt64}}
ranges := []types.IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}}
if b.ctx.GetClient().IsRequestTypeSupported(kv.ReqTypeDAG, kv.ReqSubTypeBasic) {
e := &TableReaderExecutor{
table: table,
Expand Down
6 changes: 5 additions & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -443,7 +443,11 @@ func (e *SelectionExec) initController() error {
}
x.ranges = ranges
case *XSelectIndexExec:
accessCondition, newConds, _, accessInAndEqCount := ranger.DetachIndexScanConditions(newConds, x.index)
var (
accessCondition []expression.Expression
accessInAndEqCount int
)
accessCondition, newConds, _, accessInAndEqCount = ranger.DetachIndexScanConditions(newConds, x.index)
idxConds, tblConds := ranger.DetachIndexFilterConditions(newConds, x.index.Columns, x.tableInfo)
x.indexConditionPBExpr, _, _ = expression.ExpressionsToPB(sc, idxConds, client)
tableConditionPBExpr, _, _ := expression.ExpressionsToPB(sc, tblConds, client)
Expand Down
5 changes: 3 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -404,7 +404,8 @@ func (s *testSuite) TestIssue2612(c *C) {
c.Assert(err, IsNil)
row, err := rs.Next()
c.Assert(err, IsNil)
row.Data[0].GetMysqlDuration().String()
str := row.Data[0].GetMysqlDuration().String()
c.Assert(str, Equals, "-46:09:02")
}

// TestIssue345 is related with https://github.com/pingcap/tidb/issues/345
Expand Down Expand Up @@ -1236,7 +1237,7 @@ func (s *testSuite) TestBuiltin(c *C) {
tk.MustExec("create table t (a varchar(255), b int)")
for i, d := range data {
tk.MustExec(fmt.Sprintf("insert into t values('%s', %d)", d.val, i))
result := tk.MustQuery(fmt.Sprintf("select * from t where a %s '%s'", queryOp, d.pattern))
result = tk.MustQuery(fmt.Sprintf("select * from t where a %s '%s'", queryOp, d.pattern))
if d.result == 1 {
rowStr := fmt.Sprintf("%s %d", d.val, i)
result.Check(testkit.Rows(rowStr))
Expand Down
3 changes: 2 additions & 1 deletion executor/merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,8 @@ func (e *MergeJoinExec) computeCrossProduct() error {
for _, lRow := range e.leftRows {
// make up for outer join since we ignored single table conditions previously
if e.leftFilter != nil {
matched, err := expression.EvalBool(e.leftFilter, lRow.Data, e.ctx)
var matched bool
matched, err = expression.EvalBool(e.leftFilter, lRow.Data, e.ctx)
if err != nil {
return errors.Trace(err)
}
Expand Down
12 changes: 6 additions & 6 deletions executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,9 +64,9 @@ func updateRecord(ctx context.Context, h int64, oldData, newData []types.Datum,
if newData[i].IsNull() {
return false, errors.Errorf("Column '%v' cannot be null", col.Name.O)
}
val, err := newData[i].ToInt64(sc)
if err != nil {
return false, errors.Trace(err)
val, errTI := newData[i].ToInt64(sc)
if errTI != nil {
return false, errors.Trace(errTI)
}
t.RebaseAutoID(val, true)
}
Expand Down Expand Up @@ -106,9 +106,9 @@ func updateRecord(ctx context.Context, h int64, oldData, newData []types.Datum,
// Fill values into on-update-now fields, only if they are really changed.
for i, col := range t.Cols() {
if mysql.HasOnUpdateNowFlag(col.Flag) && !modified[i] && !onUpdateSpecified[i] {
v, err := expression.GetTimeValue(ctx, expression.CurrentTimestamp, col.Tp, col.Decimal)
if err != nil {
return false, errors.Trace(err)
v, errGT := expression.GetTimeValue(ctx, expression.CurrentTimestamp, col.Tp, col.Decimal)
if errGT != nil {
return false, errors.Trace(errGT)
}
newData[i] = v
}
Expand Down
6 changes: 4 additions & 2 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -367,7 +367,8 @@ func (b *builtinCastIntAsDecimalSig) evalDecimal(row []types.Datum) (res *types.
if !mysql.HasUnsignedFlag(b.args[0].GetType().Flag) {
res = types.NewDecFromInt(val)
} else {
uVal, err := types.ConvertIntToUint(val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
var uVal uint64
uVal, err = types.ConvertIntToUint(val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
if err != nil {
return res, false, errors.Trace(err)
}
Expand All @@ -390,7 +391,8 @@ func (b *builtinCastIntAsStringSig) evalString(row []types.Datum) (res string, i
if !mysql.HasUnsignedFlag(b.args[0].GetType().Flag) {
res = strconv.FormatInt(val, 10)
} else {
uVal, err := types.ConvertIntToUint(val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
var uVal uint64
uVal, err = types.ConvertIntToUint(val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
if err != nil {
return res, false, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -962,7 +962,8 @@ func (s *builtinCompareSig) eval(row []types.Datum) (d types.Datum, err error) {
}

if s.op != opcode.NullEQ {
if aa, bb, err := types.CoerceDatum(sc, a, b); err == nil {
var aa, bb types.Datum
if aa, bb, err = types.CoerceDatum(sc, a, b); err == nil {
a = aa
b = bb
}
Expand Down
24 changes: 16 additions & 8 deletions expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,10 +94,12 @@ func (s *testEvaluatorSuite) TestJSONExtract(c *C) {
c.Assert(err, IsNil)
switch x := t.Expected.(type) {
case string:
j1, err := json.ParseFromString(x)
var j1 json.JSON
j1, err = json.ParseFromString(x)
c.Assert(err, IsNil)
j2 := d.GetMysqlJSON()
cmp, err := json.CompareJSON(j1, j2)
var cmp int
cmp, err = json.CompareJSON(j1, j2)
c.Assert(err, IsNil)
c.Assert(cmp, Equals, 0)
}
Expand Down Expand Up @@ -134,10 +136,12 @@ func (s *testEvaluatorSuite) TestJSONSetInsertReplace(c *C) {
c.Assert(err, IsNil)
switch x := t.Expected.(type) {
case string:
j1, err := json.ParseFromString(x)
var j1 json.JSON
j1, err = json.ParseFromString(x)
c.Assert(err, IsNil)
j2 := d.GetMysqlJSON()
cmp, err := json.CompareJSON(j1, j2)
var cmp int
cmp, err = json.CompareJSON(j1, j2)
c.Assert(err, IsNil)
c.Assert(cmp, Equals, 0)
}
Expand Down Expand Up @@ -228,10 +232,12 @@ func (s *testEvaluatorSuite) TestJSONObject(c *C) {
c.Assert(err, IsNil)
switch x := t.Expected.(type) {
case string:
j1, err := json.ParseFromString(x)
var j1 json.JSON
j1, err = json.ParseFromString(x)
c.Assert(err, IsNil)
j2 := d.GetMysqlJSON()
cmp, err := json.CompareJSON(j1, j2)
var cmp int
cmp, err = json.CompareJSON(j1, j2)
c.Assert(err, IsNil)
c.Assert(cmp, Equals, 0)
}
Expand Down Expand Up @@ -277,10 +283,12 @@ func (s *testEvaluatorSuite) TestJSONORemove(c *C) {
c.Assert(err, IsNil)
switch x := t.Expected.(type) {
case string:
j1, err := json.ParseFromString(x)
var j1 json.JSON
j1, err = json.ParseFromString(x)
c.Assert(err, IsNil)
j2 := d.GetMysqlJSON()
cmp, err := json.CompareJSON(j1, j2)
var cmp int
cmp, err = json.CompareJSON(j1, j2)
c.Assert(err, IsNil)
c.Assert(cmp, Equals, 0)
}
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -1947,7 +1947,8 @@ func (b *builtinExportSetSig) eval(row []types.Datum) (d types.Datum, err error)
)
switch len(args) {
case 5:
arg, err := args[4].ToInt64(b.ctx.GetSessionVars().StmtCtx)
var arg int64
arg, err = args[4].ToInt64(b.ctx.GetSessionVars().StmtCtx)
if err != nil {
return d, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion expression/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func getTimeValue(ctx context.Context, v interface{}, tp byte, fsp int) (d types
if upperX == CurrentTimestamp {
value.Time = types.FromGoTime(defaultTime)
if tp == mysql.TypeTimestamp {
err := value.ConvertTimeZone(time.Local, ctx.GetSessionVars().GetTimeZone())
err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().GetTimeZone())
if err != nil {
return d, errors.Trace(err)
}
Expand Down
Loading

0 comments on commit dff73cc

Please sign in to comment.