mirror of
https://github.com/minio/minio.git
synced 2025-04-22 03:24:38 -04:00
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
|
// erasureCreateFile - writes an entire stream by erasure coding to
|
||||||
// all the disks, writes also calculate individual block's checksum
|
// all the disks, writes also calculate individual block's checksum
|
||||||
// for future bit-rot protection.
|
// 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.
|
// Allocated blockSized buffer for reading from incoming stream.
|
||||||
buf := make([]byte, blockSize)
|
buf := make([]byte, blockSize)
|
||||||
|
|
||||||
|
@ -19,7 +19,9 @@ package cmd
|
|||||||
import "encoding/hex"
|
import "encoding/hex"
|
||||||
|
|
||||||
// Heals the erasure coded file. reedsolomon.Reconstruct() is used to reconstruct the missing parts.
|
// 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
|
var offset int64
|
||||||
remainingSize := size
|
remainingSize := size
|
||||||
|
|
||||||
|
@ -17,7 +17,6 @@
|
|||||||
package cmd
|
package cmd
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"encoding/hex"
|
|
||||||
"errors"
|
"errors"
|
||||||
"io"
|
"io"
|
||||||
"sync"
|
"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.
|
// 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.
|
// WaitGroup to synchronise the read go-routines.
|
||||||
wg := &sync.WaitGroup{}
|
wg := &sync.WaitGroup{}
|
||||||
|
|
||||||
@ -125,11 +126,15 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||||||
go func(index int) {
|
go func(index int) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
|
|
||||||
// Verify bit rot for the file on this disk.
|
// evaluate if we need to perform bit-rot checking
|
||||||
if !bitRotVerify(index) {
|
needBitRotVerification := true
|
||||||
// So that we don't read from this disk for the next block.
|
if brVerifiers[index].isVerified {
|
||||||
orderedDisks[index] = nil
|
needBitRotVerification = false
|
||||||
return
|
// if file has bit-rot, do not reuse disk
|
||||||
|
if brVerifiers[index].hasBitRot {
|
||||||
|
orderedDisks[index] = nil
|
||||||
|
return
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
buf, err := pool.Get()
|
buf, err := pool.Get()
|
||||||
@ -140,7 +145,25 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||||||
}
|
}
|
||||||
buf = buf[:curChunkSize]
|
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 {
|
if err != nil {
|
||||||
orderedDisks[index] = nil
|
orderedDisks[index] = nil
|
||||||
return
|
return
|
||||||
@ -153,12 +176,16 @@ func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []St
|
|||||||
wg.Wait()
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
// erasureReadFile - read bytes from erasure coded files and writes to given writer.
|
// erasureReadFile - read bytes from erasure coded files and writes to
|
||||||
// Erasure coded files are read block by block as per given erasureInfo and data chunks
|
// given writer. Erasure coded files are read block by block as per
|
||||||
// are decoded into a data block. Data block is trimmed for given offset and length,
|
// given erasureInfo and data chunks are decoded into a data
|
||||||
// then written to given writer. This function also supports bit-rot detection by
|
// block. Data block is trimmed for given offset and length, then
|
||||||
// verifying checksum of individual block's checksum.
|
// written to given writer. This function also supports bit-rot
|
||||||
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) {
|
// 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.
|
// Offset and length cannot be negative.
|
||||||
if offset < 0 || length < 0 {
|
if offset < 0 || length < 0 {
|
||||||
return 0, traceError(errUnexpected)
|
return 0, traceError(errUnexpected)
|
||||||
@ -169,27 +196,15 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||||||
return 0, traceError(errUnexpected)
|
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)
|
chunkSize := getChunkSize(blockSize, dataBlocks)
|
||||||
|
|
||||||
// bitRotVerify verifies if the file on a particular disk doesn't have bitrot
|
brVerifiers := make([]bitRotVerifier, len(disks))
|
||||||
// by verifying the hash of the contents of the file.
|
for i := range brVerifiers {
|
||||||
bitRotVerify := func() func(diskIndex int) bool {
|
brVerifiers[i].algo = algo
|
||||||
verified := make([]bool, len(disks))
|
brVerifiers[i].checkSum = checkSums[i]
|
||||||
// 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
|
|
||||||
}
|
|
||||||
}()
|
|
||||||
|
|
||||||
// Total bytes written to writer
|
// Total bytes written to writer
|
||||||
var bytesWritten int64
|
var bytesWritten int64
|
||||||
@ -241,7 +256,7 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||||||
return bytesWritten, err
|
return bytesWritten, err
|
||||||
}
|
}
|
||||||
// Issue a parallel read across the disks specified in readDisks.
|
// 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 isSuccessDecodeBlocks(enBlocks, dataBlocks) {
|
||||||
// If enough blocks are available to do rs.Reconstruct()
|
// If enough blocks are available to do rs.Reconstruct()
|
||||||
break
|
break
|
||||||
@ -299,27 +314,6 @@ func erasureReadFile(writer io.Writer, disks []StorageAPI, volume string, path s
|
|||||||
return bytesWritten, nil
|
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.
|
// decodeData - decode encoded blocks.
|
||||||
func decodeData(enBlocks [][]byte, dataBlocks, parityBlocks int) error {
|
func decodeData(enBlocks [][]byte, dataBlocks, parityBlocks int) error {
|
||||||
// Initialized reedsolomon.
|
// Initialized reedsolomon.
|
||||||
|
@ -213,6 +213,12 @@ func (r ReadDiskDown) ReadFile(volume string, path string, offset int64, buf []b
|
|||||||
return 0, errFaultyDisk
|
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) {
|
func TestErasureReadFileDiskFail(t *testing.T) {
|
||||||
// Initialize environment needed for the test.
|
// Initialize environment needed for the test.
|
||||||
dataBlocks := 7
|
dataBlocks := 7
|
||||||
|
@ -29,7 +29,7 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
// newHashWriters - inititialize a slice of hashes for the disk count.
|
// 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)
|
hashWriters := make([]hash.Hash, diskCount)
|
||||||
for index := range hashWriters {
|
for index := range hashWriters {
|
||||||
hashWriters[index] = newHash(algo)
|
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.
|
// 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 {
|
switch algo {
|
||||||
case sha256Algo:
|
case HashSha256:
|
||||||
// sha256 checksum specially on ARM64 platforms or whenever
|
// sha256 checksum specially on ARM64 platforms or whenever
|
||||||
// requested as dictated by `xl.json` entry.
|
// requested as dictated by `xl.json` entry.
|
||||||
h = sha256.New()
|
h = sha256.New()
|
||||||
case blake2bAlgo:
|
case HashBlake2b:
|
||||||
// ignore the error, because New512 without a key never fails
|
// ignore the error, because New512 without a key never fails
|
||||||
// New512 only returns a non-nil error, if the length of the passed
|
// 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)
|
// 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.
|
// hashSum calculates the hash of the entire path and returns.
|
||||||
func hashSum(disk StorageAPI, volume, path string, writer hash.Hash) ([]byte, error) {
|
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)
|
bufp := hashBufferPool.Get().(*[]byte)
|
||||||
defer hashBufferPool.Put(bufp)
|
defer hashBufferPool.Put(bufp)
|
||||||
|
|
||||||
@ -207,3 +207,16 @@ func copyBuffer(writer io.Writer, disk StorageAPI, volume string, path string, b
|
|||||||
// Success.
|
// Success.
|
||||||
return nil
|
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)
|
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 {
|
func (d *naughtyDisk) PrepareFile(volume, path string, length int64) error {
|
||||||
if err := d.calcError(); err != nil {
|
if err := d.calcError(); err != nil {
|
||||||
return err
|
return err
|
||||||
|
86
cmd/posix.go
86
cmd/posix.go
@ -18,6 +18,8 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"encoding/hex"
|
||||||
|
"hash"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"os"
|
"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
|
// 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
|
// read. On return, n == len(buf) if and only if err == nil. n == 0
|
||||||
// for io.EOF.
|
// for io.EOF.
|
||||||
|
//
|
||||||
// If an EOF happens after reading some but not all the bytes,
|
// If an EOF happens after reading some but not all the bytes,
|
||||||
// ReadFull returns ErrUnexpectedEOF.
|
// ReadFull returns ErrUnexpectedEOF.
|
||||||
// Additionally ReadFile also starts reading from an offset.
|
//
|
||||||
// ReadFile symantics are same as io.ReadFull
|
// Additionally ReadFile also starts reading from an offset. ReadFile
|
||||||
func (s *posix) ReadFile(volume string, path string, offset int64, buf []byte) (n int64, err error) {
|
// 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() {
|
defer func() {
|
||||||
if err == syscall.EIO {
|
if err == syscall.EIO {
|
||||||
atomic.AddInt32(&s.ioErrCount, 1)
|
atomic.AddInt32(&s.ioErrCount, 1)
|
||||||
@ -571,19 +592,66 @@ func (s *posix) ReadFile(volume string, path string, offset int64, buf []byte) (
|
|||||||
return 0, err
|
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() {
|
if !st.Mode().IsRegular() {
|
||||||
return 0, errIsNotRegular
|
return 0, errIsNotRegular
|
||||||
}
|
}
|
||||||
|
|
||||||
// Seek to requested offset.
|
// If expected hash string is empty hash verification is
|
||||||
_, err = file.Seek(offset, os.SEEK_SET)
|
// skipped.
|
||||||
if err != nil {
|
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
|
return 0, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Read full until buffer.
|
if needToHash {
|
||||||
m, err := io.ReadFull(file, buf)
|
// 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.
|
// Success.
|
||||||
return int64(m), err
|
return int64(m), err
|
||||||
|
@ -18,6 +18,8 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"crypto/sha256"
|
||||||
|
"encoding/hex"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"os"
|
"os"
|
||||||
@ -26,6 +28,8 @@ import (
|
|||||||
"strings"
|
"strings"
|
||||||
"syscall"
|
"syscall"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"golang.org/x/crypto/blake2b"
|
||||||
)
|
)
|
||||||
|
|
||||||
// creates a temp dir and sets up posix layer.
|
// 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()
|
// TestPosix posix.AppendFile()
|
||||||
func TestPosixAppendFile(t *testing.T) {
|
func TestPosixAppendFile(t *testing.T) {
|
||||||
// create posix test setup
|
// create posix test setup
|
||||||
|
@ -178,6 +178,23 @@ func (f retryStorage) ReadFile(volume, path string, offset int64, buffer []byte)
|
|||||||
return m, err
|
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.
|
// ListDir - a retryable implementation of listing directory entries.
|
||||||
func (f retryStorage) ListDir(volume, path string) (entries []string, err error) {
|
func (f retryStorage) ListDir(volume, path string) (entries []string, err error) {
|
||||||
entries, err = f.remoteStorage.ListDir(volume, path)
|
entries, err = f.remoteStorage.ListDir(volume, path)
|
||||||
|
@ -18,6 +18,8 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"crypto/sha256"
|
||||||
|
"encoding/hex"
|
||||||
"reflect"
|
"reflect"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
@ -290,6 +292,31 @@ func TestRetryStorage(t *testing.T) {
|
|||||||
if n, err = disk.ReadFile("existent", "path", 7, buf2); err != nil {
|
if n, err = disk.ReadFile("existent", "path", 7, buf2); err != nil {
|
||||||
t.Fatal(err)
|
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 {
|
if n != 5 {
|
||||||
t.Fatalf("Expected 5, got %d", n)
|
t.Fatalf("Expected 5, got %d", n)
|
||||||
}
|
}
|
||||||
|
@ -16,7 +16,10 @@
|
|||||||
|
|
||||||
package cmd
|
package cmd
|
||||||
|
|
||||||
import "errors"
|
import (
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
)
|
||||||
|
|
||||||
// errUnexpected - unexpected error, requires manual intervention.
|
// errUnexpected - unexpected error, requires manual intervention.
|
||||||
var errUnexpected = errors.New("Unexpected error, please report this issue at https://github.com/minio/minio/issues")
|
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.
|
// errVolumeAccessDenied - cannot access file, insufficient permissions.
|
||||||
var errFileAccessDenied = errors.New("file access denied")
|
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.
|
// 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,
|
||||||
|
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
|
||||||
|
@ -262,6 +262,35 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
|
|||||||
return int64(len(result)), toStorageErr(err)
|
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.
|
// ListDir - list all entries at prefix.
|
||||||
func (n *networkStorage) ListDir(volume, path string) (entries []string, err error) {
|
func (n *networkStorage) ListDir(volume, path string) (entries []string, err error) {
|
||||||
if err = n.rpcClient.Call("Storage.ListDirHandler", &ListDirArgs{
|
if err = n.rpcClient.Call("Storage.ListDirHandler", &ListDirArgs{
|
||||||
|
@ -18,6 +18,7 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"encoding/hex"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
@ -25,6 +26,8 @@ import (
|
|||||||
"net/rpc"
|
"net/rpc"
|
||||||
"runtime"
|
"runtime"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"golang.org/x/crypto/blake2b"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Tests the construction of canonical string by the
|
// 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) {
|
if !bytes.Equal(buf[4:9], buf1) {
|
||||||
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 {
|
||||||
|
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")
|
err = storageDisk.RenameFile("myvol", "file1", "myvol", "file2")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Error("Unable to initiate RenameFile", err)
|
t.Error("Unable to initiate RenameFile", err)
|
||||||
|
@ -61,6 +61,31 @@ type ReadFileArgs struct {
|
|||||||
Buffer []byte
|
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.
|
// PrepareFileArgs represents append file RPC arguments.
|
||||||
type PrepareFileArgs struct {
|
type PrepareFileArgs struct {
|
||||||
// Authentication token generated by Login.
|
// Authentication token generated by Login.
|
||||||
|
@ -160,6 +160,26 @@ func (s *storageServer) ReadFileHandler(args *ReadFileArgs, reply *[]byte) (err
|
|||||||
return 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.
|
// PrepareFileHandler - prepare file handler is rpc wrapper to prepare file.
|
||||||
func (s *storageServer) PrepareFileHandler(args *PrepareFileArgs, reply *AuthRPCReply) error {
|
func (s *storageServer) PrepareFileHandler(args *PrepareFileArgs, reply *AuthRPCReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
if err := args.IsAuthenticated(); err != nil {
|
||||||
|
@ -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.
|
// checkSumInfo - carries checksums of individual scattered parts per disk.
|
||||||
type checkSumInfo struct {
|
type checkSumInfo struct {
|
||||||
Name string `json:"name"`
|
Name string `json:"name"`
|
||||||
Algorithm string `json:"algorithm"`
|
Algorithm HashAlgo `json:"algorithm"`
|
||||||
Hash string `json:"hash"`
|
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 (
|
const (
|
||||||
// "sha256" is specifically used on arm64 bit platforms.
|
// HashBlake2b represents the Blake 2b hashing algorithm
|
||||||
sha256Algo = "sha256"
|
HashBlake2b HashAlgo = "blake2b"
|
||||||
// Rest of the platforms default to blake2b.
|
// HashSha256 represents the SHA256 hashing algorithm
|
||||||
blake2bAlgo = "blake2b"
|
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.
|
// Constant indicates current bit-rot algo used when creating objects.
|
||||||
// Depending on the architecture we are choosing a different checksum.
|
// Depending on the architecture we are choosing a different checksum.
|
||||||
var bitRotAlgo = getDefaultBitRotAlgo()
|
var bitRotAlgo = getDefaultBitRotAlgo()
|
||||||
@ -70,7 +84,7 @@ var bitRotAlgo = getDefaultBitRotAlgo()
|
|||||||
// Currently this function defaults to "blake2b" as the preferred
|
// Currently this function defaults to "blake2b" as the preferred
|
||||||
// checksum algorithm on all architectures except ARM64. On ARM64
|
// checksum algorithm on all architectures except ARM64. On ARM64
|
||||||
// we use sha256 (optimized using sha2 instructions of ARM NEON chip).
|
// we use sha256 (optimized using sha2 instructions of ARM NEON chip).
|
||||||
func getDefaultBitRotAlgo() string {
|
func getDefaultBitRotAlgo() HashAlgo {
|
||||||
switch runtime.GOARCH {
|
switch runtime.GOARCH {
|
||||||
case "arm64":
|
case "arm64":
|
||||||
// As a special case for ARM64 we use an optimized
|
// As a special case for ARM64 we use an optimized
|
||||||
@ -79,17 +93,17 @@ func getDefaultBitRotAlgo() string {
|
|||||||
// This would also allows erasure coded writes
|
// This would also allows erasure coded writes
|
||||||
// on ARM64 servers to be on-par with their
|
// on ARM64 servers to be on-par with their
|
||||||
// counter-part X86_64 servers.
|
// counter-part X86_64 servers.
|
||||||
return sha256Algo
|
return HashSha256
|
||||||
default:
|
default:
|
||||||
// Default for all other architectures we use blake2b.
|
// Default for all other architectures we use blake2b.
|
||||||
return blake2bAlgo
|
return HashBlake2b
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// erasureInfo - carries erasure coding related information, block
|
// erasureInfo - carries erasure coding related information, block
|
||||||
// distribution and checksums.
|
// distribution and checksums.
|
||||||
type erasureInfo struct {
|
type erasureInfo struct {
|
||||||
Algorithm string `json:"algorithm"`
|
Algorithm HashAlgo `json:"algorithm"`
|
||||||
DataBlocks int `json:"data"`
|
DataBlocks int `json:"data"`
|
||||||
ParityBlocks int `json:"parity"`
|
ParityBlocks int `json:"parity"`
|
||||||
BlockSize int64 `json:"blockSize"`
|
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.
|
// Get the checksums of the current part.
|
||||||
checkSums := make([]string, len(onlineDisks))
|
checkSums := make([]string, len(onlineDisks))
|
||||||
var ckSumAlgo string
|
var ckSumAlgo HashAlgo
|
||||||
for index, disk := range onlineDisks {
|
for index, disk := range onlineDisks {
|
||||||
// Disk is not found skip the checksum.
|
// Disk is not found skip the checksum.
|
||||||
if disk == nil {
|
if disk == nil {
|
||||||
|
@ -161,7 +161,7 @@ func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo {
|
|||||||
}
|
}
|
||||||
erasure.Distribution = distribution
|
erasure.Distribution = distribution
|
||||||
|
|
||||||
erasure.Algorithm = erasureResult.Get("algorithm").String()
|
erasure.Algorithm = HashAlgo(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()
|
||||||
@ -172,7 +172,7 @@ func parseXLErasureInfo(xlMetaBuf []byte) erasureInfo {
|
|||||||
for i, checkSumResult := range checkSumsResult {
|
for i, checkSumResult := range checkSumsResult {
|
||||||
checkSum := checkSumInfo{}
|
checkSum := checkSumInfo{}
|
||||||
checkSum.Name = checkSumResult.Get("name").String()
|
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()
|
checkSum.Hash = checkSumResult.Get("hash").String()
|
||||||
checkSums[i] = checkSum
|
checkSums[i] = checkSum
|
||||||
}
|
}
|
||||||
|
@ -157,7 +157,7 @@ func newTestXLMetaV1() xlMetaV1 {
|
|||||||
return xlMeta
|
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{
|
checkSum := checkSumInfo{
|
||||||
Name: name,
|
Name: name,
|
||||||
Algorithm: algo,
|
Algorithm: algo,
|
||||||
|
Loading…
x
Reference in New Issue
Block a user