Use zeropool.Pool to workaround SA6002 (#12189)

* Use zeropool.Pool to workaround SA6002

I built a tiny library called https://github.com/colega/zeropool to
workaround the SA6002 staticheck issue.

While searching for the references of that SA6002 staticheck issues on
Github first results was Prometheus itself, with quite a lot of ignores
of it.

This changes the usages of `sync.Pool` to `zeropool.Pool[T]` where a
pointer is not available.

Also added a benchmark for HeadAppender Append/Commit when series
already exist, which is one of the most usual cases IMO, as I didn't find
any.

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Improve BenchmarkHeadAppender with more cases

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* A little copying is better than a little dependency

https://www.youtube.com/watch?v=PAAkCSZUG1c&t=9m28s

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Fix imports order

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Add license header

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Copyright should be on one of the first 3 lines

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Use require.Equal for testing

I don't depend on testify in my lib, but here we have it available.

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Avoid flaky test

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

* Also use zeropool for pointsPool in engine.go

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>

---------

Signed-off-by: Oleg Zaytsev <mail@olegzaytsev.com>
This commit is contained in:
Oleg Zaytsev 2023-03-29 21:34:34 +02:00 committed by GitHub
parent 211ae4f1f0
commit 6e2905a4d4
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
8 changed files with 345 additions and 96 deletions

View file

@ -45,6 +45,7 @@ import (
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/util/stats"
"github.com/prometheus/prometheus/util/zeropool"
)
const (
@ -1794,18 +1795,16 @@ func (ev *evaluator) vectorSelectorSingle(it *storage.MemoizedSeriesIterator, no
return t, v, h, true
}
var pointPool = sync.Pool{}
var pointPool zeropool.Pool[[]Point]
func getPointSlice(sz int) []Point {
p := pointPool.Get()
if p != nil {
return p.([]Point)
if p := pointPool.Get(); p != nil {
return p
}
return make([]Point, 0, sz)
}
func putPointSlice(p []Point) {
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
pointPool.Put(p[:0])
}

View file

@ -44,6 +44,7 @@ import (
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/tsdbutil"
"github.com/prometheus/prometheus/tsdb/wlog"
"github.com/prometheus/prometheus/util/zeropool"
)
var (
@ -83,13 +84,13 @@ type Head struct {
exemplarMetrics *ExemplarMetrics
exemplars ExemplarStorage
logger log.Logger
appendPool sync.Pool
exemplarsPool sync.Pool
histogramsPool sync.Pool
floatHistogramsPool sync.Pool
metadataPool sync.Pool
seriesPool sync.Pool
bytesPool sync.Pool
appendPool zeropool.Pool[[]record.RefSample]
exemplarsPool zeropool.Pool[[]exemplarWithSeriesRef]
histogramsPool zeropool.Pool[[]record.RefHistogramSample]
floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample]
metadataPool zeropool.Pool[[]record.RefMetadata]
seriesPool zeropool.Pool[[]*memSeries]
bytesPool zeropool.Pool[[]byte]
memChunkPool sync.Pool
// All series addressable by their ID or hash.

View file

@ -199,11 +199,10 @@ func (h *Head) getAppendBuffer() []record.RefSample {
if b == nil {
return make([]record.RefSample, 0, 512)
}
return b.([]record.RefSample)
return b
}
func (h *Head) putAppendBuffer(b []record.RefSample) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.appendPool.Put(b[:0])
}
@ -212,7 +211,7 @@ func (h *Head) getExemplarBuffer() []exemplarWithSeriesRef {
if b == nil {
return make([]exemplarWithSeriesRef, 0, 512)
}
return b.([]exemplarWithSeriesRef)
return b
}
func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) {
@ -220,7 +219,6 @@ func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) {
return
}
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.exemplarsPool.Put(b[:0])
}
@ -229,11 +227,10 @@ func (h *Head) getHistogramBuffer() []record.RefHistogramSample {
if b == nil {
return make([]record.RefHistogramSample, 0, 512)
}
return b.([]record.RefHistogramSample)
return b
}
func (h *Head) putHistogramBuffer(b []record.RefHistogramSample) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.histogramsPool.Put(b[:0])
}
@ -242,11 +239,10 @@ func (h *Head) getFloatHistogramBuffer() []record.RefFloatHistogramSample {
if b == nil {
return make([]record.RefFloatHistogramSample, 0, 512)
}
return b.([]record.RefFloatHistogramSample)
return b
}
func (h *Head) putFloatHistogramBuffer(b []record.RefFloatHistogramSample) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.floatHistogramsPool.Put(b[:0])
}
@ -255,11 +251,10 @@ func (h *Head) getMetadataBuffer() []record.RefMetadata {
if b == nil {
return make([]record.RefMetadata, 0, 512)
}
return b.([]record.RefMetadata)
return b
}
func (h *Head) putMetadataBuffer(b []record.RefMetadata) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.metadataPool.Put(b[:0])
}
@ -268,11 +263,10 @@ func (h *Head) getSeriesBuffer() []*memSeries {
if b == nil {
return make([]*memSeries, 0, 512)
}
return b.([]*memSeries)
return b
}
func (h *Head) putSeriesBuffer(b []*memSeries) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.seriesPool.Put(b[:0])
}
@ -281,11 +275,10 @@ func (h *Head) getBytesBuffer() []byte {
if b == nil {
return make([]byte, 0, 1024)
}
return b.([]byte)
return b
}
func (h *Head) putBytesBuffer(b []byte) {
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.bytesPool.Put(b[:0])
}

