allow pre-allocating buffers to reduce frequent GCs during growth (#18686)

This PR also increases per node bpool memory from 1024 entries
to 2048 entries; along with that, it also moves the byte pool
centrally instead of being per pool.
This commit is contained in:
Harshavardhana 2023-12-21 08:59:38 -08:00 committed by GitHub
parent 56b7045c20
commit 7c948adf88
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 88 additions and 68 deletions

View File

@ -355,10 +355,8 @@ func buildServerCtxt(ctx *cli.Context, ctxt *serverCtxt) (err error) {
} }
// Check "no-compat" flag from command line argument. // Check "no-compat" flag from command line argument.
ctxt.StrictS3Compat = true ctxt.StrictS3Compat = !(ctx.IsSet("no-compat") || ctx.GlobalIsSet("no-compat"))
if ctx.IsSet("no-compat") || ctx.GlobalIsSet("no-compat") { ctxt.PreAllocate = ctx.IsSet("pre-allocate") || ctx.GlobalIsSet("pre-allocate")
ctxt.StrictS3Compat = false
}
switch { switch {
case ctx.IsSet("config-dir"): case ctx.IsSet("config-dir"):

View File

@ -662,12 +662,12 @@ func (er erasureObjects) PutObjectPart(ctx context.Context, bucket, object, uplo
// Account for padding and forced compression overhead and encryption. // Account for padding and forced compression overhead and encryption.
buffer = make([]byte, data.ActualSize()+256+32+32, data.ActualSize()*2+512) buffer = make([]byte, data.ActualSize()+256+32+32, data.ActualSize()*2+512)
} else { } else {
buffer = er.bp.Get() buffer = globalBytePoolCap.Get()
defer er.bp.Put(buffer) defer globalBytePoolCap.Put(buffer)
} }
case size >= fi.Erasure.BlockSize: case size >= fi.Erasure.BlockSize:
buffer = er.bp.Get() buffer = globalBytePoolCap.Get()
defer er.bp.Put(buffer) defer globalBytePoolCap.Put(buffer)
case size < fi.Erasure.BlockSize: case size < fi.Erasure.BlockSize:
// No need to allocate fully fi.Erasure.BlockSize buffer if the incoming data is smaller. // No need to allocate fully fi.Erasure.BlockSize buffer if the incoming data is smaller.
buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1)) buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1))
@ -688,10 +688,10 @@ func (er erasureObjects) PutObjectPart(ctx context.Context, bucket, object, uplo
if data.Size() > bigFileThreshold { if data.Size() > bigFileThreshold {
// Add input readahead. // Add input readahead.
// We use 2 buffers, so we always have a full buffer of input. // We use 2 buffers, so we always have a full buffer of input.
bufA := er.bp.Get() bufA := globalBytePoolCap.Get()
bufB := er.bp.Get() bufB := globalBytePoolCap.Get()
defer er.bp.Put(bufA) defer globalBytePoolCap.Put(bufA)
defer er.bp.Put(bufB) defer globalBytePoolCap.Put(bufB)
ra, err := readahead.NewReaderBuffer(data, [][]byte{bufA[:fi.Erasure.BlockSize], bufB[:fi.Erasure.BlockSize]}) ra, err := readahead.NewReaderBuffer(data, [][]byte{bufA[:fi.Erasure.BlockSize], bufB[:fi.Erasure.BlockSize]})
if err == nil { if err == nil {
toEncode = ra toEncode = ra

View File

@ -1127,8 +1127,8 @@ func (er erasureObjects) putMetacacheObject(ctx context.Context, key string, r *
case size == 0: case size == 0:
buffer = make([]byte, 1) // Allocate atleast a byte to reach EOF buffer = make([]byte, 1) // Allocate atleast a byte to reach EOF
case size >= fi.Erasure.BlockSize: case size >= fi.Erasure.BlockSize:
buffer = er.bp.Get() buffer = globalBytePoolCap.Get()
defer er.bp.Put(buffer) defer globalBytePoolCap.Put(buffer)
case size < fi.Erasure.BlockSize: case size < fi.Erasure.BlockSize:
// No need to allocate fully blockSizeV1 buffer if the incoming data is smaller. // No need to allocate fully blockSizeV1 buffer if the incoming data is smaller.
buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1)) buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1))
@ -1378,8 +1378,8 @@ func (er erasureObjects) putObject(ctx context.Context, bucket string, object st
case size == 0: case size == 0:
buffer = make([]byte, 1) // Allocate atleast a byte to reach EOF buffer = make([]byte, 1) // Allocate atleast a byte to reach EOF
case size >= fi.Erasure.BlockSize || size == -1: case size >= fi.Erasure.BlockSize || size == -1:
buffer = er.bp.Get() buffer = globalBytePoolCap.Get()
defer er.bp.Put(buffer) defer globalBytePoolCap.Put(buffer)
case size < fi.Erasure.BlockSize: case size < fi.Erasure.BlockSize:
// No need to allocate fully blockSizeV1 buffer if the incoming data is smaller. // No need to allocate fully blockSizeV1 buffer if the incoming data is smaller.
buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1)) buffer = make([]byte, size, 2*size+int64(fi.Erasure.ParityBlocks+fi.Erasure.DataBlocks-1))
@ -1438,10 +1438,10 @@ func (er erasureObjects) putObject(ctx context.Context, bucket string, object st
toEncode := io.Reader(data) toEncode := io.Reader(data)
if data.Size() > bigFileThreshold { if data.Size() > bigFileThreshold {
// We use 2 buffers, so we always have a full buffer of input. // We use 2 buffers, so we always have a full buffer of input.
bufA := er.bp.Get() bufA := globalBytePoolCap.Get()
bufB := er.bp.Get() bufB := globalBytePoolCap.Get()
defer er.bp.Put(bufA) defer globalBytePoolCap.Put(bufA)
defer er.bp.Put(bufB) defer globalBytePoolCap.Put(bufB)
ra, err := readahead.NewReaderBuffer(data, [][]byte{bufA[:fi.Erasure.BlockSize], bufB[:fi.Erasure.BlockSize]}) ra, err := readahead.NewReaderBuffer(data, [][]byte{bufA[:fi.Erasure.BlockSize], bufB[:fi.Erasure.BlockSize]})
if err == nil { if err == nil {
toEncode = ra toEncode = ra

View File

@ -38,6 +38,7 @@ import (
"github.com/minio/minio-go/v7/pkg/s3utils" "github.com/minio/minio-go/v7/pkg/s3utils"
"github.com/minio/minio-go/v7/pkg/set" "github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio-go/v7/pkg/tags" "github.com/minio/minio-go/v7/pkg/tags"
"github.com/minio/minio/internal/bpool"
"github.com/minio/minio/internal/config/storageclass" "github.com/minio/minio/internal/config/storageclass"
"github.com/minio/minio/internal/logger" "github.com/minio/minio/internal/logger"
"github.com/minio/pkg/v2/sync/errgroup" "github.com/minio/pkg/v2/sync/errgroup"
@ -82,6 +83,21 @@ func newErasureServerPools(ctx context.Context, endpointServerPools EndpointServ
} }
) )
// Maximum number of reusable buffers per node at any given point in time.
n := 1024 // single node single/multiple drives set this to 1024 entries
if globalIsDistErasure {
n = 2048
}
// Initialize byte pool once for all sets, bpool size is set to
// setCount * setDriveCount with each memory upto blockSizeV2.
globalBytePoolCap = bpool.NewBytePoolCap(n, blockSizeV2, blockSizeV2*2)
if globalServerCtxt.PreAllocate {
globalBytePoolCap.Populate()
}
var localDrives []StorageAPI var localDrives []StorageAPI
local := endpointServerPools.FirstLocal() local := endpointServerPools.FirstLocal()
for i, ep := range endpointServerPools { for i, ep := range endpointServerPools {

View File

@ -36,7 +36,6 @@ import (
"github.com/minio/madmin-go/v3" "github.com/minio/madmin-go/v3"
"github.com/minio/minio-go/v7/pkg/set" "github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio-go/v7/pkg/tags" "github.com/minio/minio-go/v7/pkg/tags"
"github.com/minio/minio/internal/bpool"
"github.com/minio/minio/internal/dsync" "github.com/minio/minio/internal/dsync"
"github.com/minio/minio/internal/logger" "github.com/minio/minio/internal/logger"
"github.com/minio/pkg/v2/console" "github.com/minio/pkg/v2/console"
@ -369,21 +368,6 @@ func newErasureSets(ctx context.Context, endpoints PoolEndpoints, storageDisks [
mutex := newNSLock(globalIsDistErasure) mutex := newNSLock(globalIsDistErasure)
// Number of buffers, max 2GB
n := (2 * humanize.GiByte) / (blockSizeV2 * 2)
// Initialize byte pool once for all sets, bpool size is set to
// setCount * setDriveCount with each memory upto blockSizeV2.
bp := bpool.NewBytePoolCap(n, blockSizeV2, blockSizeV2*2)
// Initialize byte pool for all sets, bpool size is set to
// setCount * setDriveCount with each memory upto blockSizeV1
//
// Number of buffers, max 10GiB
m := (10 * humanize.GiByte) / (blockSizeV1 * 2)
bpOld := bpool.NewBytePoolCap(m, blockSizeV1, blockSizeV1*2)
for i := 0; i < setCount; i++ { for i := 0; i < setCount; i++ {
s.erasureDisks[i] = make([]StorageAPI, setDriveCount) s.erasureDisks[i] = make([]StorageAPI, setDriveCount)
} }
@ -459,8 +443,6 @@ func newErasureSets(ctx context.Context, endpoints PoolEndpoints, storageDisks [
getLockers: s.GetLockers(i), getLockers: s.GetLockers(i),
getEndpoints: s.GetEndpoints(i), getEndpoints: s.GetEndpoints(i),
nsMutex: mutex, nsMutex: mutex,
bp: bp,
bpOld: bpOld,
} }
}(i) }(i)
} }

View File

@ -29,7 +29,6 @@ import (
"time" "time"
"github.com/minio/madmin-go/v3" "github.com/minio/madmin-go/v3"
"github.com/minio/minio/internal/bpool"
"github.com/minio/minio/internal/dsync" "github.com/minio/minio/internal/dsync"
xioutil "github.com/minio/minio/internal/ioutil" xioutil "github.com/minio/minio/internal/ioutil"
"github.com/minio/minio/internal/logger" "github.com/minio/minio/internal/logger"
@ -65,13 +64,6 @@ type erasureObjects struct {
// Locker mutex map. // Locker mutex map.
nsMutex *nsLockMap nsMutex *nsLockMap
// Byte pools used for temporary i/o buffers.
bp *bpool.BytePoolCap
// Byte pools used for temporary i/o buffers,
// legacy objects.
bpOld *bpool.BytePoolCap
} }
// NewNSLock - initialize a new namespace RWLocker instance. // NewNSLock - initialize a new namespace RWLocker instance.

View File

@ -30,6 +30,7 @@ import (
"github.com/minio/madmin-go/v3" "github.com/minio/madmin-go/v3"
"github.com/minio/minio-go/v7" "github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/set" "github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio/internal/bpool"
"github.com/minio/minio/internal/bucket/bandwidth" "github.com/minio/minio/internal/bucket/bandwidth"
"github.com/minio/minio/internal/config" "github.com/minio/minio/internal/config"
"github.com/minio/minio/internal/handlers" "github.com/minio/minio/internal/handlers"
@ -141,6 +142,7 @@ type serverCtxt struct {
JSON, Quiet bool JSON, Quiet bool
Anonymous bool Anonymous bool
StrictS3Compat bool StrictS3Compat bool
PreAllocate bool
Addr, ConsoleAddr string Addr, ConsoleAddr string
ConfigDir, CertsDir string ConfigDir, CertsDir string
configDirSet, certsDirSet bool configDirSet, certsDirSet bool
@ -225,6 +227,7 @@ var (
globalBucketMonitor *bandwidth.Monitor globalBucketMonitor *bandwidth.Monitor
globalPolicySys *PolicySys globalPolicySys *PolicySys
globalIAMSys *IAMSys globalIAMSys *IAMSys
globalBytePoolCap *bpool.BytePoolCap
globalLifecycleSys *LifecycleSys globalLifecycleSys *LifecycleSys
globalBucketSSEConfigSys *BucketSSEConfigSys globalBucketSSEConfigSys *BucketSSEConfigSys

View File

@ -1982,20 +1982,21 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
md5hex = "" // Do not try to verify the content. md5hex = "" // Do not try to verify the content.
sha256hex = "" sha256hex = ""
} }
var hashReader *hash.Reader
// Optimization: If SSE-KMS and SSE-C did not request Content-Md5. Use uuid as etag var forceMD5 []byte
if !etag.ContentMD5Requested(r.Header) && (crypto.S3KMS.IsRequested(r.Header) || crypto.SSEC.IsRequested(r.Header)) { // Optimization: If SSE-KMS and SSE-C did not request Content-Md5. Use uuid as etag. Optionally enable this also
hashReader, err = hash.NewReaderWithOpts(ctx, reader, hash.Options{ // for server that is started with `--no-compat`.
Size: size, if !etag.ContentMD5Requested(r.Header) && (crypto.S3KMS.IsRequested(r.Header) || crypto.SSEC.IsRequested(r.Header) || !globalServerCtxt.StrictS3Compat) {
MD5Hex: md5hex, forceMD5 = mustGetUUIDBytes()
SHA256Hex: sha256hex,
ActualSize: actualSize,
DisableMD5: false,
ForceMD5: mustGetUUIDBytes(),
})
} else {
hashReader, err = hash.NewReader(ctx, reader, size, md5hex, sha256hex, actualSize)
} }
hashReader, err := hash.NewReaderWithOpts(ctx, reader, hash.Options{
Size: size,
MD5Hex: md5hex,
SHA256Hex: sha256hex,
ActualSize: actualSize,
DisableMD5: false,
ForceMD5: forceMD5,
})
if err != nil { if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return return

View File

@ -728,7 +728,14 @@ func (api objectAPIHandlers) PutObjectPartHandler(w http.ResponseWriter, r *http
sha256hex = "" sha256hex = ""
} }
hashReader, err := hash.NewReader(ctx, reader, size, md5hex, sha256hex, actualSize) hashReader, err := hash.NewReaderWithOpts(ctx, reader, hash.Options{
Size: size,
MD5Hex: md5hex,
SHA256Hex: sha256hex,
ActualSize: actualSize,
DisableMD5: false,
ForceMD5: nil,
})
if err != nil { if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return return

View File

@ -74,6 +74,12 @@ var ServerFlags = []cli.Flag{
EnvVar: "MINIO_LISTENERS", EnvVar: "MINIO_LISTENERS",
Hidden: true, Hidden: true,
}, },
cli.BoolFlag{
Name: "pre-allocate",
Usage: "Number of 1MiB sized buffers to pre-allocate. Default 2048",
EnvVar: "MINIO_PRE_ALLOCATE",
Hidden: true,
},
cli.StringFlag{ cli.StringFlag{
Name: "console-address", Name: "console-address",
Usage: "bind to a specific ADDRESS:PORT for embedded Console UI, ADDRESS can be an IP or hostname", Usage: "bind to a specific ADDRESS:PORT for embedded Console UI, ADDRESS can be an IP or hostname",

View File

@ -1,4 +1,4 @@
// Copyright (c) 2015-2021 MinIO, Inc. // Copyright (c) 2015-2023 MinIO, Inc.
// //
// This file is part of MinIO Object Storage stack // This file is part of MinIO Object Storage stack
// //
@ -17,6 +17,8 @@
package bpool package bpool
import "github.com/klauspost/reedsolomon"
// BytePoolCap implements a leaky pool of []byte in the form of a bounded channel. // BytePoolCap implements a leaky pool of []byte in the form of a bounded channel.
type BytePoolCap struct { type BytePoolCap struct {
c chan []byte c chan []byte
@ -27,6 +29,12 @@ type BytePoolCap struct {
// NewBytePoolCap creates a new BytePool bounded to the given maxSize, with new // NewBytePoolCap creates a new BytePool bounded to the given maxSize, with new
// byte arrays sized based on width. // byte arrays sized based on width.
func NewBytePoolCap(maxSize int, width int, capwidth int) (bp *BytePoolCap) { func NewBytePoolCap(maxSize int, width int, capwidth int) (bp *BytePoolCap) {
if capwidth > 0 && capwidth < 64 {
panic("buffer capped with smaller than 64 bytes is not supported")
}
if capwidth > 0 && width > capwidth {
panic("buffer length cannot be > capacity of the buffer")
}
return &BytePoolCap{ return &BytePoolCap{
c: make(chan []byte, maxSize), c: make(chan []byte, maxSize),
w: width, w: width,
@ -34,18 +42,25 @@ func NewBytePoolCap(maxSize int, width int, capwidth int) (bp *BytePoolCap) {
} }
} }
// Populate - populates and pre-warms the byte pool, this function is non-blocking.
func (bp *BytePoolCap) Populate() {
for _, buf := range reedsolomon.AllocAligned(cap(bp.c), bp.wcap) {
bp.Put(buf[:bp.w])
}
}
// Get gets a []byte from the BytePool, or creates a new one if none are // Get gets a []byte from the BytePool, or creates a new one if none are
// available in the pool. // available in the pool.
func (bp *BytePoolCap) Get() (b []byte) { func (bp *BytePoolCap) Get() (b []byte) {
select { select {
case b = <-bp.c: case b = <-bp.c:
// reuse existing buffer // reuse existing buffer
default: default:
// create new buffer // create new aligned buffer
if bp.wcap > 0 { if bp.wcap > 0 {
b = make([]byte, bp.w, bp.wcap) b = reedsolomon.AllocAligned(1, bp.wcap)[0][:bp.w]
} else { } else {
b = make([]byte, bp.w) b = reedsolomon.AllocAligned(1, bp.w)[0]
} }
} }
return return

View File

@ -22,8 +22,8 @@ import "testing"
// Tests - bytePool functionality. // Tests - bytePool functionality.
func TestBytePool(t *testing.T) { func TestBytePool(t *testing.T) {
size := 4 size := 4
width := 10 width := 1024
capWidth := 16 capWidth := 2048
bufPool := NewBytePoolCap(size, width, capWidth) bufPool := NewBytePoolCap(size, width, capWidth)
@ -43,7 +43,7 @@ func TestBytePool(t *testing.T) {
t.Fatalf("bytepool length invalid: got %v want %v", len(b), width) t.Fatalf("bytepool length invalid: got %v want %v", len(b), width)
} }
if cap(b) != capWidth { if cap(b) != capWidth {
t.Fatalf("bytepool length invalid: got %v want %v", cap(b), capWidth) t.Fatalf("bytepool cap invalid: got %v want %v", cap(b), capWidth)
} }
bufPool.Put(b) bufPool.Put(b)