Skip to content

Commit

Permalink
ccl/sqlproxyccl: rename tenant.Resolver to tenant.Directory
Browse files Browse the repository at this point in the history
This commit completes the work of making tenant.Directory an interface.
Since we have renamed the previous Directory struct into a serviceDirectory
struct, we can now use Directory as an interface for what was previously known
as the resolver.

Release note: None
  • Loading branch information
jaylim-crl committed Mar 20, 2022
1 parent 695f8e1 commit 6123205
Show file tree
Hide file tree
Showing 10 changed files with 37 additions and 34 deletions.
6 changes: 2 additions & 4 deletions pkg/ccl/sqlproxyccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,10 +48,8 @@ type connector struct {
// Directory corresponds to the tenant directory, which will be used to
// resolve tenants to their corresponding IP addresses.
//
// TODO(jaylim-crl): Replace this with a Directory interface.
//
// NOTE: This field is required.
Directory tenant.Resolver
Directory tenant.Directory

// StartupMsg represents the startup message associated with the client.
// This will be used when establishing a pgwire connection with the SQL pod.
Expand Down Expand Up @@ -342,7 +340,7 @@ func isRetriableConnectorError(err error) bool {
// reportFailureToDirectory is a hookable function that calls the given tenant
// directory's ReportFailure method.
var reportFailureToDirectory = func(
ctx context.Context, tenantID roachpb.TenantID, addr string, directory tenant.Resolver,
ctx context.Context, tenantID roachpb.TenantID, addr string, directory tenant.Directory,
) error {
return directory.ReportFailure(ctx, tenantID, addr)
}
22 changes: 11 additions & 11 deletions pkg/ccl/sqlproxyccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -435,7 +435,7 @@ func TestConnector_dialTenantCluster(t *testing.T) {
c := &connector{
TenantID: roachpb.MakeTenantID(42),
}
c.Directory = &testResolver{
c.Directory = &testDirectory{
reportFailureFn: func(fnCtx context.Context, tenantID roachpb.TenantID, addr string) error {
reportFailureFnCount++
require.Equal(t, ctx, fnCtx)
Expand Down Expand Up @@ -485,7 +485,7 @@ func TestConnector_lookupAddr(t *testing.T) {
c = &connector{
ClusterName: "my-foo",
TenantID: roachpb.MakeTenantID(10),
Directory: &testResolver{
Directory: &testDirectory{
ensureTenantAddrFn: func(fnCtx context.Context, tenantID roachpb.TenantID, clusterName string) (string, error) {
ensureTenantAddrFnCount++
require.Equal(t, ctx, fnCtx)
Expand All @@ -508,7 +508,7 @@ func TestConnector_lookupAddr(t *testing.T) {
c = &connector{
ClusterName: "my-foo",
TenantID: roachpb.MakeTenantID(10),
Directory: &testResolver{
Directory: &testDirectory{
ensureTenantAddrFn: func(fnCtx context.Context, tenantID roachpb.TenantID, clusterName string) (string, error) {
ensureTenantAddrFnCount++
require.Equal(t, ctx, fnCtx)
Expand All @@ -531,7 +531,7 @@ func TestConnector_lookupAddr(t *testing.T) {
c = &connector{
ClusterName: "my-foo",
TenantID: roachpb.MakeTenantID(10),
Directory: &testResolver{
Directory: &testDirectory{
ensureTenantAddrFn: func(fnCtx context.Context, tenantID roachpb.TenantID, clusterName string) (string, error) {
ensureTenantAddrFnCount++
require.Equal(t, ctx, fnCtx)
Expand All @@ -554,7 +554,7 @@ func TestConnector_lookupAddr(t *testing.T) {
c = &connector{
ClusterName: "my-foo",
TenantID: roachpb.MakeTenantID(10),
Directory: &testResolver{
Directory: &testDirectory{
ensureTenantAddrFn: func(fnCtx context.Context, tenantID roachpb.TenantID, clusterName string) (string, error) {
ensureTenantAddrFnCount++
require.Equal(t, ctx, fnCtx)
Expand Down Expand Up @@ -674,31 +674,31 @@ func TestRetriableConnectorError(t *testing.T) {
require.True(t, errors.Is(err, errRetryConnectorSentinel))
}

var _ tenant.Resolver = &testResolver{}
var _ tenant.Directory = &testDirectory{}

// testResolver is a test implementation of the tenant resolver.
type testResolver struct {
// testDirectory is a test implementation of the tenant resolver.
type testDirectory struct {
ensureTenantAddrFn func(ctx context.Context, tenantID roachpb.TenantID, clusterName string) (string, error)
lookupTenantAddrsFn func(ctx context.Context, tenantID roachpb.TenantID) ([]string, error)
reportFailureFn func(ctx context.Context, tenantID roachpb.TenantID, addr string) error
}

// EnsureTenantAddr implements the Resolver interface.
func (r *testResolver) EnsureTenantAddr(
func (r *testDirectory) EnsureTenantAddr(
ctx context.Context, tenantID roachpb.TenantID, clusterName string,
) (string, error) {
return r.ensureTenantAddrFn(ctx, tenantID, clusterName)
}

// LookupTenantAddrs implements the Resolver interface.
func (r *testResolver) LookupTenantAddrs(
func (r *testDirectory) LookupTenantAddrs(
ctx context.Context, tenantID roachpb.TenantID,
) ([]string, error) {
return r.lookupTenantAddrsFn(ctx, tenantID)
}

// ReportFailure implements the Resolver interface.
func (r *testResolver) ReportFailure(
func (r *testDirectory) ReportFailure(
ctx context.Context, tenantID roachpb.TenantID, addr string,
) error {
return r.reportFailureFn(ctx, tenantID, addr)
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/sqlproxyccl/proxy_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ type ProxyOptions struct {
// connection.
RoutingRule string
// DirectoryAddr specified optional {HOSTNAME}:{PORT} for service that does
// the resolution from backend id to IP address. If specified - it will be
// the resolution from tenants to IP addresses. If specified - it will be
// used instead of the routing rule above.
DirectoryAddr string
// RatelimitBaseDelay is the initial backoff after a failed login attempt.
Expand Down Expand Up @@ -127,8 +127,8 @@ type proxyHandler struct {
// idleMonitor will detect idle connections to DRAINING pods.
idleMonitor *idle.Monitor

// directory is used to resolve backend id to IP addresses.
directory tenant.Resolver
// directory is used to resolve tenants to their corresponding IP addresses.
directory tenant.Directory

// CertManger keeps up to date the certificates used.
certManager *certmgr.CertManager
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/sqlproxyccl/proxy_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -689,7 +689,7 @@ func TestDirectoryConnect(t *testing.T) {
// Ensure that Directory.ReportFailure is being called correctly.
countReports := 0
defer testutils.TestingHook(&reportFailureToDirectory, func(
ctx context.Context, tenantID roachpb.TenantID, addr string, directory tenant.Resolver,
ctx context.Context, tenantID roachpb.TenantID, addr string, directory tenant.Directory,
) error {
require.Equal(t, roachpb.MakeTenantID(28), tenantID)
addrs, err := directory.LookupTenantAddrs(ctx, tenantID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/sqlproxyccl/tenant/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "tenant",
srcs = ["resolver.go"],
srcs = ["directory.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/tenant",
visibility = ["//visibility:public"],
deps = ["//pkg/roachpb"],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,11 +14,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

// Resolver is an interface for the tenant directory.
//
// TODO(jaylim-crl): Rename this to Directory, and the current tenant.Directory
// to tenant.directory.
type Resolver interface {
// Directory is an interface for the tenant directory.
type Directory interface {
// EnsureTenantAddr returns an IP address of one of the given tenant's SQL
// processes based on the tenantID and clusterName fields. This should block
// until the process associated with the IP is ready.
Expand Down
10 changes: 9 additions & 1 deletion pkg/ccl/sqlproxyccl/tenant/servicedir/service_directory.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,8 @@ type serviceDirectory struct {
}
}

var _ tenant.Directory = &serviceDirectory{}

// NewServiceDirectory constructs a new serviceDirectory instance that tracks
// SQL tenant processes managed by a given Directory server. The given context
// is used for tracing pod watcher activity.
Expand All @@ -107,7 +109,7 @@ type serviceDirectory struct {
// needed.
func NewServiceDirectory(
ctx context.Context, stopper *stop.Stopper, client DirectoryClient, opts ...DirOption,
) (tenant.Resolver, error) {
) (tenant.Directory, error) {
dir := &serviceDirectory{client: client, stopper: stopper}

dir.mut.tenants = make(map[roachpb.TenantID]*tenantEntry)
Expand Down Expand Up @@ -140,6 +142,8 @@ func NewServiceDirectory(
// such as the name of the cluster, before being allowed to connect. Similarly,
// if the tenant does not exist (e.g. because it was deleted), EnsureTenantAddr
// returns a GRPC NotFound error.
//
// EnsureTenantAddr implements the tenant.Directory interface.
func (d *serviceDirectory) EnsureTenantAddr(
ctx context.Context, tenantID roachpb.TenantID, clusterName string,
) (string, error) {
Expand Down Expand Up @@ -173,6 +177,8 @@ func (d *serviceDirectory) EnsureTenantAddr(
// into the directory's cache (LookupTenantAddrs will never attempt to fetch it).
// If no processes are available for the tenant, LookupTenantAddrs will return the
// empty set (unlike EnsureTenantAddr).
//
// LookupTenantAddrs implements the tenant.Directory interface.
func (d *serviceDirectory) LookupTenantAddrs(
ctx context.Context, tenantID roachpb.TenantID,
) ([]string, error) {
Expand Down Expand Up @@ -203,6 +209,8 @@ func (d *serviceDirectory) LookupTenantAddrs(
// particular pod as "unhealthy" so that it's less likely to be chosen.
// However, today there can be at most one pod for a given tenant, so it
// must always be chosen. Keep the parameter as a placeholder for the future.
//
// ReportFailure implements the tenant.Directory interface.
func (d *serviceDirectory) ReportFailure(
ctx context.Context, tenantID roachpb.TenantID, addr string,
) error {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -519,7 +519,7 @@ func newTestDirectory(
t *testing.T, opts ...servicedir.DirOption,
) (
tc serverutils.TestClusterInterface,
directory tenant.Resolver,
directory tenant.Directory,
tds *tenantdirsvr.TestDirectoryServer,
) {
tc = serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{
Expand Down
12 changes: 6 additions & 6 deletions pkg/ccl/sqlproxyccl/tenant/simpledir/simple_directory.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,18 +27,18 @@ type simpleDirectory struct {
podAddr string
}

var _ tenant.Resolver = &simpleDirectory{}
var _ tenant.Directory = &simpleDirectory{}

// NewSimpleDirectory constructs a new simple directory instance that abides to
// the Resolver interface.
func NewSimpleDirectory(podAddr string) tenant.Resolver {
// the tenant.Directory interface.
func NewSimpleDirectory(podAddr string) tenant.Directory {
return &simpleDirectory{podAddr: podAddr}
}

// EnsureTenantAddr returns the SQL pod address associated with this directory.
// If the address cannot be resolved, a GRPC NotFound error will be returned.
//
// EnsureTenantAddr implements the Resolver interface.
// EnsureTenantAddr implements the tenant.Directory interface.
func (d *simpleDirectory) EnsureTenantAddr(
ctx context.Context, tenantID roachpb.TenantID, clusterName string,
) (string, error) {
Expand All @@ -52,7 +52,7 @@ func (d *simpleDirectory) EnsureTenantAddr(
// of tenantID. If that address cannot be resolved, a GRPC NotFound error will
// be returned.
//
// LookupTenantAddrs implements the Resolver interface.
// LookupTenantAddrs implements the tenant.Directory interface.
func (d *simpleDirectory) LookupTenantAddrs(
ctx context.Context, tenantID roachpb.TenantID,
) ([]string, error) {
Expand All @@ -64,7 +64,7 @@ func (d *simpleDirectory) LookupTenantAddrs(

// ReportFailure is a no-op for the simple directory.
//
// ReportFailure implements the Resolver interface.
// ReportFailure implements the tenant.Directory interface.
func (d *simpleDirectory) ReportFailure(
ctx context.Context, tenantID roachpb.TenantID, addr string,
) error {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/cliflags/flags_mt.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ var (

DirectoryAddr = FlagInfo{
Name: "directory",
Description: "Directory address of the service doing resolution from backend id to IP.",
Description: "Directory address of the service doing resolution of tenants to their IP addresses.",
}

// TODO(chrisseto): Remove skip-verify as a CLI option. It should only be
Expand Down

0 comments on commit 6123205

Please sign in to comment.