chunked_reader_v4.go 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686
  1. package s3api
  2. // the related code is copied and modified from minio source code
  3. /*
  4. * Minio Cloud Storage, (C) 2016 Minio, Inc.
  5. *
  6. * Licensed under the Apache License, Version 2.0 (the "License");
  7. * you may not use this file except in compliance with the License.
  8. * You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. import (
  19. "bufio"
  20. "bytes"
  21. "crypto/sha1"
  22. "crypto/sha256"
  23. "encoding/base64"
  24. "encoding/hex"
  25. "errors"
  26. "fmt"
  27. "hash"
  28. "hash/crc32"
  29. "io"
  30. "net/http"
  31. "time"
  32. "github.com/seaweedfs/seaweedfs/weed/glog"
  33. "github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
  34. "github.com/seaweedfs/seaweedfs/weed/s3api/s3err"
  35. "github.com/dustin/go-humanize"
  36. "github.com/minio/crc64nvme"
  37. )
  38. // calculateSeedSignature - Calculate seed signature in accordance with
  39. // - http://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-streaming.html
  40. //
  41. // returns signature, error otherwise if the signature mismatches or any other
  42. // error while parsing and validating.
  43. func (iam *IdentityAccessManagement) calculateSeedSignature(r *http.Request) (cred *Credential, signature string, region string, service string, date time.Time, errCode s3err.ErrorCode) {
  44. // Copy request.
  45. req := *r
  46. // Save authorization header.
  47. v4Auth := req.Header.Get("Authorization")
  48. // Parse signature version '4' header.
  49. signV4Values, errCode := parseSignV4(v4Auth)
  50. if errCode != s3err.ErrNone {
  51. return nil, "", "", "", time.Time{}, errCode
  52. }
  53. contentSha256Header := req.Header.Get("X-Amz-Content-Sha256")
  54. switch contentSha256Header {
  55. // Payload for STREAMING signature should be 'STREAMING-AWS4-HMAC-SHA256-PAYLOAD'
  56. case streamingContentSHA256:
  57. glog.V(3).Infof("streaming content sha256")
  58. case streamingUnsignedPayload:
  59. glog.V(3).Infof("streaming unsigned payload")
  60. default:
  61. return nil, "", "", "", time.Time{}, s3err.ErrContentSHA256Mismatch
  62. }
  63. // Payload streaming.
  64. payload := contentSha256Header
  65. // Extract all the signed headers along with its values.
  66. extractedSignedHeaders, errCode := extractSignedHeaders(signV4Values.SignedHeaders, r)
  67. if errCode != s3err.ErrNone {
  68. return nil, "", "", "", time.Time{}, errCode
  69. }
  70. // Verify if the access key id matches.
  71. identity, cred, found := iam.lookupByAccessKey(signV4Values.Credential.accessKey)
  72. if !found {
  73. return nil, "", "", "", time.Time{}, s3err.ErrInvalidAccessKeyID
  74. }
  75. bucket, object := s3_constants.GetBucketAndObject(r)
  76. if !identity.canDo(s3_constants.ACTION_WRITE, bucket, object) {
  77. errCode = s3err.ErrAccessDenied
  78. return
  79. }
  80. // Verify if region is valid.
  81. region = signV4Values.Credential.scope.region
  82. // Extract date, if not present throw error.
  83. var dateStr string
  84. if dateStr = req.Header.Get(http.CanonicalHeaderKey("x-amz-date")); dateStr == "" {
  85. if dateStr = r.Header.Get("Date"); dateStr == "" {
  86. return nil, "", "", "", time.Time{}, s3err.ErrMissingDateHeader
  87. }
  88. }
  89. // Parse date header.
  90. date, err := time.Parse(iso8601Format, dateStr)
  91. if err != nil {
  92. return nil, "", "", "", time.Time{}, s3err.ErrMalformedDate
  93. }
  94. // Query string.
  95. queryStr := req.URL.Query().Encode()
  96. // Get canonical request.
  97. canonicalRequest := getCanonicalRequest(extractedSignedHeaders, payload, queryStr, req.URL.Path, req.Method)
  98. // Get string to sign from canonical request.
  99. stringToSign := getStringToSign(canonicalRequest, date, signV4Values.Credential.getScope())
  100. // Get hmac signing key.
  101. signingKey := getSigningKey(cred.SecretKey, signV4Values.Credential.scope.date.Format(yyyymmdd), region, signV4Values.Credential.scope.service)
  102. // Calculate signature.
  103. newSignature := getSignature(signingKey, stringToSign)
  104. // Verify if signature match.
  105. if !compareSignatureV4(newSignature, signV4Values.Signature) {
  106. return nil, "", "", "", time.Time{}, s3err.ErrSignatureDoesNotMatch
  107. }
  108. // Return calculated signature.
  109. return cred, newSignature, region, signV4Values.Credential.scope.service, date, s3err.ErrNone
  110. }
  111. const maxLineLength = 4 * humanize.KiByte // assumed <= bufio.defaultBufSize 4KiB
  112. // lineTooLong is generated as chunk header is bigger than 4KiB.
  113. var errLineTooLong = errors.New("header line too long")
  114. // Malformed encoding is generated when chunk header is wrongly formed.
  115. var errMalformedEncoding = errors.New("malformed chunked encoding")
  116. // newChunkedReader returns a new s3ChunkedReader that translates the data read from r
  117. // out of HTTP "chunked" format before returning it.
  118. // The s3ChunkedReader returns io.EOF when the final 0-length chunk is read.
  119. func (iam *IdentityAccessManagement) newChunkedReader(req *http.Request) (io.ReadCloser, s3err.ErrorCode) {
  120. glog.V(3).Infof("creating a new newSignV4ChunkedReader")
  121. contentSha256Header := req.Header.Get("X-Amz-Content-Sha256")
  122. authorizationHeader := req.Header.Get("Authorization")
  123. var ident *Credential
  124. var seedSignature, region, service string
  125. var seedDate time.Time
  126. var errCode s3err.ErrorCode
  127. switch contentSha256Header {
  128. // Payload for STREAMING signature should be 'STREAMING-AWS4-HMAC-SHA256-PAYLOAD'
  129. case streamingContentSHA256:
  130. glog.V(3).Infof("streaming content sha256")
  131. ident, seedSignature, region, service, seedDate, errCode = iam.calculateSeedSignature(req)
  132. if errCode != s3err.ErrNone {
  133. return nil, errCode
  134. }
  135. case streamingUnsignedPayload:
  136. glog.V(3).Infof("streaming unsigned payload")
  137. if authorizationHeader != "" {
  138. // We do not need to pass the seed signature to the Reader as each chunk is not signed,
  139. // but we do compute it to verify the caller has the correct permissions.
  140. _, _, _, _, _, errCode = iam.calculateSeedSignature(req)
  141. if errCode != s3err.ErrNone {
  142. return nil, errCode
  143. }
  144. }
  145. }
  146. // Get the checksum algorithm from the x-amz-trailer Header.
  147. amzTrailerHeader := req.Header.Get("x-amz-trailer")
  148. checksumAlgorithm, err := extractChecksumAlgorithm(amzTrailerHeader)
  149. if err != nil {
  150. glog.V(3).Infof("error extracting checksum algorithm: %v", err)
  151. return nil, s3err.ErrInvalidRequest
  152. }
  153. checkSumWriter := getCheckSumWriter(checksumAlgorithm)
  154. return &s3ChunkedReader{
  155. cred: ident,
  156. reader: bufio.NewReader(req.Body),
  157. seedSignature: seedSignature,
  158. seedDate: seedDate,
  159. region: region,
  160. service: service,
  161. chunkSHA256Writer: sha256.New(),
  162. checkSumAlgorithm: checksumAlgorithm.String(),
  163. checkSumWriter: checkSumWriter,
  164. state: readChunkHeader,
  165. iam: iam,
  166. }, s3err.ErrNone
  167. }
  168. func extractChecksumAlgorithm(amzTrailerHeader string) (ChecksumAlgorithm, error) {
  169. // Extract checksum algorithm from the x-amz-trailer header.
  170. switch amzTrailerHeader {
  171. case "x-amz-checksum-crc32":
  172. return ChecksumAlgorithmCRC32, nil
  173. case "x-amz-checksum-crc32c":
  174. return ChecksumAlgorithmCRC32C, nil
  175. case "x-amz-checksum-crc64nvme":
  176. return ChecksumAlgorithmCRC64NVMe, nil
  177. case "x-amz-checksum-sha1":
  178. return ChecksumAlgorithmSHA1, nil
  179. case "x-amz-checksum-sha256":
  180. return ChecksumAlgorithmSHA256, nil
  181. case "":
  182. return ChecksumAlgorithmNone, nil
  183. default:
  184. return ChecksumAlgorithmNone, errors.New("unsupported checksum algorithm '" + amzTrailerHeader + "'")
  185. }
  186. }
  187. // Represents the overall state that is required for decoding a
  188. // AWS Signature V4 chunked reader.
  189. type s3ChunkedReader struct {
  190. cred *Credential
  191. reader *bufio.Reader
  192. seedSignature string
  193. seedDate time.Time
  194. region string
  195. service string // Service from credential scope (e.g., "s3", "iam")
  196. state chunkState
  197. lastChunk bool
  198. chunkSignature string // Empty string if unsigned streaming upload.
  199. checkSumAlgorithm string // Empty string if no checksum algorithm is specified.
  200. checkSumWriter hash.Hash
  201. chunkSHA256Writer hash.Hash // Calculates sha256 of chunk data.
  202. n uint64 // Unread bytes in chunk
  203. err error
  204. iam *IdentityAccessManagement
  205. }
  206. // Read chunk reads the chunk token signature portion.
  207. func (cr *s3ChunkedReader) readS3ChunkHeader() {
  208. // Read the first chunk line until CRLF.
  209. var bytesRead, hexChunkSize, hexChunkSignature []byte
  210. bytesRead, cr.err = readChunkLine(cr.reader)
  211. // Parse s3 specific chunk extension and fetch the values.
  212. hexChunkSize, hexChunkSignature = parseS3ChunkExtension(bytesRead)
  213. if cr.err != nil {
  214. return
  215. }
  216. // <hex>;token=value - converts the hex into its uint64 form.
  217. cr.n, cr.err = parseHexUint(hexChunkSize)
  218. if cr.err != nil {
  219. return
  220. }
  221. if cr.n == 0 {
  222. cr.err = io.EOF
  223. }
  224. // Save the incoming chunk signature.
  225. if hexChunkSignature == nil {
  226. // We are using unsigned streaming upload.
  227. cr.chunkSignature = ""
  228. } else {
  229. cr.chunkSignature = string(hexChunkSignature)
  230. }
  231. }
  232. type chunkState int
  233. const (
  234. readChunkHeader chunkState = iota
  235. readChunkTrailer
  236. readChunk
  237. readTrailerChunk
  238. verifyChunk
  239. verifyChecksum
  240. eofChunk
  241. )
  242. func (cs chunkState) String() string {
  243. stateString := ""
  244. switch cs {
  245. case readChunkHeader:
  246. stateString = "readChunkHeader"
  247. case readChunkTrailer:
  248. stateString = "readChunkTrailer"
  249. case readChunk:
  250. stateString = "readChunk"
  251. case readTrailerChunk:
  252. stateString = "readTrailerChunk"
  253. case verifyChunk:
  254. stateString = "verifyChunk"
  255. case verifyChecksum:
  256. stateString = "verifyChecksum"
  257. case eofChunk:
  258. stateString = "eofChunk"
  259. }
  260. return stateString
  261. }
  262. func (cr *s3ChunkedReader) Close() (err error) {
  263. return nil
  264. }
  265. // Read - implements `io.Reader`, which transparently decodes
  266. // the incoming AWS Signature V4 streaming signature.
  267. func (cr *s3ChunkedReader) Read(buf []byte) (n int, err error) {
  268. for {
  269. switch cr.state {
  270. case readChunkHeader:
  271. cr.readS3ChunkHeader()
  272. // If we're at the end of a chunk.
  273. if cr.n == 0 && cr.err == io.EOF {
  274. cr.state = readChunkTrailer
  275. cr.lastChunk = true
  276. continue
  277. }
  278. if cr.err != nil {
  279. return 0, cr.err
  280. }
  281. cr.state = readChunk
  282. case readChunkTrailer:
  283. err = peekCRLF(cr.reader)
  284. isTrailingChunk := cr.n == 0 && cr.lastChunk
  285. if !isTrailingChunk {
  286. // If we're not in the trailing chunk, we should consume the bytes no matter what.
  287. // The error returned by peekCRLF is the same as the one by readCRLF.
  288. readCRLF(cr.reader)
  289. cr.err = err
  290. } else if err != nil && err != errMalformedEncoding {
  291. cr.err = err
  292. return 0, errMalformedEncoding
  293. } else { // equivalent to isTrailingChunk && err == errMalformedEncoding
  294. // FIXME: The "right" structure of the last chunk as provided by the examples in the
  295. // AWS documentation is "0\r\n\r\n" instead of "0\r\n", but some s3 clients when calling with
  296. // streaming-unsigned-payload-trailer omit the last CRLF. To avoid returning an error that, we need to accept both.
  297. // We arrive here when we're at the end of the 0-byte chunk, depending on the client implementation
  298. // the client may or may not send the optional CRLF after the 0-byte chunk.
  299. // If the client sends the optional CRLF, we should consume it.
  300. if err == nil {
  301. readCRLF(cr.reader)
  302. }
  303. }
  304. // If we're using unsigned streaming upload, there is no signature to verify at each chunk.
  305. if cr.chunkSignature != "" {
  306. cr.state = verifyChunk
  307. } else if cr.lastChunk {
  308. cr.state = readTrailerChunk
  309. } else {
  310. cr.state = readChunkHeader
  311. }
  312. case readTrailerChunk:
  313. // When using unsigned upload, this would be the raw contents of the trailer chunk:
  314. //
  315. // x-amz-checksum-crc32:YABb/g==\n\r\n\r\n // Trailer chunk (note optional \n character)
  316. // \r\n // CRLF
  317. //
  318. // When using signed upload with an additional checksum algorithm, this would be the raw contents of the trailer chunk:
  319. //
  320. // x-amz-checksum-crc32:YABb/g==\n\r\n // Trailer chunk (note optional \n character)
  321. // trailer-signature\r\n
  322. // \r\n // CRLF
  323. //
  324. // This implementation currently only supports the first case.
  325. // TODO: Implement the second case (signed upload with additional checksum computation for each chunk)
  326. extractedCheckSumAlgorithm, extractedChecksum := parseChunkChecksum(cr.reader)
  327. if extractedCheckSumAlgorithm.String() != cr.checkSumAlgorithm {
  328. errorMessage := fmt.Sprintf("checksum algorithm in trailer '%s' does not match the one advertised in the header '%s'", extractedCheckSumAlgorithm.String(), cr.checkSumAlgorithm)
  329. glog.V(3).Info(errorMessage)
  330. cr.err = errors.New(s3err.ErrMsgChecksumAlgorithmMismatch)
  331. return 0, cr.err
  332. }
  333. computedChecksum := cr.checkSumWriter.Sum(nil)
  334. base64Checksum := base64.StdEncoding.EncodeToString(computedChecksum)
  335. if string(extractedChecksum) != base64Checksum {
  336. glog.V(3).Infof("payload checksum '%s' does not match provided checksum '%s'", base64Checksum, string(extractedChecksum))
  337. cr.err = errors.New(s3err.ErrMsgPayloadChecksumMismatch)
  338. return 0, cr.err
  339. }
  340. // TODO: Extract signature from trailer chunk and verify it.
  341. // For now, we just read the trailer chunk and discard it.
  342. // Reading remaining CRLF.
  343. for i := 0; i < 2; i++ {
  344. cr.err = readCRLF(cr.reader)
  345. }
  346. cr.state = eofChunk
  347. case readChunk:
  348. // There is no more space left in the request buffer.
  349. if len(buf) == 0 {
  350. return n, nil
  351. }
  352. rbuf := buf
  353. // The request buffer is larger than the current chunk size.
  354. // Read only the current chunk from the underlying reader.
  355. if uint64(len(rbuf)) > cr.n {
  356. rbuf = rbuf[:cr.n]
  357. }
  358. var n0 int
  359. n0, cr.err = cr.reader.Read(rbuf)
  360. if cr.err != nil {
  361. // We have lesser than chunk size advertised in chunkHeader, this is 'unexpected'.
  362. if cr.err == io.EOF {
  363. cr.err = io.ErrUnexpectedEOF
  364. }
  365. return 0, cr.err
  366. }
  367. // Calculate sha256.
  368. cr.chunkSHA256Writer.Write(rbuf[:n0])
  369. // Compute checksum
  370. if cr.checkSumWriter != nil {
  371. cr.checkSumWriter.Write(rbuf[:n0])
  372. }
  373. // Update the bytes read into request buffer so far.
  374. n += n0
  375. buf = buf[n0:]
  376. // Update bytes to be read of the current chunk before verifying chunk's signature.
  377. cr.n -= uint64(n0)
  378. // If we're at the end of a chunk.
  379. if cr.n == 0 {
  380. cr.state = readChunkTrailer
  381. continue
  382. }
  383. case verifyChunk:
  384. // Check if we have credentials for signature verification
  385. // This handles the case where we have unsigned streaming (no cred) but chunks contain signatures
  386. //
  387. // BUG FIX for GitHub issue #6847:
  388. // Some AWS SDK versions (Java 3.7.412+, .NET 4.0.0-preview.6+) send mixed format:
  389. // - HTTP headers indicate unsigned streaming (STREAMING-UNSIGNED-PAYLOAD-TRAILER)
  390. // - But chunk data contains chunk-signature headers (normally only for signed streaming)
  391. // This causes a nil pointer dereference when trying to verify signatures without credentials
  392. if cr.cred != nil {
  393. // Normal signed streaming - verify the chunk signature
  394. // Calculate the hashed chunk.
  395. hashedChunk := hex.EncodeToString(cr.chunkSHA256Writer.Sum(nil))
  396. // Calculate the chunk signature.
  397. newSignature := cr.getChunkSignature(hashedChunk)
  398. if !compareSignatureV4(cr.chunkSignature, newSignature) {
  399. // Chunk signature doesn't match we return signature does not match.
  400. cr.err = errors.New(s3err.ErrMsgChunkSignatureMismatch)
  401. return 0, cr.err
  402. }
  403. // Newly calculated signature becomes the seed for the next chunk
  404. // this follows the chaining.
  405. cr.seedSignature = newSignature
  406. } else {
  407. // For unsigned streaming, we should not verify chunk signatures even if they are present
  408. // This fixes the bug where AWS SDKs send chunk signatures with unsigned streaming headers
  409. glog.V(3).Infof("Skipping chunk signature verification for unsigned streaming")
  410. }
  411. // Common cleanup and state transition for both signed and unsigned streaming
  412. cr.chunkSHA256Writer.Reset()
  413. if cr.lastChunk {
  414. cr.state = eofChunk
  415. } else {
  416. cr.state = readChunkHeader
  417. }
  418. case eofChunk:
  419. return n, io.EOF
  420. }
  421. }
  422. }
  423. // getChunkSignature - get chunk signature.
  424. func (cr *s3ChunkedReader) getChunkSignature(hashedChunk string) string {
  425. // Calculate string to sign.
  426. stringToSign := signV4Algorithm + "-PAYLOAD" + "\n" +
  427. cr.seedDate.Format(iso8601Format) + "\n" +
  428. getScope(cr.seedDate, cr.region, cr.service) + "\n" +
  429. cr.seedSignature + "\n" +
  430. emptySHA256 + "\n" +
  431. hashedChunk
  432. // Get hmac signing key.
  433. signingKey := getSigningKey(cr.cred.SecretKey, cr.seedDate.Format(yyyymmdd), cr.region, cr.service)
  434. // Calculate and return signature.
  435. return getSignature(signingKey, stringToSign)
  436. }
  437. func readCRLF(reader *bufio.Reader) error {
  438. buf := make([]byte, 2)
  439. _, err := io.ReadFull(reader, buf)
  440. if err != nil {
  441. return err
  442. }
  443. return checkCRLF(buf)
  444. }
  445. func peekCRLF(reader *bufio.Reader) error {
  446. buf, err := reader.Peek(2)
  447. if err != nil {
  448. return err
  449. }
  450. if err := checkCRLF(buf); err != nil {
  451. return err
  452. }
  453. return nil
  454. }
  455. func checkCRLF(buf []byte) error {
  456. if len(buf) != 2 || buf[0] != '\r' || buf[1] != '\n' {
  457. return errMalformedEncoding
  458. }
  459. return nil
  460. }
  461. func readChunkLine(b *bufio.Reader) ([]byte, error) {
  462. buf, err := b.ReadSlice('\n')
  463. if err != nil {
  464. // We always know when EOF is coming.
  465. // If the caller asked for a line, there should be a line.
  466. switch err {
  467. case io.EOF:
  468. err = io.ErrUnexpectedEOF
  469. case bufio.ErrBufferFull:
  470. err = errLineTooLong
  471. }
  472. return nil, err
  473. }
  474. if len(buf) >= maxLineLength {
  475. return nil, errLineTooLong
  476. }
  477. return trimTrailingWhitespace(buf), nil
  478. }
  479. // trimTrailingWhitespace - trim trailing white space.
  480. func trimTrailingWhitespace(b []byte) []byte {
  481. for len(b) > 0 && isASCIISpace(b[len(b)-1]) {
  482. b = b[:len(b)-1]
  483. }
  484. return b
  485. }
  486. // isASCIISpace - is ascii space?
  487. func isASCIISpace(b byte) bool {
  488. return b == ' ' || b == '\t' || b == '\n' || b == '\r'
  489. }
  490. // Constant s3 chunk encoding signature.
  491. const s3ChunkSignatureStr = ";chunk-signature="
  492. // parseS3ChunkExtension removes any s3 specific chunk-extension from buf.
  493. // For example,
  494. //
  495. // "10000;chunk-signature=..." => "10000", "chunk-signature=..."
  496. func parseS3ChunkExtension(buf []byte) ([]byte, []byte) {
  497. buf = trimTrailingWhitespace(buf)
  498. semi := bytes.Index(buf, []byte(s3ChunkSignatureStr))
  499. // Chunk signature not found, return the whole buffer.
  500. // This means we're using unsigned streaming upload.
  501. if semi == -1 {
  502. return buf, nil
  503. }
  504. return buf[:semi], parseChunkSignature(buf[semi:])
  505. }
  506. func parseChunkChecksum(b *bufio.Reader) (ChecksumAlgorithm, []byte) {
  507. // When using unsigned upload, this would be the raw contents of the trailer chunk:
  508. //
  509. // x-amz-checksum-crc32:YABb/g==\n\r\n\r\n // Trailer chunk (note optional \n character)
  510. // \r\n // CRLF
  511. //
  512. // When using signed upload with an additional checksum algorithm, this would be the raw contents of the trailer chunk:
  513. //
  514. // x-amz-checksum-crc32:YABb/g==\n\r\n // Trailer chunk (note optional \n character)
  515. // trailer-signature\r\n
  516. // \r\n // CRLF
  517. //
  518. // x-amz-checksum-crc32:YABb/g==\n
  519. bytesRead, err := readChunkLine(b)
  520. if err != nil {
  521. return ChecksumAlgorithmNone, nil
  522. }
  523. // Split on ':'
  524. parts := bytes.SplitN(bytesRead, []byte(":"), 2)
  525. checksumKey := string(parts[0])
  526. checksumValue := parts[1]
  527. // Discard all trailing whitespace characters
  528. checksumValue = trimTrailingWhitespace(checksumValue)
  529. // If the checksum key is not a supported checksum algorithm, return an error.
  530. // TODO: Bubble that error up to the caller
  531. extractedAlgorithm, err := extractChecksumAlgorithm(checksumKey)
  532. if err != nil {
  533. return ChecksumAlgorithmNone, nil
  534. }
  535. return extractedAlgorithm, checksumValue
  536. }
  537. func parseChunkSignature(chunk []byte) []byte {
  538. chunkSplits := bytes.SplitN(chunk, []byte("="), 2)
  539. return chunkSplits[1] // Keep only the signature.
  540. }
  541. func parseHexUint(v []byte) (n uint64, err error) {
  542. for i, b := range v {
  543. switch {
  544. case '0' <= b && b <= '9':
  545. b = b - '0'
  546. case 'a' <= b && b <= 'f':
  547. b = b - 'a' + 10
  548. case 'A' <= b && b <= 'F':
  549. b = b - 'A' + 10
  550. default:
  551. return 0, errors.New("invalid byte in chunk length")
  552. }
  553. if i == 16 {
  554. return 0, errors.New("http chunk length too large")
  555. }
  556. n <<= 4
  557. n |= uint64(b)
  558. }
  559. return
  560. }
  561. // Checksum Algorithm represents the various checksum algorithms supported.
  562. type ChecksumAlgorithm int
  563. const (
  564. ChecksumAlgorithmNone ChecksumAlgorithm = iota
  565. ChecksumAlgorithmCRC32
  566. ChecksumAlgorithmCRC32C
  567. ChecksumAlgorithmCRC64NVMe
  568. ChecksumAlgorithmSHA1
  569. ChecksumAlgorithmSHA256
  570. )
  571. func (ca ChecksumAlgorithm) String() string {
  572. switch ca {
  573. case ChecksumAlgorithmNone:
  574. return ""
  575. case ChecksumAlgorithmCRC32:
  576. return "x-amz-checksum-crc32"
  577. case ChecksumAlgorithmCRC32C:
  578. return "x-amz-checksum-crc32c"
  579. case ChecksumAlgorithmCRC64NVMe:
  580. return "x-amz-checksum-crc64nvme"
  581. case ChecksumAlgorithmSHA1:
  582. return "x-amz-checksum-sha1"
  583. case ChecksumAlgorithmSHA256:
  584. return "x-amz-checksum-sha256"
  585. }
  586. return ""
  587. }
  588. // getCheckSumWriter - get checksum writer.
  589. func getCheckSumWriter(checksumAlgorithm ChecksumAlgorithm) hash.Hash {
  590. switch checksumAlgorithm {
  591. case ChecksumAlgorithmCRC32:
  592. return crc32.NewIEEE()
  593. case ChecksumAlgorithmCRC32C:
  594. return crc32.New(crc32.MakeTable(crc32.Castagnoli))
  595. case ChecksumAlgorithmCRC64NVMe:
  596. return crc64nvme.New()
  597. case ChecksumAlgorithmSHA1:
  598. return sha1.New()
  599. case ChecksumAlgorithmSHA256:
  600. return sha256.New()
  601. }
  602. return nil
  603. }