Skip to content

Commit

Permalink
sql: make metrics-related tests work with test tenant
Browse files Browse the repository at this point in the history
This required to move `MustGetSQLNetworkCounter` into
`TestTenantInterface` as well as use the correct implementation of the
interface in a few places. It also removes `MustGetSQLCounter` from
`TestServerInterface` since it was redundantly duplicated in there.

This commit pulled in `ccl` into `pgwire` tests, so this commit fixes
tests in those package too. Notably, in `TestPGPreparedQuery` a couple
of test cases were removed (one for the number of tables found in
`system` database, and another for overwriting named zone) since they
don't seem to provide much value and wasn't immediately clear how to
make them work. This also pulled in `gss` auth method, but only on
linux, so this commit adjusts the test harness to ignore it in the error
hint.

Release note: None
  • Loading branch information
yuzefovich committed Jul 21, 2023
1 parent 3499ff5 commit e3b74b0
Show file tree
Hide file tree
Showing 9 changed files with 72 additions and 74 deletions.
31 changes: 11 additions & 20 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -846,11 +846,20 @@ func (t *TestTenant) DrainClients(ctx context.Context) error {
return t.drain.drainClients(ctx, nil /* reporter */)
}

// MustGetSQLCounter implements serverutils.TestTenantInterface.
// MustGetSQLCounter implements the serverutils.TestTenantInterface.
func (t *TestTenant) MustGetSQLCounter(name string) int64 {
return mustGetSQLCounterForRegistry(t.sql.metricsRegistry, name)
}

// MustGetSQLNetworkCounter implements the serverutils.TestTenantInterface.
func (t *TestTenant) MustGetSQLNetworkCounter(name string) int64 {
reg := metric.NewRegistry()
for _, m := range t.sql.pgServer.Metrics() {
reg.AddMetricStruct(m)
}
return mustGetSQLCounterForRegistry(reg, name)
}

// RangeDescIteratorFactory implements the serverutils.TestTenantInterface.
func (t *TestTenant) RangeDescIteratorFactory() interface{} {
return t.sql.execCfg.RangeDescIteratorFactory
Expand Down Expand Up @@ -1400,29 +1409,11 @@ func (ts *TestServer) MustGetSQLCounter(name string) int64 {

// MustGetSQLNetworkCounter implements the serverutils.TestServerInterface.
func (ts *TestServer) MustGetSQLNetworkCounter(name string) int64 {
var c int64
var found bool

reg := metric.NewRegistry()
for _, m := range ts.sqlServer.pgServer.Metrics() {
reg.AddMetricStruct(m)
}
reg.Each(func(n string, v interface{}) {
if name == n {
switch t := v.(type) {
case *metric.Counter:
c = t.Count()
found = true
case *metric.Gauge:
c = t.Value()
found = true
}
}
})
if !found {
panic(fmt.Sprintf("couldn't find metric %s", name))
}
return c
return mustGetSQLCounterForRegistry(reg, name)
}

// Locality returns the Locality used by the TestServer.
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/metric_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,8 +65,9 @@ func TestQueryCounts(t *testing.T) {
DisableDeleteOrphanedLeases: true,
},
}
s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())
srv, sqlDB, _ := serverutils.StartServer(t, params)
defer srv.Stopper().Stop(context.Background())
s := srv.TenantOrServer()

