Skip to content

Commit

Permalink
Use a pkg dictionary
Browse files Browse the repository at this point in the history
Signed-off-by: JmPotato <[email protected]>
  • Loading branch information
JmPotato committed Nov 25, 2024
1 parent 777867a commit c2b5d9e
Show file tree
Hide file tree
Showing 48 changed files with 66 additions and 66 deletions.
16 changes: 8 additions & 8 deletions client/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,14 +31,14 @@ import (
"github.com/pingcap/kvproto/pkg/pdpb"
"github.com/pingcap/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/clients/metastorage"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/metrics"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/client/pkg/constants"
"github.com/tikv/pd/client/pkg/utils/tlsutil"
sd "github.com/tikv/pd/client/servicediscovery"
"github.com/tikv/pd/client/utils"
"github.com/tikv/pd/client/utils/tlsutil"
"go.uber.org/zap"
)

Expand Down Expand Up @@ -263,7 +263,7 @@ func NewClientWithContext(
security SecurityOption, opts ...opt.ClientOption,
) (Client, error) {
return createClientWithKeyspace(ctx, callerComponent,
utils.NullKeyspaceID, svrAddrs, security, opts...)
constants.NullKeyspaceID, svrAddrs, security, opts...)
}

// NewClientWithKeyspace creates a client with context and the specified keyspace id.
Expand All @@ -274,9 +274,9 @@ func NewClientWithKeyspace(
keyspaceID uint32, svrAddrs []string,
security SecurityOption, opts ...opt.ClientOption,
) (Client, error) {
if keyspaceID < utils.DefaultKeyspaceID || keyspaceID > utils.MaxKeyspaceID {
if keyspaceID < constants.DefaultKeyspaceID || keyspaceID > constants.MaxKeyspaceID {
return nil, errors.Errorf("invalid keyspace id %d. It must be in the range of [%d, %d]",
keyspaceID, utils.DefaultKeyspaceID, utils.MaxKeyspaceID)
keyspaceID, constants.DefaultKeyspaceID, constants.MaxKeyspaceID)
}
return createClientWithKeyspace(ctx, callerComponent, keyspaceID,
svrAddrs, security, opts...)
Expand Down Expand Up @@ -366,7 +366,7 @@ type apiContextV2 struct {
// NewAPIContextV2 creates a API context with the specified keyspace name for V2.
func NewAPIContextV2(keyspaceName string) APIContext {
if len(keyspaceName) == 0 {
keyspaceName = utils.DefaultKeyspaceName
keyspaceName = constants.DefaultKeyspaceName
}
return &apiContextV2{keyspaceName: keyspaceName}
}
Expand Down Expand Up @@ -426,7 +426,7 @@ func newClientWithKeyspaceName(
inner: &innerClient{
// Create a PD service discovery with null keyspace id, then query the real id with the keyspace name,
// finally update the keyspace id to the PD service discovery for the following interactions.
keyspaceID: utils.NullKeyspaceID,
keyspaceID: constants.NullKeyspaceID,
updateTokenConnectionCh: make(chan struct{}, 1),
ctx: clientCtx,
cancel: clientCancel,
Expand Down
6 changes: 3 additions & 3 deletions client/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@ import (

"github.com/pingcap/errors"
"github.com/stretchr/testify/require"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/utils/testutil"
"github.com/tikv/pd/client/utils/tsoutil"
"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"
)

Expand Down
2 changes: 1 addition & 1 deletion client/http/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/pingcap/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/retry"
sd "github.com/tikv/pd/client/servicediscovery"
"go.uber.org/zap"
)
Expand Down
2 changes: 1 addition & 1 deletion client/http/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (

"github.com/stretchr/testify/require"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/retry"
)

func TestPDAllowFollowerHandleHeader(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion client/http/interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +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/retry"
"github.com/tikv/pd/client/pkg/retry"
)

// Client is a PD (Placement Driver) HTTP client.
Expand Down
2 changes: 1 addition & 1 deletion client/http/request_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ package http
import (
"fmt"

"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/retry"
"go.uber.org/zap"
)

Expand Down
2 changes: 1 addition & 1 deletion client/meta_storage_client.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/metrics"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/utils/grpcutil"
"github.com/tikv/pd/client/pkg/utils/grpcutil"
)

// metaStorageClient gets the meta storage client from current PD leader.
Expand Down
2 changes: 1 addition & 1 deletion client/opt/option_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import (
"time"

"github.com/stretchr/testify/require"
"github.com/tikv/pd/client/utils/testutil"
"github.com/tikv/pd/client/pkg/utils/testutil"
)

func TestDynamicOptionChange(t *testing.T) {
Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
func TestGetComponent(t *testing.T) {
re := require.New(t)

re.Equal(Component("github.com/tikv/pd/client/caller"), GetComponent(0))
re.Equal(Component("github.com/tikv/pd/client/pkg/caller"), GetComponent(0))
re.Equal(Component("testing"), GetComponent(1))
re.Equal(Component("runtime"), GetComponent(2))
re.Equal(Component("unknown"), GetComponent(3))
Expand Down
2 changes: 1 addition & 1 deletion client/utils/mics.go → client/pkg/constants/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package utils
package constants

const (
// DefaultKeyspaceID is the default keyspace ID.
Expand Down
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
14 changes: 7 additions & 7 deletions client/servicediscovery/pd_service_discovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,10 @@ import (
"github.com/pingcap/log"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/utils"
"github.com/tikv/pd/client/utils/grpcutil"
"github.com/tikv/pd/client/utils/tlsutil"
"github.com/tikv/pd/client/pkg/constants"
"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"
Expand Down Expand Up @@ -454,7 +454,7 @@ func NewDefaultPDServiceDiscovery(
urls []string, tlsCfg *tls.Config,
) ServiceDiscovery {
var wg sync.WaitGroup
return NewPDServiceDiscovery(ctx, cancel, &wg, nil, nil, utils.DefaultKeyspaceID, urls, tlsCfg, opt.NewOption())
return NewPDServiceDiscovery(ctx, cancel, &wg, nil, nil, constants.DefaultKeyspaceID, urls, tlsCfg, opt.NewOption())
}

// NewPDServiceDiscovery returns a new PD service discovery-based client.
Expand Down Expand Up @@ -501,7 +501,7 @@ func (c *pdServiceDiscovery) Init() error {

// We need to update the keyspace ID before we discover and update the service mode
// so that TSO in API mode can be initialized with the correct keyspace ID.
if c.keyspaceID == utils.NullKeyspaceID && c.updateKeyspaceIDFunc != nil {
if c.keyspaceID == constants.NullKeyspaceID && c.updateKeyspaceIDFunc != nil {
if err := c.initRetry(c.updateKeyspaceIDFunc); err != nil {
return err
}
Expand Down Expand Up @@ -668,7 +668,7 @@ func (c *pdServiceDiscovery) SetKeyspaceID(keyspaceID uint32) {
// GetKeyspaceGroupID returns the ID of the keyspace group
func (*pdServiceDiscovery) GetKeyspaceGroupID() uint32 {
// PD/API service only supports the default keyspace group
return utils.DefaultKeyspaceGroupID
return constants.DefaultKeyspaceGroupID
}

// DiscoverMicroservice discovers the microservice with the specified type and returns the server urls.
Expand Down
6 changes: 3 additions & 3 deletions client/servicediscovery/pd_service_discovery_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,9 @@ import (
"github.com/stretchr/testify/suite"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/utils/grpcutil"
"github.com/tikv/pd/client/utils/testutil"
"github.com/tikv/pd/client/utils/tlsutil"
"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"
Expand Down
12 changes: 6 additions & 6 deletions client/servicediscovery/tso_service_discovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ import (
"github.com/tikv/pd/client/clients/metastorage"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/utils"
"github.com/tikv/pd/client/utils/grpcutil"
"github.com/tikv/pd/client/pkg/constants"
"github.com/tikv/pd/client/pkg/utils/grpcutil"
"go.uber.org/zap"
"google.golang.org/grpc"
)
Expand Down Expand Up @@ -182,7 +182,7 @@ func NewTSOServiceDiscovery(
c.tsoServerDiscovery = &tsoServerDiscovery{urls: make([]string, 0)}
// Start with the default keyspace group. The actual keyspace group, to which the keyspace belongs,
// will be discovered later.
c.defaultDiscoveryKey = fmt.Sprintf(tsoSvcDiscoveryFormat, c.clusterID, utils.DefaultKeyspaceGroupID)
c.defaultDiscoveryKey = fmt.Sprintf(tsoSvcDiscoveryFormat, c.clusterID, constants.DefaultKeyspaceGroupID)

log.Info("created tso service discovery",
zap.Uint64("cluster-id", c.clusterID),
Expand Down Expand Up @@ -293,7 +293,7 @@ func (c *tsoServiceDiscovery) GetKeyspaceGroupID() uint32 {
c.keyspaceGroupSD.RLock()
defer c.keyspaceGroupSD.RUnlock()
if c.keyspaceGroupSD.group == nil {
return utils.DefaultKeyspaceGroupID
return constants.DefaultKeyspaceGroupID
}
return c.keyspaceGroupSD.group.Id
}
Expand Down Expand Up @@ -469,7 +469,7 @@ func (c *tsoServiceDiscovery) updateMember() error {
}
members[0].IsPrimary = true
keyspaceGroup = &tsopb.KeyspaceGroup{
Id: utils.DefaultKeyspaceGroupID,
Id: constants.DefaultKeyspaceGroupID,
Members: members,
}
}
Expand Down Expand Up @@ -554,7 +554,7 @@ func (c *tsoServiceDiscovery) findGroupByKeyspaceID(
Header: &tsopb.RequestHeader{
ClusterId: c.clusterID,
KeyspaceId: keyspaceID,
KeyspaceGroupId: utils.DefaultKeyspaceGroupID,
KeyspaceGroupId: constants.DefaultKeyspaceGroupID,
},
KeyspaceId: keyspaceID,
})
Expand Down
4 changes: 2 additions & 2 deletions client/tso_client.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,9 @@ import (
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/metrics"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/utils/grpcutil"
"github.com/tikv/pd/client/pkg/utils/tlsutil"
sd "github.com/tikv/pd/client/servicediscovery"
"github.com/tikv/pd/client/utils/grpcutil"
"github.com/tikv/pd/client/utils/tlsutil"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
Expand Down
6 changes: 3 additions & 3 deletions client/tso_dispatcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,10 @@ import (
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/metrics"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/retry"
"github.com/tikv/pd/client/pkg/utils/timerutil"
"github.com/tikv/pd/client/pkg/utils/tsoutil"
sd "github.com/tikv/pd/client/servicediscovery"
"github.com/tikv/pd/client/utils/timerutil"
"github.com/tikv/pd/client/utils/tsoutil"
"go.uber.org/zap"
)

Expand Down
4 changes: 2 additions & 2 deletions client/tso_stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/pd/client/errs"
"github.com/tikv/pd/client/metrics"
"github.com/tikv/pd/client/utils"
"github.com/tikv/pd/client/pkg/constants"
"go.uber.org/zap"
"google.golang.org/grpc"
)
Expand Down Expand Up @@ -145,7 +145,7 @@ func (s pdTSOStreamAdapter) Recv() (tsoRequestResult, error) {
physical: resp.GetTimestamp().GetPhysical(),
logical: resp.GetTimestamp().GetLogical(),
count: resp.GetCount(),
respKeyspaceGroupID: utils.DefaultKeyspaceGroupID,
respKeyspaceGroupID: constants.DefaultKeyspaceGroupID,
}, nil
}

Expand Down
12 changes: 6 additions & 6 deletions tests/integrations/client/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,9 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/client/pkg/retry"
sd "github.com/tikv/pd/client/servicediscovery"
"github.com/tikv/pd/pkg/core"
"github.com/tikv/pd/pkg/mcs/utils/constant"
Expand Down Expand Up @@ -586,7 +586,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetTsoAndRegionByFollowerFor
leader := cluster.GetLeaderServer()

follower := cluster.GetServer(cluster.GetFollower())
re.NoError(failpoint.Enable("github.com/tikv/pd/client/utils/grpcutil/unreachableNetwork2", fmt.Sprintf("return(\"%s\")", follower.GetAddr())))
re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/unreachableNetwork2", fmt.Sprintf("return(\"%s\")", follower.GetAddr())))

cli := setupCli(ctx, re, suite.endpoints, opt.WithForwardingOption(true))
defer cli.Close()
Expand Down Expand Up @@ -625,7 +625,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetTsoAndRegionByFollowerFor
return false
})

re.NoError(failpoint.Disable("github.com/tikv/pd/client/utils/grpcutil/unreachableNetwork2"))
re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/unreachableNetwork2"))
testutil.Eventually(re, func() bool {
physical, logical, err := cli.GetTS(context.TODO())
if err == nil {
Expand Down Expand Up @@ -655,7 +655,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromLeaderWhenNetwo
leader := cluster.GetLeaderServer()

follower := cluster.GetServer(cluster.GetFollower())
re.NoError(failpoint.Enable("github.com/tikv/pd/client/utils/grpcutil/unreachableNetwork2", fmt.Sprintf("return(\"%s\")", follower.GetAddr())))
re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/unreachableNetwork2", fmt.Sprintf("return(\"%s\")", follower.GetAddr())))

cli := setupCli(ctx, re, suite.endpoints)
defer cli.Close()
Expand All @@ -673,7 +673,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromLeaderWhenNetwo
re.Error(err)
re.Nil(r)

re.NoError(failpoint.Disable("github.com/tikv/pd/client/utils/grpcutil/unreachableNetwork2"))
re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/unreachableNetwork2"))
cli.GetServiceDiscovery().CheckMemberChanged()
testutil.Eventually(re, func() bool {
r, err = cli.GetRegion(context.Background(), []byte("a"))
Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/client/client_tls_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,8 @@ import (

"github.com/stretchr/testify/require"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/pkg/utils/grpcutil"
"github.com/tikv/pd/pkg/utils/netutil"
"github.com/tikv/pd/server/config"
Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/client/gc_client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"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"
Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/client/global_config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
"time"

pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/pkg/caller"

"github.com/pingcap/kvproto/pkg/pdpb"
"github.com/pingcap/log"
Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/client/http_client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client/http"
"github.com/tikv/pd/client/retry"
"github.com/tikv/pd/client/pkg/retry"
"github.com/tikv/pd/pkg/core"
"github.com/tikv/pd/pkg/keyspace"
sc "github.com/tikv/pd/pkg/schedule/config"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/client/resource_group/controller"
sd "github.com/tikv/pd/client/servicediscovery"
"github.com/tikv/pd/pkg/mcs/resourcemanager/server"
Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/mcs/testutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ import (

"github.com/stretchr/testify/require"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/caller"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/pkg/utils/testutil"
"github.com/tikv/pd/pkg/utils/tsoutil"
)
Expand Down
Loading

0 comments on commit c2b5d9e

Please sign in to comment.