Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96892: cacheutil: fix computation of entry size r=ZhouXing19 a=rafiss

fixes #89444

The previous way would always result in a fixed size for each entry, since it would just compute the size of a pointer. Now, the size of the entry must be specified when writing to the cache.

Release note: None

96899: builtins: crdb_internal.hide_sql_constants should not error on parse r=xinhaoz a=xinhaoz

Previously, if crdb_internal.hide_sql_constants is unable to parse the provided string as a sql statement, it returned an error. Instead, we should just return the empty string to make it less disruptive.

Epic: none
Fixes: #96555

Release note: None

96958: roachtest: rename SQLExperience -> SQLSessions r=jordanlewis a=rafiss

Epic: None
Release note: None

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
  • Loading branch information
3 people committed Feb 10, 2023
4 parents ad11b6b + a854ca3 + 43374b7 + 972ac84 commit 5f310bc
Show file tree
Hide file tree
Showing 31 changed files with 84 additions and 53 deletions.
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/registry/owners.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ type Owner string

// The allowable values of Owner.
const (
OwnerSQLExperience Owner = `sql-sessions`
OwnerSQLSessions Owner = `sql-sessions`
OwnerDisasterRecovery Owner = `disaster-recovery`
OwnerCDC Owner = `cdc`
OwnerKV Owner = `kv`
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/activerecord.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,7 @@ func registerActiveRecord(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "activerecord",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
NativeLibs: registry.LibGEOS,
Tags: []string{`default`, `orm`},
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/asyncpg.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ func registerAsyncpg(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "asyncpg",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1, spec.CPU(16)),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/awsdms.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func dmsDescribeTasksInput(
func registerAWSDMS(r registry.Registry) {
r.Add(registry.TestSpec{
Name: "awsdms",
Owner: registry.OwnerSQLExperience, // TODO(otan): add a migrations OWNERS team
Owner: registry.OwnerSQLSessions, // TODO(otan): add a migrations OWNERS team
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `awsdms`},
Run: runAWSDMS,
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/roachtest/tests/connection_latency.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func registerConnectionLatencyTest(r registry.Registry) {
numNodes := 3
r.Add(registry.TestSpec{
Name: fmt.Sprintf("connection_latency/nodes=%d/certs", numNodes),
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
// Add one more node for load node.
Cluster: r.MakeClusterSpec(numNodes+1, spec.Zones(regionUsCentral)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand All @@ -135,7 +135,7 @@ func registerConnectionLatencyTest(r registry.Registry) {

r.Add(registry.TestSpec{
Name: fmt.Sprintf("connection_latency/nodes=%d/multiregion/certs", numMultiRegionNodes),
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(numMultiRegionNodes+loadNodes, spec.Geo(), spec.Zones(geoZonesStr)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runConnectionLatencyTest(ctx, t, c, numMultiRegionNodes, numZones, false /*password*/)
Expand All @@ -144,7 +144,7 @@ func registerConnectionLatencyTest(r registry.Registry) {

r.Add(registry.TestSpec{
Name: fmt.Sprintf("connection_latency/nodes=%d/multiregion/password", numMultiRegionNodes),
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(numMultiRegionNodes+loadNodes, spec.Geo(), spec.Zones(geoZonesStr)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runConnectionLatencyTest(ctx, t, c, numMultiRegionNodes, numZones, true /*password*/)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/django.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ func registerDjango(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "django",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1, spec.CPU(16)),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/drain.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func registerDrain(r registry.Registry) {
{
r.Add(registry.TestSpec{
Name: "drain/early-exit-conn-wait",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runEarlyExitInConnectionWait(ctx, t, c)
Expand All @@ -41,7 +41,7 @@ func registerDrain(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "drain/warn-conn-wait-timeout",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runTestWarningForConnWait(ctx, t, c)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/flowable.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func registerFlowable(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "flowable",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runFlowable(ctx, t, c)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/gopg.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ func registerGopg(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "gopg",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/gorm.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ func registerGORM(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "gorm",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: runGORM,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/hibernate.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ func registerHibernate(r registry.Registry, opt hibernateOptions) {

r.Add(registry.TestSpec{
Name: opt.testName,
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
NativeLibs: registry.LibGEOS,
Tags: []string{`default`, `orm`},
Expand Down
14 changes: 7 additions & 7 deletions pkg/cmd/roachtest/tests/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ func registerImportNodeShutdown(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "import/nodeShutdown/worker",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(4),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
Expand All @@ -99,7 +99,7 @@ func registerImportNodeShutdown(r registry.Registry) {
})
r.Add(registry.TestSpec{
Name: "import/nodeShutdown/coordinator",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(4),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
Expand Down Expand Up @@ -162,7 +162,7 @@ func registerImportTPCC(r registry.Registry) {
timeout := 5 * time.Hour
r.Add(registry.TestSpec{
Name: testName,
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(numNodes),
Timeout: timeout,
EncryptionSupport: registry.EncryptionMetamorphic,
Expand All @@ -175,7 +175,7 @@ func registerImportTPCC(r registry.Registry) {
const geoZones = "europe-west2-b,europe-west4-b,asia-northeast1-b,us-west1-b"
r.Add(registry.TestSpec{
Name: fmt.Sprintf("import/tpcc/warehouses=%d/geo", geoWarehouses),
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(8, spec.CPU(16), spec.Geo(), spec.Zones(geoZones)),
Timeout: 5 * time.Hour,
EncryptionSupport: registry.EncryptionMetamorphic,
Expand Down Expand Up @@ -207,7 +207,7 @@ func registerImportTPCH(r registry.Registry) {
item := item
r.Add(registry.TestSpec{
Name: fmt.Sprintf(`import/tpch/nodes=%d`, item.nodes),
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(item.nodes),
Timeout: item.timeout,
EncryptionSupport: registry.EncryptionMetamorphic,
Expand Down Expand Up @@ -347,7 +347,7 @@ func runImportMixedVersion(
func registerImportMixedVersion(r registry.Registry) {
r.Add(registry.TestSpec{
Name: "import/mixed-versions",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
// Mixed-version support was added in 21.1.
Cluster: r.MakeClusterSpec(4),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down Expand Up @@ -394,7 +394,7 @@ func registerImportDecommissioned(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "import/decommissioned",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(4),
Run: runImportDecommissioned,
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/jasyncsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ func registerJasyncSQL(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "jasync",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/knex.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ func registerKnex(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "knex",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
NativeLibs: registry.LibGEOS,
Tags: []string{`default`, `orm`},
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/libpq.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ func registerLibPQ(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "lib/pq",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: runLibPQ,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/liquibase.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ func registerLiquibase(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "liquibase",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `tool`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/nodejs_postgres.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ PGSSLCERT=$HOME/certs/client.%s.crt PGSSLKEY=$HOME/certs/client.%s.key PGSSLROOT

r.Add(registry.TestSpec{
Name: "node-postgres",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/pgjdbc.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ func registerPgjdbc(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "pgjdbc",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/pgx.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ func registerPgx(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "pgx",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/pop.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ func registerPop(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "pop",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: runPop,
Expand Down
8 changes: 4 additions & 4 deletions pkg/cmd/roachtest/tests/psycopg.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,15 +81,15 @@ func registerPsycopg(r registry.Registry) {

// TODO(rafi): When psycopg 2.9.4 is released and tagged,
// use the tag version instead of the commit.
//if err := repeatGitCloneE(
// if err := repeatGitCloneE(
// ctx, t, c,
// "https://github.com/psycopg/psycopg2.git",
// "/mnt/data1/psycopg",
// supportedPsycopgTag,
// node,
//); err != nil {
// ); err != nil {
// t.Fatal(err)
//}
// }
if err = c.RunE(ctx, node, "git clone https://github.com/psycopg/psycopg2.git /mnt/data1/psycopg"); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -148,7 +148,7 @@ func registerPsycopg(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "psycopg",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/ruby_pg.go
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ func registerRubyPG(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "ruby-pg",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
NativeLibs: registry.LibGEOS,
Tags: []string{`default`, `orm`},
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/rust_postgres.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ func registerRustPostgres(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "rust-postgres",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1, spec.CPU(16)),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/sequelize.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ func registerSequelize(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "sequelize",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
NativeLibs: registry.LibGEOS,
Tags: []string{`default`, `orm`},
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/sqlalchemy.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ var supportedSQLAlchemyTag = "2.0.2"
func registerSQLAlchemy(r registry.Registry) {
r.Add(registry.TestSpec{
Name: "sqlalchemy",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/typeorm.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func registerTypeORM(r registry.Registry) {

r.Add(registry.TestSpec{
Name: "typeorm",
Owner: registry.OwnerSQLExperience,
Owner: registry.OwnerSQLSessions,
Cluster: r.MakeClusterSpec(1),
Tags: []string{`default`, `orm`},
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
14 changes: 8 additions & 6 deletions pkg/sql/cacheutil/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package cacheutil

import (
"context"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -79,12 +78,16 @@ func (c *Cache) LoadValueOutsideOfCacheSingleFlight(
// MaybeWriteBackToCache tries to put the key, value into the
// cache, and returns true if it succeeded. If the underlying system
// tables have been modified since they were read, the cache is not
// updated.
// updated. The entrySize should be the size in bytes of the key and value.
// Note that reading from system tables may give us data from a newer table
// version than the one we pass in here, that is okay since the cache will
// be invalidated upon the next read.
func (c *Cache) MaybeWriteBackToCache(
ctx context.Context, tableVersions []descpb.DescriptorVersion, key interface{}, entry interface{},
ctx context.Context,
tableVersions []descpb.DescriptorVersion,
key interface{},
value interface{},
entrySize int64,
) bool {
c.Lock()
defer c.Unlock()
Expand All @@ -98,14 +101,13 @@ func (c *Cache) MaybeWriteBackToCache(
}
}
// Table version remains the same: update map, unlock, return.
const sizeOfEntry = int(unsafe.Sizeof(entry))
if err := c.boundAccount.Grow(ctx, int64(sizeOfEntry)); err != nil {
if err := c.boundAccount.Grow(ctx, entrySize); err != nil {
// If there is no memory available to cache the entry, we can still
// proceed with authentication so that users are not locked out of
// the database.
log.Ops.Warningf(ctx, "no memory available to cache info: %v", err)
} else {
c.cache[key] = entry
c.cache[key] = value
}
return true
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/cacheutil/cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,13 +66,13 @@ func TestCache(t *testing.T) {
}()
}

wrote := cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{2, 2}, "test", "val")
wrote := cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{2, 2}, "test", "val", int64(len("test")+len("val")))
require.Equal(t, wrote, true)

wrote = cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{0, 2}, "test", "val")
wrote = cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{0, 2}, "test", "val", int64(len("test")+len("val")))
require.Equal(t, wrote, false)

wrote = cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{2, 0}, "test", "val")
wrote = cache.MaybeWriteBackToCache(ctx, []descpb.DescriptorVersion{2, 0}, "test", "val", int64(len("test")+len("val")))
require.Equal(t, wrote, false)

val, ok := cache.GetValueLocked("test")
Expand Down
Loading

0 comments on commit 5f310bc

Please sign in to comment.