diff --git a/pkg/bench/bench_test.go b/pkg/bench/bench_test.go index 37885b7e4ff1..e51bc84db62e 100644 --- a/pkg/bench/bench_test.go +++ b/pkg/bench/bench_test.go @@ -304,6 +304,36 @@ func runBenchmarkInsert(b *testing.B, db *sqlutils.SQLRunner, count int) { } +// runBenchmarkInsertLarge benchmarks inserting count large rows into a table +// where large means rows with a 1k string and 1k BYTES object. +func runBenchmarkInsertLarge(b *testing.B, db *sqlutils.SQLRunner, count int) { + defer func() { + db.Exec(b, `DROP TABLE IF EXISTS bench.insert`) + }() + + db.Exec(b, `CREATE TABLE bench.insert (k INT PRIMARY KEY, s STRING, b BYTES)`) + bigstr := strings.Repeat("x", 1<<10) + bigbytes := bytes.Repeat([]byte("x"), 1<<10) + + b.ResetTimer() + var buf bytes.Buffer + val := 0 + for i := 0; i < b.N; i++ { + buf.Reset() + buf.WriteString(`INSERT INTO bench.insert VALUES `) + for j := 0; j < count; j++ { + if j > 0 { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, "(%d, '%s', '%s')", val, bigstr, bigbytes) + val++ + } + db.Exec(b, buf.String()) + } + b.StopTimer() + +} + // runBenchmarkInsertFK benchmarks inserting count rows into a table with a // present foreign key into another table. func runBenchmarkInsertFK(b *testing.B, db *sqlutils.SQLRunner, count int) { @@ -419,6 +449,7 @@ func BenchmarkSQL(b *testing.B) { for _, runFn := range []func(*testing.B, *sqlutils.SQLRunner, int){ runBenchmarkDelete, runBenchmarkInsert, + runBenchmarkInsertLarge, runBenchmarkInsertDistinct, runBenchmarkInsertFK, runBenchmarkInsertSecondaryIndex, diff --git a/pkg/build/version.txt b/pkg/build/version.txt index 4b09e6931b3d..0236629661a7 100644 --- a/pkg/build/version.txt +++ b/pkg/build/version.txt @@ -1 +1 @@ -v23.1.0-alpha.8 +v23.2.0-alpha.00000000 diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index f0736e72158f..67e3d2c00ca4 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -369,7 +369,7 @@ go_test( "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvpb", "//pkg/kv/kvserver", - "//pkg/kv/kvserver/allocator/storepool", + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/loqrecovery", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb", diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 43b6a72fccd1..20068e5025e7 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -30,7 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" @@ -496,7 +496,7 @@ func TestPartialZip(t *testing.T) { // is no risk to see the override bumped due to a gossip update // because this setting is not otherwise set in the test cluster. s := tc.Server(0) - storepool.TimeUntilStoreDead.Override(ctx, &s.ClusterSettings().SV, storepool.TestTimeUntilStoreDead) + liveness.TimeUntilStoreDead.Override(ctx, &s.ClusterSettings().SV, liveness.TestTimeUntilStoreDead) // This last case may take a little while to converge. To make this work with datadriven and at the same // time retain the ability to use the `-rewrite` flag, we use a retry loop within that already checks the diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index 1e3e96b7f29b..053f98634661 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -138,32 +138,6 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { return c.RunE(ctx, option.NodeListOption{randomNode}, runCmd.String()) }, ) - mvt.AfterUpgradeFinalized( - "check if GC TTL is pinned", - func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { - // TODO(irfansharif): This can be removed when the predecessor version - // in this test is v23.1, where the default is 4h. This test was only to - // make sure that existing clusters that upgrade to 23.1 retained their - // existing GC TTL. - l.Printf("checking if GC TTL is pinned to 24h") - var ttlSeconds int - query := ` - SELECT - (crdb_internal.pb_to_json('cockroach.config.zonepb.ZoneConfig', raw_config_protobuf)->'gc'->'ttlSeconds')::INT - FROM crdb_internal.zones - WHERE target = 'RANGE default' - LIMIT 1 -` - if err := h.QueryRow(rng, query).Scan(&ttlSeconds); err != nil { - return fmt.Errorf("error querying GC TTL: %w", err) - } - expectedTTL := 24 * 60 * 60 // NB: 24h is what's used in the fixture - if ttlSeconds != expectedTTL { - return fmt.Errorf("unexpected GC TTL: actual (%d) != expected (%d)", ttlSeconds, expectedTTL) - } - return nil - }, - ) mvt.Run() } diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel index 1cc268247690..3d7136d60b34 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel +++ b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/kv/kvserver/allocator/load", "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/constraint", + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/raftutil", "//pkg/roachpb", @@ -52,6 +53,7 @@ go_test( "//pkg/kv/kvserver/allocator/load", "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/constraint", + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/replicastats", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index 6cf33fa4b197..67ab9270d261 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/load" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/replicastats" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -2330,7 +2331,7 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, storePool, nl := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator(st, true /* deterministic */, func(id roachpb.NodeID) (time.Duration, bool) { @@ -2719,7 +2720,7 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, storePool, nl := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator(st, true /* deterministic */, func(id roachpb.NodeID) (time.Duration, bool) { @@ -2787,7 +2788,7 @@ func TestAllocatorShouldTransferSuspected(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, clock, storePool, nl := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator(st, true /* deterministic */, func(id roachpb.NodeID) (time.Duration, bool) { @@ -5575,7 +5576,7 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, storePool, _ := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) defer stopper.Stop(ctx) @@ -7721,7 +7722,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, _, _, sp, _ := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, false, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, false, /* deterministic */ func() int { return numNodes }, livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator(st, false /* deterministic */, func(id roachpb.NodeID) (time.Duration, bool) { @@ -8445,7 +8446,7 @@ func TestAllocatorFullDisks(t *testing.T) { g := gossip.NewTest(1, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) - storepool.TimeUntilStoreDead.Override(ctx, &st.SV, storepool.TestTimeUntilStoreDeadOff) + liveness.TimeUntilStoreDead.Override(ctx, &st.SV, liveness.TestTimeUntilStoreDeadOff) const generations = 100 const nodes = 20 @@ -8903,7 +8904,7 @@ func exampleRebalancing( // adding / rebalancing ranges of random sizes. g := gossip.NewTest(1, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) - storepool.TimeUntilStoreDead.Override(ctx, &st.SV, storepool.TestTimeUntilStoreDeadOff) + liveness.TimeUntilStoreDead.Override(ctx, &st.SV, liveness.TestTimeUntilStoreDeadOff) const nodes = 20 diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/test_helpers.go b/pkg/kv/kvserver/allocator/allocatorimpl/test_helpers.go index 9532689d045f..2a103815650c 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/test_helpers.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/test_helpers.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -40,7 +41,7 @@ func CreateTestAllocatorWithKnobs( ) (*stop.Stopper, *gossip.Gossip, *storepool.StorePool, Allocator, *timeutil.ManualTime) { st := cluster.MakeTestingClusterSettings() stopper, g, manual, storePool, _ := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDeadOff, deterministic, + liveness.TestTimeUntilStoreDeadOff, deterministic, func() int { return numNodes }, livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator(st, deterministic, func(id roachpb.NodeID) (time.Duration, bool) { diff --git a/pkg/kv/kvserver/allocator/storepool/BUILD.bazel b/pkg/kv/kvserver/allocator/storepool/BUILD.bazel index cac55e29e712..20701d590024 100644 --- a/pkg/kv/kvserver/allocator/storepool/BUILD.bazel +++ b/pkg/kv/kvserver/allocator/storepool/BUILD.bazel @@ -41,6 +41,7 @@ go_test( args = ["-test.timeout=295s"], embed = [":storepool"], deps = [ + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go b/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go index 3876afa3149a..adf441258d3e 100644 --- a/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go +++ b/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go @@ -16,6 +16,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -36,7 +37,7 @@ func TestOverrideStorePoolStatusString(t *testing.T) { const nodeCount = 5 stopper, g, _, testStorePool, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return nodeCount }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -117,7 +118,7 @@ func TestOverrideStorePoolDecommissioningReplicas(t *testing.T) { const nodeCount = 5 stopper, g, _, testStorePool, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return nodeCount }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -234,7 +235,7 @@ func TestOverrideStorePoolGetStoreList(t *testing.T) { // We're going to manually mark stores dead in this test. stopper, g, _, testStorePool, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return nodeCount }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) diff --git a/pkg/kv/kvserver/allocator/storepool/store_pool.go b/pkg/kv/kvserver/allocator/storepool/store_pool.go index 6098dd8eeda3..4e80bd05495c 100644 --- a/pkg/kv/kvserver/allocator/storepool/store_pool.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool.go @@ -33,16 +33,6 @@ import ( "github.com/cockroachdb/errors" ) -const ( - // TestTimeUntilStoreDead is the test value for TimeUntilStoreDead to - // quickly mark stores as dead. - TestTimeUntilStoreDead = 5 * time.Millisecond - - // TestTimeUntilStoreDeadOff is the test value for TimeUntilStoreDead that - // prevents the store pool from marking stores as dead. - TestTimeUntilStoreDeadOff = 24 * time.Hour -) - // FailedReservationsTimeout specifies a duration during which the local // replicate queue will not consider stores which have failed a reservation a // viable target. @@ -78,32 +68,6 @@ var TimeAfterStoreSuspect = settings.RegisterDurationSetting( }, ) -const timeUntilStoreDeadSettingName = "server.time_until_store_dead" - -// TimeUntilStoreDead wraps "server.time_until_store_dead". -var TimeUntilStoreDead = func() *settings.DurationSetting { - s := settings.RegisterDurationSetting( - settings.TenantWritable, - timeUntilStoreDeadSettingName, - "the time after which if there is no new gossiped information about a store, it is considered dead", - 5*time.Minute, - func(v time.Duration) error { - // Setting this to less than the interval for gossiping stores is a big - // no-no, since this value is compared to the age of the most recent gossip - // from each store to determine whether that store is live. Put a buffer of - // 15 seconds on top to allow time for gossip to propagate. - const minTimeUntilStoreDead = gossip.StoresInterval + 15*time.Second - if v < minTimeUntilStoreDead { - return errors.Errorf("cannot set %s to less than %v: %v", - timeUntilStoreDeadSettingName, minTimeUntilStoreDead, v) - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() - // The NodeCountFunc returns a count of the total number of nodes the user // intends for their to be in the cluster. The count includes dead nodes, but // not decommissioned nodes. @@ -527,7 +491,7 @@ func (sp *StorePool) statusString(nl NodeLivenessFunc) string { var buf bytes.Buffer now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) for _, id := range ids { @@ -820,7 +784,7 @@ func (sp *StorePool) decommissioningReplicasWithLiveness( // NB: We use clock.Now() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) for _, repl := range repls { @@ -864,7 +828,7 @@ func (sp *StorePool) IsDead(storeID roachpb.StoreID) (bool, time.Duration, error // NB: We use clock.Now() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) deadAsOf := sd.LastUpdatedTime.AddDuration(timeUntilStoreDead) if now.After(deadAsOf) { @@ -940,7 +904,7 @@ func (sp *StorePool) storeStatus( // NB: We use clock.Now() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) return sd.status(now, timeUntilStoreDead, nl, timeAfterStoreSuspect), nil } @@ -974,7 +938,7 @@ func (sp *StorePool) liveAndDeadReplicasWithLiveness( defer sp.DetailsMu.Unlock() now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) for _, repl := range repls { @@ -1250,7 +1214,7 @@ func (sp *StorePool) getStoreListFromIDsLocked( var storeDescriptors []roachpb.StoreDescriptor now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) for _, storeID := range storeIDs { diff --git a/pkg/kv/kvserver/allocator/storepool/store_pool_test.go b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go index b5ab8cd5bfc7..661901fb5b89 100644 --- a/pkg/kv/kvserver/allocator/storepool/store_pool_test.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -54,7 +55,7 @@ func TestStorePoolGossipUpdate(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, _ := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 0 }, /* NodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -124,7 +125,7 @@ func TestStorePoolGetStoreList(t *testing.T) { st := cluster.MakeTestingClusterSettings() // We're going to manually mark stores dead in this test. stopper, g, _, sp, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -420,7 +421,7 @@ func TestStorePoolGetStoreDetails(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, _ := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -443,7 +444,7 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -549,7 +550,7 @@ func TestStorePoolDefaultState(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, _, _, sp, _ := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -580,7 +581,7 @@ func TestStorePoolThrottle(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, _ := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -608,13 +609,13 @@ func TestStorePoolSuspected(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDeadOff, false, /* deterministic */ + liveness.TestTimeUntilStoreDeadOff, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) now := sp.clock.Now() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&sp.st.SV) timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) // Verify a store that we haven't seen yet is unknown status. @@ -691,7 +692,7 @@ func TestGetLocalities(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, _ := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -773,7 +774,7 @@ func TestStorePoolDecommissioningReplicas(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, mnl := CreateTestStorePool(ctx, st, - TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) diff --git a/pkg/kv/kvserver/allocator/storepool/test_helpers.go b/pkg/kv/kvserver/allocator/storepool/test_helpers.go index ede0acb0f52b..d43fb11a7494 100644 --- a/pkg/kv/kvserver/allocator/storepool/test_helpers.go +++ b/pkg/kv/kvserver/allocator/storepool/test_helpers.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -83,7 +84,7 @@ func CreateTestStorePool( g := gossip.NewTest(1, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) mnl := NewMockNodeLiveness(defaultNodeStatus) - TimeUntilStoreDead.Override(ctx, &st.SV, timeUntilStoreDeadValue) + liveness.TimeUntilStoreDead.Override(ctx, &st.SV, timeUntilStoreDeadValue) storePool := NewStorePool( ambientCtx, st, diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index a670564b8871..906202747d6b 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -1003,7 +1003,7 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { } } // We need to wait until 2 and 3 are considered to be dead. - timeUntilStoreDead := storepool.TimeUntilStoreDead.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) + timeUntilStoreDead := liveness.TimeUntilStoreDead.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) wait(timeUntilStoreDead) checkDead := func(store *kvserver.Store, storeIdx int) error { diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 2ac26f013cc3..074c1080aec2 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -215,7 +216,7 @@ func (s *Store) RaftSchedulerPriorityIDs() []roachpb.RangeID { } func NewTestStorePool(cfg StoreConfig) *storepool.StorePool { - storepool.TimeUntilStoreDead.Override(context.Background(), &cfg.Settings.SV, storepool.TestTimeUntilStoreDeadOff) + liveness.TimeUntilStoreDead.Override(context.Background(), &cfg.Settings.SV, liveness.TestTimeUntilStoreDeadOff) return storepool.NewStorePool( cfg.AmbientCtx, cfg.Settings, diff --git a/pkg/kv/kvserver/liveness/BUILD.bazel b/pkg/kv/kvserver/liveness/BUILD.bazel index 3d656ef6ee9b..03aab4a872bc 100644 --- a/pkg/kv/kvserver/liveness/BUILD.bazel +++ b/pkg/kv/kvserver/liveness/BUILD.bazel @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "liveness", srcs = [ + "cache.go", "liveness.go", "storage.go", ], @@ -18,6 +19,7 @@ go_library( "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/server/telemetry", + "//pkg/settings", "//pkg/settings/cluster", "//pkg/storage", "//pkg/util/grpcutil", @@ -49,7 +51,6 @@ go_test( "//pkg/base", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator/plan", - "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/kvserver/liveness/cache.go b/pkg/kv/kvserver/liveness/cache.go new file mode 100644 index 000000000000..5ef7f7a88d52 --- /dev/null +++ b/pkg/kv/kvserver/liveness/cache.go @@ -0,0 +1,220 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package liveness + +import ( + "bytes" + "context" + + "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// cache stores updates to both Liveness records and the store descriptor map. +// It doesn't store the entire StoreDescriptor, only the time when it is +// updated. The StoreDescriptor is sent directly from nodes so doesn't require +// the liveness leaseholder to be available. +// TODO(baptist): Currently liveness does not take into account the store +// descriptor timestamps. Once all code has changed over to not directly +// checking liveness on the liveness record, then the isLive method should +// change to take this into account. Only epoch leases will use the liveness +// timestamp directly. +type cache struct { + gossip *gossip.Gossip + clock *hlc.Clock + notifyLivenessChanged func(old, new livenesspb.Liveness) + mu struct { + syncutil.RWMutex + // lastNodeUpdate stores timestamps of StoreDescriptor updates in Gossip. + // This is tracking based on NodeID, so any store that is updated on this + // node will update teh lastNodeUpdate. We don't have the ability to handle + // "1 stalled store" on a node from a liveness perspective. + lastNodeUpdate map[roachpb.NodeID]hlc.Timestamp + // nodes stores liveness records read from Gossip + nodes map[roachpb.NodeID]Record + } +} + +func newCache( + g *gossip.Gossip, clock *hlc.Clock, cbFn func(livenesspb.Liveness, livenesspb.Liveness), +) *cache { + c := cache{} + c.gossip = g + c.clock = clock + c.mu.nodes = make(map[roachpb.NodeID]Record) + c.mu.lastNodeUpdate = make(map[roachpb.NodeID]hlc.Timestamp) + + c.notifyLivenessChanged = cbFn + + // NB: we should consider moving this registration to .Start() once we + // have ensured that nobody uses the server's KV client (kv.DB) before + // nl.Start() is invoked. At the time of writing this invariant does + // not hold (which is a problem, since the node itself won't be live + // at this point, and requests routed to it will hang). + livenessRegex := gossip.MakePrefixPattern(gossip.KeyNodeLivenessPrefix) + c.gossip.RegisterCallback(livenessRegex, c.livenessGossipUpdate) + + // Enable redundant callbacks for the store keys because we use these + // callbacks as a clock to determine when a store was last updated even if it + // hasn't otherwise changed. + storeRegex := gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix) + c.gossip.RegisterCallback(storeRegex, c.storeGossipUpdate, gossip.Redundant) + return &c +} + +// selfID returns the ID for this node according to Gossip. This will be 0 +// until the node has joined the cluster. +func (c *cache) selfID() roachpb.NodeID { + return c.gossip.NodeID.Get() +} + +// livenessGossipUpdate is the gossip callback used to keep the +// in-memory liveness info up to date. +func (c *cache) livenessGossipUpdate(_ string, content roachpb.Value) { + ctx := context.TODO() + var liveness livenesspb.Liveness + if err := content.GetProto(&liveness); err != nil { + log.Errorf(ctx, "%v", err) + return + } + + c.maybeUpdate(ctx, Record{Liveness: liveness, raw: content.TagAndDataBytes()}) +} + +// storeGossipUpdate is the Gossip callback used to keep the nodeDescMap up to date. +func (c *cache) storeGossipUpdate(_ string, content roachpb.Value) { + ctx := context.TODO() + var storeDesc roachpb.StoreDescriptor + if err := content.GetProto(&storeDesc); err != nil { + log.Errorf(ctx, "%v", err) + return + } + nodeID := storeDesc.Node.NodeID + if nodeID == 0 { + log.Errorf(ctx, "unexpected update for node 0, %v", storeDesc) + return + } + c.mu.Lock() + c.mu.lastNodeUpdate[nodeID] = c.clock.Now() + c.mu.Unlock() +} + +// maybeUpdate replaces the liveness (if it appears newer) and invokes the +// registered callbacks if the node became live in the process. +func (c *cache) maybeUpdate(ctx context.Context, newLivenessRec Record) { + if newLivenessRec.Liveness == (livenesspb.Liveness{}) { + log.Fatal(ctx, "invalid new liveness record; found to be empty") + } + + shouldReplace := true + c.mu.Lock() + + // NB: shouldReplace will always be true right after a node restarts since the + // `nodes` map will be empty. This means that the callbacks called below will + // always be invoked at least once after node restarts. + oldLivenessRec, ok := c.mu.nodes[newLivenessRec.NodeID] + if ok { + shouldReplace = livenessChanged(oldLivenessRec, newLivenessRec) + } + + if shouldReplace { + c.mu.nodes[newLivenessRec.NodeID] = newLivenessRec + } + c.mu.Unlock() + + if shouldReplace { + c.notifyLivenessChanged(oldLivenessRec.Liveness, newLivenessRec.Liveness) + } +} + +// livenessChanged checks to see if the new liveness is in fact newer +// than the old liveness. +func livenessChanged(old, new Record) bool { + oldL, newL := old.Liveness, new.Liveness + + // Compare liveness information. If oldL < newL, replace. + if cmp := oldL.Compare(newL); cmp != 0 { + return cmp < 0 + } + + // If Epoch and Expiration are unchanged, assume that the update is newer + // when its draining or decommissioning field changed. + // + // Similarly, assume that the update is newer if the raw encoding is changed + // when all the fields are the same. This ensures that the CPut performed + // by updateLivenessAttempt will eventually succeed even if the proto + // encoding changes. + // + // This has false positives (in which case we're clobbering the liveness). A + // better way to handle liveness updates in general is to add a sequence + // number. + // + // See #18219. + return oldL.Draining != newL.Draining || + oldL.Membership != newL.Membership || + (oldL.Equal(newL) && !bytes.Equal(old.raw, new.raw)) +} + +// Self returns the raw, encoded value that the database has for this liveness +// record in addition to the decoded liveness proto. +func (c *cache) Self() (_ Record, ok bool) { + c.mu.RLock() + defer c.mu.RUnlock() + return c.getLivenessLocked(c.selfID()) +} + +// GetLiveness returns the liveness record for the specified nodeID. If the +// liveness record is not found (due to gossip propagation delays or due to the +// node not existing), we surface that to the caller. The record returned also +// includes the raw, encoded value that the database has for this liveness +// record in addition to the decoded liveness proto. +func (c *cache) GetLiveness(nodeID roachpb.NodeID) (_ Record, ok bool) { + c.mu.RLock() + defer c.mu.RUnlock() + return c.getLivenessLocked(nodeID) +} + +// getLivenessLocked returns the liveness record for the specified nodeID, +// consulting the in-memory cache. If nothing is found (could happen due to +// gossip propagation delays or the node not existing), we surface that to the +// caller. +func (c *cache) getLivenessLocked(nodeID roachpb.NodeID) (_ Record, ok bool) { + if l, ok := c.mu.nodes[nodeID]; ok { + return l, true + } + return Record{}, false +} + +// GetIsLiveMap returns a map of nodeID to boolean liveness status of +// each node. This excludes nodes that were removed completely (dead + +// decommissioned) +func (c *cache) GetIsLiveMap() livenesspb.IsLiveMap { + lMap := livenesspb.IsLiveMap{} + c.mu.RLock() + defer c.mu.RUnlock() + now := c.clock.Now() + for nID, l := range c.mu.nodes { + isLive := l.IsLive(now) + if l.Membership.Decommissioned() { + // This is a node that was completely removed. Skip over it. + continue + } + lMap[nID] = livenesspb.IsLiveMapEntry{ + Liveness: l.Liveness, + IsLive: isLive, + } + } + return lMap +} diff --git a/pkg/kv/kvserver/liveness/client_test.go b/pkg/kv/kvserver/liveness/client_test.go index 5187653f1891..27cc7192b8b0 100644 --- a/pkg/kv/kvserver/liveness/client_test.go +++ b/pkg/kv/kvserver/liveness/client_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/plan" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -246,7 +245,7 @@ func TestNodeLivenessStatusMap(t *testing.T) { // doesn't allow durations below 1m15s, which is much too long // for a test. // We do this in every SucceedsSoon attempt, so we'll be good. - storepool.TimeUntilStoreDead.Override(ctx, &firstServer.ClusterSettings().SV, storepool.TestTimeUntilStoreDead) + liveness.TimeUntilStoreDead.Override(ctx, &firstServer.ClusterSettings().SV, liveness.TestTimeUntilStoreDead) log.Infof(ctx, "checking expected status (%s) for node %d", expectedStatus, nodeID) resp, err := admin.Liveness(ctx, &serverpb.LivenessRequest{}) diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index f024de0529e6..3afa7007085b 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -11,7 +11,6 @@ package liveness import ( - "bytes" "context" "fmt" "strconv" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" diskStorage "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" @@ -41,6 +41,38 @@ import ( "github.com/cockroachdb/redact" ) +const ( + // TestTimeUntilStoreDead is the test value for TimeUntilStoreDead to + // quickly mark stores as dead. + TestTimeUntilStoreDead = 5 * time.Millisecond + + // TestTimeUntilStoreDeadOff is the test value for TimeUntilStoreDead that + // prevents the store pool from marking stores as dead. + TestTimeUntilStoreDeadOff = 24 * time.Hour + + timeUntilStoreDeadSettingName = "server.time_until_store_dead" +) + +// TimeUntilStoreDead wraps "server.time_until_store_dead". +var TimeUntilStoreDead = settings.RegisterDurationSetting( + settings.TenantWritable, + timeUntilStoreDeadSettingName, + "the time after which if there is no new gossiped information about a store, it is considered dead", + 5*time.Minute, + func(v time.Duration) error { + // Setting this to less than the interval for gossiping stores is a big + // no-no, since this value is compared to the age of the most recent gossip + // from each store to determine whether that store is live. Put a buffer of + // 15 seconds on top to allow time for gossip to propagate. + const minTimeUntilStoreDead = gossip.StoresInterval + 15*time.Second + if v < minTimeUntilStoreDead { + return errors.Errorf("cannot set %s to less than %v: %v", + timeUntilStoreDeadSettingName, minTimeUntilStoreDead, v) + } + return nil + }, +).WithPublic() + var ( // ErrMissingRecord is returned when asking for liveness information // about a node for which nothing is known. This happens when attempting to @@ -206,8 +238,8 @@ type NodeLiveness struct { stopper *stop.Stopper clock *hlc.Clock storage storage - gossip *gossip.Gossip livenessThreshold time.Duration + cache *cache renewalDuration time.Duration selfSem chan struct{} st *cluster.Settings @@ -237,44 +269,6 @@ type NodeLiveness struct { // Set to true once Start is called. RegisterCallback can not be called after // Start is called. started syncutil.AtomicBool - - mu struct { - syncutil.RWMutex - // nodes is an in-memory cache of liveness records that NodeLiveness - // knows about (having learnt of them through gossip or through KV). - // It's a look-aside cache, and is accessed primarily through - // `getLivenessLocked` and callers. - // - // TODO(irfansharif): The caching story for NodeLiveness is a bit - // complicated. This can be attributed to the fact that pre-20.2, we - // weren't always guaranteed for us liveness records for every given - // node. Because of this it wasn't possible to have a - // look-through cache (it wouldn't know where to fetch from if a record - // was found to be missing). - // - // Now that we're always guaranteed to have a liveness records present, - // we should change this out to be a look-through cache instead (it can - // fall back to KV when a given record is missing). This would help - // simplify our current structure where do the following: - // - // - Consult this cache to find an existing liveness record - // - If missing, fetch the record from KV - // - Update the liveness record in KV - // - Add the updated record into this cache (see `maybeUpdate`) - // - // (See `Start` for an example of this pattern.) - // - // What we want instead is a bit simpler: - // - // - Consult this cache to find an existing liveness record - // - If missing, fetch the record from KV, update and return from cache - // - Update the liveness record in KV - // - Add the updated record into this cache - // - // More concretely, we want `getLivenessRecordFromKV` to be tucked away - // within `getLivenessLocked`. - nodes map[roachpb.NodeID]Record - } } // Record is a liveness record that has been read from the database, together @@ -306,6 +300,7 @@ type NodeLivenessOptions struct { // node was permanently removed from the cluster. This method must be // idempotent as it may be invoked multiple times and defaults to a // noop. + // TODO(baptist): Change this to not take the liveness record OnNodeDecommissioned func(livenesspb.Liveness) // OnNodeDecommissioning is invoked when a node is detected to be // decommissioning. @@ -322,7 +317,6 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness { stopper: opts.Stopper, clock: opts.Clock, storage: storage{db: opts.DB}, - gossip: opts.Gossip, livenessThreshold: opts.LivenessThreshold, renewalDuration: opts.RenewalDuration, selfSem: make(chan struct{}, 1), @@ -348,24 +342,16 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness { Buckets: metric.NetworkLatencyBuckets, }), } - nl.mu.nodes = make(map[roachpb.NodeID]Record) + nl.cache = newCache(opts.Gossip, opts.Clock, nl.cacheUpdated) nl.heartbeatToken <- struct{}{} - // NB: we should consider moving this registration to .Start() once we - // have ensured that nobody uses the server's KV client (kv.DB) before - // nl.Start() is invoked. At the time of writing this invariant does - // not hold (which is a problem, since the node itself won't be live - // at this point, and requests routed to it will hang). - livenessRegex := gossip.MakePrefixPattern(gossip.KeyNodeLivenessPrefix) - nl.gossip.RegisterCallback(livenessRegex, nl.livenessGossipUpdate) - return nl } var errNodeDrainingSet = errors.New("node is already draining") func (nl *NodeLiveness) sem(nodeID roachpb.NodeID) chan struct{} { - if nodeID == nl.gossip.NodeID.Get() { + if nodeID == nl.cache.selfID() { return nl.selfSem } return nl.otherSem @@ -382,12 +368,14 @@ func (nl *NodeLiveness) SetDraining( ctx context.Context, drain bool, reporter func(int, redact.SafeString), ) error { ctx = nl.ambientCtx.AnnotateCtx(ctx) - for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { - oldLivenessRec, ok := nl.SelfEx() + retryOpts := base.DefaultRetryOptions() + retryOpts.Closer = nl.stopper.ShouldQuiesce() + for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { + oldLivenessRec, ok := nl.cache.Self() if !ok { // There was a cache miss, let's now fetch the record from KV // directly. - nodeID := nl.gossip.NodeID.Get() + nodeID := nl.cache.selfID() livenessRec, err := nl.getLivenessRecordFromKV(ctx, nodeID) if err != nil { return err @@ -474,8 +462,7 @@ func (nl *NodeLiveness) SetMembershipStatus( func (nl *NodeLiveness) setDrainingInternal( ctx context.Context, oldLivenessRec Record, drain bool, reporter func(int, redact.SafeString), ) error { - nodeID := nl.gossip.NodeID.Get() - sem := nl.sem(nodeID) + sem := nl.selfSem // Allow only one attempt to set the draining field at a time. select { case sem <- struct{}{}: @@ -505,8 +492,10 @@ func (nl *NodeLiveness) setDrainingInternal( newLiveness: newLiveness, oldRaw: oldLivenessRec.raw, } + // TODO(baptist): retry on failure. written, err := nl.updateLiveness(ctx, update, func(actual Record) error { - nl.maybeUpdate(ctx, actual) + // Handle a stale cache by updating with the value we just read. + nl.cache.maybeUpdate(ctx, actual) if actual.Draining == update.newLiveness.Draining { return errNodeDrainingSet @@ -523,10 +512,34 @@ func (nl *NodeLiveness) setDrainingInternal( return err } - nl.maybeUpdate(ctx, written) + nl.cache.maybeUpdate(ctx, written) return nil } +func (nl *NodeLiveness) cacheUpdated(old livenesspb.Liveness, new livenesspb.Liveness) { + // TODO(baptist): This won't work correctly we remove expiration timestamp. + // Need to use a different signal to determine if liveness changed. + now := nl.clock.Now() + if !old.IsLive(now) && new.IsLive(now) { + // NB: If we are not started, we don't use the onIsLive callbacks since they + // can still change. This is a bit of a tangled mess since the startup of + // liveness requires the stores to be started, but stores can't start until + // liveness can run. Ideally we could cache all these updates and call + // onIsLive as part of start. + if nl.started.Get() { + for _, fn := range nl.onIsLive { + fn(new) + } + } + } + if !old.Membership.Decommissioned() && new.Membership.Decommissioned() && nl.onNodeDecommissioned != nil { + nl.onNodeDecommissioned(new) + } + if !old.Membership.Decommissioning() && new.Membership.Decommissioning() && nl.onNodeDecommissioning != nil { + nl.onNodeDecommissioning(new.NodeID) + } +} + // CreateLivenessRecord creates a liveness record for the node specified by the // given node ID. This is typically used when adding a new node to a running // cluster, or when bootstrapping a cluster through a given node. @@ -598,8 +611,7 @@ func (nl *NodeLiveness) IsLive(nodeID roachpb.NodeID) (bool, error) { // we should be able to return ErrMissingRecord instead. return false, ErrRecordCacheMiss } - // NB: We use clock.Now().GoTime() instead of clock.PhysicalTime() in order to - // consider clock signals from other nodes. + // NB: We use clock.Now() in order to consider clock signals from other nodes. return liveness.IsLive(nl.clock.Now()), nil } @@ -651,13 +663,12 @@ func (nl *NodeLiveness) Start(ctx context.Context) { retryOpts.Closer = nl.stopper.ShouldQuiesce() nl.started.Set(true) - now := nl.clock.Now() // We may have received some liveness records from Gossip prior to Start being // called. We need to go through and notify all the callers of them now. - for _, l := range nl.GetLivenesses() { - if l.IsLive(now) { + for _, entry := range nl.GetIsLiveMap() { + if entry.IsLive { for _, fn := range nl.onIsLive { - fn(l) + fn(entry.Liveness) } } } @@ -688,7 +699,7 @@ func (nl *NodeLiveness) Start(ctx context.Context) { for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { oldLiveness, ok := nl.Self() if !ok { - nodeID := nl.gossip.NodeID.Get() + nodeID := nl.cache.selfID() liveness, err := nl.getLivenessRecordFromKV(ctx, nodeID) if err != nil { log.Infof(ctx, "unable to get liveness record from KV: %s", err) @@ -831,8 +842,7 @@ func (nl *NodeLiveness) heartbeatInternal( defer nl.metrics.HeartbeatsInFlight.Dec(1) // Allow only one heartbeat at a time. - nodeID := nl.gossip.NodeID.Get() - sem := nl.sem(nodeID) + sem := nl.selfSem select { case sem <- struct{}{}: case <-ctx.Done(): @@ -886,7 +896,7 @@ func (nl *NodeLiveness) heartbeatInternal( } written, err := nl.updateLiveness(ctx, update, func(actual Record) error { // Update liveness to actual value on mismatch. - nl.maybeUpdate(ctx, actual) + nl.cache.maybeUpdate(ctx, actual) // If the actual liveness is different than expected, but is // considered live, treat the heartbeat as a success. This can @@ -927,7 +937,7 @@ func (nl *NodeLiveness) heartbeatInternal( } log.VEventf(ctx, 1, "heartbeat %+v", written.Expiration) - nl.maybeUpdate(ctx, written) + nl.cache.maybeUpdate(ctx, written) nl.metrics.HeartbeatSuccesses.Inc(1) return nil } @@ -937,54 +947,18 @@ func (nl *NodeLiveness) heartbeatInternal( // liveness record successfully, nor received a gossip message containing // a former liveness update on restart. func (nl *NodeLiveness) Self() (_ livenesspb.Liveness, ok bool) { - rec, ok := nl.SelfEx() + rec, ok := nl.cache.Self() if !ok { return livenesspb.Liveness{}, false } return rec.Liveness, true } -// SelfEx is like Self, but returns the raw, encoded value that the database has -// for this liveness record in addition to the decoded liveness proto. -func (nl *NodeLiveness) SelfEx() (_ Record, ok bool) { - nl.mu.RLock() - defer nl.mu.RUnlock() - return nl.getLivenessLocked(nl.gossip.NodeID.Get()) -} - // GetIsLiveMap returns a map of nodeID to boolean liveness status of // each node. This excludes nodes that were removed completely (dead + // decommissioning). func (nl *NodeLiveness) GetIsLiveMap() livenesspb.IsLiveMap { - lMap := livenesspb.IsLiveMap{} - nl.mu.RLock() - defer nl.mu.RUnlock() - now := nl.clock.Now() - for nID, l := range nl.mu.nodes { - isLive := l.IsLive(now) - if !isLive && !l.Membership.Active() { - // This is a node that was completely removed. Skip over it. - continue - } - lMap[nID] = livenesspb.IsLiveMapEntry{ - Liveness: l.Liveness, - IsLive: isLive, - } - } - return lMap -} - -// GetLivenesses returns a slice containing the liveness status of -// every node on the cluster known to gossip. Callers should consider -// calling (statusServer).NodesWithLiveness() instead where possible. -func (nl *NodeLiveness) GetLivenesses() []livenesspb.Liveness { - nl.mu.RLock() - defer nl.mu.RUnlock() - livenesses := make([]livenesspb.Liveness, 0, len(nl.mu.nodes)) - for _, l := range nl.mu.nodes { - livenesses = append(livenesses, l.Liveness) - } - return livenesses + return nl.cache.GetIsLiveMap() } // GetLivenessesFromKV returns a slice containing the liveness record of all @@ -999,7 +973,7 @@ func (nl *NodeLiveness) GetLivenessesFromKV(ctx context.Context) ([]livenesspb.L livenesses := make([]livenesspb.Liveness, len(records)) for i, r := range records { livenesses[i] = r.Liveness - nl.maybeUpdate(ctx, r) + nl.cache.maybeUpdate(ctx, r) } return livenesses, nil } @@ -1010,20 +984,7 @@ func (nl *NodeLiveness) GetLivenessesFromKV(ctx context.Context) ([]livenesspb.L // includes the raw, encoded value that the database has for this liveness // record in addition to the decoded liveness proto. func (nl *NodeLiveness) GetLiveness(nodeID roachpb.NodeID) (_ Record, ok bool) { - nl.mu.RLock() - defer nl.mu.RUnlock() - return nl.getLivenessLocked(nodeID) -} - -// getLivenessLocked returns the liveness record for the specified nodeID, -// consulting the in-memory cache. If nothing is found (could happen due to -// gossip propagation delays or the node not existing), we surface that to the -// caller. -func (nl *NodeLiveness) getLivenessLocked(nodeID roachpb.NodeID) (_ Record, ok bool) { - if l, ok := nl.mu.nodes[nodeID]; ok { - return l, true - } - return Record{}, false + return nl.cache.GetLiveness(nodeID) } // getLivenessRecordFromKV fetches the liveness record from KV for a given node, @@ -1036,7 +997,7 @@ func (nl *NodeLiveness) getLivenessRecordFromKV( livenessRec, err := nl.storage.get(ctx, nodeID) if err == nil { // Update our cache with the liveness record we just found. - nl.maybeUpdate(ctx, livenessRec) + nl.cache.maybeUpdate(ctx, livenessRec) } return livenessRec, err @@ -1089,7 +1050,7 @@ func (nl *NodeLiveness) IncrementEpoch(ctx context.Context, liveness livenesspb. update.newLiveness.Epoch++ written, err := nl.updateLiveness(ctx, update, func(actual Record) error { - nl.maybeUpdate(ctx, actual) + nl.cache.maybeUpdate(ctx, actual) if actual.Epoch > liveness.Epoch { return ErrEpochAlreadyIncremented @@ -1106,7 +1067,7 @@ func (nl *NodeLiveness) IncrementEpoch(ctx context.Context, liveness livenesspb. } log.Infof(ctx, "incremented n%d liveness epoch to %d", written.NodeID, written.Epoch) - nl.maybeUpdate(ctx, written) + nl.cache.maybeUpdate(ctx, written) nl.metrics.EpochIncrements.Inc() return nil } @@ -1149,7 +1110,9 @@ func (nl *NodeLiveness) updateLiveness( if err := nl.verifyDiskHealth(ctx); err != nil { return Record{}, err } - for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { + retryOpts := base.DefaultRetryOptions() + retryOpts.Closer = nl.stopper.ShouldQuiesce() + for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { written, err := nl.updateLivenessAttempt(ctx, update, handleCondFailed) if err != nil { if errors.HasType(err, (*errRetryLiveness)(nil)) { @@ -1166,7 +1129,7 @@ func (nl *NodeLiveness) updateLiveness( if err := ctx.Err(); err != nil { return Record{}, err } - panic("unreachable; should retry until ctx canceled") + return Record{}, errors.New("retry loop ended without error - likely shutting down") } // verifyDiskHealth does a sync write to all disks before updating liveness, so @@ -1222,103 +1185,6 @@ func (nl *NodeLiveness) updateLivenessAttempt( return nl.storage.update(ctx, update, handleCondFailed) } -// maybeUpdate replaces the liveness (if it appears newer) and invokes the -// registered callbacks if the node became live in the process. -func (nl *NodeLiveness) maybeUpdate(ctx context.Context, newLivenessRec Record) { - - if newLivenessRec.Liveness == (livenesspb.Liveness{}) { - log.Fatal(ctx, "invalid new liveness record; found to be empty") - } - - var shouldReplace bool - nl.mu.Lock() - - // NB: shouldReplace will always be true right after a node restarts since the - // `nodes` map will be empty. This means that the callbacks called below will - // always be invoked at least once after node restarts. - oldLivenessRec, ok := nl.getLivenessLocked(newLivenessRec.NodeID) - if !ok { - shouldReplace = true - } else { - shouldReplace = shouldReplaceLiveness(ctx, oldLivenessRec, newLivenessRec) - } - - var onIsLive []IsLiveCallback - if shouldReplace { - nl.mu.nodes[newLivenessRec.NodeID] = newLivenessRec - // NB: If we are not started, we don't call the onIsLive callbacks since - // they can volatile. This is a bit of a tangled mess since the startup of - // liveness requires the stores to be started, but stores can't start until - // maybeUpdate can run. Ideally we could cache all these updates and - // initialize onIsLive as part of start. - if nl.started.Get() { - onIsLive = append(onIsLive, nl.onIsLive...) - } - } - nl.mu.Unlock() - - if !shouldReplace { - return - } - - now := nl.clock.Now() - if !oldLivenessRec.IsLive(now) && newLivenessRec.IsLive(now) { - for _, fn := range onIsLive { - fn(newLivenessRec.Liveness) - } - } - if newLivenessRec.Membership.Decommissioned() && nl.onNodeDecommissioned != nil { - nl.onNodeDecommissioned(newLivenessRec.Liveness) - } - if newLivenessRec.Membership.Decommissioning() && nl.onNodeDecommissioning != nil { - nl.onNodeDecommissioning(newLivenessRec.NodeID) - } -} - -// shouldReplaceLiveness checks to see if the new liveness is in fact newer -// than the old liveness. -func shouldReplaceLiveness(ctx context.Context, old, new Record) bool { - oldL, newL := old.Liveness, new.Liveness - if (oldL == livenesspb.Liveness{}) { - log.Fatal(ctx, "invalid old liveness record; found to be empty") - } - - // Compare liveness information. If oldL < newL, replace. - if cmp := oldL.Compare(newL); cmp != 0 { - return cmp < 0 - } - - // If Epoch and Expiration are unchanged, assume that the update is newer - // when its draining or decommissioning field changed. - // - // Similarly, assume that the update is newer if the raw encoding is changed - // when all of the fields are the same. This ensures that the CPut performed - // by updateLivenessAttempt will eventually succeed even if the proto - // encoding changes. - // - // This has false positives (in which case we're clobbering the liveness). A - // better way to handle liveness updates in general is to add a sequence - // number. - // - // See #18219. - return oldL.Draining != newL.Draining || - oldL.Membership != newL.Membership || - (oldL.Equal(newL) && !bytes.Equal(old.raw, new.raw)) -} - -// livenessGossipUpdate is the gossip callback used to keep the -// in-memory liveness info up to date. -func (nl *NodeLiveness) livenessGossipUpdate(_ string, content roachpb.Value) { - var liveness livenesspb.Liveness - ctx := context.TODO() - if err := content.GetProto(&liveness); err != nil { - log.Errorf(ctx, "%v", err) - return - } - - nl.maybeUpdate(ctx, Record{Liveness: liveness, raw: content.TagAndDataBytes()}) -} - // numLiveNodes is used to populate a metric that tracks the number of live // nodes in the cluster. Returns 0 if this node is not itself live, to avoid // reporting potentially inaccurate data. @@ -1328,40 +1194,35 @@ func (nl *NodeLiveness) livenessGossipUpdate(_ string, content roachpb.Value) { // nodes reporting the metric, so it's simplest to just have all live nodes // report it. func (nl *NodeLiveness) numLiveNodes() int64 { - selfID := nl.gossip.NodeID.Get() + + selfID := nl.cache.selfID() + // if our node id isn't set, don't return a count if selfID == 0 { return 0 } - nl.mu.RLock() - defer nl.mu.RUnlock() - - self, ok := nl.getLivenessLocked(selfID) - if !ok { - return 0 - } - now := nl.clock.Now() - // If this node isn't live, we don't want to report its view of node liveness - // because it's more likely to be inaccurate than the view of a live node. - if !self.IsLive(now) { - return 0 - } var liveNodes int64 - for _, l := range nl.mu.nodes { - if l.IsLive(now) { + for n, v := range nl.cache.GetIsLiveMap() { + if v.IsLive { liveNodes++ } + // If this node isn't live, we don't want to report its view of node liveness + // because it's more likely to be inaccurate than the view of a live node. + if n == selfID && !v.IsLive { + return 0 + } } return liveNodes } // GetNodeCount returns a count of the number of nodes in the cluster, // including dead nodes, but excluding decommissioning or decommissioned nodes. +// TODO(baptist): remove this method. There are better alternatives. func (nl *NodeLiveness) GetNodeCount() int { - nl.mu.RLock() - defer nl.mu.RUnlock() + nl.cache.mu.RLock() + defer nl.cache.mu.RUnlock() var count int - for _, l := range nl.mu.nodes { + for _, l := range nl.cache.mu.nodes { if l.Membership.Active() { count++ } @@ -1372,13 +1233,14 @@ func (nl *NodeLiveness) GetNodeCount() int { // GetNodeCountWithOverrides returns a count of the number of nodes in the cluster, // including dead nodes, but excluding decommissioning or decommissioned nodes, // using the provided set of liveness overrides. +// TODO(baptist): remove this method. There are better alternatives. func (nl *NodeLiveness) GetNodeCountWithOverrides( overrides map[roachpb.NodeID]livenesspb.NodeLivenessStatus, ) int { - nl.mu.RLock() - defer nl.mu.RUnlock() + nl.cache.mu.RLock() + defer nl.cache.mu.RUnlock() var count int - for _, l := range nl.mu.nodes { + for _, l := range nl.cache.mu.nodes { if l.Membership.Active() { if overrideStatus, ok := overrides[l.NodeID]; !ok || (overrideStatus != livenesspb.NodeLivenessStatus_DECOMMISSIONING && @@ -1409,5 +1271,5 @@ func (nl *NodeLiveness) TestingSetDecommissioningInternal( // TestingMaybeUpdate replaces the liveness (if it appears newer) and invokes // the registered callbacks if the node became live in the process. For testing. func (nl *NodeLiveness) TestingMaybeUpdate(ctx context.Context, newRec Record) { - nl.maybeUpdate(ctx, newRec) + nl.cache.maybeUpdate(ctx, newRec) } diff --git a/pkg/kv/kvserver/liveness/liveness_test.go b/pkg/kv/kvserver/liveness/liveness_test.go index f3597cacde36..f1987eb1399a 100644 --- a/pkg/kv/kvserver/liveness/liveness_test.go +++ b/pkg/kv/kvserver/liveness/liveness_test.go @@ -11,7 +11,6 @@ package liveness import ( - "context" "fmt" "testing" @@ -126,7 +125,7 @@ func TestShouldReplaceLiveness(t *testing.T) { }, } { t.Run("", func(t *testing.T) { - if act := shouldReplaceLiveness(context.Background(), test.old, test.new); act != test.exp { + if act := livenessChanged(test.old, test.new); act != test.exp { t.Errorf("unexpected update: %+v", test) } }) diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index c078152c7344..c4d7a2cb8b76 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -750,14 +750,14 @@ func TestNodeLivenessGetLivenesses(t *testing.T) { nl := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) actualLMapNodes := make(map[roachpb.NodeID]struct{}) originalExpiration := testStartTime + nl.GetLivenessThreshold().Nanoseconds() - for _, l := range nl.GetLivenesses() { + for id, l := range nl.GetIsLiveMap() { if a, e := l.Epoch, int64(1); a != e { t.Errorf("liveness record had epoch %d, wanted %d", a, e) } if a, e := l.Expiration.WallTime, originalExpiration; a < e { t.Errorf("liveness record had expiration %d, wanted %d", a, e) } - actualLMapNodes[l.NodeID] = struct{}{} + actualLMapNodes[id] = struct{}{} } expectedLMapNodes := map[roachpb.NodeID]struct{}{1: {}, 2: {}, 3: {}} if !reflect.DeepEqual(actualLMapNodes, expectedLMapNodes) { @@ -781,7 +781,7 @@ func TestNodeLivenessGetLivenesses(t *testing.T) { // Verify that node liveness receives the change. actualLMapNodes = make(map[roachpb.NodeID]struct{}) - for _, l := range nl.GetLivenesses() { + for id, l := range nl.GetIsLiveMap() { if a, e := l.Epoch, int64(1); a != e { t.Errorf("liveness record had epoch %d, wanted %d", a, e) } @@ -792,7 +792,7 @@ func TestNodeLivenessGetLivenesses(t *testing.T) { if a, e := l.Expiration.WallTime, expectedExpiration; a < e { t.Errorf("liveness record had expiration %d, wanted %d", a, e) } - actualLMapNodes[l.NodeID] = struct{}{} + actualLMapNodes[id] = struct{}{} } if !reflect.DeepEqual(actualLMapNodes, expectedLMapNodes) { t.Errorf("got liveness map nodes %+v; wanted %+v", actualLMapNodes, expectedLMapNodes) @@ -1214,13 +1214,9 @@ func TestNodeLivenessNoRetryOnAmbiguousResultCausedByCancellation(t *testing.T) func verifyNodeIsDecommissioning(t *testing.T, tc *testcluster.TestCluster, nodeID roachpb.NodeID) { testutils.SucceedsSoon(t, func() error { for _, s := range tc.Servers { - for _, liv := range s.NodeLiveness().(*liveness.NodeLiveness).GetLivenesses() { - if liv.NodeID != nodeID { - continue - } - if !liv.Membership.Decommissioning() { - return errors.Errorf("unexpected Membership value of %v for node %v", liv.Membership, liv.NodeID) - } + liv, _ := s.NodeLiveness().(*liveness.NodeLiveness).GetLiveness(nodeID) + if !liv.Membership.Decommissioning() { + return errors.Errorf("unexpected Membership value of %v for node %v", liv.Membership, liv.NodeID) } } return nil diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index 589b2b2f4aca..bb8968d92917 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/load" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" @@ -44,7 +45,7 @@ func TestStorePoolUpdateLocalStore(t *testing.T) { // We're going to manually mark stores dead in this test. st := cluster.MakeTestingClusterSettings() stopper, g, _, sp, _ := storepool.CreateTestStorePool(ctx, st, - storepool.TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) @@ -216,7 +217,7 @@ func TestStorePoolUpdateLocalStoreBeforeGossip(t *testing.T) { cfg := TestStoreConfig(clock) var stopper *stop.Stopper stopper, _, _, cfg.StorePool, _ = storepool.CreateTestStorePool(ctx, cfg.Settings, - storepool.TestTimeUntilStoreDead, false, /* deterministic */ + liveness.TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 9d98c2d38b68..69a85785958e 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -2217,7 +2217,7 @@ func getLivenessStatusMap( if err != nil { return nil, err } - threshold := storepool.TimeUntilStoreDead.Get(&st.SV) + threshold := liveness.TimeUntilStoreDead.Get(&st.SV) statusMap := make(map[roachpb.NodeID]livenesspb.NodeLivenessStatus, len(livenesses)) for _, liveness := range livenesses { @@ -2238,7 +2238,7 @@ func getLivenessResponse( return nil, serverError(ctx, err) } - threshold := storepool.TimeUntilStoreDead.Get(&st.SV) + threshold := liveness.TimeUntilStoreDead.Get(&st.SV) statusMap := make(map[roachpb.NodeID]livenesspb.NodeLivenessStatus, len(livenesses)) for _, liveness := range livenesses { diff --git a/pkg/sql/colenc/inverted.go b/pkg/sql/colenc/inverted.go index ce1cbec7ffe1..238ffedf0166 100644 --- a/pkg/sql/colenc/inverted.go +++ b/pkg/sql/colenc/inverted.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) @@ -32,7 +33,7 @@ func invertedColToDatum(vec coldata.Vec, row int) tree.Datum { return tree.NewDJSON(vec.JSON().Get(row)) case types.StringFamily: b := vec.Bytes().Get(row) - s := unsafeConvertBytesToString(b) + s := encoding.UnsafeConvertBytesToString(b) return tree.NewDString(s) } // This handles arrays, geo etc. diff --git a/pkg/sql/colenc/key.go b/pkg/sql/colenc/key.go index 598cc708d051..0fe6a062a6b9 100644 --- a/pkg/sql/colenc/key.go +++ b/pkg/sql/colenc/key.go @@ -11,8 +11,6 @@ package colenc import ( - "unsafe" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -144,9 +142,9 @@ func encodeKeys[T []byte | roachpb.Key]( } s := ss.Get(r + start) if dir == encoding.Ascending { - kys[r] = encoding.EncodeStringAscending(b, unsafeConvertBytesToString(s)) + kys[r] = encoding.EncodeStringAscending(b, encoding.UnsafeConvertBytesToString(s)) } else { - kys[r] = encoding.EncodeStringDescending(b, unsafeConvertBytesToString(s)) + kys[r] = encoding.EncodeStringDescending(b, encoding.UnsafeConvertBytesToString(s)) } } case types.TimestampFamily, types.TimestampTZFamily: @@ -249,7 +247,3 @@ func (b *BatchEncoder) encodeIndexKey( } return nil } - -func unsafeConvertBytesToString(b []byte) string { - return *(*string)(unsafe.Pointer(&b)) -} diff --git a/pkg/sql/colenc/legacy.go b/pkg/sql/colenc/legacy.go index 5e15dffa19c9..fafff4b0ca54 100644 --- a/pkg/sql/colenc/legacy.go +++ b/pkg/sql/colenc/legacy.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc/valueside" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -66,7 +67,7 @@ func MarshalLegacy(colType *types.T, vec coldata.Vec, row int) (roachpb.Value, e switch vec.Type().Family() { case types.StringFamily, types.BytesFamily, types.UuidFamily, types.EnumFamily: b := vec.Bytes().Get(row) - r.SetString(unsafeConvertBytesToString(b)) + r.SetString(encoding.UnsafeConvertBytesToString(b)) return r, nil } case types.TimestampFamily, types.TimestampTZFamily: diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 2224b270a026..225cd805f122 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//pkg/sql/sqltelemetry", # keep "//pkg/sql/types", # keep "//pkg/util/duration", # keep + "//pkg/util/encoding", # keep "//pkg/util/json", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep "@com_github_cockroachdb_errors//:errors", # keep diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index 83bc025d8523..54249b5d2545 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) @@ -24176,7 +24177,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24210,7 +24211,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24242,7 +24243,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24272,7 +24273,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24327,7 +24328,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24359,7 +24360,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24388,7 +24389,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24416,7 +24417,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24471,7 +24472,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24503,7 +24504,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24532,7 +24533,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24560,7 +24561,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24615,7 +24616,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24647,7 +24648,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24676,7 +24677,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24704,7 +24705,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24865,7 +24866,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24897,7 +24898,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24927,7 +24928,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24955,7 +24956,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index 730a4cee0724..0125456a41b4 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) diff --git a/pkg/sql/colexec/colexecprojconst/BUILD.bazel b/pkg/sql/colexec/colexecprojconst/BUILD.bazel index a649cf9c3042..63e8517756b9 100644 --- a/pkg/sql/colexec/colexecprojconst/BUILD.bazel +++ b/pkg/sql/colexec/colexecprojconst/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//pkg/sql/sqltelemetry", # keep "//pkg/sql/types", "//pkg/util/duration", # keep + "//pkg/util/encoding", # keep "//pkg/util/json", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go index f697fafc897a..f3e8a408d688 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -22395,7 +22396,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22426,7 +22427,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22457,7 +22458,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22485,7 +22486,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22540,7 +22541,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22568,7 +22569,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22596,7 +22597,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22621,7 +22622,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22676,7 +22677,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22704,7 +22705,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22732,7 +22733,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22757,7 +22758,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22812,7 +22813,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22840,7 +22841,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22868,7 +22869,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22893,7 +22894,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -23048,7 +23049,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23077,7 +23078,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23106,7 +23107,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23132,7 +23133,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index b9ad2c124e0f..58a7d8d5da73 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go index ce8dc945d82e..04012e174f75 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -22422,7 +22423,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22453,7 +22454,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22484,7 +22485,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22512,7 +22513,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22567,7 +22568,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22596,7 +22597,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22624,7 +22625,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22650,7 +22651,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22705,7 +22706,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22734,7 +22735,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22762,7 +22763,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22788,7 +22789,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22843,7 +22844,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22872,7 +22873,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22900,7 +22901,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22926,7 +22927,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -23081,7 +23082,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23110,7 +23111,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23139,7 +23140,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23165,7 +23166,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go index f7a896d38cbe..2f896143e79c 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go @@ -684,7 +684,7 @@ if _j == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) }`, vecVariable, idxVariable)) case treebin.Minus: return fmt.Sprintf(` @@ -736,7 +736,7 @@ if _j == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) }`, vecVariable, idxVariable)) case treebin.Minus: return fmt.Sprintf(` @@ -787,7 +787,7 @@ if _path == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) } `, vecVariable, idxVariable)) default: diff --git a/pkg/sql/copy_file_upload.go b/pkg/sql/copy_file_upload.go index 504fc83f1e2b..088fe49bca1d 100644 --- a/pkg/sql/copy_file_upload.go +++ b/pkg/sql/copy_file_upload.go @@ -194,7 +194,7 @@ func (f *fileUploadMachine) run(ctx context.Context) error { func (f *fileUploadMachine) writeFile(ctx context.Context, finalBatch bool) error { for i := 0; i < f.c.rows.Len(); i++ { r := f.c.rows.At(i) - b := []byte(*r[0].(*tree.DBytes)) + b := r[0].(*tree.DBytes).UnsafeBytes() n, err := f.w.Write(b) if err != nil { return err diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go index e02b9effdd93..c5b9163ed2bb 100644 --- a/pkg/sql/lex/encode.go +++ b/pkg/sql/lex/encode.go @@ -142,28 +142,39 @@ func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix // according to the encoding specification in "be". // When using the Hex format, the caller is responsible for skipping the // "\x" prefix, if any. See DecodeRawBytesToByteArrayAuto() below for -// an alternative. -func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error) { +// an alternative. If no conversion is necessary the input is returned, +// callers should not assume a copy is made. +func DecodeRawBytesToByteArray(data []byte, be BytesEncodeFormat) ([]byte, error) { switch be { case BytesEncodeHex: - return hex.DecodeString(data) + res := make([]byte, hex.DecodedLen(len(data))) + n, err := hex.Decode(res, data) + return res[:n], err case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. // See https://www.postgresql.org/docs/current/static/datatype-binary.html#AEN5667 - res := make([]byte, 0, len(data)) + res := data + copied := false for i := 0; i < len(data); i++ { ch := data[i] if ch != '\\' { - res = append(res, ch) + if copied { + res = append(res, ch) + } continue } if i >= len(data)-1 { return nil, pgerror.New(pgcode.InvalidEscapeSequence, "bytea encoded value ends with escape character") } + if !copied { + res = make([]byte, 0, len(data)) + res = append(res, data[:i]...) + copied = true + } if data[i+1] == '\\' { res = append(res, '\\') i++ @@ -188,7 +199,9 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error return res, nil case BytesEncodeBase64: - return base64.StdEncoding.DecodeString(data) + res := make([]byte, base64.StdEncoding.DecodedLen(len(data))) + n, err := base64.StdEncoding.Decode(res, data) + return res[:n], err default: return nil, errors.AssertionFailedf("unhandled format: %s", be) @@ -200,9 +213,9 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error // and escape. func DecodeRawBytesToByteArrayAuto(data []byte) ([]byte, error) { if len(data) >= 2 && data[0] == '\\' && (data[1] == 'x' || data[1] == 'X') { - return DecodeRawBytesToByteArray(string(data[2:]), BytesEncodeHex) + return DecodeRawBytesToByteArray(data[2:], BytesEncodeHex) } - return DecodeRawBytesToByteArray(string(data), BytesEncodeEscape) + return DecodeRawBytesToByteArray(data, BytesEncodeEscape) } func (f BytesEncodeFormat) String() string { diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go index f36bfb82e683..9cd63c864443 100644 --- a/pkg/sql/lex/encode_test.go +++ b/pkg/sql/lex/encode_test.go @@ -64,7 +64,7 @@ func TestByteArrayDecoding(t *testing.T) { if s.auto { dec, err = lex.DecodeRawBytesToByteArrayAuto([]byte(s.in)) } else { - dec, err = lex.DecodeRawBytesToByteArray(s.in, s.inFmt) + dec, err = lex.DecodeRawBytesToByteArray([]byte(s.in), s.inFmt) } if s.err != "" { if err == nil { @@ -124,7 +124,7 @@ func TestByteArrayEncoding(t *testing.T) { enc = enc[2:] } - dec, err := lex.DecodeRawBytesToByteArray(enc, format) + dec, err := lex.DecodeRawBytesToByteArray([]byte(enc), format) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 96fac77d5abe..171d47bc5f19 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1264,10 +1264,11 @@ func (t *logicTest) newTestServerCluster(bootstrapBinaryPath string, upgradeBina testserver.PollListenURLTimeoutOpt(120), } if strings.Contains(upgradeBinaryPath, "cockroach-short") { - // If we're using a cockroach-short binary, that means it was locally - // built, so we need to opt-in to development upgrades. + // If we're using a cockroach-short binary, that means it was + // locally built, so we need to opt-out of version offsetting to + // better simulate a real upgrade path. opts = append(opts, testserver.EnvVarOpt([]string{ - "COCKROACH_UPGRADE_TO_DEV_VERSION=true", + "COCKROACH_TESTING_FORCE_RELEASE_BRANCH=true", })) } diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_can_login b/pkg/sql/logictest/testdata/logic_test/mixed_version_can_login index c55c2ba5f016..bb1b67b440bc 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_can_login +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_can_login @@ -1,5 +1,7 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master +skip 103956 + query I SELECT 1 ---- @@ -8,7 +10,7 @@ SELECT 1 query T SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 statement ok SELECT crdb_internal.create_tenant(1000) @@ -21,20 +23,20 @@ SELECT crdb_internal.create_tenant(1001) upgrade 1 query B -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true query T nodeidx=2 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 # Verify that a non-root user can login on the upgraded node. user testuser nodeidx=0 query B -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true @@ -42,7 +44,7 @@ true user root nodeidx=1 query B -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true @@ -52,7 +54,7 @@ user testuser nodeidx=2 query T SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 # Verify that a root user can login on the non-upgraded node. user root nodeidx=2 @@ -60,12 +62,12 @@ user root nodeidx=2 query T SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 upgrade 2 query B -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_insights_queries b/pkg/sql/logictest/testdata/logic_test/mixed_version_insights_queries index 9f078adf7d34..6cf49144ec66 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_insights_queries +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_insights_queries @@ -1,21 +1,21 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master -# Verify that all nodes are running 22.2 binaries. +# Verify that all nodes are running the previous version. query T nodeidx=0 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=1 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=2 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 upgrade 1 diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_new_system_privileges b/pkg/sql/logictest/testdata/logic_test/mixed_version_new_system_privileges index 85e208b51c20..5a3d63d418ed 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_new_system_privileges +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_new_system_privileges @@ -1,5 +1,9 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master +# This test should only be executed to test upgrades from 22.2 -> 23.2 +# (experimental version skipping). +skip 103956 + upgrade 0 upgrade 1 diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_partially_visible_index b/pkg/sql/logictest/testdata/logic_test/mixed_version_partially_visible_index index 54d2ce9ae9b5..574ce92f9146 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_partially_visible_index +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_partially_visible_index @@ -1,21 +1,21 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master -# Verify that all nodes are running 22.2 binaries. +# Verify that all nodes are running the previous version. query T nodeidx=0 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=1 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=2 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 upgrade 0 diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_range_tombstones b/pkg/sql/logictest/testdata/logic_test/mixed_version_range_tombstones index 9c52fa9ca0e1..bf9ec4191a88 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_range_tombstones +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_range_tombstones @@ -1,5 +1,9 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master +# This test should only be executed to test upgrades from 22.2 -> 23.2 +# (experimental version skipping). +skip 103956 + query I SELECT 1 ---- diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_role_members_user_ids b/pkg/sql/logictest/testdata/logic_test/mixed_version_role_members_user_ids index 874f6f535ce2..6a0c0d12d3e6 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_role_members_user_ids +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_role_members_user_ids @@ -1,21 +1,21 @@ # LogicTest: cockroach-go-testserver-upgrade-to-master -# Verify that all nodes are running 22.2 binaries. +# Verify that all nodes are running previous version binaries. query T nodeidx=0 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=1 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 query T nodeidx=2 SELECT crdb_internal.node_executable_version() ---- -22.2 +23.1 # Create test users. @@ -28,26 +28,26 @@ CREATE USER testuser2 statement ok CREATE USER testuser3 -query TTB +query TTBTT SELECT * FROM system.role_members ---- -admin root true +admin root true 2 1 statement ok GRANT testuser1 TO testuser2 -query TTB +query TTBTT SELECT * FROM system.role_members ---- -admin root true -testuser1 testuser2 false +admin root true 2 1 +testuser1 testuser2 false 101 102 upgrade 1 -# Test that there are no problems creating role memberships on a mixed 22.2/23.1 cluster. +# Test that there are no problems creating role memberships on a mixed-version cluster. query B nodeidx=1 -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true @@ -58,23 +58,23 @@ GRANT testuser1 TO testuser3 user root nodeidx=0 -query TTB +query TTBTT SELECT * FROM system.role_members ---- -admin root true -testuser1 testuser2 false -testuser1 testuser3 false +admin root true 2 1 +testuser1 testuser2 false 101 102 +testuser1 testuser3 false 101 103 statement ok GRANT testuser2 TO testuser3 -query TTB +query TTBTT SELECT * FROM system.role_members ---- -admin root true -testuser1 testuser2 false -testuser1 testuser3 false -testuser2 testuser3 false +admin root true 2 1 +testuser1 testuser2 false 101 102 +testuser1 testuser3 false 101 103 +testuser2 testuser3 false 102 103 upgrade 0 @@ -83,16 +83,16 @@ upgrade 2 # Verify that all nodes are now running 23.1 binaries. query B nodeidx=0 -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true query B nodeidx=1 -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true query B nodeidx=2 -SELECT crdb_internal.node_executable_version() SIMILAR TO '1000023.1-%' +SELECT crdb_internal.node_executable_version() SIMILAR TO '23.1-%' ---- true diff --git a/pkg/sql/logictest/testdata/logic_test/mixed_version_system_privileges_user_id b/pkg/sql/logictest/testdata/logic_test/mixed_version_system_privileges_user_id index 00676b9724e1..cfe56f8054b3 100644 --- a/pkg/sql/logictest/testdata/logic_test/mixed_version_system_privileges_user_id +++ b/pkg/sql/logictest/testdata/logic_test/mixed_version_system_privileges_user_id @@ -20,7 +20,7 @@ user root nodeidx=2 statement ok GRANT SYSTEM CANCELQUERY TO testuser1 -query TTTT +query TTTT rowsort SELECT username, path, privileges, grant_options FROM system.privileges ---- public /vtable/crdb_internal/tables {} {} diff --git a/pkg/sql/opt/memo/interner.go b/pkg/sql/opt/memo/interner.go index 2e0f86c470d1..db43fb6748d9 100644 --- a/pkg/sql/opt/memo/interner.go +++ b/pkg/sql/opt/memo/interner.go @@ -361,7 +361,7 @@ func (h *hasher) HashDatum(val tree.Datum) { case *tree.DString: h.HashString(string(*t)) case *tree.DBytes: - h.HashBytes([]byte(*t)) + h.HashBytes(t.UnsafeBytes()) case *tree.DDate: h.HashUint64(uint64(t.PGEpochDays())) case *tree.DTime: @@ -843,7 +843,7 @@ func (h *hasher) IsDatumEqual(l, r tree.Datum) bool { return lt.Locale == rt.Locale && h.IsStringEqual(lt.Contents, rt.Contents) case *tree.DBytes: rt := r.(*tree.DBytes) - return bytes.Equal([]byte(*lt), []byte(*rt)) + return bytes.Equal(lt.UnsafeBytes(), rt.UnsafeBytes()) case *tree.DDate: rt := r.(*tree.DDate) return lt.Date == rt.Date diff --git a/pkg/sql/optionalnodeliveness/node_liveness.go b/pkg/sql/optionalnodeliveness/node_liveness.go index 29861cdb26d1..f14d943667bc 100644 --- a/pkg/sql/optionalnodeliveness/node_liveness.go +++ b/pkg/sql/optionalnodeliveness/node_liveness.go @@ -22,7 +22,6 @@ import ( // Interface is the interface used in Container. type Interface interface { Self() (livenesspb.Liveness, bool) - GetLivenesses() []livenesspb.Liveness GetLiveness(nodeID roachpb.NodeID) (liveness.Record, bool) GetLivenessesFromKV(ctx context.Context) ([]livenesspb.Liveness, error) IsAvailable(roachpb.NodeID) bool diff --git a/pkg/sql/pgwire/pgwirebase/BUILD.bazel b/pkg/sql/pgwire/pgwirebase/BUILD.bazel index 9ad52b9b1ad2..9bbcdc6bb80a 100644 --- a/pkg/sql/pgwire/pgwirebase/BUILD.bazel +++ b/pkg/sql/pgwire/pgwirebase/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/sql/types", "//pkg/util/bitarray", "//pkg/util/duration", + "//pkg/util/encoding", "//pkg/util/errorutil/unimplemented", "//pkg/util/ipaddr", "//pkg/util/timeofday", diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index 21aa603309e9..1da69d3ebb7d 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/timeofday" @@ -324,31 +325,33 @@ func DecodeDatum( ctx context.Context, evalCtx *eval.Context, typ *types.T, code FormatCode, b []byte, ) (tree.Datum, error) { id := typ.Oid() + // Use a direct string pointing to b where we are sure we aren't retaining this string. + bs := encoding.UnsafeConvertBytesToString(b) switch code { case FormatText: switch id { case oid.T_record: - d, _, err := tree.ParseDTupleFromString(evalCtx, string(b), typ) + d, _, err := tree.ParseDTupleFromString(evalCtx, bs, typ) if err != nil { return nil, err } return d, nil case oid.T_bool: - t, err := strconv.ParseBool(string(b)) + t, err := strconv.ParseBool(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.MakeDBool(tree.DBool(t)), nil case oid.T_bit, oid.T_varbit: - t, err := tree.ParseDBitArray(string(b)) + t, err := tree.ParseDBitArray(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return t, nil case oid.T_int2, oid.T_int4, oid.T_int8: - i, err := strconv.ParseInt(string(b), 10, 64) + i, err := strconv.ParseInt(bs, 10, 64) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.NewDInt(tree.DInt(i)), nil case oid.T_oid, @@ -362,106 +365,108 @@ func DecodeDatum( oid.T_regnamespace, oid.T_regprocedure, oid.T_regdictionary: - return eval.ParseDOid(ctx, evalCtx, string(b), typ) + return eval.ParseDOid(ctx, evalCtx, bs, typ) case oid.T_float4, oid.T_float8: - f, err := strconv.ParseFloat(string(b), 64) + f, err := strconv.ParseFloat(bs, 64) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.NewDFloat(tree.DFloat(f)), nil case oidext.T_box2d: - d, err := tree.ParseDBox2D(string(b)) + d, err := tree.ParseDBox2D(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oidext.T_geography: - d, err := tree.ParseDGeography(string(b)) + d, err := tree.ParseDGeography(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oidext.T_geometry: - d, err := tree.ParseDGeometry(string(b)) + d, err := tree.ParseDGeometry(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_void: return tree.DVoidDatum, nil case oid.T_numeric: - d, err := tree.ParseDDecimal(string(b)) + d, err := tree.ParseDDecimal(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_bytea: res, err := lex.DecodeRawBytesToByteArrayAuto(b) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } - return tree.NewDBytes(tree.DBytes(res)), nil + // Note: we could use encoding.UnsafeConvertBytesToString here if + // we were guaranteed all callers never mutated b. + return tree.NewDBytes(tree.DBytes(string(res))), nil case oid.T_timestamp: - d, _, err := tree.ParseDTimestamp(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestamp(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_timestamptz: - d, _, err := tree.ParseDTimestampTZ(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestampTZ(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_date: - d, _, err := tree.ParseDDate(evalCtx, string(b)) + d, _, err := tree.ParseDDate(evalCtx, bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_time: - d, _, err := tree.ParseDTime(nil, string(b), time.Microsecond) + d, _, err := tree.ParseDTime(nil, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_timetz: - d, _, err := tree.ParseDTimeTZ(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimeTZ(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_interval: - d, err := tree.ParseDInterval(evalCtx.GetIntervalStyle(), string(b)) + d, err := tree.ParseDInterval(evalCtx.GetIntervalStyle(), bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_uuid: - d, err := tree.ParseDUuidFromString(string(b)) + d, err := tree.ParseDUuidFromString(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_inet: - d, err := tree.ParseDIPAddrFromINetString(string(b)) + d, err := tree.ParseDIPAddrFromINetString(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_jsonb, oid.T_json: if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(bs) case oid.T_tsquery: - ret, err := tsearch.ParseTSQuery(string(b)) + ret, err := tsearch.ParseTSQuery(bs) if err != nil { return nil, err } return &tree.DTSQuery{TSQuery: ret}, nil case oid.T_tsvector: - ret, err := tsearch.ParseTSVector(string(b)) + ret, err := tsearch.ParseTSVector(bs) if err != nil { return nil, err } @@ -473,7 +478,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.NewDString(string(b)), nil + return tree.NewDString(bs), nil } case FormatBinary: switch id { @@ -613,7 +618,7 @@ func DecodeDatum( decDigits = decDigits[:len(decDigits)-int(overScale)] } - decString := string(decDigits) + decString := encoding.UnsafeConvertBytesToString(decDigits) if _, ok := alloc.dd.Coeff.SetString(decString, 10); !ok { return nil, pgerror.Newf(pgcode.Syntax, "could not parse %q as type decimal", decString) } @@ -640,6 +645,7 @@ func DecodeDatum( return &alloc.dd, nil case oid.T_bytea: + // Note: there's an implicit string cast here reallocating b. return tree.NewDBytes(tree.DBytes(b)), nil case oid.T_timestamp: if len(b) < 8 { @@ -698,7 +704,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(bs) case oid.T_jsonb: if len(b) < 1 { return nil, NewProtocolViolationErrorf("no data to decode") @@ -711,7 +717,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(encoding.UnsafeConvertBytesToString(b)) case oid.T_varbit, oid.T_bit: if len(b) < 4 { return nil, NewProtocolViolationErrorf("insufficient data: %d", len(b)) @@ -801,7 +807,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - e, err := tree.MakeDEnumFromLogicalRepresentation(typ, string(b)) + e, err := tree.MakeDEnumFromLogicalRepresentation(typ, bs) if err != nil { return nil, err } @@ -812,15 +818,21 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. return tree.NewDString(string(b)), nil case oid.T_bpchar: if err := validateStringBytes(b); err != nil { return nil, err } // Trim the trailing spaces + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. sv := strings.TrimRight(string(b), " ") return tree.NewDString(sv), nil case oid.T_char: + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. sv := string(b) // Always truncate to 1 byte, and handle the null byte specially. if len(b) >= 1 { @@ -835,6 +847,8 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. return tree.NewDName(string(b)), nil } diff --git a/pkg/sql/rowenc/valueside/encode.go b/pkg/sql/rowenc/valueside/encode.go index 33b548b42bea..506cd0911efc 100644 --- a/pkg/sql/rowenc/valueside/encode.go +++ b/pkg/sql/rowenc/valueside/encode.go @@ -48,11 +48,11 @@ func Encode(appendTo []byte, colID ColumnIDDelta, val tree.Datum, scratch []byte case *tree.DDecimal: return encoding.EncodeDecimalValue(appendTo, uint32(colID), &t.Decimal), nil case *tree.DString: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DBytes: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DEncodedKey: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DDate: return encoding.EncodeIntValue(appendTo, uint32(colID), t.UnixEpochDaysWithOrig()), nil case *tree.DBox2D: @@ -102,7 +102,7 @@ func Encode(appendTo []byte, colID ColumnIDDelta, val tree.Datum, scratch []byte case *tree.DTuple: return encodeTuple(t, appendTo, uint32(colID), scratch) case *tree.DCollatedString: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(t.Contents)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeContentBytes()), nil case *tree.DOid: return encoding.EncodeIntValue(appendTo, uint32(colID), int64(t.Oid)), nil case *tree.DEnum: diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 0f083bda30d2..4db1b5f73348 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -436,20 +436,20 @@ var regularBuiltins = map[string]builtinDefinition{ Types: tree.ParamTypes{{Name: "str", Typ: types.Bytes}, {Name: "enc", Typ: types.String}}, ReturnType: tree.FixedReturnType(types.String), Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { - str := []byte(tree.MustBeDBytes(args[0])) + str := string(tree.MustBeDBytes(args[0])) enc := CleanEncodingName(string(tree.MustBeDString(args[1]))) switch enc { // All the following are aliases to each other in PostgreSQL. case "utf8", "unicode", "cp65001": - if !utf8.Valid(str) { + if !utf8.Valid(encoding.UnsafeConvertStringToBytes(str)) { return nil, newDecodeError("UTF8") } - return tree.NewDString(string(str)), nil + return tree.NewDString(str), nil // All the following are aliases to each other in PostgreSQL. case "latin1", "iso88591", "cp28591": var buf strings.Builder - for _, c := range str { + for _, c := range encoding.UnsafeConvertStringToBytes(str) { buf.WriteRune(rune(c)) } return tree.NewDString(buf.String()), nil @@ -1170,11 +1170,11 @@ var regularBuiltins = map[string]builtinDefinition{ return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for decode()") } - res, err := lex.DecodeRawBytesToByteArray(data, be) + res, err := lex.DecodeRawBytesToByteArray(encoding.UnsafeConvertStringToBytes(data), be) if err != nil { return nil, err } - return tree.NewDBytes(tree.DBytes(res)), nil + return tree.NewDBytes(tree.DBytes(encoding.UnsafeConvertBytesToString(res))), nil }, Info: "Decodes `data` using `format` (`hex` / `escape` / `base64`).", Volatility: volatility.Immutable, diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 23f9c67b3573..daeb11f14200 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/storageparam" "github.com/cockroachdb/cockroach/pkg/sql/storageparam/indexstorageparam" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -760,7 +761,7 @@ var geoBuiltins = map[string]builtinDefinition{ if asString == nil { return tree.DNull, nil } - g, err := geo.ParseGeometryFromGeoJSON([]byte(*asString)) + g, err := geo.ParseGeometryFromGeoJSON(encoding.UnsafeConvertStringToBytes(*asString)) if err != nil { return nil, err } @@ -1063,7 +1064,7 @@ var geoBuiltins = map[string]builtinDefinition{ if asString == nil { return tree.DNull, nil } - g, err := geo.ParseGeographyFromGeoJSON([]byte(*asString)) + g, err := geo.ParseGeographyFromGeoJSON(encoding.UnsafeConvertStringToBytes(*asString)) if err != nil { return nil, err } diff --git a/pkg/sql/sem/eval/cast.go b/pkg/sql/sem/eval/cast.go index 9ac346449193..ec9de9fa5549 100644 --- a/pkg/sql/sem/eval/cast.go +++ b/pkg/sql/sem/eval/cast.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/timeofday" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" @@ -617,7 +618,7 @@ func performCastWithoutPrecisionTruncation( if t == nil { return tree.DNull, nil } - g, err := geo.ParseGeographyFromGeoJSON([]byte(*t)) + g, err := geo.ParseGeographyFromGeoJSON(encoding.UnsafeConvertStringToBytes(*t)) if err != nil { return nil, err } @@ -665,7 +666,7 @@ func performCastWithoutPrecisionTruncation( if t == nil { return tree.DNull, nil } - g, err := geo.ParseGeometryFromGeoJSON([]byte(*t)) + g, err := geo.ParseGeometryFromGeoJSON(encoding.UnsafeConvertStringToBytes(*t)) if err != nil { return nil, err } diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index bb3e895c24a9..285f4334fb60 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -374,11 +374,11 @@ func ParseDBool(s string) (*DBool, error) { // the beginning), and the escaped format, which supports "\\" and // octal escapes. func ParseDByte(s string) (*DBytes, error) { - res, err := lex.DecodeRawBytesToByteArrayAuto([]byte(s)) + res, err := lex.DecodeRawBytesToByteArrayAuto(encoding.UnsafeConvertStringToBytes(s)) if err != nil { return nil, MakeParseError(s, types.Bytes, err) } - return NewDBytes(DBytes(res)), nil + return NewDBytes(DBytes(encoding.UnsafeConvertBytesToString(res))), nil } // ParseDUuidFromString parses and returns the *DUuid Datum value represented @@ -1359,6 +1359,13 @@ func (d *DString) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "Unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DString) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DCollatedString is the Datum for strings with a locale. The struct members // are intended to be immutable. type DCollatedString struct { @@ -1513,6 +1520,13 @@ func (d *DCollatedString) IsComposite() bool { return true } +// UnsafeContentBytes returns the raw bytes avoiding allocation. It is "unsafe" +// because the contract is that callers must not to mutate the bytes but there +// is nothing stopping that from happening. +func (d *DCollatedString) UnsafeContentBytes() []byte { + return encoding.UnsafeConvertStringToBytes(d.Contents) +} + // DBytes is the bytes Datum. The underlying type is a string because we want // the immutability, but this may contain arbitrary bytes. type DBytes string @@ -1645,6 +1659,13 @@ func (d *DBytes) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DBytes) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DEncodedKey is a special Datum of types.EncodedKey type, used to pass through // encoded key data. It is similar to DBytes, except when it comes to // encoding/decoding. It is currently used to pass around inverted index keys, @@ -1717,6 +1738,13 @@ func (d *DEncodedKey) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DEncodedKey) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DUuid is the UUID Datum. type DUuid struct { uuid.UUID diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index ea2fdd22301f..44589a03326e 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -202,7 +202,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto HistogramWindowInterval: cfg.HistogramWindowInterval, Engines: []storage.Engine{ltc.Eng}, }) - storepool.TimeUntilStoreDead.Override(ctx, &cfg.Settings.SV, storepool.TestTimeUntilStoreDead) + liveness.TimeUntilStoreDead.Override(ctx, &cfg.Settings.SV, liveness.TestTimeUntilStoreDead) cfg.StorePool = storepool.NewStorePool( cfg.AmbientCtx, cfg.Settings, diff --git a/pkg/ts/BUILD.bazel b/pkg/ts/BUILD.bazel index 222127854855..623bae9caef7 100644 --- a/pkg/ts/BUILD.bazel +++ b/pkg/ts/BUILD.bazel @@ -25,7 +25,7 @@ go_library( "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvpb", "//pkg/kv/kvserver", - "//pkg/kv/kvserver/allocator/storepool", + "//pkg/kv/kvserver/liveness", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/ts/server.go b/pkg/ts/server.go index 607be760ede7..ba301d473381 100644 --- a/pkg/ts/server.go +++ b/pkg/ts/server.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/ts/tspb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -240,7 +240,7 @@ func (s *Server) Query( // dead. This is a conservatively long span, but gives us a good indication of // when a gap likely indicates an outage (and thus missing values should not // be interpolated). - interpolationLimit := storepool.TimeUntilStoreDead.Get(&s.db.st.SV).Nanoseconds() + interpolationLimit := liveness.TimeUntilStoreDead.Get(&s.db.st.SV).Nanoseconds() // Get the estimated number of nodes on the cluster, used to compute more // accurate memory usage estimates. Set a minimum of 1 in order to avoid diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index fb8cfcfb5bd6..4a3e4a86846d 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -844,7 +844,7 @@ func prettyPrintInvertedIndexKey(b []byte) (string, []byte, error) { switch tempB[i+1] { case escapedTerm: if len(tempB[:i]) > 0 { - outBytes = outBytes + strconv.Quote(unsafeString(tempB[:i])) + outBytes = outBytes + strconv.Quote(UnsafeConvertBytesToString(tempB[:i])) } else { lenOut := len(outBytes) if lenOut > 1 && outBytes[lenOut-1] == '/' { @@ -853,7 +853,7 @@ func prettyPrintInvertedIndexKey(b []byte) (string, []byte, error) { } return outBytes, tempB[i+escapeLength:], nil case escapedJSONObjectKeyTerm: - outBytes = outBytes + strconv.Quote(unsafeString(tempB[:i])) + "/" + outBytes = outBytes + strconv.Quote(UnsafeConvertBytesToString(tempB[:i])) + "/" case escapedJSONArray: outBytes = outBytes + "Arr/" default: @@ -957,11 +957,12 @@ func EncodeStringDescending(b []byte, s string) []byte { return EncodeBytesDescending(b, arg) } -// unsafeString performs an unsafe conversion from a []byte to a string. The -// returned string will share the underlying memory with the []byte which thus -// allows the string to be mutable through the []byte. We're careful to use -// this method only in situations in which the []byte will not be modified. -func unsafeString(b []byte) string { +// UnsafeConvertBytesToString performs an unsafe conversion from a []byte to a +// string. The returned string will share the underlying memory with the +// []byte which thus allows the string to be mutable through the []byte. We're +// careful to use this method only in situations in which the []byte will not +// be modified. +func UnsafeConvertBytesToString(b []byte) string { return *(*string)(unsafe.Pointer(&b)) } @@ -972,7 +973,7 @@ func unsafeString(b []byte) string { // string may share storage with the input buffer. func DecodeUnsafeStringAscending(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesAscending(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // DecodeUnsafeStringAscendingDeepCopy is the same as @@ -980,7 +981,7 @@ func DecodeUnsafeStringAscending(b []byte, r []byte) ([]byte, string, error) { // with the input buffer. func DecodeUnsafeStringAscendingDeepCopy(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesAscendingDeepCopy(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // DecodeUnsafeStringDescending decodes a string value from the input buffer which @@ -991,7 +992,7 @@ func DecodeUnsafeStringAscendingDeepCopy(b []byte, r []byte) ([]byte, string, er // buffer. func DecodeUnsafeStringDescending(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesDescending(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // EncodeNullAscending encodes a NULL value. The encodes bytes are appended to the diff --git a/pkg/util/goschedstats/BUILD.bazel b/pkg/util/goschedstats/BUILD.bazel index 83c8fc3bc62f..201e0fe5a447 100644 --- a/pkg/util/goschedstats/BUILD.bazel +++ b/pkg/util/goschedstats/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "runnable.go", "runtime_go1.19.go", + "runtime_go1.20.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/goschedstats", visibility = ["//visibility:public"], diff --git a/pkg/util/goschedstats/runtime_go1.19.go b/pkg/util/goschedstats/runtime_go1.19.go index 90d9b345910a..a49a37b9f54f 100644 --- a/pkg/util/goschedstats/runtime_go1.19.go +++ b/pkg/util/goschedstats/runtime_go1.19.go @@ -7,7 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -// + // The structure definitions in this file have been cross-checked against go1.19. // Before allowing newer versions, please check that the structures // still match with those in go/src/runtime. diff --git a/pkg/util/goschedstats/runtime_go1.20.go b/pkg/util/goschedstats/runtime_go1.20.go new file mode 100644 index 000000000000..d3e899bcb44b --- /dev/null +++ b/pkg/util/goschedstats/runtime_go1.20.go @@ -0,0 +1,200 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// The structure definitions in this file have been cross-checked against go1.20. +// Before allowing newer versions, please check that the structures +// still match with those in go/src/runtime. + +//go:build gc && go1.20 && !go1.21 +// +build gc,go1.20,!go1.21 + +package goschedstats + +import ( + "sync/atomic" + _ "unsafe" // required by go:linkname +) + +type puintptr uintptr + +type muintptr uintptr + +type guintptr uintptr + +type sysmontick struct { + schedtick uint32 + schedwhen int64 + syscalltick uint32 + syscallwhen int64 +} + +type pageCache struct { + base uintptr // base address of the chunk + cache uint64 // 64-bit bitmap representing free pages (1 means free) + scav uint64 // 64-bit bitmap representing scavenged pages (1 means scavenged) +} + +// A _defer holds an entry on the list of deferred calls. +// If you add a field here, add code to clear it in deferProcStack. +// This struct must match the code in cmd/compile/internal/ssagen/ssa.go:deferstruct +// and cmd/compile/internal/ssagen/ssa.go:(*state).call. +// Some defers will be allocated on the stack and some on the heap. +// All defers are logically part of the stack, so write barriers to +// initialize them are not required. All defers must be manually scanned, +// and for heap defers, marked. +type _defer struct { + // The rest of the fields aren't important. +} + +// Per-thread (in Go, per-P) cache for small objects. +// This includes a small object cache and local allocation stats. +// No locking needed because it is per-thread (per-P). +// +// mcaches are allocated from non-GC'd memory, so any heap pointers +// must be specially handled. +type mcache struct { + // The rest of the fields aren't important. +} + +type p struct { + id int32 + status uint32 // one of pidle/prunning/... + link puintptr + schedtick uint32 // incremented on every scheduler call + syscalltick uint32 // incremented on every system call + sysmontick sysmontick // last tick observed by sysmon + m muintptr // back-link to associated m (nil if idle) + mcache *mcache + pcache pageCache + raceprocctx uintptr + + deferpool []*_defer // pool of available defer structs (see panic.go) + deferpoolbuf [32]*_defer + + // Cache of goroutine ids, amortizes accesses to runtime·sched.goidgen. + goidcache uint64 + goidcacheend uint64 + + // Queue of runnable goroutines. Accessed without lock. + runqhead uint32 + runqtail uint32 + runq [256]guintptr + // runnext, if non-nil, is a runnable G that was ready'd by + // the current G and should be run next instead of what's in + // runq if there's time remaining in the running G's time + // slice. It will inherit the time left in the current time + // slice. If a set of goroutines is locked in a + // communicate-and-wait pattern, this schedules that set as a + // unit and eliminates the (potentially large) scheduling + // latency that otherwise arises from adding the ready'd + // goroutines to the end of the run queue. + // + // Note that while other P's may atomically CAS this to zero, + // only the owner P can CAS it to a valid G. + runnext guintptr + + // The rest of the fields aren't important. +} + +type lockRankStruct struct{} + +// Mutual exclusion locks. In the uncontended case, +// as fast as spin locks (just a few user-level instructions), +// but on the contention path they sleep in the kernel. +// A zeroed Mutex is unlocked (no need to initialize each lock). +// Initialization is helpful for static lock ranking, but not required. +type mutex struct { + // Empty struct if lock ranking is disabled, otherwise includes the lock rank + lockRankStruct + // Futex-based impl treats it as uint32 key, + // while sema-based impl as M* waitm. + // Used to be a union, but unions break precise GC. + key uintptr +} + +// A gQueue is a dequeue of Gs linked through g.schedlink. A G can only +// be on one gQueue or gList at a time. +type gQueue struct { + head guintptr + tail guintptr +} + +type schedt struct { + goidgen atomic.Uint64 + lastpoll atomic.Int64 // time of last network poll, 0 if currently polling + pollUntil atomic.Int64 // time to which current poll is sleeping + + lock mutex + + // When increasing nmidle, nmidlelocked, nmsys, or nmfreed, be + // sure to call checkdead(). + + midle muintptr // idle m's waiting for work + nmidle int32 // number of idle m's waiting for work + nmidlelocked int32 // number of locked m's waiting for work + mnext int64 // number of m's that have been created and next M ID + maxmcount int32 // maximum number of m's allowed (or die) + nmsys int32 // number of system m's not counted for deadlock + nmfreed int64 // cumulative number of freed m's + + ngsys atomic.Int32 // number of system goroutines + + pidle puintptr // idle p's + npidle atomic.Int32 + nmspinning atomic.Int32 // See "Worker thread parking/unparking" comment in proc.go. + needspinning atomic.Uint32 // See "Delicate dance" comment in proc.go. Boolean. Must hold sched.lock to set to 1. + + // Global runnable queue. + runq gQueue + runqsize int32 + + // The rest of the fields aren't important. +} + +//go:linkname allp runtime.allp +var allp []*p + +//go:linkname sched runtime.sched +var sched schedt + +//go:linkname lock runtime.lock +func lock(l *mutex) + +//go:linkname unlock runtime.unlock +func unlock(l *mutex) + +func numRunnableGoroutines() (numRunnable int, numProcs int) { + lock(&sched.lock) + numRunnable = int(sched.runqsize) + numProcs = len(allp) + + // Note that holding sched.lock prevents the number of Ps from changing, so + // it's safe to loop over allp. + for _, p := range allp { + // Retry loop for concurrent updates. + for { + h := atomic.LoadUint32(&p.runqhead) + t := atomic.LoadUint32(&p.runqtail) + next := atomic.LoadUintptr((*uintptr)(&p.runnext)) + runnable := int32(t - h) + if atomic.LoadUint32(&p.runqhead) != h || runnable < 0 { + // A concurrent update messed with us; try again. + continue + } + if next != 0 { + runnable++ + } + numRunnable += int(runnable) + break + } + } + unlock(&sched.lock) + return numRunnable, numProcs +}