erasure: simplify XL backend operations (#4649) (#4758)

This change provides new implementations of the XL backend operations:
 - create file
 - read   file
 - heal   file
Further this change adds table based tests for all three operations.

This affects also the bitrot algorithm integration. Algorithms are now
integrated in an idiomatic way (like crypto.Hash).
Fixes #4696
Fixes #4649
Fixes #4359
This commit is contained in:
Andreas Auernhammer 2017-08-14 18:08:42 -07:00 committed by Dee Koder
parent 617f2394fb
commit 85fcee1919
33 changed files with 1238 additions and 1863 deletions

View File

@ -17,128 +17,79 @@
package cmd package cmd
import ( import (
"encoding/hex"
"hash" "hash"
"io" "io"
"sync"
"github.com/klauspost/reedsolomon"
) )
// erasureCreateFile - writes an entire stream by erasure coding to // CreateFile creates a new bitrot encoded file spread over all available disks. CreateFile will create
// all the disks, writes also calculate individual block's checksum // the file at the given volume and path. It will read from src until an io.EOF occurs. The given algorithm will
// for future bit-rot protection. // be used to protect the erasure encoded file.
func erasureCreateFile(disks []StorageAPI, volume, path string, reader io.Reader, allowEmpty bool, blockSize int64, func (s *ErasureStorage) CreateFile(src io.Reader, volume, path string, buffer []byte, algorithm BitrotAlgorithm, writeQuorum int) (f ErasureFileInfo, err error) {
dataBlocks, parityBlocks int, algo HashAlgo, writeQuorum int) (newDisks []StorageAPI, bytesWritten int64, checkSums []string, err error) { if !algorithm.Available() {
return f, traceError(errBitrotHashAlgoInvalid)
// Allocated blockSized buffer for reading from incoming stream.
buf := make([]byte, blockSize)
hashWriters := newHashWriters(len(disks), algo)
// Read until io.EOF, erasure codes data and writes to all disks.
for {
var blocks [][]byte
n, rErr := io.ReadFull(reader, buf)
// FIXME: this is a bug in Golang, n == 0 and err ==
// io.ErrUnexpectedEOF for io.ReadFull function.
if n == 0 && rErr == io.ErrUnexpectedEOF {
return nil, 0, nil, traceError(rErr)
} }
if rErr == io.EOF { f.Checksums = make([][]byte, len(s.disks))
// We have reached EOF on the first byte read, io.Reader hashers := make([]hash.Hash, len(s.disks))
// must be 0bytes, we don't need to erasure code for i := range hashers {
// data. Will create a 0byte file instead. hashers[i] = algorithm.New()
if bytesWritten == 0 && allowEmpty {
blocks = make([][]byte, len(disks))
newDisks, rErr = appendFile(disks, volume, path, blocks, hashWriters, writeQuorum)
if rErr != nil {
return nil, 0, nil, rErr
} }
} // else we have reached EOF after few reads, no need to errChans, errors := make([]chan error, len(s.disks)), make([]error, len(s.disks))
// add an additional 0bytes at the end. for i := range errChans {
errChans[i] = make(chan error, 1) // create buffered channel to let finished go-routines die early
}
blocks, n := [][]byte{}, len(buffer)
for n == len(buffer) {
n, err = io.ReadFull(src, buffer)
if n == 0 && err == io.EOF {
if f.Size != 0 { // don't write empty block if we have written to the disks
break break
} }
if rErr != nil && rErr != io.ErrUnexpectedEOF { blocks = make([][]byte, len(s.disks)) // write empty block
return nil, 0, nil, traceError(rErr) } else if err == nil || (n > 0 && err == io.ErrUnexpectedEOF) {
} blocks, err = s.ErasureEncode(buffer[:n])
if n > 0 {
// Returns encoded blocks.
var enErr error
blocks, enErr = encodeData(buf[0:n], dataBlocks, parityBlocks)
if enErr != nil {
return nil, 0, nil, enErr
}
// Write to all disks.
if newDisks, err = appendFile(disks, volume, path, blocks, hashWriters, writeQuorum); err != nil {
return nil, 0, nil, err
}
bytesWritten += int64(n)
}
}
checkSums = make([]string, len(disks))
for i := range checkSums {
checkSums[i] = hex.EncodeToString(hashWriters[i].Sum(nil))
}
return newDisks, bytesWritten, checkSums, nil
}
// encodeData - encodes incoming data buffer into
// dataBlocks+parityBlocks returns a 2 dimensional byte array.
func encodeData(dataBuffer []byte, dataBlocks, parityBlocks int) ([][]byte, error) {
rs, err := reedsolomon.New(dataBlocks, parityBlocks)
if err != nil { if err != nil {
return nil, traceError(err) return f, err
} }
// Split the input buffer into data and parity blocks. } else {
var blocks [][]byte return f, traceError(err)
blocks, err = rs.Split(dataBuffer)
if err != nil {
return nil, traceError(err)
} }
// Encode parity blocks using data blocks. for i := range errChans { // span workers
err = rs.Encode(blocks) go erasureAppendFile(s.disks[i], volume, path, hashers[i], blocks[i], errChans[i])
if err != nil { }
return nil, traceError(err) for i := range errChans { // what until all workers are finished
errors[i] = <-errChans[i]
}
if err = reduceWriteQuorumErrs(errors, objectOpIgnoredErrs, writeQuorum); err != nil {
return f, err
}
s.disks = evalDisks(s.disks, errors)
f.Size += int64(n)
} }
// Return encoded blocks. f.Algorithm = algorithm
return blocks, nil for i, disk := range s.disks {
} if disk == OfflineDisk {
// appendFile - append data buffer at path.
func appendFile(disks []StorageAPI, volume, path string, enBlocks [][]byte, hashWriters []hash.Hash, writeQuorum int) ([]StorageAPI, error) {
var wg = &sync.WaitGroup{}
var wErrs = make([]error, len(disks))
// Write encoded data to quorum disks in parallel.
for index, disk := range disks {
if disk == nil {
wErrs[index] = traceError(errDiskNotFound)
continue continue
} }
wg.Add(1) f.Checksums[i] = hashers[i].Sum(nil)
// Write encoded data in routine. }
go func(index int, disk StorageAPI) { return f, nil
defer wg.Done() }
wErr := disk.AppendFile(volume, path, enBlocks[index])
if wErr != nil { // erasureAppendFile appends the content of buf to the file on the given disk and updates computes
wErrs[index] = traceError(wErr) // the hash of the written data. It sends the write error (or nil) over the error channel.
func erasureAppendFile(disk StorageAPI, volume, path string, hash hash.Hash, buf []byte, errChan chan<- error) {
if disk == OfflineDisk {
errChan <- traceError(errDiskNotFound)
return return
} }
err := disk.AppendFile(volume, path, buf)
// Calculate hash for each blocks. if err != nil {
hashWriters[index].Write(enBlocks[index]) errChan <- err
return
// Successfully wrote.
wErrs[index] = nil
}(index, disk)
} }
hash.Write(buf)
// Wait for all the appends to finish. errChan <- err
wg.Wait()
return evalDisks(disks, wErrs), reduceWriteQuorumErrs(wErrs, objectOpIgnoredErrs, writeQuorum)
} }

View File