var testcases = []queryCounter{
// The counts are deltas for each query.
Expand Down Expand Up @@ -118,7 +119,7 @@ func TestQueryCounts(t *testing.T) {
}

// Force metric snapshot refresh.
if err := s.WriteSummaries(); err != nil {
if err := srv.WriteSummaries(); err != nil {
t.Fatal(err)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/metric_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (

// initializeQueryCounter returns a queryCounter that accounts for system
// upgrades that may have run DDL statements.
func initializeQueryCounter(s serverutils.TestServerInterface) queryCounter {
func initializeQueryCounter(s serverutils.TestTenantInterface) queryCounter {
return queryCounter{
txnBeginCount: s.MustGetSQLCounter(sql.MetaTxnBeginStarted.Name),
selectCount: s.MustGetSQLCounter(sql.MetaSelectStarted.Name),
Expand All @@ -44,7 +44,7 @@ func initializeQueryCounter(s serverutils.TestServerInterface) queryCounter {
}

func checkCounterDelta(
s serverutils.TestServerInterface, meta metric.Metadata, init, delta int64,
s serverutils.TestTenantInterface, meta metric.Metadata, init, delta int64,
) (int64, error) {
actual := s.MustGetSQLCounter(meta.Name)
if actual != (init + delta) {
Expand All @@ -54,7 +54,7 @@ func checkCounterDelta(
return actual, nil
}

func checkCounterGE(s serverutils.TestServerInterface, meta metric.Metadata, e int64) error {
func checkCounterGE(s serverutils.TestTenantInterface, meta metric.Metadata, e int64) error {
if a := s.MustGetSQLCounter(meta.Name); a < e {
return errors.Errorf("stat %s: expected: actual %d >= %d",
meta.Name, a, e)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ go_test(
shard_count = 16,
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/cloud/impl:cloudimpl",
"//pkg/col/coldata",
"//pkg/col/coldataext",
Expand All @@ -121,7 +122,6 @@ go_test(
"//pkg/server/telemetry",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog/bootstrap",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/clusterunique",
"//pkg/sql/colconv",
Expand Down
26 changes: 19 additions & 7 deletions pkg/sql/pgwire/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,17 +218,21 @@ func hbaRunTest(t *testing.T, insecure bool) {
defer cleanup()

s, conn, _ := serverutils.StartServer(t,
base.TestServerArgs{Insecure: insecure, SocketFile: maybeSocketFile})
base.TestServerArgs{
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(107310),
Insecure: insecure,
SocketFile: maybeSocketFile,
})
defer s.Stopper().Stop(context.Background())

// Enable conn/auth logging.
// We can't use the cluster settings to do this, because
// cluster settings propagate asynchronously.
testServer := s.(*server.TestServer)
pgServer := testServer.PGServer().(*pgwire.Server)
pgServer := s.TenantOrServer().PGServer().(*pgwire.Server)
pgServer.TestingEnableConnLogging()
pgServer.TestingEnableAuthLogging()
testServer.PGPreServer().(*pgwire.PreServeConnHandler).TestingAcceptSystemIdentityOption(true)
s.TenantOrServer().PGPreServer().(*pgwire.PreServeConnHandler).TestingAcceptSystemIdentityOption(true)

httpClient, err := s.GetAdminHTTPClient()
if err != nil {
Expand Down Expand Up @@ -602,6 +606,13 @@ func fmtErr(err error) string {
}
if pqErr.Hint != "" {
hint := strings.Replace(pqErr.Hint, stdstrings.IssueReferral, "<STANDARD REFERRAL>", 1)
if strings.Contains(hint, "Supported methods:") {
// Depending on whether the test is running on linux or not
// (or, more specifically, whether gss build tag is set),
// "gss" method might not be included, so we remove it here
// and not include into the expected output.
hint = strings.Replace(hint, "gss, ", "", 1)
}
errStr += "\nHINT: " + hint
}
if pqErr.Detail != "" {
Expand Down Expand Up @@ -640,10 +651,9 @@ func TestClientAddrOverride(t *testing.T) {
// Enable conn/auth logging.
// We can't use the cluster settings to do this, because
// cluster settings for booleans propagate asynchronously.
testServer := s.(*server.TestServer)
pgServer := testServer.PGServer().(*pgwire.Server)
pgServer := s.TenantOrServer().PGServer().(*pgwire.Server)
pgServer.TestingEnableAuthLogging()
pgPreServer := testServer.PGPreServer().(*pgwire.PreServeConnHandler)
pgPreServer := s.TenantOrServer().PGPreServer().(*pgwire.PreServeConnHandler)

testCases := []struct {
specialAddr string
Expand Down Expand Up @@ -783,7 +793,9 @@ func TestSSLSessionVar(t *testing.T) {
defer sc.Close(t)

// Start a server.
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(107310),
})
s.(*server.TestServer).Cfg.AcceptSQLWithoutTLS = true
ctx := context.Background()
defer s.Stopper().Stop(ctx)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2037,7 +2037,7 @@ func TestPGWireRejectsNewConnIfTooManyConns(t *testing.T) {
}

getConnectionCount := func() int {
return int(testServer.SQLServer().(*sql.Server).GetConnectionCount())
return int(testServer.TenantOrServer().SQLServer().(*sql.Server).GetConnectionCount())
}

requireConnectionCount := func(t *testing.T, expectedCount int) {
Expand Down
59 changes: 28 additions & 31 deletions pkg/sql/pgwire/pgwire_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,10 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -71,13 +71,13 @@ func trivialQuery(pgURL url.URL) error {
func TestPGWireDrainClient(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
params := base.TestServerArgs{Insecure: true}
s, _, _ := serverutils.StartServer(t, params)

srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{Insecure: true})
ctx := context.Background()
defer s.Stopper().Stop(ctx)
defer srv.Stopper().Stop(ctx)
tt := srv.TenantOrServer()

host, port, err := net.SplitHostPort(s.ServingSQLAddr())
host, port, err := net.SplitHostPort(srv.ServingSQLAddr())
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -106,7 +106,7 @@ func TestPGWireDrainClient(t *testing.T) {
go func() {
defer close(errChan)
errChan <- func() error {
return s.DrainClients(ctx)
return tt.DrainClients(ctx)
}()
}()

Expand All @@ -133,7 +133,7 @@ func TestPGWireDrainClient(t *testing.T) {
}
}

if !s.(*server.TestServer).PGServer().(*pgwire.Server).IsDraining() {
if !tt.PGServer().(*pgwire.Server).IsDraining() {
t.Fatal("server should be draining, but is not")
}
}
Expand All @@ -143,8 +143,8 @@ func TestPGWireDrainClient(t *testing.T) {
func TestPGWireDrainOngoingTxns(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
params := base.TestServerArgs{Insecure: true}
s, _, _ := serverutils.StartServer(t, params)

s, _, _ := serverutils.StartServer(t, base.TestServerArgs{Insecure: true})
ctx := context.Background()
defer s.Stopper().Stop(ctx)

Expand All @@ -166,7 +166,7 @@ func TestPGWireDrainOngoingTxns(t *testing.T) {
}
defer db.Close()

pgServer := s.(*server.TestServer).PGServer().(*pgwire.Server)
pgServer := s.TenantOrServer().PGServer().(*pgwire.Server)

// Make sure that the server reports correctly the case in which a
// connection did not respond to cancellation in time.
Expand Down Expand Up @@ -568,9 +568,6 @@ func TestPGPreparedQuery(t *testing.T) {
Results("users", "users_user_id_idx", false, 1, "user_id", "user_id", "ASC", false, false, true, 1).
Results("users", "users_user_id_idx", false, 2, "username", "username", "ASC", true, true, true, 1),
}},
{"SHOW TABLES FROM system", []preparedQueryTest{
baseTest.Results("public", "comments", "table", "node", 0, gosql.NullString{}).Others(bootstrap.NumSystemTablesForSystemTenant - 1),
}},
{"SHOW SCHEMAS FROM system", []preparedQueryTest{
baseTest.Results("crdb_internal", gosql.NullString{}).Others(4),
}},
Expand Down Expand Up @@ -822,15 +819,12 @@ func TestPGPreparedQuery(t *testing.T) {
{"EXPERIMENTAL SCRUB TABLE system.locations", []preparedQueryTest{
baseTest.SetArgs(),
}},
{"ALTER RANGE liveness CONFIGURE ZONE = $1", []preparedQueryTest{
{"ALTER RANGE default CONFIGURE ZONE = $1", []preparedQueryTest{
baseTest.SetArgs("num_replicas: 1"),
}},
{"ALTER RANGE liveness CONFIGURE ZONE USING num_replicas = $1", []preparedQueryTest{
{"ALTER RANGE default CONFIGURE ZONE USING num_replicas = $1", []preparedQueryTest{
baseTest.SetArgs(1),
}},
{"ALTER RANGE liveness CONFIGURE ZONE = $1", []preparedQueryTest{
baseTest.SetArgs(gosql.NullString{}),
}},
{"TRUNCATE TABLE d.str", []preparedQueryTest{
baseTest.SetArgs(),
}},
Expand All @@ -851,8 +845,10 @@ func TestPGPreparedQuery(t *testing.T) {
// }},
}

ctx := context.Background()
s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
defer s.Stopper().Stop(ctx)
sql.SecondaryTenantZoneConfigsEnabled.Override(ctx, &s.TenantOrServer().ClusterSettings().SV, true)

pgURL, cleanupFn := sqlutils.PGUrl(t, s.ServingSQLAddr(), t.Name(), url.User(username.RootUser))
defer cleanupFn()
Expand Down Expand Up @@ -1600,14 +1596,14 @@ func TestPGCommandTags(t *testing.T) {
// checkSQLNetworkMetrics returns the server's pgwire bytesIn/bytesOut and an
// error if the bytesIn/bytesOut don't satisfy the given minimums and maximums.
func checkSQLNetworkMetrics(
s serverutils.TestServerInterface, minBytesIn, minBytesOut, maxBytesIn, maxBytesOut int64,
srv serverutils.TestServerInterface, minBytesIn, minBytesOut, maxBytesIn, maxBytesOut int64,
) (int64, int64, error) {
if err := s.WriteSummaries(); err != nil {
if err := srv.WriteSummaries(); err != nil {
return -1, -1, err
}

bytesIn := s.MustGetSQLNetworkCounter(pgwire.MetaBytesIn.Name)
bytesOut := s.MustGetSQLNetworkCounter(pgwire.MetaBytesOut.Name)
bytesIn := srv.TenantOrServer().MustGetSQLNetworkCounter(pgwire.MetaBytesIn.Name)
bytesOut := srv.TenantOrServer().MustGetSQLNetworkCounter(pgwire.MetaBytesOut.Name)
if a, min := bytesIn, minBytesIn; a < min {
return bytesIn, bytesOut, errors.Errorf("bytesin %d < expected min %d", a, min)
}
Expand All @@ -1626,28 +1622,29 @@ func checkSQLNetworkMetrics(
func TestSQLNetworkMetrics(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
defer ccl.TestingEnableEnterprise()()

s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(context.Background())

// Setup pgwire client.
pgURL, cleanupFn := sqlutils.PGUrl(
t, s.ServingSQLAddr(), t.Name(), url.User(username.RootUser))
t, srv.ServingSQLAddr(), t.Name(), url.User(username.RootUser))
defer cleanupFn()

const minbytes = 10
const maxbytes = 2 * 1024

// Make sure we're starting at 0.
if _, _, err := checkSQLNetworkMetrics(s, 0, 0, 0, 0); err != nil {
if _, _, err := checkSQLNetworkMetrics(srv, 0, 0, 0, 0); err != nil {
t.Fatal(err)
}

// A single query should give us some I/O.
if err := trivialQuery(pgURL); err != nil {
t.Fatal(err)
}
bytesIn, bytesOut, err := checkSQLNetworkMetrics(s, minbytes, minbytes, maxbytes, maxbytes)
bytesIn, bytesOut, err := checkSQLNetworkMetrics(srv, minbytes, minbytes, maxbytes, maxbytes)
if err != nil {
t.Fatal(err)
}
Expand All @@ -1656,15 +1653,15 @@ func TestSQLNetworkMetrics(t *testing.T) {
}

// A second query should give us more I/O.
_, _, err = checkSQLNetworkMetrics(s, bytesIn+minbytes, bytesOut+minbytes, maxbytes, maxbytes)
_, _, err = checkSQLNetworkMetrics(srv, bytesIn+minbytes, bytesOut+minbytes, maxbytes, maxbytes)
if err != nil {
t.Fatal(err)
}

// Verify connection counter.
expectConns := func(n int) {
testutils.SucceedsSoon(t, func() error {
if conns := s.MustGetSQLNetworkCounter(pgwire.MetaConns.Name); conns != int64(n) {
if conns := srv.TenantOrServer().MustGetSQLNetworkCounter(pgwire.MetaConns.Name); conns != int64(n) {
return errors.Errorf("connections %d != expected %d", conns, n)
}
return nil
Expand Down
7 changes: 0 additions & 7 deletions pkg/testutils/serverutils/test_server_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,13 +183,6 @@ type TestServerInterface interface {
// with DistSQL at the same time.
SetDistSQLSpanResolver(spanResolver interface{})

// MustGetSQLCounter returns the value of a counter metric from the server's
// SQL Executor. Runs in O(# of metrics) time, which is fine for test code.
MustGetSQLCounter(name string) int64
// MustGetSQLNetworkCounter returns the value of a counter metric from the
// server's SQL server. Runs in O(# of metrics) time, which is fine for test
// code.
MustGetSQLNetworkCounter(name string) int64
// WriteSummaries records summaries of time-series data, which is required for
// any tests that query server stats.
WriteSummaries() error
Expand Down
6 changes: 5 additions & 1 deletion pkg/testutils/serverutils/test_tenant_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,9 +226,13 @@ type TestTenantInterface interface {
// SystemConfigProvider provides access to the system config.
SystemConfigProvider() config.SystemConfigProvider

// MustGetSQLCounter returns the value of a counter metric from the server's
// MustGetSQLCounter returns the value of a counter metric from the tenant's
// SQL Executor. Runs in O(# of metrics) time, which is fine for test code.
MustGetSQLCounter(name string) int64
// MustGetSQLNetworkCounter returns the value of a counter metric from the
// tenant's SQL server. Runs in O(# of metrics) time, which is fine for test
// code.
MustGetSQLNetworkCounter(name string) int64

// Codec returns this tenant's codec (or keys.SystemSQLCodec if this is the
// system tenant).
Expand Down

0 comments on commit e3b74b0

Please sign in to comment.