View file

@ -79,9 +79,9 @@ func newTestHead(t testing.TB, chunkRange int64, compressWAL, oooEnabled bool) (
func BenchmarkCreateSeries(b *testing.B) {
series := genSeries(b.N, 10, 0, 0)
h, _ := newTestHead(b, 10000, false, false)
defer func() {
b.Cleanup(func() {
require.NoError(b, h.Close())
}()
})
b.ReportAllocs()
b.ResetTimer()
@ -91,6 +91,49 @@ func BenchmarkCreateSeries(b *testing.B) {
}
}
func BenchmarkHeadAppender_Append_Commit_ExistingSeries(b *testing.B) {
seriesCounts := []int{100, 1000, 10000}
series := genSeries(10000, 10, 0, 0)
for _, seriesCount := range seriesCounts {
b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) {
for _, samplesPerAppend := range []int64{1, 2, 5, 100} {
b.Run(fmt.Sprintf("%d samples per append", samplesPerAppend), func(b *testing.B) {
h, _ := newTestHead(b, 10000, false, false)
b.Cleanup(func() { require.NoError(b, h.Close()) })
ts := int64(1000)
append := func() error {
var err error
app := h.Appender(context.Background())
for _, s := range series[:seriesCount] {
var ref storage.SeriesRef
for sampleIndex := int64(0); sampleIndex < samplesPerAppend; sampleIndex++ {
ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex))
if err != nil {
return err
}
}
}
ts += 1000 // should increment more than highest samplesPerAppend
return app.Commit()
}
// Init series, that's not what we're benchmarking here.
require.NoError(b, append())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
require.NoError(b, append())
}
})
}
})
}
}
func populateTestWAL(t testing.TB, w *wlog.WL, recs []interface{}) {
var enc record.Encoder
for _, r := range recs {

View file

@ -40,6 +40,7 @@ import (
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wlog"
"github.com/prometheus/prometheus/util/zeropool"
)
// histogramRecord combines both RefHistogramSample and RefFloatHistogramSample
@ -74,41 +75,14 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
decoded = make(chan interface{}, 10)
decodeErr, seriesCreationErr error
seriesPool = sync.Pool{
New: func() interface{} {
return []record.RefSeries{}
},
}
samplesPool = sync.Pool{
New: func() interface{} {
return []record.RefSample{}
},
}
tstonesPool = sync.Pool{
New: func() interface{} {
return []tombstones.Stone{}
},
}
exemplarsPool = sync.Pool{
New: func() interface{} {
return []record.RefExemplar{}
},
}
histogramsPool = sync.Pool{
New: func() interface{} {
return []record.RefHistogramSample{}
},
}
floatHistogramsPool = sync.Pool{
New: func() interface{} {
return []record.RefFloatHistogramSample{}
},
}
metadataPool = sync.Pool{
New: func() interface{} {
return []record.RefMetadata{}
},
}
seriesPool zeropool.Pool[[]record.RefSeries]
samplesPool zeropool.Pool[[]record.RefSample]
tstonesPool zeropool.Pool[[]tombstones.Stone]
exemplarsPool zeropool.Pool[[]record.RefExemplar]
histogramsPool zeropool.Pool[[]record.RefHistogramSample]
floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample]
metadataPool zeropool.Pool[[]record.RefMetadata]
)
defer func() {
@ -167,7 +141,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
rec := r.Record()
switch dec.Type(rec) {
case record.Series:
series := seriesPool.Get().([]record.RefSeries)[:0]
series := seriesPool.Get()[:0]
series, err = dec.Series(rec, series)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -179,7 +153,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- series
case record.Samples:
samples := samplesPool.Get().([]record.RefSample)[:0]
samples := samplesPool.Get()[:0]
samples, err = dec.Samples(rec, samples)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -191,7 +165,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- samples
case record.Tombstones:
tstones := tstonesPool.Get().([]tombstones.Stone)[:0]
tstones := tstonesPool.Get()[:0]
tstones, err = dec.Tombstones(rec, tstones)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -203,7 +177,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- tstones
case record.Exemplars:
exemplars := exemplarsPool.Get().([]record.RefExemplar)[:0]
exemplars := exemplarsPool.Get()[:0]
exemplars, err = dec.Exemplars(rec, exemplars)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -215,7 +189,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- exemplars
case record.HistogramSamples:
hists := histogramsPool.Get().([]record.RefHistogramSample)[:0]
hists := histogramsPool.Get()[:0]
hists, err = dec.HistogramSamples(rec, hists)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -227,7 +201,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- hists
case record.FloatHistogramSamples:
hists := floatHistogramsPool.Get().([]record.RefFloatHistogramSample)[:0]
hists := floatHistogramsPool.Get()[:0]
hists, err = dec.FloatHistogramSamples(rec, hists)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -239,7 +213,7 @@ func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
decoded <- hists
case record.Metadata:
meta := metadataPool.Get().([]record.RefMetadata)[:0]
meta := metadataPool.Get()[:0]
meta, err := dec.Metadata(rec, meta)
if err != nil {
decodeErr = &wlog.CorruptionErr{
@ -278,7 +252,6 @@ Outer:
idx := uint64(mSeries.ref) % uint64(concurrency)
processors[idx].input <- walSubsetProcessorInputItem{walSeriesRef: walSeries.Ref, existingSeries: mSeries}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
seriesPool.Put(v)
case []record.RefSample:
samples := v
@ -315,7 +288,6 @@ Outer:
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
samplesPool.Put(v)
case []tombstones.Stone:
for _, s := range v {
@ -330,13 +302,11 @@ Outer:
h.tombstones.AddInterval(storage.SeriesRef(s.Ref), itv)
}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
tstonesPool.Put(v)
case []record.RefExemplar:
for _, e := range v {
exemplarsInput <- e
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
exemplarsPool.Put(v)
case []record.RefHistogramSample:
samples := v
@ -373,7 +343,6 @@ Outer:
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
histogramsPool.Put(v)
case []record.RefFloatHistogramSample:
samples := v
@ -410,7 +379,6 @@ Outer:
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
floatHistogramsPool.Put(v)
case []record.RefMetadata:
for _, m := range v {
@ -425,7 +393,6 @@ Outer:
Help: m.Help,
}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
metadataPool.Put(v)
default:
panic(fmt.Errorf("unexpected decoded type: %T", d))
@ -793,7 +760,6 @@ func (h *Head) loadWBL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
samplesPool.Put(d)
case []record.RefMmapMarker:
markers := v

View file

@ -38,6 +38,7 @@ import (
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wlog"
"github.com/prometheus/prometheus/util/zeropool"
)
// WALEntryType indicates what data a WAL entry contains.
@ -870,9 +871,9 @@ func (r *walReader) Read(
// Historically, the processing is the bottleneck with reading and decoding using only
// 15% of the CPU.
var (
seriesPool sync.Pool
samplePool sync.Pool
deletePool sync.Pool
seriesPool zeropool.Pool[[]record.RefSeries]
samplePool zeropool.Pool[[]record.RefSample]
deletePool zeropool.Pool[[]tombstones.Stone]
)
donec := make(chan struct{})
datac := make(chan interface{}, 100)
@ -886,19 +887,16 @@ func (r *walReader) Read(
if seriesf != nil {
seriesf(v)
}
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
seriesPool.Put(v[:0])
case []record.RefSample:
if samplesf != nil {
samplesf(v)
}
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
samplePool.Put(v[:0])
case []tombstones.Stone:
if deletesf != nil {
deletesf(v)
}
//nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
deletePool.Put(v[:0])
default:
level.Error(r.logger).Log("msg", "unexpected data type")
@ -915,11 +913,9 @@ func (r *walReader) Read(
// Those should generally be caught by entry decoding before.
switch et {
case WALEntrySeries:
var series []record.RefSeries
if v := seriesPool.Get(); v == nil {
series := seriesPool.Get()
if series == nil {
series = make([]record.RefSeries, 0, 512)
} else {
series = v.([]record.RefSeries)
}
err = r.decodeSeries(flag, b, &series)
@ -936,11 +932,9 @@ func (r *walReader) Read(
}
}
case WALEntrySamples:
var samples []record.RefSample
if v := samplePool.Get(); v == nil {
samples := samplePool.Get()
if samples == nil {
samples = make([]record.RefSample, 0, 512)
} else {
samples = v.([]record.RefSample)
}
err = r.decodeSamples(flag, b, &samples)
@ -958,11 +952,9 @@ func (r *walReader) Read(
}
}
case WALEntryDeletes:
var deletes []tombstones.Stone
if v := deletePool.Get(); v == nil {
deletes := deletePool.Get()
if deletes == nil {
deletes = make([]tombstones.Stone, 0, 512)
} else {
deletes = v.([]tombstones.Stone)
}
err = r.decodeDeletes(flag, b, &deletes)

77
util/zeropool/pool.go Normal file
View file

@ -0,0 +1,77 @@
// Copyright 2023 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//
// Package zeropool provides a zero-allocation type-safe alternative for sync.Pool, used to workaround staticheck SA6002.
// The contents of this package are brought from https://github.com/colega/zeropool because "little copying is better than little dependency".
package zeropool
import "sync"
// Pool is a type-safe pool of items that does not allocate pointers to items.
// That is not entirely true, it does allocate sometimes, but not most of the time,
// just like the usual sync.Pool pools items most of the time, except when they're evicted.
// It does that by storing the allocated pointers in a secondary pool instead of letting them go,
// so they can be used later to store the items again.
//
// Zero value of Pool[T] is valid, and it will return zero values of T if nothing is pooled.
type Pool[T any] struct {
// items holds pointers to the pooled items, which are valid to be used.
items sync.Pool
// pointers holds just pointers to the pooled item types.
// The values referenced by pointers are not valid to be used (as they're used by some other caller)
// and it is safe to overwrite these pointers.
pointers sync.Pool
}
// New creates a new Pool[T] with the given function to create new items.
// A Pool must not be copied after first use.
func New[T any](item func() T) Pool[T] {
return Pool[T]{
items: sync.Pool{
New: func() interface{} {
val := item()
return &val
},
},
}
}
// Get returns an item from the pool, creating a new one if necessary.
// Get may be called concurrently from multiple goroutines.
func (p *Pool[T]) Get() T {
pooled := p.items.Get()
if pooled == nil {
// The only way this can happen is when someone is using the zero-value of zeropool.Pool, and items pool is empty.
// We don't have a pointer to store in p.pointers, so just return the empty value.
var zero T
return zero
}
ptr := pooled.(*T)
item := *ptr // ptr still holds a reference to a copy of item, but nobody will use it.
p.pointers.Put(ptr)
return item
}
// Put adds an item to the pool.
func (p *Pool[T]) Put(item T) {
var ptr *T
if pooled := p.pointers.Get(); pooled != nil {
ptr = pooled.(*T)
} else {
ptr = new(T)
}
*ptr = item
p.items.Put(ptr)
}

178
util/zeropool/pool_test.go Normal file
View file

@ -0,0 +1,178 @@
// Copyright 2023 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zeropool_test
import (
"math"
"sync"
"testing"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
"github.com/prometheus/prometheus/util/zeropool"
)
func TestPool(t *testing.T) {
t.Run("provides correct values", func(t *testing.T) {
pool := zeropool.New(func() []byte { return make([]byte, 1024) })
item1 := pool.Get()
require.Equal(t, 1024, len(item1))
item2 := pool.Get()
require.Equal(t, 1024, len(item2))
pool.Put(item1)
pool.Put(item2)
item1 = pool.Get()
require.Equal(t, 1024, len(item1))
item2 = pool.Get()
require.Equal(t, 1024, len(item2))
})
t.Run("is not racy", func(t *testing.T) {
pool := zeropool.New(func() []byte { return make([]byte, 1024) })
const iterations = 1e6
const concurrency = math.MaxUint8
var counter atomic.Int64
do := make(chan struct{}, 1e6)
for i := 0; i < iterations; i++ {
do <- struct{}{}
}
close(do)
run := make(chan struct{})
done := sync.WaitGroup{}
done.Add(concurrency)
for i := 0; i < concurrency; i++ {
go func(worker int) {
<-run
for range do {
item := pool.Get()
item[0] = byte(worker)
counter.Add(1) // Counts and also adds some delay to add raciness.
if item[0] != byte(worker) {
panic("wrong value")
}
pool.Put(item)
}
done.Done()
}(i)
}
close(run)
done.Wait()
t.Logf("Done %d iterations", counter.Load())
})
t.Run("does not allocate", func(t *testing.T) {
pool := zeropool.New(func() []byte { return make([]byte, 1024) })
// Warm up, this will alloate one slice.
slice := pool.Get()
pool.Put(slice)
allocs := testing.AllocsPerRun(1000, func() {
slice := pool.Get()
pool.Put(slice)
})
// Don't compare to 0, as when passing all the tests the GC could flush the pools during this test and we would allocate.
// Just check that it's less than 1 on average, which is mostly the same thing.
require.Less(t, allocs, 1., "Should not allocate.")
})
t.Run("zero value is valid", func(t *testing.T) {
var pool zeropool.Pool[[]byte]
slice := pool.Get()
pool.Put(slice)
allocs := testing.AllocsPerRun(1000, func() {
slice := pool.Get()
pool.Put(slice)
})
// Don't compare to 0, as when passing all the tests the GC could flush the pools during this test and we would allocate.
// Just check that it's less than 1 on average, which is mostly the same thing.
require.Less(t, allocs, 1., "Should not allocate.")
})
}
func BenchmarkZeropoolPool(b *testing.B) {
pool := zeropool.New(func() []byte { return make([]byte, 1024) })
// Warmup
item := pool.Get()
pool.Put(item)
b.ResetTimer()
for i := 0; i < b.N; i++ {
item := pool.Get()
pool.Put(item)
}
}
// BenchmarkSyncPoolValue uses sync.Pool to store values, which makes an allocation on each Put call.
func BenchmarkSyncPoolValue(b *testing.B) {
pool := sync.Pool{New: func() any {
return make([]byte, 1024)
}}
// Warmup
item := pool.Get().([]byte)
pool.Put(item) //nolint:staticcheck // This allocates.
b.ResetTimer()
for i := 0; i < b.N; i++ {
item := pool.Get().([]byte)
pool.Put(item) //nolint:staticcheck // This allocates.
}
}
// BenchmarkSyncPoolNewPointer uses sync.Pool to store pointers, but it calls Put with a new pointer every time.
func BenchmarkSyncPoolNewPointer(b *testing.B) {
pool := sync.Pool{New: func() any {
v := make([]byte, 1024)
return &v
}}
// Warmup
item := pool.Get().(*[]byte)
pool.Put(item) //nolint:staticcheck // This allocates.
b.ResetTimer()
for i := 0; i < b.N; i++ {
item := pool.Get().(*[]byte)
buf := *item
pool.Put(&buf) //nolint:staticcheck // New pointer.
}
}
// BenchmarkSyncPoolPointer illustrates the optimal usage of sync.Pool, not always possible.
func BenchmarkSyncPoolPointer(b *testing.B) {
pool := sync.Pool{New: func() any {
v := make([]byte, 1024)
return &v
}}
// Warmup
item := pool.Get().(*[]byte)
pool.Put(item)
b.ResetTimer()
for i := 0; i < b.N; i++ {
item := pool.Get().(*[]byte)
pool.Put(item)
}
}