Skip to content

Commit

Permalink
*: fix static check issues (pingcap#14286)
Browse files Browse the repository at this point in the history
  • Loading branch information
Rustin170506 authored and sre-bot committed Jan 6, 2020
1 parent 4f242fe commit 174ff27
Show file tree
Hide file tree
Showing 13 changed files with 65 additions and 80 deletions.
1 change: 0 additions & 1 deletion bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -753,7 +753,6 @@ func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan
}
terror.Call(recordSets[0].Close)
resultChan <- err
return
}

// HandleEvolvePlanTask tries to evolve one plan task.
Expand Down
2 changes: 2 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -469,6 +469,7 @@ func (s *testSuite2) TestAdminCheckPartitionTableFailed(c *C) {
err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), int64(i+8))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
tk.MustExec("admin check table admin_test_p")
}

Expand All @@ -491,6 +492,7 @@ func (s *testSuite2) TestAdminCheckPartitionTableFailed(c *C) {
err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), int64(i))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
tk.MustExec("admin check table admin_test_p")
}
}
Expand Down
40 changes: 16 additions & 24 deletions executor/aggfuncs/func_first_row.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,13 +95,12 @@ func (e *firstRow4Int) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalInt(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalInt(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, input
break
}
return nil
}
Expand Down Expand Up @@ -142,13 +141,12 @@ func (e *firstRow4Float32) UpdatePartialResult(sctx sessionctx.Context, rowsInGr
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalReal(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalReal(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, float32(input)
break
}
return nil
}
Expand Down Expand Up @@ -188,13 +186,12 @@ func (e *firstRow4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGr
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalReal(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalReal(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, input
break
}
return nil
}
Expand Down Expand Up @@ -234,13 +231,12 @@ func (e *firstRow4String) UpdatePartialResult(sctx sessionctx.Context, rowsInGro
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalString(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalString(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, stringutil.Copy(input)
break
}
return nil
}
Expand Down Expand Up @@ -281,13 +277,12 @@ func (e *firstRow4Time) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalTime(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalTime(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, input
break
}
return nil
}
Expand Down Expand Up @@ -328,13 +323,12 @@ func (e *firstRow4Duration) UpdatePartialResult(sctx sessionctx.Context, rowsInG
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalDuration(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalDuration(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, input
break
}
return nil
}
Expand Down Expand Up @@ -374,13 +368,12 @@ func (e *firstRow4JSON) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalJSON(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalJSON(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull, p.val = true, isNull, input.Copy()
break
}
return nil
}
Expand Down Expand Up @@ -420,16 +413,15 @@ func (e *firstRow4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGr
if p.gotFirstRow {
return nil
}
for _, row := range rowsInGroup {
input, isNull, err := e.args[0].EvalDecimal(sctx, row)
if len(rowsInGroup) > 0 {
input, isNull, err := e.args[0].EvalDecimal(sctx, rowsInGroup[0])
if err != nil {
return err
}
p.gotFirstRow, p.isNull = true, isNull
if input != nil {
p.val = *input
}
break
}
return nil
}
Expand Down
4 changes: 2 additions & 2 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -608,7 +608,7 @@ type AnalyzeFastExec struct {

func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild *bool, err *error, sampTasks *[]*AnalyzeFastTask) {
defer func() {
if *needRebuild == true {
if *needRebuild {
for ok := true; ok; _, ok = <-e.sampLocs {
// Do nothing, just clear the channel.
}
Expand Down Expand Up @@ -1023,7 +1023,7 @@ func (e *AnalyzeFastExec) buildColumnStats(ID int64, collector *statistics.Sampl
}

func (e *AnalyzeFastExec) buildIndexStats(idxInfo *model.IndexInfo, collector *statistics.SampleCollector, rowCount int64) (*statistics.Histogram, *statistics.CMSketch, error) {
data := make([][][]byte, len(idxInfo.Columns), len(idxInfo.Columns))
data := make([][][]byte, len(idxInfo.Columns))
for _, sample := range collector.Samples {
var preLen int
remained := sample.Value.GetBytes()
Expand Down
2 changes: 0 additions & 2 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,10 +242,8 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) {
}
err = mockExec.TestFastSample()
c.Assert(err, IsNil)
vals := make([][]string, 0)
c.Assert(len(mockExec.Collectors), Equals, 3)
for i := 0; i < 2; i++ {
vals = append(vals, make([]string, 0))
samples := mockExec.Collectors[i].Samples
c.Assert(len(samples), Equals, 20)
for j := 1; j < 20; j++ {
Expand Down
2 changes: 1 addition & 1 deletion executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {

if e.idxInfo != nil {
// `SELECT a, b FROM t WHERE (a, b) IN ((1, 2), (1, 2), (2, 1), (1, 2))` should not return duplicated rows
dedup := make(map[hack.MutableString]struct{}, 0)
dedup := make(map[hack.MutableString]struct{})
keys := make([]kv.Key, 0, len(e.idxVals))
for _, idxVals := range e.idxVals {
idxKey, err1 := encodeIndexKey(e.base(), e.tblInfo, e.idxInfo, idxVals)
Expand Down
8 changes: 2 additions & 6 deletions executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,18 +257,14 @@ func getDbFromResultNode(resultNode ast.ResultSetNode) []string { //may have dup
if x.Left != nil {
dbs := getDbFromResultNode(x.Left)
if dbs != nil {
for _, db := range dbs {
dbLabels = append(dbLabels, db)
}
dbLabels = append(dbLabels, dbs...)
}
}

if x.Right != nil {
dbs := getDbFromResultNode(x.Right)
if dbs != nil {
for _, db := range dbs {
dbLabels = append(dbLabels, db)
}
dbLabels = append(dbLabels, dbs...)
}
}
}
Expand Down
10 changes: 4 additions & 6 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1191,13 +1191,11 @@ func (e *TableScanExec) nextChunk4InfoSchema(ctx context.Context, chk *chunk.Chu

// nextHandle gets the unique handle for next row.
func (e *TableScanExec) nextHandle() (handle int64, found bool, err error) {
for {
handle, found, err = e.t.Seek(e.ctx, e.seekHandle)
if err != nil || !found {
return 0, false, err
}
return handle, true, nil
handle, found, err = e.t.Seek(e.ctx, e.seekHandle)
if err != nil || !found {
return 0, false, err
}
return handle, true, nil
}

func (e *TableScanExec) getRow(handle int64) ([]types.Datum, error) {
Expand Down
16 changes: 10 additions & 6 deletions executor/explain_unit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,22 +71,26 @@ func TestExplainAnalyzeInvokeNextAndClose(t *testing.T) {
explain: nil,
}
// mockErrorOperator returns errors
mockOper := mockErrorOperator{baseExec, false, false}
explainExec.analyzeExec = &mockOper
mockOpr := mockErrorOperator{baseExec, false, false}
explainExec.analyzeExec = &mockOpr
tmpCtx := context.Background()
_, err := explainExec.generateExplainInfo(tmpCtx)

expectedStr := "next error, close error"
if err.Error() != expectedStr || !mockOper.closed {
if err != nil && (err.Error() != expectedStr || !mockOpr.closed) {
t.Errorf(err.Error())
}
// mockErrorOperator panic
mockOper = mockErrorOperator{baseExec, true, false}
explainExec.analyzeExec = &mockOper
mockOpr = mockErrorOperator{baseExec, true, false}
explainExec.analyzeExec = &mockOpr
defer func() {
if panicErr := recover(); panicErr == nil || !mockOper.closed {
if panicErr := recover(); panicErr == nil || !mockOpr.closed {
t.Errorf("panic test failed: without panic or close() is not called")
}
}()

_, err = explainExec.generateExplainInfo(tmpCtx)
if err != nil {
t.Errorf(err.Error())
}
}
1 change: 0 additions & 1 deletion executor/index_lookup_hash_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -519,7 +519,6 @@ func (iw *indexHashJoinInnerWorker) buildHashTableForOuterResult(ctx context.Con
task.lookupMap.Put(h.Sum64(), rowPtr)
}
}
return
}

func (iw *indexHashJoinInnerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask) error {
Expand Down
4 changes: 2 additions & 2 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ func (e *HashJoinExec) handleUnmatchedRowsFromHashTableInMemory(workerID uint) {
for i := int(workerID); i < numChks; i += int(e.concurrency) {
chk := e.rowContainer.GetChunk(i)
for j := 0; j < chk.NumRows(); j++ {
if e.outerMatchedStatus[i].UnsafeIsSet(j) == false { // process unmatched outer rows
if !e.outerMatchedStatus[i].UnsafeIsSet(j) { // process unmatched outer rows
e.joiners[workerID].onMissMatch(false, chk.GetRow(j), joinResult.chk)
}
if joinResult.chk.IsFull() {
Expand Down Expand Up @@ -376,7 +376,7 @@ func (e *HashJoinExec) handleUnmatchedRowsFromHashTableInDisk(workerID uint) {
e.joinResultCh <- joinResult
return
}
if e.outerMatchedStatus[i].UnsafeIsSet(j) == false { // process unmatched outer rows
if !e.outerMatchedStatus[i].UnsafeIsSet(j) { // process unmatched outer rows
e.joiners[workerID].onMissMatch(false, row, joinResult.chk)
}
if joinResult.chk.IsFull() {
Expand Down
3 changes: 1 addition & 2 deletions executor/load_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,7 @@ func (e *LoadDataInfo) CommitWork(ctx context.Context) error {
case <-e.QuitCh:
err = errors.New("commit forced to quit")
logutil.Logger(ctx).Error("commit forced to quit, possible preparation failed")
break
return err
case commitTask, ok := <-e.commitTaskQueue:
if ok {
start := time.Now()
Expand All @@ -257,7 +257,6 @@ func (e *LoadDataInfo) CommitWork(ctx context.Context) error {
zap.Int("tasks in queue", len(e.commitTaskQueue)))
} else {
end = true
break
}
}
if err != nil {
Expand Down
52 changes: 25 additions & 27 deletions executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,41 +163,39 @@ func (us *UnionScanExec) Next(ctx context.Context, req *chunk.Chunk) error {

// getOneRow gets one result row from dirty table or child.
func (us *UnionScanExec) getOneRow(ctx context.Context) ([]types.Datum, error) {
for {
snapshotRow, err := us.getSnapshotRow(ctx)
snapshotRow, err := us.getSnapshotRow(ctx)
if err != nil {
return nil, err
}
addedRow := us.getAddedRow()
var row []types.Datum
var isSnapshotRow bool
if addedRow == nil {
row = snapshotRow
isSnapshotRow = true
} else if snapshotRow == nil {
row = addedRow
} else {
isSnapshotRow, err = us.shouldPickFirstRow(snapshotRow, addedRow)
if err != nil {
return nil, err
}
addedRow := us.getAddedRow()
var row []types.Datum
var isSnapshotRow bool
if addedRow == nil {
if isSnapshotRow {
row = snapshotRow
isSnapshotRow = true
} else if snapshotRow == nil {
row = addedRow
} else {
isSnapshotRow, err = us.shouldPickFirstRow(snapshotRow, addedRow)
if err != nil {
return nil, err
}
if isSnapshotRow {
row = snapshotRow
} else {
row = addedRow
}
}
if row == nil {
return nil, nil
row = addedRow
}
}
if row == nil {
return nil, nil
}

if isSnapshotRow {
us.cursor4SnapshotRows++
} else {
us.cursor4AddRows++
}
return row, nil
if isSnapshotRow {
us.cursor4SnapshotRows++
} else {
us.cursor4AddRows++
}
return row, nil
}

func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, error) {
Expand Down

0 comments on commit 174ff27

Please sign in to comment.