diff --git a/client/Makefile b/client/Makefile index 2916ed444a2..fd34ef157bf 100644 --- a/client/Makefile +++ b/client/Makefile @@ -31,9 +31,12 @@ install-tools: cd .. && $(MAKE) install-tools static: install-tools + @ echo "gofmt ..." @ gofmt -s -l -d . 2>&1 | awk '{ print } END { if (NR > 0) { exit 1 } }' - @ golangci-lint run -c ../.golangci.yml ./... - @ revive -formatter friendly -config ../revive.toml . + @ echo "golangci-lint ..." + @ golangci-lint run -c ../.golangci.yml --verbose ./... + @ echo "revive ..." + @ revive -formatter friendly -config ../revive.toml ./... tidy: @ go mod tidy diff --git a/client/errs/errno.go b/client/errs/errno.go index 98fe9451d1e..646af81929d 100644 --- a/client/errs/errno.go +++ b/client/errs/errno.go @@ -32,7 +32,8 @@ const ( // NotServedErr indicates an tso node/pod received the requests for the keyspace groups which are not served by it. // Note: keep the same as the ones defined on the server side, because the client side checks if an error message // contains this string to judge whether the leader is changed. - NotServedErr = "is not served" + NotServedErr = "is not served" + // RetryTimeoutErr indicates the server is busy. RetryTimeoutErr = "retry timeout" ) @@ -91,6 +92,7 @@ var ( ErrClientResourceGroupThrottled = errors.Normalize("exceeded resource group quota limitation", errors.RFCCodeText("PD:client:ErrClientResourceGroupThrottled")) ) +// ErrClientGetResourceGroup is the error type for getting resource group. type ErrClientGetResourceGroup struct { ResourceGroupName string Cause string diff --git a/client/resource_group/controller/config.go b/client/resource_group/controller/config.go index cbf0be54c8d..2095bc60601 100644 --- a/client/resource_group/controller/config.go +++ b/client/resource_group/controller/config.go @@ -77,8 +77,8 @@ const ( defaultAvgBatchProportion = 0.7 ) -// ControllerConfig is the configuration of the resource manager controller which includes some option for client needed. -type ControllerConfig struct { +// Config is the configuration of the resource manager controller which includes some option for client needed. +type Config struct { // EnableDegradedMode is to control whether resource control client enable degraded mode when server is disconnect. DegradedModeWaitDuration string `toml:"degraded-mode-wait-duration" json:"degraded-mode-wait-duration"` @@ -87,9 +87,9 @@ type ControllerConfig struct { RequestUnit RequestUnitConfig `toml:"request-unit" json:"request-unit"` } -// DefaultControllerConfig returns the default resource manager controller configuration. -func DefaultControllerConfig() *ControllerConfig { - return &ControllerConfig{ +// DefaultConfig returns the default resource manager controller configuration. +func DefaultConfig() *Config { + return &Config{ DegradedModeWaitDuration: defaultDegradedModeWaitDuration, RequestUnit: DefaultRequestUnitConfig(), } @@ -130,10 +130,10 @@ func DefaultRequestUnitConfig() RequestUnitConfig { } } -// Config is the configuration of the resource units, which gives the read/write request +// RUConfig is the configuration of the resource units, which gives the read/write request // units or request resource cost standards. It should be calculated by a given `RequestUnitConfig` // or `RequestResourceConfig`. -type Config struct { +type RUConfig struct { // RU model config ReadBaseCost RequestUnit ReadPerBatchBaseCost RequestUnit @@ -148,16 +148,16 @@ type Config struct { DegradedModeWaitDuration time.Duration } -// DefaultConfig returns the default configuration. -func DefaultConfig() *Config { - return GenerateConfig( - DefaultControllerConfig(), +// DefaultRUConfig returns the default configuration. +func DefaultRUConfig() *RUConfig { + return GenerateRUConfig( + DefaultConfig(), ) } -// GenerateConfig generates the configuration by the given request unit configuration. -func GenerateConfig(config *ControllerConfig) *Config { - cfg := &Config{ +// GenerateRUConfig generates the configuration by the given request unit configuration. +func GenerateRUConfig(config *Config) *RUConfig { + cfg := &RUConfig{ ReadBaseCost: RequestUnit(config.RequestUnit.ReadBaseCost), ReadPerBatchBaseCost: RequestUnit(config.RequestUnit.ReadPerBatchBaseCost), ReadBytesCost: RequestUnit(config.RequestUnit.ReadCostPerByte), diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index cc4595f7e42..7eb985af1ba 100755 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -78,25 +78,25 @@ type ResourceControlCreateOption func(controller *ResourceGroupsController) // EnableSingleGroupByKeyspace is the option to enable single group by keyspace feature. func EnableSingleGroupByKeyspace() ResourceControlCreateOption { return func(controller *ResourceGroupsController) { - controller.config.isSingleGroupByKeyspace = true + controller.ruConfig.isSingleGroupByKeyspace = true } } // WithMaxWaitDuration is the option to set the max wait duration for acquiring token buckets. func WithMaxWaitDuration(d time.Duration) ResourceControlCreateOption { return func(controller *ResourceGroupsController) { - controller.config.maxWaitDuration = d + controller.ruConfig.maxWaitDuration = d } } var _ ResourceGroupKVInterceptor = (*ResourceGroupsController)(nil) -// ResourceGroupsController impls ResourceGroupKVInterceptor. +// ResourceGroupsController implements ResourceGroupKVInterceptor. type ResourceGroupsController struct { clientUniqueID uint64 provider ResourceGroupProvider groupsController sync.Map - config *Config + ruConfig *RUConfig loopCtx context.Context loopCancel func() @@ -128,19 +128,19 @@ func NewResourceGroupController( requestUnitConfig *RequestUnitConfig, opts ...ResourceControlCreateOption, ) (*ResourceGroupsController, error) { - controllerConfig, err := loadServerConfig(ctx, provider) + config, err := loadServerConfig(ctx, provider) if err != nil { return nil, err } if requestUnitConfig != nil { - controllerConfig.RequestUnit = *requestUnitConfig + config.RequestUnit = *requestUnitConfig } - log.Info("load resource controller config", zap.Reflect("config", controllerConfig)) - config := GenerateConfig(controllerConfig) + log.Info("load resource controller config", zap.Reflect("config", config)) + ruConfig := GenerateRUConfig(config) controller := &ResourceGroupsController{ clientUniqueID: clientUniqueID, provider: provider, - config: config, + ruConfig: ruConfig, lowTokenNotifyChan: make(chan struct{}, 1), tokenResponseChan: make(chan []*rmpb.TokenBucketResponse, 1), tokenBucketUpdateChan: make(chan *groupCostController, maxNotificationChanLen), @@ -148,30 +148,30 @@ func NewResourceGroupController( for _, opt := range opts { opt(controller) } - controller.calculators = []ResourceCalculator{newKVCalculator(controller.config), newSQLCalculator(controller.config)} + controller.calculators = []ResourceCalculator{newKVCalculator(controller.ruConfig), newSQLCalculator(controller.ruConfig)} return controller, nil } -func loadServerConfig(ctx context.Context, provider ResourceGroupProvider) (*ControllerConfig, error) { +func loadServerConfig(ctx context.Context, provider ResourceGroupProvider) (*Config, error) { items, _, err := provider.LoadGlobalConfig(ctx, nil, controllerConfigPath) if err != nil { return nil, err } if len(items) == 0 { log.Warn("[resource group controller] server does not save config, load config failed") - return DefaultControllerConfig(), nil + return DefaultConfig(), nil } - controllerConfig := &ControllerConfig{} - err = json.Unmarshal(items[0].PayLoad, controllerConfig) + config := &Config{} + err = json.Unmarshal(items[0].PayLoad, config) if err != nil { return nil, err } - return controllerConfig, nil + return config, nil } // GetConfig returns the config of controller. It's only used for test. -func (c *ResourceGroupsController) GetConfig() *Config { - return c.config +func (c *ResourceGroupsController) GetConfig() *RUConfig { + return c.ruConfig } // Source List @@ -184,8 +184,8 @@ const ( func (c *ResourceGroupsController) Start(ctx context.Context) { c.loopCtx, c.loopCancel = context.WithCancel(ctx) go func() { - if c.config.DegradedModeWaitDuration > 0 { - c.run.responseDeadline = time.NewTimer(c.config.DegradedModeWaitDuration) + if c.ruConfig.DegradedModeWaitDuration > 0 { + c.run.responseDeadline = time.NewTimer(c.ruConfig.DegradedModeWaitDuration) c.run.responseDeadline.Stop() defer c.run.responseDeadline.Stop() } @@ -214,11 +214,11 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { log.Warn("load resource group revision failed", zap.Error(err)) } var watchChannel chan []*meta_storagepb.Event - if !c.config.isSingleGroupByKeyspace { + if !c.ruConfig.isSingleGroupByKeyspace { watchChannel, err = c.provider.Watch(ctx, pd.GroupSettingsPathPrefixBytes, pd.WithRev(revision), pd.WithPrefix()) } watchRetryTimer := time.NewTimer(watchRetryInterval) - if err == nil || c.config.isSingleGroupByKeyspace { + if err == nil || c.ruConfig.isSingleGroupByKeyspace { watchRetryTimer.Stop() } defer watchRetryTimer.Stop() @@ -259,7 +259,7 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { c.executeOnAllGroups((*groupCostController).resetEmergencyTokenAcquisition) case resp, ok := <-watchChannel: failpoint.Inject("disableWatch", func() { - if c.config.isSingleGroupByKeyspace { + if c.ruConfig.isSingleGroupByKeyspace { panic("disableWatch") } }) @@ -335,7 +335,7 @@ func (c *ResourceGroupsController) tryGetResourceGroup(ctx context.Context, name return gc, nil } // Initialize the resource group controller. - gc, err := newGroupCostController(group, c.config, c.lowTokenNotifyChan, c.tokenBucketUpdateChan) + gc, err := newGroupCostController(group, c.ruConfig, c.lowTokenNotifyChan, c.tokenBucketUpdateChan) if err != nil { return nil, err } @@ -425,8 +425,8 @@ func (c *ResourceGroupsController) sendTokenBucketRequests(ctx context.Context, TargetRequestPeriodMs: uint64(defaultTargetPeriod / time.Millisecond), ClientUniqueId: c.clientUniqueID, } - if c.config.DegradedModeWaitDuration > 0 && c.responseDeadlineCh == nil { - c.run.responseDeadline.Reset(c.config.DegradedModeWaitDuration) + if c.ruConfig.DegradedModeWaitDuration > 0 && c.responseDeadlineCh == nil { + c.run.responseDeadline.Reset(c.ruConfig.DegradedModeWaitDuration) c.responseDeadlineCh = c.run.responseDeadline.C } go func() { @@ -485,7 +485,7 @@ type groupCostController struct { // invariant attributes name string mode rmpb.GroupMode - mainCfg *Config + mainCfg *RUConfig // meta info meta *rmpb.ResourceGroup metaLock sync.RWMutex @@ -574,7 +574,7 @@ type tokenCounter struct { func newGroupCostController( group *rmpb.ResourceGroup, - mainCfg *Config, + mainCfg *RUConfig, lowRUNotifyChan chan struct{}, tokenBucketUpdateChan chan *groupCostController, ) (*groupCostController, error) { @@ -1118,9 +1118,8 @@ func (gc *groupCostController) onRequestWait( sub(gc.mu.consumption, delta) gc.mu.Unlock() return nil, nil, err - } else { - gc.successfulRequestDuration.Observe(d.Seconds()) } + gc.successfulRequestDuration.Observe(d.Seconds()) } gc.mu.Lock() diff --git a/client/resource_group/controller/controller_test.go b/client/resource_group/controller/controller_test.go index d3b2a29c211..165d501ddb1 100644 --- a/client/resource_group/controller/controller_test.go +++ b/client/resource_group/controller/controller_test.go @@ -42,7 +42,7 @@ func createTestGroupCostController(re *require.Assertions) *groupCostController } ch1 := make(chan struct{}) ch2 := make(chan *groupCostController) - gc, err := newGroupCostController(group, DefaultConfig(), ch1, ch2) + gc, err := newGroupCostController(group, DefaultRUConfig(), ch1, ch2) re.NoError(err) return gc } diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index a8038c6d8e5..f89ab17514c 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -285,6 +285,7 @@ type tokenBucketReconfigureArgs struct { NotifyThreshold float64 } +// LimiterOption configures Limiter. type LimiterOption func(*Limiter) func resetLowProcess() func(*Limiter) { @@ -386,6 +387,7 @@ func (lim *Limiter) reserveN(now time.Time, n float64, maxFutureReserve time.Dur return r } +// ResetRemainingNotifyTimes resets the remaining notify times to 3. func (lim *Limiter) ResetRemainingNotifyTimes() { lim.mu.Lock() defer lim.mu.Unlock() diff --git a/client/resource_group/controller/model.go b/client/resource_group/controller/model.go index 55a6f9ec939..81f7e52716e 100644 --- a/client/resource_group/controller/model.go +++ b/client/resource_group/controller/model.go @@ -65,13 +65,13 @@ type ResourceCalculator interface { // KVCalculator is used to calculate the KV-side consumption. type KVCalculator struct { - *Config + *RUConfig } var _ ResourceCalculator = (*KVCalculator)(nil) -func newKVCalculator(cfg *Config) *KVCalculator { - return &KVCalculator{Config: cfg} +func newKVCalculator(cfg *RUConfig) *KVCalculator { + return &KVCalculator{RUConfig: cfg} } // Trickle ... @@ -146,13 +146,13 @@ func (kc *KVCalculator) payBackWriteCost(consumption *rmpb.Consumption, req Requ // SQLCalculator is used to calculate the SQL-side consumption. type SQLCalculator struct { - *Config + *RUConfig } var _ ResourceCalculator = (*SQLCalculator)(nil) -func newSQLCalculator(cfg *Config) *SQLCalculator { - return &SQLCalculator{Config: cfg} +func newSQLCalculator(cfg *RUConfig) *SQLCalculator { + return &SQLCalculator{RUConfig: cfg} } // Trickle update sql layer CPU consumption. diff --git a/client/resource_group/controller/testutil.go b/client/resource_group/controller/testutil.go index 81dc8a154b0..4df8c9bba0d 100644 --- a/client/resource_group/controller/testutil.go +++ b/client/resource_group/controller/testutil.go @@ -51,7 +51,7 @@ func (tri *TestRequestInfo) StoreID() uint64 { return tri.storeID } -// ReplicaNums implements the RequestInfo interface. +// ReplicaNumber implements the RequestInfo interface. func (tri *TestRequestInfo) ReplicaNumber() int64 { return 1 } @@ -63,6 +63,7 @@ type TestResponseInfo struct { succeed bool } +// NewTestResponseInfo creates a new TestResponseInfo. func NewTestResponseInfo(readBytes uint64, kvCPU time.Duration, succeed bool) *TestResponseInfo { return &TestResponseInfo{ readBytes: readBytes, diff --git a/tests/integrations/client/Makefile b/tests/integrations/client/Makefile index 71f6297270c..2d0cf748599 100644 --- a/tests/integrations/client/Makefile +++ b/tests/integrations/client/Makefile @@ -18,9 +18,12 @@ PATH := $(GO_TOOLS_BIN_PATH):$(PATH) SHELL := env PATH='$(PATH)' GOBIN='$(GO_TOOLS_BIN_PATH)' $(shell which bash) static: install-tools + @ echo "gofmt ..." @ gofmt -s -l -d . 2>&1 | awk '{ print } END { if (NR > 0) { exit 1 } }' - @ golangci-lint run ./... - @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml . + @ echo "golangci-lint ..." + @ golangci-lint run -c $(ROOT_PATH)/.golangci.yml --verbose ./... + @ echo "revive ..." + @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml ./... tidy: @ go mod tidy diff --git a/tests/integrations/mcs/Makefile b/tests/integrations/mcs/Makefile index 2628ea9e437..11862fc9e6c 100644 --- a/tests/integrations/mcs/Makefile +++ b/tests/integrations/mcs/Makefile @@ -18,9 +18,12 @@ PATH := $(GO_TOOLS_BIN_PATH):$(PATH) SHELL := env PATH='$(PATH)' GOBIN='$(GO_TOOLS_BIN_PATH)' $(shell which bash) static: install-tools + @ echo "gofmt ..." @ gofmt -s -l -d . 2>&1 | awk '{ print } END { if (NR > 0) { exit 1 } }' - @ golangci-lint run ./... - @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml . + @ echo "golangci-lint ..." + @ golangci-lint run -c $(ROOT_PATH)/.golangci.yml --verbose ./... + @ echo "revive ..." + @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml ./... tidy: @ go mod tidy diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 4bdefefbb81..d393dfe03ad 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -1081,7 +1081,7 @@ func (suite *resourceManagerClientTestSuite) TestLoadRequestUnitConfig() { re.NoError(err) config := ctr.GetConfig() re.NotNil(config) - expectedConfig := controller.DefaultConfig() + expectedConfig := controller.DefaultRUConfig() re.Equal(expectedConfig.ReadBaseCost, config.ReadBaseCost) re.Equal(expectedConfig.ReadBytesCost, config.ReadBytesCost) re.Equal(expectedConfig.WriteBaseCost, config.WriteBaseCost) @@ -1099,9 +1099,9 @@ func (suite *resourceManagerClientTestSuite) TestLoadRequestUnitConfig() { re.NoError(err) config = ctr.GetConfig() re.NotNil(config) - controllerConfig := controller.DefaultControllerConfig() + controllerConfig := controller.DefaultConfig() controllerConfig.RequestUnit = *ruConfig - expectedConfig = controller.GenerateConfig(controllerConfig) + expectedConfig = controller.GenerateRUConfig(controllerConfig) re.Equal(expectedConfig.ReadBaseCost, config.ReadBaseCost) re.Equal(expectedConfig.ReadBytesCost, config.ReadBytesCost) re.Equal(expectedConfig.WriteBaseCost, config.WriteBaseCost) diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index 45646dfa48e..625a702ad39 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -79,7 +79,7 @@ func (s *tsoProxyTestSuite) SetupSuite() { } // Create some TSO client streams with different context. - s.streams, s.cleanupFuncs = createTSOStreams(re, s.ctx, s.backendEndpoints, 200) + s.streams, s.cleanupFuncs = createTSOStreams(s.ctx, re, s.backendEndpoints, 200) } func (s *tsoProxyTestSuite) TearDownSuite() { @@ -107,7 +107,7 @@ func (s *tsoProxyTestSuite) TestTSOProxyWorksWithCancellation() { go func() { defer wg.Done() for i := 0; i < 3; i++ { - streams, cleanupFuncs := createTSOStreams(re, s.ctx, s.backendEndpoints, 10) + streams, cleanupFuncs := createTSOStreams(s.ctx, re, s.backendEndpoints, 10) for j := 0; j < 10; j++ { s.verifyTSOProxy(s.ctx, streams, cleanupFuncs, 10, true) } @@ -148,7 +148,7 @@ func TestTSOProxyStress(t *testing.T) { log.Info("start a new round of stress test", zap.Int("round-id", i), zap.Int("clients-count", len(streams)+clientsIncr)) streamsTemp, cleanupFuncsTemp := - createTSOStreams(re, s.ctx, s.backendEndpoints, clientsIncr) + createTSOStreams(s.ctx, re, s.backendEndpoints, clientsIncr) streams = append(streams, streamsTemp...) cleanupFuncs = append(cleanupFuncs, cleanupFuncsTemp...) s.verifyTSOProxy(ctxTimeout, streams, cleanupFuncs, 50, false) @@ -201,7 +201,7 @@ func (s *tsoProxyTestSuite) TestTSOProxyRecvFromClientTimeout() { // Enable the failpoint to make the TSO Proxy's grpc stream timeout on the server side to be 1 second. re.NoError(failpoint.Enable("github.com/tikv/pd/server/tsoProxyRecvFromClientTimeout", `return(1)`)) - streams, cleanupFuncs := createTSOStreams(re, s.ctx, s.backendEndpoints, 1) + streams, cleanupFuncs := createTSOStreams(s.ctx, re, s.backendEndpoints, 1) // Sleep 2 seconds to make the TSO Proxy's grpc stream timeout on the server side. time.Sleep(2 * time.Second) err := streams[0].Send(s.defaultReq) @@ -220,7 +220,7 @@ func (s *tsoProxyTestSuite) TestTSOProxyFailToSendToClient() { // Enable the failpoint to make the TSO Proxy's grpc stream timeout on the server side to be 1 second. re.NoError(failpoint.Enable("github.com/tikv/pd/server/tsoProxyFailToSendToClient", `return(true)`)) - streams, cleanupFuncs := createTSOStreams(re, s.ctx, s.backendEndpoints, 1) + streams, cleanupFuncs := createTSOStreams(s.ctx, re, s.backendEndpoints, 1) err := streams[0].Send(s.defaultReq) re.NoError(err) _, err = streams[0].Recv() @@ -238,7 +238,7 @@ func (s *tsoProxyTestSuite) TestTSOProxySendToTSOTimeout() { // Enable the failpoint to make the TSO Proxy's grpc stream timeout on the server side to be 1 second. re.NoError(failpoint.Enable("github.com/tikv/pd/server/tsoProxySendToTSOTimeout", `return(true)`)) - streams, cleanupFuncs := createTSOStreams(re, s.ctx, s.backendEndpoints, 1) + streams, cleanupFuncs := createTSOStreams(s.ctx, re, s.backendEndpoints, 1) err := streams[0].Send(s.defaultReq) re.NoError(err) _, err = streams[0].Recv() @@ -256,7 +256,7 @@ func (s *tsoProxyTestSuite) TestTSOProxyRecvFromTSOTimeout() { // Enable the failpoint to make the TSO Proxy's grpc stream timeout on the server side to be 1 second. re.NoError(failpoint.Enable("github.com/tikv/pd/server/tsoProxyRecvFromTSOTimeout", `return(true)`)) - streams, cleanupFuncs := createTSOStreams(re, s.ctx, s.backendEndpoints, 1) + streams, cleanupFuncs := createTSOStreams(s.ctx, re, s.backendEndpoints, 1) err := streams[0].Send(s.defaultReq) re.NoError(err) _, err = streams[0].Recv() @@ -369,7 +369,7 @@ func (s *tsoProxyTestSuite) generateRequests(requestsPerClient int) []*pdpb.TsoR // createTSOStreams creates multiple TSO client streams, and each stream uses a different gRPC connection // to simulate multiple clients. func createTSOStreams( - re *require.Assertions, ctx context.Context, + ctx context.Context, re *require.Assertions, backendEndpoints string, clientCount int, ) ([]pdpb.PD_TsoClient, []testutil.CleanupFunc) { cleanupFuncs := make([]testutil.CleanupFunc, clientCount) @@ -475,7 +475,7 @@ func benchmarkTSOProxyNClients(clientCount int, b *testing.B) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - streams, cleanupFuncs := createTSOStreams(re, ctx, suite.backendEndpoints, clientCount) + streams, cleanupFuncs := createTSOStreams(ctx, re, suite.backendEndpoints, clientCount) // Benchmark TSO proxy b.ResetTimer() diff --git a/tests/integrations/tso/Makefile b/tests/integrations/tso/Makefile index 38f5cfa164a..25896ca50e4 100644 --- a/tests/integrations/tso/Makefile +++ b/tests/integrations/tso/Makefile @@ -18,9 +18,12 @@ PATH := $(GO_TOOLS_BIN_PATH):$(PATH) SHELL := env PATH='$(PATH)' GOBIN='$(GO_TOOLS_BIN_PATH)' $(shell which bash) static: install-tools + @ echo "gofmt ..." @ gofmt -s -l -d . 2>&1 | awk '{ print } END { if (NR > 0) { exit 1 } }' - @ golangci-lint run ./... - @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml . + @ echo "golangci-lint ..." + @ golangci-lint run -c $(ROOT_PATH)/.golangci.yml --verbose ./... + @ echo "revive ..." + @ revive -formatter friendly -config $(ROOT_PATH)/revive.toml ./... tidy: @ go mod tidy