| 
									
										
										
										
											2016-03-28 12:52:38 +08:00
										 |  |  | /* | 
					
						
							|  |  |  |  * Minio Cloud Storage, (C) 2016 Minio, Inc. | 
					
						
							|  |  |  |  * | 
					
						
							|  |  |  |  * Licensed under the Apache License, Version 2.0 (the "License"); | 
					
						
							|  |  |  |  * you may not use this file except in compliance with the License. | 
					
						
							|  |  |  |  * You may obtain a copy of the License at | 
					
						
							|  |  |  |  * | 
					
						
							|  |  |  |  *     http://www.apache.org/licenses/LICENSE-2.0
 | 
					
						
							|  |  |  |  * | 
					
						
							|  |  |  |  * Unless required by applicable law or agreed to in writing, software | 
					
						
							|  |  |  |  * distributed under the License is distributed on an "AS IS" BASIS, | 
					
						
							|  |  |  |  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | 
					
						
							|  |  |  |  * See the License for the specific language governing permissions and | 
					
						
							|  |  |  |  * limitations under the License. | 
					
						
							|  |  |  |  */ | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-08-19 07:23:42 +08:00
										 |  |  | package cmd | 
					
						
							| 
									
										
										
										
											2016-03-28 12:52:38 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | import ( | 
					
						
							|  |  |  | 	"encoding/hex" | 
					
						
							|  |  |  | 	"errors" | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 	"io" | 
					
						
							| 
									
										
										
										
											2016-06-22 05:34:11 +08:00
										 |  |  | 	"sync" | 
					
						
							| 
									
										
										
										
											2016-06-02 07:43:31 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	"github.com/klauspost/reedsolomon" | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 	"github.com/minio/minio/pkg/bpool" | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | ) | 
					
						
							| 
									
										
										
										
											2016-03-28 12:52:38 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | // isSuccessDecodeBlocks - do we have all the blocks to be
 | 
					
						
							|  |  |  | // successfully decoded?. Input encoded blocks ordered matrix.
 | 
					
						
							|  |  |  | func isSuccessDecodeBlocks(enBlocks [][]byte, dataBlocks int) bool { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	// Count number of data and parity blocks that were read.
 | 
					
						
							|  |  |  | 	var successDataBlocksCount = 0 | 
					
						
							|  |  |  | 	var successParityBlocksCount = 0 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	for index := range enBlocks { | 
					
						
							|  |  |  | 		if enBlocks[index] == nil { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		// block index lesser than data blocks, update data block count.
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		if index < dataBlocks { | 
					
						
							|  |  |  | 			successDataBlocksCount++ | 
					
						
							|  |  |  | 			continue | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		} // else { // update parity block count.
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		successParityBlocksCount++ | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-08-15 16:25:41 +08:00
										 |  |  | 	// Returns true if we have atleast dataBlocks parity.
 | 
					
						
							|  |  |  | 	return successDataBlocksCount == dataBlocks || successDataBlocksCount+successParityBlocksCount >= dataBlocks | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // isSuccessDataBlocks - do we have all the data blocks?
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | // Input encoded blocks ordered matrix.
 | 
					
						
							|  |  |  | func isSuccessDataBlocks(enBlocks [][]byte, dataBlocks int) bool { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	// Count number of data blocks that were read.
 | 
					
						
							|  |  |  | 	var successDataBlocksCount = 0 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	for index := range enBlocks[:dataBlocks] { | 
					
						
							|  |  |  | 		if enBlocks[index] == nil { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		// block index lesser than data blocks, update data block count.
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		if index < dataBlocks { | 
					
						
							|  |  |  | 			successDataBlocksCount++ | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	// Returns true if we have atleast the dataBlocks.
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	return successDataBlocksCount >= dataBlocks | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-27 13:05:48 +08:00
										 |  |  | // Return readable disks slice from which we can read parallelly.
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | func getReadDisks(orderedDisks []StorageAPI, index int, dataBlocks int) (readDisks []StorageAPI, nextIndex int, err error) { | 
					
						
							|  |  |  | 	readDisks = make([]StorageAPI, len(orderedDisks)) | 
					
						
							|  |  |  | 	dataDisks := 0 | 
					
						
							|  |  |  | 	parityDisks := 0 | 
					
						
							|  |  |  | 	// Count already read data and parity chunks.
 | 
					
						
							|  |  |  | 	for i := 0; i < index; i++ { | 
					
						
							|  |  |  | 		if orderedDisks[i] == nil { | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		if i < dataBlocks { | 
					
						
							|  |  |  | 			dataDisks++ | 
					
						
							|  |  |  | 		} else { | 
					
						
							|  |  |  | 			parityDisks++ | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Sanity checks - we should never have this situation.
 | 
					
						
							|  |  |  | 	if dataDisks == dataBlocks { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return nil, 0, traceError(errUnexpected) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-08-15 16:25:41 +08:00
										 |  |  | 	if dataDisks+parityDisks >= dataBlocks { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return nil, 0, traceError(errUnexpected) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Find the disks from which next set of parallel reads should happen.
 | 
					
						
							|  |  |  | 	for i := index; i < len(orderedDisks); i++ { | 
					
						
							|  |  |  | 		if orderedDisks[i] == nil { | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		if i < dataBlocks { | 
					
						
							|  |  |  | 			dataDisks++ | 
					
						
							|  |  |  | 		} else { | 
					
						
							|  |  |  | 			parityDisks++ | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		readDisks[i] = orderedDisks[i] | 
					
						
							|  |  |  | 		if dataDisks == dataBlocks { | 
					
						
							|  |  |  | 			return readDisks, i + 1, nil | 
					
						
							| 
									
										
										
										
											2016-08-15 16:25:41 +08:00
										 |  |  | 		} else if dataDisks+parityDisks == dataBlocks { | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 			return readDisks, i + 1, nil | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 	return nil, 0, traceError(errXLReadQuorum) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | // parallelRead - reads chunks in parallel from the disks specified in []readDisks.
 | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | func parallelRead(volume, path string, readDisks []StorageAPI, orderedDisks []StorageAPI, enBlocks [][]byte, blockOffset int64, curChunkSize int64, bitRotVerify func(diskIndex int) bool, pool *bpool.BytePool) { | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 	// WaitGroup to synchronise the read go-routines.
 | 
					
						
							|  |  |  | 	wg := &sync.WaitGroup{} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Read disks in parallel.
 | 
					
						
							|  |  |  | 	for index := range readDisks { | 
					
						
							|  |  |  | 		if readDisks[index] == nil { | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		wg.Add(1) | 
					
						
							|  |  |  | 		// Reads chunk from readDisk[index] in routine.
 | 
					
						
							|  |  |  | 		go func(index int) { | 
					
						
							|  |  |  | 			defer wg.Done() | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 			// 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 | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 			buf, err := pool.Get() | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 			if err != nil { | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 				errorIf(err, "unable to get buffer from byte pool") | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 				orderedDisks[index] = nil | 
					
						
							|  |  |  | 				return | 
					
						
							|  |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 			buf = buf[:curChunkSize] | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 			_, err = readDisks[index].ReadFile(volume, path, blockOffset, buf) | 
					
						
							|  |  |  | 			if err != nil { | 
					
						
							|  |  |  | 				orderedDisks[index] = nil | 
					
						
							|  |  |  | 				return | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 			enBlocks[index] = buf | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 		}(index) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Waiting for first routines to finish.
 | 
					
						
							|  |  |  | 	wg.Wait() | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | // 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
 | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | // 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
 | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | // verifying checksum of individual block's checksum.
 | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | 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) { | 
					
						
							| 
									
										
										
										
											2016-07-07 16:30:34 +08:00
										 |  |  | 	// Offset and length cannot be negative.
 | 
					
						
							|  |  |  | 	if offset < 0 || length < 0 { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return 0, traceError(errUnexpected) | 
					
						
							| 
									
										
										
										
											2016-07-07 16:30:34 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 	// Can't request more data than what is available.
 | 
					
						
							|  |  |  | 	if offset+length > totalLength { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return 0, traceError(errUnexpected) | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 	// chunkSize is the amount of data that needs to be read from each disk at a time.
 | 
					
						
							|  |  |  | 	chunkSize := getChunkSize(blockSize, dataBlocks) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	// bitRotVerify verifies if the file on a particular disk doesn't have bitrot
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	// by verifying the hash of the contents of the file.
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 	bitRotVerify := func() func(diskIndex int) bool { | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 		verified := make([]bool, len(disks)) | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		// Return closure so that we have reference to []verified and
 | 
					
						
							| 
									
										
										
										
											2016-06-27 13:05:48 +08:00
										 |  |  | 		// not recalculate the hash on it every time the function is
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		// called for the same disk.
 | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 		return func(diskIndex int) bool { | 
					
						
							|  |  |  | 			if verified[diskIndex] { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 				// Already validated.
 | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 				return true | 
					
						
							|  |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 			// Is this a valid block?
 | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | 			isValid := isValidBlock(disks[diskIndex], volume, path, checkSums[diskIndex], algo) | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 			verified[diskIndex] = isValid | 
					
						
							|  |  |  | 			return isValid | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	}() | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Total bytes written to writer
 | 
					
						
							| 
									
										
										
										
											2017-02-25 01:20:40 +08:00
										 |  |  | 	var bytesWritten int64 | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 	startBlock := offset / blockSize | 
					
						
							|  |  |  | 	endBlock := (offset + length) / blockSize | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// curChunkSize = chunk size for the current block in the for loop below.
 | 
					
						
							|  |  |  | 	// curBlockSize = block size for the current block in the for loop below.
 | 
					
						
							|  |  |  | 	// curChunkSize and curBlockSize can change for the last block if totalLength%blockSize != 0
 | 
					
						
							|  |  |  | 	curChunkSize := chunkSize | 
					
						
							|  |  |  | 	curBlockSize := blockSize | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// For each block, read chunk from each disk. If we are able to read all the data disks then we don't
 | 
					
						
							|  |  |  | 	// need to read parity disks. If one of the data disk is missing we need to read DataBlocks+1 number
 | 
					
						
							|  |  |  | 	// of disks. Once read, we Reconstruct() missing data if needed and write it to the given writer.
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 	for block := startBlock; block <= endBlock; block++ { | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 		// Mark all buffers as unused at the start of the loop so that the buffers
 | 
					
						
							|  |  |  | 		// can be reused.
 | 
					
						
							|  |  |  | 		pool.Reset() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		// Each element of enBlocks holds curChunkSize'd amount of data read from its corresponding disk.
 | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 		enBlocks := make([][]byte, len(disks)) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		if ((offset + bytesWritten) / blockSize) == (totalLength / blockSize) { | 
					
						
							|  |  |  | 			// This is the last block for which curBlockSize and curChunkSize can change.
 | 
					
						
							|  |  |  | 			// For ex. if totalLength is 15M and blockSize is 10MB, curBlockSize for
 | 
					
						
							|  |  |  | 			// the last block should be 5MB.
 | 
					
						
							|  |  |  | 			curBlockSize = totalLength % blockSize | 
					
						
							|  |  |  | 			curChunkSize = getChunkSize(curBlockSize, dataBlocks) | 
					
						
							| 
									
										
										
										
											2016-06-22 05:34:11 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-21 12:40:10 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 		// NOTE: That for the offset calculation we have to use chunkSize and
 | 
					
						
							|  |  |  | 		// not curChunkSize. If we use curChunkSize for offset calculation
 | 
					
						
							|  |  |  | 		// then it can result in wrong offset for the last block.
 | 
					
						
							|  |  |  | 		blockOffset := block * chunkSize | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		// nextIndex - index from which next set of parallel reads
 | 
					
						
							|  |  |  | 		// should happen.
 | 
					
						
							|  |  |  | 		nextIndex := 0 | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 		for { | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 			// readDisks - disks from which we need to read in parallel.
 | 
					
						
							|  |  |  | 			var readDisks []StorageAPI | 
					
						
							|  |  |  | 			var err error | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			// get readable disks slice from which we can read parallelly.
 | 
					
						
							|  |  |  | 			readDisks, nextIndex, err = getReadDisks(disks, nextIndex, dataBlocks) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 			if err != nil { | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 				return bytesWritten, err | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			// Issue a parallel read across the disks specified in readDisks.
 | 
					
						
							| 
									
										
										
										
											2016-07-26 05:17:01 +08:00
										 |  |  | 			parallelRead(volume, path, readDisks, disks, enBlocks, blockOffset, curChunkSize, bitRotVerify, pool) | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			if isSuccessDecodeBlocks(enBlocks, dataBlocks) { | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 				// If enough blocks are available to do rs.Reconstruct()
 | 
					
						
							|  |  |  | 				break | 
					
						
							|  |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			if nextIndex == len(disks) { | 
					
						
							| 
									
										
										
										
											2016-06-28 04:24:55 +08:00
										 |  |  | 				// No more disks to read from.
 | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 				return bytesWritten, traceError(errXLReadQuorum) | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			// We do not have enough enough data blocks to reconstruct the data
 | 
					
						
							|  |  |  | 			// hence continue the for-loop till we have enough data blocks.
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-25 09:00:34 +08:00
										 |  |  | 		// If we have all the data blocks no need to decode, continue to write.
 | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 		if !isSuccessDataBlocks(enBlocks, dataBlocks) { | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 			// Reconstruct the missing data blocks.
 | 
					
						
							| 
									
										
										
										
											2016-07-16 23:35:30 +08:00
										 |  |  | 			if err := decodeData(enBlocks, dataBlocks, parityBlocks); err != nil { | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 				return bytesWritten, err | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | 			} | 
					
						
							| 
									
										
										
										
											2016-05-29 06:13:15 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		// Offset in enBlocks from where data should be read from.
 | 
					
						
							| 
									
										
										
										
											2017-02-25 01:20:40 +08:00
										 |  |  | 		var enBlocksOffset int64 | 
					
						
							| 
									
										
										
										
											2016-07-19 14:56:16 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		// Total data to be read from enBlocks.
 | 
					
						
							|  |  |  | 		enBlocksLength := curBlockSize | 
					
						
							| 
									
										
										
										
											2016-07-19 14:56:16 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		// If this is the start block then enBlocksOffset might not be 0.
 | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 		if block == startBlock { | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 			enBlocksOffset = offset % blockSize | 
					
						
							|  |  |  | 			enBlocksLength -= enBlocksOffset | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-05-30 06:38:14 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		remaining := length - bytesWritten | 
					
						
							|  |  |  | 		if remaining < enBlocksLength { | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 			// We should not send more data than what was requested.
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 			enBlocksLength = remaining | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-23 03:55:23 +08:00
										 |  |  | 		// Write data blocks.
 | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 		n, err := writeDataBlocks(writer, enBlocks, dataBlocks, enBlocksOffset, enBlocksLength) | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			return bytesWritten, err | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 		// Update total bytes written.
 | 
					
						
							| 
									
										
										
										
											2016-06-23 03:55:23 +08:00
										 |  |  | 		bytesWritten += n | 
					
						
							| 
									
										
										
										
											2016-07-20 16:30:30 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 		if bytesWritten == length { | 
					
						
							|  |  |  | 			// Done writing all the requested data.
 | 
					
						
							|  |  |  | 			break | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2016-06-01 11:23:31 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	// Success.
 | 
					
						
							| 
									
										
										
										
											2016-06-20 04:35:26 +08:00
										 |  |  | 	return bytesWritten, nil | 
					
						
							| 
									
										
										
										
											2016-03-28 12:52:38 +08:00
										 |  |  | } | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | // isValidBlock - calculates the checksum hash for the block and
 | 
					
						
							|  |  |  | // validates if its correct returns true for valid cases, false otherwise.
 | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | func isValidBlock(disk StorageAPI, volume, path, checkSum, checkSumAlgo string) (ok bool) { | 
					
						
							| 
									
										
										
										
											2016-06-26 18:32:49 +08:00
										 |  |  | 	// Disk is not available, not a valid block.
 | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 	if disk == nil { | 
					
						
							|  |  |  | 		return false | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | 	// Checksum not available, not a valid block.
 | 
					
						
							|  |  |  | 	if checkSum == "" { | 
					
						
							|  |  |  | 		return false | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	// Read everything for a given block and calculate hash.
 | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | 	hashWriter := newHash(checkSumAlgo) | 
					
						
							| 
									
										
										
										
											2016-06-23 00:05:03 +08:00
										 |  |  | 	hashBytes, err := hashSum(disk, volume, path, hashWriter) | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	if err != nil { | 
					
						
							| 
									
										
										
										
											2016-06-26 18:32:49 +08:00
										 |  |  | 		errorIf(err, "Unable to calculate checksum %s/%s", volume, path) | 
					
						
							|  |  |  | 		return false | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-07-28 17:20:34 +08:00
										 |  |  | 	return hex.EncodeToString(hashBytes) == checkSum | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // decodeData - decode encoded blocks.
 | 
					
						
							|  |  |  | func decodeData(enBlocks [][]byte, dataBlocks, parityBlocks int) error { | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 	// Initialized reedsolomon.
 | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	rs, err := reedsolomon.New(dataBlocks, parityBlocks) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return traceError(err) | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// Reconstruct encoded blocks.
 | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	err = rs.Reconstruct(enBlocks) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return traceError(err) | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	// Verify reconstructed blocks (parity).
 | 
					
						
							|  |  |  | 	ok, err := rs.Verify(enBlocks) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return traceError(err) | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 	if !ok { | 
					
						
							|  |  |  | 		// Blocks cannot be reconstructed, corrupted data.
 | 
					
						
							| 
									
										
										
										
											2016-11-16 10:14:23 +08:00
										 |  |  | 		err = errors.New("Verification failed after reconstruction, data likely corrupted") | 
					
						
							| 
									
										
										
										
											2016-08-26 00:39:01 +08:00
										 |  |  | 		return traceError(err) | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2016-06-24 17:06:23 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// Success.
 | 
					
						
							| 
									
										
										
										
											2016-06-02 16:49:46 +08:00
										 |  |  | 	return nil | 
					
						
							|  |  |  | } |