Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
101693: sql: avoid string to byte conversion copies on insert path r=cucaroach a=cucaroach

Originated from: #91930

Incorporate those changes and add a benchmark and add some more changes based on benchmark results.

Broken into several commits:

### json: avoid copying json.AsText where possible
    
Epic: None
Release note: None


### pgwire: avoid some string to byte copies

Most parse routines don't retain the input string pointer so we can
use a pointer to the input bytes in those cases.

Release note: None
Epic: None

### builtins: avoid some []byte to string copies in decode

Release note: None
Epic: None



### valueside: remove string to []byte copies

Release note: None
Epic: None


### colenc: use new UnsafeConvertBytesToString

Cosmetic change to use the new UnsafeConvertBytesToString.

Release note: None
Epic: None



### opt: avoid allocation in DBytes interning

DBytes is a string under the covers but we usually operate on them
with []byte APIs, avoid copies in these cases.

Release note: None
Epic: None


### tree: allow access to DBytes/DString/DEncodedKey raw bytes

Epic: None
Release note: None


### encoding: unsafeString -> UnsafeConvertBytesToString

Make public for use elsewhere.

Release note: None
Epic: None

### lex: avoid some string to byte slice copies

Facilitate some copy avoidance by using []byte instead of string.
Only copy when necessary in some cases.

Release note: None
Epic: None

### bench: add a large insert benchmark

This benchmark shows the reducation in allocations by the copy avoidance
changes.  Together the changes result in:

```
name                                     old time/op    new time/op    delta
SQL/Cockroach/InsertLarge/count=1000-10    18.7ms ± 4%    19.1ms ±26%     ~     (p=0.780 n=9+10)

name                                     old alloc/op   new alloc/op   delta
SQL/Cockroach/InsertLarge/count=1000-10    49.7MB ± 4%    39.7MB ±10%  -20.19%  (p=0.000 n=9+9)

name                                     old allocs/op  new allocs/op  delta
SQL/Cockroach/InsertLarge/count=1000-10     39.0k ± 5%     29.7k ± 8%  -23.83%  (p=0.000 n=10+9)
```

Release note: None
Epic: None


103551: build: make master `v23.2.0-alpha.00000000` r=rickystewart,srosenberg,postamar a=renatolabs

v23.1.1 has been released, `master` should now be considered to be 23.2.

Epic: none

Release note: None

103859: goschedstats: support go1.20 r=rickystewart a=ajwerner

There was one field added to the `p` that matters: needspinning (see golang/go@8cb350d).

All of the relevant symbols and structs were re-copied. Some `uintptr`s were replaced with real pointers, and some things were ported to use new atomic types.

Relates to #96443.

Epic: none

Release note: None

103948: liveness: split liveness cache out r=erikgrinaker a=andrewbaptist

Move the liveness caching methods to a new file.

Only look at the last 2 commits - I will rebase on master once those are merged.

Epic: none

Release note: None


Co-authored-by: Tommy Reilly <[email protected]>
Co-authored-by: Renato Costa <[email protected]>
Co-authored-by: ajwerner <[email protected]>
Co-authored-by: Andrew Baptist <[email protected]>
  • Loading branch information
5 people committed May 31, 2023
5 parents 853a4d2 + e67b8ed + c1776af + 8b33327 + 8235d54 commit 774684d
Show file tree
Hide file tree
Showing 61 changed files with 897 additions and 568 deletions.
31 changes: 31 additions & 0 deletions pkg/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -419,6 +449,7 @@ func BenchmarkSQL(b *testing.B) {
for _, runFn := range []func(*testing.B, *sqlutils.SQLRunner, int){
runBenchmarkDelete,
runBenchmarkInsert,
runBenchmarkInsertLarge,
runBenchmarkInsertDistinct,
runBenchmarkInsertFK,
runBenchmarkInsertSecondaryIndex,
Expand Down
2 changes: 1 addition & 1 deletion pkg/build/version.txt
Original file line number Diff line number Diff line change
@@ -1 +1 @@
v23.1.0-alpha.8
v23.2.0-alpha.00000000
2 changes: 1 addition & 1 deletion pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/zip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
26 changes: 0 additions & 26 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
15 changes: 8 additions & 7 deletions pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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

Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/allocator/allocatorimpl/test_helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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) {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/allocator/storepool/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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)
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down
48 changes: 6 additions & 42 deletions pkg/kv/kvserver/allocator/storepool/store_pool.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down
Loading

0 comments on commit 774684d

Please sign in to comment.