mirror of https://github.com/minio/minio.git
Add new ReadFileWithVerify storage-layer API (#4349)
This is an enhancement to the XL/distributed-XL mode. FS mode is unaffected. The ReadFileWithVerify storage-layer call is similar to ReadFile with the additional functionality of performing bit-rot checking. It accepts additional parameters for a hashing algorithm to use and the expected hex-encoded hash string. This patch provides significant performance improvement because: 1. combines the step of reading the file (during erasure-decoding/reconstruction) with bit-rot verification; 2. limits the number of file-reads; and 3. avoids transferring the file over the network for bit-rot verification. ReadFile API is implemented as ReadFileWithVerify with empty hashing arguments. Credits to AB and Harsha for the algorithmic improvement. Fixes #4236.
This commit is contained in:
parent
cae4683971
commit
8975da4e84
|
@ -28,7 +28,9 @@ import (
|
|||
// erasureCreateFile - writes an entire stream by erasure coding to
|
||||
// all the disks, writes also calculate individual block's checksum
|
||||
// for future bit-rot protection.
|
||||
func erasureCreateFile(disks []StorageAPI, volume, path string, reader io.Reader, allowEmpty bool, blockSize int64, dataBlocks int, parityBlocks int, algo string, writeQuorum int) (bytesWritten int64, checkSums []string, err error) {
|
||||
func erasureCreateFile(disks []StorageAPI, volume, path string, reader io.Reader, allowEmpty bool, blockSize int64,
|
||||
dataBlocks, parityBlocks int, algo HashAlgo, writeQuorum int) (bytesWritten int64, checkSums []string, err error) {
|
||||
|
||||
// Allocated blockSized buffer for reading from incoming stream.
|
||||
buf := make([]byte, blockSize)
|
||||
|
||||
|
|
|
@ -19,7 +19,9 @@ package cmd
|
|||
import "encoding/hex"
|
||||
|
||||
// Heals the erasure coded file. reedsolomon.Reconstruct() is used to reconstruct the missing parts.
|
||||
func erasureHealFile(latestDisks []StorageAPI, outDatedDisks []StorageAPI, volume, path, healBucket, healPath string, size int64, blockSize int64, dataBlocks int, parityBlocks int, algo string) (checkSums []string, err error) {
|
||||
func erasureHealFile(latestDisks []StorageAPI, outDatedDisks []StorageAPI, volume, path, healBucket, healPath string,
|
||||
size, blockSize int64, dataBlocks, parityBlocks int, algo HashAlgo) (checkSums []string, err error) {
|
||||
|
||||
var offset int64
|
||||
remainingSize := size
|
||||
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package cmd
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"io"
|
||||
"sync"
|
||||
|
@ -111,7 +110,9 @@ func getReadDisks(orderedDisks []StorageAPI, index int, dataBlocks int) (readDis
|
|||
}
|
||||
|
||||
// parallelRead - reads chunks in parallel from the disks specified in []readDisks.
|
||||
func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []StorageAPI, enBlocks [][]byte, blockOffset int64, curChunkSize int64, bitRotVerify func(diskIndex int) bool, pool *bpool.BytePool) {
|
||||
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{}
|
||||
|
||||
|
@ -125,12 +126,16 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||
go func(index int) {
|
||||
defer wg.Done()
|
||||
|
||||
// Verify bit rot for the file on this disk.
|
||||
if !bitRotVerify(index) {
|
||||
// So that we don't read from this disk for the next block.
|
||||
// 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 {
|
||||
|
@ -140,7 +145,25 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||
}
|
||||
buf = buf[:curChunkSize]
|
||||
|
||||
_, err = readDisks[index].ReadFile(volume, path, blockOffset, buf)
|
||||
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
|
||||
|
@ -153,12 +176,16 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||
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 string, path string, offset int64, length int64, totalLength int64, blockSize int64, dataBlocks int, parityBlocks int, checkSums []string, algo string, pool *bpool.BytePool) (int64, error) {
|
||||
// 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 {
|
||||
return 0, traceError(errUnexpected)
|
||||
|
@ -169,27 +196,15 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||
return 0, traceError(errUnexpected)
|
||||
}
|
||||
|
||||
// chunkSize is the amount of data that needs to be read from each disk at a time.
|
||||
// chunkSize is the amount of data that needs to be read from
|
||||
// each disk at a time.
|
||||
chunkSize := getChunkSize(blockSize, dataBlocks)
|
||||
|
||||
// bitRotVerify verifies if the file on a particular disk doesn't have bitrot
|
||||
// by verifying the hash of the contents of the file.
|
||||
bitRotVerify := func() func(diskIndex int) bool {
|
||||
verified := make([]bool, len(disks))
|
||||
// Return closure so that we have reference to []verified and
|
||||
// not recalculate the hash on it every time the function is
|
||||
// called for the same disk.
|
||||
return func(diskIndex int) bool {
|
||||
if verified[diskIndex] {
|
||||
// Already validated.
|
||||
return true
|
||||
brVerifiers := make([]bitRotVerifier, len(disks))
|
||||
for i := range brVerifiers {
|
||||
brVerifiers[i].algo = algo
|
||||
brVerifiers[i].checkSum = checkSums[i]
|
||||
}
|
||||
// Is this a valid block?
|
||||
isValid := isValidBlock(disks[diskIndex], volume, path, checkSums[diskIndex], algo)
|
||||
verified[diskIndex] = isValid
|
||||
return isValid
|
||||
}
|
||||
}()
|
||||
|
||||
// Total bytes written to writer
|
||||
var bytesWritten int64
|
||||
|
@ -241,7 +256,7 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||
return bytesWritten, err
|
||||
}
|
||||
// Issue a parallel read across the disks specified in readDisks.
|
||||
parallelRead(volume, path, readDisks, disks, enBlocks, blockOffset, curChunkSize, bitRotVerify, pool)
|
||||
parallelRead(volume, path, readDisks, disks, enBlocks, blockOffset, curChunkSize, brVerifiers, pool)
|
||||
if isSuccessDecodeBlocks(enBlocks, dataBlocks) {
|
||||
// If enough blocks are available to do rs.Reconstruct()
|
||||
break
|
||||
|
@ -299,27 +314,6 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||
return bytesWritten, nil
|
||||
}
|
||||
|
||||
// isValidBlock - calculates the checksum hash for the block and
|
||||
// validates if its correct returns true for valid cases, false otherwise.
|
||||
func isValidBlock(disk StorageAPI, volume, path, checkSum, checkSumAlgo string) (ok bool) {
|
||||
// Disk is not available, not a valid block.
|
||||
if disk == nil {
|
||||
return false
|
||||
}
|
||||
// Checksum not available, not a valid block.
|
||||
if checkSum == "" {
|
||||
return false
|
||||
}
|
||||
// Read everything for a given block and calculate hash.
|
||||
hashWriter := newHash(checkSumAlgo)
|
||||
hashBytes, err := hashSum(disk, volume, path, hashWriter)
|
||||
if err != nil {
|
||||
errorIf(err, "Unable to calculate checksum %s/%s", volume, path)
|
||||
return false
|
||||
}
|
||||
return hex.EncodeToString(hashBytes) == checkSum
|
||||
}
|
||||
|
||||
// decodeData - decode encoded blocks.
|
||||
func decodeData(enBlocks [][]byte, dataBlocks, parityBlocks int) error {
|
||||
// Initialized reedsolomon.
|
||||
|
|
|
@ -213,6 +213,12 @@ func (r ReadDiskDown) ReadFile(volume string, path string, offset int64, buf []b
|
|||
return 0, errFaultyDisk
|
||||
}
|
||||
|
||||
func (r ReadDiskDown) ReadFileWithVerify(volume string, path string, offset int64, buf []byte,
|
||||
algo HashAlgo, expectedHash string) (n int64, err error) {
|
||||
|
||||
return 0, errFaultyDisk
|
||||
}
|
||||
|
||||
func TestErasureReadFileDiskFail(t *testing.T) {
|
||||
// Initialize environment needed for the test.
|
||||
dataBlocks := 7
|
||||
|
|
|
@ -29,7 +29,7 @@ import (
|
|||
)
|
||||
|
||||
// newHashWriters - inititialize a slice of hashes for the disk count.
|
||||
func newHashWriters(diskCount int, algo string) []hash.Hash {
|
||||
func newHashWriters(diskCount int, algo HashAlgo) []hash.Hash {
|
||||
hashWriters := make([]hash.Hash, diskCount)
|
||||
for index := range hashWriters {
|
||||
hashWriters[index] = newHash(algo)
|
||||
|
@ -38,13 +38,13 @@ func newHashWriters(diskCount int, algo string) []hash.Hash {
|
|||
}
|
||||
|
||||
// newHash - gives you a newly allocated hash depending on the input algorithm.
|
||||
func newHash(algo string) (h hash.Hash) {
|
||||
func newHash(algo HashAlgo) (h hash.Hash) {
|
||||
switch algo {
|
||||
case sha256Algo:
|
||||
case HashSha256:
|
||||
// sha256 checksum specially on ARM64 platforms or whenever
|
||||
// requested as dictated by `xl.json` entry.
|
||||
h = sha256.New()
|
||||
case blake2bAlgo:
|
||||
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)
|
||||
|
@ -71,7 +71,7 @@ var hashBufferPool = sync.Pool{
|
|||
|
||||
// hashSum calculates the hash of the entire path and returns.
|
||||
func hashSum(disk StorageAPI, volume, path string, writer hash.Hash) ([]byte, error) {
|
||||
// Fetch staging a new staging buffer from the pool.
|
||||
// Fetch a new staging buffer from the pool.
|
||||
bufp := hashBufferPool.Get().(*[]byte)
|
||||
defer hashBufferPool.Put(bufp)
|
||||
|
||||
|
@ -207,3 +207,16 @@ func copyBuffer(writer io.Writer, disk StorageAPI, volume string, path string, b
|
|||
// 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
|
||||
}
|
||||
|
|
|
@ -122,6 +122,15 @@ func (d *naughtyDisk) ReadFile(volume string, path string, offset int64, buf []b
|
|||
return d.disk.ReadFile(volume, path, offset, buf)
|
||||
}
|
||||
|
||||
func (d *naughtyDisk) ReadFileWithVerify(volume, path string, offset int64,
|
||||
buf []byte, algo HashAlgo, expectedHash string) (n int64, err error) {
|
||||
|
||||
if err := d.calcError(); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
return d.disk.ReadFileWithVerify(volume, path, offset, buf, algo, expectedHash)
|
||||
}
|
||||
|
||||
func (d *naughtyDisk) PrepareFile(volume, path string, length int64) error {
|
||||
if err := d.calcError(); err != nil {
|
||||
return err
|
||||
|
|
78
cmd/posix.go
78
cmd/posix.go
|
@ -18,6 +18,8 @@ package cmd
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"hash"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
|
@ -512,11 +514,30 @@ func (s *posix) ReadAll(volume, path string) (buf []byte, err error) {
|
|||
// number of bytes copied. The error is EOF only if no bytes were
|
||||
// read. On return, n == len(buf) if and only if err == nil. n == 0
|
||||
// for io.EOF.
|
||||
//
|
||||
// If an EOF happens after reading some but not all the bytes,
|
||||
// ReadFull returns ErrUnexpectedEOF.
|
||||
// Additionally ReadFile also starts reading from an offset.
|
||||
// ReadFile symantics are same as io.ReadFull
|
||||
func (s *posix) ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error) {
|
||||
//
|
||||
// Additionally ReadFile also starts reading from an offset. ReadFile
|
||||
// semantics are same as io.ReadFull.
|
||||
func (s *posix) ReadFile(volume, path string, offset int64, buf []byte) (n int64, err error) {
|
||||
|
||||
return s.ReadFileWithVerify(volume, path, offset, buf, "", "")
|
||||
}
|
||||
|
||||
// ReadFileWithVerify is the same as ReadFile but with hashsum
|
||||
// verification: the operation will fail if the hash verification
|
||||
// fails.
|
||||
//
|
||||
// The `expectedHash` is the expected hex-encoded hash string for
|
||||
// verification. With an empty expected hash string, hash verification
|
||||
// is skipped. An empty HashAlgo defaults to `blake2b`.
|
||||
//
|
||||
// The function takes care to minimize the number of disk read
|
||||
// operations.
|
||||
func (s *posix) ReadFileWithVerify(volume, path string, offset int64, buf []byte,
|
||||
algo HashAlgo, expectedHash string) (n int64, err error) {
|
||||
|
||||
defer func() {
|
||||
if err == syscall.EIO {
|
||||
atomic.AddInt32(&s.ioErrCount, 1)
|
||||
|
@ -571,19 +592,66 @@ func (s *posix) ReadFile(volume string, path string, offset int64, buf []byte) (
|
|||
return 0, err
|
||||
}
|
||||
|
||||
// Verify if its not a regular file, since subsequent Seek is undefined.
|
||||
// Verify it is a regular file, otherwise subsequent Seek is
|
||||
// undefined.
|
||||
if !st.Mode().IsRegular() {
|
||||
return 0, errIsNotRegular
|
||||
}
|
||||
|
||||
// If expected hash string is empty hash verification is
|
||||
// skipped.
|
||||
needToHash := expectedHash != ""
|
||||
var hasher hash.Hash
|
||||
|
||||
if needToHash {
|
||||
// If the hashing algo is invalid, return an error.
|
||||
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
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Seek to requested offset.
|
||||
_, err = file.Seek(offset, os.SEEK_SET)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
|
||||
// Read full until buffer.
|
||||
// Read until buffer is full.
|
||||
m, err := io.ReadFull(file, buf)
|
||||
if err == io.EOF {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
if needToHash {
|
||||
// Continue computing hash with buf.
|
||||
_, err = hasher.Write(buf)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
// Continue computing hash until end of file.
|
||||
_, err = io.Copy(hasher, file)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
// Verify the computed hash.
|
||||
computedHash := hex.EncodeToString(hasher.Sum(nil))
|
||||
if computedHash != expectedHash {
|
||||
return 0, hashMismatchError{expectedHash, computedHash}
|
||||
}
|
||||
}
|
||||
|
||||
// Success.
|
||||
return int64(m), err
|
||||
|
|
|
@ -18,6 +18,8 @@ package cmd
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/sha256"
|
||||
"encoding/hex"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
|
@ -26,6 +28,8 @@ import (
|
|||
"strings"
|
||||
"syscall"
|
||||
"testing"
|
||||
|
||||
"golang.org/x/crypto/blake2b"
|
||||
)
|
||||
|
||||
// creates a temp dir and sets up posix layer.
|
||||
|
@ -1017,6 +1021,115 @@ func TestPosixReadFile(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
// TestPosixReadFileWithVerify - tests the posix level
|
||||
// ReadFileWithVerify API. Only tests hashing related
|
||||
// functionality. Other functionality is tested with
|
||||
// TestPosixReadFile.
|
||||
func TestPosixReadFileWithVerify(t *testing.T) {
|
||||
// create posix test setup
|
||||
posixStorage, path, err := newPosixTestSetup()
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create posix test setup, %s", err)
|
||||
}
|
||||
defer removeAll(path)
|
||||
|
||||
volume := "success-vol"
|
||||
// Setup test environment.
|
||||
if err = posixStorage.MakeVol(volume); err != nil {
|
||||
t.Fatalf("Unable to create volume, %s", err)
|
||||
}
|
||||
|
||||
blakeHash := func(s string) string {
|
||||
k := blake2b.Sum512([]byte(s))
|
||||
return hex.EncodeToString(k[:])
|
||||
}
|
||||
|
||||
sha256Hash := func(s string) string {
|
||||
k := sha256.Sum256([]byte(s))
|
||||
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 {
|
||||
case err == nil && testCase.expectedErr != nil:
|
||||
t.Errorf("Test %d: Expected error %v but got none.", i+1, testCase.expectedErr)
|
||||
case err == nil && n != int64(testCase.bufSize):
|
||||
t.Errorf("Test %d: %d bytes were expected, but %d were written", i+1, testCase.bufSize, n)
|
||||
case err == nil && !bytes.Equal(testCase.expectedBuf, buf):
|
||||
t.Errorf("Test %d: Expected bytes: %v, but got: %v", i+1, testCase.expectedBuf, buf)
|
||||
case err != nil && err != testCase.expectedErr:
|
||||
t.Errorf("Test %d: Expected error: %v, but got: %v", i+1, testCase.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestPosix posix.AppendFile()
|
||||
func TestPosixAppendFile(t *testing.T) {
|
||||
// create posix test setup
|
||||
|
|
|
@ -178,6 +178,23 @@ func (f retryStorage) ReadFile(volume, path string, offset int64, buffer []byte)
|
|||
return m, err
|
||||
}
|
||||
|
||||
// ReadFileWithVerify - a retryable implementation of reading at
|
||||
// offset from a file with verification.
|
||||
func (f retryStorage) ReadFileWithVerify(volume, path string, offset int64, buffer []byte,
|
||||
algo HashAlgo, expectedHash string) (m int64, err error) {
|
||||
|
||||
m, err = f.remoteStorage.ReadFileWithVerify(volume, path, offset, buffer,
|
||||
algo, expectedHash)
|
||||
if err == errDiskNotFound {
|
||||
err = f.reInit()
|
||||
if err == nil {
|
||||
return f.remoteStorage.ReadFileWithVerify(volume, path,
|
||||
offset, buffer, algo, expectedHash)
|
||||
}
|
||||
}
|
||||
return m, err
|
||||
}
|
||||
|
||||
// ListDir - a retryable implementation of listing directory entries.
|
||||
func (f retryStorage) ListDir(volume, path string) (entries []string, err error) {
|
||||
entries, err = f.remoteStorage.ListDir(volume, path)
|
||||
|
|
|
@ -18,6 +18,8 @@ package cmd
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/sha256"
|
||||
"encoding/hex"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
@ -290,6 +292,31 @@ func TestRetryStorage(t *testing.T) {
|
|||
if n, err = disk.ReadFile("existent", "path", 7, buf2); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err != nil {
|
||||
t.Error("Error in ReadFile", err)
|
||||
}
|
||||
if n != 5 {
|
||||
t.Fatalf("Expected 5, got %d", n)
|
||||
}
|
||||
if !bytes.Equal(buf2, []byte("World")) {
|
||||
t.Fatalf("Expected `World`, got %s", string(buf2))
|
||||
}
|
||||
}
|
||||
|
||||
sha256Hash := func(s string) string {
|
||||
k := sha256.Sum256([]byte(s))
|
||||
return hex.EncodeToString(k[:])
|
||||
}
|
||||
for _, disk := range storageDisks {
|
||||
var buf2 = make([]byte, 5)
|
||||
var n int64
|
||||
if n, err = disk.ReadFileWithVerify("existent", "path", 7, buf2,
|
||||
HashSha256, sha256Hash("Hello, World")); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err != nil {
|
||||
t.Error("Error in ReadFileWithVerify", err)
|
||||
}
|
||||
if n != 5 {
|
||||
t.Fatalf("Expected 5, got %d", n)
|
||||
}
|
||||
|
|
|
@ -16,7 +16,10 @@
|
|||
|
||||
package cmd
|
||||
|
||||
import "errors"
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// errUnexpected - unexpected error, requires manual intervention.
|
||||
var errUnexpected = errors.New("Unexpected error, please report this issue at https://github.com/minio/minio/issues")
|
||||
|
@ -68,3 +71,21 @@ var errVolumeAccessDenied = errors.New("volume access denied")
|
|||
|
||||
// errVolumeAccessDenied - cannot access file, insufficient permissions.
|
||||
var errFileAccessDenied = errors.New("file access denied")
|
||||
|
||||
// errBitrotHashAlgoInvalid - the algo for bit-rot hash
|
||||
// verification is empty or invalid.
|
||||
var errBitrotHashAlgoInvalid = errors.New("bit-rot hash algorithm is invalid")
|
||||
|
||||
// hashMisMatchError - represents a bit-rot hash verification failure
|
||||
// error.
|
||||
type hashMismatchError struct {
|
||||
expected string
|
||||
computed string
|
||||
}
|
||||
|
||||
// error method for the hashMismatchError
|
||||
func (h hashMismatchError) Error() string {
|
||||
return fmt.Sprintf(
|
||||
"Bitrot verification mismatch - expected %v, received %v",
|
||||
h.expected, h.computed)
|
||||
}
|
||||
|
|
|
@ -37,6 +37,8 @@ type StorageAPI interface {
|
|||
// File operations.
|
||||
ListDir(volume, dirPath string) ([]string, error)
|
||||
ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error)
|
||||
ReadFileWithVerify(volume string, path string, offset int64, buf []byte,
|
||||
algo HashAlgo, expectedHash string) (n int64, err error)
|
||||
PrepareFile(volume string, path string, len int64) (err error)
|
||||
AppendFile(volume string, path string, buf []byte) (err error)
|
||||
RenameFile(srcVolume, srcPath, dstVolume, dstPath string) error
|
||||
|
|
|
@ -262,6 +262,35 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
|
|||
return int64(len(result)), toStorageErr(err)
|
||||
}
|
||||
|
||||
// ReadFileWithVerify - reads a file at remote path and fills the buffer.
|
||||
func (n *networkStorage) ReadFileWithVerify(volume string, path string, offset int64,
|
||||
buffer []byte, algo HashAlgo, expectedHash string) (m int64, err error) {
|
||||
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
// Recover any panic from allocation, and return error.
|
||||
err = bytes.ErrTooLarge
|
||||
}
|
||||
}() // Do not crash the server.
|
||||
|
||||
var result []byte
|
||||
err = n.rpcClient.Call("Storage.ReadFileWithVerifyHandler",
|
||||
&ReadFileWithVerifyArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
Offset: offset,
|
||||
Buffer: buffer,
|
||||
Algo: algo,
|
||||
ExpectedHash: expectedHash,
|
||||
}, &result)
|
||||
|
||||
// Copy results to buffer.
|
||||
copy(buffer, result)
|
||||
|
||||
// Return length of result, err if any.
|
||||
return int64(len(result)), toStorageErr(err)
|
||||
}
|
||||
|
||||
// ListDir - list all entries at prefix.
|
||||
func (n *networkStorage) ListDir(volume, path string) (entries []string, err error) {
|
||||
if err = n.rpcClient.Call("Storage.ListDirHandler", &ListDirArgs{
|
||||
|
|
|
@ -18,6 +18,7 @@ package cmd
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
|
@ -25,6 +26,8 @@ import (
|
|||
"net/rpc"
|
||||
"runtime"
|
||||
"testing"
|
||||
|
||||
"golang.org/x/crypto/blake2b"
|
||||
)
|
||||
|
||||
// Tests the construction of canonical string by the
|
||||
|
@ -387,6 +390,24 @@ func (s *TestRPCStorageSuite) testRPCStorageFileOps(t *testing.T) {
|
|||
if !bytes.Equal(buf[4:9], buf1) {
|
||||
t.Errorf("Expected %s, got %s", string(buf[4:9]), string(buf1))
|
||||
}
|
||||
|
||||
blakeHash := func(s string) string {
|
||||
k := blake2b.Sum512([]byte(s))
|
||||
return hex.EncodeToString(k[:])
|
||||
}
|
||||
buf2 := make([]byte, 2)
|
||||
n, err = storageDisk.ReadFileWithVerify("myvol", "file1", 1,
|
||||
buf2, HashBlake2b, blakeHash(string(buf)))
|
||||
if err != nil {
|
||||
t.Error("Error in ReadFileWithVerify", err)
|
||||
}
|
||||
if n != 2 {
|
||||
t.Errorf("Expected `2`, got %d", n)
|
||||
}
|
||||
if !bytes.Equal(buf[1:3], buf2) {
|
||||
t.Errorf("Expected %s, got %s", string(buf[1:3]), string(buf2))
|
||||
}
|
||||
|
||||
err = storageDisk.RenameFile("myvol", "file1", "myvol", "file2")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate RenameFile", err)
|
||||
|
|
|
@ -61,6 +61,31 @@ type ReadFileArgs struct {
|
|||
Buffer []byte
|
||||
}
|
||||
|
||||
// ReadFileWithVerifyArgs represents read file RPC arguments.
|
||||
type ReadFileWithVerifyArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
|
||||
// Starting offset to start reading into Buffer.
|
||||
Offset int64
|
||||
|
||||
// Data buffer read from the path at offset.
|
||||
Buffer []byte
|
||||
|
||||
// Algorithm used in bit-rot hash computation.
|
||||
Algo HashAlgo
|
||||
|
||||
// Stored hash value (hex-encoded) used to compare with
|
||||
// computed value.
|
||||
ExpectedHash string
|
||||
}
|
||||
|
||||
// PrepareFileArgs represents append file RPC arguments.
|
||||
type PrepareFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
|
|
|
@ -160,6 +160,26 @@ func (s *storageServer) ReadFileHandler(args *ReadFileArgs, reply *[]byte) (err
|
|||
return err
|
||||
}
|
||||
|
||||
// ReadFileWithVerifyHandler - read file with verify handler is rpc wrapper to read file with verify.
|
||||
func (s *storageServer) ReadFileWithVerifyHandler(args *ReadFileWithVerifyArgs, reply *[]byte) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
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
|
||||
// when we have short read i.e `io.ErrUnexpectedEOF` treat it as good condition and copy
|
||||
// the buffer properly.
|
||||
if err == io.ErrUnexpectedEOF {
|
||||
// Reset to nil as good condition.
|
||||
err = nil
|
||||
}
|
||||
*reply = args.Buffer[0:n]
|
||||
return err
|
||||
}
|
||||
|
||||
// PrepareFileHandler - prepare file handler is rpc wrapper to prepare file.
|
||||
func (s *storageServer) PrepareFileHandler(args *PrepareFileArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
|
|
|
@ -50,18 +50,32 @@ func (t byObjectPartNumber) Less(i, j int) bool { return t[i].Number < t[j].Numb
|
|||
// checkSumInfo - carries checksums of individual scattered parts per disk.
|
||||
type checkSumInfo struct {
|
||||
Name string `json:"name"`
|
||||
Algorithm string `json:"algorithm"`
|
||||
Algorithm HashAlgo `json:"algorithm"`
|
||||
Hash string `json:"hash"`
|
||||
}
|
||||
|
||||
// Various algorithms supported by bit-rot protection feature.
|
||||
// HashAlgo - represents a supported hashing algorithm for bitrot
|
||||
// verification.
|
||||
type HashAlgo string
|
||||
|
||||
const (
|
||||
// "sha256" is specifically used on arm64 bit platforms.
|
||||
sha256Algo = "sha256"
|
||||
// Rest of the platforms default to blake2b.
|
||||
blake2bAlgo = "blake2b"
|
||||
// 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
|
||||
}
|
||||
}
|
||||
|
||||
// Constant indicates current bit-rot algo used when creating objects.
|
||||
// Depending on the architecture we are choosing a different checksum.
|
||||
var bitRotAlgo = getDefaultBitRotAlgo()
|
||||
|
@ -70,7 +84,7 @@ var bitRotAlgo = getDefaultBitRotAlgo()
|
|||
// Currently this function defaults to "blake2b" as the preferred
|
||||
// checksum algorithm on all architectures except ARM64. On ARM64
|
||||
// we use sha256 (optimized using sha2 instructions of ARM NEON chip).
|
||||
func getDefaultBitRotAlgo() string {
|
||||
func getDefaultBitRotAlgo() HashAlgo {
|
||||
switch runtime.GOARCH {
|
||||
case "arm64":
|
||||
// As a special case for ARM64 we use an optimized
|
||||
|
@ -79,17 +93,17 @@ func getDefaultBitRotAlgo() string {
|
|||
// This would also allows erasure coded writes
|
||||
// on ARM64 servers to be on-par with their
|
||||
// counter-part X86_64 servers.
|
||||
return sha256Algo
|
||||
return HashSha256
|
||||
default:
|
||||
// Default for all other architectures we use blake2b.
|
||||
return blake2bAlgo
|
||||
return HashBlake2b
|
||||
}
|
||||
}
|
||||
|
||||
// erasureInfo - carries erasure coding related information, block
|
||||
// distribution and checksums.
|
||||
type erasureInfo struct {
|
||||
Algorithm string `json:"algorithm"`
|
||||
Algorithm HashAlgo `json:"algorithm"`
|
||||
DataBlocks int `json:"data"`
|
||||
ParityBlocks int `json:"parity"`
|
||||
BlockSize int64 `json:"blockSize"`
|
||||
|
|
|
@ -266,7 +266,7 @@ func (xl xlObjects) GetObject(bucket, object string, startOffset int64, length i
|
|||
|
||||
// Get the checksums of the current part.
|
||||
checkSums := make([]string, len(onlineDisks))
|
||||
var ckSumAlgo string
|
||||
var ckSumAlgo HashAlgo
|
||||
for index, disk := range onlineDisks {
|
||||
// Disk is not found skip the checksum.
|
||||
if disk == nil {
|
||||
|
|
|
@ -161,7 +161,7 @@ func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo {
|
|||
}
|
||||
erasure.Distribution = distribution
|
||||
|
||||
erasure.Algorithm = erasureResult.Get("algorithm").String()
|
||||
erasure.Algorithm = HashAlgo(erasureResult.Get("algorithm").String())
|
||||
erasure.DataBlocks = int(erasureResult.Get("data").Int())
|
||||
erasure.ParityBlocks = int(erasureResult.Get("parity").Int())
|
||||
erasure.BlockSize = erasureResult.Get("blockSize").Int()
|
||||
|
@ -172,7 +172,7 @@ func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo {
|
|||
for i, checkSumResult := range checkSumsResult {
|
||||
checkSum := checkSumInfo{}
|
||||
checkSum.Name = checkSumResult.Get("name").String()
|
||||
checkSum.Algorithm = checkSumResult.Get("algorithm").String()
|
||||
checkSum.Algorithm = HashAlgo(checkSumResult.Get("algorithm").String())
|
||||
checkSum.Hash = checkSumResult.Get("hash").String()
|
||||
checkSums[i] = checkSum
|
||||
}
|
||||
|
|
|
@ -157,7 +157,7 @@ func newTestXLMetaV1() xlMetaV1 {
|
|||
return xlMeta
|
||||
}
|
||||
|
||||
func (m *xlMetaV1) AddTestObjectCheckSum(checkSumNum int, name string, hash string, algo string) {
|
||||
func (m *xlMetaV1) AddTestObjectCheckSum(checkSumNum int, name string, hash string, algo HashAlgo) {
|
||||
checkSum := checkSumInfo{
|
||||
Name: name,
|
||||
Algorithm: algo,
|
||||
|
|
Loading…
Reference in New Issue