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:
Aditya Manthramurthy 2017-05-16 14:21:52 -07:00 committed by Harshavardhana
parent cae4683971
commit 8975da4e84
20 changed files with 471 additions and 88 deletions

View file

@ -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)

View file

@ -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

View file

@ -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,11 +126,15 @@ 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.
orderedDisks[index] = nil
return
// 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()
@ -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
}
// Is this a valid block?
isValid := isValidBlock(disks[diskIndex], volume, path, checkSums[diskIndex], algo)
verified[diskIndex] = isValid
return isValid
}
}()
brVerifiers := make([]bitRotVerifier, len(disks))
for i := range brVerifiers {
brVerifiers[i].algo = algo
brVerifiers[i].checkSum = checkSums[i]
}
// 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.

View file

@ -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

View file

@ -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
}

View file

@ -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

View file

@ -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
}
// Seek to requested offset.
_, err = file.Seek(offset, os.SEEK_SET)
if err != nil {
// 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 until buffer is full.
m, err := io.ReadFull(file, buf)
if err == io.EOF {
return 0, err
}
// Read full until buffer.
m, err := io.ReadFull(file, buf)
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

View file

@ -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

View file

@ -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)

View file

@ -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)
}

View file

@ -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)
}

View file

@ -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

View file

@ -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{

View file

@ -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)

View file

@ -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.

View file

@ -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 {

View file

@ -49,19 +49,33 @@ 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"`
Hash string `json:"hash"`
Name string `json:"name"`
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"`

View file

@ -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 {

View file

@ -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
}

View file

@ -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,