@ -19,186 +19,104 @@ package cmd
import ( import (
"bytes" "bytes"
"crypto/rand" "crypto/rand"
"io"
"testing" "testing"
humanize "github.com/dustin/go-humanize" humanize "github.com/dustin/go-humanize"
"github.com/klauspost/reedsolomon"
) )
// Simulates a faulty disk for AppendFile() type badDisk struct{ StorageAPI }
type AppendDiskDown struct {
*posix
}
func (a AppendDiskDown) AppendFile(volume string, path string, buf []byte) error { func (a badDisk) AppendFile(volume string, path string, buf []byte) error {
return errFaultyDisk return errFaultyDisk
} }
// Test erasureCreateFile() const oneMiByte = 1 * humanize.MiByte
func TestErasureCreateFile(t *testing.T) {
// Initialize environment needed for the test.
dataBlocks := 7
parityBlocks := 7
blockSize := int64(blockSizeV1)
setup, err := newErasureTestSetup(dataBlocks, parityBlocks, blockSize)
if err != nil {
t.Error(err)
return
}
defer setup.Remove()
disks := setup.disks var erasureCreateFileTests = []struct {
dataBlocks int
// Prepare a slice of 1MiB with random data. onDisks, offDisks int
data := make([]byte, 1*humanize.MiByte) blocksize, data int64
_, err = rand.Read(data) offset int
if err != nil { algorithm BitrotAlgorithm
t.Fatal(err) shouldFail, shouldFailQuorum bool
}
// Test when all disks are up.
_, size, _, err := erasureCreateFile(disks, "testbucket", "testobject1", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if err != nil {
t.Fatal(err)
}
if size != int64(len(data)) {
t.Errorf("erasureCreateFile returned %d, expected %d", size, len(data))
}
// 2 disks down.
disks[4] = AppendDiskDown{disks[4].(*posix)}
disks[5] = AppendDiskDown{disks[5].(*posix)}
// Test when two disks are down.
_, size, _, err = erasureCreateFile(disks, "testbucket", "testobject2", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if err != nil {
t.Fatal(err)
}
if size != int64(len(data)) {
t.Errorf("erasureCreateFile returned %d, expected %d", size, len(data))
}
// 4 more disks down. 6 disks down in total.
disks[6] = AppendDiskDown{disks[6].(*posix)}
disks[7] = AppendDiskDown{disks[7].(*posix)}
disks[8] = AppendDiskDown{disks[8].(*posix)}
disks[9] = AppendDiskDown{disks[9].(*posix)}
_, size, _, err = erasureCreateFile(disks, "testbucket", "testobject3", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if err != nil {
t.Fatal(err)
}
if size != int64(len(data)) {
t.Errorf("erasureCreateFile returned %d, expected %d", size, len(data))
}
// 1 more disk down. 7 disk down in total. Should return quorum error.
disks[10] = AppendDiskDown{disks[10].(*posix)}
_, _, _, err = erasureCreateFile(disks, "testbucket", "testobject4", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if errorCause(err) != errXLWriteQuorum {
t.Errorf("erasureCreateFile return value: expected errXLWriteQuorum, got %s", err)
}
}
// TestErasureEncode checks for encoding for different data sets.
func TestErasureEncode(t *testing.T) {
// Collection of cases for encode cases.
testEncodeCases := []struct {
inputData []byte
inputDataBlocks int
inputParityBlocks int
shouldPass bool
expectedErr error
}{ }{
// TestCase - 1. {dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 0
// Regular data encoded. {dataBlocks: 3, onDisks: 6, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 1, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 1
{ {dataBlocks: 4, onDisks: 8, offDisks: 2, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 2, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 2
[]byte("Lorem Ipsum is simply dummy text of the printing and typesetting industry. Lorem Ipsum has been the industry's standard dummy text ever since the 1500s, when an unknown printer took a galley of type and scrambled it to make a type specimen book. It has survived not only five centuries, but also the leap into electronic typesetting, remaining essentially unchanged. It was popularised in the 1960s with the release of Letraset sheets containing Lorem Ipsum passages, and more recently with desktop publishing software like Aldus PageMaker including versions of Lorem Ipsum."), {dataBlocks: 5, onDisks: 10, offDisks: 3, blocksize: int64(blockSizeV1), data: oneMiByte, offset: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 3
8, {dataBlocks: 6, onDisks: 12, offDisks: 4, blocksize: int64(blockSizeV1), data: oneMiByte, offset: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 4
8, {dataBlocks: 7, onDisks: 14, offDisks: 5, blocksize: int64(blockSizeV1), data: 0, offset: 0, shouldFail: false, algorithm: SHA256, shouldFailQuorum: false}, // 5
true, {dataBlocks: 8, onDisks: 16, offDisks: 7, blocksize: int64(blockSizeV1), data: 0, offset: 0, shouldFail: false, algorithm: DefaultBitrotAlgorithm, shouldFailQuorum: false}, // 6
nil, {dataBlocks: 2, onDisks: 4, offDisks: 2, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: true}, // 7
}, {dataBlocks: 4, onDisks: 8, offDisks: 4, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: SHA256, shouldFail: false, shouldFailQuorum: true}, // 8
// TestCase - 2. {dataBlocks: 7, onDisks: 14, offDisks: 7, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 9
// Empty data errors out. {dataBlocks: 8, onDisks: 16, offDisks: 8, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 10
{ {dataBlocks: 5, onDisks: 10, offDisks: 3, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 11
[]byte(""), {dataBlocks: 6, onDisks: 12, offDisks: 5, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 102, algorithm: 0, shouldFail: true, shouldFailQuorum: false}, // 12
8, {dataBlocks: 3, onDisks: 6, offDisks: 1, blocksize: int64(blockSizeV1), data: oneMiByte, offset: oneMiByte / 2, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 13
8, {dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(oneMiByte / 2), data: oneMiByte, offset: oneMiByte/2 + 1, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 14
false, {dataBlocks: 4, onDisks: 8, offDisks: 0, blocksize: int64(oneMiByte - 1), data: oneMiByte, offset: oneMiByte - 1, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 15
reedsolomon.ErrShortData, {dataBlocks: 8, onDisks: 12, offDisks: 2, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 2, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 16
}, {dataBlocks: 8, onDisks: 10, offDisks: 1, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 17
// TestCase - 3. {dataBlocks: 10, onDisks: 14, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 17, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 18
// Single byte encoded. {dataBlocks: 2, onDisks: 6, offDisks: 2, blocksize: int64(oneMiByte), data: oneMiByte, offset: oneMiByte / 2, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 19
{ {dataBlocks: 10, onDisks: 16, offDisks: 8, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 20
[]byte("1"),
4,
4,
true,
nil,
},
// TestCase - 4.
// test case with negative data block.
{
[]byte("1"),
-1,
8,
false,
reedsolomon.ErrInvShardNum,
},
// TestCase - 5.
// test case with negative parity block.
{
[]byte("1"),
8,
-1,
false,
reedsolomon.ErrInvShardNum,
},
// TestCase - 6.
// test case with zero data block.
{
[]byte("1"),
0,
8,
false,
reedsolomon.ErrInvShardNum,
},
// TestCase - 7.
// test case with zero parity block.
{
[]byte("1"),
8,
0,
false,
reedsolomon.ErrInvShardNum,
},
// TestCase - 8.
// test case with data + parity blocks > 256.
// expected to fail with Error Max Shard number.
{
[]byte("1"),
129,
128,
false,
reedsolomon.ErrMaxShardNum,
},
} }
// Test encode cases. func TestErasureCreateFile(t *testing.T) {
for i, testCase := range testEncodeCases { for i, test := range erasureCreateFileTests {
_, actualErr := encodeData(testCase.inputData, testCase.inputDataBlocks, testCase.inputParityBlocks) setup, err := newErasureTestSetup(test.dataBlocks, test.onDisks-test.dataBlocks, test.blocksize)
if actualErr != nil && testCase.shouldPass { if err != nil {
t.Errorf("Test %d: Expected to pass but failed instead with \"%s\"", i+1, actualErr) t.Fatalf("Test %d: failed to create test setup: %v", i, err)
} }
if actualErr == nil && !testCase.shouldPass { storage, err := NewErasureStorage(setup.disks, test.dataBlocks, test.onDisks-test.dataBlocks)
t.Errorf("Test %d: Expected to fail with error <Error> \"%v\", but instead passed", i+1, testCase.expectedErr) if err != nil {
setup.Remove()
t.Fatalf("Test %d: failed to create ErasureStorage: %v", i, err)
} }
// Failed as expected, but does it fail for the expected reason. buffer := make([]byte, test.blocksize, 2*test.blocksize)
if actualErr != nil && !testCase.shouldPass {
if errorCause(actualErr) != testCase.expectedErr { data := make([]byte, test.data)
t.Errorf("Test %d: Expected Error to be \"%v\", but instead found \"%v\" ", i+1, testCase.expectedErr, actualErr) if _, err = io.ReadFull(rand.Reader, data); err != nil {
setup.Remove()
t.Fatalf("Test %d: failed to generate random test data: %v", i, err)
}
algorithm := test.algorithm
if !algorithm.Available() {
algorithm = DefaultBitrotAlgorithm
}
file, err := storage.CreateFile(bytes.NewReader(data[test.offset:]), "testbucket", "object", buffer, test.algorithm, test.dataBlocks+1)
if err != nil && !test.shouldFail {
t.Errorf("Test %d: should pass but failed with: %v", i, err)
}
if err == nil && test.shouldFail {
t.Errorf("Test %d: should fail but it passed", i)
}
if err == nil {
if length := int64(len(data[test.offset:])); file.Size != length {
t.Errorf("Test %d: invalid number of bytes written: got: #%d want #%d", i, file.Size, length)
}
for j := range storage.disks[:test.offDisks] {
storage.disks[j] = badDisk{nil}
}
if test.offDisks > 0 {
storage.disks[0] = OfflineDisk
}
file, err = storage.CreateFile(bytes.NewReader(data[test.offset:]), "testbucket", "object2", buffer, test.algorithm, test.dataBlocks+1)
if err != nil && !test.shouldFailQuorum {
t.Errorf("Test %d: should pass but failed with: %v", i, err)
}
if err == nil && test.shouldFailQuorum {
t.Errorf("Test %d: should fail but it passed", i)
}
if err == nil {
if length := int64(len(data[test.offset:])); file.Size != length {
t.Errorf("Test %d: invalid number of bytes written: got: #%d want #%d", i, file.Size, length)
} }
} }
} }
setup.Remove()
}
} }

View File

@ -16,71 +16,75 @@
package cmd package cmd
import "encoding/hex" import "hash"
// Heals the erasure coded file. reedsolomon.Reconstruct() is used to reconstruct the missing parts. // HealFile tries to reconstruct a bitrot encoded file spread over all available disks. HealFile will read the valid parts of the file,
func erasureHealFile(latestDisks []StorageAPI, outDatedDisks []StorageAPI, volume, path, healBucket, healPath string, // reconstruct the missing data and write the reconstructed parts back to the disks.
size, blockSize int64, dataBlocks, parityBlocks int, algo HashAlgo) (checkSums []string, err error) { // It will try to read the valid parts from the file under the given volume and path and tries to reconstruct the file under the given
// healVolume and healPath. The given algorithm will be used to verify the valid parts and to protect the reconstructed file.
var offset int64 func (s ErasureStorage) HealFile(offlineDisks []StorageAPI, volume, path string, blocksize int64, healVolume, healPath string, size int64, algorithm BitrotAlgorithm, checksums [][]byte) (f ErasureFileInfo, err error) {
remainingSize := size if !algorithm.Available() {
return f, traceError(errBitrotHashAlgoInvalid)
// Hash for bitrot protection.
hashWriters := newHashWriters(len(outDatedDisks), bitRotAlgo)
for remainingSize > 0 {
curBlockSize := blockSize
if remainingSize < curBlockSize {
curBlockSize = remainingSize
} }
f.Checksums = make([][]byte, len(s.disks))
// Calculate the block size that needs to be read from each disk. hashers, verifiers := make([]hash.Hash, len(s.disks)), make([]*BitrotVerifier, len(s.disks))
curEncBlockSize := getChunkSize(curBlockSize, dataBlocks) for i, disk := range s.disks {
if disk == OfflineDisk {
// Memory for reading data from disks and reconstructing missing data using erasure coding. hashers[i] = algorithm.New()
enBlocks := make([][]byte, len(latestDisks)) } else {
verifiers[i] = NewBitrotVerifier(algorithm, checksums[i])
// Read data from the latest disks. f.Checksums[i] = checksums[i]
// FIXME: no need to read from all the disks. dataBlocks+1 is enough. }
for index, disk := range latestDisks { }
if disk == nil { blocks := make([][]byte, len(s.disks))
chunksize := getChunkSize(blocksize, s.dataBlocks)
for offset := int64(0); offset < size; offset += blocksize {
if size < blocksize {
blocksize = size
chunksize = getChunkSize(blocksize, s.dataBlocks)
}
numReads := 0
for i, disk := range s.disks {
if disk != OfflineDisk {
if blocks[i] == nil {
blocks[i] = make([]byte, chunksize)
}
blocks[i] = blocks[i][:chunksize]
if !verifiers[i].IsVerified() {
_, err = disk.ReadFileWithVerify(volume, path, offset, blocks[i], verifiers[i])
} else {
_, err = disk.ReadFile(volume, path, offset, blocks[i])
}
if err != nil {
blocks[i] = nil
} else {
numReads++
}
if numReads == s.dataBlocks { // we have enough data to reconstruct
break
}
}
}
if err = s.ErasureDecodeDataAndParityBlocks(blocks); err != nil {
return f, err
}
for i, disk := range s.disks {
if disk != OfflineDisk {
continue continue
} }
enBlocks[index] = make([]byte, curEncBlockSize) if err = offlineDisks[i].AppendFile(healVolume, healPath, blocks[i]); err != nil {
_, err := disk.ReadFile(volume, path, offset, enBlocks[index]) return f, traceError(err)
if err != nil { }
enBlocks[index] = nil hashers[i].Write(blocks[i])
} }
} }
f.Size = size
// Reconstruct any missing data and parity blocks. f.Algorithm = algorithm
err := decodeDataAndParity(enBlocks, dataBlocks, parityBlocks) for i, disk := range s.disks {
if err != nil { if disk != OfflineDisk {
return nil, err
}
// Write to the healPath file.
for index, disk := range outDatedDisks {
if disk == nil {
continue continue
} }
err := disk.AppendFile(healBucket, healPath, enBlocks[index]) f.Checksums[i] = hashers[i].Sum(nil)
if err != nil {
return nil, traceError(err)
} }
hashWriters[index].Write(enBlocks[index]) return f, nil
}
remainingSize -= curBlockSize
offset += curEncBlockSize
}
// Checksums for the bit rot.
checkSums = make([]string, len(outDatedDisks))
for index, disk := range outDatedDisks {
if disk == nil {
continue
}
checkSums[index] = hex.EncodeToString(hashWriters[index].Sum(nil))
}
return checkSums, nil
} }

View File

@ -19,111 +19,124 @@ package cmd
import ( import (
"bytes" "bytes"
"crypto/rand" "crypto/rand"
"os" "io"
"path" "reflect"
"testing" "testing"
humanize "github.com/dustin/go-humanize"
) )
// Test erasureHealFile() var erasureHealFileTests = []struct {
dataBlocks int
disks, offDisks, badDisks, badOffDisks int
blocksize, size int64
algorithm BitrotAlgorithm
shouldFail bool
shouldFailQuorum bool
}{
{dataBlocks: 2, disks: 4, offDisks: 1, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 0
{dataBlocks: 3, disks: 6, offDisks: 2, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 1
{dataBlocks: 4, disks: 8, offDisks: 2, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 2
{dataBlocks: 5, disks: 10, offDisks: 3, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 3
{dataBlocks: 6, disks: 12, offDisks: 2, badDisks: 3, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 4
{dataBlocks: 7, disks: 14, offDisks: 4, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 5
{dataBlocks: 8, disks: 16, offDisks: 6, badDisks: 1, badOffDisks: 1, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 6
{dataBlocks: 7, disks: 14, offDisks: 2, badDisks: 3, badOffDisks: 0, blocksize: int64(oneMiByte / 2), size: oneMiByte, algorithm: BLAKE2b512, shouldFail: true, shouldFailQuorum: false}, // 7
{dataBlocks: 6, disks: 12, offDisks: 1, badDisks: 0, badOffDisks: 1, blocksize: int64(oneMiByte - 1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: true, shouldFailQuorum: false}, // 8
{dataBlocks: 5, disks: 10, offDisks: 3, badDisks: 0, badOffDisks: 3, blocksize: int64(oneMiByte / 2), size: oneMiByte, algorithm: SHA256, shouldFail: true, shouldFailQuorum: false}, // 9
{dataBlocks: 4, disks: 8, offDisks: 1, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 10
{dataBlocks: 2, disks: 4, offDisks: 1, badDisks: 0, badOffDisks: 1, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 11
{dataBlocks: 6, disks: 12, offDisks: 8, badDisks: 3, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 12
{dataBlocks: 7, disks: 14, offDisks: 3, badDisks: 4, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 13
{dataBlocks: 7, disks: 14, offDisks: 6, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 14
{dataBlocks: 8, disks: 16, offDisks: 4, badDisks: 5, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 15
{dataBlocks: 2, disks: 4, offDisks: 0, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 16
{dataBlocks: 2, disks: 4, offDisks: 0, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: 0, shouldFail: true, shouldFailQuorum: false}, // 17
{dataBlocks: 12, disks: 16, offDisks: 2, badDisks: 1, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 18
{dataBlocks: 6, disks: 8, offDisks: 1, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 19
{dataBlocks: 7, disks: 10, offDisks: 1, badDisks: 0, badOffDisks: 0, blocksize: int64(blockSizeV1), size: oneMiByte, algorithm: 0, shouldFail: true, shouldFailQuorum: false}, // 20
}
func TestErasureHealFile(t *testing.T) { func TestErasureHealFile(t *testing.T) {
// Initialize environment needed for the test. for i, test := range erasureHealFileTests {
dataBlocks := 7 setup, err := newErasureTestSetup(test.dataBlocks, test.disks-test.dataBlocks, test.blocksize)
parityBlocks := 7
blockSize := int64(blockSizeV1)
setup, err := newErasureTestSetup(dataBlocks, parityBlocks, blockSize)
if err != nil { if err != nil {
t.Error(err) t.Fatalf("Test %d: failed to setup XL environment: %v", i, err)
return
} }
defer setup.Remove() storage, err := NewErasureStorage(setup.disks, test.dataBlocks, test.disks-test.dataBlocks)
disks := setup.disks
// Prepare a slice of 1MiB with random data.
data := make([]byte, 1*humanize.MiByte)
_, err = rand.Read(data)
if err != nil { if err != nil {
t.Fatal(err) setup.Remove()
t.Fatalf("Test %d: failed to create ErasureStorage: %v", i, err)
} }
// Create a test file. offline := make([]StorageAPI, len(storage.disks))
_, size, checkSums, err := erasureCreateFile(disks, "testbucket", "testobject1", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1) copy(offline, storage.disks)
data := make([]byte, test.size)
if _, err = io.ReadFull(rand.Reader, data); err != nil {
setup.Remove()
t.Fatalf("Test %d: failed to create random test data: %v", i, err)
}
algorithm := test.algorithm
if !algorithm.Available() {
algorithm = DefaultBitrotAlgorithm
}
buffer := make([]byte, test.blocksize, 2*test.blocksize)
file, err := storage.CreateFile(bytes.NewReader(data), "testbucket", "testobject", buffer, algorithm, test.dataBlocks+1)
if err != nil { if err != nil {
t.Fatal(err) setup.Remove()
} t.Fatalf("Test %d: failed to create random test data: %v", i, err)
if size != int64(len(data)) {
t.Errorf("erasureCreateFile returned %d, expected %d", size, len(data))
} }
latest := make([]StorageAPI, len(disks)) // Slice of latest disks info, err := storage.HealFile(offline, "testbucket", "testobject", test.blocksize, "testbucket", "healedobject", test.size, test.algorithm, file.Checksums)
outDated := make([]StorageAPI, len(disks)) // Slice of outdated disks if err != nil && !test.shouldFail {
t.Errorf("Test %d: should pass but it failed with: %v", i, err)
// Test case when one part needs to be healed.
dataPath := path.Join(setup.diskPaths[0], "testbucket", "testobject1")
err = os.Remove(dataPath)
if err != nil {
t.Fatal(err)
} }
copy(latest, disks) if err == nil && test.shouldFail {
latest[0] = nil t.Errorf("Test %d: should fail but it passed", i)
outDated[0] = disks[0]
healCheckSums, err := erasureHealFile(latest, outDated, "testbucket", "testobject1", "testbucket", "testobject1", 1*humanize.MiByte, blockSize, dataBlocks, parityBlocks, bitRotAlgo)
if err != nil {
t.Fatal(err)
} }
// Checksum of the healed file should match.
if checkSums[0] != healCheckSums[0] {
t.Error("Healing failed, data does not match.")
}
// Test case when parityBlocks number of disks need to be healed.
// Should succeed.
copy(latest, disks)
for index := 0; index < parityBlocks; index++ {
dataPath := path.Join(setup.diskPaths[index], "testbucket", "testobject1")
err = os.Remove(dataPath)
if err != nil {
t.Fatal(err)
}
latest[index] = nil
outDated[index] = disks[index]
}
healCheckSums, err = erasureHealFile(latest, outDated, "testbucket", "testobject1", "testbucket", "testobject1", 1*humanize.MiByte, blockSize, dataBlocks, parityBlocks, bitRotAlgo)
if err != nil {
t.Fatal(err)
}
// Checksums of the healed files should match.
for index := 0; index < parityBlocks; index++ {
if checkSums[index] != healCheckSums[index] {
t.Error("Healing failed, data does not match.")
}
}
for index := dataBlocks; index < len(disks); index++ {
if healCheckSums[index] != "" {
t.Errorf("expected healCheckSums[%d] to be empty", index)
}
}
// Test case when parityBlocks+1 number of disks need to be healed.
// Should fail.
copy(latest, disks)
for index := 0; index < parityBlocks+1; index++ {
dataPath := path.Join(setup.diskPaths[index], "testbucket", "testobject1")
err = os.Remove(dataPath)
if err != nil {
t.Fatal(err)
}
latest[index] = nil
outDated[index] = disks[index]
}
_, err = erasureHealFile(latest, outDated, "testbucket", "testobject1", "testbucket", "testobject1", 1*humanize.MiByte, blockSize, dataBlocks, parityBlocks, bitRotAlgo)
if err == nil { if err == nil {
t.Error("Expected erasureHealFile() to fail when the number of available disks <= parityBlocks") if info.Size != test.size {
t.Errorf("Test %d: healed wrong number of bytes: got: #%d want: #%d", i, info.Size, test.size)
}
if info.Algorithm != test.algorithm {
t.Errorf("Test %d: healed with wrong algorithm: got: %v want: %v", i, info.Algorithm, test.algorithm)
}
if !reflect.DeepEqual(info.Checksums, file.Checksums) {
t.Errorf("Test %d: heal returned different bitrot keys", i)
}
}
if err == nil && !test.shouldFail {
for j := 0; j < len(storage.disks); j++ {
if j < test.offDisks {
storage.disks[j] = OfflineDisk
} else {
offline[j] = OfflineDisk
}
}
for j := 0; j < test.badDisks; j++ {
storage.disks[test.offDisks+j] = badDisk{nil}
}
for j := 0; j < test.badOffDisks; j++ {
offline[j] = badDisk{nil}
}
info, err := storage.HealFile(offline, "testbucket", "testobject", test.blocksize, "testbucket", "healedobject", test.size, test.algorithm, file.Checksums)
if err != nil && !test.shouldFailQuorum {
t.Errorf("Test %d: should pass but it failed with: %v", i, err)
}
if err == nil && test.shouldFailQuorum {
t.Errorf("Test %d: should fail but it passed", i)
}
if err == nil {
if info.Size != test.size {
t.Errorf("Test %d: healed wrong number of bytes: got: #%d want: #%d", i, info.Size, test.size)
}
if info.Algorithm != test.algorithm {
t.Errorf("Test %d: healed with wrong algorithm: got: %v want: %v", i, info.Algorithm, test.algorithm)
}
if !reflect.DeepEqual(info.Checksums, file.Checksums) {
t.Errorf("Test %d: heal returned different bitrot checksums", i)
}
}
}
setup.Remove()
} }
} }

View File

@ -18,321 +18,150 @@ package cmd
import ( import (
"io" "io"
"sync"
"github.com/klauspost/reedsolomon"
"github.com/minio/minio/pkg/bpool" "github.com/minio/minio/pkg/bpool"
) )
// isSuccessDecodeBlocks - do we have all the blocks to be // ReadFile reads as much data as requested from the file under the given volume and path and writes the data to the provided writer.
// successfully decoded?. Input encoded blocks ordered matrix. // The algorithm and the keys/checksums are used to verify the integrity of the given file. ReadFile will read data from the given offset
func isSuccessDecodeBlocks(enBlocks [][]byte, dataBlocks int) bool { // up to the given length. If parts of the file are corrupted ReadFile tries to reconstruct the data.
// Count number of data and parity blocks that were read. 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) {
var successDataBlocksCount = 0
var successParityBlocksCount = 0
for index := range enBlocks {
if enBlocks[index] == nil {
continue
}
// block index lesser than data blocks, update data block count.
if index < dataBlocks {
successDataBlocksCount++
continue
} // else { // update parity block count.
successParityBlocksCount++
}
// Returns true if we have atleast dataBlocks parity.
return successDataBlocksCount == dataBlocks || successDataBlocksCount+successParityBlocksCount >= dataBlocks
}
// isSuccessDataBlocks - do we have all the data blocks?
// Input encoded blocks ordered matrix.
func isSuccessDataBlocks(enBlocks [][]byte, dataBlocks int) bool {
// Count number of data blocks that were read.
var successDataBlocksCount = 0
for index := range enBlocks[:dataBlocks] {
if enBlocks[index] == nil {
continue
}
// block index lesser than data blocks, update data block count.
if index < dataBlocks {
successDataBlocksCount++
}
}
// Returns true if we have atleast the dataBlocks.
return successDataBlocksCount >= dataBlocks
}
// Return readable disks slice from which we can read parallelly.
func getReadDisks(orderedDisks []StorageAPI, index int, dataBlocks int) (readDisks []StorageAPI, nextIndex int, err error) {
readDisks = make([]StorageAPI, len(orderedDisks))
dataDisks := 0
parityDisks := 0
// Count already read data and parity chunks.
for i := 0; i < index; i++ {
if orderedDisks[i] == nil {
continue
}
if i < dataBlocks {
dataDisks++
} else {
parityDisks++
}
}
// Sanity checks - we should never have this situation.
if dataDisks == dataBlocks {
return nil, 0, traceError(errUnexpected)
}
if dataDisks+parityDisks >= dataBlocks {
return nil, 0, traceError(errUnexpected)
}
// Find the disks from which next set of parallel reads should happen.
for i := index; i < len(orderedDisks); i++ {
if orderedDisks[i] == nil {
continue
}
if i < dataBlocks {
dataDisks++
} else {
parityDisks++
}
readDisks[i] = orderedDisks[i]
if dataDisks == dataBlocks {
return readDisks, i + 1, nil
} else if dataDisks+parityDisks == dataBlocks {
return readDisks, i + 1, nil
}
}
return nil, 0, traceError(errXLReadQuorum)
}
// parallelRead - reads chunks in parallel from the disks specified in []readDisks.
func parallelRead(volume, path string, readDisks, orderedDisks []StorageAPI, enBlocks [][]byte,
blockOffset, curChunkSize int64, brVerifiers []bitRotVerifier, pool *bpool.BytePool) {
// WaitGroup to synchronise the read go-routines.
wg := &sync.WaitGroup{}
// Read disks in parallel.
for index := range readDisks {
if readDisks[index] == nil {
continue
}
wg.Add(1)
// Reads chunk from readDisk[index] in routine.
go func(index int) {
defer wg.Done()
// evaluate if we need to perform bit-rot checking
needBitRotVerification := true
if brVerifiers[index].isVerified {
needBitRotVerification = false
// if file has bit-rot, do not reuse disk
if brVerifiers[index].hasBitRot {
orderedDisks[index] = nil
return
}
}
buf, err := pool.Get()
if err != nil {
errorIf(err, "unable to get buffer from byte pool")
orderedDisks[index] = nil
return
}
buf = buf[:curChunkSize]
if needBitRotVerification {
_, err = readDisks[index].ReadFileWithVerify(
volume, path, blockOffset, buf,
brVerifiers[index].algo,
brVerifiers[index].checkSum)
} else {
_, err = readDisks[index].ReadFile(volume, path,
blockOffset, buf)
}
// if bit-rot verification was done, store the
// result of verification so we can skip
// re-doing it next time
if needBitRotVerification {
brVerifiers[index].isVerified = true
_, ok := err.(hashMismatchError)
brVerifiers[index].hasBitRot = ok
}
if err != nil {
orderedDisks[index] = nil
return
}
enBlocks[index] = buf
}(index)
}
// Waiting for first routines to finish.
wg.Wait()
}
// erasureReadFile - read bytes from erasure coded files and writes to
// given writer. Erasure coded files are read block by block as per
// given erasureInfo and data chunks are decoded into a data
// block. Data block is trimmed for given offset and length, then
// written to given writer. This function also supports bit-rot
// detection by verifying checksum of individual block's checksum.
func erasureReadFile(writer io.Writer, disks []StorageAPI, volume, path string,
offset, length, totalLength, blockSize int64, dataBlocks, parityBlocks int,
checkSums []string, algo HashAlgo, pool *bpool.BytePool) (int64, error) {
// Offset and length cannot be negative.
if offset < 0 || length < 0 { if offset < 0 || length < 0 {
return 0, traceError(errUnexpected) return f, traceError(errUnexpected)
} }
// Can't request more data than what is available.
if offset+length > totalLength { if offset+length > totalLength {
return 0, traceError(errUnexpected) return f, traceError(errUnexpected)
}
if !algorithm.Available() {
return f, traceError(errBitrotHashAlgoInvalid)
} }
// chunkSize is the amount of data that needs to be read from f.Checksums = make([][]byte, len(s.disks))
// each disk at a time. verifiers := make([]*BitrotVerifier, len(s.disks))
chunkSize := getChunkSize(blockSize, dataBlocks) for i, disk := range s.disks {
if disk == OfflineDisk {
brVerifiers := make([]bitRotVerifier, len(disks)) continue
for i := range brVerifiers {
brVerifiers[i].algo = algo
brVerifiers[i].checkSum = checkSums[i]
} }
verifiers[i] = NewBitrotVerifier(algorithm, checksums[i])
}
errChans := make([]chan error, len(s.disks))
for i := range errChans {
errChans[i] = make(chan error, 1)
}
lastBlock := totalLength / blocksize
startOffset := offset % blocksize
chunksize := getChunkSize(blocksize, s.dataBlocks)
// Total bytes written to writer blocks := make([][]byte, len(s.disks))
var bytesWritten int64 for off := offset / blocksize; length > 0; off++ {
blockOffset := off * chunksize
startBlock := offset / blockSize
endBlock := (offset + length) / blockSize
// curChunkSize = chunk size for the current block in the for loop below.
// curBlockSize = block size for the current block in the for loop below.
// curChunkSize and curBlockSize can change for the last block if totalLength%blockSize != 0
curChunkSize := chunkSize
curBlockSize := blockSize
// For each block, read chunk from each disk. If we are able to read all the data disks then we don't
// need to read parity disks. If one of the data disk is missing we need to read DataBlocks+1 number
// of disks. Once read, we Reconstruct() missing data if needed and write it to the given writer.
for block := startBlock; block <= endBlock; block++ {
// Mark all buffers as unused at the start of the loop so that the buffers
// can be reused.
pool.Reset() pool.Reset()
// Each element of enBlocks holds curChunkSize'd amount of data read from its corresponding disk. if currentBlock := (offset + f.Size) / blocksize; currentBlock == lastBlock {
enBlocks := make([][]byte, len(disks)) blocksize = totalLength % blocksize
chunksize = getChunkSize(blocksize, s.dataBlocks)
if ((offset + bytesWritten) / blockSize) == (totalLength / blockSize) { }
// This is the last block for which curBlockSize and curChunkSize can change. err = s.readConcurrent(volume, path, blockOffset, chunksize, blocks, verifiers, errChans, pool)
// For ex. if totalLength is 15M and blockSize is 10MB, curBlockSize for if err != nil {
// the last block should be 5MB. return f, traceError(errXLReadQuorum)
curBlockSize = totalLength % blockSize
curChunkSize = getChunkSize(curBlockSize, dataBlocks)
} }
// NOTE: That for the offset calculation we have to use chunkSize and writeLength := blocksize - startOffset
// not curChunkSize. If we use curChunkSize for offset calculation if length < writeLength {
// then it can result in wrong offset for the last block. writeLength = length
blockOffset := block * chunkSize }
n, err := writeDataBlocks(writer, blocks, s.dataBlocks, startOffset, writeLength)
if err != nil {
return f, err
}
startOffset = 0
f.Size += int64(n)
length -= int64(n)
}
// nextIndex - index from which next set of parallel reads f.Algorithm = algorithm
// should happen. for i, disk := range s.disks {
nextIndex := 0 if disk == OfflineDisk {
continue
}
f.Checksums[i] = verifiers[i].Sum(nil)
}
return f, nil
}
for { func erasureCountMissingBlocks(blocks [][]byte, limit int) int {
// readDisks - disks from which we need to read in parallel. missing := 0
var readDisks []StorageAPI for i := range blocks[:limit] {
if blocks[i] == nil {
missing++
}
}
return missing
}
// 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) {
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)
mustReconstruct := missingDataBlocks > 0
if mustReconstruct {
requiredReads := s.dataBlocks + missingDataBlocks
if requiredReads > s.dataBlocks+s.parityBlocks {
return errXLReadQuorum
}
erasureReadBlocksConcurrent(s.disks[s.dataBlocks:requiredReads], volume, path, offset, blocks[s.dataBlocks:requiredReads], verifiers[s.dataBlocks:requiredReads], errs[s.dataBlocks:requiredReads], errChans[s.dataBlocks:requiredReads])
if erasureCountMissingBlocks(blocks, requiredReads) > 0 {
erasureReadBlocksConcurrent(s.disks[requiredReads:], volume, path, offset, blocks[requiredReads:], verifiers[requiredReads:], errs[requiredReads:], errChans[requiredReads:])
}
}
if err = reduceReadQuorumErrs(errs, []error{}, s.dataBlocks); err != nil {
return err
}
if mustReconstruct {
if err = s.ErasureDecodeDataBlocks(blocks); err != nil {
return err
}
}
return nil
}
// erasureReadBlocksConcurrent reads all data from each disk to each data block in parallel.
// Therefore disks, blocks, verifiers errors and locks must have the same length.
func erasureReadBlocksConcurrent(disks []StorageAPI, volume, path string, offset int64, blocks [][]byte, verifiers []*BitrotVerifier, errors []error, errChans []chan error) {
for i := range errChans {
go erasureReadFromFile(disks[i], volume, path, offset, blocks[i], verifiers[i], errChans[i])
}
for i := range errChans {
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
}
}
}
// erasureReadFromFile reads data from the disk to buffer in parallel.
// It sends the returned error through the error channel.
func erasureReadFromFile(disk StorageAPI, volume, path string, offset int64, buffer []byte, verifier *BitrotVerifier, errChan chan<- error) {
if disk == OfflineDisk {
errChan <- traceError(errDiskNotFound)
return
}
var err error var err error
// get readable disks slice from which we can read parallelly. if !verifier.IsVerified() {
readDisks, nextIndex, err = getReadDisks(disks, nextIndex, dataBlocks) _, err = disk.ReadFileWithVerify(volume, path, offset, buffer, verifier)
if err != nil { } else {
return bytesWritten, err _, err = disk.ReadFile(volume, path, offset, buffer)
} }
// Issue a parallel read across the disks specified in readDisks. errChan <- err
parallelRead(volume, path, readDisks, disks, enBlocks, blockOffset, curChunkSize, brVerifiers, pool)
if isSuccessDecodeBlocks(enBlocks, dataBlocks) {
// If enough blocks are available to do rs.Reconstruct()
break
}
if nextIndex == len(disks) {
// No more disks to read from.
return bytesWritten, traceError(errXLReadQuorum)
}
// We do not have enough enough data blocks to reconstruct the data
// hence continue the for-loop till we have enough data blocks.
}
// If we have all the data blocks no need to decode, continue to write.
if !isSuccessDataBlocks(enBlocks, dataBlocks) {
// Reconstruct the missing data blocks.
if err := decodeMissingData(enBlocks, dataBlocks, parityBlocks); err != nil {
return bytesWritten, err
}
}
// Offset in enBlocks from where data should be read from.
var enBlocksOffset int64
// Total data to be read from enBlocks.
enBlocksLength := curBlockSize
// If this is the start block then enBlocksOffset might not be 0.
if block == startBlock {
enBlocksOffset = offset % blockSize
enBlocksLength -= enBlocksOffset
}
remaining := length - bytesWritten
if remaining < enBlocksLength {
// We should not send more data than what was requested.
enBlocksLength = remaining
}
// Write data blocks.
n, err := writeDataBlocks(writer, enBlocks, dataBlocks, enBlocksOffset, enBlocksLength)
if err != nil {
return bytesWritten, err
}
// Update total bytes written.
bytesWritten += n
if bytesWritten == length {
// Done writing all the requested data.
break
}
}
// Success.
return bytesWritten, nil
}
// decodeMissingData - decode any missing data blocks.
func decodeMissingData(enBlocks [][]byte, dataBlocks, parityBlocks int) error {
// Initialize reedsolomon.
rs, err := reedsolomon.New(dataBlocks, parityBlocks)
if err != nil {
return traceError(err)
}
// Reconstruct any missing data blocks.
return rs.ReconstructData(enBlocks)
}
// decodeDataAndParity - decode all encoded data and parity blocks.
func decodeDataAndParity(enBlocks [][]byte, dataBlocks, parityBlocks int) error {
// Initialize reedsolomon.
rs, err := reedsolomon.New(dataBlocks, parityBlocks)
if err != nil {
return traceError(err)
}
// Reconstruct encoded blocks.
return rs.Reconstruct(enBlocks)
} }

View File

@ -18,358 +18,148 @@ package cmd
import ( import (
"bytes" "bytes"
crand "crypto/rand"
"io"
"math/rand" "math/rand"
"testing" "testing"
"reflect"
humanize "github.com/dustin/go-humanize" humanize "github.com/dustin/go-humanize"
"github.com/minio/minio/pkg/bpool" "github.com/minio/minio/pkg/bpool"
) )
// Tests getReadDisks which returns readable disks slice from which we can func (d badDisk) ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error) {
// read parallelly.
func testGetReadDisks(t *testing.T, xl *xlObjects) {
d := xl.storageDisks
testCases := []struct {
index int // index argument for getReadDisks
argDisks []StorageAPI // disks argument for getReadDisks
retDisks []StorageAPI // disks return value from getReadDisks
nextIndex int // return value from getReadDisks
err error // error return value from getReadDisks
}{
// Test case - 1.
// When all disks are available, should return data disks.
{
0,
[]StorageAPI{d[0], d[1], d[2], d[3], d[4], d[5], d[6], d[7], d[8], d[9], d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{d[0], d[1], d[2], d[3], d[4], d[5], d[6], d[7], nil, nil, nil, nil, nil, nil, nil, nil},
8,
nil,
},
// Test case - 2.
// If a parity disk is down, should return all data disks.
{
0,
[]StorageAPI{d[0], d[1], d[2], d[3], d[4], d[5], d[6], d[7], d[8], nil, d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{d[0], d[1], d[2], d[3], d[4], d[5], d[6], d[7], nil, nil, nil, nil, nil, nil, nil, nil},
8,
nil,
},
// Test case - 3.
// If a data disk is down, should return 7 data and 1 parity.
{
0,
[]StorageAPI{nil, d[1], d[2], d[3], d[4], d[5], d[6], d[7], d[8], d[9], d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{nil, d[1], d[2], d[3], d[4], d[5], d[6], d[7], d[8], nil, nil, nil, nil, nil, nil, nil},
9,
nil,
},
// Test case - 4.
// If 7 data disks are down, should return 1 data and 7 parity.
{
0,
[]StorageAPI{nil, nil, nil, nil, nil, nil, nil, d[7], d[8], d[9], d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{nil, nil, nil, nil, nil, nil, nil, d[7], d[8], d[9], d[10], d[11], d[12], d[13], d[14], nil},
15,
nil,
},
// Test case - 5.
// When 2 disks fail during parallelRead, next call to getReadDisks should return 3 disks
{
8,
[]StorageAPI{nil, nil, d[2], d[3], d[4], d[5], d[6], d[7], d[8], d[9], d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{nil, nil, nil, nil, nil, nil, nil, nil, d[8], d[9], nil, nil, nil, nil, nil, nil},
10,
nil,
},
// Test case - 6.
// If 2 disks again fail from the 3 disks returned previously, return next 2 disks
{
11,
[]StorageAPI{nil, nil, d[2], d[3], d[4], d[5], d[6], d[7], nil, nil, d[10], d[11], d[12], d[13], d[14], d[15]},
[]StorageAPI{nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, d[11], nil, nil, nil, nil},
12,
nil,
},
// Test case - 7.
// No more disks are available for read, return error
{
13,
[]StorageAPI{nil, nil, d[2], d[3], d[4], d[5], d[6], d[7], nil, nil, d[10], nil, nil, nil, nil, nil},
nil,
0,
errXLReadQuorum,
},
}
for i, test := range testCases {
disks, nextIndex, err := getReadDisks(test.argDisks, test.index, xl.dataBlocks)
if errorCause(err) != test.err {
t.Errorf("test-case %d - expected error : %s, got : %s", i+1, test.err, err)
continue
}
if test.nextIndex != nextIndex {
t.Errorf("test-case %d - expected nextIndex: %d, got : %d", i+1, test.nextIndex, nextIndex)
continue
}
if !reflect.DeepEqual(test.retDisks, disks) {
t.Errorf("test-case %d : incorrect disks returned. expected %+v, got %+v", i+1, test.retDisks, disks)
continue
}
}
}
// Test for isSuccessDataBlocks and isSuccessDecodeBlocks.
func TestIsSuccessBlocks(t *testing.T) {
dataBlocks := 8
testCases := []struct {
enBlocks [][]byte // data and parity blocks.
successData bool // expected return value of isSuccessDataBlocks()
successDecode bool // expected return value of isSuccessDecodeBlocks()
}{
{
// When all data and partity blocks are available.
[][]byte{
{'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
{'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
},
true,
true,
},
{
// When one data block is not available.
[][]byte{
nil, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
{'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
},
false,
true,
},
{
// When one data and all parity are available, enough for reedsolomon.Reconstruct()
[][]byte{
nil, nil, nil, nil, nil, nil, nil, {'a'},
{'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
},
false,
true,
},
{
// When all data disks are not available, enough for reedsolomon.Reconstruct()
[][]byte{
nil, nil, nil, nil, nil, nil, nil, nil,
{'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
},
false,
true,
},
{
// Not enough disks for reedsolomon.Reconstruct()
[][]byte{
nil, nil, nil, nil, nil, nil, nil, nil,
nil, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'}, {'a'},
},
false,
false,
},
}
for i, test := range testCases {
got := isSuccessDataBlocks(test.enBlocks, dataBlocks)
if test.successData != got {
t.Errorf("test-case %d : expected %v got %v", i+1, test.successData, got)
}
got = isSuccessDecodeBlocks(test.enBlocks, dataBlocks)
if test.successDecode != got {
t.Errorf("test-case %d : expected %v got %v", i+1, test.successDecode, got)
}
}
}
// Wrapper function for testGetReadDisks, testShuffleDisks.
func TestErasureReadUtils(t *testing.T) {
nDisks := 16
disks, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
objLayer, _, err := initObjectLayer(mustGetNewEndpointList(disks...))
if err != nil {
removeRoots(disks)
t.Fatal(err)
}
defer removeRoots(disks)
xl := objLayer.(*xlObjects)
testGetReadDisks(t, xl)
}
// Simulates a faulty disk for ReadFile()
type ReadDiskDown struct {
*posix
}
func (r ReadDiskDown) ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error) {
return 0, errFaultyDisk return 0, errFaultyDisk
} }
func (r ReadDiskDown) ReadFileWithVerify(volume string, path string, offset int64, buf []byte, func (d badDisk) ReadFileWithVerify(volume string, path string, offset int64, buf []byte, verifier *BitrotVerifier) (n int64, err error) {
algo HashAlgo, expectedHash string) (n int64, err error) {
return 0, errFaultyDisk return 0, errFaultyDisk
} }
func TestErasureReadFileDiskFail(t *testing.T) { var erasureReadFileTests = []struct {
// Initialize environment needed for the test. dataBlocks int
dataBlocks := 7 onDisks, offDisks int
parityBlocks := 7 blocksize, data int64
blockSize := int64(blockSizeV1) offset int64
setup, err := newErasureTestSetup(dataBlocks, parityBlocks, blockSize) length int64
if err != nil { algorithm BitrotAlgorithm
t.Error(err) shouldFail, shouldFailQuorum bool
return
}
defer setup.Remove()
disks := setup.disks
// Prepare a slice of 1humanize.MiByte with random data.
data := make([]byte, 1*humanize.MiByte)
length := int64(len(data))
_, err = rand.Read(data)
if err != nil {
t.Fatal(err)
}
// Create a test file to read from.
_, size, checkSums, err := erasureCreateFile(disks, "testbucket", "testobject", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if err != nil {
t.Fatal(err)
}
if size != length {
t.Errorf("erasureCreateFile returned %d, expected %d", size, length)
}
// create byte pool which will be used by erasureReadFile for
// reading from disks and erasure decoding.
chunkSize := getChunkSize(blockSize, dataBlocks)
pool := bpool.NewBytePool(chunkSize, len(disks))
buf := &bytes.Buffer{}
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", 0, length, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool)
if err != nil {
t.Error(err)
}
if !bytes.Equal(buf.Bytes(), data) {
t.Error("Contents of the erasure coded file differs")
}
// 2 disks down. Read should succeed.
disks[4] = ReadDiskDown{disks[4].(*posix)}
disks[5] = ReadDiskDown{disks[5].(*posix)}
buf.Reset()
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", 0, length, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool)
if err != nil {
t.Error(err)
}
if !bytes.Equal(buf.Bytes(), data) {
t.Error("Contents of the erasure coded file differs")
}
// 4 more disks down. 6 disks down in total. Read should succeed.
disks[6] = ReadDiskDown{disks[6].(*posix)}
disks[8] = ReadDiskDown{disks[8].(*posix)}
disks[9] = ReadDiskDown{disks[9].(*posix)}
disks[11] = ReadDiskDown{disks[11].(*posix)}
buf.Reset()
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", 0, length, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool)
if err != nil {
t.Error(err)
}
if !bytes.Equal(buf.Bytes(), data) {
t.Error("Contents of the erasure coded file differs")
}
// 2 more disk down. 8 disks down in total. Read should fail.
disks[12] = ReadDiskDown{disks[12].(*posix)}
disks[13] = ReadDiskDown{disks[13].(*posix)}
buf.Reset()
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", 0, length, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool)
if errorCause(err) != errXLReadQuorum {
t.Fatal("expected errXLReadQuorum error")
}
}
func TestErasureReadFileOffsetLength(t *testing.T) {
// Initialize environment needed for the test.
dataBlocks := 7
parityBlocks := 7
blockSize := int64(1 * humanize.MiByte)
setup, err := newErasureTestSetup(dataBlocks, parityBlocks, blockSize)
if err != nil {
t.Error(err)
return
}
defer setup.Remove()
disks := setup.disks
// Prepare a slice of 5humanize.MiByte with random data.
data := make([]byte, 5*humanize.MiByte)
length := int64(len(data))
_, err = rand.Read(data)
if err != nil {
t.Fatal(err)
}
// Create a test file to read from.
_, size, checkSums, err := erasureCreateFile(disks, "testbucket", "testobject", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1)
if err != nil {
t.Fatal(err)
}
if size != length {
t.Errorf("erasureCreateFile returned %d, expected %d", size, length)
}
testCases := []struct {
offset, length int64
}{ }{
// Full file. {dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 0
{0, length}, {dataBlocks: 3, onDisks: 6, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 1
// Read nothing. {dataBlocks: 4, onDisks: 8, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 2
{length, 0}, {dataBlocks: 5, onDisks: 10, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 1, length: oneMiByte - 1, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 3
// 2nd block. {dataBlocks: 6, onDisks: 12, offDisks: 0, blocksize: int64(oneMiByte), data: oneMiByte, offset: oneMiByte, length: 0, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 4
{blockSize, blockSize}, {dataBlocks: 7, onDisks: 14, offDisks: 0, blocksize: int64(oneMiByte), data: oneMiByte, offset: 3, length: 1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 5
// Test cases for random offsets and lengths. {dataBlocks: 8, onDisks: 16, offDisks: 0, blocksize: int64(oneMiByte), data: oneMiByte, offset: 4, length: 8 * 1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 6
{blockSize - 1, 2}, {dataBlocks: 7, onDisks: 14, offDisks: 7, blocksize: int64(blockSizeV1), data: oneMiByte, offset: oneMiByte, length: 1, algorithm: DefaultBitrotAlgorithm, shouldFail: true, shouldFailQuorum: false}, // 7
{blockSize - 1, blockSize + 1}, {dataBlocks: 6, onDisks: 12, offDisks: 6, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 8
{blockSize + 1, blockSize - 1}, {dataBlocks: 5, onDisks: 10, offDisks: 5, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 9
{blockSize + 1, blockSize}, {dataBlocks: 4, onDisks: 8, offDisks: 4, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 10
{blockSize + 1, blockSize + 1}, {dataBlocks: 3, onDisks: 6, offDisks: 3, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 11
{blockSize*2 - 1, blockSize + 1}, {dataBlocks: 2, onDisks: 4, offDisks: 2, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 12
{length - 1, 1}, {dataBlocks: 2, onDisks: 4, offDisks: 1, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 13
{length - blockSize, blockSize}, {dataBlocks: 3, onDisks: 6, offDisks: 2, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 14
{length - blockSize - 1, blockSize}, {dataBlocks: 4, onDisks: 8, offDisks: 3, blocksize: int64(2 * oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 15
{length - blockSize - 1, blockSize + 1}, {dataBlocks: 5, onDisks: 10, offDisks: 6, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 16
{dataBlocks: 5, onDisks: 10, offDisks: 2, blocksize: int64(blockSizeV1), data: 2 * oneMiByte, offset: oneMiByte, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 17
{dataBlocks: 5, onDisks: 10, offDisks: 1, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 18
{dataBlocks: 6, onDisks: 12, offDisks: 3, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: SHA256, shouldFail: false, shouldFailQuorum: false}, // 19
{dataBlocks: 6, onDisks: 12, offDisks: 7, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 20
{dataBlocks: 8, onDisks: 16, offDisks: 8, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 21
{dataBlocks: 8, onDisks: 16, offDisks: 9, blocksize: int64(oneMiByte), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 22
{dataBlocks: 8, onDisks: 16, offDisks: 7, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 23
{dataBlocks: 2, onDisks: 4, offDisks: 1, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 24
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 25
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: oneMiByte, offset: 0, length: oneMiByte, algorithm: 0, shouldFail: true, shouldFailQuorum: false}, // 26
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(blockSizeV1) + 1, offset: 0, length: int64(blockSizeV1) + 1, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 27
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 12, length: int64(blockSizeV1) + 17, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 28
{dataBlocks: 3, onDisks: 6, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 1023, length: int64(blockSizeV1) + 1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 29
{dataBlocks: 4, onDisks: 8, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 11, length: int64(blockSizeV1) + 2*1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 30
{dataBlocks: 6, onDisks: 12, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 512, length: int64(blockSizeV1) + 8*1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 31
{dataBlocks: 8, onDisks: 16, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: int64(blockSizeV1), length: int64(blockSizeV1) - 1, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 32
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(oneMiByte), offset: -1, length: 3, algorithm: DefaultBitrotAlgorithm, shouldFail: true, shouldFailQuorum: false}, // 33
{dataBlocks: 2, onDisks: 4, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(oneMiByte), offset: 1024, length: -1, algorithm: DefaultBitrotAlgorithm, shouldFail: true, shouldFailQuorum: false}, // 34
{dataBlocks: 4, onDisks: 6, offDisks: 0, blocksize: int64(blockSizeV1), data: int64(blockSizeV1), offset: 0, length: int64(blockSizeV1), algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 35
{dataBlocks: 4, onDisks: 6, offDisks: 1, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 12, length: int64(blockSizeV1) + 17, algorithm: BLAKE2b512, shouldFail: false, shouldFailQuorum: false}, // 36
{dataBlocks: 4, onDisks: 6, offDisks: 3, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 1023, length: int64(blockSizeV1) + 1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: true}, // 37
{dataBlocks: 8, onDisks: 12, offDisks: 4, blocksize: int64(blockSizeV1), data: int64(2 * blockSizeV1), offset: 11, length: int64(blockSizeV1) + 2*1024, algorithm: DefaultBitrotAlgorithm, shouldFail: false, shouldFailQuorum: false}, // 38
} }
chunkSize := getChunkSize(blockSize, dataBlocks)
pool := bpool.NewBytePool(chunkSize, len(disks))
// Compare the data read from file with "data" byte array. func TestErasureReadFile(t *testing.T) {
for i, testCase := range testCases { for i, test := range erasureReadFileTests {
expected := data[testCase.offset:(testCase.offset + testCase.length)] setup, err := newErasureTestSetup(test.dataBlocks, test.onDisks-test.dataBlocks, test.blocksize)
buf := &bytes.Buffer{}
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", testCase.offset, testCase.length, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool)
if err != nil { if err != nil {
t.Error(err) t.Fatalf("Test %d: failed to create test setup: %v", i, err)
continue
} }
got := buf.Bytes() storage, err := NewErasureStorage(setup.disks, test.dataBlocks, test.onDisks-test.dataBlocks)
if !bytes.Equal(expected, got) { if err != nil {
t.Errorf("Test %d : read data is different from what was expected", i+1) setup.Remove()
t.Fatalf("Test %d: failed to create ErasureStorage: %v", i, err)
} }
data := make([]byte, test.data)
if _, err = io.ReadFull(crand.Reader, data); err != nil {
setup.Remove()
t.Fatalf("Test %d: failed to generate random test data: %v", i, err)
}
writeAlgorithm := test.algorithm
if !test.algorithm.Available() {
writeAlgorithm = DefaultBitrotAlgorithm
}
buffer := make([]byte, test.blocksize, 2*test.blocksize)
file, err := storage.CreateFile(bytes.NewReader(data[:]), "testbucket", "object", buffer, writeAlgorithm, test.dataBlocks+1)
if err != nil {
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)
if err != nil && !test.shouldFail {
t.Errorf("Test %d: should pass but failed with: %v", i, err)
}
if err == nil && test.shouldFail {
t.Errorf("Test %d: should fail but it passed", i)
}
if err == nil {
if readInfo.Size != test.length {
t.Errorf("Test %d: read returns wrong number of bytes: got: #%d want: #%d", i, readInfo.Size, test.length)
}
if readInfo.Algorithm != test.algorithm {
t.Errorf("Test %d: read returns wrong algorithm: got: %v want: %v", i, readInfo.Algorithm, test.algorithm)
}
if content := writer.Bytes(); !bytes.Equal(content, data[test.offset:test.offset+test.length]) {
t.Errorf("Test %d: read retruns wrong file content", i)
}
}
if err == nil && !test.shouldFail {
writer.Reset()
for j := range storage.disks[:test.offDisks] {
storage.disks[j] = badDisk{nil}
}
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)
if err != nil && !test.shouldFailQuorum {
t.Errorf("Test %d: should pass but failed with: %v", i, err)
}
if err == nil && test.shouldFailQuorum {
t.Errorf("Test %d: should fail but it passed", i)
}
if !test.shouldFailQuorum {
if readInfo.Size != test.length {
t.Errorf("Test %d: read returns wrong number of bytes: got: #%d want: #%d", i, readInfo.Size, test.length)
}
if readInfo.Algorithm != test.algorithm {
t.Errorf("Test %d: read returns wrong algorithm: got: %v want: %v", i, readInfo.Algorithm, test.algorithm)
}
if content := writer.Bytes(); !bytes.Equal(content, data[test.offset:test.offset+test.length]) {
t.Errorf("Test %d: read retruns wrong file content", i)
}
}
}
setup.Remove()
} }
} }
@ -390,8 +180,10 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
} }
defer setup.Remove() defer setup.Remove()
disks := setup.disks storage, err := NewErasureStorage(setup.disks, dataBlocks, parityBlocks)
if err != nil {
t.Fatalf("failed to create ErasureStorage: %v", err)
}
// Prepare a slice of 5MiB with random data. // Prepare a slice of 5MiB with random data.
data := make([]byte, 5*humanize.MiByte) data := make([]byte, 5*humanize.MiByte)
length := int64(len(data)) length := int64(len(data))
@ -404,12 +196,13 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
iterations := 10000 iterations := 10000
// Create a test file to read from. // Create a test file to read from.
_, size, checkSums, err := erasureCreateFile(disks, "testbucket", "testobject", bytes.NewReader(data), true, blockSize, dataBlocks, parityBlocks, bitRotAlgo, dataBlocks+1) buffer := make([]byte, blockSize, 2*blockSize)
file, err := storage.CreateFile(bytes.NewReader(data), "testbucket", "testobject", buffer, DefaultBitrotAlgorithm, dataBlocks+1)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
if size != length { if file.Size != length {
t.Errorf("erasureCreateFile returned %d, expected %d", size, length) t.Errorf("erasureCreateFile returned %d, expected %d", file.Size, length)
} }
// To generate random offset/length. // To generate random offset/length.
@ -418,7 +211,7 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
// create pool buffer which will be used by erasureReadFile for // create pool buffer which will be used by erasureReadFile for
// reading from disks and erasure decoding. // reading from disks and erasure decoding.
chunkSize := getChunkSize(blockSize, dataBlocks) chunkSize := getChunkSize(blockSize, dataBlocks)
pool := bpool.NewBytePool(chunkSize, len(disks)) pool := bpool.NewBytePool(chunkSize, len(storage.disks))
buf := &bytes.Buffer{} buf := &bytes.Buffer{}
@ -429,7 +222,7 @@ func TestErasureReadFileRandomOffsetLength(t *testing.T) {
expected := data[offset : offset+readLen] expected := data[offset : offset+readLen]
_, err = erasureReadFile(buf, disks, "testbucket", "testobject", offset, readLen, length, blockSize, dataBlocks, parityBlocks, checkSums, bitRotAlgo, pool) _, err = storage.ReadFile(buf, "testbucket", "testobject", offset, readLen, length, file.Checksums, DefaultBitrotAlgorithm, blockSize, pool)
if err != nil { if err != nil {
t.Fatal(err, offset, readLen) t.Fatal(err, offset, readLen)
} }

View File

@ -18,72 +18,11 @@ package cmd
import ( import (
"bytes" "bytes"
"errors"
"hash"
"io" "io"
"sync"
"github.com/klauspost/reedsolomon" "github.com/klauspost/reedsolomon"
"github.com/minio/sha256-simd"
"golang.org/x/crypto/blake2b"
) )
// newHashWriters - inititialize a slice of hashes for the disk count.
func newHashWriters(diskCount int, algo HashAlgo) []hash.Hash {
hashWriters := make([]hash.Hash, diskCount)
for index := range hashWriters {
hashWriters[index] = newHash(algo)
}
return hashWriters
}
// newHash - gives you a newly allocated hash depending on the input algorithm.
func newHash(algo HashAlgo) (h hash.Hash) {
switch algo {
case HashSha256:
// sha256 checksum specially on ARM64 platforms or whenever
// requested as dictated by `xl.json` entry.
h = sha256.New()
case HashBlake2b:
// ignore the error, because New512 without a key never fails
// New512 only returns a non-nil error, if the length of the passed
// key > 64 bytes - but we use blake2b as hash function (no key)
h, _ = blake2b.New512(nil)
// Add new hashes here.
default:
// Default to blake2b.
// ignore the error, because New512 without a key never fails
// New512 only returns a non-nil error, if the length of the passed
// key > 64 bytes - but we use blake2b as hash function (no key)
h, _ = blake2b.New512(nil)
}
return h
}
// Hash buffer pool is a pool of reusable
// buffers used while checksumming a stream.
var hashBufferPool = sync.Pool{
New: func() interface{} {
b := make([]byte, readSizeV1)
return &b
},
}
// hashSum calculates the hash of the entire path and returns.
func hashSum(disk StorageAPI, volume, path string, writer hash.Hash) ([]byte, error) {
// Fetch a new staging buffer from the pool.
bufp := hashBufferPool.Get().(*[]byte)
defer hashBufferPool.Put(bufp)
// Copy entire buffer to writer.
if err := copyBuffer(writer, disk, volume, path, *bufp); err != nil {
return nil, err
}
// Return the final hash sum.
return writer.Sum(nil), nil
}
// getDataBlockLen - get length of data blocks from encoded blocks. // getDataBlockLen - get length of data blocks from encoded blocks.
func getDataBlockLen(enBlocks [][]byte, dataBlocks int) int { func getDataBlockLen(enBlocks [][]byte, dataBlocks int) int {
size := 0 size := 0
@ -166,57 +105,3 @@ func writeDataBlocks(dst io.Writer, enBlocks [][]byte, dataBlocks int, offset in
func getChunkSize(blockSize int64, dataBlocks int) int64 { func getChunkSize(blockSize int64, dataBlocks int) int64 {
return (blockSize + int64(dataBlocks) - 1) / int64(dataBlocks) return (blockSize + int64(dataBlocks) - 1) / int64(dataBlocks)
} }
// copyBuffer - copies from disk, volume, path to input writer until either EOF
// is reached at volume, path or an error occurs. A success copyBuffer returns
// err == nil, not err == EOF. Because copyBuffer is defined to read from path
// until EOF. It does not treat an EOF from ReadFile an error to be reported.
// Additionally copyBuffer stages through the provided buffer; otherwise if it
// has zero length, returns error.
func copyBuffer(writer io.Writer, disk StorageAPI, volume string, path string, buf []byte) error {
// Error condition of zero length buffer.
if buf != nil && len(buf) == 0 {
return errors.New("empty buffer in readBuffer")
}
// Starting offset for Reading the file.
var startOffset int64
// Read until io.EOF.
for {
n, err := disk.ReadFile(volume, path, startOffset, buf)
if n > 0 {
m, wErr := writer.Write(buf[:n])
if wErr != nil {
return wErr
}
if int64(m) != n {
return io.ErrShortWrite
}
}
if err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
break
}
return err
}
// Progress the offset.
startOffset += n
}
// Success.
return nil
}
// bitRotVerifier - type representing bit-rot verification process for
// a single under-lying object (currently whole files)
type bitRotVerifier struct {
// has the bit-rot verification been done?
isVerified bool
// is the data free of bit-rot?
hasBitRot bool
// hashing algorithm
algo HashAlgo
// hex-encoded expected raw-hash value
checkSum string
}

View File

@ -16,23 +16,7 @@
package cmd package cmd
import ( import "testing"
"bytes"
"errors"
"io"
"os"
"testing"
)
// Test validates the number hash writers returned.
func TestNewHashWriters(t *testing.T) {
diskNum := 8
hashWriters := newHashWriters(diskNum, bitRotAlgo)
if len(hashWriters) != diskNum {
t.Errorf("Expected %d hashWriters, but instead got %d", diskNum, len(hashWriters))
}
}
// Tests validate the output of getChunkSize. // Tests validate the output of getChunkSize.
func TestGetChunkSize(t *testing.T) { func TestGetChunkSize(t *testing.T) {
@ -67,95 +51,3 @@ func TestGetChunkSize(t *testing.T) {
} }
} }
} }
// TestCopyBuffer - Tests validate the result and errors produced when `copyBuffer` is called with sample inputs.
func TestCopyBuffer(t *testing.T) {
// create posix test setup
disk, diskPath, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer os.RemoveAll(diskPath)
volume := "success-vol"
// Setup test environment.
if err = disk.MakeVol(volume); err != nil {
t.Fatalf("Unable to create volume, %s", err)
}
// creating io.Writer for the input to copyBuffer.
buffers := []*bytes.Buffer{
new(bytes.Buffer),
new(bytes.Buffer),
new(bytes.Buffer),
}
testFile := "testFile"
testContent := []byte("hello, world")
err = disk.AppendFile(volume, testFile, testContent)
if err != nil {
t.Fatalf("AppendFile failed: <ERROR> %s", err)
}
testCases := []struct {
writer io.Writer
disk StorageAPI
volume string
path string
buf []byte
expectedResult []byte
// flag to indicate whether test case should pass.
shouldPass bool
expectedErr error
}{
// Test case - 1.
// case with empty buffer.
{nil, nil, "", "", []byte{}, nil, false, errors.New("empty buffer in readBuffer")},
// Test case - 2.
// Test case with empty volume.
{buffers[0], disk, "", "", make([]byte, 5), nil, false, errInvalidArgument},
// Test case - 3.
// Test case with non existent volume.
{buffers[0], disk, "abc", "", make([]byte, 5), nil, false, errVolumeNotFound},
// Test case - 4.
// Test case with empty filename/path.
{buffers[0], disk, volume, "", make([]byte, 5), nil, false, errIsNotRegular},
// Test case - 5.
// Test case with non existent file name.
{buffers[0], disk, volume, "abcd", make([]byte, 5), nil, false, errFileNotFound},
// Test case - 6.
// Test case where the writer returns EOF.
{NewEOFWriter(buffers[0], 3), disk, volume, testFile, make([]byte, 5), nil, false, io.EOF},
// Test case - 7.
// Test case to produce io.ErrShortWrite, the TruncateWriter returns after writing 3 bytes.
{TruncateWriter(buffers[1], 3), disk, volume, testFile, make([]byte, 5), nil, false, io.ErrShortWrite},
// Teset case - 8.
// Valid case, expected to read till EOF and write the contents into the writer.
{buffers[2], disk, volume, testFile, make([]byte, 5), testContent, true, nil},
}
// iterate over the test cases and call copy Buffer with data.
for i, testCase := range testCases {
actualErr := copyBuffer(testCase.writer, testCase.disk, testCase.volume, testCase.path, testCase.buf)
if actualErr != nil && testCase.shouldPass {
t.Errorf("Test %d: Expected to pass but failed instead with \"%s\"", i+1, actualErr)
}
if actualErr == nil && !testCase.shouldPass {
t.Errorf("Test %d: Expected to fail with error <Error> \"%v\", but instead passed", i+1, testCase.expectedErr)
}
// Failed as expected, but does it fail for the expected reason.
if actualErr != nil && !testCase.shouldPass {
if testCase.expectedErr.Error() != actualErr.Error() {
t.Errorf("Test %d: Expected Error to be \"%v\", but instead found \"%v\" ", i+1, testCase.expectedErr, actualErr)
}
}
// test passed as expected, asserting the result.
if actualErr == nil && testCase.shouldPass {
if !bytes.Equal(testCase.expectedResult, buffers[2].Bytes()) {
t.Errorf("Test %d: copied buffer differs from the expected one.", i+1)
}
}
}
}

115
cmd/erasure.go Normal file
View File

@ -0,0 +1,115 @@
/*
* Minio Cloud Storage, (C) 2017 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 cmd
import (
"crypto/subtle"
"hash"
"github.com/klauspost/reedsolomon"
)
// OfflineDisk represents an unavailable disk.
var OfflineDisk StorageAPI // zero value is nil
// ErasureFileInfo contains information about an erasure file operation (create, read, heal).
type ErasureFileInfo struct {
Size int64
Algorithm BitrotAlgorithm
Checksums [][]byte
}
// ErasureStorage represents an array of disks.
// The disks contain erasure coded and bitrot-protected data.
type ErasureStorage struct {
disks []StorageAPI
erasure reedsolomon.Encoder
dataBlocks, parityBlocks int
}
// NewErasureStorage creates a new ErasureStorage. The storage erasure codes and protects all data written to
// the disks.
func NewErasureStorage(disks []StorageAPI, dataBlocks, parityBlocks int) (s ErasureStorage, err error) {
erasure, err := reedsolomon.New(dataBlocks, parityBlocks)
if err != nil {
return s, traceErrorf("failed to create erasure coding: %v", err)
}
s = ErasureStorage{
disks: make([]StorageAPI, len(disks)),
erasure: erasure,
dataBlocks: dataBlocks,
parityBlocks: parityBlocks,
}
copy(s.disks, disks)
return
}
// ErasureEncode encodes the given data and returns the erasure-coded data.
// It returns an error if the erasure coding failed.
func (s *ErasureStorage) ErasureEncode(data []byte) ([][]byte, error) {
encoded, err := s.erasure.Split(data)
if err != nil {
return nil, traceErrorf("failed to split data: %v", err)
}
if err = s.erasure.Encode(encoded); err != nil {
return nil, traceErrorf("failed to encode data: %v", err)
}
return encoded, nil
}
// ErasureDecodeDataBlocks decodes the given erasure-coded data.
// It only decodes the data blocks but does not verify them.
// It returns an error if the decoding failed.
func (s *ErasureStorage) ErasureDecodeDataBlocks(data [][]byte) error {
if err := s.erasure.ReconstructData(data); err != nil {
return traceErrorf("failed to reconstruct data: %v", err)
}
return nil
}
// ErasureDecodeDataAndParityBlocks decodes the given erasure-coded data and verifies it.
// It returns an error if the decoding failed.
func (s *ErasureStorage) ErasureDecodeDataAndParityBlocks(data [][]byte) error {
if err := s.erasure.Reconstruct(data); err != nil {
return traceErrorf("failed to reconstruct data: %v", err)
}
return nil
}
// NewBitrotVerifier returns a new BitrotVerifier implementing the given algorithm.
func NewBitrotVerifier(algorithm BitrotAlgorithm, checksum []byte) *BitrotVerifier {
return &BitrotVerifier{algorithm.New(), algorithm, checksum, false}
}
// BitrotVerifier can be used to verify protected data.
type BitrotVerifier struct {
hash.Hash
algorithm BitrotAlgorithm
sum []byte
verified bool
}
// Verify returns true iff the computed checksum of the verifier matches the the checksum provided when the verifier
// was created.
func (v *BitrotVerifier) Verify() bool {
v.verified = true
return subtle.ConstantTimeCompare(v.Sum(nil), v.sum) == 1
}
// IsVerified returns true iff Verify was called at least once.
func (v *BitrotVerifier) IsVerified() bool { return v.verified }

View File

@ -18,161 +18,91 @@ package cmd
import ( import (
"bytes" "bytes"
"crypto/rand"
"io"
"os" "os"
"testing" "testing"
) )
// mustEncodeData - encodes data slice and provides encoded 2 dimensional slice. var erasureDecodeTests = []struct {
func mustEncodeData(data []byte, dataBlocks, parityBlocks int) [][]byte { dataBlocks, parityBlocks int
encodedData, err := encodeData(data, dataBlocks, parityBlocks) missingData, missingParity int
if err != nil { reconstructParity bool
// Upon failure panic this function. shouldFail bool
panic(err)
}
return encodedData
}
// Generates good encoded data with one parity block and data block missing.
func getGoodEncodedData(data []byte, dataBlocks, parityBlocks int) [][]byte {
encodedData := mustEncodeData(data, dataBlocks, parityBlocks)
encodedData[3] = nil
encodedData[1] = nil
return encodedData
}
// Generates bad encoded data with one parity block and data block with garbage data.
func getBadEncodedData(data []byte, dataBlocks, parityBlocks int) [][]byte {
encodedData := mustEncodeData(data, dataBlocks, parityBlocks)
encodedData[3] = []byte("garbage")
encodedData[1] = []byte("garbage")
return encodedData
}
// Generates encoded data with all data blocks missing.
func getMissingData(data []byte, dataBlocks, parityBlocks int) [][]byte {
encodedData := mustEncodeData(data, dataBlocks, parityBlocks)
for i := 0; i < dataBlocks+1; i++ {
encodedData[i] = nil
}
return encodedData
}
// Generates encoded data with less number of blocks than expected data blocks.
func getInsufficientData(data []byte, dataBlocks, parityBlocks int) [][]byte {
encodedData := mustEncodeData(data, dataBlocks, parityBlocks)
// Return half of the data.
return encodedData[:dataBlocks/2]
}
// Represents erasure encoding matrix dataBlocks and paritBlocks.
type encodingMatrix struct {
dataBlocks int
parityBlocks int
}
// List of encoding matrices the tests will run on.
var encodingMatrices = []encodingMatrix{
{3, 3}, // 3 data, 3 parity blocks.
{4, 4}, // 4 data, 4 parity blocks.
{5, 5}, // 5 data, 5 parity blocks.
{6, 6}, // 6 data, 6 parity blocks.
{7, 7}, // 7 data, 7 parity blocks.
{8, 8}, // 8 data, 8 parity blocks.
}
// Tests erasure decoding functionality for various types of inputs.
func TestErasureDecode(t *testing.T) {
data := []byte("Lorem Ipsum is simply dummy text of the printing and typesetting industry. Lorem Ipsum has been the industry's standard dummy text ever since the 1500s, when an unknown printer took a galley of type and scrambled it to make a type specimen book. It has survived not only five centuries, but also the leap into electronic typesetting, remaining essentially unchanged. It was popularised in the 1960s with the release of Letraset sheets containing Lorem Ipsum passages, and more recently with desktop publishing software like Aldus PageMaker including versions of Lorem Ipsum.")
// List of decoding cases
// - validates bad encoded data.
// - validates good encoded data.
// - validates insufficient data.
testDecodeCases := []struct {
enFn func([]byte, int, int) [][]byte
shouldPass bool
}{ }{
// Generates bad encoded data. {dataBlocks: 2, parityBlocks: 2, missingData: 0, missingParity: 0, reconstructParity: true, shouldFail: false},
{ {dataBlocks: 3, parityBlocks: 3, missingData: 1, missingParity: 0, reconstructParity: true, shouldFail: false},
enFn: getBadEncodedData, {dataBlocks: 4, parityBlocks: 4, missingData: 2, missingParity: 0, reconstructParity: false, shouldFail: false},
shouldPass: false, {dataBlocks: 5, parityBlocks: 5, missingData: 0, missingParity: 1, reconstructParity: true, shouldFail: false},
}, {dataBlocks: 6, parityBlocks: 6, missingData: 0, missingParity: 2, reconstructParity: true, shouldFail: false},
// Generates good encoded data. {dataBlocks: 7, parityBlocks: 7, missingData: 1, missingParity: 1, reconstructParity: false, shouldFail: false},
{ {dataBlocks: 8, parityBlocks: 8, missingData: 3, missingParity: 2, reconstructParity: false, shouldFail: false},
enFn: getGoodEncodedData, {dataBlocks: 2, parityBlocks: 2, missingData: 2, missingParity: 1, reconstructParity: true, shouldFail: true},
shouldPass: true, {dataBlocks: 4, parityBlocks: 2, missingData: 2, missingParity: 2, reconstructParity: false, shouldFail: true},
}, {dataBlocks: 8, parityBlocks: 4, missingData: 2, missingParity: 2, reconstructParity: false, shouldFail: false},
// Generates missing data.
{
enFn: getMissingData,
shouldPass: false,
},
// Generates short data.
{
enFn: getInsufficientData,
shouldPass: false,
},
} }
// Validates all decode tests. func TestErasureDecode(t *testing.T) {
for i, testCase := range testDecodeCases { data := make([]byte, 256)
for _, encodingMatrix := range encodingMatrices { if _, err := io.ReadFull(rand.Reader, data); err != nil {
t.Fatalf("Failed to read random data: %v", err)
}
for i, test := range erasureDecodeTests {
buffer := make([]byte, len(data), 2*len(data))
copy(buffer, data)
// Encoding matrix. disks := make([]StorageAPI, test.dataBlocks+test.parityBlocks)
dataBlocks := encodingMatrix.dataBlocks storage, err := NewErasureStorage(disks, test.dataBlocks, test.parityBlocks)
parityBlocks := encodingMatrix.parityBlocks if err != nil {
t.Fatalf("Test %d: failed to create erasure storage: %v", i, err)
// Data block size. }
blockSize := len(data) encoded, err := storage.ErasureEncode(buffer)
if err != nil {
// Test decoder for just the missing data blocks t.Fatalf("Test %d: failed to encode data: %v", i, err)
{
// Generates encoded data based on type of testCase function.
encodedData := testCase.enFn(data, dataBlocks, parityBlocks)
// Decodes the data.
err := decodeMissingData(encodedData, dataBlocks, parityBlocks)
if err != nil && testCase.shouldPass {
t.Errorf("Test %d: Expected to pass by failed instead with %s", i+1, err)
} }
// Proceed to extract the data blocks. for j := range encoded[:test.missingData] {
decodedDataWriter := new(bytes.Buffer) encoded[j] = nil
_, err = writeDataBlocks(decodedDataWriter, encodedData, dataBlocks, 0, int64(blockSize)) }
if err != nil && testCase.shouldPass { for j := test.dataBlocks; j < test.dataBlocks+test.missingParity; j++ {
t.Errorf("Test %d: Expected to pass by failed instead with %s", i+1, err) encoded[j] = nil
} }
// Validate if decoded data is what we expected. if test.reconstructParity {
if bytes.Equal(decodedDataWriter.Bytes(), data) != testCase.shouldPass { err = storage.ErasureDecodeDataAndParityBlocks(encoded)
err := errUnexpected } else {
t.Errorf("Test %d: Expected to pass by failed instead %s", i+1, err) err = storage.ErasureDecodeDataBlocks(encoded)
}
if err == nil && test.shouldFail {
t.Errorf("Test %d: test should fail but it passed", i)
}
if err != nil && !test.shouldFail {
t.Errorf("Test %d: test should pass but it failed: %v", i, err)
}
decoded := encoded
if !test.shouldFail {
if test.reconstructParity {
for j := range decoded {
if decoded[j] == nil {
t.Errorf("Test %d: failed to reconstruct shard %d", i, j)
}
}
} else {
for j := range decoded[:test.dataBlocks] {
if decoded[j] == nil {
t.Errorf("Test %d: failed to reconstruct data shard %d", i, j)
}
} }
} }
// Test decoder for all missing data and parity blocks decodedData := new(bytes.Buffer)
{ if _, err = writeDataBlocks(decodedData, decoded, test.dataBlocks, 0, int64(len(data))); err != nil {
// Generates encoded data based on type of testCase function. t.Errorf("Test %d: failed to write data blocks: %v", i, err)
encodedData := testCase.enFn(data, dataBlocks, parityBlocks)
// Decodes the data.
err := decodeDataAndParity(encodedData, dataBlocks, parityBlocks)
if err != nil && testCase.shouldPass {
t.Errorf("Test %d: Expected to pass by failed instead with %s", i+1, err)
}
// Proceed to extract the data blocks.
decodedDataWriter := new(bytes.Buffer)
_, err = writeDataBlocks(decodedDataWriter, encodedData, dataBlocks, 0, int64(blockSize))
if err != nil && testCase.shouldPass {
t.Errorf("Test %d: Expected to pass by failed instead with %s", i+1, err)
}
// Validate if decoded data is what we expected.
if bytes.Equal(decodedDataWriter.Bytes(), data) != testCase.shouldPass {
err := errUnexpected
t.Errorf("Test %d: Expected to pass by failed instead %s", i+1, err)
} }
if !bytes.Equal(decodedData.Bytes(), data) {
t.Errorf("Test %d: Decoded data does not match original data: got: %v want: %v", i, decodedData.Bytes(), data)
} }
} }
} }

View File

@ -151,3 +151,8 @@ func isErr(err error, errs ...error) bool {
} }
return false return false
} }
// traceErrorf behaves like fmt.traceErrorf but also traces the returned error.
func traceErrorf(format string, args ...interface{}) error {
return traceError(fmt.Errorf(format, args...))
}

View File

@ -122,13 +122,11 @@ func (d *naughtyDisk) ReadFile(volume string, path string, offset int64, buf []b
return d.disk.ReadFile(volume, path, offset, buf) return d.disk.ReadFile(volume, path, offset, buf)
} }
func (d *naughtyDisk) ReadFileWithVerify(volume, path string, offset int64, func (d *naughtyDisk) ReadFileWithVerify(volume, path string, offset int64, buf []byte, verifier *BitrotVerifier) (n int64, err error) {
buf []byte, algo HashAlgo, expectedHash string) (n int64, err error) {
if err := d.calcError(); err != nil { if err := d.calcError(); err != nil {
return 0, err return 0, err
} }
return d.disk.ReadFileWithVerify(volume, path, offset, buf, algo, expectedHash) return d.disk.ReadFileWithVerify(volume, path, offset, buf, verifier)
} }
func (d *naughtyDisk) PrepareFile(volume, path string, length int64) error { func (d *naughtyDisk) PrepareFile(volume, path string, length int64) error {

View File

@ -17,9 +17,7 @@
package cmd package cmd
import ( import (
"bytes"
"encoding/hex" "encoding/hex"
"hash"
"io" "io"
"io/ioutil" "io/ioutil"
"os" "os"
@ -539,8 +537,7 @@ func (s *posix) ReadAll(volume, path string) (buf []byte, err error) {
// Additionally ReadFile also starts reading from an offset. ReadFile // Additionally ReadFile also starts reading from an offset. ReadFile
// semantics are same as io.ReadFull. // semantics are same as io.ReadFull.
func (s *posix) ReadFile(volume, path string, offset int64, buf []byte) (n int64, err error) { func (s *posix) ReadFile(volume, path string, offset int64, buf []byte) (n int64, err error) {
return s.ReadFileWithVerify(volume, path, offset, buf, &BitrotVerifier{verified: true})
return s.ReadFileWithVerify(volume, path, offset, buf, "", "")
} }
// ReadFileWithVerify is the same as ReadFile but with hashsum // ReadFileWithVerify is the same as ReadFile but with hashsum
@ -553,9 +550,7 @@ func (s *posix) ReadFile(volume, path string, offset int64, buf []byte) (n int64
// //
// The function takes care to minimize the number of disk read // The function takes care to minimize the number of disk read
// operations. // operations.
func (s *posix) ReadFileWithVerify(volume, path string, offset int64, buf []byte, func (s *posix) ReadFileWithVerify(volume, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (n int64, err error) {
algo HashAlgo, expectedHash string) (n int64, err error) {
defer func() { defer func() {
if err == syscall.EIO { if err == syscall.EIO {
atomic.AddInt32(&s.ioErrCount, 1) atomic.AddInt32(&s.ioErrCount, 1)
@ -616,62 +611,34 @@ func (s *posix) ReadFileWithVerify(volume, path string, offset int64, buf []byte
return 0, errIsNotRegular return 0, errIsNotRegular
} }
// If expected hash string is empty hash verification is if !verifier.IsVerified() {
// skipped. bufp := s.pool.Get().(*[]byte)
needToHash := expectedHash != "" defer s.pool.Put(bufp)
var hasher hash.Hash
if needToHash { if offset != 0 {
// If the hashing algo is invalid, return an error. if _, err = io.CopyBuffer(verifier, io.LimitReader(file, offset), *bufp); err != nil {
if !isValidHashAlgo(algo) {
return 0, errBitrotHashAlgoInvalid
}
// Compute hash of object from start to the byte at
// (offset - 1), and as a result of this read, seek to
// `offset`.
hasher = newHash(algo)
if offset > 0 {
_, err = io.CopyN(hasher, file, offset)
if err != nil {
return 0, err return 0, err
} }
} }
} else { if _, err = file.Read(buffer); err != nil {
// Seek to requested offset.
_, err = file.Seek(offset, os.SEEK_SET)
if err != nil {
return 0, err return 0, err
} }
} if _, err = verifier.Write(buffer); err != nil {
// Read until buffer is full.
m, err := io.ReadFull(file, buf)
if err == io.EOF {
return 0, err return 0, err
} }
if _, err = io.CopyBuffer(verifier, file, *bufp); err != nil {
if needToHash {
// Continue computing hash with buf.
_, err = hasher.Write(buf)
if err != nil {
return 0, err return 0, err
} }
if !verifier.Verify() {
// Continue computing hash until end of file. return 0, hashMismatchError{hex.EncodeToString(verifier.sum), hex.EncodeToString(verifier.Sum(nil))}
_, err = io.Copy(hasher, file) }
if err != nil { return int64(len(buffer)), err
return 0, err
} }
// Verify the computed hash. m, err := file.ReadAt(buffer, offset)
computedHash := hex.EncodeToString(hasher.Sum(nil)) if m > 0 && m < len(buffer) {
if computedHash != expectedHash { err = io.ErrUnexpectedEOF
return 0, hashMismatchError{expectedHash, computedHash}
} }
}
// Success.
return int64(m), err return int64(m), err
} }
@ -811,17 +778,8 @@ func (s *posix) AppendFile(volume, path string, buf []byte) (err error) {
if err != nil { if err != nil {
return err return err
} }
_, err = w.Write(buf)
// Close upon return. w.Close()
defer w.Close()
bufp := s.pool.Get().(*[]byte)
// Reuse buffer.
defer s.pool.Put(bufp)
// Return io.Copy
_, err = io.CopyBuffer(w, bytes.NewReader(buf), *bufp)
return err return err
} }

View File

@ -18,7 +18,7 @@ package cmd
import ( import (
"bytes" "bytes"
"crypto/sha256" "crypto/rand"
"encoding/hex" "encoding/hex"
"io" "io"
"io/ioutil" "io/ioutil"
@ -30,8 +30,6 @@ import (
"testing" "testing"
"github.com/minio/minio/pkg/disk" "github.com/minio/minio/pkg/disk"
"golang.org/x/crypto/blake2b"
) )
// creates a temp dir and sets up posix layer. // creates a temp dir and sets up posix layer.
@ -945,14 +943,14 @@ func TestPosixReadFile(t *testing.T) {
func() error { func() error {
if runtime.GOOS == globalWindowsOSName { if runtime.GOOS == globalWindowsOSName {
return &os.PathError{ return &os.PathError{
Op: "seek", Op: "read",
Path: (slashpath.Join(path, "success-vol", "myobject")), Path: slashpath.Join(path, "success-vol", "myobject"),
Err: syscall.Errno(0x83), // ERROR_NEGATIVE_SEEK Err: syscall.Errno(0x57), // invalid parameter
} }
} }
return &os.PathError{ return &os.PathError{
Op: "seek", Op: "read",
Path: (slashpath.Join(path, "success-vol", "myobject")), Path: slashpath.Join(path, "success-vol", "myobject"),
Err: os.ErrInvalid, Err: os.ErrInvalid,
} }
}(), }(),
@ -1079,111 +1077,77 @@ func TestPosixReadFile(t *testing.T) {
} }
} }
var posixReadFileWithVerifyTests = []struct {
file string
offset int
length int
algorithm BitrotAlgorithm
expError error
}{
{file: "myobject", offset: 0, length: 100, algorithm: SHA256, expError: nil}, // 0
{file: "myobject", offset: 25, length: 74, algorithm: SHA256, expError: nil}, // 1
{file: "myobject", offset: 29, length: 70, algorithm: SHA256, expError: nil}, // 2
{file: "myobject", offset: 100, length: 0, algorithm: SHA256, expError: nil}, // 3
{file: "myobject", offset: 1, length: 120, algorithm: SHA256, expError: hashMismatchError{}}, // 4
{file: "myobject", offset: 3, length: 1100, algorithm: SHA256, expError: nil}, // 5
{file: "myobject", offset: 2, length: 100, algorithm: SHA256, expError: hashMismatchError{}}, // 6
{file: "myobject", offset: 1000, length: 1001, algorithm: SHA256, expError: nil}, // 7
{file: "myobject", offset: 0, length: 100, algorithm: BLAKE2b512, expError: hashMismatchError{}}, // 8
{file: "myobject", offset: 25, length: 74, algorithm: BLAKE2b512, expError: nil}, // 9
{file: "myobject", offset: 29, length: 70, algorithm: BLAKE2b512, expError: hashMismatchError{}}, // 10
{file: "myobject", offset: 100, length: 0, algorithm: BLAKE2b512, expError: nil}, // 11
{file: "myobject", offset: 1, length: 120, algorithm: BLAKE2b512, expError: nil}, // 12
{file: "myobject", offset: 3, length: 1100, algorithm: BLAKE2b512, expError: nil}, // 13
{file: "myobject", offset: 2, length: 100, algorithm: BLAKE2b512, expError: nil}, // 14
{file: "myobject", offset: 1000, length: 1001, algorithm: BLAKE2b512, expError: nil}, // 15
}
// TestPosixReadFileWithVerify - tests the posix level // TestPosixReadFileWithVerify - tests the posix level
// ReadFileWithVerify API. Only tests hashing related // ReadFileWithVerify API. Only tests hashing related
// functionality. Other functionality is tested with // functionality. Other functionality is tested with
// TestPosixReadFile. // TestPosixReadFile.
func TestPosixReadFileWithVerify(t *testing.T) { func TestPosixReadFileWithVerify(t *testing.T) {
// create posix test setup volume, object := "test-vol", "myobject"
posixStorage, path, err := newPosixTestSetup() posixStorage, path, err := newPosixTestSetup()
if err != nil { if err != nil {
os.RemoveAll(path)
t.Fatalf("Unable to create posix test setup, %s", err) t.Fatalf("Unable to create posix test setup, %s", err)
} }
defer os.RemoveAll(path)
volume := "success-vol"
// Setup test environment.
if err = posixStorage.MakeVol(volume); err != nil { if err = posixStorage.MakeVol(volume); err != nil {
t.Fatalf("Unable to create volume, %s", err) os.RemoveAll(path)
t.Fatalf("Unable to create volume %s: %v", volume, err)
}
data := make([]byte, 8*1024)
if _, err = io.ReadFull(rand.Reader, data); err != nil {
os.RemoveAll(path)
t.Fatalf("Unable to create generate random data: %v", err)
}
if err = posixStorage.AppendFile(volume, object, data); err != nil {
os.RemoveAll(path)
t.Fatalf("Unable to create object: %v", err)
} }
blakeHash := func(s string) string { for i, test := range posixReadFileWithVerifyTests {
k := blake2b.Sum512([]byte(s)) h := test.algorithm.New()
return hex.EncodeToString(k[:]) h.Write(data)
if test.expError != nil {
expected := h.Sum(nil)
h.Write([]byte{0})
test.expError = hashMismatchError{hex.EncodeToString(h.Sum(nil)), hex.EncodeToString(expected)}
} }
sha256Hash := func(s string) string { buffer := make([]byte, test.length)
k := sha256.Sum256([]byte(s)) n, err := posixStorage.ReadFileWithVerify(volume, test.file, int64(test.offset), buffer, NewBitrotVerifier(test.algorithm, h.Sum(nil)))
return hex.EncodeToString(k[:])
}
testCases := []struct {
fileName string
offset int64
bufSize int
algo HashAlgo
expectedHash string
expectedBuf []byte
expectedErr error
}{
// Hash verification is skipped with empty expected
// hash - 1
{
"myobject", 0, 5, HashBlake2b, "",
[]byte("Hello"), nil,
},
// Hash verification failure case - 2
{
"myobject", 0, 5, HashBlake2b, "a",
[]byte(""),
hashMismatchError{"a", blakeHash("Hello, world!")},
},
// Hash verification success with full content requested - 3
{
"myobject", 0, 13, HashBlake2b, blakeHash("Hello, world!"),
[]byte("Hello, world!"), nil,
},
// Hash verification success with full content and Sha256 - 4
{
"myobject", 0, 13, HashSha256, sha256Hash("Hello, world!"),
[]byte("Hello, world!"), nil,
},
// Hash verification success with partial content requested - 5
{
"myobject", 7, 4, HashBlake2b, blakeHash("Hello, world!"),
[]byte("worl"), nil,
},
// Hash verification success with partial content and Sha256 - 6
{
"myobject", 7, 4, HashSha256, sha256Hash("Hello, world!"),
[]byte("worl"), nil,
},
// Empty hash-algo returns error - 7
{
"myobject", 7, 4, "", blakeHash("Hello, world!"),
[]byte("worl"), errBitrotHashAlgoInvalid,
},
// Empty content hash verification with empty
// hash-algo algo returns error - 8
{
"myobject", 7, 0, "", blakeHash("Hello, world!"),
[]byte(""), errBitrotHashAlgoInvalid,
},
}
// Create file used in testcases
err = posixStorage.AppendFile(volume, "myobject", []byte("Hello, world!"))
if err != nil {
t.Fatalf("Failure in test setup: %v\n", err)
}
// Validate each test case.
for i, testCase := range testCases {
var n int64
// Common read buffer.
var buf = make([]byte, testCase.bufSize)
n, err = posixStorage.ReadFileWithVerify(volume, testCase.fileName, testCase.offset, buf, testCase.algo, testCase.expectedHash)
switch { switch {
case err == nil && testCase.expectedErr != nil: case err == nil && test.expError != nil:
t.Errorf("Test %d: Expected error %v but got none.", i+1, testCase.expectedErr) t.Errorf("Test %d: Expected error %v but got none.", i, test.expError)
case err == nil && n != int64(testCase.bufSize): case err == nil && n != int64(test.length):
t.Errorf("Test %d: %d bytes were expected, but %d were written", i+1, testCase.bufSize, n) t.Errorf("Test %d: %d bytes were expected, but %d were written", i, test.length, n)
case err == nil && !bytes.Equal(testCase.expectedBuf, buf): case err == nil && !bytes.Equal(data[test.offset:test.offset+test.length], buffer):
t.Errorf("Test %d: Expected bytes: %v, but got: %v", i+1, testCase.expectedBuf, buf) t.Errorf("Test %d: Expected bytes: %v, but got: %v", i, data[test.offset:test.offset+test.length], buffer)
case err != nil && err != testCase.expectedErr: case err != nil && err != test.expError:
t.Errorf("Test %d: Expected error: %v, but got: %v", i+1, testCase.expectedErr, err) t.Errorf("Test %d: Expected error: %v, but got: %v", i, test.expError, err)
} }
} }
} }

View File

@ -221,18 +221,11 @@ func (f *retryStorage) ReadFile(volume, path string, offset int64, buffer []byte
// ReadFileWithVerify - a retryable implementation of reading at // ReadFileWithVerify - a retryable implementation of reading at
// offset from a file with verification. // offset from a file with verification.
func (f *retryStorage) ReadFileWithVerify(volume, path string, offset int64, buffer []byte, func (f retryStorage) ReadFileWithVerify(volume, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (m int64, err error) {
algo HashAlgo, expectedHash string) (m int64, err error) {
if f.IsOffline() {
return m, errDiskNotFound
}
m, err = f.remoteStorage.ReadFileWithVerify(volume, path, offset, buffer, m, err = f.remoteStorage.ReadFileWithVerify(volume, path, offset, buffer, verifier)
algo, expectedHash)
if f.reInitUponDiskNotFound(err) { if f.reInitUponDiskNotFound(err) {
m, err = f.remoteStorage.ReadFileWithVerify(volume, path, m, err = f.remoteStorage.ReadFileWithVerify(volume, path, offset, buffer, verifier)
offset, buffer, algo, expectedHash)
return m, retryToStorageErr(err)
} }
return m, retryToStorageErr(err) return m, retryToStorageErr(err)
} }

View File

@ -19,7 +19,6 @@ package cmd
import ( import (
"bytes" "bytes"
"crypto/sha256" "crypto/sha256"
"encoding/hex"
"errors" "errors"
"os" "os"
"reflect" "reflect"
@ -305,15 +304,15 @@ func TestRetryStorage(t *testing.T) {
} }
} }
sha256Hash := func(s string) string { sha256Hash := func(b []byte) []byte {
k := sha256.Sum256([]byte(s)) k := sha256.Sum256(b)
return hex.EncodeToString(k[:]) return k[:]
} }
for _, disk := range storageDisks { for _, disk := range storageDisks {
var buf2 = make([]byte, 5) var buf2 = make([]byte, 5)
verifier := NewBitrotVerifier(SHA256, sha256Hash([]byte("Hello, World")))
var n int64 var n int64
if n, err = disk.ReadFileWithVerify("existent", "path", 7, buf2, if n, err = disk.ReadFileWithVerify("existent", "path", 7, buf2, verifier); err != nil {
HashSha256, sha256Hash("Hello, World")); err != nil {
t.Fatal(err) t.Fatal(err)
} }
if err != nil { if err != nil {

View File

@ -16,7 +16,11 @@
package cmd package cmd
import "github.com/minio/minio/pkg/disk" import (
"io"
"github.com/minio/minio/pkg/disk"
)
// StorageAPI interface. // StorageAPI interface.
type StorageAPI interface { type StorageAPI interface {
@ -37,8 +41,7 @@ type StorageAPI interface {
// File operations. // File operations.
ListDir(volume, dirPath string) ([]string, error) ListDir(volume, dirPath string) ([]string, error)
ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error) ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error)
ReadFileWithVerify(volume string, path string, offset int64, buf []byte, ReadFileWithVerify(volume string, path string, offset int64, buf []byte, verifier *BitrotVerifier) (n int64, err error)
algo HashAlgo, expectedHash string) (n int64, err error)
PrepareFile(volume string, path string, len int64) (err error) PrepareFile(volume string, path string, len int64) (err error)
AppendFile(volume string, path string, buf []byte) (err error) AppendFile(volume string, path string, buf []byte) (err error)
RenameFile(srcVolume, srcPath, dstVolume, dstPath string) error RenameFile(srcVolume, srcPath, dstVolume, dstPath string) error
@ -48,3 +51,47 @@ type StorageAPI interface {
// Read all. // Read all.
ReadAll(volume string, path string) (buf []byte, err error) ReadAll(volume string, path string) (buf []byte, err error)
} }
// storageReader is an io.Reader view of a disk
type storageReader struct {
storage StorageAPI
volume, path string
offset int64
}
func (r *storageReader) Read(p []byte) (n int, err error) {
nn, err := r.storage.ReadFile(r.volume, r.path, r.offset, p)
r.offset += nn
n = int(nn)
if err == io.ErrUnexpectedEOF && nn > 0 {
err = io.EOF
}
return
}
// storageWriter is a io.Writer view of a disk.
type storageWriter struct {
storage StorageAPI
volume, path string
}
func (w *storageWriter) Write(p []byte) (n int, err error) {
err = w.storage.AppendFile(w.volume, w.path, p)
if err == nil {
n = len(p)
}
return
}
// StorageWriter returns a new io.Writer which appends data to the file
// at the given disk, volume and path.
func StorageWriter(storage StorageAPI, volume, path string) io.Writer {
return &storageWriter{storage, volume, path}
}
// StorageReader returns a new io.Reader which reads data to the file
// at the given disk, volume, path and offset.
func StorageReader(storage StorageAPI, volume, path string, offset int64) io.Reader {
return &storageReader{storage, volume, path, offset}
}

View File

@ -262,8 +262,7 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
} }
// ReadFileWithVerify - reads a file at remote path and fills the buffer. // ReadFileWithVerify - reads a file at remote path and fills the buffer.
func (n *networkStorage) ReadFileWithVerify(volume string, path string, offset int64, func (n *networkStorage) ReadFileWithVerify(volume string, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (m int64, err error) {
buffer []byte, algo HashAlgo, expectedHash string) (m int64, err error) {
defer func() { defer func() {
if r := recover(); r != nil { if r := recover(); r != nil {
@ -279,8 +278,8 @@ func (n *networkStorage) ReadFileWithVerify(volume string, path string, offset i
Path: path, Path: path,
Offset: offset, Offset: offset,
Buffer: buffer, Buffer: buffer,
Algo: algo, Algo: verifier.algorithm,
ExpectedHash: expectedHash, ExpectedHash: verifier.sum,
}, &result) }, &result)
// Copy results to buffer. // Copy results to buffer.

View File

@ -18,7 +18,6 @@ package cmd
import ( import (
"bytes" "bytes"
"encoding/hex"
"errors" "errors"
"fmt" "fmt"
"io" "io"
@ -391,13 +390,13 @@ func (s *TestRPCStorageSuite) testRPCStorageFileOps(t *testing.T) {
t.Errorf("Expected %s, got %s", string(buf[4:9]), string(buf1)) t.Errorf("Expected %s, got %s", string(buf[4:9]), string(buf1))
} }
blakeHash := func(s string) string { blakeHash := func(b []byte) []byte {
k := blake2b.Sum512([]byte(s)) k := blake2b.Sum512(b)
return hex.EncodeToString(k[:]) return k[:]
} }
verifier := NewBitrotVerifier(BLAKE2b512, blakeHash(buf))
buf2 := make([]byte, 2) buf2 := make([]byte, 2)
n, err = storageDisk.ReadFileWithVerify("myvol", "file1", 1, n, err = storageDisk.ReadFileWithVerify("myvol", "file1", 1, buf2, verifier)
buf2, HashBlake2b, blakeHash(string(buf)))
if err != nil { if err != nil {
t.Error("Error in ReadFileWithVerify", err) t.Error("Error in ReadFileWithVerify", err)
} }

View File

@ -79,11 +79,10 @@ type ReadFileWithVerifyArgs struct {
Buffer []byte Buffer []byte
// Algorithm used in bit-rot hash computation. // Algorithm used in bit-rot hash computation.
Algo HashAlgo Algo BitrotAlgorithm
// Stored hash value (hex-encoded) used to compare with // Stored hash value used to compare with computed value.
// computed value. ExpectedHash []byte
ExpectedHash string
} }
// PrepareFileArgs represents append file RPC arguments. // PrepareFileArgs represents append file RPC arguments.

View File

@ -164,10 +164,7 @@ func (s *storageServer) ReadFileWithVerifyHandler(args *ReadFileWithVerifyArgs,
if err = args.IsAuthenticated(); err != nil { if err = args.IsAuthenticated(); err != nil {
return err return err
} }
n, err := s.storage.ReadFileWithVerify(args.Vol, args.Path, args.Offset, args.Buffer, NewBitrotVerifier(args.Algo, args.ExpectedHash))
var n int64
n, err = s.storage.ReadFileWithVerify(args.Vol, args.Path, args.Offset, args.Buffer,
args.Algo, args.ExpectedHash)
// Sending an error over the rpc layer, would cause unmarshalling to fail. In situations // Sending an error over the rpc layer, would cause unmarshalling to fail. In situations
// when we have short read i.e `io.ErrUnexpectedEOF` treat it as good condition and copy // when we have short read i.e `io.ErrUnexpectedEOF` treat it as good condition and copy
// the buffer properly. // the buffer properly.

View File

@ -17,11 +17,23 @@
package cmd package cmd
import ( import (
"encoding/hex" "crypto/subtle"
"path/filepath" "path/filepath"
"sync"
"time" "time"
"io"
) )
// healBufferPool is a pool of reusable buffers used to verify a stream
// while healing.
var healBufferPool = sync.Pool{
New: func() interface{} {
b := make([]byte, readSizeV1)
return &b
},
}
// commonTime returns a maximally occurring time from a list of time. // commonTime returns a maximally occurring time from a list of time.
func commonTime(modTimes []time.Time) (modTime time.Time, count int) { func commonTime(modTimes []time.Time) (modTime time.Time, count int) {
var maxima int // Counter for remembering max occurrence of elements. var maxima int // Counter for remembering max occurrence of elements.
@ -252,36 +264,32 @@ func xlHealStat(xl xlObjects, partsMetadata []xlMetaV1, errs []error) HealObject
// calculating blake2b checksum. // calculating blake2b checksum.
func disksWithAllParts(onlineDisks []StorageAPI, partsMetadata []xlMetaV1, errs []error, bucket, object string) ([]StorageAPI, []error, error) { func disksWithAllParts(onlineDisks []StorageAPI, partsMetadata []xlMetaV1, errs []error, bucket, object string) ([]StorageAPI, []error, error) {
availableDisks := make([]StorageAPI, len(onlineDisks)) availableDisks := make([]StorageAPI, len(onlineDisks))
buffer := healBufferPool.Get().(*[]byte)
defer healBufferPool.Put(buffer)
for diskIndex, onlineDisk := range onlineDisks { for diskIndex, onlineDisk := range onlineDisks {
if onlineDisk == nil { if onlineDisk == OfflineDisk {
continue continue
} }
// disk has a valid xl.json but may not have all the // disk has a valid xl.json but may not have all the
// parts. This is considered an outdated disk, since // parts. This is considered an outdated disk, since
// it needs healing too. // it needs healing too.
for _, part := range partsMetadata[diskIndex].Parts { for _, part := range partsMetadata[diskIndex].Parts {
// compute blake2b sum of part.
partPath := filepath.Join(object, part.Name) partPath := filepath.Join(object, part.Name)
checkSumInfo := partsMetadata[diskIndex].Erasure.GetCheckSumInfo(part.Name) checkSumInfo := partsMetadata[diskIndex].Erasure.GetChecksumInfo(part.Name)
hash := newHash(checkSumInfo.Algorithm) hash := checkSumInfo.Algorithm.New()
blakeBytes, hErr := hashSum(onlineDisk, bucket, partPath, hash) _, hErr := io.CopyBuffer(hash, StorageReader(onlineDisk, bucket, partPath, 0), *buffer)
if hErr == errFileNotFound { if hErr == errFileNotFound {
errs[diskIndex] = errFileNotFound errs[diskIndex] = errFileNotFound
availableDisks[diskIndex] = nil availableDisks[diskIndex] = OfflineDisk
break break
} }
if hErr != nil && hErr != errFileNotFound { if hErr != nil && hErr != errFileNotFound {
return nil, nil, traceError(hErr) return nil, nil, traceError(hErr)
} }
if subtle.ConstantTimeCompare(hash.Sum(nil), checkSumInfo.Hash) != 1 {
blakeSum := hex.EncodeToString(blakeBytes)
// if blake2b sum doesn't match for a part
// then this disk is outdated and needs
// healing.
if blakeSum != checkSumInfo.Hash {
errs[diskIndex] = errFileNotFound errs[diskIndex] = errFileNotFound
availableDisks[diskIndex] = nil availableDisks[diskIndex] = OfflineDisk
break break
} }
availableDisks[diskIndex] = onlineDisk availableDisks[diskIndex] = onlineDisk

View File

@ -87,12 +87,12 @@ func TestCommonTime(t *testing.T) {
// partsMetaFromModTimes - returns slice of modTimes given metadata of // partsMetaFromModTimes - returns slice of modTimes given metadata of
// an object part. // an object part.
func partsMetaFromModTimes(modTimes []time.Time, checksums []checkSumInfo) []xlMetaV1 { func partsMetaFromModTimes(modTimes []time.Time, algorithm BitrotAlgorithm, checksums []ChecksumInfo) []xlMetaV1 {
var partsMetadata []xlMetaV1 var partsMetadata []xlMetaV1
for _, modTime := range modTimes { for _, modTime := range modTimes {
partsMetadata = append(partsMetadata, xlMetaV1{ partsMetadata = append(partsMetadata, xlMetaV1{
Erasure: erasureInfo{ Erasure: ErasureInfo{
Checksum: checksums, Checksums: checksums,
}, },
Stat: statInfo{ Stat: statInfo{
ModTime: modTime, ModTime: modTime,
@ -270,7 +270,7 @@ func TestListOnlineDisks(t *testing.T) {
} }
partsMetadata := partsMetaFromModTimes(test.modTimes, xlMeta.Erasure.Checksum) partsMetadata := partsMetaFromModTimes(test.modTimes, DefaultBitrotAlgorithm, xlMeta.Erasure.Checksums)
onlineDisks, modTime := listOnlineDisks(xlDisks, partsMetadata, test.errs) onlineDisks, modTime := listOnlineDisks(xlDisks, partsMetadata, test.errs)
availableDisks, newErrs, _ := disksWithAllParts(onlineDisks, partsMetadata, test.errs, bucket, object) availableDisks, newErrs, _ := disksWithAllParts(onlineDisks, partsMetadata, test.errs, bucket, object)
@ -357,8 +357,7 @@ func TestDisksWithAllParts(t *testing.T) {
t.Fatalf("Failed to make a bucket %v", err) t.Fatalf("Failed to make a bucket %v", err)
} }
_, err = obj.PutObject(bucket, object, int64(len(data)), _, err = obj.PutObject(bucket, object, int64(len(data)), bytes.NewReader(data), nil, "")
bytes.NewReader(data), nil, "")
if err != nil { if err != nil {
t.Fatalf("Failed to putObject %v", err) t.Fatalf("Failed to putObject %v", err)
} }
@ -368,8 +367,6 @@ func TestDisksWithAllParts(t *testing.T) {
t.Fatalf("Failed to read xl meta data %v", err) t.Fatalf("Failed to read xl meta data %v", err)
} }
// Replace the default blake2b erasure algorithm to HashSha256 and test that
// disks are excluded
diskFailures := make(map[int]string) diskFailures := make(map[int]string)
// key = disk index, value = part name with hash mismatch // key = disk index, value = part name with hash mismatch
diskFailures[0] = "part.3" diskFailures[0] = "part.3"
@ -377,16 +374,15 @@ func TestDisksWithAllParts(t *testing.T) {
diskFailures[15] = "part.2" diskFailures[15] = "part.2"
for diskIndex, partName := range diskFailures { for diskIndex, partName := range diskFailures {
for index, info := range partsMetadata[diskIndex].Erasure.Checksum { for index, info := range partsMetadata[diskIndex].Erasure.Checksums {
if info.Name == partName { if info.Name == partName {
partsMetadata[diskIndex].Erasure.Checksum[index].Algorithm = HashSha256 partsMetadata[diskIndex].Erasure.Checksums[index].Hash[0]++
} }
} }
} }
errs = make([]error, len(xlDisks)) errs = make([]error, len(xlDisks))
filteredDisks, errs, err := filteredDisks, errs, err := disksWithAllParts(xlDisks, partsMetadata, errs, bucket, object)
disksWithAllParts(xlDisks, partsMetadata, errs, bucket, object)
if err != nil { if err != nil {
t.Errorf("Unexpected error: %s", err) t.Errorf("Unexpected error: %s", err)
} }
@ -396,7 +392,6 @@ func TestDisksWithAllParts(t *testing.T) {
} }
for diskIndex, disk := range filteredDisks { for diskIndex, disk := range filteredDisks {
if _, ok := diskFailures[diskIndex]; ok { if _, ok := diskFailures[diskIndex]; ok {
if disk != nil { if disk != nil {
t.Errorf("Disk not filtered as expected, disk: %d", diskIndex) t.Errorf("Disk not filtered as expected, disk: %d", diskIndex)
@ -422,8 +417,7 @@ func TestDisksWithAllParts(t *testing.T) {
t.Fatalf("Failed to read xl meta data %v", err) t.Fatalf("Failed to read xl meta data %v", err)
} }
filteredDisks, errs, err = filteredDisks, errs, err = disksWithAllParts(xlDisks, partsMetadata, errs, bucket, object)
disksWithAllParts(xlDisks, partsMetadata, errs, bucket, object)
if err != nil { if err != nil {
t.Errorf("Unexpected error: %s", err) t.Errorf("Unexpected error: %s", err)
} }

View File

@ -446,30 +446,35 @@ func healObject(storageDisks []StorageAPI, bucket string, object string, quorum
// Checksum of the part files. checkSumInfos[index] will contain checksums // Checksum of the part files. checkSumInfos[index] will contain checksums
// of all the part files in the outDatedDisks[index] // of all the part files in the outDatedDisks[index]
checkSumInfos := make([][]checkSumInfo, len(outDatedDisks)) checksumInfos := make([][]ChecksumInfo, len(outDatedDisks))
// Heal each part. erasureHealFile() will write the healed part to // Heal each part. erasureHealFile() will write the healed part to
// .minio/tmp/uuid/ which needs to be renamed later to the final location. // .minio/tmp/uuid/ which needs to be renamed later to the final location.
storage, err := NewErasureStorage(latestDisks, latestMeta.Erasure.DataBlocks, latestMeta.Erasure.ParityBlocks)
if err != nil {
return 0, 0, toObjectErr(err, bucket, object)
}
checksums := make([][]byte, len(latestDisks))
for partIndex := 0; partIndex < len(latestMeta.Parts); partIndex++ { for partIndex := 0; partIndex < len(latestMeta.Parts); partIndex++ {
partName := latestMeta.Parts[partIndex].Name partName := latestMeta.Parts[partIndex].Name
partSize := latestMeta.Parts[partIndex].Size partSize := latestMeta.Parts[partIndex].Size
erasure := latestMeta.Erasure erasure := latestMeta.Erasure
sumInfo := latestMeta.Erasure.GetCheckSumInfo(partName) var algorithm BitrotAlgorithm
for i, disk := range storage.disks {
if disk != OfflineDisk {
info := partsMetadata[i].Erasure.GetChecksumInfo(partName)
algorithm = info.Algorithm
checksums[i] = info.Hash
}
}
// Heal the part file. // Heal the part file.
checkSums, hErr := erasureHealFile(latestDisks, outDatedDisks, file, hErr := storage.HealFile(outDatedDisks, bucket, pathJoin(object, partName), erasure.BlockSize, minioMetaTmpBucket, pathJoin(tmpID, partName), partSize, algorithm, checksums)
bucket, pathJoin(object, partName),
minioMetaTmpBucket, pathJoin(tmpID, partName),
partSize, erasure.BlockSize, erasure.DataBlocks, erasure.ParityBlocks, sumInfo.Algorithm)
if hErr != nil { if hErr != nil {
return 0, 0, toObjectErr(hErr, bucket, object) return 0, 0, toObjectErr(hErr, bucket, object)
} }
for index, sum := range checkSums { for i := range outDatedDisks {
if outDatedDisks[index] != nil { if outDatedDisks[i] != OfflineDisk {
checkSumInfos[index] = append(checkSumInfos[index], checkSumInfo{ checksumInfos[i] = append(checksumInfos[i], ChecksumInfo{partName, file.Algorithm, file.Checksums[i]})
Name: partName,
Algorithm: sumInfo.Algorithm,
Hash: sum,
})
} }
} }
} }
@ -480,7 +485,7 @@ func healObject(storageDisks []StorageAPI, bucket string, object string, quorum
continue continue
} }
partsMetadata[index] = latestMeta partsMetadata[index] = latestMeta
partsMetadata[index].Erasure.Checksum = checkSumInfos[index] partsMetadata[index].Erasure.Checksums = checksumInfos[index]
} }
// Generate and write `xl.json` generated from other disks. // Generate and write `xl.json` generated from other disks.

View File

@ -491,8 +491,7 @@ func TestHealObjectXL(t *testing.T) {
var uploadedParts []completePart var uploadedParts []completePart
for _, partID := range []int{2, 1} { for _, partID := range []int{2, 1} {
pInfo, err1 := obj.PutObjectPart(bucket, object, uploadID, partID, pInfo, err1 := obj.PutObjectPart(bucket, object, uploadID, partID, int64(len(data)), bytes.NewReader(data), "", "")
int64(len(data)), bytes.NewReader(data), "", "")
if err1 != nil { if err1 != nil {
t.Fatalf("Failed to upload a part - %v", err1) t.Fatalf("Failed to upload a part - %v", err1)
} }

View File

@ -17,20 +17,97 @@
package cmd package cmd
import ( import (
"crypto"
"crypto/sha256"
"encoding/hex"
"encoding/json" "encoding/json"
"errors" "errors"
"fmt"
"hash"
"path" "path"
"runtime" "runtime"
"sort" "sort"
"sync" "sync"
"time" "time"
"golang.org/x/crypto/blake2b"
) )
const erasureAlgorithmKlauspost = "klauspost/reedsolomon/vandermonde"
// DefaultBitrotAlgorithm is the default algorithm used for bitrot protection.
var DefaultBitrotAlgorithm = BLAKE2b512
func init() {
newBLAKE2b := func() hash.Hash {
b2, _ := blake2b.New512(nil) // New512 never returns an error if the key is nil
return b2
}
crypto.RegisterHash(crypto.Hash(SHA256), sha256.New)
crypto.RegisterHash(crypto.Hash(BLAKE2b512), newBLAKE2b)
crypto.RegisterHash(crypto.Hash(HighwayHash256), nil) // TODO(aead): currently not supported, waiting for google to finish algorithm spec.
if runtime.GOARCH == "arm64" { // use SHA256 hardware implementation of arm64
DefaultBitrotAlgorithm = SHA256
}
}
// BitrotAlgorithm specifies a algorithm used for bitrot protection.
type BitrotAlgorithm crypto.Hash
const ( const (
// Erasure related constants. // SHA256 represents the SHA-256 hash function
erasureAlgorithmKlauspost = "klauspost/reedsolomon/vandermonde" SHA256 = BitrotAlgorithm(crypto.SHA256)
// HighwayHash256 represents the HighwayHash-256 hash function
HighwayHash256 = BitrotAlgorithm(crypto.SHA3_256) // we must define that HighwayHash-256 is SHA3-256 because there is no HighwayHash constant in golang/crypto yet.
// BLAKE2b512 represents the BLAKE2b-256 hash function
BLAKE2b512 = BitrotAlgorithm(crypto.SHA3_512) // we must define that BLAKE2b-512 is SHA3-512 because there is no BLAKE2b-512 constant in golang/crypto yet - FIXME: Go1.9 has BLAKE2 constants
) )
var bitrotAlgorithms = map[BitrotAlgorithm]string{
SHA256: "sha256",
BLAKE2b512: "blake2b",
HighwayHash256: "highwayhash256",
}
// New returns a new hash.Hash calculating the given bitrot algorithm. New panics
// if the algorithm is not supported or not linked into the binary.
func (a BitrotAlgorithm) New() hash.Hash {
if _, ok := bitrotAlgorithms[a]; !ok {
panic(fmt.Sprintf("bitrot algorithm #%d is not supported", a))
}
return crypto.Hash(a).New()
}
// Available reports whether the given algorihm is a supported and linked into the binary.
func (a BitrotAlgorithm) Available() bool {
_, ok := bitrotAlgorithms[a]
return ok && crypto.Hash(a).Available()
}
// String returns the string identifier for a given bitrot algorithm.
// If the algorithm is not supported String panics.
func (a BitrotAlgorithm) String() string {
if name, ok := bitrotAlgorithms[a]; ok {
return name
}
panic(fmt.Sprintf("bitrot algorithm #%d is not supported", a))
}
// BitrotAlgorithmFromString returns a bitrot algorithm from the given string representation.
// It returns 0 if the string representation does not match any supported algorithm.
// The zero value of a bitrot algorithm is never supported.
func BitrotAlgorithmFromString(s string) (a BitrotAlgorithm) {
for alg, name := range bitrotAlgorithms {
if name == s {
return alg
}
}
return
}
// objectPartInfo Info of each part kept in the multipart metadata // objectPartInfo Info of each part kept in the multipart metadata
// file after CompleteMultipartUpload() is called. // file after CompleteMultipartUpload() is called.
type objectPartInfo struct { type objectPartInfo struct {
@ -47,91 +124,92 @@ func (t byObjectPartNumber) Len() int { return len(t) }
func (t byObjectPartNumber) Swap(i, j int) { t[i], t[j] = t[j], t[i] } func (t byObjectPartNumber) Swap(i, j int) { t[i], t[j] = t[j], t[i] }
func (t byObjectPartNumber) Less(i, j int) bool { return t[i].Number < t[j].Number } func (t byObjectPartNumber) Less(i, j int) bool { return t[i].Number < t[j].Number }
// checkSumInfo - carries checksums of individual scattered parts per disk. // ChecksumInfo - carries checksums of individual scattered parts per disk.
type checkSumInfo struct { type ChecksumInfo struct {
Name string
Algorithm BitrotAlgorithm
Hash []byte
}
// MarshalJSON marshals the ChecksumInfo struct
func (c ChecksumInfo) MarshalJSON() ([]byte, error) {
type checksuminfo struct {
Name string `json:"name"` Name string `json:"name"`
Algorithm HashAlgo `json:"algorithm"` Algorithm string `json:"algorithm"`
Hash string `json:"hash"` Hash string `json:"hash"`
} }
// HashAlgo - represents a supported hashing algorithm for bitrot info := checksuminfo{
// verification. Name: c.Name,
type HashAlgo string Algorithm: c.Algorithm.String(),
Hash: hex.EncodeToString(c.Hash),
const (
// HashBlake2b represents the Blake 2b hashing algorithm
HashBlake2b HashAlgo = "blake2b"
// HashSha256 represents the SHA256 hashing algorithm
HashSha256 HashAlgo = "sha256"
)
// isValidHashAlgo - function that checks if the hash algorithm is
// valid (known and used).
func isValidHashAlgo(algo HashAlgo) bool {
switch algo {
case HashSha256, HashBlake2b:
return true
default:
return false
} }
return json.Marshal(info)
} }
// Constant indicates current bit-rot algo used when creating objects. // UnmarshalJSON unmarshals the the given data into the ChecksumInfo struct
// Depending on the architecture we are choosing a different checksum. func (c *ChecksumInfo) UnmarshalJSON(data []byte) error {
var bitRotAlgo = getDefaultBitRotAlgo() type checksuminfo struct {
Name string `json:"name"`
// Get the default bit-rot algo depending on the architecture. Algorithm string `json:"algorithm"`
// Currently this function defaults to "blake2b" as the preferred Hash string `json:"hash"`
// checksum algorithm on all architectures except ARM64. On ARM64
// we use sha256 (optimized using sha2 instructions of ARM NEON chip).
func getDefaultBitRotAlgo() HashAlgo {
switch runtime.GOARCH {
case "arm64":
// As a special case for ARM64 we use an optimized
// version of hash i.e sha256. This is done so that
// blake2b is sub-optimal and slower on ARM64.
// This would also allows erasure coded writes
// on ARM64 servers to be on-par with their
// counter-part X86_64 servers.
return HashSha256
default:
// Default for all other architectures we use blake2b.
return HashBlake2b
}
} }
// erasureInfo - carries erasure coding related information, block var info checksuminfo
// distribution and checksums. err := json.Unmarshal(data, &info)
type erasureInfo struct { if err != nil {
Algorithm HashAlgo `json:"algorithm"` return err
}
c.Algorithm = BitrotAlgorithmFromString(info.Algorithm)
if !c.Algorithm.Available() {
return errBitrotHashAlgoInvalid
}
c.Hash, err = hex.DecodeString(info.Hash)
if err != nil {
return err
}
c.Name = info.Name
return nil
}
// ErasureInfo holds erasure coding and bitrot related information.
type ErasureInfo struct {
// Algorithm is the string representation of erasure-coding-algorithm
Algorithm string `json:"algorithm"`
// DataBlocks is the number of data blocks for erasure-coding
DataBlocks int `json:"data"` DataBlocks int `json:"data"`
// ParityBlocks is the number of parity blocks for erasure-coding
ParityBlocks int `json:"parity"` ParityBlocks int `json:"parity"`
// BlockSize is the size of one erasure-coded block
BlockSize int64 `json:"blockSize"` BlockSize int64 `json:"blockSize"`
// Index is the index of the current disk
Index int `json:"index"` Index int `json:"index"`
// Distribution is the distribution of the data and parity blocks
Distribution []int `json:"distribution"` Distribution []int `json:"distribution"`
Checksum []checkSumInfo `json:"checksum,omitempty"` // Checksums holds all bitrot checksums of all erasure encoded blocks
Checksums []ChecksumInfo `json:"checksum,omitempty"`
} }
// AddCheckSum - add checksum of a part. // AddChecksumInfo adds a checksum of a part.
func (e *erasureInfo) AddCheckSumInfo(ckSumInfo checkSumInfo) { func (e *ErasureInfo) AddChecksumInfo(ckSumInfo ChecksumInfo) {
for i, sum := range e.Checksum { for i, sum := range e.Checksums {
if sum.Name == ckSumInfo.Name { if sum.Name == ckSumInfo.Name {
e.Checksum[i] = ckSumInfo e.Checksums[i] = ckSumInfo
return return
} }
} }
e.Checksum = append(e.Checksum, ckSumInfo) e.Checksums = append(e.Checksums, ckSumInfo)
} }
// GetCheckSumInfo - get checksum of a part. // GetChecksumInfo - get checksum of a part.
func (e erasureInfo) GetCheckSumInfo(partName string) (ckSum checkSumInfo) { func (e ErasureInfo) GetChecksumInfo(partName string) (ckSum ChecksumInfo) {
// Return the checksum. // Return the checksum.
for _, sum := range e.Checksum { for _, sum := range e.Checksums {
if sum.Name == partName { if sum.Name == partName {
return sum return sum
} }
} }
return checkSumInfo{Algorithm: bitRotAlgo} return ChecksumInfo{}
} }
// statInfo - carries stat information of the object. // statInfo - carries stat information of the object.
@ -146,7 +224,7 @@ type xlMetaV1 struct {
Format string `json:"format"` // Format of the current `xl.json`. Format string `json:"format"` // Format of the current `xl.json`.
Stat statInfo `json:"stat"` // Stat of the current object `xl.json`. Stat statInfo `json:"stat"` // Stat of the current object `xl.json`.
// Erasure coded info for the current object `xl.json`. // Erasure coded info for the current object `xl.json`.
Erasure erasureInfo `json:"erasure"` Erasure ErasureInfo `json:"erasure"`
// Minio release tag for current object `xl.json`. // Minio release tag for current object `xl.json`.
Minio struct { Minio struct {
Release string `json:"release"` Release string `json:"release"`
@ -177,7 +255,7 @@ func newXLMetaV1(object string, dataBlocks, parityBlocks int) (xlMeta xlMetaV1)
xlMeta.Version = xlMetaVersion xlMeta.Version = xlMetaVersion
xlMeta.Format = xlMetaFormat xlMeta.Format = xlMetaFormat
xlMeta.Minio.Release = ReleaseTag xlMeta.Minio.Release = ReleaseTag
xlMeta.Erasure = erasureInfo{ xlMeta.Erasure = ErasureInfo{
Algorithm: erasureAlgorithmKlauspost, Algorithm: erasureAlgorithmKlauspost,
DataBlocks: dataBlocks, DataBlocks: dataBlocks,
ParityBlocks: parityBlocks, ParityBlocks: parityBlocks,

View File

@ -488,6 +488,7 @@ func (xl xlObjects) newMultipartUpload(bucket string, object string, meta map[st
uploadID := mustGetUUID() uploadID := mustGetUUID()
uploadIDPath := path.Join(bucket, object, uploadID) uploadIDPath := path.Join(bucket, object, uploadID)
tempUploadIDPath := uploadID tempUploadIDPath := uploadID
// Write updated `xl.json` to all disks. // Write updated `xl.json` to all disks.
disks, err := writeSameXLMetadata(xl.storageDisks, minioMetaTmpBucket, tempUploadIDPath, xlMeta, xl.writeQuorum, xl.readQuorum) disks, err := writeSameXLMetadata(xl.storageDisks, minioMetaTmpBucket, tempUploadIDPath, xlMeta, xl.writeQuorum, xl.readQuorum)
if err != nil { if err != nil {
@ -627,23 +628,16 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
mw := io.MultiWriter(writers...) mw := io.MultiWriter(writers...)
var lreader io.Reader var lreader = data
// Limit the reader to its provided size > 0. // Limit the reader to its provided size > 0.
if size > 0 { if size > 0 {
// This is done so that we can avoid erroneous clients sending // This is done so that we can avoid erroneous clients sending
// more data than the set content size. // more data than the set content size.
lreader = io.LimitReader(data, size) lreader = io.LimitReader(data, size)
} else {
// else we read till EOF.
lreader = data
} }
// Construct a tee reader for md5sum.
teeReader := io.TeeReader(lreader, mw)
// Delete the temporary object part. If PutObjectPart succeeds there would be nothing to delete. // Delete the temporary object part. If PutObjectPart succeeds there would be nothing to delete.
defer xl.deleteObject(minioMetaTmpBucket, tmpPart) defer xl.deleteObject(minioMetaTmpBucket, tmpPart)
if size > 0 { if size > 0 {
if pErr := xl.prepareFile(minioMetaTmpBucket, tmpPartPath, size, onlineDisks, xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks); err != nil { if pErr := xl.prepareFile(minioMetaTmpBucket, tmpPartPath, size, onlineDisks, xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks); err != nil {
return pi, toObjectErr(pErr, bucket, object) return pi, toObjectErr(pErr, bucket, object)
@ -651,25 +645,26 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
} }
} }
// We always allow empty part. storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks)
allowEmpty := true if err != nil {
return pi, toObjectErr(err, bucket, object)
// Erasure code data and write across all disks. }
onlineDisks, sizeWritten, checkSums, err := erasureCreateFile(onlineDisks, minioMetaTmpBucket, tmpPartPath, teeReader, allowEmpty, xlMeta.Erasure.BlockSize, xl.dataBlocks, xl.parityBlocks, bitRotAlgo, xl.writeQuorum) buffer := make([]byte, xlMeta.Erasure.BlockSize, 2*xlMeta.Erasure.BlockSize) // alloc additional space for parity blocks created while erasure coding
file, err := storage.CreateFile(io.TeeReader(lreader, mw), minioMetaTmpBucket, tmpPartPath, buffer, DefaultBitrotAlgorithm, xl.writeQuorum)
if err != nil { if err != nil {
return pi, toObjectErr(err, bucket, object) return pi, toObjectErr(err, bucket, object)
} }
// Should return IncompleteBody{} error when reader has fewer bytes // Should return IncompleteBody{} error when reader has fewer bytes
// than specified in request header. // than specified in request header.
if sizeWritten < size { if file.Size < size {
return pi, traceError(IncompleteBody{}) return pi, traceError(IncompleteBody{})
} }
// For size == -1, perhaps client is sending in chunked encoding // For size == -1, perhaps client is sending in chunked encoding
// set the size as size that was actually written. // set the size as size that was actually written.
if size == -1 { if size == -1 {
size = sizeWritten size = file.Size
} }
// Calculate new md5sum. // Calculate new md5sum.
@ -727,16 +722,12 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
// Add the current part. // Add the current part.
xlMeta.AddObjectPart(partID, partSuffix, newMD5Hex, size) xlMeta.AddObjectPart(partID, partSuffix, newMD5Hex, size)
for index, disk := range onlineDisks { for i, disk := range onlineDisks {
if disk == nil { if disk == OfflineDisk {
continue continue
} }
partsMetadata[index].Parts = xlMeta.Parts partsMetadata[i].Parts = xlMeta.Parts
partsMetadata[index].Erasure.AddCheckSumInfo(checkSumInfo{ partsMetadata[i].Erasure.AddChecksumInfo(ChecksumInfo{partSuffix, file.Algorithm, file.Checksums[i]})
Name: partSuffix,
Hash: checkSums[index],
Algorithm: bitRotAlgo,
})
} }
// Write all the checksum metadata. // Write all the checksum metadata.

View File

@ -249,7 +249,11 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
chunkSize := getChunkSize(xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks) chunkSize := getChunkSize(xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks)
pool := bpool.NewBytePool(chunkSize, len(onlineDisks)) pool := bpool.NewBytePool(chunkSize, len(onlineDisks))
// Read from all parts. storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks)
if err != nil {
return toObjectErr(err, bucket, object)
}
checksums := make([][]byte, len(storage.disks))
for ; partIndex <= lastPartIndex; partIndex++ { for ; partIndex <= lastPartIndex; partIndex++ {
if length == totalBytesRead { if length == totalBytesRead {
break break
@ -265,33 +269,24 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
} }
// Get the checksums of the current part. // Get the checksums of the current part.
checkSums := make([]string, len(onlineDisks)) var algorithm BitrotAlgorithm
var ckSumAlgo HashAlgo for index, disk := range storage.disks {
for index, disk := range onlineDisks { if disk == OfflineDisk {
// Disk is not found skip the checksum.
if disk == nil {
checkSums[index] = ""
continue continue
} }
ckSumInfo := metaArr[index].Erasure.GetCheckSumInfo(partName) checksumInfo := metaArr[index].Erasure.GetChecksumInfo(partName)
checkSums[index] = ckSumInfo.Hash algorithm = checksumInfo.Algorithm
// Set checksum algo only once, while it is possible to have checksums[index] = checksumInfo.Hash
// different algos per block because of our `xl.json`.
// It is not a requirement, set this only once for all the disks.
if ckSumAlgo == "" {
ckSumAlgo = ckSumInfo.Algorithm
}
} }
// Start erasure decoding and writing to the client. file, err := storage.ReadFile(mw, bucket, pathJoin(object, partName), partOffset, readSize, partSize, checksums, algorithm, xlMeta.Erasure.BlockSize, pool)
n, err := erasureReadFile(mw, onlineDisks, bucket, pathJoin(object, partName), partOffset, readSize, partSize, xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks, checkSums, ckSumAlgo, pool)
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)
} }
// Track total bytes read from disk and written to the client. // Track total bytes read from disk and written to the client.
totalBytesRead += n totalBytesRead += file.Size
// partOffset will be valid only for the first part, hence reset it to 0 for // partOffset will be valid only for the first part, hence reset it to 0 for
// the remaining parts. // the remaining parts.
@ -540,6 +535,11 @@ func (xl xlObjects) PutObject(bucket string, object string, size int64, data io.
// Total size of the written object // Total size of the written object
var sizeWritten int64 var sizeWritten int64
storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks)
if err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
buffer := make([]byte, partsMetadata[0].Erasure.BlockSize, 2*partsMetadata[0].Erasure.BlockSize) // alloc additional space for parity blocks created while erasure coding
// Read data and split into parts - similar to multipart mechanism // Read data and split into parts - similar to multipart mechanism
for partIdx := 1; ; partIdx++ { for partIdx := 1; ; partIdx++ {
// Compute part name // Compute part name
@ -558,55 +558,39 @@ func (xl xlObjects) PutObject(bucket string, object string, size int64, data io.
// Hint the filesystem to pre-allocate one continuous large block. // Hint the filesystem to pre-allocate one continuous large block.
// This is only an optimization. // This is only an optimization.
if curPartSize > 0 { if curPartSize > 0 {
pErr := xl.prepareFile(minioMetaTmpBucket, tempErasureObj, curPartSize, onlineDisks, xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks) pErr := xl.prepareFile(minioMetaTmpBucket, tempErasureObj, curPartSize, storage.disks, xlMeta.Erasure.BlockSize, xlMeta.Erasure.DataBlocks)
if pErr != nil { if pErr != nil {
return ObjectInfo{}, toObjectErr(pErr, bucket, object) return ObjectInfo{}, toObjectErr(pErr, bucket, object)
} }
} }
// partReader streams at most maximum part size file, erasureErr := storage.CreateFile(io.LimitReader(teeReader, globalPutPartSize), minioMetaTmpBucket, tempErasureObj, buffer, DefaultBitrotAlgorithm, xl.writeQuorum)
partReader := io.LimitReader(teeReader, globalPutPartSize)
// Allow creating empty earsure file only when this is the first part. This flag is useful
// when size == -1 because in this case, we are not able to predict how many parts we will have.
allowEmptyPart := partIdx == 1
var partSizeWritten int64
var checkSums []string
var erasureErr error
// Erasure code data and write across all disks.
onlineDisks, partSizeWritten, checkSums, erasureErr = erasureCreateFile(onlineDisks, minioMetaTmpBucket, tempErasureObj, partReader, allowEmptyPart, partsMetadata[0].Erasure.BlockSize, partsMetadata[0].Erasure.DataBlocks, partsMetadata[0].Erasure.ParityBlocks, bitRotAlgo, xl.writeQuorum)
if erasureErr != nil { if erasureErr != nil {
return ObjectInfo{}, toObjectErr(erasureErr, minioMetaTmpBucket, tempErasureObj) return ObjectInfo{}, toObjectErr(erasureErr, minioMetaTmpBucket, tempErasureObj)
} }
// Should return IncompleteBody{} error when reader has fewer bytes // Should return IncompleteBody{} error when reader has fewer bytes
// than specified in request header. // than specified in request header.
if partSizeWritten < int64(curPartSize) { if file.Size < int64(curPartSize) {
return ObjectInfo{}, traceError(IncompleteBody{}) return ObjectInfo{}, traceError(IncompleteBody{})
} }
// Update the total written size // Update the total written size
sizeWritten += partSizeWritten sizeWritten += file.Size
// If erasure stored some data in the loop or created an empty file // allowEmpty creating empty earsure file only when this is the first part. This flag is useful
if partSizeWritten > 0 || allowEmptyPart { // when size == -1 because in this case, we are not able to predict how many parts we will have.
for index := range partsMetadata { allowEmpty := partIdx == 1
// Add the part to xl.json. if file.Size > 0 || allowEmpty {
partsMetadata[index].AddObjectPart(partIdx, partName, "", partSizeWritten) for i := range partsMetadata {
// Add part checksum info to xl.json. partsMetadata[i].AddObjectPart(partIdx, partName, "", file.Size)
partsMetadata[index].Erasure.AddCheckSumInfo(checkSumInfo{ partsMetadata[i].Erasure.AddChecksumInfo(ChecksumInfo{partName, file.Algorithm, file.Checksums[i]})
Name: partName,
Hash: checkSums[index],
Algorithm: bitRotAlgo,
})
} }
} }
// If we didn't write anything or we know that the next part doesn't have any // If we didn't write anything or we know that the next part doesn't have any
// data to write, we should quit this loop immediately // data to write, we should quit this loop immediately
if partSizeWritten == 0 { if file.Size == 0 {
break break
} }

View File

@ -17,6 +17,7 @@
package cmd package cmd
import ( import (
"encoding/hex"
"errors" "errors"
"hash/crc32" "hash/crc32"
"path" "path"
@ -149,8 +150,8 @@ func parseXLRelease(xlMetaBuf []byte) string {
return gjson.GetBytes(xlMetaBuf, "minio.release").String() return gjson.GetBytes(xlMetaBuf, "minio.release").String()
} }
func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo { func parseXLErasureInfo(xlMetaBuf []byte) (ErasureInfo, error) {
erasure := erasureInfo{} erasure := ErasureInfo{}
erasureResult := gjson.GetBytes(xlMetaBuf, "erasure") erasureResult := gjson.GetBytes(xlMetaBuf, "erasure")
// parse the xlV1Meta.Erasure.Distribution. // parse the xlV1Meta.Erasure.Distribution.
disResult := erasureResult.Get("distribution").Array() disResult := erasureResult.Get("distribution").Array()
@ -161,24 +162,28 @@ func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo {
} }
erasure.Distribution = distribution erasure.Distribution = distribution
erasure.Algorithm = HashAlgo(erasureResult.Get("algorithm").String()) erasure.Algorithm = erasureResult.Get("algorithm").String()
erasure.DataBlocks = int(erasureResult.Get("data").Int()) erasure.DataBlocks = int(erasureResult.Get("data").Int())
erasure.ParityBlocks = int(erasureResult.Get("parity").Int()) erasure.ParityBlocks = int(erasureResult.Get("parity").Int())
erasure.BlockSize = erasureResult.Get("blockSize").Int() erasure.BlockSize = erasureResult.Get("blockSize").Int()
erasure.Index = int(erasureResult.Get("index").Int()) erasure.Index = int(erasureResult.Get("index").Int())
// Pare xlMetaV1.Erasure.Checksum array.
checkSumsResult := erasureResult.Get("checksum").Array()
checkSums := make([]checkSumInfo, len(checkSumsResult))
for i, checkSumResult := range checkSumsResult {
checkSum := checkSumInfo{}
checkSum.Name = checkSumResult.Get("name").String()
checkSum.Algorithm = HashAlgo(checkSumResult.Get("algorithm").String())
checkSum.Hash = checkSumResult.Get("hash").String()
checkSums[i] = checkSum
}
erasure.Checksum = checkSums
return erasure checkSumsResult := erasureResult.Get("checksum").Array()
// Parse xlMetaV1.Erasure.Checksum array.
checkSums := make([]ChecksumInfo, len(checkSumsResult))
for i, v := range checkSumsResult {
algorithm := BitrotAlgorithmFromString(v.Get("algorithm").String())
if !algorithm.Available() {
return erasure, traceError(errBitrotHashAlgoInvalid)
}
hash, err := hex.DecodeString(v.Get("hash").String())
if err != nil {
return erasure, traceError(err)
}
checkSums[i] = ChecksumInfo{Name: v.Get("name").String(), Algorithm: algorithm, Hash: hash}
}
erasure.Checksums = checkSums
return erasure, nil
} }
func parseXLParts(xlMetaBuf []byte) []objectPartInfo { func parseXLParts(xlMetaBuf []byte) []objectPartInfo {
@ -207,8 +212,7 @@ func parseXLMetaMap(xlMetaBuf []byte) map[string]string {
} }
// Constructs XLMetaV1 using `gjson` lib to retrieve each field. // Constructs XLMetaV1 using `gjson` lib to retrieve each field.
func xlMetaV1UnmarshalJSON(xlMetaBuf []byte) (xmv xlMetaV1, e error) { func xlMetaV1UnmarshalJSON(xlMetaBuf []byte) (xlMeta xlMetaV1, e error) {
xlMeta := xlMetaV1{}
// obtain version. // obtain version.
xlMeta.Version = parseXLVersion(xlMetaBuf) xlMeta.Version = parseXLVersion(xlMetaBuf)
// obtain format. // obtain format.
@ -216,12 +220,15 @@ func xlMetaV1UnmarshalJSON(xlMetaBuf []byte) (xmv xlMetaV1, e error) {
// Parse xlMetaV1.Stat . // Parse xlMetaV1.Stat .
stat, err := parseXLStat(xlMetaBuf) stat, err := parseXLStat(xlMetaBuf)
if err != nil { if err != nil {
return xmv, err return xlMeta, err
} }
xlMeta.Stat = stat xlMeta.Stat = stat
// parse the xlV1Meta.Erasure fields. // parse the xlV1Meta.Erasure fields.
xlMeta.Erasure = parseXLErasureInfo(xlMetaBuf) xlMeta.Erasure, err = parseXLErasureInfo(xlMetaBuf)
if err != nil {
return xlMeta, err
}
// Parse the XL Parts. // Parse the XL Parts.
xlMeta.Parts = parseXLParts(xlMetaBuf) xlMeta.Parts = parseXLParts(xlMetaBuf)

View File

@ -17,6 +17,8 @@
package cmd package cmd
import ( import (
"bytes"
"encoding/hex"
"encoding/json" "encoding/json"
"errors" "errors"
"reflect" "reflect"
@ -139,7 +141,7 @@ func newTestXLMetaV1() xlMetaV1 {
xlMeta.Version = xlMetaVersion xlMeta.Version = xlMetaVersion
xlMeta.Format = xlMetaFormat xlMeta.Format = xlMetaFormat
xlMeta.Minio.Release = "test" xlMeta.Minio.Release = "test"
xlMeta.Erasure = erasureInfo{ xlMeta.Erasure = ErasureInfo{
Algorithm: "klauspost/reedsolomon/vandermonde", Algorithm: "klauspost/reedsolomon/vandermonde",
DataBlocks: 5, DataBlocks: 5,
ParityBlocks: 5, ParityBlocks: 5,
@ -158,13 +160,12 @@ func newTestXLMetaV1() xlMetaV1 {
return xlMeta return xlMeta
} }
func (m *xlMetaV1) AddTestObjectCheckSum(checkSumNum int, name string, hash string, algo HashAlgo) { func (m *xlMetaV1) AddTestObjectCheckSum(checkSumNum int, name string, algorithm BitrotAlgorithm, hash string) {
checkSum := checkSumInfo{ checksum, err := hex.DecodeString(hash)
Name: name, if err != nil {
Algorithm: algo, panic(err)
Hash: hash,
} }
m.Erasure.Checksum[checkSumNum] = checkSum m.Erasure.Checksums[checkSumNum] = ChecksumInfo{name, algorithm, checksum}
} }
// AddTestObjectPart - add a new object part in order. // AddTestObjectPart - add a new object part in order.
@ -194,14 +195,14 @@ func getXLMetaBytes(totalParts int) []byte {
func getSampleXLMeta(totalParts int) xlMetaV1 { func getSampleXLMeta(totalParts int) xlMetaV1 {
xlMeta := newTestXLMetaV1() xlMeta := newTestXLMetaV1()
// Number of checksum info == total parts. // Number of checksum info == total parts.
xlMeta.Erasure.Checksum = make([]checkSumInfo, totalParts) xlMeta.Erasure.Checksums = make([]ChecksumInfo, totalParts)
// total number of parts. // total number of parts.
xlMeta.Parts = make([]objectPartInfo, totalParts) xlMeta.Parts = make([]objectPartInfo, totalParts)
for i := 0; i < totalParts; i++ { for i := 0; i < totalParts; i++ {
partName := "part." + strconv.Itoa(i+1) partName := "part." + strconv.Itoa(i+1)
// hard coding hash and algo value for the checksum, Since we are benchmarking the parsing of xl.json the magnitude doesn't affect the test, // hard coding hash and algo value for the checksum, Since we are benchmarking the parsing of xl.json the magnitude doesn't affect the test,
// The magnitude doesn't make a difference, only the size does. // The magnitude doesn't make a difference, only the size does.
xlMeta.AddTestObjectCheckSum(i, partName, "a23f5eff248c4372badd9f3b2455a285cd4ca86c3d9a570b091d3fc5cd7ca6d9484bbea3f8c5d8d4f84daae96874419eda578fd736455334afbac2c924b3915a", "blake2b") xlMeta.AddTestObjectCheckSum(i, partName, BLAKE2b512, "a23f5eff248c4372badd9f3b2455a285cd4ca86c3d9a570b091d3fc5cd7ca6d9484bbea3f8c5d8d4f84daae96874419eda578fd736455334afbac2c924b3915a")
xlMeta.AddTestObjectPart(i, partName, "d3fdd79cc3efd5fe5c068d7be397934b", 67108864) xlMeta.AddTestObjectPart(i, partName, "d3fdd79cc3efd5fe5c068d7be397934b", 67108864)
} }
return xlMeta return xlMeta
@ -248,18 +249,18 @@ func compareXLMetaV1(t *testing.T, unMarshalXLMeta, gjsonXLMeta xlMetaV1) {
} }
} }
if len(unMarshalXLMeta.Erasure.Checksum) != len(gjsonXLMeta.Erasure.Checksum) { if len(unMarshalXLMeta.Erasure.Checksums) != len(gjsonXLMeta.Erasure.Checksums) {
t.Errorf("Expected the size of Erasure Checksum to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Checksum), len(gjsonXLMeta.Erasure.Checksum)) t.Errorf("Expected the size of Erasure Checksums to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Checksums), len(gjsonXLMeta.Erasure.Checksums))
} else { } else {
for i := 0; i < len(unMarshalXLMeta.Erasure.Checksum); i++ { for i := 0; i < len(unMarshalXLMeta.Erasure.Checksums); i++ {
if unMarshalXLMeta.Erasure.Checksum[i].Name != gjsonXLMeta.Erasure.Checksum[i].Name { if unMarshalXLMeta.Erasure.Checksums[i].Name != gjsonXLMeta.Erasure.Checksums[i].Name {
t.Errorf("Expected the Erasure Checksum Name to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksum[i].Name, gjsonXLMeta.Erasure.Checksum[i].Name) t.Errorf("Expected the Erasure Checksum Name to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Name, gjsonXLMeta.Erasure.Checksums[i].Name)
} }
if unMarshalXLMeta.Erasure.Checksum[i].Algorithm != gjsonXLMeta.Erasure.Checksum[i].Algorithm { if unMarshalXLMeta.Erasure.Checksums[i].Algorithm != gjsonXLMeta.Erasure.Checksums[i].Algorithm {
t.Errorf("Expected the Erasure Checksum Algorithm to be \"%s\", got \"%s.\"", unMarshalXLMeta.Erasure.Checksum[i].Algorithm, gjsonXLMeta.Erasure.Checksum[i].Algorithm) t.Errorf("Expected the Erasure Checksum Algorithm to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Algorithm, gjsonXLMeta.Erasure.Checksums[i].Algorithm)
} }
if unMarshalXLMeta.Erasure.Checksum[i] != gjsonXLMeta.Erasure.Checksum[i] { if !bytes.Equal(unMarshalXLMeta.Erasure.Checksums[i].Hash, gjsonXLMeta.Erasure.Checksums[i].Hash) {
t.Errorf("Expected the Erasure Checksum Hash to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksum[i].Hash, gjsonXLMeta.Erasure.Checksum[i].Hash) t.Errorf("Expected the Erasure Checksum Hash to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Hash, gjsonXLMeta.Erasure.Checksums[i].Hash)
} }
} }
} }
@ -304,12 +305,12 @@ func TestGetXLMetaV1GJson1(t *testing.T) {
var unMarshalXLMeta xlMetaV1 var unMarshalXLMeta xlMetaV1
if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil { if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil {
t.Errorf("Unmarshalling failed") t.Errorf("Unmarshalling failed: %v", err)
} }
gjsonXLMeta, err := xlMetaV1UnmarshalJSON(xlMetaJSON) gjsonXLMeta, err := xlMetaV1UnmarshalJSON(xlMetaJSON)
if err != nil { if err != nil {
t.Errorf("gjson parsing of XLMeta failed") t.Errorf("gjson parsing of XLMeta failed: %v", err)
} }
compareXLMetaV1(t, unMarshalXLMeta, gjsonXLMeta) compareXLMetaV1(t, unMarshalXLMeta, gjsonXLMeta)
} }
@ -322,11 +323,11 @@ func TestGetXLMetaV1GJson10(t *testing.T) {
var unMarshalXLMeta xlMetaV1 var unMarshalXLMeta xlMetaV1
if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil { if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil {
t.Errorf("Unmarshalling failed") t.Errorf("Unmarshalling failed: %v", err)
} }
gjsonXLMeta, err := xlMetaV1UnmarshalJSON(xlMetaJSON) gjsonXLMeta, err := xlMetaV1UnmarshalJSON(xlMetaJSON)
if err != nil { if err != nil {
t.Errorf("gjson parsing of XLMeta failed") t.Errorf("gjson parsing of XLMeta failed: %v", err)
} }
compareXLMetaV1(t, unMarshalXLMeta, gjsonXLMeta) compareXLMetaV1(t, unMarshalXLMeta, gjsonXLMeta)
} }

View File

@ -22,6 +22,19 @@ To get the package use the standard:
go get github.com/klauspost/reedsolomon go get github.com/klauspost/reedsolomon
``` ```
# Changes
## July 20, 2017
`ReconstructData` added to [`Encoder`](https://godoc.org/github.com/klauspost/reedsolomon#Encoder) interface. This can cause compatibility issues if you implement your own Encoder. A simple workaround can be added:
```Go
func (e *YourEnc) ReconstructData(shards [][]byte) error {
return ReconstructData(shards)
}
```
You can of course also do your own implementation. The [`StreamEncoder`](https://godoc.org/github.com/klauspost/reedsolomon#StreamEncoder) handles this without modifying the interface. This is a good lesson on why returning interfaces is not a good design.
# Usage # Usage
This section assumes you know the basics of Reed-Solomon encoding. A good start is this [Backblaze blog post](https://www.backblaze.com/blog/reed-solomon/). This section assumes you know the basics of Reed-Solomon encoding. A good start is this [Backblaze blog post](https://www.backblaze.com/blog/reed-solomon/).

View File

@ -616,7 +616,7 @@ func (r reedSolomon) reconstruct(shards [][]byte, dataOnly bool) error {
var ErrShortData = errors.New("not enough data to fill the number of requested shards") var ErrShortData = errors.New("not enough data to fill the number of requested shards")
// Split a data slice into the number of shards given to the encoder, // Split a data slice into the number of shards given to the encoder,
// and create empty parity shards. // and create empty parity shards if necessary.
// //
// The data will be split into equally sized shards. // The data will be split into equally sized shards.
// If the data size isn't divisible by the number of shards, // If the data size isn't divisible by the number of shards,
@ -631,12 +631,19 @@ func (r reedSolomon) Split(data []byte) ([][]byte, error) {
if len(data) == 0 { if len(data) == 0 {
return nil, ErrShortData return nil, ErrShortData
} }
// Calculate number of bytes per shard. // Calculate number of bytes per data shard.
perShard := (len(data) + r.DataShards - 1) / r.DataShards perShard := (len(data) + r.DataShards - 1) / r.DataShards
if cap(data) > len(data) {
data = data[:cap(data)]
}
// Only allocate memory if necessary
if len(data) < (r.Shards * perShard) {
// Pad data to r.Shards*perShard. // Pad data to r.Shards*perShard.
padding := make([]byte, (r.Shards*perShard)-len(data)) padding := make([]byte, (r.Shards*perShard)-len(data))
data = append(data, padding...) data = append(data, padding...)
}
// Split into equal-length shards. // Split into equal-length shards.
dst := make([][]byte, r.Shards) dst := make([][]byte, r.Shards)

6
vendor/vendor.json vendored
View File

@ -243,10 +243,10 @@
"revisionTime": "2016-10-16T15:41:25Z" "revisionTime": "2016-10-16T15:41:25Z"
}, },
{ {
"checksumSHA1": "DnS7x0Gqc93p4hQ88FgE35vfIRw=", "checksumSHA1": "gYAsuckCW3o4veePKZzEHvCcJro=",
"path": "github.com/klauspost/reedsolomon", "path": "github.com/klauspost/reedsolomon",
"revision": "a9202d772777d8d2264c3e0c6159be5047697380", "revision": "48a4fd05f1730dd3ef9c3f9e943f6091d063f2c4",
"revisionTime": "2017-07-19T04:51:23Z" "revisionTime": "2017-07-22T14:16:58Z"
}, },
{ {
"checksumSHA1": "dNYxHiBLalTqluak2/Z8c3RsSEM=", "checksumSHA1": "dNYxHiBLalTqluak2/Z8c3RsSEM=",