Add staticcheck linter and fix existing problems (#27174)

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
pull/26369/head
congqixia 2023-09-19 10:05:22 +08:00 committed by GitHub
parent 4feb3fa7c6
commit cc9974979f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 50 additions and 41 deletions

View File

@ -11,6 +11,7 @@ run:
linters:
disable-all: true
enable:
- staticcheck
- typecheck
- goimports
- misspell
@ -82,6 +83,11 @@ issues:
- G402
# Use of weak random number generator math/rand
- G404
# Unused parameters
- SA1019
# defer return errors
- SA5001
# Maximum issues count per one linter. Set to 0 to disable. Default is 50.
max-issues-per-linter: 0
# Maximum count of issues with the same text. Set to 0 to disable. Default is 3.

View File

@ -419,7 +419,7 @@ func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) {
require.True(t, has)
call.Unset()
call = mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil)
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil)
err = c.handleMergeCompactionResult(plan, compactionResult2)
assert.Error(t, err)
}

View File

@ -465,9 +465,7 @@ func Test_compactionTrigger_force(t *testing.T) {
})
t.Run(tt.name+" with DiskANN index", func(t *testing.T) {
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
// Collection 1000 means it has DiskANN index
segment.CollectionID = 1000
}

View File

@ -351,6 +351,7 @@ func TestMeta_Basic(t *testing.T) {
catalog = datacoord.NewCatalog(metakv, "", "")
meta, err = newMeta(context.TODO(), catalog, nil)
assert.NoError(t, err)
assert.NotNil(t, meta)
})
t.Run("Test GetCount", func(t *testing.T) {

View File

@ -11,6 +11,7 @@ import (
"strings"
"github.com/milvus-io/milvus/pkg/util/parameterutil.go"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/util/merr"
@ -274,20 +275,23 @@ func checkAndSetData(body string, collDescResp *milvuspb.DescribeCollectionRespo
for mapKey, mapValue := range data.Map() {
if !containsString(fieldNames, mapKey) {
mapValueStr := mapValue.String()
if mapValue.Type == gjson.True || mapValue.Type == gjson.False {
switch mapValue.Type {
case gjson.True, gjson.False:
reallyData[mapKey] = cast.ToBool(mapValueStr)
} else if mapValue.Type == gjson.String {
case gjson.String:
reallyData[mapKey] = mapValueStr
} else if mapValue.Type == gjson.Number {
case gjson.Number:
if strings.Contains(mapValue.Raw, ".") {
reallyData[mapKey] = cast.ToFloat64(mapValue.Raw)
} else {
reallyData[mapKey] = cast.ToInt64(mapValueStr)
}
} else if mapValue.Type == gjson.JSON {
case gjson.JSON:
reallyData[mapKey] = mapValue.Value()
} else {
case gjson.Null:
// skip null
default:
log.Warn("unknown json type found", zap.Int("mapValue.Type", int(mapValue.Type)))
}
}
}

View File

@ -321,7 +321,7 @@ func TestPrimaryField(t *testing.T) {
}
func TestInsertWithDynamicFields(t *testing.T) {
body := "{\"data\": {\"id\": 0, \"book_id\": 1, \"book_intro\": [0.1, 0.2], \"word_count\": 2}}"
body := "{\"data\": {\"id\": 0, \"book_id\": 1, \"book_intro\": [0.1, 0.2], \"word_count\": 2, \"classified\": false, \"databaseID\": null}}"
req := InsertReq{}
coll := generateCollectionSchema(false)
err := checkAndSetData(body, &milvuspb.DescribeCollectionResponse{
@ -336,7 +336,7 @@ func TestInsertWithDynamicFields(t *testing.T) {
assert.Equal(t, err, nil)
assert.Equal(t, fieldsData[len(fieldsData)-1].IsDynamic, true)
assert.Equal(t, fieldsData[len(fieldsData)-1].Type, schemapb.DataType_JSON)
assert.Equal(t, string(fieldsData[len(fieldsData)-1].GetScalars().GetJsonData().GetData()[0]), "{\"id\":0}")
assert.Equal(t, string(fieldsData[len(fieldsData)-1].GetScalars().GetJsonData().GetData()[0]), "{\"classified\":false,\"id\":0}")
}
func TestSerialize(t *testing.T) {

View File

@ -372,10 +372,6 @@ func Test_AlterSegments(t *testing.T) {
opGroupCount := 0
metakv := mocks.NewMetaKv(t)
metakv.EXPECT().MultiSave(mock.Anything).RunAndReturn(func(m map[string]string) error {
var ks []string
for k := range m {
ks = append(ks, k)
}
maps.Copy(savedKvs, m)
opGroupCount++
return nil

View File

@ -1794,7 +1794,7 @@ func Test_JSONContainsAny(t *testing.T) {
assert.False(t, plan.GetVectorAnns().GetPredicates().GetJsonContainsExpr().GetElementsSameType())
expr = `JSON_CONTAINS_ANY(A, 1)`
plan, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{
_, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{
Topk: 0,
MetricType: "",
SearchParams: "",

View File

@ -207,6 +207,8 @@ func newMockDmlTask(ctx context.Context) *mockDmlTask {
return &mockDmlTask{
mockTask: newMockTask(ctx),
vchans: vchans,
pchans: pchans,
}
}

View File

@ -1,6 +1,7 @@
package proxy
import (
"context"
"testing"
"github.com/milvus-io/milvus/internal/proto/planpb"
@ -16,13 +17,14 @@ func Test_createMilvusReducer(t *testing.T) {
},
}
var r milvusReducer
ctx := context.Background()
r = createMilvusReducer(nil, nil, nil, nil, n, "")
r = createMilvusReducer(ctx, nil, nil, nil, n, "")
_, ok := r.(*defaultLimitReducer)
assert.True(t, ok)
n.Node.(*planpb.PlanNode_Query).Query.IsCount = true
r = createMilvusReducer(nil, nil, nil, nil, n, "")
r = createMilvusReducer(ctx, nil, nil, nil, n, "")
_, ok = r.(*cntReducer)
assert.True(t, ok)
}

View File

@ -525,10 +525,9 @@ func validateSchema(coll *schemapb.CollectionSchema) error {
if err4 != nil {
return err4
}
} else {
// in C++, default type will be specified
// do nothing
}
// in C++, default type will be specified
// do nothing
} else {
if len(field.IndexParams) != 0 {
return fmt.Errorf("index params is not empty for scalar field: %s(%d)", field.Name, field.FieldID)

View File

@ -397,10 +397,8 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceMultiRound() {
balancer := suite.balancer
//1. set up target for multi collections
collections := make([]*meta.Collection, 0, len(balanceCase.collectionIDs))
for i := range balanceCase.collectionIDs {
collection := utils.CreateTestCollection(balanceCase.collectionIDs[i], int32(balanceCase.replicaIDs[i]))
collections = append(collections, collection)
suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, balanceCase.collectionIDs[i]).Return(
nil, balanceCase.collectionsSegments[i], nil)

View File

@ -54,7 +54,7 @@ func (replica *Replica) AddNode(nodes ...int64) {
func (replica *Replica) GetNodes() []int64 {
replica.rwmutex.RLock()
defer replica.rwmutex.RUnlock()
if replica != nil {
if replica.nodes != nil {
return replica.nodes.Collect()
}
return nil
@ -63,7 +63,7 @@ func (replica *Replica) GetNodes() []int64 {
func (replica *Replica) Len() int {
replica.rwmutex.RLock()
defer replica.rwmutex.RUnlock()
if replica != nil {
if replica.nodes != nil {
return replica.nodes.Len()
}
@ -73,7 +73,7 @@ func (replica *Replica) Len() int {
func (replica *Replica) Contains(node int64) bool {
replica.rwmutex.RLock()
defer replica.rwmutex.RUnlock()
if replica != nil {
if replica.nodes != nil {
return replica.nodes.Contain(node)
}

View File

@ -575,7 +575,6 @@ func (sd *shardDelegator) readDeleteFromMsgstream(ctx context.Context, position
// reach safe ts
if safeTs <= msgPack.EndPositions[0].GetTimestamp() {
hasMore = false
break
}
}
}

View File

@ -152,7 +152,7 @@ func (mgr *segmentManager) Put(segmentType SegmentType, segments ...Segment) {
var replacedSegment []Segment
mgr.mu.Lock()
defer mgr.mu.Unlock()
targetMap := mgr.growingSegments
var targetMap map[int64]Segment
switch segmentType {
case SegmentTypeGrowing:
targetMap = mgr.growingSegments

View File

@ -1771,6 +1771,7 @@ func (suite *ServiceSuite) TestSyncDistribution_Normal() {
req.Actions = []*querypb.SyncAction{syncVersionAction}
status, err = suite.node.SyncDistribution(ctx, req)
suite.NoError(err)
suite.Equal(commonpb.ErrorCode_Success, status.GetErrorCode())
}
func (suite *ServiceSuite) TestSyncDistribution_ReleaseResultCheck() {

View File

@ -573,6 +573,7 @@ func TestImportManager_ImportJob(t *testing.T) {
rowReq.Files = []string{"f1.json"}
mgr = newImportManager(context.TODO(), mockKv, idAlloc, importServiceFunc, callGetSegmentStates, nil, nil)
resp = mgr.importJob(context.TODO(), rowReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, len(rowReq.Files), len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
@ -586,6 +587,7 @@ func TestImportManager_ImportJob(t *testing.T) {
// since the importServiceFunc return error, tasks will be kept in pending list
mgr = newImportManager(context.TODO(), mockKv, idAlloc, importServiceFunc, callGetSegmentStates, nil, nil)
resp = mgr.importJob(context.TODO(), colReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, 1, len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
@ -600,12 +602,14 @@ func TestImportManager_ImportJob(t *testing.T) {
// row-based case, since the importServiceFunc return success, tasks will be sent to working list
mgr = newImportManager(context.TODO(), mockKv, idAlloc, importServiceFunc, callGetSegmentStates, nil, nil)
resp = mgr.importJob(context.TODO(), rowReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, len(rowReq.Files), len(mgr.workingTasks))
// column-based case, since the importServiceFunc return success, tasks will be sent to working list
mgr = newImportManager(context.TODO(), mockKv, idAlloc, importServiceFunc, callGetSegmentStates, nil, nil)
resp = mgr.importJob(context.TODO(), colReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, 1, len(mgr.workingTasks))
@ -630,9 +634,11 @@ func TestImportManager_ImportJob(t *testing.T) {
// the first task is sent to working list, and 1 task left in pending list
mgr = newImportManager(context.TODO(), mockKv, idAlloc, importServiceFunc, callGetSegmentStates, nil, nil)
resp = mgr.importJob(context.TODO(), rowReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, 1, len(mgr.workingTasks))
resp = mgr.importJob(context.TODO(), rowReq, colID, 0)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, 1, len(mgr.pendingTasks))
assert.Equal(t, 1, len(mgr.workingTasks))

View File

@ -1107,21 +1107,18 @@ func newMockStepExecutor() *mockStepExecutor {
func (m mockStepExecutor) Start() {
if m.StartFunc != nil {
m.StartFunc()
} else {
}
}
func (m mockStepExecutor) Stop() {
if m.StopFunc != nil {
m.StopFunc()
} else {
}
}
func (m mockStepExecutor) AddSteps(s *stepStack) {
if m.AddStepsFunc != nil {
m.AddStepsFunc(s)
} else {
}
}

View File

@ -1101,7 +1101,7 @@ func TestIndexFileBinlog(t *testing.T) {
//descriptor data fix, field id
fID := UnsafeReadInt64(buf, pos)
assert.Equal(t, fieldID, fieldID)
assert.Equal(t, fieldID, fID)
pos += int(unsafe.Sizeof(fID))
//descriptor data fix, start time stamp
@ -1230,7 +1230,7 @@ func TestIndexFileBinlogV2(t *testing.T) {
//descriptor data fix, field id
fID := UnsafeReadInt64(buf, pos)
assert.Equal(t, fieldID, fieldID)
assert.Equal(t, fieldID, fID)
pos += int(unsafe.Sizeof(fID))
//descriptor data fix, start time stamp

View File

@ -465,7 +465,7 @@ func TestVectorChunkManager_Read(t *testing.T) {
r, err = vcm.Mmap(ctx, "not exist")
assert.Error(t, err)
assert.Nil(t, nil)
assert.Nil(t, r)
}
content, err = vcm.ReadAt(ctx, "109", 9999, 8*4)

View File

@ -322,6 +322,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
return errors.New("import task was canceled")
}
// nolint
// this break means we require the first node must be RowRootNode
// once the RowRootNode is parsed, just finish
break

View File

@ -28,6 +28,7 @@ func TestExporterV2(t *testing.T) {
ts.assertMessagesContains("traceID=mock-trace")
ts.CleanBuffer()
// nolint
Ctx(nil).Info("empty context")
ts.assertMessagesNotContains("traceID")

View File

@ -860,7 +860,6 @@ func (ms *MqTtMsgStream) Seek(ctx context.Context, msgPositions []*msgpb.MsgPosi
}
if tsMsg.Type() == commonpb.MsgType_TimeTick && tsMsg.BeginTs() >= mp.Timestamp {
runLoop = false
break
} else if tsMsg.BeginTs() > mp.Timestamp {
ctx, _ := ExtractCtx(tsMsg, msg.Properties())
tsMsg.SetTraceCtx(ctx)

View File

@ -353,12 +353,12 @@ func TestGetIfPresentExpired(t *testing.T) {
c := NewCache(WithExpireAfterWrite[int, string](1*time.Second), WithInsertionListener(insFunc))
defer c.Close()
v, ok := c.GetIfPresent(0)
_, ok := c.GetIfPresent(0)
assert.False(t, ok)
wg.Add(1)
c.Put(0, "0")
v, ok = c.GetIfPresent(0)
v, ok := c.GetIfPresent(0)
assert.True(t, ok)
assert.Equal(t, "0", v)

View File

@ -42,27 +42,26 @@ func TestEtcd(t *testing.T) {
assert.False(t, resp.Count < 1)
assert.Equal(t, string(resp.Kvs[0].Value), "value")
etcdCli, err = GetEtcdClient(false, true, []string{},
_, err = GetEtcdClient(false, true, []string{},
"../../../configs/cert/client.pem",
"../../../configs/cert/client.key",
"../../../configs/cert/ca.pem",
"some not right word")
assert.Error(t, err)
etcdCli, err = GetEtcdClient(false, true, []string{},
_, err = GetEtcdClient(false, true, []string{},
"../../../configs/cert/client.pem",
"../../../configs/cert/client.key",
"wrong/file",
"1.2")
assert.Error(t, err)
etcdCli, err = GetEtcdClient(false, true, []string{},
_, err = GetEtcdClient(false, true, []string{},
"wrong/file",
"../../../configs/cert/client.key",
"../../../configs/cert/ca.pem",
"1.2")
assert.Error(t, err)
}
func Test_buildKvGroup(t *testing.T) {