mirror of
https://github.com/minio/minio.git
synced 2024-12-24 06:05:55 -05:00
optimize memory allocs during reconstruct (#4964)
The reedsolomon library now avoids allocations during reconstruction. This change exploits that to reduce memory allocs and GC preasure during healing and reading.
This commit is contained in:
parent
4879cd73f8
commit
02af37a394
@ -70,6 +70,10 @@ func (s ErasureStorage) HealFile(staleDisks []StorageAPI, volume, path string,
|
||||
// Scan part files on disk, block-by-block reconstruct it and
|
||||
// write to stale disks.
|
||||
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
||||
blocks := make([][]byte, len(s.disks))
|
||||
for i := range blocks {
|
||||
blocks[i] = make([]byte, chunksize)
|
||||
}
|
||||
var chunkOffset, blockOffset int64
|
||||
for ; blockOffset < size; blockOffset += blocksize {
|
||||
// last iteration may have less than blocksize data
|
||||
@ -77,40 +81,35 @@ func (s ErasureStorage) HealFile(staleDisks []StorageAPI, volume, path string,
|
||||
if size < blockOffset+blocksize {
|
||||
blocksize = size - blockOffset
|
||||
chunksize = getChunkSize(blocksize, s.dataBlocks)
|
||||
for i := range blocks {
|
||||
blocks[i] = blocks[i][:chunksize]
|
||||
}
|
||||
}
|
||||
|
||||
// read a chunk from each disk, until we have
|
||||
// `s.dataBlocks` number of chunks set to non-nil in
|
||||
// `blocks`
|
||||
blocks := make([][]byte, len(s.disks))
|
||||
var buffer []byte
|
||||
numReads := 0
|
||||
for i, disk := range s.disks {
|
||||
// skip reading from unavailable or stale disks
|
||||
if disk == nil || staleDisks[i] != nil {
|
||||
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||
continue
|
||||
}
|
||||
// allocate buffer only when needed - when
|
||||
// reads fail, the buffer can be reused
|
||||
if int64(len(buffer)) != chunksize {
|
||||
buffer = make([]byte, chunksize)
|
||||
}
|
||||
_, err = disk.ReadFile(volume, path, chunkOffset, buffer, verifiers[i])
|
||||
_, err = disk.ReadFile(volume, path, chunkOffset, blocks[i], verifiers[i])
|
||||
if err != nil {
|
||||
// LOG FIXME: add a conditional log
|
||||
// for read failures, once per-disk
|
||||
// per-function-invocation.
|
||||
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||
continue
|
||||
}
|
||||
|
||||
// read was successful, so set the buffer as
|
||||
// blocks[i], and reset buffer to nil to force
|
||||
// allocation on next iteration
|
||||
blocks[i], buffer = buffer, nil
|
||||
|
||||
numReads++
|
||||
if numReads == s.dataBlocks {
|
||||
// we have enough data to reconstruct
|
||||
// mark all other blocks as missing
|
||||
for j := i + 1; j < len(blocks); j++ {
|
||||
blocks[j] = blocks[j][:0] // mark shard as missing
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
@ -18,14 +18,12 @@ package cmd
|
||||
|
||||
import (
|
||||
"io"
|
||||
|
||||
"github.com/minio/minio/pkg/bpool"
|
||||
)
|
||||
|
||||
// ReadFile reads as much data as requested from the file under the given volume and path and writes the data to the provided writer.
|
||||
// The algorithm and the keys/checksums are used to verify the integrity of the given file. ReadFile will read data from the given offset
|
||||
// up to the given length. If parts of the file are corrupted ReadFile tries to reconstruct the data.
|
||||
func (s ErasureStorage) ReadFile(writer io.Writer, volume, path string, offset, length int64, totalLength int64, checksums [][]byte, algorithm BitrotAlgorithm, blocksize int64, pool *bpool.BytePool) (f ErasureFileInfo, err error) {
|
||||
func (s ErasureStorage) ReadFile(writer io.Writer, volume, path string, offset, length int64, totalLength int64, checksums [][]byte, algorithm BitrotAlgorithm, blocksize int64) (f ErasureFileInfo, err error) {
|
||||
if offset < 0 || length < 0 {
|
||||
return f, traceError(errUnexpected)
|
||||
}
|
||||
@ -53,15 +51,20 @@ func (s ErasureStorage) ReadFile(writer io.Writer, volume, path string, offset,
|
||||
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
||||
|
||||
blocks := make([][]byte, len(s.disks))
|
||||
for i := range blocks {
|
||||
blocks[i] = make([]byte, chunksize)
|
||||
}
|
||||
for off := offset / blocksize; length > 0; off++ {
|
||||
blockOffset := off * chunksize
|
||||
pool.Reset()
|
||||
|
||||
if currentBlock := (offset + f.Size) / blocksize; currentBlock == lastBlock {
|
||||
blocksize = totalLength % blocksize
|
||||
chunksize = getChunkSize(blocksize, s.dataBlocks)
|
||||
for i := range blocks {
|
||||
blocks[i] = blocks[i][:chunksize]
|
||||
}
|
||||
}
|
||||
err = s.readConcurrent(volume, path, blockOffset, chunksize, blocks, verifiers, errChans, pool)
|
||||
err = s.readConcurrent(volume, path, blockOffset, blocks, verifiers, errChans)
|
||||
if err != nil {
|
||||
return f, traceError(errXLReadQuorum)
|
||||
}
|
||||
@ -92,7 +95,7 @@ func (s ErasureStorage) ReadFile(writer io.Writer, volume, path string, offset,
|
||||
func erasureCountMissingBlocks(blocks [][]byte, limit int) int {
|
||||
missing := 0
|
||||
for i := range blocks[:limit] {
|
||||
if blocks[i] == nil {
|
||||
if len(blocks[i]) == 0 {
|
||||
missing++
|
||||
}
|
||||
}
|
||||
@ -101,15 +104,8 @@ func erasureCountMissingBlocks(blocks [][]byte, limit int) int {
|
||||
|
||||
// readConcurrent reads all requested data concurrently from the disks into blocks. It returns an error if
|
||||
// too many disks failed while reading.
|
||||
func (s *ErasureStorage) readConcurrent(volume, path string, offset int64, length int64, blocks [][]byte, verifiers []*BitrotVerifier, errChans []chan error, pool *bpool.BytePool) (err error) {
|
||||
func (s *ErasureStorage) readConcurrent(volume, path string, offset int64, blocks [][]byte, verifiers []*BitrotVerifier, errChans []chan error) (err error) {
|
||||
errs := make([]error, len(s.disks))
|
||||
for i := range blocks {
|
||||
blocks[i], err = pool.Get()
|
||||
if err != nil {
|
||||
return traceErrorf("failed to get new buffer from pool: %v", err)
|
||||
}
|
||||
blocks[i] = blocks[i][:length]
|
||||
}
|
||||
|
||||
erasureReadBlocksConcurrent(s.disks[:s.dataBlocks], volume, path, offset, blocks[:s.dataBlocks], verifiers[:s.dataBlocks], errs[:s.dataBlocks], errChans[:s.dataBlocks])
|
||||
missingDataBlocks := erasureCountMissingBlocks(blocks, s.dataBlocks)
|
||||
@ -145,7 +141,7 @@ func erasureReadBlocksConcurrent(disks []StorageAPI, volume, path string, offset
|
||||
errors[i] = <-errChans[i] // blocks until the go routine 'i' is done - no data race
|
||||
if errors[i] != nil {
|
||||
disks[i] = OfflineDisk
|
||||
blocks[i] = nil
|
||||
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -24,7 +24,6 @@ import (
|
||||
"testing"
|
||||
|
||||
humanize "github.com/dustin/go-humanize"
|
||||
"github.com/minio/minio/pkg/bpool"
|
||||
)
|
||||
|
||||
func (d badDisk) ReadFile(volume string, path string, offset int64, buf []byte, verifier *BitrotVerifier) (n int64, err error) {
|
||||
@ -108,9 +107,8 @@ func TestErasureReadFile(t *testing.T) {
|
||||
setup.Remove()
|
||||
t.Fatalf("Test %d: failed to create erasure test file: %v", i, err)
|
||||
}
|
||||
pool := bpool.NewBytePool(getChunkSize(test.blocksize, test.dataBlocks), len(storage.disks))
|
||||
writer := bytes.NewBuffer(nil)
|
||||
readInfo, err := storage.ReadFile(writer, "testbucket", "object", test.offset, test.length, test.data, file.Checksums, test.algorithm, test.blocksize, pool)
|
||||
readInfo, err := storage.ReadFile(writer, "testbucket", "object", test.offset, test.length, test.data, file.Checksums, test.algorithm, test.blocksize)
|
||||
if err != nil && !test.shouldFail {
|
||||
t.Errorf("Test %d: should pass but failed with: %v", i, err)
|
||||
}
|
||||
@ -136,7 +134,7 @@ func TestErasureReadFile(t *testing.T) {
|
||||
if test.offDisks > 0 {
|
||||
storage.disks[0] = OfflineDisk
|
||||
}
|
||||
readInfo, err = storage.ReadFile(writer, "testbucket", "object", test.offset, test.length, test.data, file.Checksums, test.algorithm, test.blocksize, pool)
|
||||
readInfo, err = storage.ReadFile(writer, "testbucket", "object", test.offset, test.length, test.data, file.Checksums, test.algorithm, test.blocksize)
|
||||
if err != nil && !test.shouldFailQuorum {
|
||||
t.Errorf("Test %d: should pass but failed with: %v", i, err)
|
||||
}
|
||||
@ -204,11 +202,6 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
|
||||
// To generate random offset/length.
|
||||
r := rand.New(rand.NewSource(UTCNow().UnixNano()))
|
||||
|
||||
// create pool buffer which will be used by erasureReadFile for
|
||||
// reading from disks and erasure decoding.
|
||||
chunkSize := getChunkSize(blockSize, dataBlocks)
|
||||
pool := bpool.NewBytePool(chunkSize, len(storage.disks))
|
||||
|
||||
buf := &bytes.Buffer{}
|
||||
|
||||
// Verify erasureReadFile() for random offsets and lengths.
|
||||
@ -218,7 +211,7 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
|
||||
|
||||
expected := data[offset : offset+readLen]
|
||||
|
||||
_, err = storage.ReadFile(buf, "testbucket", "testobject", offset, readLen, length, file.Checksums, DefaultBitrotAlgorithm, blockSize, pool)
|
||||
_, err = storage.ReadFile(buf, "testbucket", "testobject", offset, readLen, length, file.Checksums, DefaultBitrotAlgorithm, blockSize)
|
||||
if err != nil {
|
||||
t.Fatal(err, offset, readLen)
|
||||
}
|
||||
|
@ -24,7 +24,6 @@ import (
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/minio/minio/pkg/bpool"
|
||||
"github.com/minio/minio/pkg/mimedb"
|
||||
"github.com/minio/minio/pkg/objcache"
|
||||
)
|
||||
@ -242,10 +241,6 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
|
||||
}
|
||||
|
||||
var totalBytesRead int64
|
||||
|
||||
chunkSize := getChunkSize(xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks)
|
||||
pool := bpool.NewBytePool(chunkSize, len(onlineDisks))
|
||||
|
||||
storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks)
|
||||
if err != nil {
|
||||
return toObjectErr(err, bucket, object)
|
||||
@ -276,7 +271,7 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
|
||||
checksums[index] = checksumInfo.Hash
|
||||
}
|
||||
|
||||
file, err := storage.ReadFile(mw, bucket, pathJoin(object, partName), partOffset, readSize, partSize, checksums, algorithm, xlMeta.Erasure.BlockSize, pool)
|
||||
file, err := storage.ReadFile(mw, bucket, pathJoin(object, partName), partOffset, readSize, partSize, checksums, algorithm, xlMeta.Erasure.BlockSize)
|
||||
if err != nil {
|
||||
errorIf(err, "Unable to read %s of the object `%s/%s`.", partName, bucket, object)
|
||||
return toObjectErr(err, bucket, object)
|
||||
|
@ -1,69 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
*
|
||||
* 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 bpool implements a fixed size pool of byte slices.
|
||||
package bpool
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// ErrBpoolNoFree - Normally this error should never be returned, this error
|
||||
// indicates a bug in the package consumer.
|
||||
var ErrBpoolNoFree = errors.New("no free byte slice in pool")
|
||||
|
||||
// BytePool - temporary pool of byte slices.
|
||||
type BytePool struct {
|
||||
buf [][]byte // array of byte slices
|
||||
used []bool // indicates if a buf[i] is in use
|
||||
size int64 // size of buf[i]
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
// Get - Returns an unused byte slice.
|
||||
func (b *BytePool) Get() (buf []byte, err error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for i := 0; i < len(b.used); i++ {
|
||||
if !b.used[i] {
|
||||
b.used[i] = true
|
||||
if b.buf[i] == nil {
|
||||
b.buf[i] = make([]byte, b.size)
|
||||
}
|
||||
return b.buf[i], nil
|
||||
}
|
||||
}
|
||||
return nil, ErrBpoolNoFree
|
||||
}
|
||||
|
||||
// Reset - Marks all slices as unused.
|
||||
func (b *BytePool) Reset() {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for i := 0; i < len(b.used); i++ {
|
||||
b.used[i] = false
|
||||
}
|
||||
}
|
||||
|
||||
// NewBytePool - Returns new pool.
|
||||
// size - length of each slice.
|
||||
// n - number of slices in the pool.
|
||||
func NewBytePool(size int64, n int) *BytePool {
|
||||
used := make([]bool, n)
|
||||
buf := make([][]byte, n)
|
||||
return &BytePool{buf, used, size, sync.Mutex{}}
|
||||
}
|
@ -1,53 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
*
|
||||
* 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 bpool
|
||||
|
||||
import "testing"
|
||||
|
||||
func TestBytePool(t *testing.T) {
|
||||
size := int64(10)
|
||||
n := 16
|
||||
pool := NewBytePool(size, n)
|
||||
enBlocks := make([][]byte, n)
|
||||
|
||||
// Allocates all the 16 byte slices in the pool.
|
||||
alloc := func() {
|
||||
for i := range enBlocks {
|
||||
var err error
|
||||
enBlocks[i], err = pool.Get()
|
||||
if err != nil {
|
||||
t.Fatal("expected nil, got", err)
|
||||
}
|
||||
// Make sure the slice length is as expected.
|
||||
if len(enBlocks[i]) != int(size) {
|
||||
t.Fatalf("expected size %d, got %d", len(enBlocks[i]), size)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Allocate everything in the pool.
|
||||
alloc()
|
||||
// Any Get() will fail when the pool does not have any free buffer.
|
||||
_, err := pool.Get()
|
||||
if err == nil {
|
||||
t.Fatalf("expected %s, got nil", err)
|
||||
}
|
||||
// Reset - so that all the buffers are marked as unused.
|
||||
pool.Reset()
|
||||
// Allocation of all the buffers in the pool should succeed now.
|
||||
alloc()
|
||||
}
|
28
vendor/github.com/klauspost/reedsolomon/reedsolomon.go
generated
vendored
28
vendor/github.com/klauspost/reedsolomon/reedsolomon.go
generated
vendored
@ -40,7 +40,9 @@ type Encoder interface {
|
||||
// ones that don't have data.
|
||||
//
|
||||
// The length of the array must be equal to the total number of shards.
|
||||
// You indicate that a shard is missing by setting it to nil.
|
||||
// You indicate that a shard is missing by setting it to nil or zero-length.
|
||||
// If a shard is zero-length but has sufficient capacity, that memory will
|
||||
// be used, otherwise a new []byte will be allocated.
|
||||
//
|
||||
// If there are too few shards to reconstruct the missing
|
||||
// ones, ErrTooFewShards will be returned.
|
||||
@ -55,7 +57,9 @@ type Encoder interface {
|
||||
// data shards that don't have data.
|
||||
//
|
||||
// The length of the array must be equal to Shards.
|
||||
// You indicate that a shard is missing by setting it to nil.
|
||||
// You indicate that a shard is missing by setting it to nil or zero-length.
|
||||
// If a shard is zero-length but has sufficient capacity, that memory will
|
||||
// be used, otherwise a new []byte will be allocated.
|
||||
//
|
||||
// If there are too few shards to reconstruct the missing
|
||||
// ones, ErrTooFewShards will be returned.
|
||||
@ -547,7 +551,9 @@ func shardSize(shards [][]byte) int {
|
||||
// ones that don't have data.
|
||||
//
|
||||
// The length of the array must be equal to Shards.
|
||||
// You indicate that a shard is missing by setting it to nil.
|
||||
// You indicate that a shard is missing by setting it to nil or zero-length.
|
||||
// If a shard is zero-length but has sufficient capacity, that memory will
|
||||
// be used, otherwise a new []byte will be allocated.
|
||||
//
|
||||
// If there are too few shards to reconstruct the missing
|
||||
// ones, ErrTooFewShards will be returned.
|
||||
@ -564,7 +570,9 @@ func (r reedSolomon) Reconstruct(shards [][]byte) error {
|
||||
// data shards that don't have data.
|
||||
//
|
||||
// The length of the array must be equal to Shards.
|
||||
// You indicate that a shard is missing by setting it to nil.
|
||||
// You indicate that a shard is missing by setting it to nil or zero-length.
|
||||
// If a shard is zero-length but has sufficient capacity, that memory will
|
||||
// be used, otherwise a new []byte will be allocated.
|
||||
//
|
||||
// If there are too few shards to reconstruct the missing
|
||||
// ones, ErrTooFewShards will be returned.
|
||||
@ -682,7 +690,11 @@ func (r reedSolomon) reconstruct(shards [][]byte, dataOnly bool) error {
|
||||
|
||||
for iShard := 0; iShard < r.DataShards; iShard++ {
|
||||
if len(shards[iShard]) == 0 {
|
||||
shards[iShard] = make([]byte, shardSize)
|
||||
if cap(shards[iShard]) >= shardSize {
|
||||
shards[iShard] = shards[iShard][0:shardSize]
|
||||
} else {
|
||||
shards[iShard] = make([]byte, shardSize)
|
||||
}
|
||||
outputs[outputCount] = shards[iShard]
|
||||
matrixRows[outputCount] = dataDecodeMatrix[iShard]
|
||||
outputCount++
|
||||
@ -704,7 +716,11 @@ func (r reedSolomon) reconstruct(shards [][]byte, dataOnly bool) error {
|
||||
outputCount = 0
|
||||
for iShard := r.DataShards; iShard < r.Shards; iShard++ {
|
||||
if len(shards[iShard]) == 0 {
|
||||
shards[iShard] = make([]byte, shardSize)
|
||||
if cap(shards[iShard]) >= shardSize {
|
||||
shards[iShard] = shards[iShard][0:shardSize]
|
||||
} else {
|
||||
shards[iShard] = make([]byte, shardSize)
|
||||
}
|
||||
outputs[outputCount] = shards[iShard]
|
||||
matrixRows[outputCount] = r.parity[iShard-r.DataShards]
|
||||
outputCount++
|
||||
|
6
vendor/vendor.json
vendored
6
vendor/vendor.json
vendored
@ -243,10 +243,10 @@
|
||||
"revisionTime": "2016-10-16T15:41:25Z"
|
||||
},
|
||||
{
|
||||
"checksumSHA1": "R9saYJznxosfknAq2aPnVKxqI3w=",
|
||||
"checksumSHA1": "sGHmZAWf2bzBFBwL8HPg4u9aJAA=",
|
||||
"path": "github.com/klauspost/reedsolomon",
|
||||
"revision": "87ba8262ab3d167ae4d38e22796312cd2a9d0b19",
|
||||
"revisionTime": "2017-08-26T09:54:10Z"
|
||||
"revision": "ddcafc661e43ab1786575c0fc4b5b935b121de05",
|
||||
"revisionTime": "2017-09-20T19:08:25Z"
|
||||
},
|
||||
{
|
||||
"checksumSHA1": "dNYxHiBLalTqluak2/Z8c3RsSEM=",
|
||||
|
Loading…
Reference in New Issue
Block a user