mirror of
https://github.com/versity/versitygw.git
synced 2025-12-23 05:05:16 +00:00
Merge pull request #1664 from versity/sis/malformed-streaming-upload
fix: fixes unsigned streaming upload parsing and checksum calculation
This commit is contained in:
@@ -40,6 +40,7 @@ import (
|
||||
"github.com/versity/versitygw/backend"
|
||||
"github.com/versity/versitygw/backend/meta"
|
||||
"github.com/versity/versitygw/debuglogger"
|
||||
"github.com/versity/versitygw/s3api/middlewares"
|
||||
"github.com/versity/versitygw/s3api/utils"
|
||||
"github.com/versity/versitygw/s3err"
|
||||
"github.com/versity/versitygw/s3response"
|
||||
@@ -2501,53 +2502,56 @@ func (p *Posix) UploadPartWithPostFunc(ctx context.Context, input *s3.UploadPart
|
||||
hash := md5.New()
|
||||
tr := io.TeeReader(r, hash)
|
||||
|
||||
hashConfigs := []hashConfig{
|
||||
{input.ChecksumCRC32, utils.HashTypeCRC32},
|
||||
{input.ChecksumCRC32C, utils.HashTypeCRC32C},
|
||||
{input.ChecksumSHA1, utils.HashTypeSha1},
|
||||
{input.ChecksumSHA256, utils.HashTypeSha256},
|
||||
{input.ChecksumCRC64NVME, utils.HashTypeCRC64NVME},
|
||||
}
|
||||
chRdr, chunkUpload := input.Body.(middlewares.ChecksumReader)
|
||||
isTrailingChecksum := chunkUpload && chRdr.Algorithm() != ""
|
||||
|
||||
var hashRdr *utils.HashReader
|
||||
for _, config := range hashConfigs {
|
||||
if config.value != nil {
|
||||
hashRdr, err = utils.NewHashReader(tr, *config.value, config.hashType)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("initialize hash reader: %w", err)
|
||||
|
||||
if !isTrailingChecksum {
|
||||
hashConfigs := []hashConfig{
|
||||
{input.ChecksumCRC32, utils.HashTypeCRC32},
|
||||
{input.ChecksumCRC32C, utils.HashTypeCRC32C},
|
||||
{input.ChecksumSHA1, utils.HashTypeSha1},
|
||||
{input.ChecksumSHA256, utils.HashTypeSha256},
|
||||
{input.ChecksumCRC64NVME, utils.HashTypeCRC64NVME},
|
||||
}
|
||||
|
||||
for _, config := range hashConfigs {
|
||||
if config.value != nil {
|
||||
hashRdr, err = utils.NewHashReader(tr, *config.value, config.hashType)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
|
||||
tr = hashRdr
|
||||
}
|
||||
|
||||
tr = hashRdr
|
||||
}
|
||||
}
|
||||
|
||||
// If only the checksum algorithm is provided register
|
||||
// a new HashReader to calculate the object checksum
|
||||
if hashRdr == nil && input.ChecksumAlgorithm != "" {
|
||||
hashRdr, err = utils.NewHashReader(tr, "", utils.HashType(strings.ToLower(string(input.ChecksumAlgorithm))))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
|
||||
tr = hashRdr
|
||||
}
|
||||
|
||||
checksums, chErr := p.retrieveChecksums(nil, bucket, mpPath)
|
||||
if chErr != nil && !errors.Is(chErr, meta.ErrNoSuchKey) {
|
||||
return nil, fmt.Errorf("retreive mp checksum: %w", chErr)
|
||||
}
|
||||
|
||||
var inputChAlgo utils.HashType
|
||||
if isTrailingChecksum {
|
||||
inputChAlgo = utils.HashType(chRdr.Algorithm())
|
||||
}
|
||||
if hashRdr != nil {
|
||||
inputChAlgo = hashRdr.Type()
|
||||
}
|
||||
|
||||
// If checksum isn't provided for the part,
|
||||
// but it has been provided on mp initalization
|
||||
// and checksum type is 'COMPOSITE', return mismatch error
|
||||
if hashRdr == nil && chErr == nil && checksums.Type == types.ChecksumTypeComposite {
|
||||
if inputChAlgo == "" && checksums.Type == types.ChecksumTypeComposite {
|
||||
return nil, s3err.GetChecksumTypeMismatchErr(checksums.Algorithm, "null")
|
||||
}
|
||||
|
||||
// Check if the provided checksum algorithm match
|
||||
// the one specified on mp initialization
|
||||
if hashRdr != nil && chErr == nil && checksums.Type != "" {
|
||||
algo := types.ChecksumAlgorithm(strings.ToUpper(string(hashRdr.Type())))
|
||||
if inputChAlgo != "" && checksums.Type != "" {
|
||||
algo := types.ChecksumAlgorithm(strings.ToUpper(string(inputChAlgo)))
|
||||
if checksums.Algorithm != algo {
|
||||
return nil, s3err.GetChecksumTypeMismatchErr(checksums.Algorithm, algo)
|
||||
}
|
||||
@@ -2556,11 +2560,13 @@ func (p *Posix) UploadPartWithPostFunc(ctx context.Context, input *s3.UploadPart
|
||||
// if no checksum algorithm or precalculated checksum is
|
||||
// provided, but one has been on multipart upload initialization,
|
||||
// anyways calculate and store the uploaded part checksum
|
||||
if hashRdr == nil && checksums.Algorithm != "" {
|
||||
hashRdr, err = utils.NewHashReader(tr, "", utils.HashType(strings.ToLower(string(checksums.Algorithm))))
|
||||
if inputChAlgo == "" && checksums.Algorithm != "" {
|
||||
hashType := utils.HashType(strings.ToLower(string(checksums.Algorithm)))
|
||||
hashRdr, err = utils.NewHashReader(tr, "", hashType)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
inputChAlgo = hashType
|
||||
|
||||
tr = hashRdr
|
||||
}
|
||||
@@ -2587,14 +2593,21 @@ func (p *Posix) UploadPartWithPostFunc(ctx context.Context, input *s3.UploadPart
|
||||
ETag: &etag,
|
||||
}
|
||||
|
||||
if hashRdr != nil {
|
||||
if inputChAlgo != "" {
|
||||
checksum := s3response.Checksum{
|
||||
Algorithm: input.ChecksumAlgorithm,
|
||||
}
|
||||
|
||||
// Validate the provided checksum
|
||||
sum := hashRdr.Sum()
|
||||
switch hashRdr.Type() {
|
||||
var sum string
|
||||
if isTrailingChecksum {
|
||||
sum = chRdr.Checksum()
|
||||
}
|
||||
if hashRdr != nil {
|
||||
sum = hashRdr.Sum()
|
||||
}
|
||||
|
||||
// Assign the checksum
|
||||
switch inputChAlgo {
|
||||
case utils.HashTypeCRC32:
|
||||
checksum.CRC32 = &sum
|
||||
res.ChecksumCRC32 = &sum
|
||||
@@ -2880,10 +2893,6 @@ func (p *Posix) PutObjectWithPostFunc(ctx context.Context, po s3response.PutObje
|
||||
if po.Key == nil {
|
||||
return s3response.PutObjectOutput{}, s3err.GetAPIError(s3err.ErrNoSuchKey)
|
||||
}
|
||||
// Override the checksum algorithm with default: CRC64NVME
|
||||
if po.ChecksumAlgorithm == "" {
|
||||
po.ChecksumAlgorithm = types.ChecksumAlgorithmCrc64nvme
|
||||
}
|
||||
if !p.isBucketValid(*po.Bucket) {
|
||||
return s3response.PutObjectOutput{}, s3err.GetAPIError(s3err.ErrInvalidBucketName)
|
||||
}
|
||||
@@ -3017,35 +3026,54 @@ func (p *Posix) PutObjectWithPostFunc(ctx context.Context, po s3response.PutObje
|
||||
hash := md5.New()
|
||||
rdr := io.TeeReader(po.Body, hash)
|
||||
|
||||
hashConfigs := []hashConfig{
|
||||
{po.ChecksumCRC32, utils.HashTypeCRC32},
|
||||
{po.ChecksumCRC32C, utils.HashTypeCRC32C},
|
||||
{po.ChecksumSHA1, utils.HashTypeSha1},
|
||||
{po.ChecksumSHA256, utils.HashTypeSha256},
|
||||
{po.ChecksumCRC64NVME, utils.HashTypeCRC64NVME},
|
||||
}
|
||||
var hashRdr *utils.HashReader
|
||||
|
||||
for _, config := range hashConfigs {
|
||||
if config.value != nil {
|
||||
hashRdr, err = utils.NewHashReader(rdr, *config.value, config.hashType)
|
||||
chRdr, chunkUpload := po.Body.(middlewares.ChecksumReader)
|
||||
isTrailingChecksum := chunkUpload && chRdr.Algorithm() != ""
|
||||
|
||||
if !isTrailingChecksum {
|
||||
hashConfigs := []hashConfig{
|
||||
{po.ChecksumCRC32, utils.HashTypeCRC32},
|
||||
{po.ChecksumCRC32C, utils.HashTypeCRC32C},
|
||||
{po.ChecksumSHA1, utils.HashTypeSha1},
|
||||
{po.ChecksumSHA256, utils.HashTypeSha256},
|
||||
{po.ChecksumCRC64NVME, utils.HashTypeCRC64NVME},
|
||||
}
|
||||
|
||||
for _, config := range hashConfigs {
|
||||
if config.value != nil {
|
||||
hashRdr, err = utils.NewHashReader(rdr, *config.value, config.hashType)
|
||||
if err != nil {
|
||||
return s3response.PutObjectOutput{}, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
|
||||
rdr = hashRdr
|
||||
}
|
||||
}
|
||||
|
||||
// If only the checksum algorithm is provided register
|
||||
// a new HashReader to calculate the object checksum
|
||||
// This can never happen with PutObject direct call
|
||||
// it's there for CopyObject to add a new checksum
|
||||
if hashRdr == nil && po.ChecksumAlgorithm != "" {
|
||||
hashRdr, err = utils.NewHashReader(rdr, "", utils.HashType(strings.ToLower(string(po.ChecksumAlgorithm))))
|
||||
if err != nil {
|
||||
return s3response.PutObjectOutput{}, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
|
||||
rdr = hashRdr
|
||||
}
|
||||
}
|
||||
|
||||
// If only the checksum algorithm is provided register
|
||||
// a new HashReader to calculate the object checksum
|
||||
if hashRdr == nil && po.ChecksumAlgorithm != "" {
|
||||
hashRdr, err = utils.NewHashReader(rdr, "", utils.HashType(strings.ToLower(string(po.ChecksumAlgorithm))))
|
||||
if err != nil {
|
||||
return s3response.PutObjectOutput{}, fmt.Errorf("initialize hash reader: %w", err)
|
||||
if hashRdr == nil {
|
||||
// if no precalculated checksum or sdk checksum algorithm is provided
|
||||
// and no streaming upload has checksum, default to crc64nvme
|
||||
hashRdr, err = utils.NewHashReader(rdr, "", utils.HashTypeCRC64NVME)
|
||||
if err != nil {
|
||||
return s3response.PutObjectOutput{}, fmt.Errorf("initialize hash reader: %w", err)
|
||||
}
|
||||
|
||||
rdr = hashRdr
|
||||
}
|
||||
|
||||
rdr = hashRdr
|
||||
}
|
||||
|
||||
_, err = io.Copy(f, rdr)
|
||||
@@ -3095,15 +3123,23 @@ func (p *Posix) PutObjectWithPostFunc(ctx context.Context, po s3response.PutObje
|
||||
}
|
||||
}
|
||||
|
||||
var chAlgo utils.HashType
|
||||
var sum string
|
||||
|
||||
if isTrailingChecksum {
|
||||
chAlgo = utils.HashType(chRdr.Algorithm())
|
||||
sum = chRdr.Checksum()
|
||||
} else if hashRdr != nil {
|
||||
chAlgo = hashRdr.Type()
|
||||
sum = hashRdr.Sum()
|
||||
}
|
||||
|
||||
checksum := s3response.Checksum{}
|
||||
|
||||
// Store the calculated checksum in the object metadata
|
||||
if hashRdr != nil {
|
||||
// The checksum type is always FULL_OBJECT for PutObject
|
||||
if sum != "" {
|
||||
checksum.Type = types.ChecksumTypeFullObject
|
||||
|
||||
sum := hashRdr.Sum()
|
||||
switch hashRdr.Type() {
|
||||
switch chAlgo {
|
||||
case utils.HashTypeCRC32:
|
||||
checksum.CRC32 = &sum
|
||||
checksum.Algorithm = types.ChecksumAlgorithmCrc32
|
||||
@@ -3120,7 +3156,6 @@ func (p *Posix) PutObjectWithPostFunc(ctx context.Context, po s3response.PutObje
|
||||
checksum.CRC64NVME = &sum
|
||||
checksum.Algorithm = types.ChecksumAlgorithmCrc64nvme
|
||||
}
|
||||
|
||||
err := p.storeChecksums(f.File(), *po.Bucket, *po.Key, checksum)
|
||||
if err != nil {
|
||||
return s3response.PutObjectOutput{}, fmt.Errorf("store checksum: %w", err)
|
||||
|
||||
@@ -22,17 +22,83 @@ import (
|
||||
"github.com/versity/versitygw/s3api/utils"
|
||||
)
|
||||
|
||||
// ChecksumReader extends io.Reader with checksum-related metadata.
|
||||
// It is used to differentiate normal readers from readers that can
|
||||
// report a checksum and the algorithm used to produce it.
|
||||
type ChecksumReader interface {
|
||||
io.Reader
|
||||
Algorithm() string
|
||||
Checksum() string
|
||||
}
|
||||
|
||||
// NewChecksumReader wraps a stackedReader and returns a reader that
|
||||
// preserves checksum behavior when the *original* bodyReader implemented
|
||||
// ChecksumReader.
|
||||
//
|
||||
// If bodyReader already supports ChecksumReader, we wrap stackedReader
|
||||
// with MockChecksumReader so that reading continues from stackedReader,
|
||||
// but Algorithm() and Checksum() still delegate to the underlying reader.
|
||||
//
|
||||
// If bodyReader is not a ChecksumReader, we simply return stackedReader.
|
||||
func NewChecksumReader(bodyReader io.Reader, stackedReader io.Reader) io.Reader {
|
||||
_, ok := bodyReader.(ChecksumReader)
|
||||
if ok {
|
||||
return &MockChecksumReader{rdr: stackedReader}
|
||||
}
|
||||
|
||||
return stackedReader
|
||||
}
|
||||
|
||||
// MockChecksumReader is a wrapper around an io.Reader that forwards Read()
|
||||
// but also conditionally exposes checksum metadata if the underlying reader
|
||||
// implements the ChecksumReader interface.
|
||||
type MockChecksumReader struct {
|
||||
rdr io.Reader
|
||||
}
|
||||
|
||||
// Read simply forwards data reads to the underlying reader.
|
||||
func (rr *MockChecksumReader) Read(buffer []byte) (int, error) {
|
||||
return rr.rdr.Read(buffer)
|
||||
}
|
||||
|
||||
// Algorithm returns the checksum algorithm used by the underlying reader,
|
||||
// but only if the wrapped reader implements ChecksumReader.
|
||||
func (rr *MockChecksumReader) Algorithm() string {
|
||||
r, ok := rr.rdr.(ChecksumReader)
|
||||
if ok {
|
||||
return r.Algorithm()
|
||||
}
|
||||
|
||||
return ""
|
||||
}
|
||||
|
||||
// Checksum returns the checksum value from the underlying reader,
|
||||
// if it implements ChecksumReader. Otherwise returns an empty string.
|
||||
func (rr *MockChecksumReader) Checksum() string {
|
||||
r, ok := rr.rdr.(ChecksumReader)
|
||||
if ok {
|
||||
return r.Checksum()
|
||||
}
|
||||
|
||||
return ""
|
||||
}
|
||||
|
||||
var _ ChecksumReader = &MockChecksumReader{}
|
||||
|
||||
func wrapBodyReader(ctx *fiber.Ctx, wr func(io.Reader) io.Reader) {
|
||||
r, ok := utils.ContextKeyBodyReader.Get(ctx).(io.Reader)
|
||||
rdr, ok := utils.ContextKeyBodyReader.Get(ctx).(io.Reader)
|
||||
if !ok {
|
||||
r = ctx.Request().BodyStream()
|
||||
rdr = ctx.Request().BodyStream()
|
||||
// Override the body reader with an empty reader to prevent panics
|
||||
// in case of unexpected or malformed HTTP requests.
|
||||
if r == nil {
|
||||
r = bytes.NewBuffer([]byte{})
|
||||
if rdr == nil {
|
||||
rdr = bytes.NewBuffer([]byte{})
|
||||
}
|
||||
}
|
||||
|
||||
r = wr(r)
|
||||
r := wr(rdr)
|
||||
// Ensure checksum behavior is stacked if the original body reader had it.
|
||||
r = NewChecksumReader(rdr, r)
|
||||
|
||||
utils.ContextKeyBodyReader.Set(ctx, r)
|
||||
}
|
||||
|
||||
@@ -170,7 +170,7 @@ func NewChunkReader(ctx *fiber.Ctx, r io.Reader, authdata AuthData, region, secr
|
||||
//TODO: not sure if InvalidRequest should be returned in this case
|
||||
if err != nil {
|
||||
debuglogger.Logf("invalid value for 'X-Amz-Decoded-Content-Length': %v", decContLengthStr)
|
||||
return nil, s3err.GetAPIError(s3err.ErrInvalidRequest)
|
||||
return nil, s3err.GetAPIError(s3err.ErrMissingContentLength)
|
||||
}
|
||||
|
||||
if decContLength > maxObjSizeLimit {
|
||||
@@ -189,10 +189,6 @@ func NewChunkReader(ctx *fiber.Ctx, r io.Reader, authdata AuthData, region, secr
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if contentSha256 != payloadTypeStreamingSigned && checksumType == "" {
|
||||
debuglogger.Logf("empty value for required trailer header 'X-Amz-Trailer': %v", checksumType)
|
||||
return nil, s3err.GetAPIError(s3err.ErrTrailerHeaderNotSupported)
|
||||
}
|
||||
|
||||
switch contentSha256 {
|
||||
case payloadTypeStreamingUnsignedTrailer:
|
||||
|
||||
@@ -21,7 +21,6 @@ import (
|
||||
"crypto/sha256"
|
||||
"encoding/base64"
|
||||
"errors"
|
||||
"fmt"
|
||||
"hash"
|
||||
"hash/crc32"
|
||||
"hash/crc64"
|
||||
@@ -30,7 +29,9 @@ import (
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/service/s3/types"
|
||||
"github.com/versity/versitygw/debuglogger"
|
||||
"github.com/versity/versitygw/s3err"
|
||||
)
|
||||
|
||||
var (
|
||||
@@ -39,20 +40,25 @@ var (
|
||||
)
|
||||
|
||||
type UnsignedChunkReader struct {
|
||||
reader *bufio.Reader
|
||||
checksumType checksumType
|
||||
expectedChecksum string
|
||||
hasher hash.Hash
|
||||
stash []byte
|
||||
offset int
|
||||
reader *bufio.Reader
|
||||
checksumType checksumType
|
||||
parsedChecksum string
|
||||
hasher hash.Hash
|
||||
stash []byte
|
||||
offset int
|
||||
}
|
||||
|
||||
func NewUnsignedChunkReader(r io.Reader, ct checksumType) (*UnsignedChunkReader, error) {
|
||||
hasher, err := getHasher(ct)
|
||||
var hasher hash.Hash
|
||||
var err error
|
||||
if ct != "" {
|
||||
hasher, err = getHasher(ct)
|
||||
}
|
||||
if err != nil {
|
||||
debuglogger.Logf("failed to initialize hash calculator: %v", err)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
debuglogger.Infof("initializing unsigned chunk reader")
|
||||
return &UnsignedChunkReader{
|
||||
reader: bufio.NewReader(r),
|
||||
@@ -62,6 +68,16 @@ func NewUnsignedChunkReader(r io.Reader, ct checksumType) (*UnsignedChunkReader,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Algorithm returns the checksum algorithm
|
||||
func (ucr *UnsignedChunkReader) Algorithm() string {
|
||||
return strings.TrimPrefix(string(ucr.checksumType), "x-amz-checksum-")
|
||||
}
|
||||
|
||||
// Checksum returns the parsed trailing checksum
|
||||
func (ucr *UnsignedChunkReader) Checksum() string {
|
||||
return ucr.parsedChecksum
|
||||
}
|
||||
|
||||
func (ucr *UnsignedChunkReader) Read(p []byte) (int, error) {
|
||||
// First read any stashed data
|
||||
if len(ucr.stash) != 0 {
|
||||
@@ -87,7 +103,10 @@ func (ucr *UnsignedChunkReader) Read(p []byte) (int, error) {
|
||||
// Stop reading parsing payloads as 0 sized chunk is reached
|
||||
break
|
||||
}
|
||||
rdr := io.TeeReader(ucr.reader, ucr.hasher)
|
||||
var rdr io.Reader = ucr.reader
|
||||
if ucr.hasher != nil {
|
||||
rdr = io.TeeReader(ucr.reader, ucr.hasher)
|
||||
}
|
||||
payload := make([]byte, chunkSize)
|
||||
// Read and cache the payload
|
||||
_, err = io.ReadFull(rdr, payload)
|
||||
@@ -167,6 +186,7 @@ func (ucr *UnsignedChunkReader) extractChunkSize() (int64, error) {
|
||||
// Reads and validates the trailer at the end
|
||||
func (ucr *UnsignedChunkReader) readTrailer() error {
|
||||
var trailerBuffer bytes.Buffer
|
||||
var hasChecksum bool
|
||||
|
||||
for {
|
||||
v, err := ucr.reader.ReadByte()
|
||||
@@ -178,9 +198,27 @@ func (ucr *UnsignedChunkReader) readTrailer() error {
|
||||
return err
|
||||
}
|
||||
if v != '\r' {
|
||||
hasChecksum = true
|
||||
trailerBuffer.WriteByte(v)
|
||||
continue
|
||||
}
|
||||
|
||||
if !hasChecksum {
|
||||
// in case the payload doesn't contain trailer
|
||||
// the first 2 bytes(\r\n) have been read
|
||||
// only read the last byte: \n
|
||||
err := ucr.readAndSkip('\n')
|
||||
if err != nil {
|
||||
debuglogger.Logf("failed to read chunk last byte: \\n: %v", err)
|
||||
if err == io.EOF {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
break
|
||||
}
|
||||
|
||||
var tmp [3]byte
|
||||
_, err = io.ReadFull(ucr.reader, tmp[:])
|
||||
if err != nil {
|
||||
@@ -200,20 +238,35 @@ func (ucr *UnsignedChunkReader) readTrailer() error {
|
||||
// Parse the trailer
|
||||
trailerHeader := trailerBuffer.String()
|
||||
trailerHeader = strings.TrimSpace(trailerHeader)
|
||||
if trailerHeader == "" {
|
||||
if ucr.checksumType != "" {
|
||||
debuglogger.Logf("expected %s checksum in the paylod, but it's missing", ucr.checksumType)
|
||||
return s3err.GetAPIError(s3err.ErrMalformedTrailer)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
trailerHeaderParts := strings.Split(trailerHeader, ":")
|
||||
if len(trailerHeaderParts) != 2 {
|
||||
debuglogger.Logf("invalid trailer header parts: %v", trailerHeaderParts)
|
||||
return errMalformedEncoding
|
||||
return s3err.GetAPIError(s3err.ErrMalformedTrailer)
|
||||
}
|
||||
|
||||
if trailerHeaderParts[0] != string(ucr.checksumType) {
|
||||
debuglogger.Logf("invalid checksum type: %v", trailerHeaderParts[0])
|
||||
//TODO: handle the error
|
||||
return errMalformedEncoding
|
||||
checksumKey := checksumType(trailerHeaderParts[0])
|
||||
checksum := trailerHeaderParts[1]
|
||||
|
||||
if !checksumKey.isValid() {
|
||||
debuglogger.Logf("invalid checksum header key: %s", checksumKey)
|
||||
return s3err.GetAPIError(s3err.ErrMalformedTrailer)
|
||||
}
|
||||
|
||||
ucr.expectedChecksum = trailerHeaderParts[1]
|
||||
debuglogger.Infof("parsed the trailing header:\n%v:%v", trailerHeaderParts[0], trailerHeaderParts[1])
|
||||
if checksumKey != ucr.checksumType {
|
||||
debuglogger.Logf("incorrect checksum type (expected): %s, (actual): %s", ucr.checksumType, checksumKey)
|
||||
return s3err.GetAPIError(s3err.ErrMalformedTrailer)
|
||||
}
|
||||
|
||||
ucr.parsedChecksum = checksum
|
||||
debuglogger.Infof("parsed the trailing header:\n%v:%v", checksumKey, checksum)
|
||||
|
||||
// Validate checksum
|
||||
return ucr.validateChecksum()
|
||||
@@ -221,17 +274,30 @@ func (ucr *UnsignedChunkReader) readTrailer() error {
|
||||
|
||||
// Validates the trailing checksum sent at the end
|
||||
func (ucr *UnsignedChunkReader) validateChecksum() error {
|
||||
csum := ucr.hasher.Sum(nil)
|
||||
checksum := base64.StdEncoding.EncodeToString(csum)
|
||||
algo := types.ChecksumAlgorithm(strings.ToUpper(strings.TrimPrefix(string(ucr.checksumType), "x-amz-checksum-")))
|
||||
// validate the checksum
|
||||
if !IsValidChecksum(ucr.parsedChecksum, algo) {
|
||||
debuglogger.Logf("invalid checksum: (algo): %s, (checksum): %s", algo, ucr.parsedChecksum)
|
||||
return s3err.GetInvalidTrailingChecksumHeaderErr(string(ucr.checksumType))
|
||||
}
|
||||
|
||||
if checksum != ucr.expectedChecksum {
|
||||
debuglogger.Logf("incorrect checksum: (expected): %v, (got): %v", ucr.expectedChecksum, checksum)
|
||||
return fmt.Errorf("actual checksum: %v, expected checksum: %v", checksum, ucr.expectedChecksum)
|
||||
checksum := ucr.calculateChecksum()
|
||||
|
||||
// compare the calculated and parsed checksums
|
||||
if checksum != ucr.parsedChecksum {
|
||||
debuglogger.Logf("incorrect checksum: (expected): %v, (got): %v", ucr.parsedChecksum, checksum)
|
||||
return s3err.GetChecksumBadDigestErr(algo)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// calculateChecksum calculates the checksum with the unsigned reader hasher
|
||||
func (ucr *UnsignedChunkReader) calculateChecksum() string {
|
||||
csum := ucr.hasher.Sum(nil)
|
||||
return base64.StdEncoding.EncodeToString(csum)
|
||||
}
|
||||
|
||||
// Retruns the hash calculator based on the hash type provided
|
||||
func getHasher(ct checksumType) (hash.Hash, error) {
|
||||
switch ct {
|
||||
|
||||
@@ -512,14 +512,33 @@ func ParseChecksumHeadersAndSdkAlgo(ctx *fiber.Ctx) (types.ChecksumAlgorithm, Ch
|
||||
return sdkAlgorithm, checksums, err
|
||||
}
|
||||
|
||||
if len(checksums) == 0 && sdkAlgorithm != "" {
|
||||
if ctx.Get("X-Amz-Trailer") == "" {
|
||||
// This is a special case when x-amz-trailer is there
|
||||
// it means the upload is done with chunked encoding
|
||||
// where the checksum verification is handled in the chunk reader
|
||||
trailer := strings.ToUpper(ctx.Get("X-Amz-Trailer"))
|
||||
|
||||
if len(checksums) != 0 && trailer != "" {
|
||||
// both x-amz-trailer and one of x-amz-checksum-* is not allowed
|
||||
debuglogger.Logf("x-amz-checksum-* header is used with x-amz-trailer: trailer: %s", trailer)
|
||||
return sdkAlgorithm, checksums, s3err.GetAPIError(s3err.ErrMultipleChecksumHeaders)
|
||||
}
|
||||
|
||||
trailerAlgo := strings.TrimPrefix(trailer, "X-AMZ-CHECKSUM-")
|
||||
|
||||
if sdkAlgorithm != "" {
|
||||
if len(checksums) == 0 && trailerAlgo == "" {
|
||||
// in case x-amz-sdk-algorithm is specified, but no corresponging
|
||||
// x-amz-checksum-* or x-amz-trailer is sent
|
||||
debuglogger.Logf("'x-amz-sdk-checksum-algorithm : %s' is used without corresponding x-amz-checksum-* header", sdkAlgorithm)
|
||||
return sdkAlgorithm, checksums, s3err.GetAPIError(s3err.ErrChecksumSDKAlgoMismatch)
|
||||
}
|
||||
|
||||
if trailerAlgo != "" && string(sdkAlgorithm) != trailerAlgo {
|
||||
// x-amz-sdk-checksum-algorithm and x-amz-trailer should match
|
||||
debuglogger.Logf("x-amz-sdk-checksum-algorithm: (%s) and x-amz-trailer: (%s) doesn't match", sdkAlgorithm, trailerAlgo)
|
||||
return sdkAlgorithm, checksums, s3err.GetInvalidChecksumHeaderErr("x-amz-sdk-checksum-algorithm")
|
||||
}
|
||||
}
|
||||
|
||||
if trailerAlgo != "" {
|
||||
sdkAlgorithm = types.ChecksumAlgorithm(trailerAlgo)
|
||||
}
|
||||
|
||||
for al, val := range checksums {
|
||||
|
||||
@@ -178,6 +178,7 @@ const (
|
||||
ErrNotModified
|
||||
ErrInvalidLocationConstraint
|
||||
ErrInvalidArgument
|
||||
ErrMalformedTrailer
|
||||
|
||||
// Non-AWS errors
|
||||
ErrExistingObjectIsDirectory
|
||||
@@ -792,6 +793,11 @@ var errorCodeResponse = map[ErrorCode]APIError{
|
||||
Description: "",
|
||||
HTTPStatusCode: http.StatusBadRequest,
|
||||
},
|
||||
ErrMalformedTrailer: {
|
||||
Code: "MalformedTrailerError",
|
||||
Description: "The request contained trailing data that was not well-formed or did not conform to our published schema.",
|
||||
HTTPStatusCode: http.StatusBadRequest,
|
||||
},
|
||||
|
||||
// non aws errors
|
||||
ErrExistingObjectIsDirectory: {
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
423
tests/integration/unsigned_streaming_payload_trailer.go
Normal file
423
tests/integration/unsigned_streaming_payload_trailer.go
Normal file
@@ -0,0 +1,423 @@
|
||||
package integration
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/service/s3"
|
||||
"github.com/aws/aws-sdk-go-v2/service/s3/types"
|
||||
"github.com/versity/versitygw/s3err"
|
||||
)
|
||||
|
||||
func UnsignedStreaminPayloadTrailer_malformed_trailer(s *S3Conf) error {
|
||||
testName := "UnsignedStreaminPayloadTrailer_malformed_trailer"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-obj"
|
||||
for i, test := range []struct {
|
||||
trailer string
|
||||
decContentLength string
|
||||
payload string
|
||||
}{
|
||||
// missing trailer in the payload
|
||||
{"x-amz-checksum-crc64nvme", "5", "5\r\nhello\r\n0\r\n\r\n"},
|
||||
// empty checksum key
|
||||
{"x-amz-checksum-crc64nvme", "5", "5\r\nhello\r\n0\r\n:M3eFcAZSQlc=\r\n\r\n"},
|
||||
// missing x-amz-trailer
|
||||
{"", "5", "5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:M3eFcAZSQlc=\r\n\r\n"},
|
||||
// invalid trailer in payload
|
||||
{"x-amz-checksum-crc64nvme", "5", "5\r\nhello\r\n0\r\ninvalid_trailer:M3eFcAZSQlc=\r\n\r\n"},
|
||||
} {
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": test.decContentLength,
|
||||
}
|
||||
if test.trailer != "" {
|
||||
reqHeaders["x-amz-trailer"] = test.trailer
|
||||
}
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, []byte(test.payload), reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if err := compareS3ApiError(s3err.GetAPIError(s3err.ErrMalformedTrailer), apiErr); err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_missing_invalid_dec_content_length(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_missing_invalid_dec_content_length"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "object"
|
||||
for i, clength := range []string{"", "abc", "12x"} {
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-trailer": "x-amz-checksum-crc64nvme",
|
||||
}
|
||||
if clength != "" {
|
||||
reqHeaders["x-amz-decoded-content-length"] = clength
|
||||
}
|
||||
body := []byte("5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:M3eFcAZSQlc=\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if err := compareS3ApiError(s3err.GetAPIError(s3err.ErrMissingContentLength), apiErr); err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_invalid_trailing_checksum(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_invalid_trailing_checksum"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "5",
|
||||
"x-amz-trailer": "x-amz-checksum-crc64nvme",
|
||||
}
|
||||
|
||||
body := []byte("5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:invalid_checksum\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return compareS3ApiError(s3err.GetInvalidTrailingChecksumHeaderErr("x-amz-checksum-crc64nvme"), apiErr)
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_incorrect_trailing_checksum(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_incorrect_trailing_checksum"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "5",
|
||||
"x-amz-trailer": "x-amz-checksum-crc64nvme",
|
||||
}
|
||||
|
||||
// valid crc64nvme, but incorrect
|
||||
body := []byte("5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:QFRKMGE3tuw=\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return compareS3ApiError(s3err.GetChecksumBadDigestErr("CRC64NVME"), apiErr)
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_multiple_checksum_headers(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_multiple_checksum_headers"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
body := []byte("5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:M3eFcAZSQlc=\r\n\r\n")
|
||||
|
||||
for i, test := range []struct {
|
||||
key string
|
||||
value string
|
||||
}{
|
||||
{"crc32", "NhCmhg=="},
|
||||
{"crc32c", "+Cy97w=="},
|
||||
{"crc64nvme", "QFRKMGE3tuw="},
|
||||
{"sha1", "qvTGHdzF6KLavt4PO0gs2a6pQ00="},
|
||||
{"sha256", "LPJNul+wow4m6DsqxbninhsWHlwfp0JecwQzYpOLmCQ="},
|
||||
} {
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "5",
|
||||
"x-amz-trailer": "x-amz-checksum-crc64nvme",
|
||||
fmt.Sprintf("x-amz-checksum-%s", test.key): test.value,
|
||||
}
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if err := compareS3ApiError(s3err.GetAPIError(s3err.ErrMultipleChecksumHeaders), apiErr); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_sdk_algo_and_trailer_mismatch(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_sdk_algo_and_trailer_mismatch"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "5",
|
||||
"x-amz-trailer": "x-amz-checksum-crc64nvme",
|
||||
"x-amz-sdk-checksum-algorithm": "sha1",
|
||||
}
|
||||
|
||||
// valid crc64nvme, but incorrect
|
||||
body := []byte("5\r\nhello\r\n0\r\nx-amz-checksum-crc64nvme:M3eFcAZSQlc=\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return compareS3ApiError(s3err.GetInvalidChecksumHeaderErr("x-amz-sdk-checksum-algorithm"), apiErr)
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_no_trailer_should_calculate_crc64nvme(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_no_trailer_should_calculate_crc64nvme"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "11",
|
||||
}
|
||||
|
||||
body := []byte("B\r\nhello world\r\n0\r\n\r\n")
|
||||
|
||||
headers, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if apiErr != nil {
|
||||
return fmt.Errorf("%s: %s", apiErr.Code, apiErr.Message)
|
||||
}
|
||||
|
||||
csum := headers["x-amz-checksum-crc64nvme"]
|
||||
expectedCsum := "jSnVw/bqjr4="
|
||||
if csum != expectedCsum {
|
||||
return fmt.Errorf("expected the crc64nvme to be %s, instead got %s", expectedCsum, csum)
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_no_payload_trailer_only_headers(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_no_payload_trailer_only_headers"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
body := []byte("7\r\nabcdefg\r\n0\r\n\r\n")
|
||||
|
||||
for i, test := range []struct {
|
||||
key string
|
||||
value string
|
||||
}{
|
||||
{"crc32", "MSpqpg=="},
|
||||
{"crc32c", "5if0QQ=="},
|
||||
{"crc64nvme", "SmzZ/LTp1CA="},
|
||||
{"sha1", "L7XhNBn8iSRoZeejJPR27GJOh0A="},
|
||||
{"sha256", "fRpUEnsiJQL1t5tfsIAwYRUqRPkrN+I8ZSe69mXU2po="},
|
||||
} {
|
||||
csumHdr := fmt.Sprintf("x-amz-checksum-%s", test.key)
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "7",
|
||||
csumHdr: test.value,
|
||||
}
|
||||
|
||||
headers, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body, reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if apiErr != nil {
|
||||
return fmt.Errorf("test %v failed: (%s) %s", i+1, apiErr.Code, apiErr.Message)
|
||||
}
|
||||
|
||||
if headers[csumHdr] != test.value {
|
||||
return fmt.Errorf("expected the %s to be %s, instead got %s", csumHdr, test.value, headers[csumHdr])
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_success_both_sdk_algo_and_trailer(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_success_both_sdk_algo_and_trailer"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
|
||||
for i, test := range []struct {
|
||||
key string
|
||||
value string
|
||||
}{
|
||||
{"crc32", "MSpqpg=="},
|
||||
{"crc32c", "5if0QQ=="},
|
||||
{"crc64nvme", "SmzZ/LTp1CA="},
|
||||
{"sha1", "L7XhNBn8iSRoZeejJPR27GJOh0A="},
|
||||
{"sha256", "fRpUEnsiJQL1t5tfsIAwYRUqRPkrN+I8ZSe69mXU2po="},
|
||||
} {
|
||||
csumHdr := fmt.Sprintf("x-amz-checksum-%s", test.key)
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "7",
|
||||
"x-amz-sdk-checksum-algorithm": strings.ToUpper(test.key),
|
||||
"x-amz-trailer": csumHdr,
|
||||
}
|
||||
body := bytes.NewBuffer([]byte("7\r\nabcdefg\r\n0\r\n"))
|
||||
|
||||
_, err := body.WriteString(fmt.Sprintf("%s:%s\r\n\r\n", csumHdr, test.value))
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
headers, apiErr, err := testUnsignedStreamingPayloadTrailerObjectPut(s, bucket, object, body.Bytes(), reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if apiErr != nil {
|
||||
return fmt.Errorf("test %v failed: (%s) %s", i+1, apiErr.Code, apiErr.Message)
|
||||
}
|
||||
|
||||
if headers[csumHdr] != test.value {
|
||||
return fmt.Errorf("expected the %s to be %s, instead got %s", csumHdr, test.value, headers[csumHdr])
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_UploadPart_no_trailer_composite_checksum(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_UploadPart_no_trailer_composite_checksum"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
mp, err := createMp(s3client, bucket, object, withChecksumType(types.ChecksumTypeComposite), withChecksum(types.ChecksumAlgorithmCrc32))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "7",
|
||||
}
|
||||
|
||||
body := []byte("7\r\nabcdefg\r\n0\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerUploadPart(s, bucket, object, mp.UploadId, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return compareS3ApiError(s3err.GetChecksumTypeMismatchErr(types.ChecksumAlgorithmCrc32, "null"), apiErr)
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_UploadPart_no_trailer_full_object(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_UploadPart_no_trailer_composite_checksum"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
mp, err := createMp(s3client, bucket, object, withChecksumType(types.ChecksumTypeFullObject), withChecksum(types.ChecksumAlgorithmCrc32))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "7",
|
||||
}
|
||||
|
||||
body := []byte("7\r\nabcdefg\r\n0\r\n\r\n")
|
||||
|
||||
headers, apiErr, err := testUnsignedStreamingPayloadTrailerUploadPart(s, bucket, object, mp.UploadId, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if apiErr != nil {
|
||||
return fmt.Errorf("(%s) %s", apiErr.Code, apiErr.Message)
|
||||
}
|
||||
|
||||
expectedCsum := "MSpqpg=="
|
||||
actualCsum := headers["x-amz-checksum-crc32"]
|
||||
|
||||
if expectedCsum != actualCsum {
|
||||
return fmt.Errorf("expected the crc32 checksum to be %s, instead got %s", expectedCsum, actualCsum)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_UploadPart_trailer_and_mp_algo_mismatch(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_UploadPart_trailer_and_mp_algo_mismatch"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
mp, err := createMp(s3client, bucket, object, withChecksumType(types.ChecksumTypeFullObject), withChecksum(types.ChecksumAlgorithmCrc32))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "7",
|
||||
"x-amz-trailer": "x-amz-checksum-sha256",
|
||||
}
|
||||
|
||||
body := []byte("7\r\nabcdefg\r\n0\r\nx-amz-checksum-sha256:fRpUEnsiJQL1t5tfsIAwYRUqRPkrN+I8ZSe69mXU2po=\r\n\r\n")
|
||||
|
||||
_, apiErr, err := testUnsignedStreamingPayloadTrailerUploadPart(s, bucket, object, mp.UploadId, body, reqHeaders)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return compareS3ApiError(s3err.GetChecksumTypeMismatchErr(types.ChecksumAlgorithmCrc32, types.ChecksumAlgorithmSha256), apiErr)
|
||||
})
|
||||
}
|
||||
|
||||
func UnsignedStreamingPayloadTrailer_UploadPart_success_with_trailer(s *S3Conf) error {
|
||||
testName := "UnsignedStreamingPayloadTrailer_UploadPart_success_with_trailer"
|
||||
return actionHandler(s, testName, func(s3client *s3.Client, bucket string) error {
|
||||
object := "my-object"
|
||||
|
||||
for i, test := range []struct {
|
||||
key string
|
||||
value string
|
||||
}{
|
||||
{"crc32", "QWaN2w=="},
|
||||
{"crc32c", "R/I7iQ=="},
|
||||
{"crc64nvme", "dPVWc2vU1+Q="},
|
||||
{"sha1", "YR/1TvTYOJz5gtqVFoBJBtmTibY="},
|
||||
{"sha256", "eXuwq/95jXIAr3aF3KeQHt/8Ur8mUA1b2XKCZY7iQVI="},
|
||||
} {
|
||||
mp, err := createMp(s3client, bucket, object, withChecksum(types.ChecksumAlgorithm(strings.ToUpper(test.key))))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
csumHdr := fmt.Sprintf("x-amz-checksum-%s", test.key)
|
||||
reqHeaders := map[string]string{
|
||||
"x-amz-decoded-content-length": "10",
|
||||
"x-amz-sdk-checksum-algorithm": test.key,
|
||||
"x-amz-trailer": csumHdr,
|
||||
}
|
||||
body := bytes.NewBuffer([]byte("A\r\ndummy data\r\n0\r\n"))
|
||||
|
||||
_, err = body.WriteString(fmt.Sprintf("%s:%s\r\n\r\n", csumHdr, test.value))
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
headers, apiErr, err := testUnsignedStreamingPayloadTrailerUploadPart(s, bucket, object, mp.UploadId, body.Bytes(), reqHeaders)
|
||||
if err != nil {
|
||||
return fmt.Errorf("test %v failed: %w", i+1, err)
|
||||
}
|
||||
|
||||
if apiErr != nil {
|
||||
return fmt.Errorf("test %v failed: (%s) %s", i+1, apiErr.Code, apiErr.Message)
|
||||
}
|
||||
|
||||
if headers[csumHdr] != test.value {
|
||||
return fmt.Errorf("expected the %s to be %s, instead got %s", csumHdr, test.value, headers[csumHdr])
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
@@ -371,11 +371,19 @@ func checkHTTPResponseApiErr(resp *http.Response, apiErr s3err.APIError) error {
|
||||
if resp.StatusCode != apiErr.HTTPStatusCode {
|
||||
return fmt.Errorf("expected response status code to be %v, instead got %v", apiErr.HTTPStatusCode, resp.StatusCode)
|
||||
}
|
||||
if errResp.Code != apiErr.Code {
|
||||
return fmt.Errorf("expected error code to be %v, instead got %v", apiErr.Code, errResp.Code)
|
||||
return compareS3ApiError(apiErr, &errResp)
|
||||
}
|
||||
|
||||
func compareS3ApiError(expected s3err.APIError, received *s3err.APIErrorResponse) error {
|
||||
if received == nil {
|
||||
return fmt.Errorf("expected %w, received nil", expected)
|
||||
}
|
||||
if errResp.Message != apiErr.Description {
|
||||
return fmt.Errorf("expected error message to be %v, instead got %v", apiErr.Description, errResp.Message)
|
||||
|
||||
if received.Code != expected.Code {
|
||||
return fmt.Errorf("expected error code to be %v, instead got %v", expected.Code, received.Code)
|
||||
}
|
||||
if received.Message != expected.Description {
|
||||
return fmt.Errorf("expected error message to be %v, instead got %v", expected.Description, received.Message)
|
||||
}
|
||||
|
||||
return nil
|
||||
@@ -2010,3 +2018,78 @@ func putBucketPolicy(client *s3.Client, bucket, policy string) error {
|
||||
cancel()
|
||||
return err
|
||||
}
|
||||
|
||||
func sendSignedRequest(s *S3Conf, req *http.Request, cancel context.CancelFunc) (map[string]string, *s3err.APIErrorResponse, error) {
|
||||
signer := v4.NewSigner()
|
||||
signErr := signer.SignHTTP(req.Context(), aws.Credentials{AccessKeyID: s.awsID, SecretAccessKey: s.awsSecret}, req, "STREAMING-UNSIGNED-PAYLOAD-TRAILER", "s3", s.awsRegion, time.Now())
|
||||
if signErr != nil {
|
||||
cancel()
|
||||
return nil, nil, fmt.Errorf("failed to sign the request: %w", signErr)
|
||||
}
|
||||
|
||||
resp, err := s.httpClient.Do(req)
|
||||
cancel()
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("failed to send the request: %w", err)
|
||||
}
|
||||
|
||||
if resp.StatusCode >= 300 {
|
||||
defer resp.Body.Close()
|
||||
bodyBytes, err := io.ReadAll(resp.Body)
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("failed to read the request body: %w", err)
|
||||
}
|
||||
|
||||
var errResp s3err.APIErrorResponse
|
||||
err = xml.Unmarshal(bodyBytes, &errResp)
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("failed to unmarshal response body: %w", err)
|
||||
}
|
||||
|
||||
return nil, &errResp, nil
|
||||
}
|
||||
|
||||
headers := map[string]string{}
|
||||
for key, val := range resp.Header {
|
||||
headers[strings.ToLower(key)] = val[0]
|
||||
}
|
||||
|
||||
return headers, nil, nil
|
||||
}
|
||||
|
||||
func testUnsignedStreamingPayloadTrailerObjectPut(s *S3Conf, bucket, object string, body []byte, reqHeaders map[string]string) (map[string]string, *s3err.APIErrorResponse, error) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), shortTimeout)
|
||||
req, err := http.NewRequestWithContext(ctx, http.MethodPut, s.endpoint+"/"+bucket+"/"+object, bytes.NewReader(body))
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, nil, fmt.Errorf("failed to create a request: %w", err)
|
||||
}
|
||||
|
||||
req.Header.Add("x-amz-content-sha256", "STREAMING-UNSIGNED-PAYLOAD-TRAILER")
|
||||
for key, val := range reqHeaders {
|
||||
req.Header.Add(key, val)
|
||||
}
|
||||
|
||||
return sendSignedRequest(s, req, cancel)
|
||||
}
|
||||
|
||||
func testUnsignedStreamingPayloadTrailerUploadPart(s *S3Conf, bucket, object string, uploadId *string, body []byte, reqHeaders map[string]string) (map[string]string, *s3err.APIErrorResponse, error) {
|
||||
if uploadId == nil {
|
||||
return nil, nil, fmt.Errorf("empty upload id")
|
||||
}
|
||||
|
||||
uri := fmt.Sprintf("%s/%s/%s?uploadId=%s&partNumber=%v", s.endpoint, bucket, object, *uploadId, 1)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), shortTimeout)
|
||||
req, err := http.NewRequestWithContext(ctx, http.MethodPut, uri, bytes.NewReader(body))
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, nil, fmt.Errorf("failed to create a request: %w", err)
|
||||
}
|
||||
|
||||
req.Header.Add("x-amz-content-sha256", "STREAMING-UNSIGNED-PAYLOAD-TRAILER")
|
||||
for key, val := range reqHeaders {
|
||||
req.Header.Add(key, val)
|
||||
}
|
||||
|
||||
return sendSignedRequest(s, req, cancel)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user