From 0b757ce255ffc91dcf4f2a543b54005e52bb1325 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Tue, 10 Dec 2024 18:45:39 +0800 Subject: [PATCH 01/10] cluster: fix tso fallback due raft cluster did not stop tso service (part2) (#8885) ref tikv/pd#8477, close tikv/pd#8889 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- server/cluster/cluster.go | 30 ++++++++- server/server.go | 10 ++- tests/server/api/api_test.go | 2 +- tests/server/cluster/cluster_test.go | 99 ++++++++++++++++++++++++++-- 4 files changed, 131 insertions(+), 10 deletions(-) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 9b4630964b9..10e9bf7ff1a 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -315,7 +315,7 @@ func (c *RaftCluster) InitCluster( } // Start starts a cluster. -func (c *RaftCluster) Start(s Server) error { +func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { c.Lock() defer c.Unlock() @@ -324,11 +324,29 @@ func (c *RaftCluster) Start(s Server) error { return nil } c.isAPIServiceMode = s.IsAPIServiceMode() - err := c.InitCluster(s.GetAllocator(), s.GetPersistOptions(), s.GetHBStreams(), s.GetKeyspaceGroupManager()) + err = c.InitCluster(s.GetAllocator(), s.GetPersistOptions(), s.GetHBStreams(), s.GetKeyspaceGroupManager()) if err != nil { return err } - c.checkTSOService() + // We should not manage tso service when bootstrap try to start raft cluster. + // It only is controlled by leader election. + // Ref: https://github.com/tikv/pd/issues/8836 + if !bootstrap { + c.checkTSOService() + } + defer func() { + if !bootstrap && err != nil { + c.stopTSOJobsIfNeeded() + } + }() + failpoint.Inject("raftClusterReturn", func(val failpoint.Value) { + if val, ok := val.(bool); (ok && val) || !ok { + err = errors.New("raftClusterReturn") + } else { + err = nil + } + failpoint.Return(err) + }) cluster, err := c.LoadClusterInfo() if err != nil { return err @@ -2554,3 +2572,9 @@ func (c *RaftCluster) SetServiceIndependent(name string) { func (c *RaftCluster) UnsetServiceIndependent(name string) { c.independentServices.Delete(name) } + +// GetGlobalTSOAllocator return global tso allocator +// It only is used for test. +func (c *RaftCluster) GetGlobalTSOAllocator() tso.Allocator { + return c.tsoAllocator.GetAllocator() +} diff --git a/server/server.go b/server/server.go index d7bdd92d96d..3ed3c9514ff 100644 --- a/server/server.go +++ b/server/server.go @@ -758,7 +758,7 @@ func (s *Server) bootstrapCluster(req *pdpb.BootstrapRequest) (*pdpb.BootstrapRe log.Warn("flush the bootstrap region failed", errs.ZapError(err)) } - if err := s.cluster.Start(s); err != nil { + if err := s.cluster.Start(s, true); err != nil { return nil, err } @@ -776,7 +776,7 @@ func (s *Server) createRaftCluster() error { return nil } - return s.cluster.Start(s) + return s.cluster.Start(s, false) } func (s *Server) stopRaftCluster() { @@ -2097,3 +2097,9 @@ func (s *Server) GetMaxResetTSGap() time.Duration { func (s *Server) SetClient(client *clientv3.Client) { s.client = client } + +// GetGlobalTSOAllocator return global tso allocator +// It only is used for test. +func (s *Server) GetGlobalTSOAllocator() tso.Allocator { + return s.cluster.GetGlobalTSOAllocator() +} diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 14df5ff8eea..e1e4db3a26d 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -925,7 +925,7 @@ func TestSendApiWhenRestartRaftCluster(t *testing.T) { output := sendRequest(re, leader.GetAddr()+"/pd/api/v1/min-resolved-ts", http.MethodGet, http.StatusInternalServerError) re.Contains(string(output), "TiKV cluster not bootstrapped, please start TiKV first") - err = rc.Start(leader.GetServer()) + err = rc.Start(leader.GetServer(), false) re.NoError(err) rc = leader.GetRaftCluster() re.NotNil(rc) diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index df0cf7d38a3..dfdb9cb8685 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -578,7 +578,7 @@ func TestRaftClusterRestart(t *testing.T) { re.NotNil(rc) rc.Stop() - err = rc.Start(leaderServer.GetServer()) + err = rc.Start(leaderServer.GetServer(), false) re.NoError(err) rc = leaderServer.GetRaftCluster() @@ -621,7 +621,7 @@ func TestRaftClusterMultipleRestart(t *testing.T) { for range 100 { // See https://github.com/tikv/pd/issues/8543 rc.Wait() - err = rc.Start(leaderServer.GetServer()) + err = rc.Start(leaderServer.GetServer(), false) re.NoError(err) time.Sleep(time.Millisecond) rc.Stop() @@ -629,6 +629,97 @@ func TestRaftClusterMultipleRestart(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs")) } +// TestRaftClusterStartTSOJob is used to test whether tso job service is normally closed +// when raft cluster is stopped ahead of time. +// Ref: https://github.com/tikv/pd/issues/8836 +func TestRaftClusterStartTSOJob(t *testing.T) { + re := require.New(t) + name := "pd1" + // case 1: normal start + ctx, cancel := context.WithCancel(context.Background()) + tc, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + re.NoError(tc.RunInitialServers()) + re.NotEmpty(tc.WaitLeader()) + leaderServer := tc.GetLeaderServer() + re.NotNil(leaderServer) + leaderServer.BootstrapCluster() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + tc.Destroy() + cancel() + // case 2: return ahead of time but no error when start raft cluster + re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/raftClusterReturn", `return(false)`)) + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + tc.WaitLeader() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/raftClusterReturn")) + tc.Destroy() + cancel() + // case 3: meet error when start raft cluster + re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/raftClusterReturn", `return(true)`)) + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + tc.WaitLeader() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return !allocator.IsInitialize() + }) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/raftClusterReturn")) + tc.Destroy() + cancel() + // case 4: multiple bootstrap in 3 pd cluster + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 3, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + re.NoError(tc.RunInitialServers()) + re.NotEmpty(tc.WaitLeader()) + leaderServer = tc.GetLeaderServer() + re.NotNil(leaderServer) + name = leaderServer.GetLeader().GetName() + wg := sync.WaitGroup{} + for range 3 { + wg.Add(1) + go func() { + leaderServer.BootstrapCluster() + wg.Done() + }() + } + wg.Wait() + testutil.Eventually(re, func() bool { + allocator := leaderServer.GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + re.NoError(tc.ResignLeader()) + re.NotEmpty(tc.WaitLeader()) + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return !allocator.IsInitialize() + }) + tc.Destroy() + cancel() +} + func newMetaStore(storeID uint64, addr, version string, state metapb.StoreState, deployPath string) *metapb.Store { return &metapb.Store{Id: storeID, Address: addr, Version: version, State: state, DeployPath: deployPath} } @@ -1437,7 +1528,7 @@ func TestTransferLeaderForScheduler(t *testing.T) { tc.WaitLeader() leaderServer = tc.GetLeaderServer() rc1 := leaderServer.GetServer().GetRaftCluster() - rc1.Start(leaderServer.GetServer()) + rc1.Start(leaderServer.GetServer(), false) re.NoError(err) re.NotNil(rc1) // region heartbeat @@ -1457,7 +1548,7 @@ func TestTransferLeaderForScheduler(t *testing.T) { tc.WaitLeader() leaderServer = tc.GetLeaderServer() rc = leaderServer.GetServer().GetRaftCluster() - rc.Start(leaderServer.GetServer()) + rc.Start(leaderServer.GetServer(), false) re.NotNil(rc) // region heartbeat id = leaderServer.GetAllocator() From fef6424e4f14affbc20e45cf20b24196ef112aa7 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 11 Dec 2024 11:21:54 +0800 Subject: [PATCH 02/10] *: fix typo in license (#8884) ref tikv/pd#4399 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/resource_group/controller/config.go | 2 +- client/resource_group/controller/controller.go | 2 +- client/resource_group/controller/controller_test.go | 2 +- client/resource_group/controller/limiter.go | 2 +- client/resource_group/controller/limiter_test.go | 2 +- client/resource_group/controller/model.go | 2 +- client/resource_group/controller/testutil.go | 2 +- client/resource_group/controller/util.go | 2 +- pkg/mcs/resourcemanager/server/config_test.go | 2 +- pkg/mcs/resourcemanager/server/token_buckets.go | 2 +- pkg/mcs/resourcemanager/server/token_buckets_test.go | 2 +- pkg/window/counter.go | 2 +- pkg/window/counter_test.go | 2 +- pkg/window/policy.go | 2 +- pkg/window/policy_test.go | 2 +- pkg/window/reduce.go | 2 +- pkg/window/window.go | 2 +- pkg/window/window_test.go | 2 +- 18 files changed, 18 insertions(+), 18 deletions(-) diff --git a/client/resource_group/controller/config.go b/client/resource_group/controller/config.go index 96c783455bb..3008d7b6e77 100644 --- a/client/resource_group/controller/config.go +++ b/client/resource_group/controller/config.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index 83bd21b1eed..46401aad1ff 100644 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/controller_test.go b/client/resource_group/controller/controller_test.go index 882f99a6868..254df36020e 100644 --- a/client/resource_group/controller/controller_test.go +++ b/client/resource_group/controller/controller_test.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index 5d9823312ca..cd05adb7dfa 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/limiter_test.go b/client/resource_group/controller/limiter_test.go index 9afebcb3d53..24cdee0bbc3 100644 --- a/client/resource_group/controller/limiter_test.go +++ b/client/resource_group/controller/limiter_test.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/model.go b/client/resource_group/controller/model.go index 9e86de69abb..88c18dc25cf 100644 --- a/client/resource_group/controller/model.go +++ b/client/resource_group/controller/model.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/testutil.go b/client/resource_group/controller/testutil.go index 01a9c3af1fc..de71cff4d0b 100644 --- a/client/resource_group/controller/testutil.go +++ b/client/resource_group/controller/testutil.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/util.go b/client/resource_group/controller/util.go index e3450e0ae0d..3b491e02c8f 100644 --- a/client/resource_group/controller/util.go +++ b/client/resource_group/controller/util.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/config_test.go b/pkg/mcs/resourcemanager/server/config_test.go index ae9dfc2cad3..afe22356def 100644 --- a/pkg/mcs/resourcemanager/server/config_test.go +++ b/pkg/mcs/resourcemanager/server/config_test.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/token_buckets.go b/pkg/mcs/resourcemanager/server/token_buckets.go index e0777b419eb..f72d0dfcc88 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets.go +++ b/pkg/mcs/resourcemanager/server/token_buckets.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/token_buckets_test.go b/pkg/mcs/resourcemanager/server/token_buckets_test.go index 8ac3ec4a3ba..3d9cbd3f628 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets_test.go +++ b/pkg/mcs/resourcemanager/server/token_buckets_test.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/counter.go b/pkg/window/counter.go index 84325cdc14b..c56c202a414 100644 --- a/pkg/window/counter.go +++ b/pkg/window/counter.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/counter_test.go b/pkg/window/counter_test.go index dc43e4fee3f..2cb25babdba 100644 --- a/pkg/window/counter_test.go +++ b/pkg/window/counter_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/policy.go b/pkg/window/policy.go index fed4fedc32a..14e33e3ee74 100644 --- a/pkg/window/policy.go +++ b/pkg/window/policy.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/policy_test.go b/pkg/window/policy_test.go index f4ae9989e19..936360ccb2b 100644 --- a/pkg/window/policy_test.go +++ b/pkg/window/policy_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/reduce.go b/pkg/window/reduce.go index 23fa87177f2..0df21ff4c4f 100644 --- a/pkg/window/reduce.go +++ b/pkg/window/reduce.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/window.go b/pkg/window/window.go index 80fb5bb5714..6d7a54131ce 100644 --- a/pkg/window/window.go +++ b/pkg/window/window.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/window_test.go b/pkg/window/window_test.go index 59fb2ee0bbb..4cedd12ae01 100644 --- a/pkg/window/window_test.go +++ b/pkg/window/window_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. From fbfcdb8a4613bc1f05e0feabbc8325a90da0518a Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 11 Dec 2024 14:57:27 +0800 Subject: [PATCH 03/10] client: reduce redundant concepts in the client filenames (#8894) ref tikv/pd#8690 Reduce the redundant concepts in the client filenames. Signed-off-by: JmPotato --- client/clients/router/{router_client.go => client.go} | 0 client/clients/tso/{tso_client.go => client.go} | 0 client/clients/tso/{tso_dispatcher.go => dispatcher.go} | 0 client/clients/tso/{tso_dispatcher_test.go => dispatcher_test.go} | 0 client/clients/tso/{tso_request.go => request.go} | 0 client/clients/tso/{tso_request_test.go => request_test.go} | 0 client/clients/tso/{tso_stream.go => stream.go} | 0 client/clients/tso/{tso_stream_test.go => stream_test.go} | 0 8 files changed, 0 insertions(+), 0 deletions(-) rename client/clients/router/{router_client.go => client.go} (100%) rename client/clients/tso/{tso_client.go => client.go} (100%) rename client/clients/tso/{tso_dispatcher.go => dispatcher.go} (100%) rename client/clients/tso/{tso_dispatcher_test.go => dispatcher_test.go} (100%) rename client/clients/tso/{tso_request.go => request.go} (100%) rename client/clients/tso/{tso_request_test.go => request_test.go} (100%) rename client/clients/tso/{tso_stream.go => stream.go} (100%) rename client/clients/tso/{tso_stream_test.go => stream_test.go} (100%) diff --git a/client/clients/router/router_client.go b/client/clients/router/client.go similarity index 100% rename from client/clients/router/router_client.go rename to client/clients/router/client.go diff --git a/client/clients/tso/tso_client.go b/client/clients/tso/client.go similarity index 100% rename from client/clients/tso/tso_client.go rename to client/clients/tso/client.go diff --git a/client/clients/tso/tso_dispatcher.go b/client/clients/tso/dispatcher.go similarity index 100% rename from client/clients/tso/tso_dispatcher.go rename to client/clients/tso/dispatcher.go diff --git a/client/clients/tso/tso_dispatcher_test.go b/client/clients/tso/dispatcher_test.go similarity index 100% rename from client/clients/tso/tso_dispatcher_test.go rename to client/clients/tso/dispatcher_test.go diff --git a/client/clients/tso/tso_request.go b/client/clients/tso/request.go similarity index 100% rename from client/clients/tso/tso_request.go rename to client/clients/tso/request.go diff --git a/client/clients/tso/tso_request_test.go b/client/clients/tso/request_test.go similarity index 100% rename from client/clients/tso/tso_request_test.go rename to client/clients/tso/request_test.go diff --git a/client/clients/tso/tso_stream.go b/client/clients/tso/stream.go similarity index 100% rename from client/clients/tso/tso_stream.go rename to client/clients/tso/stream.go diff --git a/client/clients/tso/tso_stream_test.go b/client/clients/tso/stream_test.go similarity index 100% rename from client/clients/tso/tso_stream_test.go rename to client/clients/tso/stream_test.go From 5d62787565f7d238f386fd702e89a1921060f545 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Wed, 11 Dec 2024 16:17:27 +0800 Subject: [PATCH 04/10] tso/local: remove local tso completely (#8864) close tikv/pd#8802 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 16 ++--- client/clients/tso/client.go | 16 ++--- pkg/mcs/tso/server/config.go | 5 +- pkg/mcs/tso/server/config_test.go | 5 -- pkg/mcs/tso/server/server.go | 9 --- pkg/storage/endpoint/service_middleware.go | 4 +- pkg/storage/endpoint/tso.go | 6 +- pkg/tso/allocator_manager.go | 30 ---------- pkg/tso/global_allocator.go | 22 +------ pkg/tso/keyspace_group_manager.go | 6 +- pkg/tso/keyspace_group_manager_test.go | 1 - pkg/tso/testutil.go | 1 - pkg/tso/tso.go | 58 ++++--------------- pkg/utils/keypath/key_path.go | 14 ----- pkg/utils/tsoutil/tso_dispatcher.go | 14 ++--- pkg/utils/tsoutil/tso_request.go | 16 +++-- server/config/config.go | 7 +-- server/server.go | 5 -- tests/cluster.go | 5 -- .../mcs/tso/keyspace_group_manager_test.go | 2 - 20 files changed, 53 insertions(+), 189 deletions(-) diff --git a/client/client.go b/client/client.go index c271f10591d..519fd478bb3 100644 --- a/client/client.go +++ b/client/client.go @@ -509,10 +509,10 @@ func (c *client) GetTSAsync(ctx context.Context) tso.TSFuture { return c.inner.dispatchTSORequestWithRetry(ctx) } -// GetLocalTSAsync implements the TSOClient interface. -// -// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the -// parameters passed in, this method will default to returning the global TSO. +// Deprecated: the Local TSO feature has been deprecated. Regardless of the +// parameters passed, the behavior of this interface will be equivalent to +// `GetTSAsync`. If you want to use a separately deployed TSO service, +// please refer to the deployment of the TSO microservice. func (c *client) GetLocalTSAsync(ctx context.Context, _ string) tso.TSFuture { return c.GetTSAsync(ctx) } @@ -523,10 +523,10 @@ func (c *client) GetTS(ctx context.Context) (physical int64, logical int64, err return resp.Wait() } -// GetLocalTS implements the TSOClient interface. -// -// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the -// parameters passed in, this method will default to returning the global TSO. +// Deprecated: the Local TSO feature has been deprecated. Regardless of the +// parameters passed, the behavior of this interface will be equivalent to +// `GetTS`. If you want to use a separately deployed TSO service, +// please refer to the deployment of the TSO microservice. func (c *client) GetLocalTS(ctx context.Context, _ string) (physical int64, logical int64, err error) { return c.GetTS(ctx) } diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index 9c7075fe3bb..68e2163d191 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -56,15 +56,15 @@ type Client interface { // the TSO microservice. GetMinTS(ctx context.Context) (int64, int64, error) - // GetLocalTS gets a local timestamp from PD or TSO microservice. - // - // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the - // parameters passed in, this method will default to returning the global TSO. + // Deprecated: the Local TSO feature has been deprecated. Regardless of the + // parameters passed, the behavior of this interface will be equivalent to + // `GetTS`. If you want to use a separately deployed TSO service, + // please refer to the deployment of the TSO microservice. GetLocalTS(ctx context.Context, _ string) (int64, int64, error) - // GetLocalTSAsync gets a local timestamp from PD or TSO microservice, without block the caller. - // - // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the - // parameters passed in, this method will default to returning the global TSO. + // Deprecated: the Local TSO feature has been deprecated. Regardless of the + // parameters passed, the behavior of this interface will be equivalent to + // `GetTSAsync`. If you want to use a separately deployed TSO service, + // please refer to the deployment of the TSO microservice. GetLocalTSAsync(ctx context.Context, _ string) TSFuture } diff --git a/pkg/mcs/tso/server/config.go b/pkg/mcs/tso/server/config.go index 0973042b912..2aaa54114da 100644 --- a/pkg/mcs/tso/server/config.go +++ b/pkg/mcs/tso/server/config.go @@ -64,10 +64,7 @@ type Config struct { // the primary/leader again. Etcd only supports seconds TTL, so here is second too. LeaderLease int64 `toml:"lease" json:"lease"` - // EnableLocalTSO is used to enable the Local TSO Allocator feature, - // which allows the PD server to generate Local TSO for certain DC-level transactions. - // To make this feature meaningful, user has to set the "zone" label for the PD server - // to indicate which DC this PD belongs to. + // Deprecated EnableLocalTSO bool `toml:"enable-local-tso" json:"enable-local-tso"` // TSOSaveInterval is the interval to save timestamp. diff --git a/pkg/mcs/tso/server/config_test.go b/pkg/mcs/tso/server/config_test.go index 2bd27a67492..2bafec30aa9 100644 --- a/pkg/mcs/tso/server/config_test.go +++ b/pkg/mcs/tso/server/config_test.go @@ -36,7 +36,6 @@ func TestConfigBasic(t *testing.T) { re.Equal(defaultBackendEndpoints, cfg.BackendEndpoints) re.Equal(defaultListenAddr, cfg.ListenAddr) re.Equal(constant.DefaultLeaderLease, cfg.LeaderLease) - re.False(cfg.EnableLocalTSO) re.True(cfg.EnableGRPCGateway) re.Equal(defaultTSOSaveInterval, cfg.TSOSaveInterval.Duration) re.Equal(defaultTSOUpdatePhysicalInterval, cfg.TSOUpdatePhysicalInterval.Duration) @@ -48,7 +47,6 @@ func TestConfigBasic(t *testing.T) { cfg.ListenAddr = "test-listen-addr" cfg.AdvertiseListenAddr = "test-advertise-listen-addr" cfg.LeaderLease = 123 - cfg.EnableLocalTSO = true cfg.TSOSaveInterval.Duration = time.Duration(10) * time.Second cfg.TSOUpdatePhysicalInterval.Duration = time.Duration(100) * time.Millisecond cfg.MaxResetTSGap.Duration = time.Duration(1) * time.Hour @@ -58,7 +56,6 @@ func TestConfigBasic(t *testing.T) { re.Equal("test-listen-addr", cfg.GetListenAddr()) re.Equal("test-advertise-listen-addr", cfg.GetAdvertiseListenAddr()) re.Equal(int64(123), cfg.GetLeaderLease()) - re.True(cfg.EnableLocalTSO) re.Equal(time.Duration(10)*time.Second, cfg.TSOSaveInterval.Duration) re.Equal(time.Duration(100)*time.Millisecond, cfg.TSOUpdatePhysicalInterval.Duration) re.Equal(time.Duration(1)*time.Hour, cfg.MaxResetTSGap.Duration) @@ -74,7 +71,6 @@ name = "tso-test-name" data-dir = "/var/lib/tso" enable-grpc-gateway = false lease = 123 -enable-local-tso = true tso-save-interval = "10s" tso-update-physical-interval = "100ms" max-gap-reset-ts = "1h" @@ -92,7 +88,6 @@ max-gap-reset-ts = "1h" re.Equal("test-advertise-listen-addr", cfg.GetAdvertiseListenAddr()) re.Equal("/var/lib/tso", cfg.DataDir) re.Equal(int64(123), cfg.GetLeaderLease()) - re.True(cfg.EnableLocalTSO) re.Equal(time.Duration(10)*time.Second, cfg.TSOSaveInterval.Duration) re.Equal(time.Duration(100)*time.Millisecond, cfg.TSOUpdatePhysicalInterval.Duration) re.Equal(time.Duration(1)*time.Hour, cfg.MaxResetTSGap.Duration) diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 04e81c2d48e..d2974075e94 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -272,15 +272,6 @@ func (s *Server) GetTSOAllocatorManager(keyspaceGroupID uint32) (*tso.AllocatorM return s.keyspaceGroupManager.GetAllocatorManager(keyspaceGroupID) } -// IsLocalRequest checks if the forwarded host is the current host -func (*Server) IsLocalRequest(forwardedHost string) bool { - // TODO: Check if the forwarded host is the current host. - // The logic is depending on etcd service mode -- if the TSO service - // uses the embedded etcd, check against ClientUrls; otherwise check - // against the cluster membership. - return forwardedHost == "" -} - // ValidateInternalRequest checks if server is closed, which is used to validate // the gRPC communication between TSO servers internally. // TODO: Check if the sender is from the global TSO allocator diff --git a/pkg/storage/endpoint/service_middleware.go b/pkg/storage/endpoint/service_middleware.go index 3859dab4d62..35f0606f9d0 100644 --- a/pkg/storage/endpoint/service_middleware.go +++ b/pkg/storage/endpoint/service_middleware.go @@ -29,7 +29,7 @@ type ServiceMiddlewareStorage interface { var _ ServiceMiddlewareStorage = (*StorageEndpoint)(nil) -// LoadServiceMiddlewareConfig loads service middleware config from keypath.KeyspaceGroupLocalTSPath then unmarshal it to cfg. +// LoadServiceMiddlewareConfig loads service middleware config from ServiceMiddlewarePath then unmarshal it to cfg. func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) { value, err := se.Load(keypath.ServiceMiddlewarePath) if err != nil || value == "" { @@ -42,7 +42,7 @@ func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) { return true, nil } -// SaveServiceMiddlewareConfig stores marshallable cfg to the keypath.KeyspaceGroupLocalTSPath. +// SaveServiceMiddlewareConfig stores marshallable cfg to the ServiceMiddlewarePath. func (se *StorageEndpoint) SaveServiceMiddlewareConfig(cfg any) error { return se.saveJSON(keypath.ServiceMiddlewarePath, cfg) } diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index a656f6d2945..77841529e98 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -37,9 +37,9 @@ type TSOStorage interface { var _ TSOStorage = (*StorageEndpoint)(nil) -// LoadTimestamp will get all time windows of Local/Global TSOs from etcd and return the biggest one. -// For the Global TSO, loadTimestamp will get all Local and Global TSO time windows persisted in etcd and choose the biggest one. -// For the Local TSO, loadTimestamp will only get its own dc-location time window persisted before. +// LoadTimestamp will get all time windows of Global TSOs from etcd and return the biggest one. +// TODO: Due to local TSO is deprecated, maybe we do not need to load timestamp +// by prefix, we can just load the timestamp by the key. func (se *StorageEndpoint) LoadTimestamp(prefix string) (time.Time, error) { prefixEnd := clientv3.GetPrefixRangeEnd(prefix) keys, values, err := se.LoadRange(prefix, prefixEnd, 0) diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 8d5589143aa..65f61e819d1 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -16,8 +16,6 @@ package tso import ( "context" - "math" - "path" "runtime/trace" "strconv" "sync" @@ -43,8 +41,6 @@ const ( checkStep = time.Minute patrolStep = time.Second defaultAllocatorLeaderLease = 3 - localTSOAllocatorEtcdPrefix = "lta" - localTSOSuffixEtcdPrefix = "lts" ) var ( @@ -217,17 +213,6 @@ func (am *AllocatorManager) getGroupIDStr() string { return strconv.FormatUint(uint64(am.kgID), 10) } -// GetTimestampPath returns the timestamp path in etcd. -func (am *AllocatorManager) GetTimestampPath() string { - if am == nil { - return "" - } - - am.mu.RLock() - defer am.mu.RUnlock() - return path.Join(am.rootPath, am.mu.allocatorGroup.allocator.GetTimestampPath()) -} - // tsoAllocatorLoop is used to run the TSO Allocator updating daemon. func (am *AllocatorManager) tsoAllocatorLoop() { defer logutil.LogPanic() @@ -254,21 +239,6 @@ func (am *AllocatorManager) GetMember() ElectionMember { return am.member } -// GetSuffixBits calculates the bits of suffix sign -// by the max number of suffix so far, -// which will be used in the TSO logical part. -func (am *AllocatorManager) GetSuffixBits() int { - am.mu.RLock() - defer am.mu.RUnlock() - return CalSuffixBits(am.mu.maxSuffix) -} - -// CalSuffixBits calculates the bits of suffix by the max suffix sign. -func CalSuffixBits(maxSuffix int32) int { - // maxSuffix + 1 because we have the Global TSO holds 0 as the suffix sign - return int(math.Ceil(math.Log2(float64(maxSuffix + 1)))) -} - // AllocatorDaemon is used to update every allocator's TSO and check whether we have // any new local allocator that needs to be set up. func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 740317c676a..2fe0df3e000 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -47,14 +47,6 @@ type Allocator interface { IsInitialize() bool // UpdateTSO is used to update the TSO in memory and the time window in etcd. UpdateTSO() error - // GetTimestampPath returns the timestamp path in etcd, which is: - // 1. for the default keyspace group: - // a. timestamp in /pd/{cluster_id}/timestamp - // b. lta/{dc-location}/timestamp in /pd/{cluster_id}/lta/{dc-location}/timestamp - // 1. for the non-default keyspace groups: - // a. {group}/gts/timestamp in /ms/{cluster_id}/tso/{group}/gta/timestamp - // b. {group}/lts/{dc-location}/timestamp in /ms/{cluster_id}/tso/{group}/lta/{dc-location}/timestamp - GetTimestampPath() string // SetTSO sets the physical part with given TSO. It's mainly used for BR restore. // Cannot set the TSO smaller than now in any case. // if ignoreSmaller=true, if input ts is smaller than current, ignore silently, else return error @@ -68,6 +60,8 @@ type Allocator interface { } // GlobalTSOAllocator is the global single point TSO allocator. +// TODO: Local TSO allocator is deprecated now, we can update the name to +// TSOAllocator and remove the `Global` concept. type GlobalTSOAllocator struct { ctx context.Context cancel context.CancelFunc @@ -132,19 +126,9 @@ func (gta *GlobalTSOAllocator) getGroupID() uint32 { return gta.am.getGroupID() } -// GetTimestampPath returns the timestamp path in etcd. -func (gta *GlobalTSOAllocator) GetTimestampPath() string { - if gta == nil || gta.timestampOracle == nil { - return "" - } - return gta.timestampOracle.GetTimestampPath() -} - // Initialize will initialize the created global TSO allocator. func (gta *GlobalTSOAllocator) Initialize(int) error { gta.tsoAllocatorRoleGauge.Set(1) - // The suffix of a Global TSO should always be 0. - gta.timestampOracle.suffix = 0 return gta.timestampOracle.SyncTimestamp() } @@ -175,7 +159,7 @@ func (gta *GlobalTSOAllocator) GenerateTSO(ctx context.Context, count uint32) (p return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested pd %s of cluster", errs.NotLeaderErr)) } - return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count, 0) + return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count) } // Reset is used to reset the TSO allocator. diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 86b43d0de45..bb5fb4587f7 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -66,7 +66,7 @@ const ( type state struct { syncutil.RWMutex // ams stores the allocator managers of the keyspace groups. Each keyspace group is - // assigned with an allocator manager managing its global/local tso allocators. + // assigned with an allocator manager managing its global tso allocators. // Use a fixed size array to maximize the efficiency of concurrent access to // different keyspace groups for tso service. ams [constant.MaxKeyspaceGroupCountInUse]*AllocatorManager @@ -790,8 +790,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro am := NewAllocatorManager(kgm.ctx, group.ID, participant, tsRootPath, storage, kgm.cfg) am.startGlobalAllocatorLoop() log.Info("created allocator manager", - zap.Uint32("keyspace-group-id", group.ID), - zap.String("timestamp-path", am.GetTimestampPath())) + zap.Uint32("keyspace-group-id", group.ID)) kgm.Lock() group.KeyspaceLookupTable = make(map[uint32]struct{}) for _, kid := range group.Keyspaces { @@ -1517,7 +1516,6 @@ func (kgm *KeyspaceGroupManager) deletedGroupCleaner() { log.Info("delete the keyspace group tso key", zap.Uint32("keyspace-group-id", groupID)) // Clean up the remaining TSO keys. - // TODO: support the Local TSO Allocator clean up. err := kgm.tsoSvcStorage.DeleteTimestamp( keypath.TimestampPath( keypath.KeyspaceGroupGlobalTSPath(groupID), diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index dea0b00f4f0..be3d53785cd 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -87,7 +87,6 @@ func (suite *keyspaceGroupManagerTestSuite) createConfig() *TestServiceConfig { ListenAddr: addr, AdvertiseListenAddr: addr, LeaderLease: constant.DefaultLeaderLease, - LocalTSOEnabled: false, TSOUpdatePhysicalInterval: 50 * time.Millisecond, TSOSaveInterval: time.Duration(constant.DefaultLeaderLease) * time.Second, MaxResetTSGap: time.Hour * 24, diff --git a/pkg/tso/testutil.go b/pkg/tso/testutil.go index e3d04f55813..336d1414d98 100644 --- a/pkg/tso/testutil.go +++ b/pkg/tso/testutil.go @@ -29,7 +29,6 @@ type TestServiceConfig struct { ListenAddr string // Address the service listens on. AdvertiseListenAddr string // Address the service advertises to the clients. LeaderLease int64 // Leader lease. - LocalTSOEnabled bool // Whether local TSO is enabled. TSOUpdatePhysicalInterval time.Duration // Interval to update TSO in physical storage. TSOSaveInterval time.Duration // Interval to save TSO to physical storage. MaxResetTSGap time.Duration // Maximum gap to reset TSO. diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 38a4c989093..0210c98626b 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -43,8 +43,6 @@ const ( // When a TSO's logical time reaches this limit, // the physical time will be forced to increase. maxLogical = int64(1 << 18) - // MaxSuffixBits indicates the max number of suffix bits. - MaxSuffixBits = 4 // jetLagWarningThreshold is the warning threshold of jetLag in `timestampOracle.UpdateTimestamp`. // In case of small `updatePhysicalInterval`, the `3 * updatePhysicalInterval` would also is small, // and trigger unnecessary warnings about clock offset. @@ -55,9 +53,8 @@ const ( // tsoObject is used to store the current TSO in memory with a RWMutex lock. type tsoObject struct { syncutil.RWMutex - physical time.Time - logical int64 - updateTime time.Time + physical time.Time + logical int64 } // timestampOracle is used to maintain the logic of TSO. @@ -75,7 +72,6 @@ type timestampOracle struct { tsoMux *tsoObject // last timestamp window stored in etcd lastSavedTime atomic.Value // stored as time.Time - suffix int // pre-initialized metrics metrics *tsoMetrics @@ -92,7 +88,6 @@ func (t *timestampOracle) setTSOPhysical(next time.Time, force bool) { if typeutil.SubTSOPhysicalByWallClock(next, t.tsoMux.physical) > 0 { t.tsoMux.physical = next t.tsoMux.logical = 0 - t.tsoMux.updateTime = time.Now() } } @@ -106,23 +101,17 @@ func (t *timestampOracle) getTSO() (time.Time, int64) { } // generateTSO will add the TSO's logical part with the given count and returns the new TSO result. -func (t *timestampOracle) generateTSO(ctx context.Context, count int64, suffixBits int) (physical int64, logical int64, lastUpdateTime time.Time) { +func (t *timestampOracle) generateTSO(ctx context.Context, count int64) (physical int64, logical int64) { defer trace.StartRegion(ctx, "timestampOracle.generateTSO").End() t.tsoMux.Lock() defer t.tsoMux.Unlock() if t.tsoMux.physical == typeutil.ZeroTime { - return 0, 0, typeutil.ZeroTime + return 0, 0 } physical = t.tsoMux.physical.UnixNano() / int64(time.Millisecond) t.tsoMux.logical += count logical = t.tsoMux.logical - if suffixBits > 0 && t.suffix >= 0 { - logical = t.calibrateLogical(logical, suffixBits) - } - // Return the last update time - lastUpdateTime = t.tsoMux.updateTime - t.tsoMux.updateTime = time.Now() - return physical, logical, lastUpdateTime + return physical, logical } func (t *timestampOracle) getLastSavedTime() time.Time { @@ -133,28 +122,6 @@ func (t *timestampOracle) getLastSavedTime() time.Time { return last.(time.Time) } -// Because the Local TSO in each Local TSO Allocator is independent, so they are possible -// to be the same at sometimes, to avoid this case, we need to use the logical part of the -// Local TSO to do some differentiating work. -// For example, we have three DCs: dc-1, dc-2 and dc-3. The bits of suffix is defined by -// the const suffixBits. Then, for dc-2, the suffix may be 1 because it's persisted -// in etcd with the value of 1. -// Once we get a normal TSO like this (18 bits): xxxxxxxxxxxxxxxxxx. We will make the TSO's -// low bits of logical part from each DC looks like: -// -// global: xxxxxxxxxx00000000 -// dc-1: xxxxxxxxxx00000001 -// dc-2: xxxxxxxxxx00000010 -// dc-3: xxxxxxxxxx00000011 -func (t *timestampOracle) calibrateLogical(rawLogical int64, suffixBits int) int64 { - return rawLogical< 0)) @@ -209,7 +176,7 @@ func (t *timestampOracle) SyncTimestamp() error { }) save := next.Add(t.saveInterval) start := time.Now() - if err = t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err = t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { t.metrics.errSaveSyncTSEvent.Inc() return err } @@ -277,7 +244,7 @@ func (t *timestampOracle) resetUserTimestamp(leadership *election.Leadership, ts if typeutil.SubRealTimeByWallClock(t.getLastSavedTime(), nextPhysical) <= UpdateTimestampGuard { save := nextPhysical.Add(t.saveInterval) start := time.Now() - if err := t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err := t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { t.metrics.errSaveResetTSEvent.Inc() return err } @@ -287,7 +254,6 @@ func (t *timestampOracle) resetUserTimestamp(leadership *election.Leadership, ts // save into memory only if nextPhysical or nextLogical is greater. t.tsoMux.physical = nextPhysical t.tsoMux.logical = int64(nextLogical) - t.tsoMux.updateTime = time.Now() t.metrics.resetTSOOKEvent.Inc() return nil } @@ -361,10 +327,10 @@ func (t *timestampOracle) UpdateTimestamp() error { if typeutil.SubRealTimeByWallClock(t.getLastSavedTime(), next) <= UpdateTimestampGuard { save := next.Add(t.saveInterval) start := time.Now() - if err := t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err := t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { log.Warn("save timestamp failed", logutil.CondUint32("keyspace-group-id", t.keyspaceGroupID, t.keyspaceGroupID > 0), - zap.String("timestamp-path", t.GetTimestampPath()), + zap.String("timestamp-path", keypath.TimestampPath(t.tsPath)), zap.Error(err)) t.metrics.errSaveUpdateTSEvent.Inc() return err @@ -381,7 +347,7 @@ func (t *timestampOracle) UpdateTimestamp() error { var maxRetryCount = 10 // getTS is used to get a timestamp. -func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leadership, count uint32, suffixBits int) (pdpb.Timestamp, error) { +func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leadership, count uint32) (pdpb.Timestamp, error) { defer trace.StartRegion(ctx, "timestampOracle.getTS").End() var resp pdpb.Timestamp if count == 0 { @@ -399,7 +365,7 @@ func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leader return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") } // Get a new TSO result with the given count - resp.Physical, resp.Logical, _ = t.generateTSO(ctx, int64(count), suffixBits) + resp.Physical, resp.Logical = t.generateTSO(ctx, int64(count)) if resp.GetPhysical() == 0 { return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory has been reset") } @@ -416,7 +382,6 @@ func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leader if !leadership.Check() { return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested %s anymore", errs.NotLeaderErr)) } - resp.SuffixBits = uint32(suffixBits) return resp, nil } t.metrics.exceededMaxRetryEvent.Inc() @@ -430,6 +395,5 @@ func (t *timestampOracle) ResetTimestamp() { log.Info("reset the timestamp in memory", logutil.CondUint32("keyspace-group-id", t.keyspaceGroupID, t.keyspaceGroupID > 0)) t.tsoMux.physical = typeutil.ZeroTime t.tsoMux.logical = 0 - t.tsoMux.updateTime = typeutil.ZeroTime t.lastSavedTime.Store(typeutil.ZeroTime) } diff --git a/pkg/utils/keypath/key_path.go b/pkg/utils/keypath/key_path.go index 4d59fafe16f..1a56ad3330a 100644 --- a/pkg/utils/keypath/key_path.go +++ b/pkg/utils/keypath/key_path.go @@ -370,20 +370,6 @@ func TimestampPath(tsPath string) string { return path.Join(tsPath, TimestampKey) } -// FullTimestampPath returns the full timestamp path. -// 1. for the default keyspace group: -// /pd/{cluster_id}/timestamp -// 2. for the non-default keyspace groups: -// /ms/{cluster_id}/tso/{group}/gta/timestamp -func FullTimestampPath(groupID uint32) string { - rootPath := TSOSvcRootPath() - tsPath := TimestampPath(KeyspaceGroupGlobalTSPath(groupID)) - if groupID == constant.DefaultKeyspaceGroupID { - rootPath = LegacyRootPath() - } - return path.Join(rootPath, tsPath) -} - const ( registryKey = "registry" ) diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index c4aa96274e1..86afe75e6c5 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -171,23 +171,23 @@ func (s *TSODispatcher) processRequests(forwardStream stream, requests []Request s.tsoProxyBatchSize.Observe(float64(count)) // Split the response ts := resp.GetTimestamp() - physical, logical, suffixBits := ts.GetPhysical(), ts.GetLogical(), ts.GetSuffixBits() + physical, logical := ts.GetPhysical(), ts.GetLogical() // `logical` is the largest ts's logical part here, we need to do the subtracting before we finish each TSO request. // This is different from the logic of client batch, for example, if we have a largest ts whose logical part is 10, // count is 5, then the splitting results should be 5 and 10. - firstLogical := addLogical(logical, -int64(count), suffixBits) - return s.finishRequest(requests, physical, firstLogical, suffixBits) + firstLogical := addLogical(logical, -int64(count)) + return s.finishRequest(requests, physical, firstLogical) } // Because of the suffix, we need to shift the count before we add it to the logical part. -func addLogical(logical, count int64, suffixBits uint32) int64 { - return logical + count< Date: Tue, 17 Dec 2024 14:49:54 +0800 Subject: [PATCH 05/10] client/batch: allow tokenCh of batch controller to be nil (#8903) ref tikv/pd#8690 Allow `tokenCh` of batch controller be nil. Signed-off-by: JmPotato --- client/pkg/batch/batch_controller.go | 37 +++++++++----- client/pkg/batch/batch_controller_test.go | 61 ++++++++++++++++++++++- 2 files changed, 83 insertions(+), 15 deletions(-) diff --git a/client/pkg/batch/batch_controller.go b/client/pkg/batch/batch_controller.go index 32f0aaba1ae..322502b754a 100644 --- a/client/pkg/batch/batch_controller.go +++ b/client/pkg/batch/batch_controller.go @@ -60,13 +60,18 @@ func NewController[T any](maxBatchSize int, finisher FinisherFunc[T], bestBatchO // It returns nil error if everything goes well, otherwise a non-nil error which means we should stop the service. // It's guaranteed that if this function failed after collecting some requests, then these requests will be cancelled // when the function returns, so the caller don't need to clear them manually. +// `tokenCh` is an optional parameter: +// - If it's nil, the batching process will not wait for the token to arrive to continue. +// - If it's not nil, the batching process will wait for a token to arrive before continuing. +// The token will be given back if any error occurs, otherwise it's the caller's responsibility +// to decide when to recycle the signal. func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-chan T, tokenCh chan struct{}, maxBatchWaitInterval time.Duration) (errRet error) { var tokenAcquired bool defer func() { if errRet != nil { // Something went wrong when collecting a batch of requests. Release the token and cancel collected requests // if any. - if tokenAcquired { + if tokenAcquired && tokenCh != nil { tokenCh <- struct{}{} } bc.FinishCollectedRequests(bc.finisher, errRet) @@ -80,6 +85,9 @@ func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-c // If the batch size reaches the maxBatchSize limit but the token haven't arrived yet, don't receive more // requests, and return when token is ready. if bc.collectedRequestCount >= bc.maxBatchSize && !tokenAcquired { + if tokenCh == nil { + return nil + } select { case <-ctx.Done(): return ctx.Err() @@ -88,20 +96,23 @@ func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-c } } - select { - case <-ctx.Done(): - return ctx.Err() - case req := <-requestCh: - // Start to batch when the first request arrives. - bc.pushRequest(req) - // A request arrives but the token is not ready yet. Continue waiting, and also allowing collecting the next - // request if it arrives. - continue - case <-tokenCh: - tokenAcquired = true + if tokenCh != nil { + select { + case <-ctx.Done(): + return ctx.Err() + case req := <-requestCh: + // Start to batch when the first request arrives. + bc.pushRequest(req) + // A request arrives but the token is not ready yet. Continue waiting, and also allowing collecting the next + // request if it arrives. + continue + case <-tokenCh: + tokenAcquired = true + } } - // The token is ready. If the first request didn't arrive, wait for it. + // After the token is ready or it's working without token, + // wait for the first request to arrive. if bc.collectedRequestCount == 0 { select { case <-ctx.Done(): diff --git a/client/pkg/batch/batch_controller_test.go b/client/pkg/batch/batch_controller_test.go index 7c9ffa6944f..92aef14bd35 100644 --- a/client/pkg/batch/batch_controller_test.go +++ b/client/pkg/batch/batch_controller_test.go @@ -21,9 +21,11 @@ import ( "github.com/stretchr/testify/require" ) +const testMaxBatchSize = 20 + func TestAdjustBestBatchSize(t *testing.T) { re := require.New(t) - bc := NewController[int](20, nil, nil) + bc := NewController[int](testMaxBatchSize, nil, nil) re.Equal(defaultBestBatchSize, bc.bestBatchSize) bc.AdjustBestBatchSize() re.Equal(defaultBestBatchSize-1, bc.bestBatchSize) @@ -52,7 +54,7 @@ type testRequest struct { func TestFinishCollectedRequests(t *testing.T) { re := require.New(t) - bc := NewController[*testRequest](20, nil, nil) + bc := NewController[*testRequest](testMaxBatchSize, nil, nil) // Finish with zero request count. re.Zero(bc.collectedRequestCount) bc.FinishCollectedRequests(nil, nil) @@ -81,3 +83,58 @@ func TestFinishCollectedRequests(t *testing.T) { re.Equal(context.Canceled, requests[i].err) } } + +func TestFetchPendingRequests(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + re := require.New(t) + bc := NewController[int](testMaxBatchSize, nil, nil) + requestCh := make(chan int, testMaxBatchSize+1) + // Fetch a nil `tokenCh`. + requestCh <- 1 + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Empty(requestCh) + re.Equal(1, bc.collectedRequestCount) + // Fetch a nil `tokenCh` with max batch size. + for i := range testMaxBatchSize { + requestCh <- i + } + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Empty(requestCh) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Fetch a nil `tokenCh` with max batch size + 1. + for i := range testMaxBatchSize + 1 { + requestCh <- i + } + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Len(requestCh, 1) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Drain the requestCh. + <-requestCh + // Fetch a non-nil `tokenCh`. + tokenCh := make(chan struct{}, 1) + requestCh <- 1 + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Empty(requestCh) + re.Equal(1, bc.collectedRequestCount) + // Fetch a non-nil `tokenCh` with max batch size. + for i := range testMaxBatchSize { + requestCh <- i + } + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Empty(requestCh) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Fetch a non-nil `tokenCh` with max batch size + 1. + for i := range testMaxBatchSize + 1 { + requestCh <- i + } + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Len(requestCh, 1) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Drain the requestCh. + <-requestCh +} From e907aac6fc34cab0c2b0762ec95ab8355ca532fd Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 17 Dec 2024 15:23:16 +0800 Subject: [PATCH 06/10] *: enable gci (#8925) ref tikv/pd#4322 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- .golangci.yml | 8 ++++++++ client/circuitbreaker/circuit_breaker.go | 6 +++--- client/circuitbreaker/circuit_breaker_test.go | 4 ++-- client/client.go | 9 +++++---- client/client_test.go | 3 ++- client/clients/metastorage/client.go | 1 + client/clients/router/client.go | 1 + client/clients/tso/client.go | 12 +++++++----- client/clients/tso/dispatcher.go | 4 +++- client/clients/tso/dispatcher_test.go | 8 +++++--- client/clients/tso/request.go | 1 + client/clients/tso/request_test.go | 3 ++- client/clients/tso/stream.go | 8 +++++--- client/clients/tso/stream_test.go | 8 +++++--- client/errs/errs.go | 3 ++- client/gc_client.go | 4 +++- client/http/client.go | 6 ++++-- client/http/client_test.go | 1 + client/http/interface.go | 1 + client/http/request_info.go | 3 ++- client/http/types.go | 1 + client/inner_client.go | 7 ++++--- client/keyspace_client.go | 2 ++ client/meta_storage_client.go | 4 +++- client/opt/option.go | 7 ++++--- client/opt/option_test.go | 1 + client/pkg/retry/backoff.go | 3 ++- client/pkg/retry/backoff_test.go | 3 ++- client/pkg/utils/grpcutil/grpcutil.go | 12 +++++++----- client/pkg/utils/testutil/check_env_linux.go | 3 ++- client/pkg/utils/testutil/tempurl.go | 3 ++- client/pkg/utils/tlsutil/tlsconfig.go | 1 + client/pkg/utils/tlsutil/url.go | 3 ++- .../resource_group/controller/controller.go | 8 +++++--- .../controller/controller_test.go | 6 ++++-- client/resource_group/controller/limiter.go | 6 ++++-- .../resource_group/controller/model_test.go | 3 ++- client/resource_manager_client.go | 4 +++- .../servicediscovery/pd_service_discovery.go | 12 +++++++----- .../pd_service_discovery_test.go | 16 +++++++++------- .../servicediscovery/tso_service_discovery.go | 6 ++++-- cmd/pd-server/main.go | 6 ++++-- pkg/audit/audit.go | 6 ++++-- pkg/audit/audit_test.go | 1 + pkg/autoscaling/calculation.go | 8 +++++--- pkg/autoscaling/calculation_test.go | 1 + pkg/autoscaling/handler.go | 3 ++- pkg/autoscaling/prometheus.go | 8 +++++--- pkg/autoscaling/service.go | 5 +++-- pkg/autoscaling/types.go | 3 ++- pkg/cache/cache_test.go | 1 + pkg/cache/ttl.go | 4 +++- pkg/cgroup/cgmon.go | 6 ++++-- pkg/cgroup/cgroup.go | 3 ++- pkg/core/factory_test.go | 2 ++ pkg/core/region.go | 4 +++- pkg/core/region_test.go | 4 +++- pkg/core/region_tree.go | 4 +++- pkg/core/region_tree_test.go | 3 ++- pkg/core/store.go | 4 +++- pkg/core/store_option.go | 1 + pkg/core/store_stats.go | 1 + pkg/core/store_stats_test.go | 3 ++- pkg/core/store_test.go | 4 +++- pkg/core/storelimit/limit_test.go | 1 + pkg/dashboard/adapter/manager.go | 1 + pkg/dashboard/adapter/redirector.go | 1 + pkg/dashboard/distroutil/distro.go | 3 ++- pkg/dashboard/keyvisual/input/core.go | 4 +++- .../uiserver/embedded_assets_rewriter.go | 1 + pkg/election/leadership.go | 8 +++++--- pkg/election/leadership_test.go | 8 +++++--- pkg/election/lease.go | 6 ++++-- pkg/election/lease_test.go | 1 + pkg/encryption/config.go | 1 + pkg/encryption/config_test.go | 1 + pkg/encryption/crypter.go | 1 + pkg/encryption/crypter_test.go | 3 ++- pkg/encryption/key_manager.go | 8 +++++--- pkg/encryption/key_manager_test.go | 6 ++++-- pkg/encryption/kms.go | 2 ++ pkg/encryption/master_key.go | 1 + pkg/encryption/master_key_test.go | 3 ++- pkg/encryption/region_crypter.go | 1 + pkg/encryption/region_crypter_test.go | 3 ++- pkg/errs/errs.go | 3 ++- pkg/errs/errs_test.go | 5 +++-- pkg/gc/safepoint.go | 1 + pkg/gc/safepoint_test.go | 1 + pkg/gc/safepoint_v2.go | 4 +++- pkg/gctuner/memory_limit_tuner.go | 4 +++- pkg/gctuner/memory_limit_tuner_test.go | 4 +++- pkg/gctuner/tuner.go | 4 +++- pkg/id/id.go | 8 +++++--- pkg/id/id_test.go | 1 + pkg/keyspace/keyspace.go | 4 +++- pkg/keyspace/keyspace_test.go | 6 ++++-- pkg/keyspace/tso_keyspace_group.go | 8 +++++--- pkg/keyspace/tso_keyspace_group_test.go | 1 + pkg/keyspace/util.go | 1 + pkg/keyspace/util_test.go | 1 + pkg/mcs/discovery/discover.go | 6 ++++-- pkg/mcs/discovery/discover_test.go | 1 + pkg/mcs/discovery/register.go | 6 ++++-- pkg/mcs/discovery/register_test.go | 5 +++-- pkg/mcs/discovery/registry_entry.go | 3 ++- pkg/mcs/metastorage/server/grpc_service.go | 12 +++++++----- pkg/mcs/metastorage/server/manager.go | 6 ++++-- pkg/mcs/registry/registry.go | 6 ++++-- pkg/mcs/resourcemanager/server/apis/v1/api.go | 2 ++ pkg/mcs/resourcemanager/server/config.go | 6 ++++-- .../resourcemanager/server/grpc_service.go | 10 ++++++---- pkg/mcs/resourcemanager/server/manager.go | 6 ++++-- .../resourcemanager/server/metrics_test.go | 3 ++- .../resourcemanager/server/resource_group.go | 4 +++- .../server/resource_group_test.go | 3 ++- .../resourcemanager/server/token_buckets.go | 3 ++- .../server/token_buckets_test.go | 3 ++- pkg/mcs/scheduling/server/apis/v1/api.go | 4 +++- pkg/mcs/scheduling/server/cluster.go | 4 +++- pkg/mcs/scheduling/server/config/config.go | 6 ++++-- pkg/mcs/scheduling/server/config/watcher.go | 8 +++++--- pkg/mcs/scheduling/server/grpc_service.go | 10 ++++++---- pkg/mcs/scheduling/server/meta/watcher.go | 8 +++++--- pkg/mcs/scheduling/server/rule/watcher.go | 8 +++++--- .../scheduling/server/rule/watcher_test.go | 5 +++-- pkg/mcs/scheduling/server/server.go | 8 +++++--- pkg/mcs/scheduling/server/testutil.go | 4 +++- pkg/mcs/server/server.go | 6 ++++-- pkg/mcs/tso/server/apis/v1/api.go | 6 ++++-- pkg/mcs/tso/server/config.go | 6 ++++-- pkg/mcs/tso/server/config_test.go | 1 + pkg/mcs/tso/server/grpc_service.go | 8 +++++--- pkg/mcs/tso/server/server.go | 12 +++++++----- pkg/mcs/tso/server/testutil.go | 3 ++- pkg/mcs/utils/expected_primary.go | 6 ++++-- pkg/mcs/utils/util.go | 16 +++++++++------- pkg/member/member.go | 8 +++++--- pkg/member/participant.go | 6 ++++-- pkg/memory/meminfo.go | 8 +++++--- pkg/mock/mockcluster/mockcluster.go | 2 ++ pkg/mock/mockhbstream/mockhbstream.go | 1 + pkg/mock/mockhbstream/mockhbstream_test.go | 4 +++- pkg/mock/mockserver/mockserver.go | 1 + pkg/ratelimit/controller_test.go | 3 ++- pkg/ratelimit/limiter.go | 3 ++- pkg/ratelimit/limiter_test.go | 3 ++- pkg/ratelimit/ratelimiter.go | 3 ++- pkg/ratelimit/runner.go | 3 ++- pkg/replication/replication_mode.go | 4 +++- pkg/replication/replication_mode_test.go | 4 +++- pkg/response/region.go | 2 ++ pkg/response/region_test.go | 3 ++- pkg/response/store.go | 1 + pkg/schedule/checker/checker_controller.go | 4 +++- pkg/schedule/checker/joint_state_checker.go | 1 + .../checker/joint_state_checker_test.go | 4 +++- pkg/schedule/checker/learner_checker.go | 1 + pkg/schedule/checker/learner_checker_test.go | 4 +++- pkg/schedule/checker/merge_checker.go | 1 + pkg/schedule/checker/merge_checker_test.go | 6 ++++-- .../checker/priority_inspector_test.go | 1 + pkg/schedule/checker/replica_checker.go | 4 +++- pkg/schedule/checker/replica_checker_test.go | 6 ++++-- pkg/schedule/checker/replica_strategy.go | 4 +++- pkg/schedule/checker/rule_checker.go | 4 +++- pkg/schedule/checker/rule_checker_test.go | 5 +++-- pkg/schedule/checker/split_checker.go | 1 + pkg/schedule/checker/split_checker_test.go | 1 + pkg/schedule/config/config.go | 1 + pkg/schedule/config/config_provider.go | 2 ++ pkg/schedule/config/store_config.go | 4 +++- pkg/schedule/config/util_test.go | 3 ++- pkg/schedule/coordinator.go | 4 +++- pkg/schedule/filter/candidates_test.go | 4 +++- pkg/schedule/filter/counter_test.go | 1 + pkg/schedule/filter/filters.go | 4 +++- pkg/schedule/filter/filters_test.go | 4 +++- pkg/schedule/filter/healthy_test.go | 4 +++- pkg/schedule/filter/region_filters_test.go | 4 +++- pkg/schedule/handler/handler.go | 4 +++- pkg/schedule/hbstream/heartbeat_streams.go | 4 +++- pkg/schedule/labeler/labeler.go | 4 +++- pkg/schedule/labeler/labeler_test.go | 4 +++- pkg/schedule/labeler/rules.go | 4 +++- pkg/schedule/operator/builder.go | 1 + pkg/schedule/operator/builder_test.go | 4 +++- pkg/schedule/operator/create_operator.go | 4 +++- pkg/schedule/operator/create_operator_test.go | 6 ++++-- pkg/schedule/operator/metrics.go | 1 + pkg/schedule/operator/operator.go | 4 +++- pkg/schedule/operator/operator_controller.go | 4 +++- .../operator/operator_controller_test.go | 6 ++++-- pkg/schedule/operator/operator_test.go | 4 +++- pkg/schedule/operator/step.go | 4 +++- pkg/schedule/operator/step_test.go | 4 +++- pkg/schedule/operator/test_util.go | 1 + .../operator/waiting_operator_test.go | 4 +++- pkg/schedule/placement/fit.go | 1 + pkg/schedule/placement/fit_region_test.go | 1 + pkg/schedule/placement/fit_test.go | 4 +++- .../placement/label_constraint_test.go | 1 + pkg/schedule/placement/region_rule_cache.go | 1 + .../placement/region_rule_cache_test.go | 4 +++- pkg/schedule/placement/rule_list.go | 1 + pkg/schedule/placement/rule_manager.go | 6 ++++-- pkg/schedule/placement/rule_manager_test.go | 4 +++- pkg/schedule/plan/balance_plan_test.go | 4 +++- pkg/schedule/plugin_interface.go | 4 +++- pkg/schedule/prepare_checker.go | 4 +++- pkg/schedule/scatter/region_scatterer.go | 4 +++- pkg/schedule/scatter/region_scatterer_test.go | 4 +++- .../schedulers/balance_benchmark_test.go | 4 +++- pkg/schedule/schedulers/balance_leader.go | 6 ++++-- .../schedulers/balance_leader_test.go | 4 +++- pkg/schedule/schedulers/balance_region.go | 4 +++- .../schedulers/balance_region_test.go | 4 +++- pkg/schedule/schedulers/balance_witness.go | 8 +++++--- .../schedulers/balance_witness_test.go | 1 + pkg/schedule/schedulers/base_scheduler.go | 1 + pkg/schedule/schedulers/config.go | 4 +++- pkg/schedule/schedulers/config_test.go | 1 + pkg/schedule/schedulers/evict_leader.go | 6 ++++-- pkg/schedule/schedulers/evict_leader_test.go | 4 +++- pkg/schedule/schedulers/evict_slow_store.go | 6 ++++-- .../schedulers/evict_slow_store_test.go | 4 +++- pkg/schedule/schedulers/evict_slow_trend.go | 6 ++++-- .../schedulers/evict_slow_trend_test.go | 4 +++- pkg/schedule/schedulers/grant_hot_region.go | 6 ++++-- pkg/schedule/schedulers/grant_leader.go | 6 ++++-- pkg/schedule/schedulers/grant_leader_test.go | 1 + pkg/schedule/schedulers/hot_region.go | 6 ++++-- pkg/schedule/schedulers/hot_region_config.go | 6 ++++-- .../schedulers/hot_region_rank_v2_test.go | 1 + pkg/schedule/schedulers/hot_region_test.go | 4 +++- pkg/schedule/schedulers/label.go | 4 +++- pkg/schedule/schedulers/metrics.go | 1 + pkg/schedule/schedulers/random_merge.go | 1 + pkg/schedule/schedulers/random_merge_test.go | 1 + pkg/schedule/schedulers/range_cluster.go | 1 + pkg/schedule/schedulers/scatter_range.go | 4 +++- pkg/schedule/schedulers/scatter_range_test.go | 4 +++- pkg/schedule/schedulers/scheduler.go | 4 +++- .../schedulers/scheduler_controller.go | 4 +++- pkg/schedule/schedulers/scheduler_test.go | 4 +++- pkg/schedule/schedulers/shuffle_hot_region.go | 6 ++++-- pkg/schedule/schedulers/shuffle_leader.go | 1 + pkg/schedule/schedulers/shuffle_region.go | 1 + .../schedulers/shuffle_region_config.go | 3 ++- pkg/schedule/schedulers/split_bucket.go | 4 +++- pkg/schedule/schedulers/split_bucket_test.go | 4 +++- .../schedulers/transfer_witness_leader.go | 1 + .../transfer_witness_leader_test.go | 4 +++- pkg/schedule/schedulers/utils.go | 4 +++- pkg/schedule/schedulers/utils_test.go | 4 +++- pkg/schedule/splitter/region_splitter.go | 4 +++- pkg/schedule/splitter/region_splitter_test.go | 1 + pkg/slice/slice_test.go | 1 + pkg/statistics/buckets/hot_bucket_cache.go | 4 +++- .../buckets/hot_bucket_cache_test.go | 3 ++- .../buckets/hot_bucket_task_test.go | 3 ++- pkg/statistics/hot_cache.go | 4 +++- pkg/statistics/hot_cache_test.go | 1 + pkg/statistics/hot_peer.go | 4 +++- pkg/statistics/hot_peer_cache.go | 4 +++- pkg/statistics/hot_peer_cache_test.go | 4 +++- pkg/statistics/region_collection.go | 4 +++- pkg/statistics/region_collection_test.go | 4 +++- pkg/statistics/store.go | 4 +++- pkg/statistics/store_collection.go | 1 + pkg/statistics/store_collection_test.go | 4 +++- pkg/statistics/store_load_test.go | 1 + pkg/statistics/store_test.go | 4 +++- pkg/statistics/utils/kind_test.go | 4 +++- pkg/storage/endpoint/cluster_id.go | 6 ++++-- pkg/storage/endpoint/cluster_id_test.go | 1 + pkg/storage/endpoint/config.go | 3 ++- pkg/storage/endpoint/gc_safe_point.go | 4 +++- pkg/storage/endpoint/keyspace.go | 4 +++- pkg/storage/endpoint/meta.go | 2 ++ pkg/storage/endpoint/resource_group.go | 1 + pkg/storage/endpoint/safepoint_v2.go | 6 ++++-- pkg/storage/endpoint/tso.go | 6 ++++-- pkg/storage/endpoint/tso_keyspace_group.go | 3 ++- pkg/storage/endpoint/util.go | 3 ++- pkg/storage/etcd_backend.go | 3 ++- pkg/storage/hot_region_storage.go | 10 ++++++---- pkg/storage/hot_region_storage_test.go | 1 + pkg/storage/keyspace_test.go | 4 +++- pkg/storage/kv/etcd_kv.go | 6 ++++-- pkg/storage/kv/kv_test.go | 3 ++- pkg/storage/kv/levedb_kv.go | 4 +++- pkg/storage/kv/mem_kv.go | 2 ++ pkg/storage/leveldb_backend.go | 4 +++- pkg/storage/leveldb_backend_test.go | 1 + pkg/storage/region_storage.go | 2 ++ pkg/storage/region_storage_test.go | 4 +++- pkg/storage/storage.go | 4 +++- pkg/storage/storage_gc_test.go | 4 +++- pkg/storage/storage_test.go | 6 ++++-- pkg/storage/storage_tso_test.go | 1 + pkg/syncer/client.go | 14 ++++++++------ pkg/syncer/client_test.go | 8 +++++--- pkg/syncer/history_buffer.go | 4 +++- pkg/syncer/history_buffer_test.go | 4 +++- pkg/syncer/server.go | 8 +++++--- pkg/systimemon/systimemon.go | 4 +++- pkg/tso/admin.go | 3 ++- pkg/tso/allocator_manager.go | 6 ++++-- pkg/tso/global_allocator.go | 6 ++++-- pkg/tso/keyspace_group_manager.go | 8 +++++--- pkg/tso/keyspace_group_manager_test.go | 10 ++++++---- pkg/tso/tso.go | 6 ++++-- pkg/tso/util_test.go | 1 + .../unsafe_recovery_controller.go | 4 +++- .../unsafe_recovery_controller_test.go | 4 +++- pkg/utils/apiutil/apiutil.go | 6 ++++-- .../apiutil/multiservicesapi/middleware.go | 4 +++- pkg/utils/apiutil/serverapi/middleware.go | 6 ++++-- pkg/utils/configutil/configutil.go | 4 +++- pkg/utils/etcdutil/etcdutil.go | 16 +++++++++------- pkg/utils/etcdutil/etcdutil_test.go | 10 ++++++---- pkg/utils/etcdutil/health_checker.go | 8 +++++--- pkg/utils/etcdutil/testutil.go | 5 +++-- pkg/utils/grpcutil/grpcutil.go | 10 ++++++---- pkg/utils/grpcutil/grpcutil_test.go | 6 ++++-- pkg/utils/logutil/log.go | 6 ++++-- pkg/utils/metricutil/metricutil.go | 4 +++- pkg/utils/metricutil/metricutil_test.go | 1 + pkg/utils/operatorutil/operator_check.go | 1 + pkg/utils/tempurl/check_env_linux.go | 2 ++ pkg/utils/tempurl/tempurl.go | 1 + pkg/utils/testutil/api_check.go | 1 + pkg/utils/testutil/testutil.go | 5 +++-- pkg/utils/tsoutil/tso_dispatcher.go | 8 +++++--- pkg/utils/tsoutil/tso_proto_factory.go | 4 +++- pkg/utils/tsoutil/tso_request.go | 4 +++- pkg/utils/typeutil/size.go | 1 + pkg/versioninfo/feature.go | 4 +++- pkg/versioninfo/versioninfo.go | 4 +++- plugin/scheduler_example/evict_leader.go | 4 +++- server/api/admin.go | 6 ++++-- server/api/admin_test.go | 4 +++- server/api/checker.go | 3 ++- server/api/cluster.go | 3 ++- server/api/cluster_test.go | 4 +++- server/api/config.go | 4 +++- server/api/diagnostic.go | 3 ++- server/api/diagnostic_test.go | 4 +++- server/api/etcd_api_test.go | 1 + server/api/health.go | 3 ++- server/api/health_test.go | 1 + server/api/hot_status.go | 3 ++- server/api/hot_status_test.go | 1 + server/api/label.go | 4 +++- server/api/label_test.go | 4 +++- server/api/log.go | 4 +++- server/api/log_test.go | 4 +++- server/api/member.go | 6 ++++-- server/api/member_test.go | 6 ++++-- server/api/middleware.go | 6 ++++-- server/api/min_resolved_ts.go | 3 ++- server/api/min_resolved_ts_test.go | 4 +++- server/api/operator.go | 3 ++- server/api/plugin_disable.go | 3 ++- server/api/pprof.go | 6 ++++-- server/api/pprof_test.go | 1 + server/api/region.go | 4 +++- server/api/region_label.go | 3 ++- server/api/region_label_test.go | 4 +++- server/api/region_test.go | 6 ++++-- server/api/replication_mode.go | 3 ++- server/api/router.go | 4 +++- server/api/rule.go | 3 ++- server/api/scheduler.go | 6 ++++-- server/api/server.go | 3 ++- server/api/server_test.go | 8 +++++--- server/api/service_gc_safepoint.go | 3 ++- server/api/service_gc_safepoint_test.go | 4 +++- server/api/service_middleware.go | 5 +++-- server/api/service_middleware_test.go | 4 +++- server/api/stats.go | 3 ++- server/api/stats_test.go | 4 +++- server/api/status.go | 3 ++- server/api/status_test.go | 1 + server/api/store.go | 4 +++- server/api/store_test.go | 6 ++++-- server/api/trend.go | 3 ++- server/api/trend_test.go | 4 +++- server/api/tso.go | 3 ++- server/api/unsafe_operation.go | 3 ++- server/api/unsafe_operation_test.go | 4 +++- server/api/version.go | 3 ++- server/api/version_test.go | 4 +++- server/apiv2/handlers/keyspace.go | 2 ++ server/apiv2/handlers/micro_service.go | 1 + server/apiv2/handlers/tso_keyspace_group.go | 2 ++ server/apiv2/middlewares/bootstrap_checker.go | 1 + server/apiv2/middlewares/redirector.go | 4 +++- server/apiv2/router.go | 1 + server/cluster/cluster.go | 6 ++++-- server/cluster/cluster_test.go | 4 +++- server/cluster/cluster_worker.go | 4 +++- server/cluster/cluster_worker_test.go | 4 +++- server/cluster/scheduling_controller.go | 1 + server/config/config.go | 10 ++++++---- server/config/config_test.go | 1 + server/config/persist_options.go | 6 ++++-- .../service_middleware_persist_options.go | 1 + server/config/util.go | 1 + server/forward.go | 10 ++++++---- server/gc_service.go | 8 +++++--- server/grpc_service.go | 14 ++++++++------ server/handler.go | 4 +++- server/join/join.go | 8 +++++--- server/join/join_test.go | 1 + server/keyspace_service.go | 6 ++++-- server/server.go | 19 +++++++++++-------- server/server_test.go | 7 ++++--- server/testutil.go | 6 ++++-- server/util.go | 6 ++++-- tests/autoscaling/autoscaling_test.go | 3 ++- tests/cluster.go | 4 +++- tests/compatibility/version_upgrade_test.go | 4 +++- tests/config.go | 1 + tests/dashboard/race_test.go | 1 + tests/dashboard/service_test.go | 3 ++- tests/integrations/client/client_test.go | 14 ++++++++------ tests/integrations/client/client_tls_test.go | 5 +++-- tests/integrations/client/gc_client_test.go | 12 +++++++----- .../integrations/client/global_config_test.go | 13 +++++++------ tests/integrations/client/http_client_test.go | 8 +++++--- tests/integrations/client/keyspace_test.go | 4 +++- .../mcs/discovery/register_test.go | 3 ++- .../mcs/keyspace/tso_keyspace_group_test.go | 4 +++- tests/integrations/mcs/members/member_test.go | 4 +++- .../resourcemanager/resource_manager_test.go | 10 ++++++---- tests/integrations/mcs/scheduling/api_test.go | 7 +++++-- .../mcs/scheduling/config_test.go | 4 +++- .../integrations/mcs/scheduling/meta_test.go | 4 +++- .../integrations/mcs/scheduling/rule_test.go | 1 + .../mcs/scheduling/server_test.go | 8 +++++--- tests/integrations/mcs/testutil.go | 1 + tests/integrations/mcs/tso/api_test.go | 4 +++- .../mcs/tso/keyspace_group_manager_test.go | 6 ++++-- tests/integrations/mcs/tso/proxy_test.go | 12 +++++++----- tests/integrations/mcs/tso/server_test.go | 14 ++++++++------ tests/integrations/realcluster/cluster.go | 3 ++- .../realcluster/cluster_id_test.go | 1 + .../integrations/realcluster/etcd_key_test.go | 1 + tests/integrations/realcluster/mock_db.go | 3 ++- .../realcluster/reboot_pd_test.go | 1 + .../realcluster/scheduler_test.go | 6 ++++-- tests/integrations/tso/client_test.go | 4 +++- tests/integrations/tso/consistency_test.go | 6 ++++-- tests/integrations/tso/server_test.go | 6 ++++-- tests/integrations/tso/testutil.go | 3 ++- tests/registry/registry_test.go | 12 +++++++----- tests/scheduling_cluster.go | 1 + tests/server/api/api_test.go | 8 +++++--- tests/server/api/checker_test.go | 1 + tests/server/api/operator_test.go | 4 +++- tests/server/api/region_test.go | 6 ++++-- tests/server/api/rule_test.go | 4 +++- tests/server/api/scheduler_test.go | 6 ++++-- tests/server/api/testutil.go | 1 + tests/server/apiv2/handlers/keyspace_test.go | 8 +++++--- tests/server/apiv2/handlers/testutil.go | 4 +++- .../apiv2/handlers/tso_keyspace_group_test.go | 1 + tests/server/cluster/cluster_test.go | 8 +++++--- tests/server/cluster/cluster_work_test.go | 4 +++- tests/server/config/config_test.go | 1 + tests/server/id/id_test.go | 6 ++++-- tests/server/join/join_fail/join_fail_test.go | 4 +++- tests/server/join/join_test.go | 1 + tests/server/keyspace/keyspace_test.go | 4 +++- tests/server/member/member_test.go | 6 ++++-- .../region_syncer/region_syncer_test.go | 6 ++++-- tests/server/server_test.go | 6 ++++-- .../server/storage/hot_region_storage_test.go | 4 +++- tests/server/tso/tso_test.go | 6 ++++-- tests/server/watch/leader_watch_test.go | 6 ++++-- tests/testutil.go | 6 ++++-- tests/tso_cluster.go | 4 +++- .../pd-analysis/analysis/transfer_counter.go | 3 ++- tools/pd-analysis/main.go | 4 +++- tools/pd-api-bench/cases/cases.go | 6 ++++-- tools/pd-api-bench/cases/controller.go | 6 ++++-- tools/pd-api-bench/config/config.go | 6 ++++-- tools/pd-api-bench/main.go | 14 ++++++++------ tools/pd-backup/main.go | 3 ++- tools/pd-backup/pdbackup/backup.go | 3 ++- tools/pd-backup/pdbackup/backup_test.go | 7 ++++--- tools/pd-backup/tests/backup_test.go | 3 ++- tools/pd-ctl/main.go | 4 +++- tools/pd-ctl/pdctl/command/config_command.go | 1 + tools/pd-ctl/pdctl/command/global.go | 6 ++++-- tools/pd-ctl/pdctl/command/hot_command.go | 4 +++- .../pd-ctl/pdctl/command/keyspace_command.go | 1 + .../pdctl/command/keyspace_group_command.go | 1 + tools/pd-ctl/pdctl/command/label_command.go | 1 + tools/pd-ctl/pdctl/command/log_command.go | 1 + tools/pd-ctl/pdctl/command/operator.go | 3 ++- tools/pd-ctl/pdctl/command/plugin_command.go | 1 + tools/pd-ctl/pdctl/command/region_command.go | 3 ++- tools/pd-ctl/pdctl/command/scheduler.go | 3 ++- tools/pd-ctl/pdctl/command/store_command.go | 6 ++++-- tools/pd-ctl/pdctl/command/tso_command.go | 1 + tools/pd-ctl/pdctl/ctl.go | 1 + tools/pd-ctl/tests/cluster/cluster_test.go | 4 +++- .../tests/completion/completion_test.go | 1 + tools/pd-ctl/tests/config/config_test.go | 6 ++++-- tools/pd-ctl/tests/global_test.go | 4 +++- tools/pd-ctl/tests/health/health_test.go | 3 ++- tools/pd-ctl/tests/helper.go | 1 + tools/pd-ctl/tests/hot/hot_test.go | 6 ++++-- .../tests/keyspace/keyspace_group_test.go | 4 +++- tools/pd-ctl/tests/keyspace/keyspace_test.go | 6 ++++-- tools/pd-ctl/tests/label/label_test.go | 4 +++- tools/pd-ctl/tests/log/log_test.go | 4 +++- tools/pd-ctl/tests/member/member_test.go | 4 +++- tools/pd-ctl/tests/operator/operator_test.go | 4 +++- tools/pd-ctl/tests/region/region_test.go | 4 +++- .../resource_manager_command_test.go | 1 + .../pd-ctl/tests/safepoint/safepoint_test.go | 1 + .../pd-ctl/tests/scheduler/scheduler_test.go | 6 ++++-- tools/pd-ctl/tests/store/store_test.go | 6 ++++-- tools/pd-ctl/tests/tso/tso_test.go | 1 + .../tests/unsafe/unsafe_operation_test.go | 1 + tools/pd-heartbeat-bench/config/config.go | 6 ++++-- tools/pd-heartbeat-bench/main.go | 8 +++++--- tools/pd-heartbeat-bench/metrics/util.go | 3 ++- tools/pd-recover/main.go | 6 ++++-- tools/pd-simulator/main.go | 6 ++++-- .../simulator/cases/balance_leader.go | 2 ++ .../simulator/cases/balance_region.go | 1 + tools/pd-simulator/simulator/cases/cases.go | 1 + .../cases/diagnose_label_isolation.go | 4 +++- .../simulator/cases/diagnose_rule.go | 4 +++- .../pd-simulator/simulator/cases/hot_read.go | 2 ++ .../pd-simulator/simulator/cases/hot_write.go | 2 ++ .../simulator/cases/import_data.go | 4 +++- .../simulator/cases/makeup_down_replica.go | 2 ++ .../simulator/cases/region_merge.go | 2 ++ .../simulator/cases/region_split.go | 2 ++ .../simulator/cases/scale_tikv.go | 1 + .../simulator/cases/stable_env.go | 2 ++ tools/pd-simulator/simulator/client.go | 8 +++++--- tools/pd-simulator/simulator/config/config.go | 1 + tools/pd-simulator/simulator/conn.go | 1 + tools/pd-simulator/simulator/drive.go | 8 +++++--- tools/pd-simulator/simulator/event.go | 4 +++- tools/pd-simulator/simulator/node.go | 4 +++- tools/pd-simulator/simulator/raft.go | 4 +++- tools/pd-simulator/simulator/simutil/key.go | 1 + .../simulator/simutil/key_test.go | 4 +++- .../pd-simulator/simulator/simutil/logger.go | 3 ++- tools/pd-simulator/simulator/task.go | 4 +++- tools/pd-tso-bench/main.go | 10 ++++++---- tools/pd-ut/alloc/check_env_linux.go | 3 ++- tools/pd-ut/alloc/server.go | 4 +++- tools/pd-ut/alloc/tempurl.go | 1 + tools/pd-ut/ut.go | 3 ++- tools/regions-dump/main.go | 6 ++++-- tools/stores-dump/main.go | 6 ++++-- 565 files changed, 1634 insertions(+), 719 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index e13feb04ba5..a44052b22e8 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -28,6 +28,7 @@ linters: - protogetter - reassign - intrange + - gci linters-settings: gocritic: # Which checks should be disabled; can't be combined with 'enabled-checks'; default is empty @@ -233,6 +234,13 @@ linters-settings: desc: "Use 'sync/atomic' instead of 'go.uber.org/atomic'" - pkg: github.com/pkg/errors desc: "Use 'github.com/pingcap/errors' instead of 'github.com/pkg/errors'" + gci: + sections: + - standard + - default + - prefix(github.com/pingcap) + - prefix(github.com/tikv/pd) + - blank issues: exclude-rules: - path: (_test\.go|pkg/mock/.*\.go|tests/.*\.go) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/circuitbreaker/circuit_breaker.go index b5a4c53ebb5..26d1b642ea4 100644 --- a/client/circuitbreaker/circuit_breaker.go +++ b/client/circuitbreaker/circuit_breaker.go @@ -19,13 +19,13 @@ import ( "sync" "time" - "github.com/tikv/pd/client/errs" - "github.com/prometheus/client_golang/prometheus" - m "github.com/tikv/pd/client/metrics" "go.uber.org/zap" "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" + m "github.com/tikv/pd/client/metrics" ) // Overloading is a type describing service return value diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/circuitbreaker/circuit_breaker_test.go index ca77b7f9f99..6a726028cd8 100644 --- a/client/circuitbreaker/circuit_breaker_test.go +++ b/client/circuitbreaker/circuit_breaker_test.go @@ -18,9 +18,9 @@ import ( "testing" "time" - "github.com/tikv/pd/client/errs" - "github.com/stretchr/testify/require" + + "github.com/tikv/pd/client/errs" ) // advance emulate the state machine clock moves forward by the given duration diff --git a/client/client.go b/client/client.go index 519fd478bb3..31bc72c0a77 100644 --- a/client/client.go +++ b/client/client.go @@ -22,15 +22,17 @@ import ( "sync" "time" - cb "github.com/tikv/pd/client/circuitbreaker" - "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/clients/tso" @@ -41,7 +43,6 @@ import ( "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) // GlobalConfigItem standard format of KV pair in GlobalConfig client diff --git a/client/client_test.go b/client/client_test.go index f4f914900cd..305d054fa18 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -20,11 +20,12 @@ import ( "time" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/pkg/utils/testutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/client/clients/metastorage/client.go b/client/clients/metastorage/client.go index dba1127f9f5..baac1ab7539 100644 --- a/client/clients/metastorage/client.go +++ b/client/clients/metastorage/client.go @@ -18,6 +18,7 @@ import ( "context" "github.com/pingcap/kvproto/pkg/meta_storagepb" + "github.com/tikv/pd/client/opt" ) diff --git a/client/clients/router/client.go b/client/clients/router/client.go index 667c82a6805..48cebfa950e 100644 --- a/client/clients/router/client.go +++ b/client/clients/router/client.go @@ -20,6 +20,7 @@ import ( "net/url" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/client/opt" ) diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index 68e2163d191..c26dd25f2ad 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -22,9 +22,16 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" @@ -32,11 +39,6 @@ import ( "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" ) const ( diff --git a/client/clients/tso/dispatcher.go b/client/clients/tso/dispatcher.go index 1b805395904..bdac8096f85 100644 --- a/client/clients/tso/dispatcher.go +++ b/client/clients/tso/dispatcher.go @@ -25,9 +25,12 @@ import ( "time" "github.com/opentracing/opentracing-go" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/client/pkg/utils/timerutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) // deadline is used to control the TS request timeout manually, diff --git a/client/clients/tso/dispatcher_test.go b/client/clients/tso/dispatcher_test.go index 2b5fd1e52e8..cefc53f3944 100644 --- a/client/clients/tso/dispatcher_test.go +++ b/client/clients/tso/dispatcher_test.go @@ -22,13 +22,15 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/zap/zapcore" + + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/tikv/pd/client/opt" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap/zapcore" ) type mockTSOServiceProvider struct { diff --git a/client/clients/tso/request.go b/client/clients/tso/request.go index 0c9f54f8b2b..9c0d8e6bea6 100644 --- a/client/clients/tso/request.go +++ b/client/clients/tso/request.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/tikv/pd/client/metrics" ) diff --git a/client/clients/tso/request_test.go b/client/clients/tso/request_test.go index 6887ee28124..4be50eaea68 100644 --- a/client/clients/tso/request_test.go +++ b/client/clients/tso/request_test.go @@ -18,8 +18,9 @@ import ( "context" "testing" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" + + "github.com/pingcap/errors" ) func TestTsoRequestWait(t *testing.T) { diff --git a/client/clients/tso/stream.go b/client/clients/tso/stream.go index 6baf63c8882..291d1d31b65 100644 --- a/client/clients/tso/stream.go +++ b/client/clients/tso/stream.go @@ -23,17 +23,19 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" - "go.uber.org/zap" - "google.golang.org/grpc" ) // TSO Stream Builder Factory diff --git a/client/clients/tso/stream_test.go b/client/clients/tso/stream_test.go index 0244c06e024..4791b90bb81 100644 --- a/client/clients/tso/stream_test.go +++ b/client/clients/tso/stream_test.go @@ -21,12 +21,14 @@ import ( "testing" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/client/errs" "go.uber.org/zap/zapcore" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" ) const mockStreamURL = "mock:///" diff --git a/client/errs/errs.go b/client/errs/errs.go index 67a5dd8ec92..868faa6a77a 100644 --- a/client/errs/errs.go +++ b/client/errs/errs.go @@ -17,10 +17,11 @@ package errs import ( "strings" - "github.com/pingcap/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" "google.golang.org/grpc/codes" + + "github.com/pingcap/errors" ) // IsLeaderChange will determine whether there is a leader/primary change. diff --git a/client/gc_client.go b/client/gc_client.go index f30521905c3..45fc8b40b91 100644 --- a/client/gc_client.go +++ b/client/gc_client.go @@ -19,11 +19,13 @@ import ( "time" "github.com/opentracing/opentracing-go" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" - "go.uber.org/zap" ) // GCClient is a client for doing GC diff --git a/client/http/client.go b/client/http/client.go index c813474fcf6..fa9801cf764 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -23,13 +23,15 @@ import ( "net/http" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/retry" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) const ( diff --git a/client/http/client_test.go b/client/http/client_test.go index a14691eed02..b6c2105bd4a 100644 --- a/client/http/client_test.go +++ b/client/http/client_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/http/interface.go b/client/http/interface.go index 772599e27fb..1aabd1ae331 100644 --- a/client/http/interface.go +++ b/client/http/interface.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/http/request_info.go b/client/http/request_info.go index 1e3449b59a0..d1930800304 100644 --- a/client/http/request_info.go +++ b/client/http/request_info.go @@ -17,8 +17,9 @@ package http import ( "fmt" - "github.com/tikv/pd/client/pkg/retry" "go.uber.org/zap" + + "github.com/tikv/pd/client/pkg/retry" ) // The following constants are the names of the requests. diff --git a/client/http/types.go b/client/http/types.go index cab564e99ac..83e8badf334 100644 --- a/client/http/types.go +++ b/client/http/types.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" + pd "github.com/tikv/pd/client/clients/router" ) diff --git a/client/inner_client.go b/client/inner_client.go index ae15c763854..045a9a6eed8 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -6,20 +6,21 @@ import ( "sync" "time" + "go.uber.org/zap" + "google.golang.org/grpc" "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/status" ) const ( diff --git a/client/keyspace_client.go b/client/keyspace_client.go index ce0cc0bc426..84bc29054eb 100644 --- a/client/keyspace_client.go +++ b/client/keyspace_client.go @@ -19,8 +19,10 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" ) diff --git a/client/meta_storage_client.go b/client/meta_storage_client.go index 45bcb8d3d65..fbabd60debd 100644 --- a/client/meta_storage_client.go +++ b/client/meta_storage_client.go @@ -19,10 +19,12 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" diff --git a/client/opt/option.go b/client/opt/option.go index 9a80a895cc0..c7a0bb17195 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -18,12 +18,13 @@ import ( "sync/atomic" "time" - cb "github.com/tikv/pd/client/circuitbreaker" + "github.com/prometheus/client_golang/prometheus" + "google.golang.org/grpc" "github.com/pingcap/errors" - "github.com/prometheus/client_golang/prometheus" + + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" - "google.golang.org/grpc" ) const ( diff --git a/client/opt/option_test.go b/client/opt/option_test.go index 26760fac7f2..fa0decbb3a1 100644 --- a/client/opt/option_test.go +++ b/client/opt/option_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/client/pkg/utils/testutil" ) diff --git a/client/pkg/retry/backoff.go b/client/pkg/retry/backoff.go index b0524e6c139..e3b331582d6 100644 --- a/client/pkg/retry/backoff.go +++ b/client/pkg/retry/backoff.go @@ -21,10 +21,11 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "go.uber.org/zap" ) // Option is used to customize the backoffer. diff --git a/client/pkg/retry/backoff_test.go b/client/pkg/retry/backoff_test.go index 12891d822aa..b01a753b374 100644 --- a/client/pkg/retry/backoff_test.go +++ b/client/pkg/retry/backoff_test.go @@ -22,9 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "go.uber.org/zap" + + "github.com/pingcap/log" ) func TestBackoffer(t *testing.T) { diff --git a/client/pkg/utils/grpcutil/grpcutil.go b/client/pkg/utils/grpcutil/grpcutil.go index 29967263dd3..b73d117fe84 100644 --- a/client/pkg/utils/grpcutil/grpcutil.go +++ b/client/pkg/utils/grpcutil/grpcutil.go @@ -21,17 +21,19 @@ import ( "sync" "time" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/tikv/pd/client/errs" - "github.com/tikv/pd/client/pkg/retry" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/backoff" "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/pkg/retry" ) const ( diff --git a/client/pkg/utils/testutil/check_env_linux.go b/client/pkg/utils/testutil/check_env_linux.go index df45f359eb3..ebe6a8e0663 100644 --- a/client/pkg/utils/testutil/check_env_linux.go +++ b/client/pkg/utils/testutil/check_env_linux.go @@ -18,8 +18,9 @@ package testutil import ( "github.com/cakturk/go-netstat/netstat" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) func environmentCheck(addr string) bool { diff --git a/client/pkg/utils/testutil/tempurl.go b/client/pkg/utils/testutil/tempurl.go index 71d51176106..d948160ba42 100644 --- a/client/pkg/utils/testutil/tempurl.go +++ b/client/pkg/utils/testutil/tempurl.go @@ -20,8 +20,9 @@ import ( "sync" "time" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) var ( diff --git a/client/pkg/utils/tlsutil/tlsconfig.go b/client/pkg/utils/tlsutil/tlsconfig.go index 88d797d3b3a..65e0ce9542f 100644 --- a/client/pkg/utils/tlsutil/tlsconfig.go +++ b/client/pkg/utils/tlsutil/tlsconfig.go @@ -40,6 +40,7 @@ import ( "fmt" "github.com/pingcap/errors" + "github.com/tikv/pd/client/errs" ) diff --git a/client/pkg/utils/tlsutil/url.go b/client/pkg/utils/tlsutil/url.go index ccc312d195b..abe4132a6a9 100644 --- a/client/pkg/utils/tlsutil/url.go +++ b/client/pkg/utils/tlsutil/url.go @@ -19,8 +19,9 @@ import ( "net/url" "strings" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) const ( diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index 46401aad1ff..2265053fdd2 100644 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -24,18 +24,20 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "golang.org/x/exp/slices" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" - "go.uber.org/zap" - "golang.org/x/exp/slices" ) const ( diff --git a/client/resource_group/controller/controller_test.go b/client/resource_group/controller/controller_test.go index 254df36020e..f0bdc62d6d3 100644 --- a/client/resource_group/controller/controller_test.go +++ b/client/resource_group/controller/controller_test.go @@ -24,11 +24,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index cd05adb7dfa..c8843da00bd 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -25,10 +25,12 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" - "github.com/tikv/pd/client/errs" "go.uber.org/zap" + + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" ) // Limit defines the maximum frequency of some events. diff --git a/client/resource_group/controller/model_test.go b/client/resource_group/controller/model_test.go index 594091da364..99cac79c25a 100644 --- a/client/resource_group/controller/model_test.go +++ b/client/resource_group/controller/model_test.go @@ -17,8 +17,9 @@ package controller import ( "testing" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestGetRUValueFromConsumption(t *testing.T) { diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index 3cf2970109f..0c481631b93 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -19,14 +19,16 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" - "go.uber.org/zap" ) type actionType int diff --git a/client/servicediscovery/pd_service_discovery.go b/client/servicediscovery/pd_service_discovery.go index bd4e442030a..619d4196408 100644 --- a/client/servicediscovery/pd_service_discovery.go +++ b/client/servicediscovery/pd_service_discovery.go @@ -25,21 +25,23 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/retry" "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" ) const ( diff --git a/client/servicediscovery/pd_service_discovery_test.go b/client/servicediscovery/pd_service_discovery_test.go index e8e7ef14e44..dc0a0bd4511 100644 --- a/client/servicediscovery/pd_service_discovery_test.go +++ b/client/servicediscovery/pd_service_discovery_test.go @@ -25,21 +25,23 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/client/errs" - "github.com/tikv/pd/client/opt" - "github.com/tikv/pd/client/pkg/utils/grpcutil" - "github.com/tikv/pd/client/pkg/utils/testutil" - "github.com/tikv/pd/client/pkg/utils/tlsutil" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" pb "google.golang.org/grpc/examples/helloworld/helloworld" "google.golang.org/grpc/health" healthpb "google.golang.org/grpc/health/grpc_health_v1" "google.golang.org/grpc/metadata" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/pdpb" + + "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/opt" + "github.com/tikv/pd/client/pkg/utils/grpcutil" + "github.com/tikv/pd/client/pkg/utils/testutil" + "github.com/tikv/pd/client/pkg/utils/tlsutil" ) type testGRPCServer struct { diff --git a/client/servicediscovery/tso_service_discovery.go b/client/servicediscovery/tso_service_discovery.go index 3fd27837f95..1d2130db804 100644 --- a/client/servicediscovery/tso_service_discovery.go +++ b/client/servicediscovery/tso_service_discovery.go @@ -25,17 +25,19 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/utils/grpcutil" - "go.uber.org/zap" - "google.golang.org/grpc" ) const ( diff --git a/cmd/pd-server/main.go b/cmd/pd-server/main.go index 0181d4c47aa..165bcd2a12f 100644 --- a/cmd/pd-server/main.go +++ b/cmd/pd-server/main.go @@ -22,8 +22,11 @@ import ( "syscall" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" - "github.com/pingcap/log" "github.com/spf13/cobra" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/autoscaling" "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/pkg/errs" @@ -41,7 +44,6 @@ import ( "github.com/tikv/pd/server/apiv2" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/join" - "go.uber.org/zap" // register microservice API _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/install" diff --git a/pkg/audit/audit.go b/pkg/audit/audit.go index f84d035f8c9..eba9722f44f 100644 --- a/pkg/audit/audit.go +++ b/pkg/audit/audit.go @@ -17,10 +17,12 @@ package audit import ( "net/http" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" - "github.com/tikv/pd/pkg/utils/requestutil" "go.uber.org/zap" + + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/utils/requestutil" ) const ( diff --git a/pkg/audit/audit_test.go b/pkg/audit/audit_test.go index 9066d81ebe3..c210d91f8b2 100644 --- a/pkg/audit/audit_test.go +++ b/pkg/audit/audit_test.go @@ -27,6 +27,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/requestutil" "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/autoscaling/calculation.go b/pkg/autoscaling/calculation.go index 43aa2972ed8..41db5f6cf78 100644 --- a/pkg/autoscaling/calculation.go +++ b/pkg/autoscaling/calculation.go @@ -20,17 +20,19 @@ import ( "strings" "time" + promClient "github.com/prometheus/client_golang/api" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - promClient "github.com/prometheus/client_golang/api" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/filter" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/autoscaling/calculation_test.go b/pkg/autoscaling/calculation_test.go index 9eb4ad648df..c2a70f8920e 100644 --- a/pkg/autoscaling/calculation_test.go +++ b/pkg/autoscaling/calculation_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/autoscaling/handler.go b/pkg/autoscaling/handler.go index ea248fdcc55..52b342f4a85 100644 --- a/pkg/autoscaling/handler.go +++ b/pkg/autoscaling/handler.go @@ -19,9 +19,10 @@ import ( "io" "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // HTTPHandler is a handler to handle the auto scaling HTTP request. diff --git a/pkg/autoscaling/prometheus.go b/pkg/autoscaling/prometheus.go index 91b813b6ef2..43c47c8c898 100644 --- a/pkg/autoscaling/prometheus.go +++ b/pkg/autoscaling/prometheus.go @@ -22,13 +22,15 @@ import ( "strings" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" promClient "github.com/prometheus/client_golang/api" promAPI "github.com/prometheus/client_golang/api/prometheus/v1" promModel "github.com/prometheus/common/model" - "github.com/tikv/pd/pkg/errs" "go.uber.org/zap" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) const ( diff --git a/pkg/autoscaling/service.go b/pkg/autoscaling/service.go index a8d84b2e598..304f1aa1188 100644 --- a/pkg/autoscaling/service.go +++ b/pkg/autoscaling/service.go @@ -18,11 +18,12 @@ import ( "context" "net/http" + "github.com/unrolled/render" + "github.com/urfave/negroni" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/serverapi" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "github.com/urfave/negroni" ) const autoScalingPrefix = "/autoscaling" diff --git a/pkg/autoscaling/types.go b/pkg/autoscaling/types.go index 53c614312b2..bee0d157191 100644 --- a/pkg/autoscaling/types.go +++ b/pkg/autoscaling/types.go @@ -20,8 +20,9 @@ import ( "regexp" "strings" - "github.com/tikv/pd/pkg/utils/etcdutil" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/pkg/utils/etcdutil" ) // Strategy within a HTTP request provides rules and resources to help make decision for auto scaling. diff --git a/pkg/cache/cache_test.go b/pkg/cache/cache_test.go index 75f26cfed33..d7d7365a344 100644 --- a/pkg/cache/cache_test.go +++ b/pkg/cache/cache_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/cache/ttl.go b/pkg/cache/ttl.go index 2aa39f6c6fd..ba7e3b67330 100644 --- a/pkg/cache/ttl.go +++ b/pkg/cache/ttl.go @@ -18,10 +18,12 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type ttlCacheItem struct { diff --git a/pkg/cgroup/cgmon.go b/pkg/cgroup/cgmon.go index 407e50f50c7..4be033facb1 100644 --- a/pkg/cgroup/cgmon.go +++ b/pkg/cgroup/cgmon.go @@ -21,10 +21,12 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/shirou/gopsutil/v3/mem" - bs "github.com/tikv/pd/pkg/basicserver" "go.uber.org/zap" + + "github.com/pingcap/log" + + bs "github.com/tikv/pd/pkg/basicserver" ) const ( diff --git a/pkg/cgroup/cgroup.go b/pkg/cgroup/cgroup.go index 133bd3158c8..0093020a0ac 100644 --- a/pkg/cgroup/cgroup.go +++ b/pkg/cgroup/cgroup.go @@ -26,9 +26,10 @@ import ( "strconv" "strings" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "go.uber.org/zap" ) // CPUQuotaStatus presents the status of how CPU quota is used diff --git a/pkg/core/factory_test.go b/pkg/core/factory_test.go index 9f0ba883885..a81da9af081 100644 --- a/pkg/core/factory_test.go +++ b/pkg/core/factory_test.go @@ -18,7 +18,9 @@ import ( "testing" "github.com/gogo/protobuf/proto" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/core/region.go b/pkg/core/region.go index b5ead86b3f1..5f5a4a5f2e0 100644 --- a/pkg/core/region.go +++ b/pkg/core/region.go @@ -29,17 +29,19 @@ import ( "github.com/docker/go-units" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/core/region_test.go b/pkg/core/region_test.go index 8c30efb2769..51ba5fe96dc 100644 --- a/pkg/core/region_test.go +++ b/pkg/core/region_test.go @@ -23,10 +23,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mock/mockid" diff --git a/pkg/core/region_tree.go b/pkg/core/region_tree.go index 12e2c5c8878..6efafd133cf 100644 --- a/pkg/core/region_tree.go +++ b/pkg/core/region_tree.go @@ -18,12 +18,14 @@ import ( "bytes" "math/rand" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/btree" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) type regionItem struct { diff --git a/pkg/core/region_tree_test.go b/pkg/core/region_tree_test.go index 0dedd91be9e..4d18394aa70 100644 --- a/pkg/core/region_tree_test.go +++ b/pkg/core/region_tree_test.go @@ -19,9 +19,10 @@ import ( "math/rand" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestRegionInfo(t *testing.T) { diff --git a/pkg/core/store.go b/pkg/core/store.go index 5edf59f6dce..9ec9a44bfc8 100644 --- a/pkg/core/store.go +++ b/pkg/core/store.go @@ -20,15 +20,17 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/core/store_option.go b/pkg/core/store_option.go index 3d05a0fb6e1..80fb09853e2 100644 --- a/pkg/core/store_option.go +++ b/pkg/core/store_option.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/core/store_stats.go b/pkg/core/store_stats.go index d68f8b8e43c..6c6a6420d56 100644 --- a/pkg/core/store_stats.go +++ b/pkg/core/store_stats.go @@ -16,6 +16,7 @@ package core import ( "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/core/store_stats_test.go b/pkg/core/store_stats_test.go index 9f2969cd81f..47f30dbd27b 100644 --- a/pkg/core/store_stats_test.go +++ b/pkg/core/store_stats_test.go @@ -18,9 +18,10 @@ import ( "testing" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestStoreStats(t *testing.T) { diff --git a/pkg/core/store_test.go b/pkg/core/store_test.go index 5cb324e5635..7c5aaa98289 100644 --- a/pkg/core/store_test.go +++ b/pkg/core/store_test.go @@ -21,9 +21,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/core/storelimit/limit_test.go b/pkg/core/storelimit/limit_test.go index cfe805935a5..eb0bde2732f 100644 --- a/pkg/core/storelimit/limit_test.go +++ b/pkg/core/storelimit/limit_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/dashboard/adapter/manager.go b/pkg/dashboard/adapter/manager.go index 293d8ad6549..648f5562419 100644 --- a/pkg/dashboard/adapter/manager.go +++ b/pkg/dashboard/adapter/manager.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/pingcap/tidb-dashboard/pkg/apiserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/server" diff --git a/pkg/dashboard/adapter/redirector.go b/pkg/dashboard/adapter/redirector.go index c1d845065b7..5d0fd1801cd 100644 --- a/pkg/dashboard/adapter/redirector.go +++ b/pkg/dashboard/adapter/redirector.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb-dashboard/pkg/apiserver" "github.com/pingcap/tidb-dashboard/pkg/utils" + "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/dashboard/distroutil/distro.go b/pkg/dashboard/distroutil/distro.go index a19db806d70..0bc7b1d031b 100644 --- a/pkg/dashboard/distroutil/distro.go +++ b/pkg/dashboard/distroutil/distro.go @@ -18,9 +18,10 @@ import ( "os" "path/filepath" + "go.uber.org/zap" + "github.com/pingcap/log" "github.com/pingcap/tidb-dashboard/util/distro" - "go.uber.org/zap" ) const ( diff --git a/pkg/dashboard/keyvisual/input/core.go b/pkg/dashboard/keyvisual/input/core.go index 3ca5f96cd81..0219d6e0b77 100644 --- a/pkg/dashboard/keyvisual/input/core.go +++ b/pkg/dashboard/keyvisual/input/core.go @@ -15,11 +15,13 @@ package input import ( + "go.uber.org/zap" + "github.com/pingcap/log" regionpkg "github.com/pingcap/tidb-dashboard/pkg/keyvisual/region" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/server" - "go.uber.org/zap" ) const limit = 1024 diff --git a/pkg/dashboard/uiserver/embedded_assets_rewriter.go b/pkg/dashboard/uiserver/embedded_assets_rewriter.go index d19db01936f..f0f9f0c1e51 100644 --- a/pkg/dashboard/uiserver/embedded_assets_rewriter.go +++ b/pkg/dashboard/uiserver/embedded_assets_rewriter.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb-dashboard/pkg/config" "github.com/pingcap/tidb-dashboard/pkg/uiserver" + "github.com/tikv/pd/pkg/dashboard/distroutil" ) diff --git a/pkg/election/leadership.go b/pkg/election/leadership.go index ec64a003c53..bb0ce7bf361 100644 --- a/pkg/election/leadership.go +++ b/pkg/election/leadership.go @@ -19,18 +19,20 @@ import ( "sync/atomic" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/election/leadership_test.go b/pkg/election/leadership_test.go index eab842ca6e5..fc158e6f73a 100644 --- a/pkg/election/leadership_test.go +++ b/pkg/election/leadership_test.go @@ -21,12 +21,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/pingcap/failpoint" + + "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/testutil" ) const defaultLeaseTimeout = 1 diff --git a/pkg/election/lease.go b/pkg/election/lease.go index 21bd43018b5..1dff5ea7a99 100644 --- a/pkg/election/lease.go +++ b/pkg/election/lease.go @@ -19,13 +19,15 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/election/lease_test.go b/pkg/election/lease_test.go index 3a02de97239..b099d0c0d5e 100644 --- a/pkg/election/lease_test.go +++ b/pkg/election/lease_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/encryption/config.go b/pkg/encryption/config.go index d4d257a8a43..38222c56b5d 100644 --- a/pkg/encryption/config.go +++ b/pkg/encryption/config.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/encryption/config_test.go b/pkg/encryption/config_test.go index 4134d46c2f3..2ab5ae68dfa 100644 --- a/pkg/encryption/config_test.go +++ b/pkg/encryption/config_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/encryption/crypter.go b/pkg/encryption/crypter.go index 7e69854c5a8..5c91228c150 100644 --- a/pkg/encryption/crypter.go +++ b/pkg/encryption/crypter.go @@ -22,6 +22,7 @@ import ( "io" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/crypter_test.go b/pkg/encryption/crypter_test.go index 9ac72bd7813..ff660cc1a9d 100644 --- a/pkg/encryption/crypter_test.go +++ b/pkg/encryption/crypter_test.go @@ -19,8 +19,9 @@ import ( "encoding/hex" "testing" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/encryptionpb" ) func TestEncryptionMethodSupported(t *testing.T) { diff --git a/pkg/encryption/key_manager.go b/pkg/encryption/key_manager.go index 621b1b9742f..54e5fa01b35 100644 --- a/pkg/encryption/key_manager.go +++ b/pkg/encryption/key_manager.go @@ -20,15 +20,17 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/encryption/key_manager_test.go b/pkg/encryption/key_manager_test.go index f387497c2e9..52e91296314 100644 --- a/pkg/encryption/key_manager_test.go +++ b/pkg/encryption/key_manager_test.go @@ -24,13 +24,15 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // #nosec G101 diff --git a/pkg/encryption/kms.go b/pkg/encryption/kms.go index 99dcf9619a3..3836cf53db6 100644 --- a/pkg/encryption/kms.go +++ b/pkg/encryption/kms.go @@ -22,7 +22,9 @@ import ( "github.com/aws/aws-sdk-go-v2/credentials/stscreds" "github.com/aws/aws-sdk-go-v2/service/kms" "github.com/aws/aws-sdk-go-v2/service/sts" + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/master_key.go b/pkg/encryption/master_key.go index 31d9a0cb591..aff480386ac 100644 --- a/pkg/encryption/master_key.go +++ b/pkg/encryption/master_key.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/master_key_test.go b/pkg/encryption/master_key_test.go index d6d7845284a..0a8e99bf75a 100644 --- a/pkg/encryption/master_key_test.go +++ b/pkg/encryption/master_key_test.go @@ -20,8 +20,9 @@ import ( "path/filepath" "testing" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/encryptionpb" ) func TestPlaintextMasterKey(t *testing.T) { diff --git a/pkg/encryption/region_crypter.go b/pkg/encryption/region_crypter.go index 458c5b67d7b..96826d8bef6 100644 --- a/pkg/encryption/region_crypter.go +++ b/pkg/encryption/region_crypter.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/encryption/region_crypter_test.go b/pkg/encryption/region_crypter_test.go index b1ca558063c..1b6f910b0d5 100644 --- a/pkg/encryption/region_crypter_test.go +++ b/pkg/encryption/region_crypter_test.go @@ -19,10 +19,11 @@ import ( "crypto/cipher" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" ) type testKeyManager struct { diff --git a/pkg/errs/errs.go b/pkg/errs/errs.go index 5746b282f10..84cc90312d1 100644 --- a/pkg/errs/errs.go +++ b/pkg/errs/errs.go @@ -15,9 +15,10 @@ package errs import ( - "github.com/pingcap/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + + "github.com/pingcap/errors" ) // ZapError is used to make the log output easier. diff --git a/pkg/errs/errs_test.go b/pkg/errs/errs_test.go index 01b7de461b8..80722f4a8c4 100644 --- a/pkg/errs/errs_test.go +++ b/pkg/errs/errs_test.go @@ -20,10 +20,11 @@ import ( "strings" "testing" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "go.uber.org/zap" + + "github.com/pingcap/errors" + "github.com/pingcap/log" ) // testingWriter is a WriteSyncer that writes to the the messages. diff --git a/pkg/gc/safepoint.go b/pkg/gc/safepoint.go index c1b4687e109..5bc64ae9a90 100644 --- a/pkg/gc/safepoint.go +++ b/pkg/gc/safepoint.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/config" diff --git a/pkg/gc/safepoint_test.go b/pkg/gc/safepoint_test.go index 62ce9c086fc..b67f7b4fe44 100644 --- a/pkg/gc/safepoint_test.go +++ b/pkg/gc/safepoint_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/server/config" diff --git a/pkg/gc/safepoint_v2.go b/pkg/gc/safepoint_v2.go index 665249bcab0..449be8f3d59 100644 --- a/pkg/gc/safepoint_v2.go +++ b/pkg/gc/safepoint_v2.go @@ -18,16 +18,18 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) var ( diff --git a/pkg/gctuner/memory_limit_tuner.go b/pkg/gctuner/memory_limit_tuner.go index 8a852b191d8..10e28842938 100644 --- a/pkg/gctuner/memory_limit_tuner.go +++ b/pkg/gctuner/memory_limit_tuner.go @@ -20,12 +20,14 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + util "github.com/tikv/pd/pkg/gogc" "github.com/tikv/pd/pkg/memory" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // GlobalMemoryLimitTuner only allow one memory limit tuner in one process diff --git a/pkg/gctuner/memory_limit_tuner_test.go b/pkg/gctuner/memory_limit_tuner_test.go index 5e5f84ccbac..6c0aaca685d 100644 --- a/pkg/gctuner/memory_limit_tuner_test.go +++ b/pkg/gctuner/memory_limit_tuner_test.go @@ -20,8 +20,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/memory" ) diff --git a/pkg/gctuner/tuner.go b/pkg/gctuner/tuner.go index 74932fe174b..bc63b36c5a2 100644 --- a/pkg/gctuner/tuner.go +++ b/pkg/gctuner/tuner.go @@ -20,9 +20,11 @@ import ( "strconv" "sync/atomic" + "go.uber.org/zap" + "github.com/pingcap/log" + util "github.com/tikv/pd/pkg/gogc" - "go.uber.org/zap" ) var ( diff --git a/pkg/id/id.go b/pkg/id/id.go index eb2788fc656..cb38c23268d 100644 --- a/pkg/id/id.go +++ b/pkg/id/id.go @@ -15,16 +15,18 @@ package id import ( - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) type label string diff --git a/pkg/id/id_test.go b/pkg/id/id_test.go index e08c0e93367..9cd8493c430 100644 --- a/pkg/id/id_test.go +++ b/pkg/id/id_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 08bb51da936..4a50f36169f 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -20,10 +20,13 @@ import ( "strconv" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/keyspace/keyspace_test.go b/pkg/keyspace/keyspace_test.go index 3c259649cd3..57aeb70341d 100644 --- a/pkg/keyspace/keyspace_test.go +++ b/pkg/keyspace/keyspace_test.go @@ -23,10 +23,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index fa26b4cd0cb..dc414cfad3a 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -23,10 +23,15 @@ import ( "sync" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/balancer" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -38,9 +43,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 5878d7d907f..c615627be71 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 91d07676205..95b3162eed0 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/keyspace/util_test.go b/pkg/keyspace/util_test.go index ab544b21a5d..f938904c709 100644 --- a/pkg/keyspace/util_test.go +++ b/pkg/keyspace/util_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/mcs/discovery/discover.go b/pkg/mcs/discovery/discover.go index d4234df893d..f46c72495c6 100644 --- a/pkg/mcs/discovery/discover.go +++ b/pkg/mcs/discovery/discover.go @@ -15,15 +15,17 @@ package discovery import ( + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Discover is used to get all the service instances of the specified service name. diff --git a/pkg/mcs/discovery/discover_test.go b/pkg/mcs/discovery/discover_test.go index fd66ddcad18..81af0d524b7 100644 --- a/pkg/mcs/discovery/discover_test.go +++ b/pkg/mcs/discovery/discover_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/mcs/discovery/register.go b/pkg/mcs/discovery/register.go index 5ab0ceabfce..e62239a4694 100644 --- a/pkg/mcs/discovery/register.go +++ b/pkg/mcs/discovery/register.go @@ -19,12 +19,14 @@ import ( "fmt" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // DefaultLeaseInSeconds is the default lease time in seconds. diff --git a/pkg/mcs/discovery/register_test.go b/pkg/mcs/discovery/register_test.go index bdaf7e379a4..b03543f624e 100644 --- a/pkg/mcs/discovery/register_test.go +++ b/pkg/mcs/discovery/register_test.go @@ -22,10 +22,11 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/testutil" ) func TestRegister(t *testing.T) { diff --git a/pkg/mcs/discovery/registry_entry.go b/pkg/mcs/discovery/registry_entry.go index db4ac44a2cc..887a8eb7aea 100644 --- a/pkg/mcs/discovery/registry_entry.go +++ b/pkg/mcs/discovery/registry_entry.go @@ -17,8 +17,9 @@ package discovery import ( "encoding/json" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) // ServiceRegistryEntry is the registry entry of a service diff --git a/pkg/mcs/metastorage/server/grpc_service.go b/pkg/mcs/metastorage/server/grpc_service.go index 32b3788906d..00f4efb56fd 100644 --- a/pkg/mcs/metastorage/server/grpc_service.go +++ b/pkg/mcs/metastorage/server/grpc_service.go @@ -19,17 +19,19 @@ import ( "fmt" "net/http" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var ( diff --git a/pkg/mcs/metastorage/server/manager.go b/pkg/mcs/metastorage/server/manager.go index 49fc58c6b7d..0c03e4c7d03 100644 --- a/pkg/mcs/metastorage/server/manager.go +++ b/pkg/mcs/metastorage/server/manager.go @@ -15,12 +15,14 @@ package server import ( + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Manager is the manager of resource group. diff --git a/pkg/mcs/registry/registry.go b/pkg/mcs/registry/registry.go index c6470b34dc4..6a01f091e52 100644 --- a/pkg/mcs/registry/registry.go +++ b/pkg/mcs/registry/registry.go @@ -20,10 +20,12 @@ import ( "fmt" "net/http" - "github.com/pingcap/log" - bs "github.com/tikv/pd/pkg/basicserver" "go.uber.org/zap" "google.golang.org/grpc" + + "github.com/pingcap/log" + + bs "github.com/tikv/pd/pkg/basicserver" ) var ( diff --git a/pkg/mcs/resourcemanager/server/apis/v1/api.go b/pkg/mcs/resourcemanager/server/apis/v1/api.go index 891072e898f..e142dbbc69a 100644 --- a/pkg/mcs/resourcemanager/server/apis/v1/api.go +++ b/pkg/mcs/resourcemanager/server/apis/v1/api.go @@ -25,7 +25,9 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" + rmserver "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/pkg/mcs/resourcemanager/server/config.go b/pkg/mcs/resourcemanager/server/config.go index 360f3b169b9..415a87d5e79 100644 --- a/pkg/mcs/resourcemanager/server/config.go +++ b/pkg/mcs/resourcemanager/server/config.go @@ -22,16 +22,18 @@ import ( "time" "github.com/BurntSushi/toml" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/grpc_service.go b/pkg/mcs/resourcemanager/server/grpc_service.go index 21681bc0759..6c0d7ce0120 100644 --- a/pkg/mcs/resourcemanager/server/grpc_service.go +++ b/pkg/mcs/resourcemanager/server/grpc_service.go @@ -20,17 +20,19 @@ import ( "net/http" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var ( diff --git a/pkg/mcs/resourcemanager/server/manager.go b/pkg/mcs/resourcemanager/server/manager.go index 7f7e710b3fb..2ccfd44c418 100644 --- a/pkg/mcs/resourcemanager/server/manager.go +++ b/pkg/mcs/resourcemanager/server/manager.go @@ -23,11 +23,14 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/jsonutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/metrics_test.go b/pkg/mcs/resourcemanager/server/metrics_test.go index d69d364b64b..4c3ec7ce5ef 100644 --- a/pkg/mcs/resourcemanager/server/metrics_test.go +++ b/pkg/mcs/resourcemanager/server/metrics_test.go @@ -18,8 +18,9 @@ import ( "fmt" "testing" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestMaxPerSecCostTracker(t *testing.T) { diff --git a/pkg/mcs/resourcemanager/server/resource_group.go b/pkg/mcs/resourcemanager/server/resource_group.go index 6a5d06da6f7..65d2959e870 100644 --- a/pkg/mcs/resourcemanager/server/resource_group.go +++ b/pkg/mcs/resourcemanager/server/resource_group.go @@ -20,12 +20,14 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // ResourceGroup is the definition of a resource group, for REST API. diff --git a/pkg/mcs/resourcemanager/server/resource_group_test.go b/pkg/mcs/resourcemanager/server/resource_group_test.go index 96b15d14293..8f058a212bb 100644 --- a/pkg/mcs/resourcemanager/server/resource_group_test.go +++ b/pkg/mcs/resourcemanager/server/resource_group_test.go @@ -5,8 +5,9 @@ import ( "testing" "github.com/brianvoe/gofakeit/v6" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestPatchResourceGroup(t *testing.T) { diff --git a/pkg/mcs/resourcemanager/server/token_buckets.go b/pkg/mcs/resourcemanager/server/token_buckets.go index f72d0dfcc88..50dc78c9d68 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets.go +++ b/pkg/mcs/resourcemanager/server/token_buckets.go @@ -19,9 +19,10 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/token_buckets_test.go b/pkg/mcs/resourcemanager/server/token_buckets_test.go index 3d9cbd3f628..b56ccb6ab96 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets_test.go +++ b/pkg/mcs/resourcemanager/server/token_buckets_test.go @@ -20,8 +20,9 @@ import ( "testing" "time" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestGroupTokenBucketUpdateAndPatch(t *testing.T) { diff --git a/pkg/mcs/scheduling/server/apis/v1/api.go b/pkg/mcs/scheduling/server/apis/v1/api.go index c374456a6eb..3d2d0005a24 100644 --- a/pkg/mcs/scheduling/server/apis/v1/api.go +++ b/pkg/mcs/scheduling/server/apis/v1/api.go @@ -27,8 +27,11 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/unrolled/render" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" scheserver "github.com/tikv/pd/pkg/mcs/scheduling/server" mcsutils "github.com/tikv/pd/pkg/mcs/utils" @@ -44,7 +47,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - "github.com/unrolled/render" ) // APIPathPrefix is the prefix of the API path. diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index e4949446da9..20e5acca379 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -7,12 +7,15 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Cluster is used to manage all information for scheduling purpose. diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index 45a606720a0..784d1f45a82 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -27,10 +27,13 @@ import ( "github.com/BurntSushi/toml" "github.com/coreos/go-semver/semver" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -43,7 +46,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index f0af61c9dd2..f499a0d7d50 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -23,15 +23,17 @@ import ( "time" "github.com/coreos/go-semver/semver" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any configuration changes. diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index f4fe606b403..440b2d47d4f 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -21,10 +21,16 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -33,10 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // gRPC errors diff --git a/pkg/mcs/scheduling/server/meta/watcher.go b/pkg/mcs/scheduling/server/meta/watcher.go index 9d54a636b9e..27fe6687f3d 100644 --- a/pkg/mcs/scheduling/server/meta/watcher.go +++ b/pkg/mcs/scheduling/server/meta/watcher.go @@ -20,15 +20,17 @@ import ( "sync" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any meta changes. diff --git a/pkg/mcs/scheduling/server/rule/watcher.go b/pkg/mcs/scheduling/server/rule/watcher.go index 790dd9c2f81..cc6480a0cb4 100644 --- a/pkg/mcs/scheduling/server/rule/watcher.go +++ b/pkg/mcs/scheduling/server/rule/watcher.go @@ -19,7 +19,12 @@ import ( "strings" "sync" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/checker" "github.com/tikv/pd/pkg/schedule/labeler" @@ -27,9 +32,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any Placement Rule changes. diff --git a/pkg/mcs/scheduling/server/rule/watcher_test.go b/pkg/mcs/scheduling/server/rule/watcher_test.go index 40469eef2a8..5b3d49a53f1 100644 --- a/pkg/mcs/scheduling/server/rule/watcher_test.go +++ b/pkg/mcs/scheduling/server/rule/watcher_test.go @@ -24,14 +24,15 @@ import ( "time" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" ) const ( diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 31c8d307adb..8c9972d5eec 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -28,6 +28,10 @@ import ( "time" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/spf13/cobra" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -35,7 +39,7 @@ import ( "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/spf13/cobra" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" @@ -62,8 +66,6 @@ import ( "github.com/tikv/pd/pkg/utils/memberutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" ) var _ bs.Server = (*Server)(nil) diff --git a/pkg/mcs/scheduling/server/testutil.go b/pkg/mcs/scheduling/server/testutil.go index 312365c81ab..794a1bf520c 100644 --- a/pkg/mcs/scheduling/server/testutil.go +++ b/pkg/mcs/scheduling/server/testutil.go @@ -18,9 +18,11 @@ import ( "context" "os" - "github.com/pingcap/log" "github.com/spf13/pflag" "github.com/stretchr/testify/require" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/pkg/mcs/server/server.go b/pkg/mcs/server/server.go index fef05a85012..9692b52beb3 100644 --- a/pkg/mcs/server/server.go +++ b/pkg/mcs/server/server.go @@ -23,11 +23,13 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "google.golang.org/grpc" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/grpcutil" - clientv3 "go.etcd.io/etcd/client/v3" - "google.golang.org/grpc" ) // BaseServer is a basic server that provides some common functionality. diff --git a/pkg/mcs/tso/server/apis/v1/api.go b/pkg/mcs/tso/server/apis/v1/api.go index ec8782eb522..d659c13edca 100644 --- a/pkg/mcs/tso/server/apis/v1/api.go +++ b/pkg/mcs/tso/server/apis/v1/api.go @@ -23,8 +23,12 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" tsoserver "github.com/tikv/pd/pkg/mcs/tso/server" "github.com/tikv/pd/pkg/mcs/utils" @@ -34,8 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" "github.com/tikv/pd/pkg/utils/logutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/tso/server/config.go b/pkg/mcs/tso/server/config.go index 2aaa54114da..f2ee3c36a7b 100644 --- a/pkg/mcs/tso/server/config.go +++ b/pkg/mcs/tso/server/config.go @@ -22,16 +22,18 @@ import ( "time" "github.com/BurntSushi/toml" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/tso/server/config_test.go b/pkg/mcs/tso/server/config_test.go index 2bafec30aa9..08686bd5208 100644 --- a/pkg/mcs/tso/server/config_test.go +++ b/pkg/mcs/tso/server/config_test.go @@ -21,6 +21,7 @@ import ( "github.com/BurntSushi/toml" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mcs/utils/constant" ) diff --git a/pkg/mcs/tso/server/grpc_service.go b/pkg/mcs/tso/server/grpc_service.go index 3419fd16221..59abed67213 100644 --- a/pkg/mcs/tso/server/grpc_service.go +++ b/pkg/mcs/tso/server/grpc_service.go @@ -21,16 +21,18 @@ import ( "strconv" "time" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // gRPC errors diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index d2974075e94..34f51573baf 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -27,12 +27,18 @@ import ( "time" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/spf13/cobra" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/diagnosticspb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/spf13/cobra" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -49,10 +55,6 @@ import ( "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var _ bs.Server = (*Server)(nil) diff --git a/pkg/mcs/tso/server/testutil.go b/pkg/mcs/tso/server/testutil.go index 5dcfd4759b9..5ffcac48edb 100644 --- a/pkg/mcs/tso/server/testutil.go +++ b/pkg/mcs/tso/server/testutil.go @@ -17,11 +17,12 @@ package server import ( "strings" - "github.com/pingcap/kvproto/pkg/tsopb" "github.com/spf13/pflag" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/kvproto/pkg/tsopb" ) // MustNewGrpcClient must create a new TSO grpc client. diff --git a/pkg/mcs/utils/expected_primary.go b/pkg/mcs/utils/expected_primary.go index c39345b004e..b8a317ed251 100644 --- a/pkg/mcs/utils/expected_primary.go +++ b/pkg/mcs/utils/expected_primary.go @@ -20,8 +20,12 @@ import ( "math/rand" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -29,8 +33,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // GetExpectedPrimaryFlag gets the expected primary flag. diff --git a/pkg/mcs/utils/util.go b/pkg/mcs/utils/util.go index 253c846d167..68e7edb3d69 100644 --- a/pkg/mcs/utils/util.go +++ b/pkg/mcs/utils/util.go @@ -25,11 +25,18 @@ import ( "time" "github.com/gin-gonic/gin" - "github.com/pingcap/kvproto/pkg/diagnosticspb" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + + "github.com/pingcap/kvproto/pkg/diagnosticspb" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -40,11 +47,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/versioninfo" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) // PromHandler is a handler to get prometheus metrics. diff --git a/pkg/member/member.go b/pkg/member/member.go index 04e55c1a647..8b388cdee6a 100644 --- a/pkg/member/member.go +++ b/pkg/member/member.go @@ -24,18 +24,20 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) const ( diff --git a/pkg/member/participant.go b/pkg/member/participant.go index f3399f5d900..5d9129bcad5 100644 --- a/pkg/member/participant.go +++ b/pkg/member/participant.go @@ -19,16 +19,18 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) type leadershipCheckFunc func(*election.Leadership) bool diff --git a/pkg/memory/meminfo.go b/pkg/memory/meminfo.go index 64505e2e5f3..7ed1afb579b 100644 --- a/pkg/memory/meminfo.go +++ b/pkg/memory/meminfo.go @@ -17,14 +17,16 @@ package memory import ( "time" + "github.com/shirou/gopsutil/v3/mem" + "go.uber.org/zap" + "golang.org/x/exp/constraints" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/shirou/gopsutil/v3/mem" + "github.com/tikv/pd/pkg/cgroup" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "golang.org/x/exp/constraints" ) // MemTotal returns the total amount of RAM on this system diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index 5fe17e5e723..45d8e35a0bc 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -21,10 +21,12 @@ import ( "time" "github.com/docker/go-units" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/mock/mockhbstream/mockhbstream.go b/pkg/mock/mockhbstream/mockhbstream.go index ac8f246f86a..848f134c8a0 100644 --- a/pkg/mock/mockhbstream/mockhbstream.go +++ b/pkg/mock/mockhbstream/mockhbstream.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/hbstream" ) diff --git a/pkg/mock/mockhbstream/mockhbstream_test.go b/pkg/mock/mockhbstream/mockhbstream_test.go index 87a39b028b9..e4ffe8a9b20 100644 --- a/pkg/mock/mockhbstream/mockhbstream_test.go +++ b/pkg/mock/mockhbstream/mockhbstream_test.go @@ -18,10 +18,12 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/hbstream" diff --git a/pkg/mock/mockserver/mockserver.go b/pkg/mock/mockserver/mockserver.go index d79d79ffa03..3ba2abfca3f 100644 --- a/pkg/mock/mockserver/mockserver.go +++ b/pkg/mock/mockserver/mockserver.go @@ -18,6 +18,7 @@ import ( "context" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/grpcutil" diff --git a/pkg/ratelimit/controller_test.go b/pkg/ratelimit/controller_test.go index 5efa6ec1190..f42169c9902 100644 --- a/pkg/ratelimit/controller_test.go +++ b/pkg/ratelimit/controller_test.go @@ -21,8 +21,9 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) type changeAndResult struct { diff --git a/pkg/ratelimit/limiter.go b/pkg/ratelimit/limiter.go index e312066dc56..0fc3a4e6bd1 100644 --- a/pkg/ratelimit/limiter.go +++ b/pkg/ratelimit/limiter.go @@ -17,9 +17,10 @@ package ratelimit import ( "math" + "golang.org/x/time/rate" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" - "golang.org/x/time/rate" ) // DoneFunc is done function. diff --git a/pkg/ratelimit/limiter_test.go b/pkg/ratelimit/limiter_test.go index 256f9ea9ab4..2bb967677bb 100644 --- a/pkg/ratelimit/limiter_test.go +++ b/pkg/ratelimit/limiter_test.go @@ -20,8 +20,9 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) type releaseUtil struct { diff --git a/pkg/ratelimit/ratelimiter.go b/pkg/ratelimit/ratelimiter.go index b88127915c9..1c2a6a0f7b0 100644 --- a/pkg/ratelimit/ratelimiter.go +++ b/pkg/ratelimit/ratelimiter.go @@ -18,8 +18,9 @@ import ( "context" "time" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) // RateLimiter is a rate limiter based on `golang.org/x/time/rate`. diff --git a/pkg/ratelimit/runner.go b/pkg/ratelimit/runner.go index 1d65ff6a568..211a4f71be1 100644 --- a/pkg/ratelimit/runner.go +++ b/pkg/ratelimit/runner.go @@ -20,9 +20,10 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + + "github.com/pingcap/log" ) // RegionHeartbeatStageName is the name of the stage of the region heartbeat. diff --git a/pkg/replication/replication_mode.go b/pkg/replication/replication_mode.go index cba83cf1ebb..856a68a9b09 100644 --- a/pkg/replication/replication_mode.go +++ b/pkg/replication/replication_mode.go @@ -24,9 +24,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" pb "github.com/pingcap/kvproto/pkg/replication_modepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/config" - "go.uber.org/zap" ) const ( diff --git a/pkg/replication/replication_mode_test.go b/pkg/replication/replication_mode_test.go index 243d7f7d8f1..0a921bccf3a 100644 --- a/pkg/replication/replication_mode_test.go +++ b/pkg/replication/replication_mode_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/pdpb" pb "github.com/pingcap/kvproto/pkg/replication_modepb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/response/region.go b/pkg/response/region.go index 6db7f135ad8..b635780e4f2 100644 --- a/pkg/response/region.go +++ b/pkg/response/region.go @@ -18,9 +18,11 @@ import ( "context" "github.com/mailru/easyjson/jwriter" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/response/region_test.go b/pkg/response/region_test.go index de6daa2c2fe..8da4c739f79 100644 --- a/pkg/response/region_test.go +++ b/pkg/response/region_test.go @@ -18,9 +18,10 @@ import ( "encoding/json" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestPeer(t *testing.T) { diff --git a/pkg/response/store.go b/pkg/response/store.go index 8bff1e75e42..25dc7fc2176 100644 --- a/pkg/response/store.go +++ b/pkg/response/store.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/pkg/schedule/checker/checker_controller.go b/pkg/schedule/checker/checker_controller.go index 49d097daea6..aa15ee03d6f 100644 --- a/pkg/schedule/checker/checker_controller.go +++ b/pkg/schedule/checker/checker_controller.go @@ -22,8 +22,11 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/checker/joint_state_checker.go b/pkg/schedule/checker/joint_state_checker.go index 2122044e64a..4dc3922906a 100644 --- a/pkg/schedule/checker/joint_state_checker.go +++ b/pkg/schedule/checker/joint_state_checker.go @@ -16,6 +16,7 @@ package checker import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/checker/joint_state_checker_test.go b/pkg/schedule/checker/joint_state_checker_test.go index 1b9436b65fd..f649c737284 100644 --- a/pkg/schedule/checker/joint_state_checker_test.go +++ b/pkg/schedule/checker/joint_state_checker_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/checker/learner_checker.go b/pkg/schedule/checker/learner_checker.go index 8590904a760..6d830c39ad8 100644 --- a/pkg/schedule/checker/learner_checker.go +++ b/pkg/schedule/checker/learner_checker.go @@ -16,6 +16,7 @@ package checker import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/checker/learner_checker_test.go b/pkg/schedule/checker/learner_checker_test.go index b5d994d4201..3a559b86958 100644 --- a/pkg/schedule/checker/learner_checker_test.go +++ b/pkg/schedule/checker/learner_checker_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/checker/merge_checker.go b/pkg/schedule/checker/merge_checker.go index bf7fe4f2496..571ee134da0 100644 --- a/pkg/schedule/checker/merge_checker.go +++ b/pkg/schedule/checker/merge_checker.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" diff --git a/pkg/schedule/checker/merge_checker_test.go b/pkg/schedule/checker/merge_checker_test.go index 00aaafa2cfd..3737e6cc0c2 100644 --- a/pkg/schedule/checker/merge_checker_test.go +++ b/pkg/schedule/checker/merge_checker_test.go @@ -20,8 +20,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/utils/operatorutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/pkg/schedule/checker/priority_inspector_test.go b/pkg/schedule/checker/priority_inspector_test.go index 5a5d8079d93..6de76b5ccb1 100644 --- a/pkg/schedule/checker/priority_inspector_test.go +++ b/pkg/schedule/checker/priority_inspector_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" ) diff --git a/pkg/schedule/checker/replica_checker.go b/pkg/schedule/checker/replica_checker.go index fabee683c92..f1d6efe15ba 100644 --- a/pkg/schedule/checker/replica_checker.go +++ b/pkg/schedule/checker/replica_checker.go @@ -19,8 +19,11 @@ import ( "math/rand" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -29,7 +32,6 @@ import ( sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/checker/replica_checker_test.go b/pkg/schedule/checker/replica_checker_test.go index da04fb6d768..3f0f6b67d8d 100644 --- a/pkg/schedule/checker/replica_checker_test.go +++ b/pkg/schedule/checker/replica_checker_test.go @@ -20,10 +20,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/checker/replica_strategy.go b/pkg/schedule/checker/replica_strategy.go index ffb25f90ad5..3bf2c955af7 100644 --- a/pkg/schedule/checker/replica_strategy.go +++ b/pkg/schedule/checker/replica_strategy.go @@ -17,12 +17,14 @@ package checker import ( "math/rand" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" - "go.uber.org/zap" ) // ReplicaStrategy collects some utilities to manipulate region peers. It diff --git a/pkg/schedule/checker/rule_checker.go b/pkg/schedule/checker/rule_checker.go index 2d06f84fdfe..28d5988ce1c 100644 --- a/pkg/schedule/checker/rule_checker.go +++ b/pkg/schedule/checker/rule_checker.go @@ -21,9 +21,12 @@ import ( "math/rand" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) const maxPendingListLen = 100000 diff --git a/pkg/schedule/checker/rule_checker_test.go b/pkg/schedule/checker/rule_checker_test.go index 5ac67122de1..0b37ed57e7e 100644 --- a/pkg/schedule/checker/rule_checker_test.go +++ b/pkg/schedule/checker/rule_checker_test.go @@ -17,16 +17,17 @@ package checker import ( "context" "fmt" - "strconv" "strings" "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/pkg/schedule/checker/split_checker.go b/pkg/schedule/checker/split_checker.go index 3cc1664b6cc..91487949821 100644 --- a/pkg/schedule/checker/split_checker.go +++ b/pkg/schedule/checker/split_checker.go @@ -17,6 +17,7 @@ package checker import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/checker/split_checker_test.go b/pkg/schedule/checker/split_checker_test.go index 40357d2408c..be3cce84f0d 100644 --- a/pkg/schedule/checker/split_checker_test.go +++ b/pkg/schedule/checker/split_checker_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/schedule/config/config.go b/pkg/schedule/config/config.go index 5e8f2c587ac..f00cd8a943b 100644 --- a/pkg/schedule/config/config.go +++ b/pkg/schedule/config/config.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/configutil" diff --git a/pkg/schedule/config/config_provider.go b/pkg/schedule/config/config_provider.go index 5c1be1089e9..a18051ca38e 100644 --- a/pkg/schedule/config/config_provider.go +++ b/pkg/schedule/config/config_provider.go @@ -19,7 +19,9 @@ import ( "time" "github.com/coreos/go-semver/semver" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/config/store_config.go b/pkg/schedule/config/store_config.go index b7676311f3d..19cca753d1d 100644 --- a/pkg/schedule/config/store_config.go +++ b/pkg/schedule/config/store_config.go @@ -18,10 +18,12 @@ import ( "encoding/json" "reflect" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) var ( diff --git a/pkg/schedule/config/util_test.go b/pkg/schedule/config/util_test.go index 31ab3ccf2a6..bda6b9441ad 100644 --- a/pkg/schedule/config/util_test.go +++ b/pkg/schedule/config/util_test.go @@ -17,8 +17,9 @@ package config import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func TestValidateLabels(t *testing.T) { diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 739ca5c84b6..e792560cb37 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -20,9 +20,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/checker" @@ -39,7 +42,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/filter/candidates_test.go b/pkg/schedule/filter/candidates_test.go index fd03b42ace0..ce8be5ef72e 100644 --- a/pkg/schedule/filter/candidates_test.go +++ b/pkg/schedule/filter/candidates_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/plan" diff --git a/pkg/schedule/filter/counter_test.go b/pkg/schedule/filter/counter_test.go index 7c7acc5e9a5..74ff194c47a 100644 --- a/pkg/schedule/filter/counter_test.go +++ b/pkg/schedule/filter/counter_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/filter/filters.go b/pkg/schedule/filter/filters.go index 4ea59935109..efb27c3ec6d 100644 --- a/pkg/schedule/filter/filters.go +++ b/pkg/schedule/filter/filters.go @@ -17,8 +17,11 @@ package filter import ( "strconv" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) // SelectSourceStores selects stores that be selected as source store from the list. diff --git a/pkg/schedule/filter/filters_test.go b/pkg/schedule/filter/filters_test.go index f061a472d65..35ba2c33589 100644 --- a/pkg/schedule/filter/filters_test.go +++ b/pkg/schedule/filter/filters_test.go @@ -19,9 +19,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/schedule/filter/healthy_test.go b/pkg/schedule/filter/healthy_test.go index 15588352554..dd5ab8e958d 100644 --- a/pkg/schedule/filter/healthy_test.go +++ b/pkg/schedule/filter/healthy_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/filter/region_filters_test.go b/pkg/schedule/filter/region_filters_test.go index a7dd1fa932a..ddd4141f8ba 100644 --- a/pkg/schedule/filter/region_filters_test.go +++ b/pkg/schedule/filter/region_filters_test.go @@ -19,9 +19,11 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/handler/handler.go b/pkg/schedule/handler/handler.go index a8540b4b5f4..042adf96611 100644 --- a/pkg/schedule/handler/handler.go +++ b/pkg/schedule/handler/handler.go @@ -23,11 +23,14 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule" @@ -43,7 +46,6 @@ import ( "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/hbstream/heartbeat_streams.go b/pkg/schedule/hbstream/heartbeat_streams.go index ef0440a9f77..6637800e97f 100644 --- a/pkg/schedule/hbstream/heartbeat_streams.go +++ b/pkg/schedule/hbstream/heartbeat_streams.go @@ -20,17 +20,19 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Operation is detailed scheduling step of a region. diff --git a/pkg/schedule/labeler/labeler.go b/pkg/schedule/labeler/labeler.go index 7670ccdedd7..16a14068e68 100644 --- a/pkg/schedule/labeler/labeler.go +++ b/pkg/schedule/labeler/labeler.go @@ -19,7 +19,10 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/rangelist" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // RegionLabeler is utility to label regions. diff --git a/pkg/schedule/labeler/labeler_test.go b/pkg/schedule/labeler/labeler_test.go index ebd57708e47..564542727ff 100644 --- a/pkg/schedule/labeler/labeler_test.go +++ b/pkg/schedule/labeler/labeler_test.go @@ -24,8 +24,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/schedule/labeler/rules.go b/pkg/schedule/labeler/rules.go index 39a420032d8..4f5d28a8034 100644 --- a/pkg/schedule/labeler/rules.go +++ b/pkg/schedule/labeler/rules.go @@ -23,10 +23,12 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // RegionLabel is the label of a region. diff --git a/pkg/schedule/operator/builder.go b/pkg/schedule/operator/builder.go index 29b8aedf978..8fd2393da2e 100644 --- a/pkg/schedule/operator/builder.go +++ b/pkg/schedule/operator/builder.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" diff --git a/pkg/schedule/operator/builder_test.go b/pkg/schedule/operator/builder_test.go index 5e9ff7f89bb..77224a82c15 100644 --- a/pkg/schedule/operator/builder_test.go +++ b/pkg/schedule/operator/builder_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/create_operator.go b/pkg/schedule/operator/create_operator.go index 4fae7f9e3f2..2558ca8d75b 100644 --- a/pkg/schedule/operator/create_operator.go +++ b/pkg/schedule/operator/create_operator.go @@ -18,16 +18,18 @@ import ( "fmt" "math/rand" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // CreateAddPeerOperator creates an operator that adds a new peer. diff --git a/pkg/schedule/operator/create_operator_test.go b/pkg/schedule/operator/create_operator_test.go index 845255e713c..0143b1b8011 100644 --- a/pkg/schedule/operator/create_operator_test.go +++ b/pkg/schedule/operator/create_operator_test.go @@ -19,11 +19,13 @@ import ( "encoding/hex" "testing" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/metrics.go b/pkg/schedule/operator/metrics.go index 47a165500e9..25de41f0fb7 100644 --- a/pkg/schedule/operator/metrics.go +++ b/pkg/schedule/operator/metrics.go @@ -16,6 +16,7 @@ package operator import ( "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/operator/operator.go b/pkg/schedule/operator/operator.go index 8abeae54f6b..69b2504755e 100644 --- a/pkg/schedule/operator/operator.go +++ b/pkg/schedule/operator/operator.go @@ -22,8 +22,10 @@ import ( "sync/atomic" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/schedule/operator/operator_controller.go b/pkg/schedule/operator/operator_controller.go index 0478ef2b6ae..cd2470376e1 100644 --- a/pkg/schedule/operator/operator_controller.go +++ b/pkg/schedule/operator/operator_controller.go @@ -21,9 +21,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) // The source of dispatched region. diff --git a/pkg/schedule/operator/operator_controller_test.go b/pkg/schedule/operator/operator_controller_test.go index 69600f80536..3c9abe54f24 100644 --- a/pkg/schedule/operator/operator_controller_test.go +++ b/pkg/schedule/operator/operator_controller_test.go @@ -22,11 +22,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/operator/operator_test.go b/pkg/schedule/operator/operator_test.go index 22a86c789fc..6976b5ca12e 100644 --- a/pkg/schedule/operator/operator_test.go +++ b/pkg/schedule/operator/operator_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/schedule/operator/step.go b/pkg/schedule/operator/step.go index 04e41028865..6b0f0f1021f 100644 --- a/pkg/schedule/operator/step.go +++ b/pkg/schedule/operator/step.go @@ -20,17 +20,19 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/operator/step_test.go b/pkg/schedule/operator/step_test.go index 014703d00f9..f6323a6d23c 100644 --- a/pkg/schedule/operator/step_test.go +++ b/pkg/schedule/operator/step_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/test_util.go b/pkg/schedule/operator/test_util.go index 8206189daa6..f86701792dd 100644 --- a/pkg/schedule/operator/test_util.go +++ b/pkg/schedule/operator/test_util.go @@ -16,6 +16,7 @@ package operator import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" ) diff --git a/pkg/schedule/operator/waiting_operator_test.go b/pkg/schedule/operator/waiting_operator_test.go index 8a0d1875cd7..a2e333231b0 100644 --- a/pkg/schedule/operator/waiting_operator_test.go +++ b/pkg/schedule/operator/waiting_operator_test.go @@ -17,8 +17,10 @@ package operator import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/schedule/placement/fit.go b/pkg/schedule/placement/fit.go index 30530462664..bb1d642900b 100644 --- a/pkg/schedule/placement/fit.go +++ b/pkg/schedule/placement/fit.go @@ -20,6 +20,7 @@ import ( "sort" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/fit_region_test.go b/pkg/schedule/placement/fit_region_test.go index 284674b79a2..9c6e7b7faf1 100644 --- a/pkg/schedule/placement/fit_region_test.go +++ b/pkg/schedule/placement/fit_region_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/fit_test.go b/pkg/schedule/placement/fit_test.go index b12bcd7451a..f22661ba694 100644 --- a/pkg/schedule/placement/fit_test.go +++ b/pkg/schedule/placement/fit_test.go @@ -21,9 +21,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/label_constraint_test.go b/pkg/schedule/placement/label_constraint_test.go index e65676f0a5e..cd2ff5976df 100644 --- a/pkg/schedule/placement/label_constraint_test.go +++ b/pkg/schedule/placement/label_constraint_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/region_rule_cache.go b/pkg/schedule/placement/region_rule_cache.go index 79e52a49fca..4b17c3baf70 100644 --- a/pkg/schedule/placement/region_rule_cache.go +++ b/pkg/schedule/placement/region_rule_cache.go @@ -16,6 +16,7 @@ package placement import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/syncutil" diff --git a/pkg/schedule/placement/region_rule_cache_test.go b/pkg/schedule/placement/region_rule_cache_test.go index e951ea10cc5..c526a095e7b 100644 --- a/pkg/schedule/placement/region_rule_cache_test.go +++ b/pkg/schedule/placement/region_rule_cache_test.go @@ -19,9 +19,11 @@ import ( "time" "unsafe" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/rule_list.go b/pkg/schedule/placement/rule_list.go index f5ee0dada0e..3fc401163d1 100644 --- a/pkg/schedule/placement/rule_list.go +++ b/pkg/schedule/placement/rule_list.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/rangelist" ) diff --git a/pkg/schedule/placement/rule_manager.go b/pkg/schedule/placement/rule_manager.go index f44258d797c..f5101f0250c 100644 --- a/pkg/schedule/placement/rule_manager.go +++ b/pkg/schedule/placement/rule_manager.go @@ -24,8 +24,12 @@ import ( "sort" "strings" + "go.uber.org/zap" + "golang.org/x/exp/slices" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -35,8 +39,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "golang.org/x/exp/slices" ) const ( diff --git a/pkg/schedule/placement/rule_manager_test.go b/pkg/schedule/placement/rule_manager_test.go index 2e1883640d8..5f5f457da13 100644 --- a/pkg/schedule/placement/rule_manager_test.go +++ b/pkg/schedule/placement/rule_manager_test.go @@ -19,8 +19,10 @@ import ( "encoding/hex" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/pkg/schedule/plan/balance_plan_test.go b/pkg/schedule/plan/balance_plan_test.go index 59f2acc689a..8bdc2519a90 100644 --- a/pkg/schedule/plan/balance_plan_test.go +++ b/pkg/schedule/plan/balance_plan_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/plugin_interface.go b/pkg/schedule/plugin_interface.go index 62ffe2eb900..8870ca7f46d 100644 --- a/pkg/schedule/plugin_interface.go +++ b/pkg/schedule/plugin_interface.go @@ -18,10 +18,12 @@ import ( "path/filepath" "plugin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // PluginInterface is used to manage all plugin. diff --git a/pkg/schedule/prepare_checker.go b/pkg/schedule/prepare_checker.go index 187d68e3d9f..5f6b54f33ae 100644 --- a/pkg/schedule/prepare_checker.go +++ b/pkg/schedule/prepare_checker.go @@ -17,10 +17,12 @@ package schedule import ( "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type prepareChecker struct { diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index 71b2cd346c7..fdcbd705398 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -22,10 +22,13 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -36,7 +39,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const regionScatterName = "region-scatter" diff --git a/pkg/schedule/scatter/region_scatterer_test.go b/pkg/schedule/scatter/region_scatterer_test.go index fdaed888309..b86d73d4288 100644 --- a/pkg/schedule/scatter/region_scatterer_test.go +++ b/pkg/schedule/scatter/region_scatterer_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_benchmark_test.go b/pkg/schedule/schedulers/balance_benchmark_test.go index aaafa7be8ca..abf4c0b3def 100644 --- a/pkg/schedule/schedulers/balance_benchmark_test.go +++ b/pkg/schedule/schedulers/balance_benchmark_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/assert" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/balance_leader.go b/pkg/schedule/schedulers/balance_leader.go index 03f02002c74..2884e8a486f 100644 --- a/pkg/schedule/schedulers/balance_leader.go +++ b/pkg/schedule/schedulers/balance_leader.go @@ -23,7 +23,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -34,8 +38,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/typeutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/balance_leader_test.go b/pkg/schedule/schedulers/balance_leader_test.go index 940f75f78bc..36a4ee52ea6 100644 --- a/pkg/schedule/schedulers/balance_leader_test.go +++ b/pkg/schedule/schedulers/balance_leader_test.go @@ -22,9 +22,11 @@ import ( "testing" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_region.go b/pkg/schedule/schedulers/balance_region.go index 2bee521364d..6ad95201f68 100644 --- a/pkg/schedule/schedulers/balance_region.go +++ b/pkg/schedule/schedulers/balance_region.go @@ -18,8 +18,11 @@ import ( "sort" "strconv" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type balanceRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/balance_region_test.go b/pkg/schedule/schedulers/balance_region_test.go index 48a4959a170..aa8e7fc30d1 100644 --- a/pkg/schedule/schedulers/balance_region_test.go +++ b/pkg/schedule/schedulers/balance_region_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_witness.go b/pkg/schedule/schedulers/balance_witness.go index cfcafb56c57..1b23e5b4a19 100644 --- a/pkg/schedule/schedulers/balance_witness.go +++ b/pkg/schedule/schedulers/balance_witness.go @@ -23,8 +23,12 @@ import ( "strconv" "github.com/gorilla/mux" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + "github.com/unrolled/render" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -35,8 +39,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/balance_witness_test.go b/pkg/schedule/schedulers/balance_witness_test.go index b1449821236..a7edbd3ea94 100644 --- a/pkg/schedule/schedulers/balance_witness_test.go +++ b/pkg/schedule/schedulers/balance_witness_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/base_scheduler.go b/pkg/schedule/schedulers/base_scheduler.go index ef5c6bf7ae7..2534ed12281 100644 --- a/pkg/schedule/schedulers/base_scheduler.go +++ b/pkg/schedule/schedulers/base_scheduler.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/config.go b/pkg/schedule/schedulers/config.go index 78b123981fd..e172b596685 100644 --- a/pkg/schedule/schedulers/config.go +++ b/pkg/schedule/schedulers/config.go @@ -15,13 +15,15 @@ package schedulers import ( + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type schedulerConfig interface { diff --git a/pkg/schedule/schedulers/config_test.go b/pkg/schedule/schedulers/config_test.go index 9e20521854f..f97a2ad17ea 100644 --- a/pkg/schedule/schedulers/config_test.go +++ b/pkg/schedule/schedulers/config_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage" ) diff --git a/pkg/schedule/schedulers/evict_leader.go b/pkg/schedule/schedulers/evict_leader.go index defc65846ae..45285b51137 100644 --- a/pkg/schedule/schedulers/evict_leader.go +++ b/pkg/schedule/schedulers/evict_leader.go @@ -20,8 +20,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -32,8 +36,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_leader_test.go b/pkg/schedule/schedulers/evict_leader_test.go index ecd80813c0f..587a48358e9 100644 --- a/pkg/schedule/schedulers/evict_leader_test.go +++ b/pkg/schedule/schedulers/evict_leader_test.go @@ -18,9 +18,11 @@ import ( "bytes" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/schedulers/evict_slow_store.go b/pkg/schedule/schedulers/evict_slow_store.go index b1960ceca8e..8d8c014b110 100644 --- a/pkg/schedule/schedulers/evict_slow_store.go +++ b/pkg/schedule/schedulers/evict_slow_store.go @@ -19,9 +19,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -29,8 +33,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_slow_store_test.go b/pkg/schedule/schedulers/evict_slow_store_test.go index 79651fb5b5c..636d856fc14 100644 --- a/pkg/schedule/schedulers/evict_slow_store_test.go +++ b/pkg/schedule/schedulers/evict_slow_store_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/evict_slow_trend.go b/pkg/schedule/schedulers/evict_slow_trend.go index 92805587f72..cf4bf3d3b39 100644 --- a/pkg/schedule/schedulers/evict_slow_trend.go +++ b/pkg/schedule/schedulers/evict_slow_trend.go @@ -20,9 +20,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_slow_trend_test.go b/pkg/schedule/schedulers/evict_slow_trend_test.go index 4be6eeb58d9..21420cc6022 100644 --- a/pkg/schedule/schedulers/evict_slow_trend_test.go +++ b/pkg/schedule/schedulers/evict_slow_trend_test.go @@ -19,9 +19,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 88b9f5c6c93..837d7290ff7 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -21,8 +21,12 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -36,8 +40,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type grantHotRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/grant_leader.go b/pkg/schedule/schedulers/grant_leader.go index b18d1fc3397..d0cb6bae34d 100644 --- a/pkg/schedule/schedulers/grant_leader.go +++ b/pkg/schedule/schedulers/grant_leader.go @@ -19,8 +19,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type grantLeaderSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/grant_leader_test.go b/pkg/schedule/schedulers/grant_leader_test.go index 38bac3c961a..003966db73d 100644 --- a/pkg/schedule/schedulers/grant_leader_test.go +++ b/pkg/schedule/schedulers/grant_leader_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/hot_region.go b/pkg/schedule/schedulers/hot_region.go index 97a558c3fe4..622e617b729 100644 --- a/pkg/schedule/schedulers/hot_region.go +++ b/pkg/schedule/schedulers/hot_region.go @@ -23,10 +23,13 @@ import ( "strconv" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -41,7 +44,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/hot_region_config.go b/pkg/schedule/schedulers/hot_region_config.go index df82ccc3afc..c458d29842f 100644 --- a/pkg/schedule/schedulers/hot_region_config.go +++ b/pkg/schedule/schedulers/hot_region_config.go @@ -22,7 +22,11 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/slice" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/hot_region_rank_v2_test.go b/pkg/schedule/schedulers/hot_region_rank_v2_test.go index f00e9dde787..d4cbfc092a0 100644 --- a/pkg/schedule/schedulers/hot_region_rank_v2_test.go +++ b/pkg/schedule/schedulers/hot_region_rank_v2_test.go @@ -19,6 +19,7 @@ import ( "github.com/docker/go-units" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index 9f79ac617c9..b37fc00cea1 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -22,9 +22,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/label.go b/pkg/schedule/schedulers/label.go index a27ea29687e..cd63b6c0511 100644 --- a/pkg/schedule/schedulers/label.go +++ b/pkg/schedule/schedulers/label.go @@ -15,7 +15,10 @@ package schedulers import ( + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -25,7 +28,6 @@ import ( "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type labelSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/metrics.go b/pkg/schedule/schedulers/metrics.go index 4afc4605f52..bd8a2b4f6ea 100644 --- a/pkg/schedule/schedulers/metrics.go +++ b/pkg/schedule/schedulers/metrics.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/schedulers/random_merge.go b/pkg/schedule/schedulers/random_merge.go index 7cd9954ce4b..ccb1bc12f9a 100644 --- a/pkg/schedule/schedulers/random_merge.go +++ b/pkg/schedule/schedulers/random_merge.go @@ -18,6 +18,7 @@ import ( "math/rand" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/random_merge_test.go b/pkg/schedule/schedulers/random_merge_test.go index 6c8ea353f2d..e7be6d589ba 100644 --- a/pkg/schedule/schedulers/random_merge_test.go +++ b/pkg/schedule/schedulers/random_merge_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/range_cluster.go b/pkg/schedule/schedulers/range_cluster.go index e83e74145f6..0b6e908b56c 100644 --- a/pkg/schedule/schedulers/range_cluster.go +++ b/pkg/schedule/schedulers/range_cluster.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/docker/go-units" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" ) diff --git a/pkg/schedule/schedulers/scatter_range.go b/pkg/schedule/schedulers/scatter_range.go index 5ba303ad05a..2e071ec6ca7 100644 --- a/pkg/schedule/schedulers/scatter_range.go +++ b/pkg/schedule/schedulers/scatter_range.go @@ -19,7 +19,10 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -28,7 +31,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) type scatterRangeSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/scatter_range_test.go b/pkg/schedule/schedulers/scatter_range_test.go index 26ac48a36bb..7125ba9a596 100644 --- a/pkg/schedule/schedulers/scatter_range_test.go +++ b/pkg/schedule/schedulers/scatter_range_test.go @@ -19,8 +19,10 @@ import ( "math/rand" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/scheduler.go b/pkg/schedule/schedulers/scheduler.go index 3f722f7f804..8976c3a1928 100644 --- a/pkg/schedule/schedulers/scheduler.go +++ b/pkg/schedule/schedulers/scheduler.go @@ -20,8 +20,11 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/config" sche "github.com/tikv/pd/pkg/schedule/core" @@ -29,7 +32,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage/endpoint" - "go.uber.org/zap" ) // Scheduler is an interface to schedule resources. diff --git a/pkg/schedule/schedulers/scheduler_controller.go b/pkg/schedule/schedulers/scheduler_controller.go index cb4ffd6f9c2..28973631570 100644 --- a/pkg/schedule/schedulers/scheduler_controller.go +++ b/pkg/schedule/schedulers/scheduler_controller.go @@ -22,7 +22,10 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const maxScheduleRetries = 10 diff --git a/pkg/schedule/schedulers/scheduler_test.go b/pkg/schedule/schedulers/scheduler_test.go index 0e06b9333e9..24df62ad3e1 100644 --- a/pkg/schedule/schedulers/scheduler_test.go +++ b/pkg/schedule/schedulers/scheduler_test.go @@ -20,8 +20,10 @@ import ( "testing" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/schedulers/shuffle_hot_region.go b/pkg/schedule/schedulers/shuffle_hot_region.go index 7517abb3c21..6da4b71ea45 100644 --- a/pkg/schedule/schedulers/shuffle_hot_region.go +++ b/pkg/schedule/schedulers/shuffle_hot_region.go @@ -18,8 +18,12 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -30,8 +34,6 @@ import ( "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type shuffleHotRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/shuffle_leader.go b/pkg/schedule/schedulers/shuffle_leader.go index e2a256af7a7..8fd5b87c1bc 100644 --- a/pkg/schedule/schedulers/shuffle_leader.go +++ b/pkg/schedule/schedulers/shuffle_leader.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/shuffle_region.go b/pkg/schedule/schedulers/shuffle_region.go index 1fbc1f08e67..2b4e2443210 100644 --- a/pkg/schedule/schedulers/shuffle_region.go +++ b/pkg/schedule/schedulers/shuffle_region.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/schedulers/shuffle_region_config.go b/pkg/schedule/schedulers/shuffle_region_config.go index 2e3394a58df..58ea25b6186 100644 --- a/pkg/schedule/schedulers/shuffle_region_config.go +++ b/pkg/schedule/schedulers/shuffle_region_config.go @@ -18,12 +18,13 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/pkg/schedule/schedulers/split_bucket.go b/pkg/schedule/schedulers/split_bucket.go index edbe2ac3545..d6aee65b181 100644 --- a/pkg/schedule/schedulers/split_bucket.go +++ b/pkg/schedule/schedulers/split_bucket.go @@ -22,8 +22,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" @@ -32,7 +35,6 @@ import ( "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/pkg/schedule/schedulers/split_bucket_test.go b/pkg/schedule/schedulers/split_bucket_test.go index 840dfa97c19..41892bb4fc4 100644 --- a/pkg/schedule/schedulers/split_bucket_test.go +++ b/pkg/schedule/schedulers/split_bucket_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/statistics/buckets" diff --git a/pkg/schedule/schedulers/transfer_witness_leader.go b/pkg/schedule/schedulers/transfer_witness_leader.go index 90191dd355c..c84f0918884 100644 --- a/pkg/schedule/schedulers/transfer_witness_leader.go +++ b/pkg/schedule/schedulers/transfer_witness_leader.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/transfer_witness_leader_test.go b/pkg/schedule/schedulers/transfer_witness_leader_test.go index b100e0a9535..e58f3dd3544 100644 --- a/pkg/schedule/schedulers/transfer_witness_leader_test.go +++ b/pkg/schedule/schedulers/transfer_witness_leader_test.go @@ -17,9 +17,11 @@ package schedulers import ( "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/schedulers/utils.go b/pkg/schedule/schedulers/utils.go index 9c1c7fe3854..9a6a6bd96fa 100644 --- a/pkg/schedule/schedulers/utils.go +++ b/pkg/schedule/schedulers/utils.go @@ -19,7 +19,10 @@ import ( "strconv" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -28,7 +31,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/statistics" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/utils_test.go b/pkg/schedule/schedulers/utils_test.go index deb7c6e1038..d85fba47ff4 100644 --- a/pkg/schedule/schedulers/utils_test.go +++ b/pkg/schedule/schedulers/utils_test.go @@ -17,8 +17,10 @@ package schedulers import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/splitter/region_splitter.go b/pkg/schedule/splitter/region_splitter.go index 37b33dad480..c5b950d73be 100644 --- a/pkg/schedule/splitter/region_splitter.go +++ b/pkg/schedule/splitter/region_splitter.go @@ -22,15 +22,17 @@ import ( "math" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/splitter/region_splitter_test.go b/pkg/schedule/splitter/region_splitter_test.go index 6f49707217e..8e59fa74382 100644 --- a/pkg/schedule/splitter/region_splitter_test.go +++ b/pkg/schedule/splitter/region_splitter_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/slice/slice_test.go b/pkg/slice/slice_test.go index 019cd49c46a..c0abb46f298 100644 --- a/pkg/slice/slice_test.go +++ b/pkg/slice/slice_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/slice" ) diff --git a/pkg/statistics/buckets/hot_bucket_cache.go b/pkg/statistics/buckets/hot_bucket_cache.go index 3fc640e8c1f..f29ef1563e8 100644 --- a/pkg/statistics/buckets/hot_bucket_cache.go +++ b/pkg/statistics/buckets/hot_bucket_cache.go @@ -19,12 +19,14 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/rangetree" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) type status int diff --git a/pkg/statistics/buckets/hot_bucket_cache_test.go b/pkg/statistics/buckets/hot_bucket_cache_test.go index e9669eb2eec..06e00687fd7 100644 --- a/pkg/statistics/buckets/hot_bucket_cache_test.go +++ b/pkg/statistics/buckets/hot_bucket_cache_test.go @@ -18,8 +18,9 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func TestPutItem(t *testing.T) { diff --git a/pkg/statistics/buckets/hot_bucket_task_test.go b/pkg/statistics/buckets/hot_bucket_task_test.go index 100ed7c818d..93551919ba2 100644 --- a/pkg/statistics/buckets/hot_bucket_task_test.go +++ b/pkg/statistics/buckets/hot_bucket_task_test.go @@ -21,8 +21,9 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func getAllBucketStats(ctx context.Context, hotCache *HotBucketCache) map[uint64][]*BucketStat { diff --git a/pkg/statistics/hot_cache.go b/pkg/statistics/hot_cache.go index ae61063646d..99ec587aea7 100644 --- a/pkg/statistics/hot_cache.go +++ b/pkg/statistics/hot_cache.go @@ -17,8 +17,10 @@ package statistics import ( "context" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/smallnest/chanx" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" diff --git a/pkg/statistics/hot_cache_test.go b/pkg/statistics/hot_cache_test.go index b0232b658d4..668c930adcc 100644 --- a/pkg/statistics/hot_cache_test.go +++ b/pkg/statistics/hot_cache_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/statistics/hot_peer.go b/pkg/statistics/hot_peer.go index c17ad5c246f..55dcd03b4ee 100644 --- a/pkg/statistics/hot_peer.go +++ b/pkg/statistics/hot_peer.go @@ -18,12 +18,14 @@ import ( "math" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type dimStat struct { diff --git a/pkg/statistics/hot_peer_cache.go b/pkg/statistics/hot_peer_cache.go index b62248062bd..77b5d567ca4 100644 --- a/pkg/statistics/hot_peer_cache.go +++ b/pkg/statistics/hot_peer_cache.go @@ -20,9 +20,11 @@ import ( "math" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" "github.com/smallnest/chanx" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/pkg/statistics/hot_peer_cache_test.go b/pkg/statistics/hot_peer_cache_test.go index 2d44b5dc783..09d267d60b9 100644 --- a/pkg/statistics/hot_peer_cache_test.go +++ b/pkg/statistics/hot_peer_cache_test.go @@ -23,9 +23,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/movingaverage" diff --git a/pkg/statistics/region_collection.go b/pkg/statistics/region_collection.go index 958ba3be5df..8d12dbf42c9 100644 --- a/pkg/statistics/region_collection.go +++ b/pkg/statistics/region_collection.go @@ -17,12 +17,14 @@ package statistics import ( "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // RegionInfoProvider is an interface to provide the region information. diff --git a/pkg/statistics/region_collection_test.go b/pkg/statistics/region_collection_test.go index 64a625a04e2..f97dffa893d 100644 --- a/pkg/statistics/region_collection_test.go +++ b/pkg/statistics/region_collection_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/pkg/statistics/store.go b/pkg/statistics/store.go index baeef0ad417..fbbfea72700 100644 --- a/pkg/statistics/store.go +++ b/pkg/statistics/store.go @@ -17,13 +17,15 @@ package statistics import ( "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/statistics/store_collection.go b/pkg/statistics/store_collection.go index f55c23b27b7..a7c77aa7b4b 100644 --- a/pkg/statistics/store_collection.go +++ b/pkg/statistics/store_collection.go @@ -19,6 +19,7 @@ import ( "strconv" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/statistics/store_collection_test.go b/pkg/statistics/store_collection_test.go index 6a0ef24aff5..93845d4de1a 100644 --- a/pkg/statistics/store_collection_test.go +++ b/pkg/statistics/store_collection_test.go @@ -20,9 +20,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/statistics/store_load_test.go b/pkg/statistics/store_load_test.go index 9d958151182..cbcbb0bacfe 100644 --- a/pkg/statistics/store_load_test.go +++ b/pkg/statistics/store_load_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/statistics/store_test.go b/pkg/statistics/store_test.go index ccf85caaa72..741fdc54fd3 100644 --- a/pkg/statistics/store_test.go +++ b/pkg/statistics/store_test.go @@ -18,9 +18,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/statistics/utils/kind_test.go b/pkg/statistics/utils/kind_test.go index 0a02ffa00c1..d0834e077da 100644 --- a/pkg/statistics/utils/kind_test.go +++ b/pkg/statistics/utils/kind_test.go @@ -17,9 +17,11 @@ package utils import ( "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/storage/endpoint/cluster_id.go b/pkg/storage/endpoint/cluster_id.go index 974b65d6c6c..f9317d44874 100644 --- a/pkg/storage/endpoint/cluster_id.go +++ b/pkg/storage/endpoint/cluster_id.go @@ -19,14 +19,16 @@ import ( "math/rand" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // InitClusterID creates a cluster ID if it hasn't existed. diff --git a/pkg/storage/endpoint/cluster_id_test.go b/pkg/storage/endpoint/cluster_id_test.go index 5ce1600044d..dcd06e15053 100644 --- a/pkg/storage/endpoint/cluster_id_test.go +++ b/pkg/storage/endpoint/cluster_id_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/endpoint/config.go b/pkg/storage/endpoint/config.go index d297562f275..0c96de0a8cc 100644 --- a/pkg/storage/endpoint/config.go +++ b/pkg/storage/endpoint/config.go @@ -18,9 +18,10 @@ import ( "encoding/json" "strings" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) // ConfigStorage defines the storage operations on the config. diff --git a/pkg/storage/endpoint/gc_safe_point.go b/pkg/storage/endpoint/gc_safe_point.go index 8a6c5c5f789..15d3b3c47a1 100644 --- a/pkg/storage/endpoint/gc_safe_point.go +++ b/pkg/storage/endpoint/gc_safe_point.go @@ -20,11 +20,13 @@ import ( "strconv" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) // ServiceSafePoint is the safepoint for a specific service diff --git a/pkg/storage/endpoint/keyspace.go b/pkg/storage/endpoint/keyspace.go index 77240541951..e1c7ae4c846 100644 --- a/pkg/storage/endpoint/keyspace.go +++ b/pkg/storage/endpoint/keyspace.go @@ -19,11 +19,13 @@ import ( "strconv" "github.com/gogo/protobuf/proto" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/pkg/storage/endpoint/meta.go b/pkg/storage/endpoint/meta.go index 176188be3f3..73b7c397ff4 100644 --- a/pkg/storage/endpoint/meta.go +++ b/pkg/storage/endpoint/meta.go @@ -21,8 +21,10 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/storage/endpoint/resource_group.go b/pkg/storage/endpoint/resource_group.go index 733e5ba2c9a..431c869ac70 100644 --- a/pkg/storage/endpoint/resource_group.go +++ b/pkg/storage/endpoint/resource_group.go @@ -16,6 +16,7 @@ package endpoint import ( "github.com/gogo/protobuf/proto" + "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/endpoint/safepoint_v2.go b/pkg/storage/endpoint/safepoint_v2.go index f16855e86d6..12659847510 100644 --- a/pkg/storage/endpoint/safepoint_v2.go +++ b/pkg/storage/endpoint/safepoint_v2.go @@ -19,12 +19,14 @@ import ( "math" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // GCSafePointV2 represents the overall safe point for a specific keyspace. diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index 77841529e98..697001dfe7c 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -19,13 +19,15 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // TSOStorage is the interface for timestamp storage. diff --git a/pkg/storage/endpoint/tso_keyspace_group.go b/pkg/storage/endpoint/tso_keyspace_group.go index b45d1f1da37..ef9a23308a6 100644 --- a/pkg/storage/endpoint/tso_keyspace_group.go +++ b/pkg/storage/endpoint/tso_keyspace_group.go @@ -18,11 +18,12 @@ import ( "context" "encoding/json" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // UserKind represents the user kind. diff --git a/pkg/storage/endpoint/util.go b/pkg/storage/endpoint/util.go index 39aa6240f5a..e8f17a3922f 100644 --- a/pkg/storage/endpoint/util.go +++ b/pkg/storage/endpoint/util.go @@ -20,10 +20,11 @@ import ( "strings" "github.com/gogo/protobuf/proto" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" - clientv3 "go.etcd.io/etcd/client/v3" ) func (se *StorageEndpoint) loadProto(key string, msg proto.Message) (bool, error) { diff --git a/pkg/storage/etcd_backend.go b/pkg/storage/etcd_backend.go index e9af5fc67f6..b967d99b37b 100644 --- a/pkg/storage/etcd_backend.go +++ b/pkg/storage/etcd_backend.go @@ -15,9 +15,10 @@ package storage import ( + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" - clientv3 "go.etcd.io/etcd/client/v3" ) // etcdBackend is a storage backend that stores data in etcd, diff --git a/pkg/storage/hot_region_storage.go b/pkg/storage/hot_region_storage.go index d323b40d435..b5ec8dda6ba 100644 --- a/pkg/storage/hot_region_storage.go +++ b/pkg/storage/hot_region_storage.go @@ -24,12 +24,15 @@ import ( "sync" "time" - "github.com/pingcap/kvproto/pkg/encryptionpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/log" "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/iterator" "github.com/syndtr/goleveldb/leveldb/util" + "go.uber.org/zap" + + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // HotRegionStorage is used to store the hot region info. diff --git a/pkg/storage/hot_region_storage_test.go b/pkg/storage/hot_region_storage_test.go index aeda8e450e7..cd713685eb0 100644 --- a/pkg/storage/hot_region_storage_test.go +++ b/pkg/storage/hot_region_storage_test.go @@ -26,6 +26,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/storage/keyspace_test.go b/pkg/storage/keyspace_test.go index a6ce94e4711..7841f5da22d 100644 --- a/pkg/storage/keyspace_test.go +++ b/pkg/storage/keyspace_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/kv/etcd_kv.go b/pkg/storage/kv/etcd_kv.go index 5945990c51b..c25f4d66060 100644 --- a/pkg/storage/kv/etcd_kv.go +++ b/pkg/storage/kv/etcd_kv.go @@ -20,13 +20,15 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/storage/kv/kv_test.go b/pkg/storage/kv/kv_test.go index f57c8145bd0..f05561b0c0b 100644 --- a/pkg/storage/kv/kv_test.go +++ b/pkg/storage/kv/kv_test.go @@ -22,8 +22,9 @@ import ( "testing" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/pkg/utils/etcdutil" ) func TestEtcd(t *testing.T) { diff --git a/pkg/storage/kv/levedb_kv.go b/pkg/storage/kv/levedb_kv.go index 6f93cd0237f..5a74c1928e8 100644 --- a/pkg/storage/kv/levedb_kv.go +++ b/pkg/storage/kv/levedb_kv.go @@ -17,9 +17,11 @@ package kv import ( "context" - "github.com/pingcap/errors" "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/util" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/storage/kv/mem_kv.go b/pkg/storage/kv/mem_kv.go index dc24ab3e0f6..cc5dca29851 100644 --- a/pkg/storage/kv/mem_kv.go +++ b/pkg/storage/kv/mem_kv.go @@ -18,8 +18,10 @@ import ( "context" "github.com/google/btree" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/storage/leveldb_backend.go b/pkg/storage/leveldb_backend.go index 8fb1db196c1..096a762db65 100644 --- a/pkg/storage/leveldb_backend.go +++ b/pkg/storage/leveldb_backend.go @@ -18,9 +18,11 @@ import ( "context" "time" + "github.com/syndtr/goleveldb/leveldb" + "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/syndtr/goleveldb/leveldb" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/storage/leveldb_backend_test.go b/pkg/storage/leveldb_backend_test.go index c6e0fcef607..0bf626834cd 100644 --- a/pkg/storage/leveldb_backend_test.go +++ b/pkg/storage/leveldb_backend_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/storage/region_storage.go b/pkg/storage/region_storage.go index 5a581ec5155..8e6a4d87429 100644 --- a/pkg/storage/region_storage.go +++ b/pkg/storage/region_storage.go @@ -18,7 +18,9 @@ import ( "context" "github.com/gogo/protobuf/proto" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/storage/region_storage_test.go b/pkg/storage/region_storage_test.go index f6670f8c82e..c0ae04c8ded 100644 --- a/pkg/storage/region_storage_test.go +++ b/pkg/storage/region_storage_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/pkg/storage/storage.go b/pkg/storage/storage.go index dce2f1712b8..34f2357c8f7 100644 --- a/pkg/storage/storage.go +++ b/pkg/storage/storage.go @@ -18,13 +18,15 @@ import ( "context" "sync/atomic" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // Storage is the interface for the backend storage of the PD. diff --git a/pkg/storage/storage_gc_test.go b/pkg/storage/storage_gc_test.go index b18fcc12afc..ffffab620b3 100644 --- a/pkg/storage/storage_gc_test.go +++ b/pkg/storage/storage_gc_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/storage_test.go b/pkg/storage/storage_test.go index ca1c8e275eb..e6c12f37cbb 100644 --- a/pkg/storage/storage_test.go +++ b/pkg/storage/storage_test.go @@ -25,13 +25,15 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) func TestBasic(t *testing.T) { diff --git a/pkg/storage/storage_tso_test.go b/pkg/storage/storage_tso_test.go index 07ea79df47c..e8b49106a09 100644 --- a/pkg/storage/storage_tso_test.go +++ b/pkg/storage/storage_tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/syncer/client.go b/pkg/syncer/client.go index 3d7f36d9114..81652045cbf 100644 --- a/pkg/syncer/client.go +++ b/pkg/syncer/client.go @@ -19,11 +19,19 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/keepalive" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/ratelimit" @@ -31,12 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/keepalive" - "google.golang.org/grpc/status" ) const ( diff --git a/pkg/syncer/client_test.go b/pkg/syncer/client_test.go index 926be3421ab..03f18fd2b76 100644 --- a/pkg/syncer/client_test.go +++ b/pkg/syncer/client_test.go @@ -19,15 +19,17 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockserver" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/grpcutil" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // For issue https://github.com/tikv/pd/issues/3936 diff --git a/pkg/syncer/history_buffer.go b/pkg/syncer/history_buffer.go index 7ff6f202ad3..cb30eb56ee3 100644 --- a/pkg/syncer/history_buffer.go +++ b/pkg/syncer/history_buffer.go @@ -17,12 +17,14 @@ package syncer import ( "strconv" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/syncer/history_buffer_test.go b/pkg/syncer/history_buffer_test.go index 6bcd58b1689..368cecf5a3b 100644 --- a/pkg/syncer/history_buffer_test.go +++ b/pkg/syncer/history_buffer_test.go @@ -17,8 +17,10 @@ package syncer import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/kv" ) diff --git a/pkg/syncer/server.go b/pkg/syncer/server.go index 6009bba1d7d..89af3f79ccc 100644 --- a/pkg/syncer/server.go +++ b/pkg/syncer/server.go @@ -22,11 +22,16 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/ratelimit" @@ -35,9 +40,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/pkg/systimemon/systimemon.go b/pkg/systimemon/systimemon.go index 75fc5e68d8b..e3aa67d8d7c 100644 --- a/pkg/systimemon/systimemon.go +++ b/pkg/systimemon/systimemon.go @@ -18,10 +18,12 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // StartMonitor calls systimeErrHandler if system time jump backward. diff --git a/pkg/tso/admin.go b/pkg/tso/admin.go index bc9fd1f853d..f74abb29fcb 100644 --- a/pkg/tso/admin.go +++ b/pkg/tso/admin.go @@ -19,9 +19,10 @@ import ( "net/http" "strconv" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" - "github.com/unrolled/render" ) // Handler defines the common behaviors of a basic tso handler. diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 65f61e819d1..fda5c6088ab 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -21,9 +21,13 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/member" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 2fe0df3e000..d6bf27878a7 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -24,9 +24,12 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" mcsutils "github.com/tikv/pd/pkg/mcs/utils" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Allocator is a Timestamp Oracle allocator. diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index bb5fb4587f7..d22d284e1be 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -25,11 +25,16 @@ import ( "sync" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + perrors "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -47,9 +52,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index be3d53785cd..b4393a23471 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -28,9 +28,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" @@ -41,9 +46,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 0210c98626b..5293780b22f 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -21,9 +21,13 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" @@ -32,8 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/util_test.go b/pkg/tso/util_test.go index f31f8781ded..c2922d49b7f 100644 --- a/pkg/tso/util_test.go +++ b/pkg/tso/util_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/unsaferecovery/unsafe_recovery_controller.go b/pkg/unsaferecovery/unsafe_recovery_controller.go index aa3cf192270..dce2069e316 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller.go @@ -23,10 +23,13 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/btree" "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) // stage is the stage of unsafe recovery. diff --git a/pkg/unsaferecovery/unsafe_recovery_controller_test.go b/pkg/unsaferecovery/unsafe_recovery_controller_test.go index feaf5ba7430..0dab6aeca1d 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller_test.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller_test.go @@ -21,11 +21,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/raft_serverpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/utils/apiutil/apiutil.go b/pkg/utils/apiutil/apiutil.go index 20465d8376c..a743c543468 100644 --- a/pkg/utils/apiutil/apiutil.go +++ b/pkg/utils/apiutil/apiutil.go @@ -33,13 +33,15 @@ import ( "github.com/gin-gonic/gin" "github.com/gorilla/mux" "github.com/joho/godotenv" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/slice" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/utils/apiutil/multiservicesapi/middleware.go b/pkg/utils/apiutil/multiservicesapi/middleware.go index 4343adcc981..5362475f9ad 100644 --- a/pkg/utils/apiutil/multiservicesapi/middleware.go +++ b/pkg/utils/apiutil/multiservicesapi/middleware.go @@ -19,11 +19,13 @@ import ( "net/url" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/utils/apiutil/serverapi/middleware.go b/pkg/utils/apiutil/serverapi/middleware.go index d6fc98082d6..85b958a5554 100644 --- a/pkg/utils/apiutil/serverapi/middleware.go +++ b/pkg/utils/apiutil/serverapi/middleware.go @@ -20,15 +20,17 @@ import ( "strings" "time" + "github.com/urfave/negroni" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/urfave/negroni" - "go.uber.org/zap" ) type runtimeServiceValidator struct { diff --git a/pkg/utils/configutil/configutil.go b/pkg/utils/configutil/configutil.go index 48be7ff8c02..fe680879f73 100644 --- a/pkg/utils/configutil/configutil.go +++ b/pkg/utils/configutil/configutil.go @@ -21,8 +21,10 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/pingcap/errors" "github.com/spf13/pflag" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 70beae943f8..3e9eaa765ab 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -24,13 +24,6 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/utils/grpcutil" - "github.com/tikv/pd/pkg/utils/logutil" - "github.com/tikv/pd/pkg/utils/syncutil" "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" @@ -39,6 +32,15 @@ import ( "go.etcd.io/etcd/server/v3/etcdserver" "go.uber.org/zap" "google.golang.org/grpc/codes" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/grpcutil" + "github.com/tikv/pd/pkg/utils/logutil" + "github.com/tikv/pd/pkg/utils/syncutil" ) const ( diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 92ec8967d03..43c25539207 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -28,18 +28,20 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/tikv/pd/pkg/utils/tempurl" - "github.com/tikv/pd/pkg/utils/testutil" "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/mvccpb" etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + + "github.com/tikv/pd/pkg/utils/syncutil" + "github.com/tikv/pd/pkg/utils/tempurl" + "github.com/tikv/pd/pkg/utils/testutil" ) func TestMain(m *testing.M) { diff --git a/pkg/utils/etcdutil/health_checker.go b/pkg/utils/etcdutil/health_checker.go index d75814ec380..eafb6293238 100644 --- a/pkg/utils/etcdutil/health_checker.go +++ b/pkg/utils/etcdutil/health_checker.go @@ -21,13 +21,15 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const pickedCountThreshold = 3 diff --git a/pkg/utils/etcdutil/testutil.go b/pkg/utils/etcdutil/testutil.go index 7cb718664d9..daca4254467 100644 --- a/pkg/utils/etcdutil/testutil.go +++ b/pkg/utils/etcdutil/testutil.go @@ -22,11 +22,12 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/tempurl" - "github.com/tikv/pd/pkg/utils/testutil" "go.etcd.io/etcd/api/v3/etcdserverpb" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/tikv/pd/pkg/utils/tempurl" + "github.com/tikv/pd/pkg/utils/testutil" ) // NewTestSingleConfig is used to create a etcd config for the unit test purpose. diff --git a/pkg/utils/grpcutil/grpcutil.go b/pkg/utils/grpcutil/grpcutil.go index 1d1e6478036..aea65dd32dd 100644 --- a/pkg/utils/grpcutil/grpcutil.go +++ b/pkg/utils/grpcutil/grpcutil.go @@ -23,10 +23,6 @@ import ( "strings" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/utils/logutil" "go.etcd.io/etcd/client/pkg/v3/transport" "go.uber.org/zap" "google.golang.org/grpc" @@ -35,6 +31,12 @@ import ( "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/logutil" ) const ( diff --git a/pkg/utils/grpcutil/grpcutil_test.go b/pkg/utils/grpcutil/grpcutil_test.go index fbcfe59f02c..db77a2e4002 100644 --- a/pkg/utils/grpcutil/grpcutil_test.go +++ b/pkg/utils/grpcutil/grpcutil_test.go @@ -7,10 +7,12 @@ import ( "path/filepath" "testing" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/errs" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + + "github.com/tikv/pd/pkg/errs" ) var ( diff --git a/pkg/utils/logutil/log.go b/pkg/utils/logutil/log.go index 4854fd7ac40..1620f9e3b70 100644 --- a/pkg/utils/logutil/log.go +++ b/pkg/utils/logutil/log.go @@ -21,10 +21,12 @@ import ( "strings" "sync/atomic" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" "go.uber.org/zap" "go.uber.org/zap/zapcore" + + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) // FileLogConfig serializes file log related config in toml/json. diff --git a/pkg/utils/metricutil/metricutil.go b/pkg/utils/metricutil/metricutil.go index 0d73c7678a8..32583248d5b 100644 --- a/pkg/utils/metricutil/metricutil.go +++ b/pkg/utils/metricutil/metricutil.go @@ -19,9 +19,11 @@ import ( "time" "unicode" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/utils/metricutil/metricutil_test.go b/pkg/utils/metricutil/metricutil_test.go index a5c183abc20..33f322d516b 100644 --- a/pkg/utils/metricutil/metricutil_test.go +++ b/pkg/utils/metricutil/metricutil_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/utils/operatorutil/operator_check.go b/pkg/utils/operatorutil/operator_check.go index 78b9e9d9bab..4e0d4332a45 100644 --- a/pkg/utils/operatorutil/operator_check.go +++ b/pkg/utils/operatorutil/operator_check.go @@ -16,6 +16,7 @@ package operatorutil import ( "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" ) diff --git a/pkg/utils/tempurl/check_env_linux.go b/pkg/utils/tempurl/check_env_linux.go index 58f902f4bb7..7e3dffc105b 100644 --- a/pkg/utils/tempurl/check_env_linux.go +++ b/pkg/utils/tempurl/check_env_linux.go @@ -18,7 +18,9 @@ package tempurl import ( "github.com/cakturk/go-netstat/netstat" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/utils/tempurl/tempurl.go b/pkg/utils/tempurl/tempurl.go index fae6f90af91..4984bb57b23 100644 --- a/pkg/utils/tempurl/tempurl.go +++ b/pkg/utils/tempurl/tempurl.go @@ -23,6 +23,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/utils/testutil/api_check.go b/pkg/utils/testutil/api_check.go index 0b714204500..3c5ffbdbe1e 100644 --- a/pkg/utils/testutil/api_check.go +++ b/pkg/utils/testutil/api_check.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/apiutil" ) diff --git a/pkg/utils/testutil/testutil.go b/pkg/utils/testutil/testutil.go index 363b3f14aef..89e706b6b63 100644 --- a/pkg/utils/testutil/testutil.go +++ b/pkg/utils/testutil/testutil.go @@ -21,11 +21,12 @@ import ( "sync" "time" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/log" ) const ( diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index 86afe75e6c5..be7d4fa6d83 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -20,15 +20,17 @@ import ( "sync" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/timerutil" - "go.uber.org/zap" - "google.golang.org/grpc" ) const ( diff --git a/pkg/utils/tsoutil/tso_proto_factory.go b/pkg/utils/tsoutil/tso_proto_factory.go index 9d8c0cfdb76..075a7024e98 100644 --- a/pkg/utils/tsoutil/tso_proto_factory.go +++ b/pkg/utils/tsoutil/tso_proto_factory.go @@ -17,10 +17,12 @@ package tsoutil import ( "context" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" + "github.com/tikv/pd/pkg/utils/grpcutil" - "google.golang.org/grpc" ) // ProtoFactory is the abstract factory for creating tso related data structures defined in the grpc service diff --git a/pkg/utils/tsoutil/tso_request.go b/pkg/utils/tsoutil/tso_request.go index d01247aa4b0..8a1a42a9d00 100644 --- a/pkg/utils/tsoutil/tso_request.go +++ b/pkg/utils/tsoutil/tso_request.go @@ -15,10 +15,12 @@ package tsoutil import ( + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" + "github.com/tikv/pd/pkg/mcs/utils/constant" - "google.golang.org/grpc" ) // Request is an interface wrapping tsopb.TsoRequest and pdpb.TsoRequest so diff --git a/pkg/utils/typeutil/size.go b/pkg/utils/typeutil/size.go index c088ca73e59..1356cef3588 100644 --- a/pkg/utils/typeutil/size.go +++ b/pkg/utils/typeutil/size.go @@ -18,6 +18,7 @@ import ( "strconv" "github.com/docker/go-units" + "github.com/pingcap/errors" ) diff --git a/pkg/versioninfo/feature.go b/pkg/versioninfo/feature.go index 21ad56e8585..24aed22dc9d 100644 --- a/pkg/versioninfo/feature.go +++ b/pkg/versioninfo/feature.go @@ -16,9 +16,11 @@ package versioninfo import ( "github.com/coreos/go-semver/semver" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // Feature supported features. diff --git a/pkg/versioninfo/versioninfo.go b/pkg/versioninfo/versioninfo.go index d6f4738d786..a6d7e910fb5 100644 --- a/pkg/versioninfo/versioninfo.go +++ b/pkg/versioninfo/versioninfo.go @@ -19,9 +19,11 @@ import ( "strings" "github.com/coreos/go-semver/semver" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // Status is the status of PD server. diff --git a/plugin/scheduler_example/evict_leader.go b/plugin/scheduler_example/evict_leader.go index eb976edf851..4e0e700e8ce 100644 --- a/plugin/scheduler_example/evict_leader.go +++ b/plugin/scheduler_example/evict_leader.go @@ -20,8 +20,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/server/api/admin.go b/server/api/admin.go index 24b9feaea75..434508c98df 100644 --- a/server/api/admin.go +++ b/server/api/admin.go @@ -22,14 +22,16 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) type adminHandler struct { diff --git a/server/api/admin_test.go b/server/api/admin_test.go index 4995c57619b..029ddce87cc 100644 --- a/server/api/admin_test.go +++ b/server/api/admin_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/replication" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/checker.go b/server/api/checker.go index 709e641c37b..adf6f7c0ccc 100644 --- a/server/api/checker.go +++ b/server/api/checker.go @@ -18,9 +18,10 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type checkerHandler struct { diff --git a/server/api/cluster.go b/server/api/cluster.go index 2c29101dd2d..e6c4954ff43 100644 --- a/server/api/cluster.go +++ b/server/api/cluster.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type clusterHandler struct { diff --git a/server/api/cluster_test.go b/server/api/cluster_test.go index 0197489ecad..48854ac3fe8 100644 --- a/server/api/cluster_test.go +++ b/server/api/cluster_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/config.go b/server/api/config.go index 511f47284a9..3eda889507f 100644 --- a/server/api/config.go +++ b/server/api/config.go @@ -24,9 +24,12 @@ import ( "strings" "time" + "github.com/unrolled/render" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" sc "github.com/tikv/pd/pkg/schedule/config" @@ -36,7 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - "github.com/unrolled/render" ) // This line is to ensure the package `sc` could always be imported so that diff --git a/server/api/diagnostic.go b/server/api/diagnostic.go index 23016519dee..8df7368f074 100644 --- a/server/api/diagnostic.go +++ b/server/api/diagnostic.go @@ -18,8 +18,9 @@ import ( "net/http" "github.com/gorilla/mux" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type diagnosticHandler struct { diff --git a/server/api/diagnostic_test.go b/server/api/diagnostic_test.go index c85d9a45369..87f652d3a13 100644 --- a/server/api/diagnostic_test.go +++ b/server/api/diagnostic_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/schedule/types" diff --git a/server/api/etcd_api_test.go b/server/api/etcd_api_test.go index cec4e6369e5..5dba77de09d 100644 --- a/server/api/etcd_api_test.go +++ b/server/api/etcd_api_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + tu "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/server/api/health.go b/server/api/health.go index 93f95e8745d..dcfc21175c3 100644 --- a/server/api/health.go +++ b/server/api/health.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/server" "github.com/tikv/pd/server/cluster" - "github.com/unrolled/render" ) type healthHandler struct { diff --git a/server/api/health_test.go b/server/api/health_test.go index 89a9627bc37..40a781a6e60 100644 --- a/server/api/health_test.go +++ b/server/api/health_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" ) diff --git a/server/api/hot_status.go b/server/api/hot_status.go index e4dbf96b904..07b3c165480 100644 --- a/server/api/hot_status.go +++ b/server/api/hot_status.go @@ -21,9 +21,10 @@ import ( "net/http" "strconv" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type hotStatusHandler struct { diff --git a/server/api/hot_status_test.go b/server/api/hot_status_test.go index 1ffb9dd787f..a18b64f1e98 100644 --- a/server/api/hot_status_test.go +++ b/server/api/hot_status_test.go @@ -23,6 +23,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/syndtr/goleveldb/leveldb" + "github.com/tikv/pd/pkg/schedule/handler" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/kv" diff --git a/server/api/label.go b/server/api/label.go index ead6b30ae26..58647e2a592 100644 --- a/server/api/label.go +++ b/server/api/label.go @@ -19,12 +19,14 @@ import ( "regexp" "strings" + "github.com/unrolled/render" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type labelsHandler struct { diff --git a/server/api/label_test.go b/server/api/label_test.go index 70e4ab80066..92a59302753 100644 --- a/server/api/label_test.go +++ b/server/api/label_test.go @@ -19,9 +19,11 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/keypath" diff --git a/server/api/log.go b/server/api/log.go index 3359e70b115..6c2182694c1 100644 --- a/server/api/log.go +++ b/server/api/log.go @@ -19,10 +19,12 @@ import ( "io" "net/http" + "github.com/unrolled/render" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type logHandler struct { diff --git a/server/api/log_test.go b/server/api/log_test.go index 2478f6f4e54..641d873192b 100644 --- a/server/api/log_test.go +++ b/server/api/log_test.go @@ -19,8 +19,10 @@ import ( "fmt" "testing" - "github.com/pingcap/log" "github.com/stretchr/testify/suite" + + "github.com/pingcap/log" + tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" ) diff --git a/server/api/member.go b/server/api/member.go index 5161322463a..e9d46459a91 100644 --- a/server/api/member.go +++ b/server/api/member.go @@ -21,16 +21,18 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) type memberHandler struct { diff --git a/server/api/member_test.go b/server/api/member_test.go index b56e84b2a1a..92fb7970f0d 100644 --- a/server/api/member_test.go +++ b/server/api/member_test.go @@ -24,13 +24,15 @@ import ( "strings" "testing" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" ) type memberTestSuite struct { diff --git a/server/api/middleware.go b/server/api/middleware.go index fd0d81412ea..818d0d84e1d 100644 --- a/server/api/middleware.go +++ b/server/api/middleware.go @@ -19,14 +19,16 @@ import ( "net/http" "time" + "github.com/unrolled/render" + "github.com/urfave/negroni" + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/audit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/requestutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/cluster" - "github.com/unrolled/render" - "github.com/urfave/negroni" ) // serviceMiddlewareBuilder is used to build service middleware for HTTP api diff --git a/server/api/min_resolved_ts.go b/server/api/min_resolved_ts.go index 1edf924370f..584eb649db7 100644 --- a/server/api/min_resolved_ts.go +++ b/server/api/min_resolved_ts.go @@ -20,9 +20,10 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type minResolvedTSHandler struct { diff --git a/server/api/min_resolved_ts_test.go b/server/api/min_resolved_ts_test.go index 88c01602eab..f999af3fe56 100644 --- a/server/api/min_resolved_ts_test.go +++ b/server/api/min_resolved_ts_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/operator.go b/server/api/operator.go index c2529240804..6bcb0699d7b 100644 --- a/server/api/operator.go +++ b/server/api/operator.go @@ -20,10 +20,11 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type operatorHandler struct { diff --git a/server/api/plugin_disable.go b/server/api/plugin_disable.go index 596cddac5d7..ea7df0577ea 100644 --- a/server/api/plugin_disable.go +++ b/server/api/plugin_disable.go @@ -20,8 +20,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type pluginHandler struct{} diff --git a/server/api/pprof.go b/server/api/pprof.go index ee4398c60b2..071dd1b2a4b 100644 --- a/server/api/pprof.go +++ b/server/api/pprof.go @@ -26,11 +26,13 @@ import ( "strconv" "time" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) // pprofHandler pprof handler diff --git a/server/api/pprof_test.go b/server/api/pprof_test.go index b43feeab108..1bb1668e374 100644 --- a/server/api/pprof_test.go +++ b/server/api/pprof_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" ) diff --git a/server/api/region.go b/server/api/region.go index ae25d659544..afc32d2e762 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -24,7 +24,10 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type regionHandler struct { diff --git a/server/api/region_label.go b/server/api/region_label.go index 7958bacd371..2fa6466d480 100644 --- a/server/api/region_label.go +++ b/server/api/region_label.go @@ -20,11 +20,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type regionLabelHandler struct { diff --git a/server/api/region_label_test.go b/server/api/region_label_test.go index 1ed0997b8e7..a35f77be1cf 100644 --- a/server/api/region_label_test.go +++ b/server/api/region_label_test.go @@ -21,8 +21,10 @@ import ( "sort" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/schedule/labeler" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/region_test.go b/server/api/region_test.go index c73dc02587d..ae91b41ef5e 100644 --- a/server/api/region_test.go +++ b/server/api/region_test.go @@ -27,10 +27,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/replication_mode.go b/server/api/replication_mode.go index 4fe2ef5da09..2e5a92c2195 100644 --- a/server/api/replication_mode.go +++ b/server/api/replication_mode.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type replicationModeHandler struct { diff --git a/server/api/router.go b/server/api/router.go index 6ea52304b63..fd999211542 100644 --- a/server/api/router.go +++ b/server/api/router.go @@ -22,13 +22,15 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/audit" "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // enableFailPointAPI enable fail point API handler. diff --git a/server/api/rule.go b/server/api/rule.go index 27a95a08269..4bd062e92c9 100644 --- a/server/api/rule.go +++ b/server/api/rule.go @@ -22,11 +22,12 @@ import ( "net/url" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type ruleHandler struct { diff --git a/server/api/scheduler.go b/server/api/scheduler.go index 8922a833a07..b2d18012c89 100644 --- a/server/api/scheduler.go +++ b/server/api/scheduler.go @@ -21,16 +21,18 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) type schedulerHandler struct { diff --git a/server/api/server.go b/server/api/server.go index 8a58669de3b..1a744635e2d 100644 --- a/server/api/server.go +++ b/server/api/server.go @@ -20,13 +20,14 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/urfave/negroni" + scheapi "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" tsoapi "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/serverapi" "github.com/tikv/pd/server" - "github.com/urfave/negroni" ) const apiPrefix = "/pd" diff --git a/server/api/server_test.go b/server/api/server_test.go index f2ff4ffb452..9b99a6c32f8 100644 --- a/server/api/server_test.go +++ b/server/api/server_test.go @@ -23,11 +23,14 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - "go.uber.org/goleak" ) var ( diff --git a/server/api/service_gc_safepoint.go b/server/api/service_gc_safepoint.go index ca29f9c352f..ab3002d1297 100644 --- a/server/api/service_gc_safepoint.go +++ b/server/api/service_gc_safepoint.go @@ -19,9 +19,10 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type serviceGCSafepointHandler struct { diff --git a/server/api/service_gc_safepoint_test.go b/server/api/service_gc_safepoint_test.go index 6325babd438..33f78570e2d 100644 --- a/server/api/service_gc_safepoint_test.go +++ b/server/api/service_gc_safepoint_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/service_middleware.go b/server/api/service_middleware.go index da40a79cbea..6f11fb58cf4 100644 --- a/server/api/service_middleware.go +++ b/server/api/service_middleware.go @@ -22,15 +22,16 @@ import ( "reflect" "strings" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/jsonutil" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - - "github.com/unrolled/render" ) type serviceMiddlewareHandler struct { diff --git a/server/api/service_middleware_test.go b/server/api/service_middleware_test.go index 2599ccbfb54..b91f07560aa 100644 --- a/server/api/service_middleware_test.go +++ b/server/api/service_middleware_test.go @@ -20,8 +20,10 @@ import ( "net/http" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/ratelimit" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/stats.go b/server/api/stats.go index 5aa8fcb72a6..8672d562fd4 100644 --- a/server/api/stats.go +++ b/server/api/stats.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type statsHandler struct { diff --git a/server/api/stats_test.go b/server/api/stats_test.go index 1485f9eb5af..9c219ea58fd 100644 --- a/server/api/stats_test.go +++ b/server/api/stats_test.go @@ -19,8 +19,10 @@ import ( "net/url" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/status.go b/server/api/status.go index e25f5da5287..3be84c3f598 100644 --- a/server/api/status.go +++ b/server/api/status.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type statusHandler struct { diff --git a/server/api/status_test.go b/server/api/status_test.go index 5444fda77b4..f75259a7728 100644 --- a/server/api/status_test.go +++ b/server/api/status_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/versioninfo" ) diff --git a/server/api/store.go b/server/api/store.go index 1342645cdb9..fb3e9cfb964 100644 --- a/server/api/store.go +++ b/server/api/store.go @@ -23,10 +23,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/response" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type storeHandler struct { diff --git a/server/api/store_test.go b/server/api/store_test.go index 47c7045ae02..dd0d1e1fe8b 100644 --- a/server/api/store_test.go +++ b/server/api/store_test.go @@ -25,10 +25,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/keypath" diff --git a/server/api/trend.go b/server/api/trend.go index 6a32b875f89..74dc43c53b5 100644 --- a/server/api/trend.go +++ b/server/api/trend.go @@ -18,13 +18,14 @@ import ( "net/http" "time" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // Trend describes the cluster's schedule trend. diff --git a/server/api/trend_test.go b/server/api/trend_test.go index 4e93d0bea93..4e4a1be1f08 100644 --- a/server/api/trend_test.go +++ b/server/api/trend_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/tso.go b/server/api/tso.go index 95096257066..eca9dc7f8b4 100644 --- a/server/api/tso.go +++ b/server/api/tso.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type tsoHandler struct { diff --git a/server/api/unsafe_operation.go b/server/api/unsafe_operation.go index dc41ec336e3..ec3da8a25be 100644 --- a/server/api/unsafe_operation.go +++ b/server/api/unsafe_operation.go @@ -17,10 +17,11 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type unsafeOperationHandler struct { diff --git a/server/api/unsafe_operation_test.go b/server/api/unsafe_operation_test.go index e708a93abfc..24cf9d7bbc3 100644 --- a/server/api/unsafe_operation_test.go +++ b/server/api/unsafe_operation_test.go @@ -19,8 +19,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/unsaferecovery" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/version.go b/server/api/version.go index 99a8a7376e5..fdc07a8b417 100644 --- a/server/api/version.go +++ b/server/api/version.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/pkg/versioninfo" "github.com/unrolled/render" + + "github.com/tikv/pd/pkg/versioninfo" ) // NOTE: This type is exported by HTTP API. Please pay more attention when modifying it. diff --git a/server/api/version_test.go b/server/api/version_test.go index 46bc80537a6..5342b1d9ebb 100644 --- a/server/api/version_test.go +++ b/server/api/version_test.go @@ -22,8 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index 89d0634ce8f..0245016734c 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -22,8 +22,10 @@ import ( "time" "github.com/gin-gonic/gin" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/server" diff --git a/server/apiv2/handlers/micro_service.go b/server/apiv2/handlers/micro_service.go index fd44665530f..c7fa0dc94f2 100644 --- a/server/apiv2/handlers/micro_service.go +++ b/server/apiv2/handlers/micro_service.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/server" "github.com/tikv/pd/server/apiv2/middlewares" diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index e99e8cf55a4..99a4e55c29f 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -20,7 +20,9 @@ import ( "strings" "github.com/gin-gonic/gin" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" diff --git a/server/apiv2/middlewares/bootstrap_checker.go b/server/apiv2/middlewares/bootstrap_checker.go index 794316d3d0f..712cf71e491 100644 --- a/server/apiv2/middlewares/bootstrap_checker.go +++ b/server/apiv2/middlewares/bootstrap_checker.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/server" ) diff --git a/server/apiv2/middlewares/redirector.go b/server/apiv2/middlewares/redirector.go index 9c2c4081175..fb6b0496bec 100644 --- a/server/apiv2/middlewares/redirector.go +++ b/server/apiv2/middlewares/redirector.go @@ -19,11 +19,13 @@ import ( "net/url" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "go.uber.org/zap" ) // Redirector is a middleware to redirect the request to the right place. diff --git a/server/apiv2/router.go b/server/apiv2/router.go index 781a1cca87b..612ba8932ff 100644 --- a/server/apiv2/router.go +++ b/server/apiv2/router.go @@ -19,6 +19,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 10e9bf7ff1a..5ecd787956d 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -30,11 +30,15 @@ import ( "time" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" @@ -69,8 +73,6 @@ import ( "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var ( diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 99bb60b5558..11b5743fa13 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -27,13 +27,15 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/server/cluster/cluster_worker.go b/server/cluster/cluster_worker.go index c14ceff2153..c24b509549a 100644 --- a/server/cluster/cluster_worker.go +++ b/server/cluster/cluster_worker.go @@ -17,10 +17,13 @@ package cluster import ( "bytes" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -30,7 +33,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) // HandleRegionHeartbeat processes RegionInfo reports from client. diff --git a/server/cluster/cluster_worker_test.go b/server/cluster/cluster_worker_test.go index afc979e2b97..150afc0c8a9 100644 --- a/server/cluster/cluster_worker_test.go +++ b/server/cluster/cluster_worker_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/storage" ) diff --git a/server/cluster/scheduling_controller.go b/server/cluster/scheduling_controller.go index b182410c016..fd9c590ef68 100644 --- a/server/cluster/scheduling_controller.go +++ b/server/cluster/scheduling_controller.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule" "github.com/tikv/pd/pkg/schedule/checker" diff --git a/server/config/config.go b/server/config/config.go index a5183f119f2..282b5264fe9 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -28,9 +28,14 @@ import ( "github.com/BurntSushi/toml" "github.com/coreos/go-semver/semver" "github.com/docker/go-units" + "github.com/spf13/pflag" + "go.etcd.io/etcd/client/pkg/v3/transport" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/errs" rm "github.com/tikv/pd/pkg/mcs/resourcemanager/server" sc "github.com/tikv/pd/pkg/schedule/config" @@ -39,9 +44,6 @@ import ( "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.etcd.io/etcd/client/pkg/v3/transport" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) // Config is the pd server configuration. diff --git a/server/config/config_test.go b/server/config/config_test.go index 78d6d25b73e..cd9447a5824 100644 --- a/server/config/config_test.go +++ b/server/config/config_test.go @@ -26,6 +26,7 @@ import ( "github.com/BurntSushi/toml" "github.com/spf13/pflag" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/storage" diff --git a/server/config/persist_options.go b/server/config/persist_options.go index c426e9d2420..59d42383743 100644 --- a/server/config/persist_options.go +++ b/server/config/persist_options.go @@ -25,10 +25,14 @@ import ( "unsafe" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -38,8 +42,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // PersistOptions wraps all configurations that need to persist to storage and diff --git a/server/config/service_middleware_persist_options.go b/server/config/service_middleware_persist_options.go index cd67c9dd1ac..ae4c0ef9b7a 100644 --- a/server/config/service_middleware_persist_options.go +++ b/server/config/service_middleware_persist_options.go @@ -19,6 +19,7 @@ import ( "sync/atomic" "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/server/config/util.go b/server/config/util.go index 2f25e711cc7..f2bc39459fd 100644 --- a/server/config/util.go +++ b/server/config/util.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" ) diff --git a/server/forward.go b/server/forward.go index 674b3e008b6..b3d0d63c81b 100644 --- a/server/forward.go +++ b/server/forward.go @@ -20,12 +20,18 @@ import ( "strings" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/grpcutil" @@ -33,10 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/server/cluster" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) func forwardTSORequest( diff --git a/server/gc_service.go b/server/gc_service.go index 114482fdd39..e9e52333905 100644 --- a/server/gc_service.go +++ b/server/gc_service.go @@ -22,16 +22,18 @@ import ( "path" "strings" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tsoutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" ) // GetGCSafePointV2 return gc safe point for the given keyspace. diff --git a/server/grpc_service.go b/server/grpc_service.go index de74916900e..d10421e87d7 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -28,6 +28,13 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/multierr" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" @@ -35,6 +42,7 @@ import ( "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -47,12 +55,6 @@ import ( "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/cluster" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/multierr" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/server/handler.go b/server/handler.go index 2ecf7763ce2..9fa0ca6a5c5 100644 --- a/server/handler.go +++ b/server/handler.go @@ -22,8 +22,11 @@ import ( "path/filepath" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/encryption" @@ -42,7 +45,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - "go.uber.org/zap" ) // SchedulerConfigHandlerPath is the api router path of the schedule config handler. diff --git a/server/join/join.go b/server/join/join.go index e77675f2196..26b6f98667d 100644 --- a/server/join/join.go +++ b/server/join/join.go @@ -21,15 +21,17 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) const ( diff --git a/server/join/join_test.go b/server/join/join_test.go index edbcb7a3077..b89a2ed10ac 100644 --- a/server/join/join_test.go +++ b/server/join/join_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 967457198a9..e110742aa45 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -20,13 +20,15 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" ) // KeyspaceServer wraps GrpcServer to provide keyspace service. diff --git a/server/server.go b/server/server.go index 7df3da153ba..a2cc32db9dd 100644 --- a/server/server.go +++ b/server/server.go @@ -32,6 +32,13 @@ import ( "github.com/coreos/go-semver/semver" "github.com/gogo/protobuf/proto" "github.com/gorilla/mux" + "go.etcd.io/etcd/api/v3/mvccpb" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -41,6 +48,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/pingcap/sysutil" + "github.com/tikv/pd/pkg/audit" bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/cgroup" @@ -53,8 +61,6 @@ import ( ms_server "github.com/tikv/pd/pkg/mcs/metastorage/server" "github.com/tikv/pd/pkg/mcs/registry" rm_server "github.com/tikv/pd/pkg/mcs/resourcemanager/server" - _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1" // init API group - _ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/ratelimit" @@ -80,12 +86,9 @@ import ( "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - "go.etcd.io/etcd/api/v3/mvccpb" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" - "google.golang.org/grpc" + + _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1" // init API group + _ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group ) const ( diff --git a/server/server_test.go b/server/server_test.go index 7dd91b9f61f..23da2078cb2 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -25,15 +25,16 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/server/testutil.go b/server/testutil.go index 2c844bf3d6b..d8bb9bb1cd9 100644 --- a/server/testutil.go +++ b/server/testutil.go @@ -23,8 +23,11 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/server/v3/embed" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - "go.etcd.io/etcd/server/v3/embed" ) // NewTestServer creates a pd server for testing. diff --git a/server/util.go b/server/util.go index 0b396b0f5df..1764e4e9850 100644 --- a/server/util.go +++ b/server/util.go @@ -22,16 +22,18 @@ import ( "github.com/coreos/go-semver/semver" "github.com/gorilla/mux" + "github.com/urfave/negroni" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/config" - "github.com/urfave/negroni" - "go.uber.org/zap" ) // CheckAndGetPDVersion checks and returns the PD version. diff --git a/tests/autoscaling/autoscaling_test.go b/tests/autoscaling/autoscaling_test.go index 10d10378323..c00bc00afe0 100644 --- a/tests/autoscaling/autoscaling_test.go +++ b/tests/autoscaling/autoscaling_test.go @@ -21,9 +21,10 @@ import ( "testing" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/cluster.go b/tests/cluster.go index 21807d2aadc..a4f445155e1 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -22,10 +22,13 @@ import ( "time" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/autoscaling" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/dashboard" @@ -46,7 +49,6 @@ import ( "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/join" - clientv3 "go.etcd.io/etcd/client/v3" ) // TestServer states. diff --git a/tests/compatibility/version_upgrade_test.go b/tests/compatibility/version_upgrade_test.go index 27eb07b0da2..7b5fd05071b 100644 --- a/tests/compatibility/version_upgrade_test.go +++ b/tests/compatibility/version_upgrade_test.go @@ -19,10 +19,12 @@ import ( "testing" "github.com/coreos/go-semver/semver" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/server" "github.com/tikv/pd/tests" ) diff --git a/tests/config.go b/tests/config.go index fcda28d4a3d..c1cb366dfa9 100644 --- a/tests/config.go +++ b/tests/config.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/server/config" ) diff --git a/tests/dashboard/race_test.go b/tests/dashboard/race_test.go index ad2d673f40d..6d117760f8f 100644 --- a/tests/dashboard/race_test.go +++ b/tests/dashboard/race_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/tests" ) diff --git a/tests/dashboard/service_test.go b/tests/dashboard/service_test.go index e689b1d4662..2c29230f787 100644 --- a/tests/dashboard/service_test.go +++ b/tests/dashboard/service_test.go @@ -26,11 +26,12 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 0462a6d9ea0..9ba63f0c83f 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -31,12 +31,18 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" @@ -56,10 +62,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/integrations/mcs" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/tests/integrations/client/client_tls_test.go b/tests/integrations/client/client_tls_test.go index fb76b4a0be1..40a36205de0 100644 --- a/tests/integrations/client/client_tls_test.go +++ b/tests/integrations/client/client_tls_test.go @@ -27,6 +27,9 @@ import ( "time" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + "google.golang.org/grpc" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -34,8 +37,6 @@ import ( "github.com/tikv/pd/pkg/utils/netutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" - "google.golang.org/grpc" ) var certScript = strings.Join([]string{".", "cert_opt.sh"}, string(filepath.Separator)) diff --git a/tests/integrations/client/gc_client_test.go b/tests/integrations/client/gc_client_test.go index 17db7345b81..0ea0c3daf19 100644 --- a/tests/integrations/client/gc_client_test.go +++ b/tests/integrations/client/gc_client_test.go @@ -20,20 +20,22 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" ) // gcClientTestReceiver is the pdpb.PD_WatchGCSafePointV2Server mock for testing. diff --git a/tests/integrations/client/global_config_test.go b/tests/integrations/client/global_config_test.go index 81aac43a293..04ef3992a2f 100644 --- a/tests/integrations/client/global_config_test.go +++ b/tests/integrations/client/global_config_test.go @@ -21,19 +21,20 @@ import ( "testing" "time" - pd "github.com/tikv/pd/client" - "github.com/tikv/pd/client/pkg/caller" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/zap" + "google.golang.org/grpc" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + + pd "github.com/tikv/pd/client" + "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" - "go.uber.org/zap" - "google.golang.org/grpc" ) const globalConfigPath = "/global/config/" diff --git a/tests/integrations/client/http_client_test.go b/tests/integrations/client/http_client_test.go index d0ff81f9fad..0cbf12261f3 100644 --- a/tests/integrations/client/http_client_test.go +++ b/tests/integrations/client/http_client_test.go @@ -25,13 +25,15 @@ import ( "testing" "time" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + pd "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/retry" "github.com/tikv/pd/pkg/core" diff --git a/tests/integrations/client/keyspace_test.go b/tests/integrations/client/keyspace_test.go index 841859bbb80..3c976ebbc57 100644 --- a/tests/integrations/client/keyspace_test.go +++ b/tests/integrations/client/keyspace_test.go @@ -19,8 +19,10 @@ import ( "math" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" diff --git a/tests/integrations/mcs/discovery/register_test.go b/tests/integrations/mcs/discovery/register_test.go index 147e16530b7..da6fa158307 100644 --- a/tests/integrations/mcs/discovery/register_test.go +++ b/tests/integrations/mcs/discovery/register_test.go @@ -21,13 +21,14 @@ import ( "time" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index c6b6070fda5..44347b4757d 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/tests/integrations/mcs/members/member_test.go b/tests/integrations/mcs/members/member_test.go index e1953de4e62..28275849073 100644 --- a/tests/integrations/mcs/members/member_test.go +++ b/tests/integrations/mcs/members/member_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + pdClient "github.com/tikv/pd/client/http" bs "github.com/tikv/pd/pkg/basicserver" tso "github.com/tikv/pd/pkg/mcs/tso/server" diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 22cb6b64c0a..2eb66df2e2b 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -26,11 +26,15 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "go.uber.org/zap" + "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/resource_group/controller" @@ -39,8 +43,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" - "go.uber.org/zap" // Register Service _ "github.com/tikv/pd/pkg/mcs/registry" diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index f3e7f235018..14b867a587d 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -10,11 +10,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" - _ "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/handler" @@ -27,6 +28,8 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/tests" + + _ "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" ) type apiTestSuite struct { diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index 6a41ad0823e..d7d200814bb 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" diff --git a/tests/integrations/mcs/scheduling/meta_test.go b/tests/integrations/mcs/scheduling/meta_test.go index 0798e9f129e..4e0d5249fdb 100644 --- a/tests/integrations/mcs/scheduling/meta_test.go +++ b/tests/integrations/mcs/scheduling/meta_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/meta" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/integrations/mcs/scheduling/rule_test.go b/tests/integrations/mcs/scheduling/rule_test.go index e8b394ec075..a137619afbf 100644 --- a/tests/integrations/mcs/scheduling/rule_test.go +++ b/tests/integrations/mcs/scheduling/rule_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index 085b87afe86..d3850e4667c 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -23,11 +23,14 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/operator" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/server" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/server/api" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/mcs/testutil.go b/tests/integrations/mcs/testutil.go index d9dc7def3d1..b605462c044 100644 --- a/tests/integrations/mcs/testutil.go +++ b/tests/integrations/mcs/testutil.go @@ -19,6 +19,7 @@ import ( "sync" "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index 7abf1503bec..91614530ef1 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + tso "github.com/tikv/pd/pkg/mcs/tso/server" apis "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" "github.com/tikv/pd/pkg/mcs/utils/constant" diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index fc90241eb81..2c19f6588e5 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -25,10 +25,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/pdpb" + pd "github.com/tikv/pd/client" clierrs "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index a15cbb1cd03..b564076c1f0 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -24,17 +24,19 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) type tsoProxyTestSuite struct { diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index b0660ed6577..09a199c2d52 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -24,10 +24,16 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -45,10 +51,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/integrations/mcs" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/realcluster/cluster.go b/tests/integrations/realcluster/cluster.go index b90ec3cb4d9..cc4f6b54713 100644 --- a/tests/integrations/realcluster/cluster.go +++ b/tests/integrations/realcluster/cluster.go @@ -23,10 +23,11 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.uber.org/zap" + + "github.com/pingcap/log" ) type clusterSuite struct { diff --git a/tests/integrations/realcluster/cluster_id_test.go b/tests/integrations/realcluster/cluster_id_test.go index b6d4bc4e74d..82233cd5b8c 100644 --- a/tests/integrations/realcluster/cluster_id_test.go +++ b/tests/integrations/realcluster/cluster_id_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/realcluster/etcd_key_test.go b/tests/integrations/realcluster/etcd_key_test.go index ee962cb0396..10c392834af 100644 --- a/tests/integrations/realcluster/etcd_key_test.go +++ b/tests/integrations/realcluster/etcd_key_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/pkg/utils/testutil" ) diff --git a/tests/integrations/realcluster/mock_db.go b/tests/integrations/realcluster/mock_db.go index 2a636b9b86b..8a21bc1bb7d 100644 --- a/tests/integrations/realcluster/mock_db.go +++ b/tests/integrations/realcluster/mock_db.go @@ -20,11 +20,12 @@ import ( "github.com/DATA-DOG/go-sqlmock" mysqldriver "github.com/go-sql-driver/mysql" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "gorm.io/driver/mysql" "gorm.io/gorm" "moul.io/zapgorm2" + + "github.com/pingcap/log" ) // TestDB is a test database diff --git a/tests/integrations/realcluster/reboot_pd_test.go b/tests/integrations/realcluster/reboot_pd_test.go index d35a921cb74..1a042872b93 100644 --- a/tests/integrations/realcluster/reboot_pd_test.go +++ b/tests/integrations/realcluster/reboot_pd_test.go @@ -20,6 +20,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/http" ) diff --git a/tests/integrations/realcluster/scheduler_test.go b/tests/integrations/realcluster/scheduler_test.go index 8456e053bf8..0e4d10acd80 100644 --- a/tests/integrations/realcluster/scheduler_test.go +++ b/tests/integrations/realcluster/scheduler_test.go @@ -21,14 +21,16 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/utils/testutil" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type schedulerSuite struct { diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index fa2cc80379d..2cda9f8734f 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -25,9 +25,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/opt" diff --git a/tests/integrations/tso/consistency_test.go b/tests/integrations/tso/consistency_test.go index 2ef2ebe3077..147f41a4591 100644 --- a/tests/integrations/tso/consistency_test.go +++ b/tests/integrations/tso/consistency_test.go @@ -20,10 +20,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" - "github.com/stretchr/testify/suite" + tso "github.com/tikv/pd/pkg/mcs/tso/server" tsopkg "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/keypath" @@ -31,7 +34,6 @@ import ( tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/tests" - "google.golang.org/grpc" ) type tsoConsistencyTestSuite struct { diff --git a/tests/integrations/tso/server_test.go b/tests/integrations/tso/server_test.go index c0d71050964..f03db197b35 100644 --- a/tests/integrations/tso/server_test.go +++ b/tests/integrations/tso/server_test.go @@ -20,16 +20,18 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" - "github.com/stretchr/testify/suite" + tso "github.com/tikv/pd/pkg/mcs/tso/server" tsopkg "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tempurl" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "google.golang.org/grpc" ) type tsoServerTestSuite struct { diff --git a/tests/integrations/tso/testutil.go b/tests/integrations/tso/testutil.go index 5eda641486f..25f10e24d65 100644 --- a/tests/integrations/tso/testutil.go +++ b/tests/integrations/tso/testutil.go @@ -15,8 +15,9 @@ package tso import ( - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/pdpb" ) const ( diff --git a/tests/registry/registry_test.go b/tests/registry/registry_test.go index 3551782d753..87d79c4f896 100644 --- a/tests/registry/registry_test.go +++ b/tests/registry/registry_test.go @@ -21,17 +21,19 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/interop/grpc_testing" + + "github.com/pingcap/failpoint" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/interop/grpc_testing" ) func TestMain(m *testing.M) { diff --git a/tests/scheduling_cluster.go b/tests/scheduling_cluster.go index 3f7c39eb81c..b5fc2429043 100644 --- a/tests/scheduling_cluster.go +++ b/tests/scheduling_cluster.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" sc "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/schedule/schedulers" diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index e1e4db3a26d..00c43d11309 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -27,11 +27,14 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" @@ -40,7 +43,6 @@ import ( "github.com/tikv/pd/server/api" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/api/checker_test.go b/tests/server/api/checker_test.go index 190e02ce607..14077ec30ae 100644 --- a/tests/server/api/checker_test.go +++ b/tests/server/api/checker_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" ) diff --git a/tests/server/api/operator_test.go b/tests/server/api/operator_test.go index 857ea6b3cdd..fd08a5ed556 100644 --- a/tests/server/api/operator_test.go +++ b/tests/server/api/operator_test.go @@ -24,8 +24,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/tests/server/api/region_test.go b/tests/server/api/region_test.go index 3a187629830..b98b4419a80 100644 --- a/tests/server/api/region_test.go +++ b/tests/server/api/region_test.go @@ -22,10 +22,12 @@ import ( "strconv" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/server/api/rule_test.go b/tests/server/api/rule_test.go index 3e8adabd4ca..3982e61b6ea 100644 --- a/tests/server/api/rule_test.go +++ b/tests/server/api/rule_test.go @@ -25,9 +25,11 @@ import ( "sync" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/tests/server/api/scheduler_test.go b/tests/server/api/scheduler_test.go index 281f7c1dc7e..1f76c469cfd 100644 --- a/tests/server/api/scheduler_test.go +++ b/tests/server/api/scheduler_test.go @@ -24,10 +24,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/slice" diff --git a/tests/server/api/testutil.go b/tests/server/api/testutil.go index 163a25c9bbb..0546bf83ee5 100644 --- a/tests/server/api/testutil.go +++ b/tests/server/api/testutil.go @@ -23,6 +23,7 @@ import ( "path" "github.com/stretchr/testify/require" + "github.com/tikv/pd/tests" ) diff --git a/tests/server/apiv2/handlers/keyspace_test.go b/tests/server/apiv2/handlers/keyspace_test.go index f3fa01851ac..ea8f14cd971 100644 --- a/tests/server/apiv2/handlers/keyspace_test.go +++ b/tests/server/apiv2/handlers/keyspace_test.go @@ -19,15 +19,17 @@ import ( "fmt" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/apiv2/handlers/testutil.go b/tests/server/apiv2/handlers/testutil.go index 1a40e8d1ac7..e5280183c52 100644 --- a/tests/server/apiv2/handlers/testutil.go +++ b/tests/server/apiv2/handlers/testutil.go @@ -21,8 +21,10 @@ import ( "io" "net/http" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/tests/server/apiv2/handlers/tso_keyspace_group_test.go b/tests/server/apiv2/handlers/tso_keyspace_group_test.go index 91a07ccd6b1..796fd514eef 100644 --- a/tests/server/apiv2/handlers/tso_keyspace_group_test.go +++ b/tests/server/apiv2/handlers/tso_keyspace_group_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index dfdb9cb8685..b7467eb99a5 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -27,11 +27,15 @@ import ( "github.com/coreos/go-semver/semver" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/dashboard" @@ -55,8 +59,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/server/api" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/tests/server/cluster/cluster_work_test.go b/tests/server/cluster/cluster_work_test.go index 9c3bf799116..e0ba916bf7b 100644 --- a/tests/server/cluster/cluster_work_test.go +++ b/tests/server/cluster/cluster_work_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/server/config/config_test.go b/tests/server/config/config_test.go index d225614fa96..bb387b68030 100644 --- a/tests/server/config/config_test.go +++ b/tests/server/config/config_test.go @@ -26,6 +26,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + cfg "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/tests/server/id/id_test.go b/tests/server/id/id_test.go index 465259063bf..e4633c480f2 100644 --- a/tests/server/id/id_test.go +++ b/tests/server/id/id_test.go @@ -19,13 +19,15 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/join/join_fail/join_fail_test.go b/tests/server/join/join_fail/join_fail_test.go index 0e376281070..a037aa9e7b5 100644 --- a/tests/server/join/join_fail/join_fail_test.go +++ b/tests/server/join/join_fail/join_fail_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/tests" ) diff --git a/tests/server/join/join_test.go b/tests/server/join/join_test.go index ea5eaaa35f4..8b0ce918377 100644 --- a/tests/server/join/join_test.go +++ b/tests/server/join/join_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/join" diff --git a/tests/server/keyspace/keyspace_test.go b/tests/server/keyspace/keyspace_test.go index a0175c1b727..5ce25794516 100644 --- a/tests/server/keyspace/keyspace_test.go +++ b/tests/server/keyspace/keyspace_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/tests/server/member/member_test.go b/tests/server/member/member_test.go index fd08e6557e8..cb8fd7bda00 100644 --- a/tests/server/member/member_test.go +++ b/tests/server/member/member_test.go @@ -25,17 +25,19 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/region_syncer/region_syncer_test.go b/tests/server/region_syncer/region_syncer_test.go index 80f5186f904..7edcaa5ad9b 100644 --- a/tests/server/region_syncer/region_syncer_test.go +++ b/tests/server/region_syncer/region_syncer_test.go @@ -19,14 +19,16 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/server_test.go b/tests/server/server_test.go index a5fcd33d2bc..77cd7aa5158 100644 --- a/tests/server/server_test.go +++ b/tests/server/server_test.go @@ -21,15 +21,17 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/storage/hot_region_storage_test.go b/tests/server/storage/hot_region_storage_test.go index 2eccb4b28e9..0643dda08d5 100644 --- a/tests/server/storage/hot_region_storage_test.go +++ b/tests/server/storage/hot_region_storage_test.go @@ -19,9 +19,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/tests/server/tso/tso_test.go b/tests/server/tso/tso_test.go index cb6b87c83d3..51189966878 100644 --- a/tests/server/tso/tso_test.go +++ b/tests/server/tso/tso_test.go @@ -19,16 +19,18 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/watch/leader_watch_test.go b/tests/server/watch/leader_watch_test.go index 39c54b5c500..0a1a53c292d 100644 --- a/tests/server/watch/leader_watch_test.go +++ b/tests/server/watch/leader_watch_test.go @@ -19,12 +19,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/testutil.go b/tests/testutil.go index b56fd245bd3..4f2a6beb261 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -29,11 +29,14 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/core" scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" @@ -46,7 +49,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "go.uber.org/zap" ) var ( diff --git a/tests/tso_cluster.go b/tests/tso_cluster.go index 28a52580a1f..a6e664abb5b 100644 --- a/tests/tso_cluster.go +++ b/tests/tso_cluster.go @@ -20,8 +20,10 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" + + "github.com/pingcap/errors" + tso "github.com/tikv/pd/pkg/mcs/tso/server" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/tools/pd-analysis/analysis/transfer_counter.go b/tools/pd-analysis/analysis/transfer_counter.go index 98d131117b4..80401c44fcc 100644 --- a/tools/pd-analysis/analysis/transfer_counter.go +++ b/tools/pd-analysis/analysis/transfer_counter.go @@ -23,8 +23,9 @@ import ( "strings" "sync" - "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" + + "github.com/tikv/pd/pkg/utils/syncutil" ) // TransferCounter is to count transfer schedule for judging whether redundant diff --git a/tools/pd-analysis/main.go b/tools/pd-analysis/main.go index 510448ddddd..3188e7707f6 100644 --- a/tools/pd-analysis/main.go +++ b/tools/pd-analysis/main.go @@ -18,9 +18,11 @@ import ( "flag" "os" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/tools/pd-analysis/analysis" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-api-bench/cases/cases.go b/tools/pd-api-bench/cases/cases.go index 118b8aaed5e..5d63b2356f6 100644 --- a/tools/pd-api-bench/cases/cases.go +++ b/tools/pd-api-bench/cases/cases.go @@ -21,12 +21,14 @@ import ( "strconv" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/opt" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-api-bench/cases/controller.go b/tools/pd-api-bench/cases/controller.go index d6bc39d6d35..c0098bced45 100644 --- a/tools/pd-api-bench/cases/controller.go +++ b/tools/pd-api-bench/cases/controller.go @@ -19,12 +19,14 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var base = int64(time.Second) / int64(time.Microsecond) diff --git a/tools/pd-api-bench/config/config.go b/tools/pd-api-bench/config/config.go index 47a02461cfd..c1c17624e2f 100644 --- a/tools/pd-api-bench/config/config.go +++ b/tools/pd-api-bench/config/config.go @@ -15,12 +15,14 @@ package config import ( + flag "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - flag "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/tools/pd-api-bench/cases" - "go.uber.org/zap" ) // Config is the heartbeat-bench configuration. diff --git a/tools/pd-api-bench/main.go b/tools/pd-api-bench/main.go index 2b96b0a4115..0d9235d5055 100644 --- a/tools/pd-api-bench/main.go +++ b/tools/pd-api-bench/main.go @@ -29,10 +29,16 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" flag "github.com/spf13/pflag" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/opt" @@ -42,10 +48,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/tools/pd-api-bench/cases" "github.com/tikv/pd/tools/pd-api-bench/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) var ( diff --git a/tools/pd-backup/main.go b/tools/pd-backup/main.go index c0ee07afd6e..e5aaad62788 100644 --- a/tools/pd-backup/main.go +++ b/tools/pd-backup/main.go @@ -21,9 +21,10 @@ import ( "strings" "time" - "github.com/tikv/pd/tools/pd-backup/pdbackup" "go.etcd.io/etcd/client/pkg/v3/transport" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/tools/pd-backup/pdbackup" ) var ( diff --git a/tools/pd-backup/pdbackup/backup.go b/tools/pd-backup/pdbackup/backup.go index d02e651b879..e3fd3e682cb 100644 --- a/tools/pd-backup/pdbackup/backup.go +++ b/tools/pd-backup/pdbackup/backup.go @@ -25,11 +25,12 @@ import ( "path" "strconv" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/tools/pd-backup/pdbackup/backup_test.go b/tools/pd-backup/pdbackup/backup_test.go index d23f1878b85..3734fb6782b 100644 --- a/tools/pd-backup/pdbackup/backup_test.go +++ b/tools/pd-backup/pdbackup/backup_test.go @@ -16,15 +16,16 @@ import ( "time" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/goleak" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/goleak" ) var ( diff --git a/tools/pd-backup/tests/backup_test.go b/tools/pd-backup/tests/backup_test.go index 05d2b7b92ed..7c8c03d96e0 100644 --- a/tools/pd-backup/tests/backup_test.go +++ b/tools/pd-backup/tests/backup_test.go @@ -23,9 +23,10 @@ import ( "time" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/tests" "github.com/tikv/pd/tools/pd-backup/pdbackup" - clientv3 "go.etcd.io/etcd/client/v3" ) func TestBackup(t *testing.T) { diff --git a/tools/pd-ctl/main.go b/tools/pd-ctl/main.go index 0d052d95680..348310f9e7a 100644 --- a/tools/pd-ctl/main.go +++ b/tools/pd-ctl/main.go @@ -20,10 +20,12 @@ import ( "os/signal" "syscall" + "go.uber.org/zap/zapcore" + "github.com/pingcap/log" + "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/pdctl/command" - "go.uber.org/zap/zapcore" ) func main() { diff --git a/tools/pd-ctl/pdctl/command/config_command.go b/tools/pd-ctl/pdctl/command/config_command.go index 126d758fe05..2e9903db550 100644 --- a/tools/pd-ctl/pdctl/command/config_command.go +++ b/tools/pd-ctl/pdctl/command/config_command.go @@ -27,6 +27,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/reflectutil" diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index c7e0cfd691d..ce79277db1b 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -24,12 +24,14 @@ import ( "os" "strings" + "github.com/spf13/cobra" + "go.etcd.io/etcd/client/pkg/v3/transport" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/spf13/cobra" + pd "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/utils/apiutil" - "go.etcd.io/etcd/client/pkg/v3/transport" ) const ( diff --git a/tools/pd-ctl/pdctl/command/hot_command.go b/tools/pd-ctl/pdctl/command/hot_command.go index 77c0ee4d7de..4a24e0d6ce4 100644 --- a/tools/pd-ctl/pdctl/command/hot_command.go +++ b/tools/pd-ctl/pdctl/command/hot_command.go @@ -22,8 +22,10 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/storage" ) diff --git a/tools/pd-ctl/pdctl/command/keyspace_command.go b/tools/pd-ctl/pdctl/command/keyspace_command.go index 93a99abc39f..2ecee481df4 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_command.go @@ -22,6 +22,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/server/apiv2/handlers" ) diff --git a/tools/pd-ctl/pdctl/command/keyspace_group_command.go b/tools/pd-ctl/pdctl/command/keyspace_group_command.go index f315417e555..ec8ec120459 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_group_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_group_command.go @@ -23,6 +23,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/tools/pd-ctl/pdctl/command/label_command.go b/tools/pd-ctl/pdctl/command/label_command.go index 6d95465392f..d6360709cea 100644 --- a/tools/pd-ctl/pdctl/command/label_command.go +++ b/tools/pd-ctl/pdctl/command/label_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/tools/pd-ctl/pdctl/command/log_command.go b/tools/pd-ctl/pdctl/command/log_command.go index 56c4438a6c3..086299433dd 100644 --- a/tools/pd-ctl/pdctl/command/log_command.go +++ b/tools/pd-ctl/pdctl/command/log_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/utils/apiutil" ) diff --git a/tools/pd-ctl/pdctl/command/operator.go b/tools/pd-ctl/pdctl/command/operator.go index 4e7771580de..8e3fcc3ead6 100644 --- a/tools/pd-ctl/pdctl/command/operator.go +++ b/tools/pd-ctl/pdctl/command/operator.go @@ -19,8 +19,9 @@ import ( "net/http" "strconv" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/plugin_command.go b/tools/pd-ctl/pdctl/command/plugin_command.go index 9716d1d7755..de530904477 100644 --- a/tools/pd-ctl/pdctl/command/plugin_command.go +++ b/tools/pd-ctl/pdctl/command/plugin_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/schedule" ) diff --git a/tools/pd-ctl/pdctl/command/region_command.go b/tools/pd-ctl/pdctl/command/region_command.go index 3536b01a606..743d1e1f4c7 100644 --- a/tools/pd-ctl/pdctl/command/region_command.go +++ b/tools/pd-ctl/pdctl/command/region_command.go @@ -26,9 +26,10 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" "github.com/spf13/pflag" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/scheduler.go b/tools/pd-ctl/pdctl/command/scheduler.go index b8e05604b16..5dc05aff62f 100644 --- a/tools/pd-ctl/pdctl/command/scheduler.go +++ b/tools/pd-ctl/pdctl/command/scheduler.go @@ -24,8 +24,9 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/store_command.go b/tools/pd-ctl/pdctl/command/store_command.go index 0ad56e0ac72..fc398902508 100644 --- a/tools/pd-ctl/pdctl/command/store_command.go +++ b/tools/pd-ctl/pdctl/command/store_command.go @@ -23,11 +23,13 @@ import ( "strconv" "strings" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/spf13/cobra" - "github.com/tikv/pd/pkg/response" "golang.org/x/text/cases" "golang.org/x/text/language" + + "github.com/pingcap/kvproto/pkg/metapb" + + "github.com/tikv/pd/pkg/response" ) var ( diff --git a/tools/pd-ctl/pdctl/command/tso_command.go b/tools/pd-ctl/pdctl/command/tso_command.go index 689420854ee..7a4e8ec6d6e 100644 --- a/tools/pd-ctl/pdctl/command/tso_command.go +++ b/tools/pd-ctl/pdctl/command/tso_command.go @@ -18,6 +18,7 @@ import ( "strconv" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/utils/tsoutil" ) diff --git a/tools/pd-ctl/pdctl/ctl.go b/tools/pd-ctl/pdctl/ctl.go index 77f1601c8f5..e975bbf7a31 100644 --- a/tools/pd-ctl/pdctl/ctl.go +++ b/tools/pd-ctl/pdctl/ctl.go @@ -24,6 +24,7 @@ import ( shellwords "github.com/mattn/go-shellwords" "github.com/spf13/cobra" "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/tools/pd-ctl/pdctl/command" ) diff --git a/tools/pd-ctl/tests/cluster/cluster_test.go b/tools/pd-ctl/tests/cluster/cluster_test.go index 46681da9319..8f087bdb094 100644 --- a/tools/pd-ctl/tests/cluster/cluster_test.go +++ b/tools/pd-ctl/tests/cluster/cluster_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + clusterpkg "github.com/tikv/pd/server/cluster" pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" diff --git a/tools/pd-ctl/tests/completion/completion_test.go b/tools/pd-ctl/tests/completion/completion_test.go index cf4717a26aa..da043d2727d 100644 --- a/tools/pd-ctl/tests/completion/completion_test.go +++ b/tools/pd-ctl/tests/completion/completion_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" ) diff --git a/tools/pd-ctl/tests/config/config_test.go b/tools/pd-ctl/tests/config/config_test.go index 6dc7e9fb269..cf9e4163457 100644 --- a/tools/pd-ctl/tests/config/config_test.go +++ b/tools/pd-ctl/tests/config/config_test.go @@ -26,10 +26,12 @@ import ( "time" "github.com/coreos/go-semver/semver" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/tools/pd-ctl/tests/global_test.go b/tools/pd-ctl/tests/global_test.go index 766e357088e..00987f2a8a1 100644 --- a/tools/pd-ctl/tests/global_test.go +++ b/tools/pd-ctl/tests/global_test.go @@ -21,8 +21,10 @@ import ( "net/http" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/health/health_test.go b/tools/pd-ctl/tests/health/health_test.go index be9d5027988..c3b9e9979a4 100644 --- a/tools/pd-ctl/tests/health/health_test.go +++ b/tools/pd-ctl/tests/health/health_test.go @@ -24,6 +24,8 @@ import ( "testing" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/server/api" "github.com/tikv/pd/server/cluster" @@ -31,7 +33,6 @@ import ( pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" ) func TestHealth(t *testing.T) { diff --git a/tools/pd-ctl/tests/helper.go b/tools/pd-ctl/tests/helper.go index bdacae48c22..8e5963440ee 100644 --- a/tools/pd-ctl/tests/helper.go +++ b/tools/pd-ctl/tests/helper.go @@ -20,6 +20,7 @@ import ( "github.com/spf13/cobra" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/tools/pd-ctl/tests/hot/hot_test.go b/tools/pd-ctl/tests/hot/hot_test.go index e12b6a39a60..d01f861d861 100644 --- a/tools/pd-ctl/tests/hot/hot_test.go +++ b/tools/pd-ctl/tests/hot/hot_test.go @@ -22,10 +22,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/handler" "github.com/tikv/pd/pkg/statistics" diff --git a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go index 9c16b0751f6..fca00f2fd3c 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go @@ -22,8 +22,10 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/keyspace/keyspace_test.go b/tools/pd-ctl/tests/keyspace/keyspace_test.go index 4aa3be1d21c..23a1148cd66 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_test.go @@ -22,10 +22,12 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/label/label_test.go b/tools/pd-ctl/tests/label/label_test.go index 057c9d9d9bb..b4fcbfc0e2d 100644 --- a/tools/pd-ctl/tests/label/label_test.go +++ b/tools/pd-ctl/tests/label/label_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server/config" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/log/log_test.go b/tools/pd-ctl/tests/log/log_test.go index 274bc4ce7df..300b016f37b 100644 --- a/tools/pd-ctl/tests/log/log_test.go +++ b/tools/pd-ctl/tests/log/log_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" diff --git a/tools/pd-ctl/tests/member/member_test.go b/tools/pd-ctl/tests/member/member_test.go index dd3e465ae38..0f7d574945f 100644 --- a/tools/pd-ctl/tests/member/member_test.go +++ b/tools/pd-ctl/tests/member/member_test.go @@ -20,8 +20,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/operator/operator_test.go b/tools/pd-ctl/tests/operator/operator_test.go index 91e97c66dbd..9e8c374ea49 100644 --- a/tools/pd-ctl/tests/operator/operator_test.go +++ b/tools/pd-ctl/tests/operator/operator_test.go @@ -22,8 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" diff --git a/tools/pd-ctl/tests/region/region_test.go b/tools/pd-ctl/tests/region/region_test.go index 03a1c04ef19..06077a6184a 100644 --- a/tools/pd-ctl/tests/region/region_test.go +++ b/tools/pd-ctl/tests/region/region_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server/api" diff --git a/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go b/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go index 3da72244215..ceab3b07ab7 100644 --- a/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go +++ b/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/utils/typeutil" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/safepoint/safepoint_test.go b/tools/pd-ctl/tests/safepoint/safepoint_test.go index 9a9c54460bd..5a7ba8a2bc8 100644 --- a/tools/pd-ctl/tests/safepoint/safepoint_test.go +++ b/tools/pd-ctl/tests/safepoint/safepoint_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server/api" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/scheduler/scheduler_test.go b/tools/pd-ctl/tests/scheduler/scheduler_test.go index 3450e59d178..787bdaa4521 100644 --- a/tools/pd-ctl/tests/scheduler/scheduler_test.go +++ b/tools/pd-ctl/tests/scheduler/scheduler_test.go @@ -22,11 +22,13 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/spf13/cobra" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/slice" diff --git a/tools/pd-ctl/tests/store/store_test.go b/tools/pd-ctl/tests/store/store_test.go index 6f704a25e8c..79a83325f8b 100644 --- a/tools/pd-ctl/tests/store/store_test.go +++ b/tools/pd-ctl/tests/store/store_test.go @@ -24,8 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/response" @@ -35,7 +38,6 @@ import ( pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" ) func TestStoreLimitV2(t *testing.T) { diff --git a/tools/pd-ctl/tests/tso/tso_test.go b/tools/pd-ctl/tests/tso/tso_test.go index 63816c40e7a..30fdda96438 100644 --- a/tools/pd-ctl/tests/tso/tso_test.go +++ b/tools/pd-ctl/tests/tso/tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" ) diff --git a/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go b/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go index da89749c2f2..652645e1570 100644 --- a/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go +++ b/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" diff --git a/tools/pd-heartbeat-bench/config/config.go b/tools/pd-heartbeat-bench/config/config.go index dc5a2a6a047..41c14845074 100644 --- a/tools/pd-heartbeat-bench/config/config.go +++ b/tools/pd-heartbeat-bench/config/config.go @@ -4,11 +4,13 @@ import ( "sync/atomic" "github.com/BurntSushi/toml" + flag "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - flag "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/configutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-heartbeat-bench/main.go b/tools/pd-heartbeat-bench/main.go index 7150c81537a..60b1db6734d 100644 --- a/tools/pd-heartbeat-bench/main.go +++ b/tools/pd-heartbeat-bench/main.go @@ -34,11 +34,15 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/spf13/pflag" + "go.etcd.io/etcd/pkg/v3/report" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/spf13/pflag" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" @@ -48,8 +52,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/tools/pd-heartbeat-bench/config" "github.com/tikv/pd/tools/pd-heartbeat-bench/metrics" - "go.etcd.io/etcd/pkg/v3/report" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-heartbeat-bench/metrics/util.go b/tools/pd-heartbeat-bench/metrics/util.go index 9a61feee420..5e709630e27 100644 --- a/tools/pd-heartbeat-bench/metrics/util.go +++ b/tools/pd-heartbeat-bench/metrics/util.go @@ -22,12 +22,13 @@ import ( "strings" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/api" v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "go.etcd.io/etcd/pkg/v3/report" "go.uber.org/zap" + + "github.com/pingcap/log" ) var ( diff --git a/tools/pd-recover/main.go b/tools/pd-recover/main.go index 3423d06bb9f..bd5ebf76d74 100644 --- a/tools/pd-recover/main.go +++ b/tools/pd-recover/main.go @@ -24,12 +24,14 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( diff --git a/tools/pd-simulator/main.go b/tools/pd-simulator/main.go index 609ed3b06e0..59a58f0a00b 100644 --- a/tools/pd-simulator/main.go +++ b/tools/pd-simulator/main.go @@ -23,8 +23,11 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/pingcap/log" flag "github.com/spf13/pflag" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/logutil" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/tools/pd-simulator/simulator/cases" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-simulator/simulator/cases/balance_leader.go b/tools/pd-simulator/simulator/cases/balance_leader.go index a6790548dc1..f0ed73a4a9c 100644 --- a/tools/pd-simulator/simulator/cases/balance_leader.go +++ b/tools/pd-simulator/simulator/cases/balance_leader.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/balance_region.go b/tools/pd-simulator/simulator/cases/balance_region.go index d4ef7ad986f..8c7bbbe19d8 100644 --- a/tools/pd-simulator/simulator/cases/balance_region.go +++ b/tools/pd-simulator/simulator/cases/balance_region.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/cases.go b/tools/pd-simulator/simulator/cases/cases.go index 026e095342b..c1de896074f 100644 --- a/tools/pd-simulator/simulator/cases/cases.go +++ b/tools/pd-simulator/simulator/cases/cases.go @@ -16,6 +16,7 @@ package cases import ( "github.com/pingcap/kvproto/pkg/metapb" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go b/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go index 8e65feefea4..06e0582f5f8 100644 --- a/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go +++ b/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go @@ -19,12 +19,14 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newLabelNotMatch1(_ *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/diagnose_rule.go b/tools/pd-simulator/simulator/cases/diagnose_rule.go index 4e7031a3a01..d1a8350dbbc 100644 --- a/tools/pd-simulator/simulator/cases/diagnose_rule.go +++ b/tools/pd-simulator/simulator/cases/diagnose_rule.go @@ -18,14 +18,16 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newRule1(_ *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/hot_read.go b/tools/pd-simulator/simulator/cases/hot_read.go index 22ff70d9312..87637063385 100644 --- a/tools/pd-simulator/simulator/cases/hot_read.go +++ b/tools/pd-simulator/simulator/cases/hot_read.go @@ -18,7 +18,9 @@ import ( "fmt" "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/hot_write.go b/tools/pd-simulator/simulator/cases/hot_write.go index adb6eb0756a..43494324936 100644 --- a/tools/pd-simulator/simulator/cases/hot_write.go +++ b/tools/pd-simulator/simulator/cases/hot_write.go @@ -18,7 +18,9 @@ import ( "fmt" "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/import_data.go b/tools/pd-simulator/simulator/cases/import_data.go index 3d329081f9e..c06e9e7dd44 100644 --- a/tools/pd-simulator/simulator/cases/import_data.go +++ b/tools/pd-simulator/simulator/cases/import_data.go @@ -21,14 +21,16 @@ import ( "github.com/docker/go-units" "github.com/go-echarts/go-echarts/charts" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newImportData(config *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/makeup_down_replica.go b/tools/pd-simulator/simulator/cases/makeup_down_replica.go index ec664e91254..d5a5021b8cf 100644 --- a/tools/pd-simulator/simulator/cases/makeup_down_replica.go +++ b/tools/pd-simulator/simulator/cases/makeup_down_replica.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/region_merge.go b/tools/pd-simulator/simulator/cases/region_merge.go index 9a278c851bd..8f19f375c8e 100644 --- a/tools/pd-simulator/simulator/cases/region_merge.go +++ b/tools/pd-simulator/simulator/cases/region_merge.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/region_split.go b/tools/pd-simulator/simulator/cases/region_split.go index 8c1f3ac7759..8299cbdf136 100644 --- a/tools/pd-simulator/simulator/cases/region_split.go +++ b/tools/pd-simulator/simulator/cases/region_split.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/scale_tikv.go b/tools/pd-simulator/simulator/cases/scale_tikv.go index 9cfe8d9dcad..49dc70e64fb 100644 --- a/tools/pd-simulator/simulator/cases/scale_tikv.go +++ b/tools/pd-simulator/simulator/cases/scale_tikv.go @@ -16,6 +16,7 @@ package cases import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/stable_env.go b/tools/pd-simulator/simulator/cases/stable_env.go index 54a9f84341f..0a0c72698d8 100644 --- a/tools/pd-simulator/simulator/cases/stable_env.go +++ b/tools/pd-simulator/simulator/cases/stable_env.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index 4de2ea52f88..4c84d308af7 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -23,18 +23,20 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + pdHttp "github.com/tikv/pd/client/http" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) // Client is a PD (Placement Driver) client. diff --git a/tools/pd-simulator/simulator/config/config.go b/tools/pd-simulator/simulator/config/config.go index ece3b6fd91f..ead9e50e6af 100644 --- a/tools/pd-simulator/simulator/config/config.go +++ b/tools/pd-simulator/simulator/config/config.go @@ -21,6 +21,7 @@ import ( "github.com/BurntSushi/toml" "github.com/docker/go-units" + pdHttp "github.com/tikv/pd/client/http" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/utils/configutil" diff --git a/tools/pd-simulator/simulator/conn.go b/tools/pd-simulator/simulator/conn.go index b1000c0f17b..332f46eb573 100644 --- a/tools/pd-simulator/simulator/conn.go +++ b/tools/pd-simulator/simulator/conn.go @@ -16,6 +16,7 @@ package simulator import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/config" ) diff --git a/tools/pd-simulator/simulator/drive.go b/tools/pd-simulator/simulator/drive.go index c8c325cfca6..0d81a2af1ab 100644 --- a/tools/pd-simulator/simulator/drive.go +++ b/tools/pd-simulator/simulator/drive.go @@ -26,10 +26,14 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus/promhttp" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/prometheus/client_golang/prometheus/promhttp" + pdHttp "github.com/tikv/pd/client/http" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" @@ -38,8 +42,6 @@ import ( "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Driver promotes the cluster status change. diff --git a/tools/pd-simulator/simulator/event.go b/tools/pd-simulator/simulator/event.go index d22f35756ef..719e12f5fea 100644 --- a/tools/pd-simulator/simulator/event.go +++ b/tools/pd-simulator/simulator/event.go @@ -22,12 +22,14 @@ import ( "strconv" "sync" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) // Event affects the status of the cluster. diff --git a/tools/pd-simulator/simulator/node.go b/tools/pd-simulator/simulator/node.go index 59b0d393c47..edacf5c4129 100644 --- a/tools/pd-simulator/simulator/node.go +++ b/tools/pd-simulator/simulator/node.go @@ -22,8 +22,11 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/utils/syncutil" @@ -32,7 +35,6 @@ import ( sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-simulator/simulator/raft.go b/tools/pd-simulator/simulator/raft.go index 7f3bf78622f..13371cefdb8 100644 --- a/tools/pd-simulator/simulator/raft.go +++ b/tools/pd-simulator/simulator/raft.go @@ -17,14 +17,16 @@ package simulator import ( "context" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) // RaftEngine records all raft information. diff --git a/tools/pd-simulator/simulator/simutil/key.go b/tools/pd-simulator/simulator/simutil/key.go index a095f9d567b..ba1a19382ed 100644 --- a/tools/pd-simulator/simulator/simutil/key.go +++ b/tools/pd-simulator/simulator/simutil/key.go @@ -18,6 +18,7 @@ import ( "bytes" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/codec" ) diff --git a/tools/pd-simulator/simulator/simutil/key_test.go b/tools/pd-simulator/simulator/simutil/key_test.go index be07037501f..23cc0720b6b 100644 --- a/tools/pd-simulator/simulator/simutil/key_test.go +++ b/tools/pd-simulator/simulator/simutil/key_test.go @@ -17,8 +17,10 @@ package simutil import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" ) diff --git a/tools/pd-simulator/simulator/simutil/logger.go b/tools/pd-simulator/simulator/simutil/logger.go index e22124f00a5..08bd0e0461f 100644 --- a/tools/pd-simulator/simulator/simutil/logger.go +++ b/tools/pd-simulator/simulator/simutil/logger.go @@ -15,8 +15,9 @@ package simutil import ( - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) // Logger is the global logger used for simulator. diff --git a/tools/pd-simulator/simulator/task.go b/tools/pd-simulator/simulator/task.go index 0921838c70b..187f08fec68 100644 --- a/tools/pd-simulator/simulator/task.go +++ b/tools/pd-simulator/simulator/task.go @@ -21,13 +21,15 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/tools/pd-analysis/analysis" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index ccb68a1c5fb..cd710470db5 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -29,15 +29,17 @@ import ( "time" "github.com/influxdata/tdigest" + "github.com/prometheus/client_golang/prometheus/promhttp" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus/promhttp" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) const ( diff --git a/tools/pd-ut/alloc/check_env_linux.go b/tools/pd-ut/alloc/check_env_linux.go index 1a51f8075cf..c4d20bf6a65 100644 --- a/tools/pd-ut/alloc/check_env_linux.go +++ b/tools/pd-ut/alloc/check_env_linux.go @@ -18,8 +18,9 @@ package alloc import ( "github.com/cakturk/go-netstat/netstat" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) func environmentCheck(addr string) bool { diff --git a/tools/pd-ut/alloc/server.go b/tools/pd-ut/alloc/server.go index ffa3bce0aa5..6e7aeeb6307 100644 --- a/tools/pd-ut/alloc/server.go +++ b/tools/pd-ut/alloc/server.go @@ -23,9 +23,11 @@ import ( "time" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/tempurl" - "go.uber.org/zap" ) var statusAddress = flag.String("status-addr", "0.0.0.0:0", "status address") diff --git a/tools/pd-ut/alloc/tempurl.go b/tools/pd-ut/alloc/tempurl.go index 2131699133a..c7492064ac2 100644 --- a/tools/pd-ut/alloc/tempurl.go +++ b/tools/pd-ut/alloc/tempurl.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" ) diff --git a/tools/pd-ut/ut.go b/tools/pd-ut/ut.go index 7bb0cf17e9f..8efacf5933a 100644 --- a/tools/pd-ut/ut.go +++ b/tools/pd-ut/ut.go @@ -33,9 +33,10 @@ import ( "sync" "time" - "github.com/tikv/pd/tools/pd-ut/alloc" "go.uber.org/zap" + "github.com/tikv/pd/tools/pd-ut/alloc" + // Set the correct value when it runs inside docker. _ "go.uber.org/automaxprocs" ) diff --git a/tools/regions-dump/main.go b/tools/regions-dump/main.go index 5ae4241cdc0..58d19ce72f2 100644 --- a/tools/regions-dump/main.go +++ b/tools/regions-dump/main.go @@ -25,12 +25,14 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/etcdutil" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( diff --git a/tools/stores-dump/main.go b/tools/stores-dump/main.go index 0409244772f..0815c6e7113 100644 --- a/tools/stores-dump/main.go +++ b/tools/stores-dump/main.go @@ -25,11 +25,13 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/etcdutil" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( From 88a761e527e54f70916b94ce2326bc7dee56a83d Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Tue, 17 Dec 2024 19:16:00 +0800 Subject: [PATCH 07/10] dep: update github.com/aws/aws-sdk-go-v2/service/kms to v1.26.3 (#8923) ref tikv/pd#4399 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 11 ++--------- tests/integrations/go.mod | 2 +- tests/integrations/go.sum | 11 ++--------- tools/go.mod | 2 +- tools/go.sum | 11 ++--------- 6 files changed, 9 insertions(+), 30 deletions(-) diff --git a/go.mod b/go.mod index 8fed2fc25fc..381d3579785 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/BurntSushi/toml v0.3.1 github.com/aws/aws-sdk-go-v2/config v1.25.12 github.com/aws/aws-sdk-go-v2/credentials v1.16.10 - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 github.com/axw/gocov v1.0.0 github.com/brianvoe/gofakeit/v6 v6.26.3 diff --git a/go.sum b/go.sum index 172a53a0909..0338557d0ab 100644 --- a/go.sum +++ b/go.sum @@ -28,7 +28,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -37,10 +36,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -49,15 +46,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/axw/gocov v1.0.0 h1:YsqYR66hUmilVr23tu8USgnJIJvnwh3n7j5zRn7x4LU= @@ -233,7 +229,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -292,8 +287,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 9d3bcea65f6..97d637697f0 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -49,7 +49,7 @@ require ( github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 // indirect diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index 2f7fd660b4f..a14e9a4ab05 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -28,7 +28,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -37,10 +36,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -49,15 +46,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -227,7 +223,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -283,8 +278,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= diff --git a/tools/go.mod b/tools/go.mod index 79bb07c91c6..7a1c7b72f36 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -60,7 +60,7 @@ require ( github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 // indirect diff --git a/tools/go.sum b/tools/go.sum index 184ab965dbb..1bfe598d503 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -26,7 +26,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -35,10 +34,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -47,15 +44,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -226,7 +222,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -285,8 +280,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= From 2d970a619a8917c35d306f401326141481c133e0 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 10:56:12 +0800 Subject: [PATCH 08/10] *: closed immediately if error rate is changed to 0 (#8887) ref tikv/pd#8678 Signed-off-by: Ryan Leung --- client/circuitbreaker/circuit_breaker.go | 27 ++- client/circuitbreaker/circuit_breaker_test.go | 18 +- client/client.go | 5 + client/metrics/metrics.go | 2 +- tests/integrations/client/client_test.go | 172 ++++++++++++++++++ 5 files changed, 205 insertions(+), 19 deletions(-) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/circuitbreaker/circuit_breaker.go index 26d1b642ea4..2c65f4f1965 100644 --- a/client/circuitbreaker/circuit_breaker.go +++ b/client/circuitbreaker/circuit_breaker.go @@ -123,6 +123,7 @@ func (cb *CircuitBreaker[T]) ChangeSettings(apply func(config *Settings)) { defer cb.mutex.Unlock() apply(cb.config) + log.Info("circuit breaker settings changed", zap.Any("config", cb.config)) } // Execute calls the given function if the CircuitBreaker is closed and returns the result of execution. @@ -238,10 +239,10 @@ func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { observedErrorRatePct := s.failureCount * 100 / total if total >= uint32(s.cb.config.ErrorRateWindow.Seconds())*s.cb.config.MinQPSForOpen && observedErrorRatePct >= s.cb.config.ErrorRateThresholdPct { // the error threshold is breached, let's move to open state and start failing all requests - log.Error("Circuit breaker tripped. Starting to fail all requests", + log.Error("circuit breaker tripped and starting to fail all requests", zap.String("name", cb.name), - zap.Uint32("observedErrorRatePct", observedErrorRatePct), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Uint32("observed-err-rate-pct", observedErrorRatePct), + zap.Any("config", cb.config)) return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen } } @@ -253,29 +254,37 @@ func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { // continue in closed state till ErrorRateWindow is over return s, nil case StateOpen: + if s.cb.config.ErrorRateThresholdPct == 0 { + return cb.newState(now, StateClosed), nil + } + if now.After(s.end) { // CoolDownInterval is over, it is time to transition to half-open state - log.Info("Circuit breaker cooldown period is over. Transitioning to half-open state to test the service", + log.Info("circuit breaker cooldown period is over. Transitioning to half-open state to test the service", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateHalfOpen), nil } else { // continue in the open state till CoolDownInterval is over return s, errs.ErrCircuitBreakerOpen } case StateHalfOpen: + if s.cb.config.ErrorRateThresholdPct == 0 { + return cb.newState(now, StateClosed), nil + } + // do we need some expire time here in case of one of pending requests is stuck forever? if s.failureCount > 0 { // there were some failures during half-open state, let's go back to open state to wait a bit longer - log.Error("Circuit breaker goes from half-open to open again as errors persist and continue to fail all requests", + log.Error("circuit breaker goes from half-open to open again as errors persist and continue to fail all requests", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen } else if s.successCount == s.cb.config.HalfOpenSuccessCount { // all probe requests are succeeded, we can move to closed state and allow all requests - log.Info("Circuit breaker is closed. Start allowing all requests", + log.Info("circuit breaker is closed and start allowing all requests", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateClosed), nil } else if s.pendingCount < s.cb.config.HalfOpenSuccessCount { // allow more probe requests and continue in half-open state diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/circuitbreaker/circuit_breaker_test.go index 6a726028cd8..07a3c06f86e 100644 --- a/client/circuitbreaker/circuit_breaker_test.go +++ b/client/circuitbreaker/circuit_breaker_test.go @@ -38,7 +38,7 @@ var settings = Settings{ var minCountToOpen = int(settings.MinQPSForOpen * uint32(settings.ErrorRateWindow.Seconds())) -func TestCircuitBreaker_Execute_Wrapper_Return_Values(t *testing.T) { +func TestCircuitBreakerExecuteWrapperReturnValues(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) originalError := errors.New("circuit breaker is open") @@ -57,7 +57,7 @@ func TestCircuitBreaker_Execute_Wrapper_Return_Values(t *testing.T) { re.Equal(42, result) } -func TestCircuitBreaker_OpenState(t *testing.T) { +func TestCircuitBreakerOpenState(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) driveQPS(cb, minCountToOpen, Yes, re) @@ -68,7 +68,7 @@ func TestCircuitBreaker_OpenState(t *testing.T) { re.Equal(StateOpen, cb.state.stateType) } -func TestCircuitBreaker_CloseState_Not_Enough_QPS(t *testing.T) { +func TestCircuitBreakerCloseStateNotEnoughQPS(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -78,7 +78,7 @@ func TestCircuitBreaker_CloseState_Not_Enough_QPS(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_CloseState_Not_Enough_Error_Rate(t *testing.T) { +func TestCircuitBreakerCloseStateNotEnoughErrorRate(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -89,7 +89,7 @@ func TestCircuitBreaker_CloseState_Not_Enough_Error_Rate(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_Half_Open_To_Closed(t *testing.T) { +func TestCircuitBreakerHalfOpenToClosed(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -107,7 +107,7 @@ func TestCircuitBreaker_Half_Open_To_Closed(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_Half_Open_To_Open(t *testing.T) { +func TestCircuitBreakerHalfOpenToOpen(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -130,7 +130,7 @@ func TestCircuitBreaker_Half_Open_To_Open(t *testing.T) { // in half open state, circuit breaker will allow only HalfOpenSuccessCount pending and should fast fail all other request till HalfOpenSuccessCount requests is completed // this test moves circuit breaker to the half open state and verifies that requests above HalfOpenSuccessCount are failing -func TestCircuitBreaker_Half_Open_Fail_Over_Pending_Count(t *testing.T) { +func TestCircuitBreakerHalfOpenFailOverPendingCount(t *testing.T) { re := require.New(t) cb := newCircuitBreakerMovedToHalfOpenState(re) @@ -176,7 +176,7 @@ func TestCircuitBreaker_Half_Open_Fail_Over_Pending_Count(t *testing.T) { re.Equal(uint32(1), cb.state.successCount) } -func TestCircuitBreaker_Count_Only_Requests_In_Same_Window(t *testing.T) { +func TestCircuitBreakerCountOnlyRequestsInSameWindow(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -211,7 +211,7 @@ func TestCircuitBreaker_Count_Only_Requests_In_Same_Window(t *testing.T) { re.Equal(uint32(1), cb.state.successCount) } -func TestCircuitBreaker_ChangeSettings(t *testing.T) { +func TestCircuitBreakerChangeSettings(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", AlwaysClosedSettings) diff --git a/client/client.go b/client/client.go index 31bc72c0a77..0e48707cd8d 100644 --- a/client/client.go +++ b/client/client.go @@ -25,6 +25,8 @@ import ( "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -661,6 +663,9 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio } resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { region, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) + failpoint.Inject("triggerCircuitBreaker", func() { + err = status.Error(codes.ResourceExhausted, "resource exhausted") + }) return region, isOverloaded(err), err }) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { diff --git a/client/metrics/metrics.go b/client/metrics/metrics.go index 3a3199c74a6..67268c826f5 100644 --- a/client/metrics/metrics.go +++ b/client/metrics/metrics.go @@ -152,7 +152,7 @@ func initMetrics(constLabels prometheus.Labels) { Namespace: "pd_client", Subsystem: "request", Name: "circuit_breaker_count", - Help: "Circuit Breaker counters", + Help: "Circuit breaker counters", ConstLabels: constLabels, }, []string{"name", "success"}) } diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 9ba63f0c83f..c0b762d0983 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -20,6 +20,7 @@ import ( "encoding/json" "fmt" "math" + "os" "path" "reflect" "sort" @@ -44,6 +45,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" pd "github.com/tikv/pd/client" + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -2049,3 +2051,173 @@ func needRetry(err error) bool { } return st.Code() == codes.ResourceExhausted } + +func TestCircuitBreaker(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 1, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + // wait cooldown + time.Sleep(time.Second) + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } +} + +func TestCircuitBreakerOpenAndChangeSettings(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 1, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + *config = cb.AlwaysClosedSettings + }) + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "ResourceExhausted") + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) +} + +func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 20, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + fname := testutil.InitTempFileLogger("info") + defer os.RemoveAll(fname) + // wait for cooldown + time.Sleep(time.Second) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + // trigger circuit breaker state to be half open + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + testutil.Eventually(re, func() bool { + b, _ := os.ReadFile(fname) + l := string(b) + // We need to check the log to see if the circuit breaker is half open + return strings.Contains(l, "Transitioning to half-open state to test the service") + }) + + // The state is half open + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + // change settings to always closed + cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + *config = cb.AlwaysClosedSettings + }) + + // It won't be changed to open state. + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.NotContains(err.Error(), "circuit breaker is open") + } + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) +} From 221877d424db768cca77bee40351cf219f75eb01 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 16:50:51 +0800 Subject: [PATCH 09/10] *: unify the error definition (#8902) ref tikv/pd#8922 Signed-off-by: Ryan Leung --- errors.toml | 155 +++++++++++++++++++++++ pkg/cgroup/cgroup_cpu.go | 6 +- pkg/cgroup/cgroup_cpu_test.go | 4 +- pkg/errs/errno.go | 75 ++++++++++- pkg/gc/safepoint_v2.go | 3 +- pkg/keyspace/keyspace.go | 43 ++++--- pkg/keyspace/tso_keyspace_group.go | 99 ++++++++------- pkg/keyspace/tso_keyspace_group_test.go | 39 +++--- pkg/keyspace/util.go | 57 --------- pkg/ratelimit/runner.go | 10 +- pkg/schedule/checker/rule_checker.go | 22 +--- pkg/schedule/scatter/region_scatterer.go | 10 +- server/apiv2/handlers/keyspace.go | 2 +- server/keyspace_service.go | 6 +- 14 files changed, 346 insertions(+), 185 deletions(-) diff --git a/errors.toml b/errors.toml index 9bfd4a79190..26b15c38077 100644 --- a/errors.toml +++ b/errors.toml @@ -61,6 +61,11 @@ error = ''' unsupported metrics type %v ''' +["PD:cgroup:ErrNoCPUControllerDetected"] +error = ''' +no cpu controller detected +''' + ["PD:checker:ErrCheckerMergeAgain"] error = ''' region will be merged again, %s @@ -71,6 +76,36 @@ error = ''' checker not found ''' +["PD:checker:ErrNoNewLeader"] +error = ''' +no new leader +''' + +["PD:checker:ErrNoStoreToAdd"] +error = ''' +no store to add peer +''' + +["PD:checker:ErrNoStoreToReplace"] +error = ''' +no store to replace peer +''' + +["PD:checker:ErrPeerCannotBeLeader"] +error = ''' +peer cannot be leader +''' + +["PD:checker:ErrPeerCannotBeWitness"] +error = ''' +peer cannot be witness +''' + +["PD:checker:ErrRegionNoLeader"] +error = ''' +region no leader +''' + ["PD:client:ErrClientCreateTSOStream"] error = ''' create TSO stream failed, %s @@ -491,6 +526,116 @@ error = ''' failed to unmarshal json ''' +["PD:keyspace:ErrExceedMaxEtcdTxnOps"] +error = ''' +exceed max etcd txn operations +''' + +["PD:keyspace:ErrIllegalOperation"] +error = ''' +unknown operation +''' + +["PD:keyspace:ErrKeyspaceExists"] +error = ''' +keyspace already exists +''' + +["PD:keyspace:ErrKeyspaceGroupExists"] +error = ''' +keyspace group already exists +''' + +["PD:keyspace:ErrKeyspaceGroupInMerging"] +error = ''' +keyspace group %v is in merging state +''' + +["PD:keyspace:ErrKeyspaceGroupInSplit"] +error = ''' +keyspace group %v is in split state +''' + +["PD:keyspace:ErrKeyspaceGroupNotEnoughReplicas"] +error = ''' +not enough replicas in the keyspace group +''' + +["PD:keyspace:ErrKeyspaceGroupNotExists"] +error = ''' +keyspace group %v does not exist +''' + +["PD:keyspace:ErrKeyspaceGroupNotInMerging"] +error = ''' +keyspace group %v is not in merging state +''' + +["PD:keyspace:ErrKeyspaceGroupNotInSplit"] +error = ''' +keyspace group %v is not in split state +''' + +["PD:keyspace:ErrKeyspaceGroupPrimaryNotFound"] +error = ''' +primary of keyspace group does not exist +''' + +["PD:keyspace:ErrKeyspaceGroupWithEmptyKeyspace"] +error = ''' +keyspace group with empty keyspace +''' + +["PD:keyspace:ErrKeyspaceNotFound"] +error = ''' +keyspace does not exist +''' + +["PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup"] +error = ''' +keyspace is not in any keyspace group +''' + +["PD:keyspace:ErrKeyspaceNotInKeyspaceGroup"] +error = ''' +keyspace is not in this keyspace group +''' + +["PD:keyspace:ErrModifyDefaultKeyspace"] +error = ''' +cannot modify default keyspace's state +''' + +["PD:keyspace:ErrModifyDefaultKeyspaceGroup"] +error = ''' +default keyspace group cannot be modified +''' + +["PD:keyspace:ErrNoAvailableNode"] +error = ''' +no available node +''' + +["PD:keyspace:ErrNodeNotInKeyspaceGroup"] +error = ''' +the tso node is not in this keyspace group +''' + +["PD:keyspace:ErrRegionSplitFailed"] +error = ''' +region split failed +''' + +["PD:keyspace:ErrRegionSplitTimeout"] +error = ''' +region split timeout +''' + +["PD:keyspace:ErrUnsupportedOperationInKeyspace"] +error = ''' +it's a unsupported operation +''' + ["PD:leveldb:ErrLevelDBClose"] error = ''' close leveldb error @@ -646,6 +791,11 @@ error = ''' failed to unmarshal proto ''' +["PD:ratelimit:ErrMaxWaitingTasksExceeded"] +error = ''' +max waiting tasks exceeded +''' + ["PD:region:ErrRegionAbnormalPeer"] error = ''' region %v has abnormal peer @@ -691,6 +841,11 @@ error = ''' invalid group settings, please check the group name, priority and the number of resources ''' +["PD:scatter:ErrEmptyRegion"] +error = ''' +empty region +''' + ["PD:schedule:ErrCreateOperator"] error = ''' unable to create operator, %s diff --git a/pkg/cgroup/cgroup_cpu.go b/pkg/cgroup/cgroup_cpu.go index 67eace5363c..e696e36fad5 100644 --- a/pkg/cgroup/cgroup_cpu.go +++ b/pkg/cgroup/cgroup_cpu.go @@ -19,9 +19,9 @@ import ( "path/filepath" "github.com/pingcap/errors" -) -var errNoCPUControllerDetected = errors.New("no cpu controller detected") + "github.com/tikv/pd/pkg/errs" +) // Helper function for getCgroupCPU. Root is always "/", except in tests. func getCgroupCPUHelper(root string) (CPUUsage, error) { @@ -32,7 +32,7 @@ func getCgroupCPUHelper(root string) (CPUUsage, error) { // No CPU controller detected if path == "" { - return CPUUsage{}, errNoCPUControllerDetected + return CPUUsage{}, errs.ErrNoCPUControllerDetected } mount, ver, err := getCgroupDetails(filepath.Join(root, procPathMountInfo), path, "cpu,cpuacct") diff --git a/pkg/cgroup/cgroup_cpu_test.go b/pkg/cgroup/cgroup_cpu_test.go index 265291163c3..441c2192e79 100644 --- a/pkg/cgroup/cgroup_cpu_test.go +++ b/pkg/cgroup/cgroup_cpu_test.go @@ -26,6 +26,8 @@ import ( "testing" "github.com/stretchr/testify/require" + + "github.com/tikv/pd/pkg/errs" ) func checkKernelVersionNewerThan(re *require.Assertions, t *testing.T, major, minor int) bool { @@ -82,7 +84,7 @@ func TestGetCgroupCPU(t *testing.T) { }() } cpu, err := GetCgroupCPU() - if err == errNoCPUControllerDetected { + if err == errs.ErrNoCPUControllerDetected { // for more information, please refer https://github.com/pingcap/tidb/pull/41347 if checkKernelVersionNewerThan(re, t, 4, 7) { re.NoError(err, "linux version > v4.7 and err still happens") diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index e5c23cffde2..fd9ff3c95ff 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -140,8 +140,69 @@ var ( // checker errors var ( - ErrCheckerNotFound = errors.Normalize("checker not found", errors.RFCCodeText("PD:checker:ErrCheckerNotFound")) - ErrCheckerMergeAgain = errors.Normalize("region will be merged again, %s", errors.RFCCodeText("PD:checker:ErrCheckerMergeAgain")) + ErrCheckerNotFound = errors.Normalize("checker not found", errors.RFCCodeText("PD:checker:ErrCheckerNotFound")) + ErrCheckerMergeAgain = errors.Normalize("region will be merged again, %s", errors.RFCCodeText("PD:checker:ErrCheckerMergeAgain")) + ErrNoStoreToAdd = errors.Normalize("no store to add peer", errors.RFCCodeText("PD:checker:ErrNoStoreToAdd")) + ErrNoStoreToReplace = errors.Normalize("no store to replace peer", errors.RFCCodeText("PD:checker:ErrNoStoreToReplace")) + ErrPeerCannotBeLeader = errors.Normalize("peer cannot be leader", errors.RFCCodeText("PD:checker:ErrPeerCannotBeLeader")) + ErrPeerCannotBeWitness = errors.Normalize("peer cannot be witness", errors.RFCCodeText("PD:checker:ErrPeerCannotBeWitness")) + ErrNoNewLeader = errors.Normalize("no new leader", errors.RFCCodeText("PD:checker:ErrNoNewLeader")) + ErrRegionNoLeader = errors.Normalize("region no leader", errors.RFCCodeText("PD:checker:ErrRegionNoLeader")) +) + +// scatter errors +var ( + ErrEmptyRegion = errors.Normalize("empty region", errors.RFCCodeText("PD:scatter:ErrEmptyRegion")) +) + +// keyspace errors +var ( + // ErrKeyspaceNotFound is used to indicate target keyspace does not exist. + ErrKeyspaceNotFound = errors.Normalize("keyspace does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotFound")) + // ErrRegionSplitTimeout indices to split region timeout + ErrRegionSplitTimeout = errors.Normalize("region split timeout", errors.RFCCodeText("PD:keyspace:ErrRegionSplitTimeout")) + // ErrRegionSplitFailed indices to split region failed + ErrRegionSplitFailed = errors.Normalize("region split failed", errors.RFCCodeText("PD:keyspace:ErrRegionSplitFailed")) + // ErrKeyspaceExists indicates target keyspace already exists. + // It's used when creating a new keyspace. + ErrKeyspaceExists = errors.Normalize("keyspace already exists", errors.RFCCodeText("PD:keyspace:ErrKeyspaceExists")) + // ErrKeyspaceGroupExists indicates target keyspace group already exists. + ErrKeyspaceGroupExists = errors.Normalize("keyspace group already exists", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupExists")) + // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. + ErrKeyspaceNotInKeyspaceGroup = errors.Normalize("keyspace is not in this keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInKeyspaceGroup")) + // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. + ErrKeyspaceNotInAnyKeyspaceGroup = errors.Normalize("keyspace is not in any keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup")) + // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. + ErrNodeNotInKeyspaceGroup = errors.Normalize("the tso node is not in this keyspace group", errors.RFCCodeText("PD:keyspace:ErrNodeNotInKeyspaceGroup")) + // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. + ErrKeyspaceGroupNotEnoughReplicas = errors.Normalize("not enough replicas in the keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotEnoughReplicas")) + // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. + ErrKeyspaceGroupWithEmptyKeyspace = errors.Normalize("keyspace group with empty keyspace", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupWithEmptyKeyspace")) + // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. + ErrModifyDefaultKeyspaceGroup = errors.Normalize("default keyspace group cannot be modified", errors.RFCCodeText("PD:keyspace:ErrModifyDefaultKeyspaceGroup")) + // ErrNoAvailableNode is used to indicate no available node in the keyspace group. + ErrNoAvailableNode = errors.Normalize("no available node", errors.RFCCodeText("PD:keyspace:ErrNoAvailableNode")) + // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. + ErrExceedMaxEtcdTxnOps = errors.Normalize("exceed max etcd txn operations", errors.RFCCodeText("PD:keyspace:ErrExceedMaxEtcdTxnOps")) + // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. + ErrModifyDefaultKeyspace = errors.Normalize("cannot modify default keyspace's state", errors.RFCCodeText("PD:keyspace:ErrModifyDefaultKeyspace")) + // ErrIllegalOperation is used to indicate this is an illegal operation. + ErrIllegalOperation = errors.Normalize("unknown operation", errors.RFCCodeText("PD:keyspace:ErrIllegalOperation")) + // ErrUnsupportedOperationInKeyspace is used to indicate this is an unsupported operation. + ErrUnsupportedOperationInKeyspace = errors.Normalize("it's a unsupported operation", errors.RFCCodeText("PD:keyspace:ErrUnsupportedOperationInKeyspace")) + // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. + ErrKeyspaceGroupPrimaryNotFound = errors.Normalize("primary of keyspace group does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupPrimaryNotFound")) + // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. + ErrKeyspaceGroupNotExists = errors.Normalize("keyspace group %v does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotExists")) + // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. + ErrKeyspaceGroupInSplit = errors.Normalize("keyspace group %v is in split state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupInSplit")) + // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. + ErrKeyspaceGroupNotInSplit = errors.Normalize("keyspace group %v is not in split state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotInSplit")) + // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. + ErrKeyspaceGroupInMerging = errors.Normalize("keyspace group %v is in merging state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupInMerging")) + // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. + ErrKeyspaceGroupNotInMerging = errors.Normalize("keyspace group %v is not in merging state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotInMerging")) + // errKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. ) // diagnostic errors @@ -229,6 +290,16 @@ var ( ErrBytesToUint64 = errors.Normalize("invalid data, must 8 bytes, but %d", errors.RFCCodeText("PD:typeutil:ErrBytesToUint64")) ) +// cgroup errors +var ( + ErrNoCPUControllerDetected = errors.Normalize("no cpu controller detected", errors.RFCCodeText("PD:cgroup:ErrNoCPUControllerDetected")) +) + +// ratelimit errors +var ( + ErrMaxWaitingTasksExceeded = errors.Normalize("max waiting tasks exceeded", errors.RFCCodeText("PD:ratelimit:ErrMaxWaitingTasksExceeded")) +) + // The third-party project error. // url errors var ( diff --git a/pkg/gc/safepoint_v2.go b/pkg/gc/safepoint_v2.go index 449be8f3d59..acdd4e6eef8 100644 --- a/pkg/gc/safepoint_v2.go +++ b/pkg/gc/safepoint_v2.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" @@ -101,7 +102,7 @@ func (manager *SafePointV2Manager) checkKeyspace(keyspaceID uint32, updateReques } // If a keyspace does not exist, then loading its gc safe point is prohibited. if meta == nil { - return keyspace.ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // If keyspace's state does not permit updating safe point, we return error. if updateRequest && !slice.Contains(allowUpdateSafePoint, meta.GetState()) { diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 4a50f36169f..c9e390df47a 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/core" @@ -153,7 +154,7 @@ func (manager *Manager) Bootstrap() error { err = manager.saveNewKeyspace(defaultKeyspaceMeta) // It's possible that default keyspace already exists in the storage (e.g. PD restart/recover), // so we ignore the keyspaceExists error. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], defaultKeyspaceMeta.GetId(), opAdd); err != nil { @@ -174,7 +175,7 @@ func (manager *Manager) Bootstrap() error { } keyspace, err := manager.CreateKeyspace(req) // Ignore the keyspaceExists error for the same reason as saving default keyspace. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], keyspace.GetId(), opAdd); err != nil { @@ -288,7 +289,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if nameExists { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } err = manager.store.SaveKeyspaceID(txn, keyspace.Id, keyspace.Name) if err != nil { @@ -301,7 +302,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if loadedMeta != nil { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } return manager.store.SaveKeyspaceMeta(txn, keyspace) }) @@ -343,7 +344,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er ranges := keyspaceRule.Data.([]*labeler.KeyRangeRule) if len(ranges) < 2 { log.Warn("[keyspace] failed to split keyspace region with insufficient range", logutil.ZapRedactString("label-rule", keyspaceRule.String())) - return ErrRegionSplitFailed + return errs.ErrRegionSplitFailed } rawLeftBound, rawRightBound := ranges[0].StartKey, ranges[0].EndKey txnLeftBound, txnRightBound := ranges[1].StartKey, ranges[1].EndKey @@ -381,7 +382,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er zap.Uint32("keyspace-id", id), zap.Error(err), ) - err = ErrRegionSplitTimeout + err = errs.ErrRegionSplitTimeout return } log.Info("[keyspace] wait region split successfully", zap.Uint32("keyspace-id", id)) @@ -407,14 +408,14 @@ func (manager *Manager) LoadKeyspace(name string) (*keyspacepb.KeyspaceMeta, err return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } meta, err = manager.store.LoadKeyspaceMeta(txn, id) if err != nil { return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -434,7 +435,7 @@ func (manager *Manager) LoadKeyspaceByID(spaceID uint32) (*keyspacepb.KeyspaceMe return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -474,7 +475,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -484,7 +485,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Only keyspace with state listed in allowChangeConfig are allowed to change their config. if !slice.Contains(allowChangeConfig, meta.GetState()) { @@ -505,7 +506,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) case OpDel: delete(meta.Config, mutation.Key) default: - return errIllegalOperation + return errs.ErrIllegalOperation } } newConfig := meta.GetConfig() @@ -553,9 +554,9 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key // Changing the state of default keyspace is not allowed. if name == constant.DefaultKeyspaceName { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), + errs.ZapError(errs.ErrModifyDefaultKeyspace), ) - return nil, ErrModifyDefaultKeyspace + return nil, errs.ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta err := manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { @@ -565,7 +566,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -575,7 +576,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -605,9 +606,9 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K // Changing the state of default keyspace is not allowed. if id == constant.DefaultKeyspaceID { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), + errs.ZapError(errs.ErrModifyDefaultKeyspace), ) - return nil, ErrModifyDefaultKeyspace + return nil, errs.ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta var err error @@ -620,7 +621,7 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -740,10 +741,10 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID return errors.Errorf("default keyspace group %d not found", constant.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsSplitting() { - return ErrKeyspaceGroupInSplit(constant.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(constant.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsMerging() { - return ErrKeyspaceGroupInMerging(constant.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(constant.DefaultKeyspaceGroupID) } keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, etcdutil.MaxEtcdTxnOps) if err != nil { diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index dc414cfad3a..a6068ffc2bb 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/pkg/balancer" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" @@ -129,7 +130,7 @@ func (m *GroupManager) Bootstrap(ctx context.Context) error { // Ignore the error if default keyspace group already exists in the storage (e.g. PD restart/recover). err := m.saveKeyspaceGroups([]*endpoint.KeyspaceGroup{defaultKeyspaceGroup}, false) - if err != nil && err != ErrKeyspaceGroupExists { + if err != nil && err != errs.ErrKeyspaceGroupExists { return err } @@ -322,7 +323,7 @@ func (m *GroupManager) DeleteKeyspaceGroupByID(id uint32) (*endpoint.KeyspaceGro return nil } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } return m.store.DeleteKeyspaceGroup(txn, id) }); err != nil { @@ -348,13 +349,13 @@ func (m *GroupManager) saveKeyspaceGroups(keyspaceGroups []*endpoint.KeyspaceGro return err } if oldKG != nil && !overwrite { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } if oldKG.IsSplitting() && overwrite { - return ErrKeyspaceGroupInSplit(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(keyspaceGroup.ID) } if oldKG.IsMerging() && overwrite { - return ErrKeyspaceGroupInMerging(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(keyspaceGroup.ID) } newKG := &endpoint.KeyspaceGroup{ ID: keyspaceGroup.ID, @@ -410,7 +411,7 @@ func (m *GroupManager) GetGroupByKeyspaceID(id uint32) (uint32, error) { } } } - return 0, ErrKeyspaceNotInAnyKeyspaceGroup + return 0, errs.ErrKeyspaceNotInAnyKeyspaceGroup } var failpointOnce sync.Once @@ -440,13 +441,13 @@ func (m *GroupManager) UpdateKeyspaceForGroup(userKind endpoint.UserKind, groupI func (m *GroupManager) updateKeyspaceForGroupLocked(userKind endpoint.UserKind, groupID uint64, keyspaceID uint32, mutation int) error { kg := m.groups[userKind].Get(uint32(groupID)) if kg == nil { - return ErrKeyspaceGroupNotExists(uint32(groupID)) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(uint32(groupID)) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(groupID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(groupID)) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(groupID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(groupID)) } changed := false @@ -500,13 +501,13 @@ func (m *GroupManager) UpdateKeyspaceGroup(oldGroupID, newGroupID string, oldUse return errors.Errorf("keyspace group %s not found in %s group", newGroupID, newUserKind) } if oldKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(oldID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(oldID)) } else if newKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(newID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(newID)) } else if oldKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(oldID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(oldID)) } else if newKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(newID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(newID)) } var updateOld, updateNew bool @@ -552,15 +553,15 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitSourceID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitSourceID) } // A keyspace group can not take part in multiple split processes. if splitSourceKg.IsSplitting() { - return ErrKeyspaceGroupInSplit(splitSourceID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(splitSourceID) } // A keyspace group can not be split when it is in merging. if splitSourceKg.IsMerging() { - return ErrKeyspaceGroupInMerging(splitSourceID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(splitSourceID) } // Build the new keyspace groups for split source and target. var startKeyspaceID, endKeyspaceID uint32 @@ -574,7 +575,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( } // Check if the source keyspace group has enough replicas. if len(splitSourceKg.Members) < constant.DefaultKeyspaceGroupReplicaCount { - return ErrKeyspaceGroupNotEnoughReplicas + return errs.ErrKeyspaceGroupNotEnoughReplicas } // Check if the new keyspace group already exists. splitTargetKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetID) @@ -582,7 +583,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitTargetKg != nil { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } // Update the old keyspace group. splitSourceKg.Keyspaces = splitSourceKeyspaces @@ -623,7 +624,7 @@ func buildSplitKeyspaces( // Split according to the new keyspace list. if newNum != 0 { if newNum > oldNum { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( oldKeyspaceMap = make(map[uint32]struct{}, oldNum) @@ -634,10 +635,10 @@ func buildSplitKeyspaces( } for _, keyspace := range new { if keyspace == constant.DefaultKeyspaceID { - return nil, nil, ErrModifyDefaultKeyspace + return nil, nil, errs.ErrModifyDefaultKeyspace } if _, ok := oldKeyspaceMap[keyspace]; !ok { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } newKeyspaceMap[keyspace] = struct{}{} } @@ -662,7 +663,7 @@ func buildSplitKeyspaces( } // Split according to the start and end keyspace ID. if startKeyspaceID == 0 && endKeyspaceID == 0 { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( newSplit = make([]uint32, 0, oldNum) @@ -681,7 +682,7 @@ func buildSplitKeyspaces( } // Check if the new keyspace list is empty. if len(newSplit) == 0 { - return nil, nil, ErrKeyspaceGroupWithEmptyKeyspace + return nil, nil, errs.ErrKeyspaceGroupWithEmptyKeyspace } // Get the split keyspace list for the old keyspace group. oldSplit := make([]uint32, 0, oldNum-len(newSplit)) @@ -705,11 +706,11 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitTargetKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetID) } // Check if it's in the split state. if !splitTargetKg.IsSplitTarget() { - return ErrKeyspaceGroupNotInSplit(splitTargetID) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetID) } // Load the split source keyspace group then. splitSourceKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetKg.SplitSource()) @@ -717,10 +718,10 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetKg.SplitSource()) } if !splitSourceKg.IsSplitSource() { - return ErrKeyspaceGroupNotInSplit(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetKg.SplitSource()) } splitTargetKg.SplitState = nil splitSourceKg.SplitState = nil @@ -765,13 +766,13 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, existMembers map[st return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } for addr := range existMembers { @@ -788,14 +789,14 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, existMembers map[st case <-ticker.C: } if m.GetNodesCount() == 0 { // double check - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } if len(existMembers) == m.GetNodesCount() { break } addr := m.nodesBalancer.Next() if addr == "" { - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } if _, ok := existMembers[addr]; ok { continue @@ -831,13 +832,13 @@ func (m *GroupManager) SetNodesForKeyspaceGroup(id uint32, nodes []string) error return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } members := make([]endpoint.KeyspaceGroupMember, 0, len(nodes)) for _, node := range nodes { @@ -868,13 +869,13 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } inKeyspaceGroup := false members := make([]endpoint.KeyspaceGroupMember, 0, len(kg.Members)) @@ -886,7 +887,7 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior members = append(members, member) } if !inKeyspaceGroup { - return ErrNodeNotInKeyspaceGroup + return errs.ErrNodeNotInKeyspaceGroup } kg.Members = members return m.store.SaveKeyspaceGroup(txn, kg) @@ -920,10 +921,10 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin // - Load and update the target keyspace group. // So we pre-check the number of operations to avoid exceeding the maximum number of etcd transaction. if (mergeListNum+1)*2 > etcdutil.MaxEtcdTxnOps { - return ErrExceedMaxEtcdTxnOps + return errs.ErrExceedMaxEtcdTxnOps } if slice.Contains(mergeList, constant.DefaultKeyspaceGroupID) { - return ErrModifyDefaultKeyspaceGroup + return errs.ErrModifyDefaultKeyspaceGroup } var ( groups = make(map[uint32]*endpoint.KeyspaceGroup, mergeListNum+1) @@ -939,15 +940,15 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin return err } if kg == nil { - return ErrKeyspaceGroupNotExists(kgID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(kgID) } // A keyspace group can not be merged if it's in splitting. if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(kgID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(kgID) } // A keyspace group can not be split when it is in merging. if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(kgID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(kgID) } groups[kgID] = kg } @@ -1013,11 +1014,11 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if mergeTargetKg == nil { - return ErrKeyspaceGroupNotExists(mergeTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(mergeTargetID) } // Check if it's in the merging state. if !mergeTargetKg.IsMergeTarget() { - return ErrKeyspaceGroupNotInMerging(mergeTargetID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(mergeTargetID) } // Make sure all merging keyspace groups are deleted. for _, kgID := range mergeTargetKg.MergeState.MergeList { @@ -1026,7 +1027,7 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if kg != nil { - return ErrKeyspaceGroupNotInMerging(kgID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(kgID) } } mergeList = mergeTargetKg.MergeState.MergeList @@ -1150,7 +1151,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if kg == nil { - return "", ErrKeyspaceGroupNotExists(id) + return "", errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } primaryPath := keypath.LeaderPath(&keypath.MsParam{ @@ -1163,7 +1164,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if !ok { - return "", ErrKeyspaceGroupPrimaryNotFound + return "", errs.ErrKeyspaceGroupPrimaryNotFound } // The format of leader name is address-groupID. contents := strings.Split(leader.GetName(), "-") diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index c615627be71..68461855c6f 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -23,6 +23,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" @@ -256,13 +257,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { re.NoError(err) // split the default keyspace err = suite.kgm.SplitKeyspaceGroupByID(0, 4, []uint32{constant.DefaultKeyspaceID}) - re.ErrorIs(err, ErrModifyDefaultKeyspace) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspace) // split the keyspace group 1 to 4 err = suite.kgm.SplitKeyspaceGroupByID(1, 4, []uint32{444}) - re.ErrorIs(err, ErrKeyspaceGroupNotEnoughReplicas) + re.ErrorIs(err, errs.ErrKeyspaceGroupNotEnoughReplicas) // split the keyspace group 2 to 4 without giving any keyspace err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) // split the keyspace group 2 to 4 err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) re.NoError(err) @@ -283,25 +284,25 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // finish the split of the keyspace group 2 err = suite.kgm.FinishSplitKeyspaceByID(2) - re.ErrorContains(err, ErrKeyspaceGroupNotInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(2).Error()) // finish the split of a non-existing keyspace group err = suite.kgm.FinishSplitKeyspaceByID(5) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // split the in-split keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) // remove the in-split keyspace group kg2, err = suite.kgm.DeleteKeyspaceGroupByID(2) re.Nil(kg2) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) kg4, err = suite.kgm.DeleteKeyspaceGroupByID(4) re.Nil(kg4) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // update the in-split keyspace group err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "2", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "4", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // finish the split of keyspace group 4 err = suite.kgm.FinishSplitKeyspaceByID(4) @@ -321,13 +322,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // split a non-existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(3, 5, nil) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(3).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(3).Error()) // split into an existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{111}) - re.ErrorIs(err, ErrKeyspaceGroupExists) + re.ErrorIs(err, errs.ErrKeyspaceGroupExists) // split with the wrong keyspaces. err = suite.kgm.SplitKeyspaceGroupByID(2, 5, []uint32{111, 222, 444}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) } func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplitRange() { @@ -449,13 +450,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { // merge a non-existing keyspace group err = suite.kgm.MergeKeyspaceGroups(4, []uint32{5}) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // merge with the number of keyspace groups exceeds the limit err = suite.kgm.MergeKeyspaceGroups(1, make([]uint32, etcdutil.MaxEtcdTxnOps/2)) - re.ErrorIs(err, ErrExceedMaxEtcdTxnOps) + re.ErrorIs(err, errs.ErrExceedMaxEtcdTxnOps) // merge the default keyspace group err = suite.kgm.MergeKeyspaceGroups(1, []uint32{constant.DefaultKeyspaceGroupID}) - re.ErrorIs(err, ErrModifyDefaultKeyspaceGroup) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspaceGroup) } func TestBuildSplitKeyspaces(t *testing.T) { @@ -484,7 +485,7 @@ func TestBuildSplitKeyspaces(t *testing.T) { { old: []uint32{1, 2, 3, 4, 5}, new: []uint32{6}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, { old: []uint32{1, 2}, @@ -545,11 +546,11 @@ func TestBuildSplitKeyspaces(t *testing.T) { old: []uint32{1, 2, 3, 4, 5}, startKeyspaceID: 7, endKeyspaceID: 10, - err: ErrKeyspaceGroupWithEmptyKeyspace, + err: errs.ErrKeyspaceGroupWithEmptyKeyspace, }, { old: []uint32{1, 2, 3, 4, 5}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, } for idx, testCase := range testCases { diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 95b3162eed0..e68a8fa60e8 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -38,60 +38,6 @@ const ( ) var ( - // ErrKeyspaceNotFound is used to indicate target keyspace does not exist. - ErrKeyspaceNotFound = errors.New("keyspace does not exist") - // ErrRegionSplitTimeout indices to split region timeout - ErrRegionSplitTimeout = errors.New("region split timeout") - // ErrRegionSplitFailed indices to split region failed - ErrRegionSplitFailed = errors.New("region split failed") - // ErrKeyspaceExists indicates target keyspace already exists. - // It's used when creating a new keyspace. - ErrKeyspaceExists = errors.New("keyspace already exists") - // ErrKeyspaceGroupExists indicates target keyspace group already exists. - ErrKeyspaceGroupExists = errors.New("keyspace group already exists") - // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. - ErrKeyspaceGroupNotExists = func(groupID uint32) error { - return errors.Errorf("keyspace group %v does not exist", groupID) - } - // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. - ErrKeyspaceGroupInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in split state", groupID) - } - // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. - ErrKeyspaceGroupNotInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in split state", groupID) - } - // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. - ErrKeyspaceGroupInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in merging state", groupID) - } - // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. - ErrKeyspaceGroupNotInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in merging state", groupID) - } - // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. - ErrKeyspaceNotInKeyspaceGroup = errors.New("keyspace is not in this keyspace group") - // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. - ErrKeyspaceNotInAnyKeyspaceGroup = errors.New("keyspace is not in any keyspace group") - // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. - ErrNodeNotInKeyspaceGroup = errors.New("the tso node is not in this keyspace group") - // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. - ErrKeyspaceGroupNotEnoughReplicas = errors.New("not enough replicas in the keyspace group") - // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. - ErrKeyspaceGroupWithEmptyKeyspace = errors.New("keyspace group with empty keyspace") - // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. - ErrModifyDefaultKeyspaceGroup = errors.New("default keyspace group cannot be modified") - // ErrNoAvailableNode is used to indicate no available node in the keyspace group. - ErrNoAvailableNode = errors.New("no available node") - // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. - ErrExceedMaxEtcdTxnOps = errors.New("exceed max etcd txn operations") - // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. - ErrModifyDefaultKeyspace = errors.New("cannot modify default keyspace's state") - errIllegalOperation = errors.New("unknown operation") - - // ErrUnsupportedOperationInKeyspace is used to indicate this is an unsupported operation. - ErrUnsupportedOperationInKeyspace = errors.New("it's a unsupported operation") - // stateTransitionTable lists all allowed next state for the given current state. // Note that transit from any state to itself is allowed for idempotence. stateTransitionTable = map[keyspacepb.KeyspaceState][]keyspacepb.KeyspaceState{ @@ -102,9 +48,6 @@ var ( } // Only keyspaces in the state specified by allowChangeConfig are allowed to change their config. allowChangeConfig = []keyspacepb.KeyspaceState{keyspacepb.KeyspaceState_ENABLED, keyspacepb.KeyspaceState_DISABLED} - - // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. - ErrKeyspaceGroupPrimaryNotFound = errors.New("primary of keyspace group does not exist") ) // validateID check if keyspace falls within the acceptable range. diff --git a/pkg/ratelimit/runner.go b/pkg/ratelimit/runner.go index 211a4f71be1..2a121d17c00 100644 --- a/pkg/ratelimit/runner.go +++ b/pkg/ratelimit/runner.go @@ -16,7 +16,6 @@ package ratelimit import ( "context" - "errors" "sync" "time" @@ -24,6 +23,8 @@ import ( "go.uber.org/zap" "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) // RegionHeartbeatStageName is the name of the stage of the region heartbeat. @@ -58,9 +59,6 @@ type Task struct { retained bool } -// ErrMaxWaitingTasksExceeded is returned when the number of waiting tasks exceeds the maximum. -var ErrMaxWaitingTasksExceeded = errors.New("max waiting tasks exceeded") - type taskID struct { id uint64 name string @@ -217,12 +215,12 @@ func (cr *ConcurrentRunner) RunTask(id uint64, name string, f func(context.Conte maxWait := time.Since(cr.pendingTasks[0].submittedAt) if maxWait > cr.maxPendingDuration { runnerFailedTasks.WithLabelValues(cr.name, task.name).Inc() - return ErrMaxWaitingTasksExceeded + return errs.ErrMaxWaitingTasksExceeded } } if pendingTaskNum > maxPendingTaskNum { runnerFailedTasks.WithLabelValues(cr.name, task.name).Inc() - return ErrMaxWaitingTasksExceeded + return errs.ErrMaxWaitingTasksExceeded } } cr.pendingTasks = append(cr.pendingTasks, task) diff --git a/pkg/schedule/checker/rule_checker.go b/pkg/schedule/checker/rule_checker.go index 28d5988ce1c..7350d92cf58 100644 --- a/pkg/schedule/checker/rule_checker.go +++ b/pkg/schedule/checker/rule_checker.go @@ -16,7 +16,6 @@ package checker import ( "context" - "errors" "math" "math/rand" "time" @@ -42,15 +41,6 @@ import ( const maxPendingListLen = 100000 -var ( - errNoStoreToAdd = errors.New("no store to add peer") - errNoStoreToReplace = errors.New("no store to replace peer") - errPeerCannotBeLeader = errors.New("peer cannot be leader") - errPeerCannotBeWitness = errors.New("peer cannot be witness") - errNoNewLeader = errors.New("no new leader") - errRegionNoLeader = errors.New("region no leader") -) - // RuleChecker fix/improve region by placement rules. type RuleChecker struct { PauseController @@ -103,7 +93,7 @@ func (c *RuleChecker) CheckWithFit(region *core.RegionInfo, fit *placement.Regio // skip no leader region if region.GetLeader() == nil { ruleCheckerRegionNoLeaderCounter.Inc() - log.Debug("fail to check region", zap.Uint64("region-id", region.GetID()), zap.Error(errRegionNoLeader)) + log.Debug("fail to check region", zap.Uint64("region-id", region.GetID()), errs.ZapError(errs.ErrRegionNoLeader)) return } @@ -230,7 +220,7 @@ func (c *RuleChecker) addRulePeer(region *core.RegionInfo, fit *placement.Region } } } - return nil, errNoStoreToAdd + return nil, errs.ErrNoStoreToAdd } peer := &metapb.Peer{StoreId: store, Role: rf.Rule.Role.MetaPeerRole(), IsWitness: isWitness} op, err := operator.CreateAddPeerOperator("add-rule-peer", c.cluster, region, peer, operator.OpReplica) @@ -262,7 +252,7 @@ func (c *RuleChecker) replaceUnexpectedRulePeer(region *core.RegionInfo, rf *pla if store == 0 { ruleCheckerNoStoreReplaceCounter.Inc() c.handleFilterState(region, filterByTempState) - return nil, errNoStoreToReplace + return nil, errs.ErrNoStoreToReplace } newPeer := &metapb.Peer{StoreId: store, Role: rf.Rule.Role.MetaPeerRole(), IsWitness: fastFailover} // pick the smallest leader store to avoid the Offline store be snapshot generator bottleneck. @@ -325,7 +315,7 @@ func (c *RuleChecker) fixLooseMatchPeer(region *core.RegionInfo, fit *placement. return operator.CreateTransferLeaderOperator("fix-leader-role", c.cluster, region, peer.GetStoreId(), []uint64{}, 0) } ruleCheckerNotAllowLeaderCounter.Inc() - return nil, errPeerCannotBeLeader + return nil, errs.ErrPeerCannotBeLeader } if region.GetLeader().GetId() == peer.GetId() && rf.Rule.Role == placement.Follower { ruleCheckerFixFollowerRoleCounter.Inc() @@ -335,14 +325,14 @@ func (c *RuleChecker) fixLooseMatchPeer(region *core.RegionInfo, fit *placement. } } ruleCheckerNoNewLeaderCounter.Inc() - return nil, errNoNewLeader + return nil, errs.ErrNoNewLeader } if core.IsVoter(peer) && rf.Rule.Role == placement.Learner { ruleCheckerDemoteVoterRoleCounter.Inc() return operator.CreateDemoteVoterOperator("fix-demote-voter", c.cluster, region, peer) } if region.GetLeader().GetId() == peer.GetId() && rf.Rule.IsWitness { - return nil, errPeerCannotBeWitness + return nil, errs.ErrPeerCannotBeWitness } if !core.IsWitness(peer) && rf.Rule.IsWitness && c.isWitnessEnabled() { c.switchWitnessCache.UpdateTTL(c.cluster.GetCheckerConfig().GetSwitchWitnessInterval()) diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index fdcbd705398..0e4dd43ecb2 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -55,8 +55,6 @@ var ( scatterUnnecessaryCounter = scatterCounter.WithLabelValues("unnecessary", "") scatterFailCounter = scatterCounter.WithLabelValues("fail", "") scatterSuccessCounter = scatterCounter.WithLabelValues("success", "") - errRegionNotFound = errors.New("region not found") - errEmptyRegion = errors.New("empty region") ) const ( @@ -169,7 +167,7 @@ func (r *RegionScatterer) ScatterRegionsByRange(startKey, endKey []byte, group s regions := r.cluster.ScanRegions(startKey, endKey, -1) if len(regions) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, nil, errEmptyRegion + return 0, nil, errs.ErrEmptyRegion } failures := make(map[uint64]error, len(regions)) regionMap := make(map[uint64]*core.RegionInfo, len(regions)) @@ -188,13 +186,13 @@ func (r *RegionScatterer) ScatterRegionsByRange(startKey, endKey []byte, group s func (r *RegionScatterer) ScatterRegionsByID(regionsID []uint64, group string, retryLimit int, skipStoreLimit bool) (int, map[uint64]error, error) { if len(regionsID) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, nil, errEmptyRegion + return 0, nil, errs.ErrEmptyRegion } if len(regionsID) == 1 { region := r.cluster.GetRegion(regionsID[0]) if region == nil { scatterSkipNoRegionCounter.Inc() - return 0, nil, errRegionNotFound + return 0, nil, errs.ErrRegionNotFound } } failures := make(map[uint64]error, len(regionsID)) @@ -230,7 +228,7 @@ func (r *RegionScatterer) ScatterRegionsByID(regionsID []uint64, group string, r func (r *RegionScatterer) scatterRegions(regions map[uint64]*core.RegionInfo, failures map[uint64]error, group string, retryLimit int, skipStoreLimit bool) (int, error) { if len(regions) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, errEmptyRegion + return 0, errs.ErrEmptyRegion } if retryLimit > maxRetryLimit { retryLimit = maxRetryLimit diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index 0245016734c..969113bb17e 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -300,7 +300,7 @@ func UpdateKeyspaceConfig(c *gin.Context) { // Check if the update is supported. for _, mutation := range mutations { if mutation.Key == keyspace.GCManagementType && mutation.Value == keyspace.KeyspaceLevelGC { - err = keyspace.ErrUnsupportedOperationInKeyspace + err = errs.ErrUnsupportedOperationInKeyspace c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return } diff --git a/server/keyspace_service.go b/server/keyspace_service.go index e110742aa45..eeca31bc942 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/tikv/pd/pkg/keyspace" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) @@ -39,9 +39,9 @@ type KeyspaceServer struct { // getErrorHeader returns corresponding ResponseHeader based on err. func getErrorHeader(err error) *pdpb.ResponseHeader { switch err { - case keyspace.ErrKeyspaceExists: + case errs.ErrKeyspaceExists: return wrapErrorToHeader(pdpb.ErrorType_DUPLICATED_ENTRY, err.Error()) - case keyspace.ErrKeyspaceNotFound: + case errs.ErrKeyspaceNotFound: return wrapErrorToHeader(pdpb.ErrorType_ENTRY_NOT_FOUND, err.Error()) default: return wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error()) From ecb31de2bc93e5c191259cbe0a4f558f0290fbb8 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 17:10:24 +0800 Subject: [PATCH 10/10] client: move circuit breaker to `pkg` (#8932) ref tikv/pd#8678, ref tikv/pd#8690 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 2 +- client/inner_client.go | 2 +- client/opt/option.go | 2 +- client/{ => pkg}/circuitbreaker/circuit_breaker.go | 0 client/{ => pkg}/circuitbreaker/circuit_breaker_test.go | 0 errors.toml | 5 ----- pkg/errs/errno.go | 1 - tests/integrations/client/client_test.go | 2 +- 8 files changed, 4 insertions(+), 10 deletions(-) rename client/{ => pkg}/circuitbreaker/circuit_breaker.go (100%) rename client/{ => pkg}/circuitbreaker/circuit_breaker_test.go (100%) diff --git a/client/client.go b/client/client.go index 0e48707cd8d..272d6c597b5 100644 --- a/client/client.go +++ b/client/client.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/clients/tso" @@ -43,6 +42,7 @@ import ( "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" ) diff --git a/client/inner_client.go b/client/inner_client.go index 045a9a6eed8..91f999dd3b5 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -15,11 +15,11 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" sd "github.com/tikv/pd/client/servicediscovery" ) diff --git a/client/opt/option.go b/client/opt/option.go index c7a0bb17195..af95a225fab 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" - cb "github.com/tikv/pd/client/circuitbreaker" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/pkg/circuitbreaker/circuit_breaker.go similarity index 100% rename from client/circuitbreaker/circuit_breaker.go rename to client/pkg/circuitbreaker/circuit_breaker.go diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/pkg/circuitbreaker/circuit_breaker_test.go similarity index 100% rename from client/circuitbreaker/circuit_breaker_test.go rename to client/pkg/circuitbreaker/circuit_breaker_test.go diff --git a/errors.toml b/errors.toml index 26b15c38077..785de6662f4 100644 --- a/errors.toml +++ b/errors.toml @@ -996,11 +996,6 @@ error = ''' get allocator manager failed, %s ''' -["PD:tso:ErrGetLocalAllocator"] -error = ''' -get local allocator failed, %s -''' - ["PD:tso:ErrGetMinTS"] error = ''' get min ts failed, %s diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index fd9ff3c95ff..30e24647a3f 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -41,7 +41,6 @@ var ( // tso errors var ( ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) - ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) ErrUpdateTimestamp = errors.Normalize("update timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrUpdateTimestamp")) diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index c0b762d0983..fadfb952e4c 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -45,10 +45,10 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" pd "github.com/tikv/pd/client" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core"