lint: update config of deprecated linter (#40763)

ref pingcap/tidb#40786
This commit is contained in:
Weizhen Wang
2023-01-28 11:57:53 +08:00
committed by GitHub
parent 1e0956d5ba
commit 76153ab9ff
39 changed files with 74 additions and 40 deletions

View File

@ -556,7 +556,7 @@ func BuildBackupRangeAndSchema(
continue
}
logger := log.With(
logger := log.L().With(
zap.String("db", dbInfo.Name.O),
zap.String("table", tableInfo.Name.O),
)

View File

@ -117,7 +117,7 @@ func (ss *Schemas) BackupSchemas(
}
workerPool.ApplyOnErrorGroup(errg, func() error {
if schema.tableInfo != nil {
logger := log.With(
logger := log.L().With(
zap.String("db", schema.dbInfo.Name.O),
zap.String("table", schema.tableInfo.Name.O),
)

View File

@ -69,6 +69,7 @@ go_library(
"@org_golang_google_grpc//backoff",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//keepalive",
"@org_golang_google_grpc//status",
"@org_golang_x_exp//slices",

View File

@ -75,6 +75,7 @@ import (
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/status"
)
@ -148,7 +149,7 @@ func (f *importClientFactoryImpl) makeConn(ctx context.Context, storeID uint64)
if err != nil {
return nil, errors.Trace(err)
}
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
if f.tls.TLSConfig() != nil {
opt = grpc.WithTransportCredentials(credentials.NewTLS(f.tls.TLSConfig()))
}

View File

@ -33,6 +33,7 @@ go_library(
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//status",
"@org_golang_x_sys//unix",
"@org_uber_go_zap//:zap",

View File

@ -28,6 +28,7 @@ import (
pd "github.com/tikv/pd/client"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
type TLS struct {
@ -104,7 +105,7 @@ func (tc *TLS) ToGRPCDialOption() grpc.DialOption {
if tc.inner != nil {
return grpc.WithTransportCredentials(credentials.NewTLS(tc.inner))
}
return grpc.WithInsecure()
return grpc.WithTransportCredentials(insecure.NewCredentials())
}
// WrapListener places a TLS layer on top of the existing listener.

View File

@ -51,5 +51,5 @@ func (r *RateTracer) RateAt(instant time.Time) float64 {
// L make a logger with the current speed.
func (r *RateTracer) L() *zap.Logger {
return log.With(zap.String("speed", fmt.Sprintf("%.2f ops/s", r.Rate())))
return log.L().With(zap.String("speed", fmt.Sprintf("%.2f ops/s", r.Rate())))
}

View File

@ -88,6 +88,7 @@ go_library(
"@org_golang_google_grpc//backoff",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//keepalive",
"@org_golang_google_grpc//status",
"@org_golang_x_exp//slices",

View File

@ -64,6 +64,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
)
@ -1298,7 +1299,7 @@ func (rc *Client) switchTiKVMode(ctx context.Context, mode import_sstpb.SwitchMo
finalStore := store
rc.workerPool.ApplyOnErrorGroup(eg,
func() error {
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
if rc.tlsConf != nil {
opt = grpc.WithTransportCredentials(credentials.NewTLS(rc.tlsConf))
}
@ -1410,7 +1411,7 @@ func (rc *Client) execChecksum(
concurrency uint,
loadStatCh chan<- *CreatedTable,
) error {
logger := log.With(
logger := log.L().With(
zap.String("db", tbl.OldTable.DB.Name.O),
zap.String("table", tbl.OldTable.Info.Name.O),
)

View File

@ -38,6 +38,7 @@ import (
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/status"
)
@ -197,7 +198,7 @@ func (ic *importClient) GetImportClient(
if err != nil {
return nil, errors.Trace(err)
}
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
if ic.tlsConf != nil {
opt = grpc.WithTransportCredentials(credentials.NewTLS(ic.tlsConf))
}

View File

@ -32,6 +32,7 @@ go_library(
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//status",
"@org_uber_go_multierr//:multierr",
"@org_uber_go_zap//:zap",

View File

@ -35,6 +35,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/status"
)
@ -200,7 +201,7 @@ func (c *pdClient) SplitRegion(ctx context.Context, regionInfo *RegionInfo, key
if err != nil {
return nil, errors.Trace(err)
}
conn, err := grpc.Dial(store.GetAddress(), grpc.WithInsecure())
conn, err := grpc.Dial(store.GetAddress(), grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
return nil, errors.Trace(err)
}
@ -336,7 +337,7 @@ func sendSplitRegionRequest(ctx context.Context, c *pdClient, regionInfo *Region
if err != nil {
return false, nil, err
}
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
if c.tlsConf != nil {
opt = grpc.WithTransportCredentials(credentials.NewTLS(c.tlsConf))
}

View File

@ -51,6 +51,7 @@ go_library(
"@org_golang_google_grpc//backoff",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//keepalive",
"@org_golang_google_grpc//status",
"@org_golang_x_net//http/httpproxy",

View File

@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/parser/types"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
// IsTypeCompatible checks whether type target is compatible with type src
@ -84,7 +85,7 @@ func IsTypeCompatible(src types.FieldType, target types.FieldType) bool {
}
func GRPCConn(ctx context.Context, storeAddr string, tlsConf *tls.Config, opts ...grpc.DialOption) (*grpc.ClientConn, error) {
secureOpt := grpc.WithInsecure()
secureOpt := grpc.WithTransportCredentials(insecure.NewCredentials())
if tlsConf != nil {
secureOpt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConf))
}

View File

@ -20,6 +20,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
)
@ -136,7 +137,7 @@ func (mgr *StoreManager) getGrpcConnLocked(ctx context.Context, storeID uint64)
if err != nil {
return nil, errors.Trace(err)
}
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
if mgr.tlsConf != nil {
opt = grpc.WithTransportCredentials(credentials.NewTLS(mgr.tlsConf))
}

View File

@ -750,26 +750,27 @@
"/build/": "no need to linter code",
"/external/": "no need to vet third party code",
".*_generated\\.go$": "ignore generated code",
".*_test\\.go$": "ignore test code"
".*_test\\.go$": "ignore test code",
"br/pkg/restore/split/client.go": "github.com/golang/protobuf deprecated",
"br/pkg/streamhelper/advancer_cliext.go": "github.com/golang/protobuf deprecated",
"br/pkg/lightning/checkpoints/checkpoints.go": "cfg.TikvImporter.Addr is deprecated",
"br/pkg/lightning/checkpoints/glue_checkpoint.go": "cfg.TikvImporter.Addr is deprecated"
},
"only_files": {
"util/gctuner": "util/gctuner",
"util/cgroup": "util/cgroup code",
"util/watcher": "util/watcher",
"br/pkg/lightning/restore/": "br/pkg/lightning/restore/",
"br/pkg/lightning/mydump/": "br/pkg/lightning/mydump/",
"br/pkg/": "br/pkg",
"executor/aggregate.go": "executor/aggregate.go",
"types/json_binary_functions.go": "types/json_binary_functions.go",
"types/json_binary_test.go": "types/json_binary_test.go",
"types/": "types",
"ddl/": "enable to ddl",
"expression/builtin_cast.go": "enable expression/builtin_cast.go",
"planner/core/plan.go": "planner/core/plan.go",
"server/conn.go": "server/conn.go",
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
"extension/": "extension code",
"resourcemanager/": "resourcemanager code",
"keyspace/": "keyspace code"
"keyspace/": "keyspace code",
"server/": "server code",
"meta": "meta code"
}
},
"SA2000": {

View File

@ -239,6 +239,7 @@ go_library(
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//status",
"@org_golang_x_exp//slices",
"@org_golang_x_sync//errgroup",

View File

@ -52,7 +52,7 @@ func (k loadStatsVarKeyType) String() string {
const LoadStatsVarKey loadStatsVarKeyType = 0
// Next implements the Executor Next interface.
func (e *LoadStatsExec) Next(ctx context.Context, req *chunk.Chunk) error {
func (e *LoadStatsExec) Next(_ context.Context, req *chunk.Chunk) error {
req.GrowAndReset(e.maxChunkSize)
if len(e.info.Path) == 0 {
return errors.New("Load Stats: file path is empty")
@ -72,7 +72,7 @@ func (e *LoadStatsExec) Close() error {
}
// Open implements the Executor Open interface.
func (e *LoadStatsExec) Open(ctx context.Context) error {
func (e *LoadStatsExec) Open(_ context.Context) error {
return nil
}

View File

@ -44,7 +44,7 @@ func (k lockStatsVarKeyType) String() string {
const LockStatsVarKey lockStatsVarKeyType = 0
// Next implements the Executor Next interface.
func (e *LockStatsExec) Next(ctx context.Context, req *chunk.Chunk) error {
func (e *LockStatsExec) Next(_ context.Context, _ *chunk.Chunk) error {
do := domain.GetDomain(e.ctx)
is := do.InfoSchema()
h := do.StatsHandle()
@ -87,7 +87,7 @@ func (e *LockStatsExec) Close() error {
}
// Open implements the Executor Open interface.
func (e *LockStatsExec) Open(ctx context.Context) error {
func (e *LockStatsExec) Open(_ context.Context) error {
return nil
}
@ -109,7 +109,7 @@ func (k unlockStatsVarKeyType) String() string {
const UnlockStatsVarKey unlockStatsVarKeyType = 0
// Next implements the Executor Next interface.
func (e *UnlockStatsExec) Next(ctx context.Context, req *chunk.Chunk) error {
func (e *UnlockStatsExec) Next(_ context.Context, _ *chunk.Chunk) error {
do := domain.GetDomain(e.ctx)
is := do.InfoSchema()
h := do.StatsHandle()
@ -152,6 +152,6 @@ func (e *UnlockStatsExec) Close() error {
}
// Open implements the Executor Open interface.
func (e *UnlockStatsExec) Open(ctx context.Context) error {
func (e *UnlockStatsExec) Open(_ context.Context) error {
return nil
}

View File

@ -49,6 +49,7 @@ import (
"golang.org/x/exp/slices"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
const clusterLogBatchSize = 256
@ -431,7 +432,7 @@ func (e *clusterLogRetriever) startRetrieving(
serversInfo []infoschema.ServerInfo,
req *diagnosticspb.SearchLogRequest) ([]chan logStreamResult, error) {
// gRPC options
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
security := config.GetGlobalConfig().Security
if len(security.ClusterSSLCA) != 0 {
clusterSecurity := security.ClusterSecurity()

View File

@ -56,6 +56,7 @@ go_library(
"@com_github_tikv_client_go_v2//tikv",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_x_exp//slices",
"@org_uber_go_zap//:zap",
],

View File

@ -58,6 +58,7 @@ import (
"golang.org/x/exp/slices"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
const (
@ -2333,7 +2334,7 @@ func serverInfoItemToRows(items []*diagnosticspb.ServerInfoItem, tp, addr string
}
func getServerInfoByGRPC(ctx context.Context, address string, tp diagnosticspb.ServerInfoType) ([]*diagnosticspb.ServerInfoItem, error) {
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
security := config.GetGlobalConfig().Security
if len(security.ClusterSSLCA) != 0 {
clusterSecurity := security.ClusterSecurity()

View File

@ -32,6 +32,7 @@ go_library(
"@io_etcd_go_etcd_client_v3//:client",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_uber_go_zap//:zap",
],
)

View File

@ -30,6 +30,7 @@ import (
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
var _ Allocator = &singlePointAlloc{}
@ -83,7 +84,7 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien
}
addr := string(resp.Kvs[0].Value)
opt := grpc.WithInsecure()
opt := grpc.WithTransportCredentials(insecure.NewCredentials())
security := config.GetGlobalConfig().Security
if len(security.ClusterSSLCA) != 0 {
clusterSecurity := security.ClusterSecurity()

View File

@ -276,7 +276,7 @@ func loadSQLMetaFile(z *zip.Reader) (uint64, error) {
}
//nolint: errcheck,all_revive
defer v.Close()
_, err = toml.DecodeReader(v, &varMap)
_, err = toml.NewDecoder(v).Decode(&varMap)
if err != nil {
return 0, errors.AddStack(err)
}

View File

@ -54,6 +54,7 @@ go_test(
"@com_github_pingcap_tipb//go-binlog",
"@com_github_stretchr_testify//require",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
"@org_uber_go_goleak//:goleak",
],
)

View File

@ -45,6 +45,7 @@ import (
"github.com/pingcap/tipb/go-binlog"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
type mockBinlogPump struct {
@ -100,7 +101,7 @@ func createBinlogSuite(t *testing.T) (s *binlogSuite) {
opt := grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) {
return net.DialTimeout("unix", addr, timeout)
})
clientCon, err := grpc.Dial(unixFile, opt, grpc.WithInsecure())
clientCon, err := grpc.Dial(unixFile, opt, grpc.WithTransportCredentials(insecure.NewCredentials()))
require.NoError(t, err)
require.NotNil(t, clientCon)

View File

@ -12,6 +12,7 @@ go_library(
"@com_github_pingcap_log//:log",
"@com_github_tikv_pd_client//:client",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
"@org_uber_go_zap//:zap",
],
)

View File

@ -29,6 +29,7 @@ import (
pd "github.com/tikv/pd/client"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
// Client is a PD (Placement Driver) client.
@ -231,7 +232,7 @@ func (c *client) getOrCreateConn(addr string) (*grpc.ClientConn, error) {
if err != nil {
return nil, err
}
cc, err := grpc.Dial(u.Host, grpc.WithInsecure())
cc, err := grpc.Dial(u.Host, grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
return nil, err
}

View File

@ -54,6 +54,7 @@ go_library(
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_pd_client//:client",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_x_exp//slices",
"@org_uber_go_zap//:zap",
],

View File

@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/store/mockstore/unistore/util/lockwaiter"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
// Follower will send detection rpc to Leader
@ -100,7 +101,7 @@ func (dt *DetectorClient) rebuildStreamClient() error {
if err != nil {
return err
}
cc, err := grpc.Dial(leaderAddr, grpc.WithInsecure())
cc, err := grpc.Dial(leaderAddr, grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
return err
}

View File

@ -21,6 +21,7 @@ go_library(
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//status",
"@org_uber_go_zap//:zap",
],
@ -41,5 +42,6 @@ go_test(
"@com_github_pingcap_tipb//go-binlog",
"@com_github_stretchr_testify//require",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
],
)

View File

@ -23,6 +23,7 @@ import (
pb "github.com/pingcap/tipb/go-binlog"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
func Benchmark100Thread(b *testing.B) {
@ -86,7 +87,7 @@ func createMockPumpsClientAndServer(b *testing.B) (*PumpsClient, *mockPumpServer
return net.DialTimeout("tcp", addr, timeout)
})
clientCon, err := grpc.Dial(addr, opt, grpc.WithInsecure())
clientCon, err := grpc.Dial(addr, opt, grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
b.Fatal(err)
}

View File

@ -27,6 +27,7 @@ import (
"github.com/pingcap/tipb/go-binlog"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
var (
@ -176,7 +177,7 @@ func TestWriteBinlog(t *testing.T) {
opt := grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) {
return net.DialTimeout(cfg.serverMode, addr, timeout)
})
clientCon, err := grpc.Dial(cfg.addr, opt, grpc.WithInsecure())
clientCon, err := grpc.Dial(cfg.addr, opt, grpc.WithTransportCredentials(insecure.NewCredentials()))
require.NoError(t, err)
require.NotNil(t, clientCon)
pumpClient := mockPumpsClient(binlog.NewPumpClient(clientCon), true)

View File

@ -29,6 +29,7 @@ import (
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
)
var (
@ -103,7 +104,7 @@ func (p *PumpStatus) createGrpcClient() error {
if p.security != nil {
clientConn, err = grpc.Dial(p.Addr, dialerOpt, grpc.WithTransportCredentials(credentials.NewTLS(p.security)))
} else {
clientConn, err = grpc.Dial(p.Addr, dialerOpt, grpc.WithInsecure())
clientConn, err = grpc.Dial(p.Addr, dialerOpt, grpc.WithTransportCredentials(insecure.NewCredentials()))
}
if err != nil {
atomic.AddInt64(&p.ErrNum, 1)

View File

@ -42,6 +42,7 @@ go_test(
"@com_github_pingcap_tipb//go-tipb",
"@com_github_stretchr_testify//require",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//keepalive",
"@org_uber_go_goleak//:goleak",
],

View File

@ -27,6 +27,7 @@ go_library(
"@com_github_wangjohn_quickselect//:quickselect",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//backoff",
"@org_golang_google_grpc//credentials/insecure",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_zap//:zap",
],

View File

@ -28,6 +28,7 @@ import (
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/credentials/insecure"
)
const (
@ -357,7 +358,7 @@ func (*SingleTargetDataSink) dial(ctx context.Context, targetRPCAddr string) (*g
dialCtx,
targetRPCAddr,
grpc.WithBlock(),
grpc.WithInsecure(),
grpc.WithTransportCredentials(insecure.NewCredentials()),
grpc.WithInitialWindowSize(grpcInitialWindowSize),
grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize),
grpc.WithDefaultCallOptions(

View File

@ -32,6 +32,7 @@ import (
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
)
@ -241,7 +242,7 @@ func TestTopSQLPubSub(t *testing.T) {
conn, err := grpc.Dial(
server.Address(),
grpc.WithBlock(),
grpc.WithInsecure(),
grpc.WithTransportCredentials(insecure.NewCredentials()),
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: 10 * time.Second,
Timeout: 3 * time.Second,
@ -363,7 +364,7 @@ func TestPubSubWhenReporterIsStopped(t *testing.T) {
conn, err := grpc.Dial(
server.Address(),
grpc.WithBlock(),
grpc.WithInsecure(),
grpc.WithTransportCredentials(insecure.NewCredentials()),
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: 10 * time.Second,
Timeout: 3 * time.Second,