mirror of
https://github.com/minio/minio.git
synced 2025-04-05 12:20:34 -04: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
|
// Scan part files on disk, block-by-block reconstruct it and
|
||||||
// write to stale disks.
|
// write to stale disks.
|
||||||
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
||||||
|
blocks := make([][]byte, len(s.disks))
|
||||||
|
for i := range blocks {
|
||||||
|
blocks[i] = make([]byte, chunksize)
|
||||||
|
}
|
||||||
var chunkOffset, blockOffset int64
|
var chunkOffset, blockOffset int64
|
||||||
for ; blockOffset < size; blockOffset += blocksize {
|
for ; blockOffset < size; blockOffset += blocksize {
|
||||||
// last iteration may have less than blocksize data
|
// 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 {
|
if size < blockOffset+blocksize {
|
||||||
blocksize = size - blockOffset
|
blocksize = size - blockOffset
|
||||||
chunksize = getChunkSize(blocksize, s.dataBlocks)
|
chunksize = getChunkSize(blocksize, s.dataBlocks)
|
||||||
|
for i := range blocks {
|
||||||
|
blocks[i] = blocks[i][:chunksize]
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// read a chunk from each disk, until we have
|
// read a chunk from each disk, until we have
|
||||||
// `s.dataBlocks` number of chunks set to non-nil in
|
// `s.dataBlocks` number of chunks set to non-nil in
|
||||||
// `blocks`
|
// `blocks`
|
||||||
blocks := make([][]byte, len(s.disks))
|
|
||||||
var buffer []byte
|
|
||||||
numReads := 0
|
numReads := 0
|
||||||
for i, disk := range s.disks {
|
for i, disk := range s.disks {
|
||||||
// skip reading from unavailable or stale disks
|
// skip reading from unavailable or stale disks
|
||||||
if disk == nil || staleDisks[i] != nil {
|
if disk == nil || staleDisks[i] != nil {
|
||||||
|
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
// allocate buffer only when needed - when
|
_, err = disk.ReadFile(volume, path, chunkOffset, blocks[i], verifiers[i])
|
||||||
// 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])
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// LOG FIXME: add a conditional log
|
// LOG FIXME: add a conditional log
|
||||||
// for read failures, once per-disk
|
// for read failures, once per-disk
|
||||||
// per-function-invocation.
|
// per-function-invocation.
|
||||||
|
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||||
continue
|
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++
|
numReads++
|
||||||
if numReads == s.dataBlocks {
|
if numReads == s.dataBlocks {
|
||||||
// we have enough data to reconstruct
|
// 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
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -18,14 +18,12 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"io"
|
"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.
|
// 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
|
// 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.
|
// 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 {
|
if offset < 0 || length < 0 {
|
||||||
return f, traceError(errUnexpected)
|
return f, traceError(errUnexpected)
|
||||||
}
|
}
|
||||||
@ -53,15 +51,20 @@ func (s ErasureStorage) ReadFile(writer io.Writer, volume, path string, offset,
|
|||||||
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
chunksize := getChunkSize(blocksize, s.dataBlocks)
|
||||||
|
|
||||||
blocks := make([][]byte, len(s.disks))
|
blocks := make([][]byte, len(s.disks))
|
||||||
|
for i := range blocks {
|
||||||
|
blocks[i] = make([]byte, chunksize)
|
||||||
|
}
|
||||||
for off := offset / blocksize; length > 0; off++ {
|
for off := offset / blocksize; length > 0; off++ {
|
||||||
blockOffset := off * chunksize
|
blockOffset := off * chunksize
|
||||||
pool.Reset()
|
|
||||||
|
|
||||||
if currentBlock := (offset + f.Size) / blocksize; currentBlock == lastBlock {
|
if currentBlock := (offset + f.Size) / blocksize; currentBlock == lastBlock {
|
||||||
blocksize = totalLength % blocksize
|
blocksize = totalLength % blocksize
|
||||||
chunksize = getChunkSize(blocksize, s.dataBlocks)
|
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 {
|
if err != nil {
|
||||||
return f, traceError(errXLReadQuorum)
|
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 {
|
func erasureCountMissingBlocks(blocks [][]byte, limit int) int {
|
||||||
missing := 0
|
missing := 0
|
||||||
for i := range blocks[:limit] {
|
for i := range blocks[:limit] {
|
||||||
if blocks[i] == nil {
|
if len(blocks[i]) == 0 {
|
||||||
missing++
|
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
|
// readConcurrent reads all requested data concurrently from the disks into blocks. It returns an error if
|
||||||
// too many disks failed while reading.
|
// 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))
|
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])
|
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)
|
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
|
errors[i] = <-errChans[i] // blocks until the go routine 'i' is done - no data race
|
||||||
if errors[i] != nil {
|
if errors[i] != nil {
|
||||||
disks[i] = OfflineDisk
|
disks[i] = OfflineDisk
|
||||||
blocks[i] = nil
|
blocks[i] = blocks[i][:0] // mark shard as missing
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -24,7 +24,6 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
humanize "github.com/dustin/go-humanize"
|
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) {
|
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()
|
setup.Remove()
|
||||||
t.Fatalf("Test %d: failed to create erasure test file: %v", i, err)
|
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)
|
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 {
|
if err != nil && !test.shouldFail {
|
||||||
t.Errorf("Test %d: should pass but failed with: %v", i, err)
|
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 {
|
if test.offDisks > 0 {
|
||||||
storage.disks[0] = OfflineDisk
|
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 {
|
if err != nil && !test.shouldFailQuorum {
|
||||||
t.Errorf("Test %d: should pass but failed with: %v", i, err)
|
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.
|
// To generate random offset/length.
|
||||||
r := rand.New(rand.NewSource(UTCNow().UnixNano()))
|
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{}
|
buf := &bytes.Buffer{}
|
||||||
|
|
||||||
// Verify erasureReadFile() for random offsets and lengths.
|
// Verify erasureReadFile() for random offsets and lengths.
|
||||||
@ -218,7 +211,7 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
|
|||||||
|
|
||||||
expected := data[offset : offset+readLen]
|
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 {
|
if err != nil {
|
||||||
t.Fatal(err, offset, readLen)
|
t.Fatal(err, offset, readLen)
|
||||||
}
|
}
|
||||||
|
@ -24,7 +24,6 @@ import (
|
|||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/minio/minio/pkg/bpool"
|
|
||||||
"github.com/minio/minio/pkg/mimedb"
|
"github.com/minio/minio/pkg/mimedb"
|
||||||
"github.com/minio/minio/pkg/objcache"
|
"github.com/minio/minio/pkg/objcache"
|
||||||
)
|
)
|
||||||
@ -242,10 +241,6 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
|
|||||||
}
|
}
|
||||||
|
|
||||||
var totalBytesRead int64
|
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)
|
storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return toObjectErr(err, bucket, object)
|
return toObjectErr(err, bucket, object)
|
||||||
@ -276,7 +271,7 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
|
|||||||
checksums[index] = checksumInfo.Hash
|
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 {
|
if err != nil {
|
||||||
errorIf(err, "Unable to read %s of the object `%s/%s`.", partName, bucket, object)
|
errorIf(err, "Unable to read %s of the object `%s/%s`.", partName, bucket, object)
|
||||||
return toObjectErr(err, 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()
|
|
||||||
}
|
|
24
vendor/github.com/klauspost/reedsolomon/reedsolomon.go
generated
vendored
24
vendor/github.com/klauspost/reedsolomon/reedsolomon.go
generated
vendored
@ -40,7 +40,9 @@ type Encoder interface {
|
|||||||
// ones that don't have data.
|
// ones that don't have data.
|
||||||
//
|
//
|
||||||
// The length of the array must be equal to the total number of shards.
|
// 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
|
// If there are too few shards to reconstruct the missing
|
||||||
// ones, ErrTooFewShards will be returned.
|
// ones, ErrTooFewShards will be returned.
|
||||||
@ -55,7 +57,9 @@ type Encoder interface {
|
|||||||
// data shards that don't have data.
|
// data shards that don't have data.
|
||||||
//
|
//
|
||||||
// The length of the array must be equal to Shards.
|
// 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
|
// If there are too few shards to reconstruct the missing
|
||||||
// ones, ErrTooFewShards will be returned.
|
// ones, ErrTooFewShards will be returned.
|
||||||
@ -547,7 +551,9 @@ func shardSize(shards [][]byte) int {
|
|||||||
// ones that don't have data.
|
// ones that don't have data.
|
||||||
//
|
//
|
||||||
// The length of the array must be equal to Shards.
|
// 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
|
// If there are too few shards to reconstruct the missing
|
||||||
// ones, ErrTooFewShards will be returned.
|
// ones, ErrTooFewShards will be returned.
|
||||||
@ -564,7 +570,9 @@ func (r reedSolomon) Reconstruct(shards [][]byte) error {
|
|||||||
// data shards that don't have data.
|
// data shards that don't have data.
|
||||||
//
|
//
|
||||||
// The length of the array must be equal to Shards.
|
// 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
|
// If there are too few shards to reconstruct the missing
|
||||||
// ones, ErrTooFewShards will be returned.
|
// 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++ {
|
for iShard := 0; iShard < r.DataShards; iShard++ {
|
||||||
if len(shards[iShard]) == 0 {
|
if len(shards[iShard]) == 0 {
|
||||||
|
if cap(shards[iShard]) >= shardSize {
|
||||||
|
shards[iShard] = shards[iShard][0:shardSize]
|
||||||
|
} else {
|
||||||
shards[iShard] = make([]byte, shardSize)
|
shards[iShard] = make([]byte, shardSize)
|
||||||
|
}
|
||||||
outputs[outputCount] = shards[iShard]
|
outputs[outputCount] = shards[iShard]
|
||||||
matrixRows[outputCount] = dataDecodeMatrix[iShard]
|
matrixRows[outputCount] = dataDecodeMatrix[iShard]
|
||||||
outputCount++
|
outputCount++
|
||||||
@ -704,7 +716,11 @@ func (r reedSolomon) reconstruct(shards [][]byte, dataOnly bool) error {
|
|||||||
outputCount = 0
|
outputCount = 0
|
||||||
for iShard := r.DataShards; iShard < r.Shards; iShard++ {
|
for iShard := r.DataShards; iShard < r.Shards; iShard++ {
|
||||||
if len(shards[iShard]) == 0 {
|
if len(shards[iShard]) == 0 {
|
||||||
|
if cap(shards[iShard]) >= shardSize {
|
||||||
|
shards[iShard] = shards[iShard][0:shardSize]
|
||||||
|
} else {
|
||||||
shards[iShard] = make([]byte, shardSize)
|
shards[iShard] = make([]byte, shardSize)
|
||||||
|
}
|
||||||
outputs[outputCount] = shards[iShard]
|
outputs[outputCount] = shards[iShard]
|
||||||
matrixRows[outputCount] = r.parity[iShard-r.DataShards]
|
matrixRows[outputCount] = r.parity[iShard-r.DataShards]
|
||||||
outputCount++
|
outputCount++
|
||||||
|
6
vendor/vendor.json
vendored
6
vendor/vendor.json
vendored
@ -243,10 +243,10 @@
|
|||||||
"revisionTime": "2016-10-16T15:41:25Z"
|
"revisionTime": "2016-10-16T15:41:25Z"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"checksumSHA1": "R9saYJznxosfknAq2aPnVKxqI3w=",
|
"checksumSHA1": "sGHmZAWf2bzBFBwL8HPg4u9aJAA=",
|
||||||
"path": "github.com/klauspost/reedsolomon",
|
"path": "github.com/klauspost/reedsolomon",
|
||||||
"revision": "87ba8262ab3d167ae4d38e22796312cd2a9d0b19",
|
"revision": "ddcafc661e43ab1786575c0fc4b5b935b121de05",
|
||||||
"revisionTime": "2017-08-26T09:54:10Z"
|
"revisionTime": "2017-09-20T19:08:25Z"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"checksumSHA1": "dNYxHiBLalTqluak2/Z8c3RsSEM=",
|
"checksumSHA1": "dNYxHiBLalTqluak2/Z8c3RsSEM=",
|
||||||
|
Loading…
x
Reference in New Issue
Block a user