Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
136780: stats: use no-op DatumAlloc when decoding EncDatums r=yuzefovich a=yuzefovich

**tree: make nil DatumAlloc act as a no-op**

This commit makes so that `nil` `DatumAlloc` value makes the alloc struct a no-op, i.e. each datum type passed by value results in a separate allocation. This can be useful when the callers will keep only a subset of all allocated datums.

Release note: None

**stats: use no-op DatumAlloc when decoding EncDatums**

This commit fixes a bounded memory leak that could previously occur due to usage of the DatumAlloc when decoding EncDatums into `tree.Datum`s during table stats collection. We have two use cases for decoded `tree.Datum`s:
- we need it when adding _all_ rows into the sketch (we compute the datum's fingerprint ("hash") to use in the distinct count estimation). This usage is very brief and we don't need the decoded datum afterwards.
- we also need it when sampling _some_ rows when we decide to keep a particular row. In this case, the datum is needed throughout the whole stats collection job (or until it's replaced by another sample) for the histogram bucket computation.

The main observation is that the DatumAlloc allocates datums in batches of 16 objects, and even if at least one of the objects is kept by the sample, then all others from the same batch are as well. We only perform memory accounting for the ones we explicitly keep, yet others would be considered live by the Go runtime, resulting in a bounded memory leak. This behavior has been present since forever, but it became more problematic in 24.2 release with the introduction of dynamically computing the sample size. To go around this problematic behavior this commit uses `nil` DatumAlloc which makes it so that every decoded datum incurs a separate allocation. This will have a minor performance hit and increase in total number of allocations, but at least most of them should be short-lived. Alternatively, we could use an "operating" DatumAlloc during fingerprinting and a no-op during the sampling, but we'd need to explicitly nil out the decoded datum after fingerprinting which would result in decoding some datums twice.

Fixes: #136394.

Release note: None

136901: kvserver: deflake TestStoreRangeMergeConcurrentRequests r=iskettaneh a=iskettaneh

Fixes: #136774

Release note: None

136994: kvserver: deflake TestLeaseRequestFromExpirationToEpochOrLeaderDoesNotRegressExpiration r=iskettaneh a=iskettaneh

This commit deflakes
TestLeaseRequestFromExpirationToEpochOrLeaderDoesNotRegressExpiration by establishing the node liveness record before pausing node liveness heartbeats.

Fixes: #136549

Release Note: None

137039: sql/tests: add `ANALYZE` to sysbench microbenchmark r=mgartner a=mgartner

Collecting table statistics manually during the benchmark setup makes
the performance more representative of a real-world workload without the
variance in results that automatic stats collection would introduce.

Epic: None

Release note: None


137049: teamcity: remove `bench` scripts r=jlinder a=rickystewart

These jobs are very flaky and extremely time-consuming at about an hour per run. Simply delete the scripts.

Closes: #136470
Epic: none
Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Ibrahim Kettaneh <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
5 people committed Dec 9, 2024
6 parents 7f75f3e + 2831511 + 5ebd0b3 + 714529c + 5e7a5b1 + 42e1e79 commit 4be77ab
Show file tree
Hide file tree
Showing 11 changed files with 185 additions and 81 deletions.
11 changes: 0 additions & 11 deletions build/teamcity/cockroach/ci/tests-aws-linux-arm64/bench.sh

This file was deleted.

18 changes: 0 additions & 18 deletions build/teamcity/cockroach/ci/tests/bench.sh

This file was deleted.

39 changes: 0 additions & 39 deletions build/teamcity/cockroach/ci/tests/bench_impl.sh

This file was deleted.

17 changes: 17 additions & 0 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1811,6 +1811,23 @@ func TestLeaseRequestFromExpirationToEpochOrLeaderDoesNotRegressExpiration(t *te
// Pause n1's node liveness heartbeats, to allow its liveness expiration to
// fall behind.
l0 := tc.Server(0).NodeLiveness().(*liveness.NodeLiveness)

// Wait for the node liveness record to have an epoch set. Otherwise,
// the test might fail later when attempting to acquire an epoch-based
// lease with this error: unable to get liveness record. This can happen
// if we pause the node liveness heartbeat before the record is even
// created.
testutils.SucceedsSoon(t, func() error {
l, ok := l0.GetLiveness(tc.Server(0).NodeID())
if !ok {
return errors.New("liveness not found")
}
if l.Epoch == 0 {
return errors.New("liveness epoch not set")
}
return nil
})

l0.PauseHeartbeatLoopForTest()
l, ok := l0.GetLiveness(tc.Server(0).NodeID())
require.True(t, ok)
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2331,9 +2331,10 @@ func TestStoreRangeMergeConcurrentRequests(t *testing.T) {
}
}

// Failures in this test often present as a deadlock. Set a short timeout to
// limit the damage, but a longer deadline for race builds.
ctx, cancel := context.WithTimeout(ctx, testutils.SucceedsSoonDuration())
// Failures in this test often present as a deadlock.
// We have a relatively high timeout since this test flakes in leader leases,
// as it keeps withdrawing/granting store liveness support.
ctx, cancel := context.WithTimeout(ctx, 3*testutils.DefaultSucceedsSoonDuration)
defer cancel()

const numGetWorkers = 16
Expand Down
24 changes: 17 additions & 7 deletions pkg/sql/rowexec/sampler.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,6 @@ func (s *samplerProcessor) mainLoop(
ctx context.Context, output execinfra.RowReceiver,
) (earlyExit bool, err error) {
rng, _ := randutil.NewPseudoRand()
var da tree.DatumAlloc
var buf []byte
rowCount := 0
lastWakeupTime := timeutil.Now()
Expand Down Expand Up @@ -316,7 +315,7 @@ func (s *samplerProcessor) mainLoop(
}

for i := range s.sketches {
if err := s.sketches[i].addRow(ctx, row, s.outTypes, &buf, &da); err != nil {
if err := s.sketches[i].addRow(ctx, row, s.outTypes, &buf); err != nil {
return false, err
}
}
Expand All @@ -325,7 +324,7 @@ func (s *samplerProcessor) mainLoop(
}

for col, invSr := range s.invSr {
if err := row[col].EnsureDecoded(s.outTypes[col], &da); err != nil {
if err := row[col].EnsureDecoded(s.outTypes[col], nil /* da */); err != nil {
return false, err
}

Expand All @@ -345,8 +344,9 @@ func (s *samplerProcessor) mainLoop(
return false, err
}
for _, key := range invKeys {
invRow[0].Datum = da.NewDBytes(tree.DBytes(key))
if err := s.invSketch[col].addRow(ctx, invRow, bytesRowType, &buf, &da); err != nil {
d := tree.DBytes(key)
invRow[0].Datum = &d
if err := s.invSketch[col].addRow(ctx, invRow, bytesRowType, &buf); err != nil {
return false, err
}
if earlyExit, err = s.sampleRow(ctx, output, invSr, invRow, rng); earlyExit || err != nil {
Expand Down Expand Up @@ -502,7 +502,7 @@ func (s *samplerProcessor) DoesNotUseTxn() bool {

// addRow adds a row to the sketch and updates row counts.
func (s *sketchInfo) addRow(
ctx context.Context, row rowenc.EncDatumRow, typs []*types.T, buf *[]byte, da *tree.DatumAlloc,
ctx context.Context, row rowenc.EncDatumRow, typs []*types.T, buf *[]byte,
) error {
var err error
s.numRows++
Expand Down Expand Up @@ -551,9 +551,19 @@ func (s *sketchInfo) addRow(
isNull := true
*buf = (*buf)[:0]
for _, col := range s.spec.Columns {
// We pass nil DatumAlloc so that each datum allocation was independent
// (to prevent bounded memory leaks like we've seen in #136394).
// TODO(yuzefovich): the problem in that issue was that the same backing
// slice of datums was shared across rows, so if a single row was kept
// as a sample, it could keep many garbage alive. To go around that we
// simply disabled the batching. We could improve that behavior by using
// a DatumAlloc in which we set typeAllocSizes in such a way that all
// columns of the same type in a single row would be backed by a single
// slice allocation.
//
// We choose to not perform the memory accounting for possibly decoded
// tree.Datum because we will lose the references to row very soon.
*buf, err = row[col].Fingerprint(ctx, typs[col], da, *buf, nil /* acc */)
*buf, err = row[col].Fingerprint(ctx, typs[col], nil /* da */, *buf, nil /* acc */)
if err != nil {
return err
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,7 @@ go_test(
"compare_test.go",
"constant_test.go",
"create_routine_test.go",
"datum_alloc_test.go",
"datum_integration_test.go",
"datum_invariants_test.go",
"datum_prev_next_test.go",
Expand Down Expand Up @@ -236,6 +237,7 @@ go_test(
"//pkg/sql/colconv",
"//pkg/sql/parser",
"//pkg/sql/randgen",
"//pkg/sql/rowenc/valueside",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/catid",
"//pkg/sql/sem/eval",
Expand Down
Loading

0 comments on commit 4be77ab

Please sign in to comment.