From 7398f3a14e5207f1ba3397c02b06de745180ad70 Mon Sep 17 00:00:00 2001 From: Henk-Jan Lebbink Date: Tue, 6 Jan 2026 18:43:25 +0100 Subject: [PATCH] added CRC32, CRC32C, SHA1, SHA256 and CRC64NVME (#195) --- Cargo.toml | 10 +- benches/s3/bench_checksums.rs | 53 + macros/src/lib.rs | 8 + macros/src/test_attr.rs | 35 +- src/s3/aws_chunked.rs | 999 +++++++++++++++ src/s3/builders/append_object.rs | 38 +- src/s3/builders/copy_object.rs | 67 +- src/s3/builders/put_object.rs | 169 ++- src/s3/client/mod.rs | 146 ++- src/s3/error.rs | 3 + src/s3/mod.rs | 1 + src/s3/response/append_object.rs | 4 +- src/s3/response/copy_object.rs | 5 +- src/s3/response/get_object.rs | 284 ++++- src/s3/response/put_object.rs | 7 +- src/s3/response/stat_object.rs | 5 +- src/s3/response_traits.rs | 89 +- src/s3/signer.rs | 419 ++++++ src/s3/types/basic_types.rs | 25 +- src/s3/types/header_constants.rs | 28 + src/s3/types/s3_request.rs | 17 + src/s3/utils.rs | 385 +++++- tests/s3/mod.rs | 2 + tests/s3/object_checksums.rs | 2031 ++++++++++++++++++++++++++++++ tests/s3/test_checksums.rs | 267 ++++ 25 files changed, 5015 insertions(+), 82 deletions(-) create mode 100644 benches/s3/bench_checksums.rs create mode 100644 src/s3/aws_chunked.rs create mode 100644 tests/s3/object_checksums.rs create mode 100644 tests/s3/test_checksums.rs diff --git a/Cargo.toml b/Cargo.toml index 3c64921..d19f0b9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,7 +24,7 @@ http2 = ["reqwest/http2"] localhost = [] [workspace.dependencies] -uuid = "1.18" +uuid = "1.19" futures-util = "0.3" futures-io = "0.3" reqwest = { version = "0.12", default-features = false } @@ -49,7 +49,7 @@ async-stream = "0.3" async-trait = "0.1" base64 = "0.22" chrono = { workspace = true, features = ["serde"] } -crc = "3.4" +crc-fast = "1.8" dashmap = "6.1.0" env_logger = "0.11" hmac = { version = "0.12", optional = true } @@ -64,6 +64,7 @@ regex = "1.12" ring = { version = "0.17", optional = true, default-features = false, features = ["alloc"] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" +sha1 = "0.10" sha2 = { version = "0.10", optional = true } urlencoding = "2.1" xmltree = "0.12" @@ -104,3 +105,8 @@ name = "load_balancing_with_hooks" name = "s3-api" path = "benches/s3/api_benchmarks.rs" harness = false + +[[bench]] +name = "bench_checksums" +path = "benches/s3/bench_checksums.rs" +harness = false diff --git a/benches/s3/bench_checksums.rs b/benches/s3/bench_checksums.rs new file mode 100644 index 0000000..a2781c4 --- /dev/null +++ b/benches/s3/bench_checksums.rs @@ -0,0 +1,53 @@ +// MinIO Rust Library for Amazon S3 Compatible Cloud Storage +// Copyright 2025 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. + +use criterion::{Criterion, Throughput, criterion_group, criterion_main}; +use minio::s3::utils::{ + crc32_checksum, crc32c, crc64nvme_checksum, md5sum_hash, sha1_hash, sha256_checksum, +}; + +fn bench_checksums(c: &mut Criterion) { + let sizes = vec![ + ("1KB", 1024), + ("10KB", 10 * 1024), + ("100KB", 100 * 1024), + ("1MB", 1024 * 1024), + ("10MB", 10 * 1024 * 1024), + ]; + + for (name, size) in sizes { + let data = vec![0u8; size]; + + let mut group = c.benchmark_group(format!("checksum_{}", name)); + group.throughput(Throughput::Bytes(size as u64)); + + group.bench_function("CRC32", |b| b.iter(|| crc32_checksum(&data))); + + group.bench_function("CRC32C", |b| b.iter(|| crc32c(&data))); + + group.bench_function("CRC64NVME", |b| b.iter(|| crc64nvme_checksum(&data))); + + group.bench_function("MD5", |b| b.iter(|| md5sum_hash(&data))); + + group.bench_function("SHA1", |b| b.iter(|| sha1_hash(&data))); + + group.bench_function("SHA256", |b| b.iter(|| sha256_checksum(&data))); + + group.finish(); + } +} + +criterion_group!(benches, bench_checksums); +criterion_main!(benches); diff --git a/macros/src/lib.rs b/macros/src/lib.rs index 2062fa0..8ba2a4a 100644 --- a/macros/src/lib.rs +++ b/macros/src/lib.rs @@ -74,6 +74,14 @@ extern crate proc_macro; /// // this test will not run if the MinIO server is NOT running in Express mode /// } /// ``` +/// - `ignore`: Mark the test as ignored (skipped by default). Run with `cargo test -- --ignored` to include. +/// ```no_run +/// use minio_common::test_context::TestContext; +/// #[minio_macros::test(ignore = "Requires newer server version")] +/// async fn my_test(ctx: TestContext, bucket_name: String) { +/// // this test is skipped by default +/// } +/// ``` #[proc_macro_attribute] pub fn test( args: proc_macro::TokenStream, diff --git a/macros/src/test_attr.rs b/macros/src/test_attr.rs index 11319de..35ead89 100644 --- a/macros/src/test_attr.rs +++ b/macros/src/test_attr.rs @@ -32,6 +32,8 @@ pub(crate) struct MacroArgs { no_bucket: darling::util::Flag, object_lock: darling::util::Flag, no_cleanup: darling::util::Flag, + /// Mark test as ignored (skipped by default, run with `cargo test -- --ignored`) + ignore: Option, } impl MacroArgs { @@ -171,28 +173,33 @@ fn generate_tokio_test_header(args: &MacroArgs, sig: TokenStream) -> TokenStream .as_ref() .map(ToString::to_string) .or(std::env::var("MINIO_TEST_TOKIO_RUNTIME_FLAVOR").ok()); - match (flavor, args.worker_threads) { + + // Generate #[ignore = "reason"] if specified + let ignore_attr = args + .ignore + .as_ref() + .map(|reason| quote!(#[ignore = #reason])); + + let tokio_attr = match (flavor, args.worker_threads) { (Some(flavor), None) => { - quote!(#[::tokio::test(flavor = #flavor)] - #sig - ) + quote!(#[::tokio::test(flavor = #flavor)]) } (None, Some(worker_threads)) => { - quote!(#[::tokio::test(worker_threads = #worker_threads)] - #sig - ) + quote!(#[::tokio::test(worker_threads = #worker_threads)]) } (None, None) => { - quote!(#[::tokio::test] - #sig - ) + quote!(#[::tokio::test]) } (Some(flavor), Some(worker_threads)) => { - quote!(#[::tokio::test(flavor = #flavor, worker_threads = #worker_threads)] - #sig - ) + quote!(#[::tokio::test(flavor = #flavor, worker_threads = #worker_threads)]) } - } + }; + + quote!( + #ignore_attr + #tokio_attr + #sig + ) } fn generate_express_skip_logic(args: &MacroArgs, span: proc_macro2::Span) -> TokenStream { diff --git a/src/s3/aws_chunked.rs b/src/s3/aws_chunked.rs new file mode 100644 index 0000000..dc97d2d --- /dev/null +++ b/src/s3/aws_chunked.rs @@ -0,0 +1,999 @@ +// MinIO Rust Library for Amazon S3 Compatible Cloud Storage +// Copyright 2025 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. + +//! AWS Chunked encoding with trailing checksum support. +//! +//! This module implements the `aws-chunked` content encoding format used by S3 +//! for streaming uploads with trailing checksums. The format allows computing +//! checksums incrementally while streaming data, with the checksum value sent +//! as a trailer at the end of the body. +//! +//! # Unsigned Protocol Format (STREAMING-UNSIGNED-PAYLOAD-TRAILER) +//! +//! ```text +//! \r\n +//! \r\n +//! ... +//! 0\r\n +//! x-amz-checksum-:\r\n +//! \r\n +//! ``` +//! +//! # Signed Protocol Format (STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER) +//! +//! ```text +//! ;chunk-signature=\r\n +//! \r\n +//! ... +//! 0;chunk-signature=\r\n +//! x-amz-checksum-:\r\n +//! x-amz-trailer-signature:\r\n +//! \r\n +//! ``` +//! +//! # Wire Format vs Canonical Form +//! +//! **Important**: There are two different line ending conventions: +//! +//! - **Wire format (HTTP protocol)**: Uses `\r\n` (CRLF) per RFC 9112 (HTTP/1.1) +//! - **Canonical form (for signing)**: Uses `\n` (LF) per AWS SigV4 spec +//! +//! When computing the trailer signature, AWS specifies: +//! ```text +//! hash('x-amz-checksum-crc32c:sOO8/Q==\n') // Note: \n not \r\n +//! ``` +//! +//! But the actual bytes sent over HTTP use CRLF line endings. +//! +//! Reference: + +use crate::s3::signer::{ChunkSigningContext, sign_chunk, sign_trailer}; +use crate::s3::utils::{ChecksumAlgorithm, b64_encode, sha256_hash}; +use bytes::Bytes; +use crc_fast::{CrcAlgorithm, Digest as CrcFastDigest}; +use futures_util::Stream; +#[cfg(feature = "ring")] +use ring::digest::{Context, SHA256}; +use sha1::{Digest as Sha1Digest, Sha1}; +#[cfg(not(feature = "ring"))] +use sha2::Sha256; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context as TaskContext, Poll}; + +/// Default chunk size for aws-chunked encoding (64 KB). +const DEFAULT_CHUNK_SIZE: usize = 64 * 1024; + +/// Incremental checksum hasher for streaming computation. +enum StreamingHasher { + Crc32(CrcFastDigest), + Crc32c(CrcFastDigest), + Crc64nvme(CrcFastDigest), + Sha1(Sha1), + #[cfg(feature = "ring")] + Sha256(Context), + #[cfg(not(feature = "ring"))] + Sha256(Sha256), +} + +impl StreamingHasher { + fn new(algorithm: ChecksumAlgorithm) -> Self { + match algorithm { + ChecksumAlgorithm::CRC32 => { + StreamingHasher::Crc32(CrcFastDigest::new(CrcAlgorithm::Crc32IsoHdlc)) + } + ChecksumAlgorithm::CRC32C => { + StreamingHasher::Crc32c(CrcFastDigest::new(CrcAlgorithm::Crc32Iscsi)) + } + ChecksumAlgorithm::CRC64NVME => { + StreamingHasher::Crc64nvme(CrcFastDigest::new(CrcAlgorithm::Crc64Nvme)) + } + ChecksumAlgorithm::SHA1 => StreamingHasher::Sha1(Sha1::new()), + #[cfg(feature = "ring")] + ChecksumAlgorithm::SHA256 => StreamingHasher::Sha256(Context::new(&SHA256)), + #[cfg(not(feature = "ring"))] + ChecksumAlgorithm::SHA256 => StreamingHasher::Sha256(Sha256::new()), + } + } + + fn update(&mut self, data: &[u8]) { + match self { + StreamingHasher::Crc32(d) => d.update(data), + StreamingHasher::Crc32c(d) => d.update(data), + StreamingHasher::Crc64nvme(d) => d.update(data), + StreamingHasher::Sha1(h) => h.update(data), + #[cfg(feature = "ring")] + StreamingHasher::Sha256(ctx) => ctx.update(data), + #[cfg(not(feature = "ring"))] + StreamingHasher::Sha256(h) => h.update(data), + } + } + + fn finalize(self) -> String { + match self { + // crc-fast returns u64; CRC32 variants need cast to u32 + StreamingHasher::Crc32(d) => b64_encode((d.finalize() as u32).to_be_bytes()), + StreamingHasher::Crc32c(d) => b64_encode((d.finalize() as u32).to_be_bytes()), + StreamingHasher::Crc64nvme(d) => b64_encode(d.finalize().to_be_bytes()), + StreamingHasher::Sha1(h) => { + let result = h.finalize(); + b64_encode(&result[..]) + } + #[cfg(feature = "ring")] + StreamingHasher::Sha256(ctx) => b64_encode(ctx.finish().as_ref()), + #[cfg(not(feature = "ring"))] + StreamingHasher::Sha256(h) => { + let result = h.finalize(); + b64_encode(&result[..]) + } + } + } +} + +/// State machine for the aws-chunked encoder. +#[derive(Clone, Copy)] +enum EncoderState { + /// Emitting data chunks + Streaming, + /// Emitting the final zero-length chunk marker + FinalChunk, + /// Emitting the trailer with checksum + Trailer, + /// Done + Done, +} + +/// AWS Chunked encoder that wraps data in aws-chunked format with trailing checksum. +/// +/// This encoder takes input data and produces output in the following format: +/// ```text +/// \r\n +/// \r\n +/// 0\r\n +/// x-amz-checksum-:\r\n +/// \r\n +/// ``` +pub struct AwsChunkedEncoder { + inner: S, + algorithm: ChecksumAlgorithm, + hasher: Option, + state: EncoderState, +} + +impl AwsChunkedEncoder { + /// Creates a new AWS chunked encoder wrapping the given stream. + pub fn new(inner: S, algorithm: ChecksumAlgorithm) -> Self { + Self { + inner, + algorithm, + hasher: Some(StreamingHasher::new(algorithm)), + state: EncoderState::Streaming, + } + } +} + +impl Stream for AwsChunkedEncoder +where + S: Stream> + Unpin, +{ + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + loop { + match self.state { + EncoderState::Streaming => { + let inner = Pin::new(&mut self.inner); + match inner.poll_next(cx) { + Poll::Ready(Some(Ok(chunk))) => { + if chunk.is_empty() { + continue; + } + + // Update checksum with raw data + if let Some(ref mut hasher) = self.hasher { + hasher.update(&chunk); + } + + // Format: \r\n\r\n + let chunk_header = format!("{:x}\r\n", chunk.len()); + let mut output = + Vec::with_capacity(chunk_header.len() + chunk.len() + 2); + output.extend_from_slice(chunk_header.as_bytes()); + output.extend_from_slice(&chunk); + output.extend_from_slice(b"\r\n"); + + return Poll::Ready(Some(Ok(Bytes::from(output)))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Input stream exhausted, move to final chunk + self.state = EncoderState::FinalChunk; + } + Poll::Pending => { + return Poll::Pending; + } + } + } + + EncoderState::FinalChunk => { + // Emit "0\r\n" for the final zero-length chunk + self.state = EncoderState::Trailer; + return Poll::Ready(Some(Ok(Bytes::from_static(b"0\r\n")))); + } + + EncoderState::Trailer => { + // Compute and emit the trailer + let hasher = self.hasher.take().expect("hasher should exist"); + let checksum_value = hasher.finalize(); + let trailer = format!( + "{}:{}\r\n\r\n", + self.algorithm.header_name(), + checksum_value + ); + + self.state = EncoderState::Done; + return Poll::Ready(Some(Ok(Bytes::from(trailer)))); + } + + EncoderState::Done => { + return Poll::Ready(None); + } + } + } + } +} + +/// Calculates the encoded length for aws-chunked format. +/// +/// For a given content length and chunk size, returns the total encoded length +/// including all chunk headers, the final zero-length chunk, and the trailer. +pub fn calculate_encoded_length( + content_length: u64, + chunk_size: usize, + algorithm: ChecksumAlgorithm, +) -> u64 { + let chunk_size = chunk_size as u64; + + // Number of full chunks + let full_chunks = content_length / chunk_size; + // Size of the last partial chunk (0 if content divides evenly) + let last_chunk_size = content_length % chunk_size; + let has_partial = if last_chunk_size > 0 { 1 } else { 0 }; + + // Each chunk: "\r\n\r\n" + // hex-size length varies based on chunk size + let hex_len_full = format!("{:x}", chunk_size).len() as u64; + let hex_len_partial = if last_chunk_size > 0 { + format!("{:x}", last_chunk_size).len() as u64 + } else { + 0 + }; + + // Full chunks overhead: hex_len + 2 (\r\n) + chunk_size + 2 (\r\n) + let full_chunk_overhead = full_chunks * (hex_len_full + 2 + chunk_size + 2); + + // Partial chunk overhead (if any) + let partial_chunk_overhead = if has_partial > 0 { + hex_len_partial + 2 + last_chunk_size + 2 + } else { + 0 + }; + + // Final chunk: "0\r\n" + let final_chunk = 3; + + // Trailer: "x-amz-checksum-:\r\n\r\n" + // Header name length + ":" + base64 checksum length + "\r\n\r\n" + let trailer_header_len = algorithm.header_name().len() as u64; + let checksum_b64_len = match algorithm { + ChecksumAlgorithm::CRC32 | ChecksumAlgorithm::CRC32C => 8, // 4 bytes -> 8 chars base64 + ChecksumAlgorithm::CRC64NVME => 12, // 8 bytes -> 12 chars base64 + ChecksumAlgorithm::SHA1 => 28, // 20 bytes -> 28 chars base64 + ChecksumAlgorithm::SHA256 => 44, // 32 bytes -> 44 chars base64 + }; + let trailer_len = trailer_header_len + 1 + checksum_b64_len + 4; // +1 for ":", +4 for "\r\n\r\n" + + full_chunk_overhead + partial_chunk_overhead + final_chunk + trailer_len +} + +/// Returns the default chunk size for aws-chunked encoding. +pub fn default_chunk_size() -> usize { + DEFAULT_CHUNK_SIZE +} + +// =========================== +// Signed AWS Chunked Encoder +// =========================== + +/// State machine for the signed aws-chunked encoder. +#[derive(Clone, Copy)] +enum SignedEncoderState { + /// Emitting signed data chunks + Streaming, + /// Emitting the final zero-length chunk with signature + FinalChunk, + /// Emitting the checksum trailer header + Trailer, + /// Emitting the trailer signature + TrailerSignature, + /// Done + Done, +} + +/// AWS Chunked encoder with chunk signing for STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER. +/// +/// Each chunk is signed using the AWS Signature V4 chunk signing algorithm. +/// The final trailer is also signed with a trailer signature. +/// +/// # Wire Format +/// +/// ```text +/// ;chunk-signature=\r\n +/// \r\n +/// 0;chunk-signature=\r\n +/// x-amz-checksum-:\r\n +/// x-amz-trailer-signature:\r\n +/// \r\n +/// ``` +pub struct SignedAwsChunkedEncoder { + inner: S, + algorithm: ChecksumAlgorithm, + hasher: Option, + state: SignedEncoderState, + + // Signing context + signing_key: Arc<[u8]>, + date_time: String, + scope: String, + + // Signature chain - each chunk's signature becomes the previous for the next + current_signature: String, + + // Store the checksum value for trailer signature computation + checksum_value: Option, +} + +impl SignedAwsChunkedEncoder { + /// Creates a new signed AWS chunked encoder wrapping the given stream. + /// + /// # Arguments + /// * `inner` - The underlying data stream + /// * `algorithm` - The checksum algorithm to use + /// * `context` - The chunk signing context from request signing + pub fn new(inner: S, algorithm: ChecksumAlgorithm, context: ChunkSigningContext) -> Self { + Self { + inner, + algorithm, + hasher: Some(StreamingHasher::new(algorithm)), + state: SignedEncoderState::Streaming, + signing_key: context.signing_key, + date_time: context.date_time, + scope: context.scope, + current_signature: context.seed_signature, + checksum_value: None, + } + } + + /// Signs a chunk and returns the signature. + fn sign_chunk_data(&mut self, chunk_hash: &str) -> String { + let signature = sign_chunk( + &self.signing_key, + &self.date_time, + &self.scope, + &self.current_signature, + chunk_hash, + ); + self.current_signature = signature.clone(); + signature + } +} + +impl Stream for SignedAwsChunkedEncoder +where + S: Stream> + Unpin, +{ + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + loop { + match self.state { + SignedEncoderState::Streaming => { + let inner = Pin::new(&mut self.inner); + match inner.poll_next(cx) { + Poll::Ready(Some(Ok(chunk))) => { + if chunk.is_empty() { + continue; + } + + // Update checksum hasher with raw data + if let Some(ref mut hasher) = self.hasher { + hasher.update(&chunk); + } + + // Compute SHA256 hash of chunk data for signing + let chunk_hash = sha256_hash(&chunk); + + // Sign the chunk + let signature = self.sign_chunk_data(&chunk_hash); + + // Format: ;chunk-signature=\r\n\r\n + let chunk_header = + format!("{:x};chunk-signature={}\r\n", chunk.len(), signature); + let mut output = + Vec::with_capacity(chunk_header.len() + chunk.len() + 2); + output.extend_from_slice(chunk_header.as_bytes()); + output.extend_from_slice(&chunk); + output.extend_from_slice(b"\r\n"); + + return Poll::Ready(Some(Ok(Bytes::from(output)))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + // Input stream exhausted, move to final chunk + self.state = SignedEncoderState::FinalChunk; + } + Poll::Pending => { + return Poll::Pending; + } + } + } + + SignedEncoderState::FinalChunk => { + // Sign the empty chunk (SHA256 of empty string) + const EMPTY_SHA256: &str = + "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"; + let signature = self.sign_chunk_data(EMPTY_SHA256); + + // Emit "0;chunk-signature=\r\n" + let final_chunk = format!("0;chunk-signature={}\r\n", signature); + + self.state = SignedEncoderState::Trailer; + return Poll::Ready(Some(Ok(Bytes::from(final_chunk)))); + } + + SignedEncoderState::Trailer => { + // Compute and store the checksum value + let hasher = self.hasher.take().expect("hasher should exist"); + let checksum_value = hasher.finalize(); + self.checksum_value = Some(checksum_value.clone()); + + // Emit the checksum trailer using CRLF (wire format per RFC 9112) + // Note: The canonical form for signing uses LF (\n), but HTTP wire + // format uses CRLF (\r\n). See module docs for details. + let trailer = format!( + "{}:{}\r\n", + self.algorithm.header_name().to_lowercase(), + checksum_value + ); + + self.state = SignedEncoderState::TrailerSignature; + return Poll::Ready(Some(Ok(Bytes::from(trailer)))); + } + + SignedEncoderState::TrailerSignature => { + // Compute the canonical trailers string for signing. + // IMPORTANT: AWS SigV4 canonical form uses LF (\n), NOT CRLF (\r\n). + // Per AWS docs: hash('x-amz-checksum-crc32c:sOO8/Q==\n') + // This differs from the wire format which uses CRLF. + let checksum_value = + self.checksum_value.as_ref().expect("checksum should exist"); + let canonical_trailers = format!( + "{}:{}\n", // LF for canonical form (signing) + self.algorithm.header_name().to_lowercase(), + checksum_value + ); + + // Hash the canonical trailers + let trailers_hash = sha256_hash(canonical_trailers.as_bytes()); + + // Sign the trailer + let trailer_signature = sign_trailer( + &self.signing_key, + &self.date_time, + &self.scope, + &self.current_signature, + &trailers_hash, + ); + + // Emit trailer signature using CRLF (wire format per RFC 9112) + // Final \r\n\r\n marks end of trailer section + let trailer_sig_line = + format!("x-amz-trailer-signature:{}\r\n\r\n", trailer_signature); + + self.state = SignedEncoderState::Done; + return Poll::Ready(Some(Ok(Bytes::from(trailer_sig_line)))); + } + + SignedEncoderState::Done => { + return Poll::Ready(None); + } + } + } + } +} + +/// Calculates the encoded length for signed aws-chunked format. +/// +/// For a given content length and chunk size, returns the total encoded length +/// including all chunk headers with signatures, the final zero-length chunk, +/// the checksum trailer, and the trailer signature. +pub fn calculate_signed_encoded_length( + content_length: u64, + chunk_size: usize, + algorithm: ChecksumAlgorithm, +) -> u64 { + let chunk_size = chunk_size as u64; + + // Number of full chunks + let full_chunks = content_length / chunk_size; + // Size of the last partial chunk (0 if content divides evenly) + let last_chunk_size = content_length % chunk_size; + let has_partial = if last_chunk_size > 0 { 1 } else { 0 }; + + // Each signed chunk: ";chunk-signature=<64-hex>\r\n\r\n" + // Signature overhead per chunk: ";chunk-signature=" (17) + 64 hex chars = 81 bytes + let signature_overhead: u64 = 81; + + let hex_len_full = format!("{:x}", chunk_size).len() as u64; + let hex_len_partial = if last_chunk_size > 0 { + format!("{:x}", last_chunk_size).len() as u64 + } else { + 0 + }; + + // Full chunks: hex_len + signature_overhead + 2 (\r\n) + chunk_size + 2 (\r\n) + let full_chunk_overhead = + full_chunks * (hex_len_full + signature_overhead + 2 + chunk_size + 2); + + // Partial chunk (if any) + let partial_chunk_overhead = if has_partial > 0 { + hex_len_partial + signature_overhead + 2 + last_chunk_size + 2 + } else { + 0 + }; + + // Final chunk: "0;chunk-signature=<64-hex>\r\n" = 1 + 81 + 2 = 84 + let final_chunk = 84; + + // Checksum trailer: ":\r\n" + // Header name is lowercase (e.g., "x-amz-checksum-crc32") + let trailer_header_len = algorithm.header_name().to_lowercase().len() as u64; + let checksum_b64_len = match algorithm { + ChecksumAlgorithm::CRC32 | ChecksumAlgorithm::CRC32C => 8, + ChecksumAlgorithm::CRC64NVME => 12, + ChecksumAlgorithm::SHA1 => 28, + ChecksumAlgorithm::SHA256 => 44, + }; + let checksum_trailer = trailer_header_len + 1 + checksum_b64_len + 2; // +1 for ":", +2 for "\r\n" + + // Trailer signature: "x-amz-trailer-signature:<64-hex>\r\n\r\n" + // = 24 + 64 + 4 = 92 bytes + let trailer_signature = 92; + + full_chunk_overhead + + partial_chunk_overhead + + final_chunk + + checksum_trailer + + trailer_signature +} + +// =========================== +// Rechunking Stream Wrapper +// =========================== + +/// A stream wrapper that re-chunks incoming data to a fixed chunk size. +/// +/// This ensures that the actual chunks produced match the chunk size assumed +/// by `calculate_encoded_length` and `calculate_signed_encoded_length`, +/// preventing Content-Length mismatches when the input stream produces +/// differently-sized chunks. +pub struct RechunkingStream { + inner: S, + chunk_size: usize, + buffer: Vec, + done: bool, +} + +impl RechunkingStream { + /// Creates a new rechunking stream wrapper. + /// + /// The wrapper buffers incoming data and emits chunks of exactly `chunk_size` bytes, + /// except for the final chunk which may be smaller. + pub fn new(inner: S, chunk_size: usize) -> Self { + Self { + inner, + chunk_size, + buffer: Vec::with_capacity(chunk_size), + done: false, + } + } + + /// Creates a new rechunking stream with the default chunk size (64 KB). + pub fn with_default_chunk_size(inner: S) -> Self { + Self::new(inner, DEFAULT_CHUNK_SIZE) + } +} + +impl Stream for RechunkingStream +where + S: Stream> + Unpin, +{ + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + let chunk_size = self.chunk_size; + + if self.done && self.buffer.is_empty() { + return Poll::Ready(None); + } + + // If we already have a full chunk buffered, emit it + if self.buffer.len() >= chunk_size { + let chunk: Vec = self.buffer.drain(..chunk_size).collect(); + return Poll::Ready(Some(Ok(Bytes::from(chunk)))); + } + + // Try to fill the buffer from the inner stream + loop { + if self.done { + // Inner stream exhausted, emit remaining buffer as final chunk + if self.buffer.is_empty() { + return Poll::Ready(None); + } + let remaining = std::mem::take(&mut self.buffer); + return Poll::Ready(Some(Ok(Bytes::from(remaining)))); + } + + let inner = Pin::new(&mut self.inner); + match inner.poll_next(cx) { + Poll::Ready(Some(Ok(chunk))) => { + if chunk.is_empty() { + continue; + } + + self.buffer.extend_from_slice(&chunk); + + // If we now have enough for a full chunk, emit it + if self.buffer.len() >= chunk_size { + let chunk: Vec = self.buffer.drain(..chunk_size).collect(); + return Poll::Ready(Some(Ok(Bytes::from(chunk)))); + } + // Otherwise continue buffering + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(Err(e))); + } + Poll::Ready(None) => { + self.done = true; + // Loop will handle emitting remaining buffer + } + Poll::Pending => { + return Poll::Pending; + } + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use futures_util::StreamExt; + + #[tokio::test] + async fn test_aws_chunked_encoder_simple() { + let data = Bytes::from("Hello, World!"); + // Use iter instead of once - iter produces an Unpin stream + let stream = futures_util::stream::iter(vec![Ok::<_, std::io::Error>(data.clone())]); + + let mut encoder = AwsChunkedEncoder::new(stream, ChecksumAlgorithm::CRC32); + let mut output = Vec::new(); + + while let Some(chunk) = encoder.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + let output_str = String::from_utf8(output).unwrap(); + + // Should start with hex size of "Hello, World!" (13 bytes = 'd') + assert!(output_str.starts_with("d\r\n")); + // Should contain the data + assert!(output_str.contains("Hello, World!")); + // Should end with trailer (header name is mixed-case per S3 spec) + assert!(output_str.contains("X-Amz-Checksum-CRC32:")); + assert!(output_str.ends_with("\r\n\r\n")); + // Should have zero-length final chunk + assert!(output_str.contains("\r\n0\r\n")); + } + + #[tokio::test] + async fn test_aws_chunked_encoder_multiple_chunks() { + let chunks = vec![ + Ok::<_, std::io::Error>(Bytes::from("Hello, ")), + Ok(Bytes::from("World!")), + ]; + let stream = futures_util::stream::iter(chunks); + + let mut encoder = AwsChunkedEncoder::new(stream, ChecksumAlgorithm::CRC64NVME); + let mut output = Vec::new(); + + while let Some(chunk) = encoder.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + let output_str = String::from_utf8(output).unwrap(); + + // Should have two chunk headers + assert!(output_str.starts_with("7\r\n")); // "Hello, " is 7 bytes + assert!(output_str.contains("6\r\n")); // "World!" is 6 bytes + assert!(output_str.contains("X-Amz-Checksum-CRC64NVME:")); + } + + #[test] + fn test_calculate_encoded_length() { + // Simple case: 100 bytes, 64KB chunks + let len = calculate_encoded_length(100, 64 * 1024, ChecksumAlgorithm::CRC32); + // 100 bytes fits in one chunk: "64\r\n" (4) + 100 + "\r\n" (2) + "0\r\n" (3) + trailer + // "64" is hex for 100, which is "64" (2 chars) + // trailer: "x-amz-checksum-crc32:" (21) + 8 (base64) + "\r\n\r\n" (4) = 33 + // Total: 2 + 2 + 100 + 2 + 3 + 33 = 142 + assert!(len > 100); // Should be larger than raw content + } + + // =========================== + // Signed Encoder Tests + // =========================== + + fn test_signing_context() -> ChunkSigningContext { + ChunkSigningContext { + signing_key: Arc::from(vec![ + // Pre-computed signing key for test credentials + 0x98, 0xf1, 0xd8, 0x89, 0xfe, 0xc4, 0xf4, 0x42, 0x1a, 0xdc, 0x52, 0x2b, 0xab, 0x0c, + 0xe1, 0xf8, 0x2c, 0x6c, 0x4e, 0x4e, 0xc3, 0x9a, 0xe1, 0xf6, 0xcc, 0xf2, 0x0e, 0x8f, + 0x40, 0x89, 0x45, 0x65, + ]), + date_time: "20130524T000000Z".to_string(), + scope: "20130524/us-east-1/s3/aws4_request".to_string(), + seed_signature: "4f232c4386841ef735655705268965c44a0e4690baa4adea153f7db9fa80a0a9" + .to_string(), + } + } + + #[tokio::test] + async fn test_signed_encoder_simple() { + let data = Bytes::from("Hello, World!"); + let stream = futures_util::stream::iter(vec![Ok::<_, std::io::Error>(data)]); + + let context = test_signing_context(); + let mut encoder = SignedAwsChunkedEncoder::new(stream, ChecksumAlgorithm::CRC32, context); + let mut output = Vec::new(); + + while let Some(chunk) = encoder.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + let output_str = String::from_utf8(output).unwrap(); + + // Should start with hex size and chunk-signature + assert!(output_str.starts_with("d;chunk-signature=")); + // Should contain the data + assert!(output_str.contains("Hello, World!")); + // Should have final chunk with signature + assert!(output_str.contains("0;chunk-signature=")); + // Should have checksum trailer (lowercase) + assert!(output_str.contains("x-amz-checksum-crc32:")); + // Should have trailer signature + assert!(output_str.contains("x-amz-trailer-signature:")); + // Should end with \r\n\r\n + assert!(output_str.ends_with("\r\n\r\n")); + } + + #[tokio::test] + async fn test_signed_encoder_multiple_chunks() { + let chunks = vec![ + Ok::<_, std::io::Error>(Bytes::from("Hello, ")), + Ok(Bytes::from("World!")), + ]; + let stream = futures_util::stream::iter(chunks); + + let context = test_signing_context(); + let mut encoder = SignedAwsChunkedEncoder::new(stream, ChecksumAlgorithm::CRC32C, context); + let mut output = Vec::new(); + + while let Some(chunk) = encoder.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + let output_str = String::from_utf8(output).unwrap(); + + // Should have two chunk signatures (different signatures due to chaining) + let sig_count = output_str.matches(";chunk-signature=").count(); + assert_eq!(sig_count, 3); // 2 data chunks + 1 final chunk + + // Should have checksum trailer + assert!(output_str.contains("x-amz-checksum-crc32c:")); + // Should have trailer signature + assert!(output_str.contains("x-amz-trailer-signature:")); + } + + #[tokio::test] + async fn test_signed_encoder_signature_is_64_hex_chars() { + let data = Bytes::from("test"); + let stream = futures_util::stream::iter(vec![Ok::<_, std::io::Error>(data)]); + + let context = test_signing_context(); + let mut encoder = SignedAwsChunkedEncoder::new(stream, ChecksumAlgorithm::CRC32, context); + let mut output = Vec::new(); + + while let Some(chunk) = encoder.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + let output_str = String::from_utf8(output).unwrap(); + + // Extract signatures and verify they're 64 hex chars + for sig_match in output_str.match_indices(";chunk-signature=") { + let start = sig_match.0 + sig_match.1.len(); + let sig = &output_str[start..start + 64]; + assert!( + sig.chars().all(|c| c.is_ascii_hexdigit()), + "Signature should be hex: {}", + sig + ); + } + + // Also check trailer signature + let trailer_sig_start = output_str.find("x-amz-trailer-signature:").unwrap() + 24; + let trailer_sig = &output_str[trailer_sig_start..trailer_sig_start + 64]; + assert!( + trailer_sig.chars().all(|c| c.is_ascii_hexdigit()), + "Trailer signature should be hex: {}", + trailer_sig + ); + } + + #[test] + fn test_calculate_signed_encoded_length() { + // 100 bytes, 64KB chunks + let len = calculate_signed_encoded_length(100, 64 * 1024, ChecksumAlgorithm::CRC32); + + // Should be larger than unsigned (due to signature overhead) + let unsigned_len = calculate_encoded_length(100, 64 * 1024, ChecksumAlgorithm::CRC32); + assert!( + len > unsigned_len, + "Signed length {} should be > unsigned length {}", + len, + unsigned_len + ); + } + + #[test] + fn test_calculate_signed_encoded_length_multiple_chunks() { + // 200KB with 64KB chunks = 3 full chunks + 1 partial + final + let content_len = 200 * 1024; + let chunk_size = 64 * 1024; + let len = + calculate_signed_encoded_length(content_len, chunk_size, ChecksumAlgorithm::SHA256); + + // Should include all overhead + assert!(len > content_len); + + // Calculate expected: signature overhead per chunk is 81 bytes + // Plus final chunk (84), checksum trailer, trailer signature (92) + } + + // =========================== + // RechunkingStream Tests + // =========================== + + #[tokio::test] + async fn test_rechunking_stream_combines_small_chunks() { + // Create many small 1KB chunks + let chunk_size = 1024; + let num_chunks = 10; + let chunks: Vec> = (0..num_chunks) + .map(|i| Ok(Bytes::from(vec![i as u8; chunk_size]))) + .collect(); + + let stream = futures_util::stream::iter(chunks); + let mut rechunker = RechunkingStream::new(stream, 4096); // 4KB target + + let mut output_chunks = Vec::new(); + while let Some(chunk) = rechunker.next().await { + output_chunks.push(chunk.unwrap()); + } + + // 10 x 1KB = 10KB, rechunked to 4KB = 2 full + 1 partial (2KB) + assert_eq!(output_chunks.len(), 3); + assert_eq!(output_chunks[0].len(), 4096); + assert_eq!(output_chunks[1].len(), 4096); + assert_eq!(output_chunks[2].len(), 2048); + + // Total bytes preserved + let total: usize = output_chunks.iter().map(|c| c.len()).sum(); + assert_eq!(total, num_chunks * chunk_size); + } + + #[tokio::test] + async fn test_rechunking_stream_passes_large_chunks() { + // Single chunk larger than target size + let data = Bytes::from(vec![42u8; 10000]); + let stream = futures_util::stream::iter(vec![Ok::<_, std::io::Error>(data)]); + + let mut rechunker = RechunkingStream::new(stream, 4096); + + let mut output_chunks = Vec::new(); + while let Some(chunk) = rechunker.next().await { + output_chunks.push(chunk.unwrap()); + } + + // 10000 bytes / 4096 = 2 full + 1 partial (1808) + assert_eq!(output_chunks.len(), 3); + assert_eq!(output_chunks[0].len(), 4096); + assert_eq!(output_chunks[1].len(), 4096); + assert_eq!(output_chunks[2].len(), 1808); + } + + #[tokio::test] + async fn test_rechunking_stream_exact_multiple() { + // Data that divides evenly into chunk size + let data = Bytes::from(vec![1u8; 8192]); + let stream = futures_util::stream::iter(vec![Ok::<_, std::io::Error>(data)]); + + let mut rechunker = RechunkingStream::new(stream, 4096); + + let mut output_chunks = Vec::new(); + while let Some(chunk) = rechunker.next().await { + output_chunks.push(chunk.unwrap()); + } + + assert_eq!(output_chunks.len(), 2); + assert_eq!(output_chunks[0].len(), 4096); + assert_eq!(output_chunks[1].len(), 4096); + } + + #[tokio::test] + async fn test_rechunking_stream_empty() { + let stream = futures_util::stream::iter(Vec::>::new()); + let mut rechunker = RechunkingStream::new(stream, 4096); + + let result = rechunker.next().await; + assert!(result.is_none()); + } + + #[tokio::test] + async fn test_rechunking_stream_preserves_data() { + // Verify data integrity through rechunking + let original: Vec = (0..=255).cycle().take(15000).collect(); + let chunks: Vec> = original + .chunks(100) // 100-byte input chunks + .map(|c| Ok(Bytes::copy_from_slice(c))) + .collect(); + + let stream = futures_util::stream::iter(chunks); + let mut rechunker = RechunkingStream::new(stream, 4096); + + let mut output = Vec::new(); + while let Some(chunk) = rechunker.next().await { + output.extend_from_slice(&chunk.unwrap()); + } + + assert_eq!(output, original); + } +} diff --git a/src/s3/builders/append_object.rs b/src/s3/builders/append_object.rs index 60494e4..489e70a 100644 --- a/src/s3/builders/append_object.rs +++ b/src/s3/builders/append_object.rs @@ -26,7 +26,9 @@ use crate::s3::response_traits::HasObjectSize; use crate::s3::segmented_bytes::SegmentedBytes; use crate::s3::sse::Sse; use crate::s3::types::{S3Api, S3Request, ToS3Request}; -use crate::s3::utils::{check_bucket_name, check_object_name, check_sse}; +use crate::s3::utils::{ + ChecksumAlgorithm, check_bucket_name, check_object_name, check_sse, compute_checksum_sb, +}; use http::Method; use std::sync::Arc; use typed_builder::TypedBuilder; @@ -64,6 +66,14 @@ pub struct AppendObject { /// Value of `x-amz-write-offset-bytes`. #[builder(!default)] // force required offset_bytes: u64, + + /// Optional checksum algorithm for data integrity verification during append. + /// + /// When specified, computes a checksum of the appended data using the selected algorithm + /// (CRC32, CRC32C, SHA1, SHA256, or CRC64NVME). The checksum is sent with the append + /// operation and verified by the server. + #[builder(default, setter(into))] + checksum_algorithm: Option, } impl S3Api for AppendObject { @@ -83,6 +93,7 @@ pub type AppendObjectBldr = AppendObjectBuilder<( (), (Arc,), (u64,), + (), )>; impl ToS3Request for AppendObject { @@ -94,6 +105,21 @@ impl ToS3Request for AppendObject { let mut headers: Multimap = self.extra_headers.unwrap_or_default(); headers.add(X_AMZ_WRITE_OFFSET_BYTES, self.offset_bytes.to_string()); + if let Some(algorithm) = self.checksum_algorithm { + let checksum_value = compute_checksum_sb(algorithm, &self.data); + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + + match algorithm { + ChecksumAlgorithm::CRC32 => headers.add(X_AMZ_CHECKSUM_CRC32, checksum_value), + ChecksumAlgorithm::CRC32C => headers.add(X_AMZ_CHECKSUM_CRC32C, checksum_value), + ChecksumAlgorithm::SHA1 => headers.add(X_AMZ_CHECKSUM_SHA1, checksum_value), + ChecksumAlgorithm::SHA256 => headers.add(X_AMZ_CHECKSUM_SHA256, checksum_value), + ChecksumAlgorithm::CRC64NVME => { + headers.add(X_AMZ_CHECKSUM_CRC64NVME, checksum_value) + } + } + } + Ok(S3Request::builder() .client(self.client) .method(Method::PUT) @@ -144,6 +170,13 @@ pub struct AppendObjectContent { /// Value of `x-amz-write-offset-bytes`. #[builder(default)] offset_bytes: u64, + /// Optional checksum algorithm for data integrity verification during append. + /// + /// When specified, computes checksums for appended data using the selected algorithm + /// (CRC32, CRC32C, SHA1, SHA256, or CRC64NVME). The checksum is computed for each + /// chunk and sent with the append operation. + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`AppendObjectContent`] that is returned by [`MinioClient::append_object_content`](crate::s3::client::MinioClient::append_object_content). @@ -162,6 +195,7 @@ pub type AppendObjectContentBldr = AppendObjectContentBuilder<( (), (), (), + (), )>; impl AppendObjectContent { @@ -229,6 +263,7 @@ impl AppendObjectContent { offset_bytes: current_file_size, sse: self.sse, data: Arc::new(seg_bytes), + checksum_algorithm: self.checksum_algorithm, }; ao.send().await } else if let Some(expected) = object_size.value() @@ -296,6 +331,7 @@ impl AppendObjectContent { sse: self.sse.clone(), data: Arc::new(part_content), offset_bytes: next_offset_bytes, + checksum_algorithm: self.checksum_algorithm, }; let resp: AppendObjectResponse = append_object.send().await?; //println!("AppendObjectResponse: object_size={:?}", resp.object_size); diff --git a/src/s3/builders/copy_object.rs b/src/s3/builders/copy_object.rs index 52ae22e..d1987db 100644 --- a/src/s3/builders/copy_object.rs +++ b/src/s3/builders/copy_object.rs @@ -23,12 +23,13 @@ use crate::s3::response::{ CopyObjectInternalResponse, CopyObjectResponse, CreateMultipartUploadResponse, StatObjectResponse, UploadPartCopyResponse, }; +use crate::s3::response_traits::HasChecksumHeaders; use crate::s3::response_traits::HasEtagFromBody; use crate::s3::sse::{Sse, SseCustomerKey}; use crate::s3::types::{Directive, PartInfo, Retention, S3Api, S3Request, ToS3Request}; use crate::s3::utils::{ - UtcTime, check_bucket_name, check_object_name, check_sse, check_ssec, to_http_header_value, - to_iso8601utc, url_encode, + ChecksumAlgorithm, UtcTime, check_bucket_name, check_object_name, check_sse, check_ssec, + to_http_header_value, to_iso8601utc, url_encode, }; use async_recursion::async_recursion; use http::Method; @@ -59,6 +60,13 @@ pub struct UploadPartCopy { part_number: u16, #[builder(default)] headers: Multimap, + /// Optional checksum algorithm for data integrity verification during part copy. + /// + /// When specified, the server computes a checksum of the copied part data using + /// this algorithm. Use the same algorithm for all parts in a multipart upload. + /// Supported algorithms: CRC32, CRC32C, SHA1, SHA256, CRC64NVME. + #[builder(default, setter(into))] + checksum_algorithm: Option, } impl S3Api for UploadPartCopy { @@ -78,6 +86,7 @@ pub type UploadPartCopyBldr = UploadPartCopyBuilder<( (String,), (), (), + (), )>; impl ToS3Request for UploadPartCopy { @@ -100,6 +109,10 @@ impl ToS3Request for UploadPartCopy { let mut headers: Multimap = self.extra_headers.unwrap_or_default(); headers.add_multimap(self.headers); + if let Some(algorithm) = self.checksum_algorithm { + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + } + let mut query_params: Multimap = self.extra_query_params.unwrap_or_default(); { query_params.add("partNumber", self.part_number.to_string()); @@ -150,6 +163,8 @@ pub struct CopyObjectInternal { metadata_directive: Option, #[builder(default, setter(into))] tagging_directive: Option, + #[builder(default, setter(into))] + checksum_algorithm: Option, } impl S3Api for CopyObjectInternal { @@ -175,6 +190,7 @@ pub type CopyObjectInternalBldr = CopyObjectInternalBuilder<( (), (), (), + (), )>; impl ToS3Request for CopyObjectInternal { @@ -261,6 +277,10 @@ impl ToS3Request for CopyObjectInternal { if let Some(v) = self.source.ssec { headers.add_multimap(v.copy_headers()); } + + if let Some(algorithm) = self.checksum_algorithm { + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + } }; Ok(S3Request::builder() @@ -310,6 +330,13 @@ pub struct CopyObject { metadata_directive: Option, #[builder(default, setter(into))] tagging_directive: Option, + /// Optional checksum algorithm for data integrity verification during copy. + /// + /// When specified, the server computes a checksum of the destination object using + /// this algorithm during the copy operation. Supported algorithms: CRC32, CRC32C, + /// SHA1, SHA256, CRC64NVME. The checksum value is included in response headers for verification. + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`CopyObject`] that is returned by [`MinioClient::copy_object`](crate::s3::client::MinioClient::copy_object). @@ -331,6 +358,7 @@ pub type CopyObjectBldr = CopyObjectBuilder<( (), (), (), + (), )>; impl CopyObject { @@ -434,6 +462,7 @@ impl CopyObject { .source(self.source) .metadata_directive(self.metadata_directive) .tagging_directive(self.tagging_directive) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await?; @@ -472,6 +501,8 @@ pub struct ComposeObjectInternal { legal_hold: bool, #[builder(default)] sources: Vec, + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`ComposeObjectInternal`] that is returned by `compose_object_internal` method. @@ -491,6 +522,7 @@ pub type ComposeObjectInternalBldr = ComposeObjectInternalBuilder<( (), (), (), + (), )>; impl ComposeObjectInternal { @@ -523,10 +555,10 @@ impl ComposeObjectInternal { .legal_hold(self.legal_hold) .source( CopySource::builder() - .bucket(&self.bucket) - .object(&self.object) + .bucket(&sources[0].bucket) + .object(&sources[0].object) .build(), - ) // TODO redundant use of bucket and object + ) .build() .send() .await @@ -554,6 +586,7 @@ impl ComposeObjectInternal { .extra_query_params(self.extra_query_params.clone()) .region(self.region.clone()) .extra_headers(Some(headers)) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await @@ -612,6 +645,7 @@ impl ComposeObjectInternal { .region(self.region.clone()) .part_number(part_number) .headers(headers) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await @@ -625,11 +659,10 @@ impl ComposeObjectInternal { Err(e) => return (Err(e.into()), upload_id), }; - parts.push(PartInfo { - number: part_number, - etag, - size, - }); + let checksum = self + .checksum_algorithm + .and_then(|alg| resp.get_checksum(alg).map(|v| (alg, v))); + parts.push(PartInfo::new(part_number, etag, size, checksum)); } else { let part_ranges = calculate_part_ranges(offset, size, MAX_PART_SIZE); for (part_offset, length) in part_ranges { @@ -648,6 +681,7 @@ impl ComposeObjectInternal { .region(self.region.clone()) .part_number(part_number) .headers(headers_copy) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await @@ -661,11 +695,10 @@ impl ComposeObjectInternal { Err(e) => return (Err(e.into()), upload_id), }; - parts.push(PartInfo { - number: part_number, - etag, - size: length, - }); + let checksum = self + .checksum_algorithm + .and_then(|alg| resp.get_checksum(alg).map(|v| (alg, v))); + parts.push(PartInfo::new(part_number, etag, length, checksum)); } } } @@ -725,6 +758,8 @@ pub struct ComposeObject { legal_hold: bool, #[builder(default)] sources: Vec, + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`ComposeObject`] that is returned by [`MinioClient::compose_object`](crate::s3::client::MinioClient::compose_object). @@ -744,6 +779,7 @@ pub type ComposeObjectBldr = ComposeObjectBuilder<( (), (), (Vec,), + (), )>; impl ComposeObject { @@ -766,6 +802,7 @@ impl ComposeObject { .retention(self.retention) .legal_hold(self.legal_hold) .sources(self.sources) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await; diff --git a/src/s3/builders/put_object.rs b/src/s3/builders/put_object.rs index 33c6697..cc62162 100644 --- a/src/s3/builders/put_object.rs +++ b/src/s3/builders/put_object.rs @@ -23,12 +23,14 @@ use crate::s3::response::{ AbortMultipartUploadResponse, CompleteMultipartUploadResponse, CreateMultipartUploadResponse, PutObjectContentResponse, PutObjectResponse, UploadPartResponse, }; -use crate::s3::response_traits::HasEtagFromHeaders; +use crate::s3::response_traits::{HasChecksumHeaders, HasEtagFromHeaders}; use crate::s3::segmented_bytes::SegmentedBytes; use crate::s3::sse::Sse; use crate::s3::types::{PartInfo, Retention, S3Api, S3Request, ToS3Request}; +use crate::s3::utils::{ + ChecksumAlgorithm, check_object_name, check_sse, compute_checksum_sb, insert, +}; use crate::s3::utils::{check_bucket_name, md5sum_hash, to_iso8601utc, url_encode}; -use crate::s3::utils::{check_object_name, check_sse, insert}; use bytes::{Bytes, BytesMut}; use http::Method; use std::{collections::HashMap, sync::Arc}; @@ -65,6 +67,13 @@ pub struct CreateMultipartUpload { legal_hold: bool, #[builder(default, setter(into))] content_type: Option, + /// Optional checksum algorithm to use for data integrity verification. + /// + /// When specified, the server will compute checksums for each uploaded part + /// using this algorithm. Supported algorithms: CRC32, CRC32C, SHA1, SHA256, CRC64NVME. + /// The checksum is included in response headers for verification. + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`CreateMultipartUpload`] that is returned by [`MinioClient::create_multipart_upload`](crate::s3::client::MinioClient::create_multipart_upload). @@ -83,6 +92,7 @@ pub type CreateMultipartUploadBldr = CreateMultipartUploadBuilder<( (), (), (), + (), )>; impl S3Api for CreateMultipartUpload { @@ -94,7 +104,7 @@ impl ToS3Request for CreateMultipartUpload { check_bucket_name(&self.bucket, true)?; check_object_name(&self.object)?; - let headers: Multimap = into_headers_put_object( + let mut headers: Multimap = into_headers_put_object( self.extra_headers, self.user_metadata, self.sse, @@ -104,6 +114,10 @@ impl ToS3Request for CreateMultipartUpload { self.content_type, )?; + if let Some(algorithm) = self.checksum_algorithm { + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + } + Ok(S3Request::builder() .client(self.client) .method(Method::POST) @@ -197,6 +211,14 @@ pub struct CompleteMultipartUpload { upload_id: String, #[builder(!default)] // force required parts: Vec, + /// Optional checksum algorithm used during multipart upload. + /// + /// When specified and all parts were uploaded with the same checksum algorithm, + /// the server will compute a composite checksum (checksum-of-checksums) for the + /// entire object. This must match the algorithm used in CreateMultipartUpload + /// and all UploadPart operations. Supported algorithms: CRC32, CRC32C, SHA1, SHA256, CRC64NVME. + #[builder(default, setter(into))] + checksum_algorithm: Option, } /// Builder type for [`CompleteMultipartUpload`] that is returned by [`MinioClient::complete_multipart_upload`](crate::s3::client::MinioClient::complete_multipart_upload). @@ -211,6 +233,7 @@ pub type CompleteMultipartUploadBldr = CompleteMultipartUploadBuilder<( (String,), (String,), (Vec,), + (), )>; impl S3Api for CompleteMultipartUpload { @@ -235,14 +258,37 @@ impl ToS3Request for CompleteMultipartUpload { // Set the capacity of the byte-buffer based on the part count - attempting // to avoid extra allocations when building the XML payload. let bytes: Bytes = { - let mut data = BytesMut::with_capacity(100 * self.parts.len() + 100); + let mut data = BytesMut::with_capacity(200 * self.parts.len() + 100); data.extend_from_slice(b""); for part in self.parts.iter() { data.extend_from_slice(b""); data.extend_from_slice(part.number.to_string().as_bytes()); data.extend_from_slice(b""); data.extend_from_slice(part.etag.as_bytes()); - data.extend_from_slice(b""); + data.extend_from_slice(b""); + if let Some((algorithm, ref value)) = part.checksum { + let (open_tag, close_tag) = match algorithm { + ChecksumAlgorithm::CRC32 => { + (&b""[..], &b""[..]) + } + ChecksumAlgorithm::CRC32C => { + (&b""[..], &b""[..]) + } + ChecksumAlgorithm::SHA1 => { + (&b""[..], &b""[..]) + } + ChecksumAlgorithm::SHA256 => { + (&b""[..], &b""[..]) + } + ChecksumAlgorithm::CRC64NVME => { + (&b""[..], &b""[..]) + } + }; + data.extend_from_slice(open_tag); + data.extend_from_slice(value.as_bytes()); + data.extend_from_slice(close_tag); + } + data.extend_from_slice(b""); } data.extend_from_slice(b""); data.freeze() @@ -252,6 +298,10 @@ impl ToS3Request for CompleteMultipartUpload { { headers.add(CONTENT_TYPE, "application/xml"); headers.add(CONTENT_MD5, md5sum_hash(bytes.as_ref())); + + if let Some(algorithm) = self.checksum_algorithm { + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + } } let mut query_params: Multimap = self.extra_query_params.unwrap_or_default(); query_params.add("uploadId", self.upload_id); @@ -311,6 +361,35 @@ pub struct UploadPart { upload_id: Option, #[builder(default, setter(into))] // force required part_number: Option, + + /// Optional checksum algorithm to use for this part's data integrity verification. + /// + /// When specified, computes a checksum of the part data using the selected algorithm + /// (CRC32, CRC32C, SHA1, SHA256, or CRC64NVME). The checksum is sent with the upload + /// and verified by the server. For multipart uploads, use the same algorithm for all parts. + #[builder(default, setter(into))] + checksum_algorithm: Option, + + /// When true and `checksum_algorithm` is set, uses trailing checksums. + /// + /// Trailing checksums use aws-chunked encoding where the checksum is computed + /// incrementally while streaming and appended at the end of the request body. + /// This avoids buffering the entire content to compute the checksum upfront. + /// + /// Defaults to false for backwards compatibility. + #[builder(default = false)] + use_trailing_checksum: bool, + + /// When true, signs each chunk with AWS Signature V4 for streaming uploads. + /// + /// Requires `use_trailing_checksum` to be true and `checksum_algorithm` to be set. + /// Uses STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER where each chunk is signed + /// and the trailer includes a trailer signature. This provides additional + /// integrity verification at the protocol level. + /// + /// Defaults to false for backwards compatibility. + #[builder(default = false)] + use_signed_streaming: bool, } /// Builder type for [`UploadPart`] that is returned by [`MinioClient::upload_part`](crate::s3::client::MinioClient::upload_part). @@ -332,6 +411,9 @@ pub type UploadPartBldr = UploadPartBuilder<( (), (Option,), (Option,), + (), + (), + (), )>; impl S3Api for UploadPart { @@ -360,7 +442,7 @@ impl ToS3Request for UploadPart { } } - let headers: Multimap = into_headers_put_object( + let mut headers: Multimap = into_headers_put_object( self.extra_headers, self.user_metadata, self.sse, @@ -370,6 +452,31 @@ impl ToS3Request for UploadPart { self.content_type, )?; + // Determine if we're using trailing checksums + let trailing_checksum = if self.use_trailing_checksum && self.checksum_algorithm.is_some() { + self.checksum_algorithm + } else { + None + }; + + // For upfront checksums (not trailing), compute and add to headers + if let Some(algorithm) = self.checksum_algorithm + && !self.use_trailing_checksum + { + let checksum_value = compute_checksum_sb(algorithm, &self.data); + headers.add(X_AMZ_CHECKSUM_ALGORITHM, algorithm.as_str().to_string()); + + match algorithm { + ChecksumAlgorithm::CRC32 => headers.add(X_AMZ_CHECKSUM_CRC32, checksum_value), + ChecksumAlgorithm::CRC32C => headers.add(X_AMZ_CHECKSUM_CRC32C, checksum_value), + ChecksumAlgorithm::SHA1 => headers.add(X_AMZ_CHECKSUM_SHA1, checksum_value), + ChecksumAlgorithm::SHA256 => headers.add(X_AMZ_CHECKSUM_SHA256, checksum_value), + ChecksumAlgorithm::CRC64NVME => { + headers.add(X_AMZ_CHECKSUM_CRC64NVME, checksum_value) + } + } + } + let mut query_params: Multimap = self.extra_query_params.unwrap_or_default(); if let Some(upload_id) = self.upload_id { @@ -388,6 +495,8 @@ impl ToS3Request for UploadPart { .object(self.object) .headers(headers) .body(self.data) + .trailing_checksum(trailing_checksum) + .use_signed_streaming(self.use_signed_streaming) .build()) } } @@ -454,6 +563,29 @@ pub struct PutObjectContent { part_size: Size, #[builder(default, setter(into))] content_type: Option, + #[builder(default, setter(into))] + checksum_algorithm: Option, + + /// When true and `checksum_algorithm` is set, uses trailing checksums. + /// + /// Trailing checksums use aws-chunked encoding where the checksum is computed + /// incrementally while streaming and appended at the end of the request body. + /// This avoids buffering the entire content to compute the checksum upfront. + /// + /// Defaults to false for backwards compatibility. + #[builder(default = false)] + use_trailing_checksum: bool, + + /// When true, signs each chunk with AWS Signature V4 for streaming uploads. + /// + /// Requires `use_trailing_checksum` to be true and `checksum_algorithm` to be set. + /// Uses STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER where each chunk is signed + /// and the trailer includes a trailer signature. This provides additional + /// integrity verification at the protocol level. + /// + /// Defaults to false for backwards compatibility. + #[builder(default = false)] + use_signed_streaming: bool, // source data #[builder(!default, setter(into))] // force required + accept Into @@ -484,6 +616,9 @@ pub type PutObjectContentBldr = PutObjectContentBuilder<( (), (), (), + (), + (), + (), (ObjectContent,), )>; @@ -542,6 +677,9 @@ impl PutObjectContent { upload_id: None, data: Arc::new(seg_bytes), content_type: self.content_type.clone(), + checksum_algorithm: self.checksum_algorithm, + use_trailing_checksum: self.use_trailing_checksum, + use_signed_streaming: self.use_signed_streaming, }) .build() .send() @@ -572,6 +710,7 @@ impl PutObjectContent { .retention(self.retention.clone()) .legal_hold(self.legal_hold) .content_type(self.content_type.clone()) + .checksum_algorithm(self.checksum_algorithm) .build() .send() .await?; @@ -667,15 +806,22 @@ impl PutObjectContent { upload_id: Some(upload_id.to_string()), data: Arc::new(part_content), content_type: self.content_type.clone(), + checksum_algorithm: self.checksum_algorithm, + use_trailing_checksum: self.use_trailing_checksum, + use_signed_streaming: self.use_signed_streaming, } .send() .await?; - parts.push(PartInfo { - number: part_number, - etag: resp.etag()?, - size: buffer_size, - }); + let checksum = self + .checksum_algorithm + .and_then(|alg| resp.get_checksum(alg).map(|v| (alg, v))); + parts.push(PartInfo::new( + part_number, + resp.etag()?, + buffer_size, + checksum, + )); // Finally, check if we are done. if buffer_size < part_size { @@ -705,6 +851,7 @@ impl PutObjectContent { region: self.region, parts, upload_id, + checksum_algorithm: self.checksum_algorithm, } .send() .await?; diff --git a/src/s3/client/mod.rs b/src/s3/client/mod.rs index 760d90f..ece55d4 100644 --- a/src/s3/client/mod.rs +++ b/src/s3/client/mod.rs @@ -57,8 +57,10 @@ use crate::s3::multimap_ext::{Multimap, MultimapExt}; use crate::s3::response::*; use crate::s3::response_traits::{HasEtagFromHeaders, HasS3Fields}; use crate::s3::segmented_bytes::SegmentedBytes; -use crate::s3::signer::{SigningKeyCache, sign_v4_s3}; -use crate::s3::utils::{EMPTY_SHA256, check_ssec_with_log, sha256_hash_sb, to_amz_date, utc_now}; +use crate::s3::signer::{SigningKeyCache, sign_v4_s3, sign_v4_s3_with_context}; +use crate::s3::utils::{ + ChecksumAlgorithm, EMPTY_SHA256, check_ssec_with_log, sha256_hash_sb, to_amz_date, utc_now, +}; mod append_object; mod bucket_exists; @@ -663,8 +665,14 @@ impl MinioClient { bucket_name: Option<&str>, object_name: Option<&str>, body: Option>, + trailing_checksum: Option, + use_signed_streaming: bool, retry: bool, ) -> Result { + use crate::s3::aws_chunked::{ + AwsChunkedEncoder, RechunkingStream, SignedAwsChunkedEncoder, + }; + let mut url = self.shared.base_url.build_url( method, region, @@ -675,19 +683,61 @@ impl MinioClient { let mut extensions = http::Extensions::default(); headers.add(HOST, url.host_header_value()); + + // Determine if we're using trailing checksums (signed or unsigned) + let use_trailing = trailing_checksum.is_some() + && matches!(*method, Method::PUT | Method::POST) + && body.is_some(); + let use_signed_trailing = use_trailing && use_signed_streaming; + let sha256: String = match *method { Method::PUT | Method::POST => { if !headers.contains_key(CONTENT_TYPE) { // Empty body with Content-Type can cause some MinIO versions to expect XML headers.add(CONTENT_TYPE, "application/octet-stream"); } - let len: usize = body.as_ref().map_or(0, |b| b.len()); - headers.add(CONTENT_LENGTH, len.to_string()); - match body { - None => EMPTY_SHA256.into(), - Some(ref v) => { - let clone = v.clone(); - async_std::task::spawn_blocking(move || sha256_hash_sb(clone)).await + let raw_len: usize = body.as_ref().map_or(0, |b| b.len()); + + if use_trailing { + // For trailing checksums, use aws-chunked encoding + let algorithm = trailing_checksum.unwrap(); + + // Set headers for aws-chunked encoding + headers.add(CONTENT_ENCODING, "aws-chunked"); + headers.add(X_AMZ_DECODED_CONTENT_LENGTH, raw_len.to_string()); + headers.add(X_AMZ_TRAILER, algorithm.header_name()); + + // Calculate the encoded length for Content-Length + let encoded_len = if use_signed_trailing { + crate::s3::aws_chunked::calculate_signed_encoded_length( + raw_len as u64, + crate::s3::aws_chunked::default_chunk_size(), + algorithm, + ) + } else { + crate::s3::aws_chunked::calculate_encoded_length( + raw_len as u64, + crate::s3::aws_chunked::default_chunk_size(), + algorithm, + ) + }; + headers.add(CONTENT_LENGTH, encoded_len.to_string()); + + // Use appropriate Content-SHA256 value + if use_signed_trailing { + STREAMING_AWS4_HMAC_SHA256_PAYLOAD_TRAILER.into() + } else { + STREAMING_UNSIGNED_PAYLOAD_TRAILER.into() + } + } else { + // Standard upfront checksum + headers.add(CONTENT_LENGTH, raw_len.to_string()); + match body { + None => EMPTY_SHA256.into(), + Some(ref v) => { + let clone = v.clone(); + async_std::task::spawn_blocking(move || sha256_hash_sb(clone)).await + } } } } @@ -723,24 +773,46 @@ impl MinioClient { headers.add("x-minio-redirect-to", url.to_string()); } - if let Some(p) = &self.shared.provider { + // For signed streaming, we need the signing context for chunk signatures + let chunk_signing_context = if let Some(p) = &self.shared.provider { let creds = p.fetch(); if creds.session_token.is_some() { headers.add(X_AMZ_SECURITY_TOKEN, creds.session_token.unwrap()); } - sign_v4_s3( - &self.shared.signing_key_cache, - method, - &url.path, - region, - headers, - query_params, - &creds.access_key, - &creds.secret_key, - &sha256, - date, - ); - } + + if use_signed_trailing { + // Use the version that returns chunk signing context + Some(sign_v4_s3_with_context( + &self.shared.signing_key_cache, + method, + &url.path, + region, + headers, + query_params, + &creds.access_key, + &creds.secret_key, + &sha256, + date, + )) + } else { + // Standard signing without context + sign_v4_s3( + &self.shared.signing_key_cache, + method, + &url.path, + region, + headers, + query_params, + &creds.access_key, + &creds.secret_key, + &sha256, + date, + ); + None + } + } else { + None + }; let mut req = self.http_client.request(method.clone(), url.to_string()); @@ -766,7 +838,27 @@ impl MinioClient { None => BodyIterator::Empty(std::iter::empty()), }; let stream = futures_util::stream::iter(iter.map(|b| -> Result<_, Error> { Ok(b) })); - req = req.body(Body::wrap_stream(stream)); + + if use_signed_trailing { + // Wrap stream with signed aws-chunked encoder for trailing checksum. + // Re-chunk the stream to match the chunk size used in Content-Length calculation, + // ensuring the actual encoded bytes match the declared Content-Length. + let algorithm = trailing_checksum.unwrap(); + let context = + chunk_signing_context.expect("signing context required for signed streaming"); + let rechunked = RechunkingStream::with_default_chunk_size(stream); + let encoder = SignedAwsChunkedEncoder::new(rechunked, algorithm, context); + req = req.body(Body::wrap_stream(encoder)); + } else if use_trailing { + // Wrap stream with unsigned aws-chunked encoder for trailing checksum. + // Re-chunk the stream to match the chunk size used in Content-Length calculation. + let algorithm = trailing_checksum.unwrap(); + let rechunked = RechunkingStream::with_default_chunk_size(stream); + let encoder = AwsChunkedEncoder::new(rechunked, algorithm); + req = req.body(Body::wrap_stream(encoder)); + } else { + req = req.body(Body::wrap_stream(stream)); + } } let resp = req.send().await; @@ -825,6 +917,8 @@ impl MinioClient { bucket_name: &Option<&str>, object_name: &Option<&str>, data: Option>, + trailing_checksum: Option, + use_signed_streaming: bool, ) -> Result { let resp: Result = self .execute_internal( @@ -835,6 +929,8 @@ impl MinioClient { bucket_name.as_deref(), object_name.as_deref(), data.as_ref().map(Arc::clone), + trailing_checksum, + use_signed_streaming, true, ) .await; @@ -859,6 +955,8 @@ impl MinioClient { bucket_name.as_deref(), object_name.as_deref(), data, + trailing_checksum, + use_signed_streaming, false, ) .await diff --git a/src/s3/error.rs b/src/s3/error.rs index 6362eb2..5343c26 100644 --- a/src/s3/error.rs +++ b/src/s3/error.rs @@ -148,6 +148,9 @@ pub enum ValidationErr { got: u32, }, + #[error("Checksum mismatch; expected: {expected}, computed: {computed}")] + ChecksumMismatch { expected: String, computed: String }, + #[error("Unknown event type: {0}")] UnknownEventType(String), diff --git a/src/s3/mod.rs b/src/s3/mod.rs index 4cf2a9f..a3b036e 100644 --- a/src/s3/mod.rs +++ b/src/s3/mod.rs @@ -15,6 +15,7 @@ //! Implementation of Simple Storage Service (aka S3) client +pub mod aws_chunked; pub mod builders; pub mod client; pub mod creds; diff --git a/src/s3/response/append_object.rs b/src/s3/response/append_object.rs index 392a31e..b41eecc 100644 --- a/src/s3/response/append_object.rs +++ b/src/s3/response/append_object.rs @@ -14,7 +14,8 @@ // limitations under the License. use crate::s3::response_traits::{ - HasBucket, HasEtagFromHeaders, HasObject, HasObjectSize, HasRegion, HasVersion, + HasBucket, HasChecksumHeaders, HasEtagFromHeaders, HasObject, HasObjectSize, HasRegion, + HasVersion, }; use crate::s3::types::S3Request; use crate::{impl_from_s3response, impl_has_s3fields}; @@ -40,3 +41,4 @@ impl HasRegion for AppendObjectResponse {} impl HasVersion for AppendObjectResponse {} impl HasEtagFromHeaders for AppendObjectResponse {} impl HasObjectSize for AppendObjectResponse {} +impl HasChecksumHeaders for AppendObjectResponse {} diff --git a/src/s3/response/copy_object.rs b/src/s3/response/copy_object.rs index 83d5b6d..affc076 100644 --- a/src/s3/response/copy_object.rs +++ b/src/s3/response/copy_object.rs @@ -13,7 +13,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::s3::response_traits::{HasBucket, HasEtagFromBody, HasObject, HasRegion, HasVersion}; +use crate::s3::response_traits::{ + HasBucket, HasChecksumHeaders, HasEtagFromBody, HasObject, HasRegion, HasVersion, +}; use crate::s3::types::S3Request; use crate::{impl_from_s3response, impl_has_s3fields}; use bytes::Bytes; @@ -35,6 +37,7 @@ impl HasObject for S3Response2 {} impl HasRegion for S3Response2 {} impl HasVersion for S3Response2 {} impl HasEtagFromBody for S3Response2 {} +impl HasChecksumHeaders for S3Response2 {} /// Represents the response of the `upload_part_copy` API call. /// This struct contains metadata and information about the part being copied during a multipart upload. diff --git a/src/s3/response/get_object.rs b/src/s3/response/get_object.rs index ca22b1b..17b7736 100644 --- a/src/s3/response/get_object.rs +++ b/src/s3/response/get_object.rs @@ -16,12 +16,22 @@ use crate::impl_has_s3fields; use crate::s3::builders::ObjectContent; use crate::s3::error::{Error, ValidationErr}; -use crate::s3::response_traits::{HasBucket, HasEtagFromHeaders, HasObject, HasRegion, HasVersion}; +use crate::s3::response_traits::{ + HasBucket, HasChecksumHeaders, HasEtagFromHeaders, HasObject, HasRegion, HasVersion, +}; use crate::s3::types::{FromS3Response, S3Request}; +use crate::s3::utils::{ChecksumAlgorithm, b64_encode, compute_checksum}; use async_trait::async_trait; use bytes::Bytes; -use futures_util::TryStreamExt; +use crc_fast::{CrcAlgorithm, Digest as CrcFastDigest}; +use futures_util::{Stream, TryStreamExt}; use http::HeaderMap; +#[cfg(feature = "ring")] +use ring::digest::{Context, SHA256}; +use sha1::{Digest as Sha1Digest, Sha1}; +#[cfg(not(feature = "ring"))] +use sha2::Sha256; +use std::io; use std::mem; use std::pin::Pin; @@ -30,12 +40,174 @@ pub type BoxedByteStream = ( Pin> + Send>>, u64, ); +use std::task::{Context as TaskContext, Poll}; + +/// Stateful checksum hasher for streaming verification. +/// +/// This enum provides incremental checksum computation across multiple data chunks, +/// enabling efficient verification of large objects without loading them entirely into memory. +/// Each variant wraps the appropriate hasher implementation for its algorithm. +/// +/// The hasher is used internally by [`GetObjectResponse::content()`] to verify checksums +/// transparently during streaming, with minimal performance overhead. +enum ChecksumHasher { + Crc32(CrcFastDigest), + Crc32c(CrcFastDigest), + Crc64nvme(CrcFastDigest), + Sha1(Sha1), + #[cfg(feature = "ring")] + Sha256(Context), + #[cfg(not(feature = "ring"))] + Sha256(Sha256), +} + +impl ChecksumHasher { + /// Creates a new checksum hasher for the specified algorithm. + /// + /// Initializes the appropriate hasher implementation with cached instances + /// for CRC variants to optimize performance. + /// + /// # Arguments + /// + /// * `algorithm` - The checksum algorithm to use for verification + fn new(algorithm: ChecksumAlgorithm) -> Self { + match algorithm { + ChecksumAlgorithm::CRC32 => { + ChecksumHasher::Crc32(CrcFastDigest::new(CrcAlgorithm::Crc32IsoHdlc)) + } + ChecksumAlgorithm::CRC32C => { + ChecksumHasher::Crc32c(CrcFastDigest::new(CrcAlgorithm::Crc32Iscsi)) + } + ChecksumAlgorithm::CRC64NVME => { + ChecksumHasher::Crc64nvme(CrcFastDigest::new(CrcAlgorithm::Crc64Nvme)) + } + ChecksumAlgorithm::SHA1 => ChecksumHasher::Sha1(Sha1::new()), + #[cfg(feature = "ring")] + ChecksumAlgorithm::SHA256 => ChecksumHasher::Sha256(Context::new(&SHA256)), + #[cfg(not(feature = "ring"))] + ChecksumAlgorithm::SHA256 => ChecksumHasher::Sha256(Sha256::new()), + } + } + + /// Updates the checksum computation with a new chunk of data. + /// + /// This method is called incrementally as data streams through, allowing + /// verification without buffering the entire object in memory. + /// + /// # Arguments + /// + /// * `data` - The next chunk of data to include in the checksum + fn update(&mut self, data: &[u8]) { + match self { + ChecksumHasher::Crc32(digest) => digest.update(data), + ChecksumHasher::Crc32c(digest) => digest.update(data), + ChecksumHasher::Crc64nvme(digest) => digest.update(data), + ChecksumHasher::Sha1(hasher) => hasher.update(data), + #[cfg(feature = "ring")] + ChecksumHasher::Sha256(ctx) => ctx.update(data), + #[cfg(not(feature = "ring"))] + ChecksumHasher::Sha256(hasher) => hasher.update(data), + } + } + + /// Completes the checksum computation and returns the base64-encoded result. + /// + /// This consumes the hasher and produces the final checksum value in the format + /// expected by S3 headers (base64-encoded). The result can be compared directly + /// with the checksum value from response headers. + /// + /// # Returns + /// + /// Base64-encoded checksum string matching the S3 header format. + fn finalize(self) -> String { + match self { + // crc-fast returns u64 for all algorithms; CRC32 variants need cast to u32 + ChecksumHasher::Crc32(digest) => b64_encode((digest.finalize() as u32).to_be_bytes()), + ChecksumHasher::Crc32c(digest) => b64_encode((digest.finalize() as u32).to_be_bytes()), + ChecksumHasher::Crc64nvme(digest) => b64_encode(digest.finalize().to_be_bytes()), + ChecksumHasher::Sha1(hasher) => { + let result = hasher.finalize(); + b64_encode(&result[..]) + } + #[cfg(feature = "ring")] + ChecksumHasher::Sha256(ctx) => b64_encode(ctx.finish().as_ref()), + #[cfg(not(feature = "ring"))] + ChecksumHasher::Sha256(hasher) => { + let result = hasher.finalize(); + b64_encode(&result[..]) + } + } + } +} + +/// A stream wrapper that computes checksum incrementally while streaming data +struct ChecksumVerifyingStream { + inner: S, + hasher: Option, + expected_checksum: String, + finished: bool, +} + +impl ChecksumVerifyingStream +where + S: Stream> + Unpin, +{ + fn new(stream: S, algorithm: ChecksumAlgorithm, expected_checksum: String) -> Self { + Self { + inner: stream, + hasher: Some(ChecksumHasher::new(algorithm)), + expected_checksum, + finished: false, + } + } +} + +impl Stream for ChecksumVerifyingStream +where + S: Stream> + Unpin, +{ + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + if self.finished { + return Poll::Ready(None); + } + + match Pin::new(&mut self.inner).poll_next(cx) { + Poll::Ready(Some(Ok(bytes))) => { + if let Some(hasher) = &mut self.hasher { + hasher.update(&bytes); + } + Poll::Ready(Some(Ok(bytes))) + } + Poll::Ready(Some(Err(e))) => { + self.finished = true; + Poll::Ready(Some(Err(io::Error::other(e)))) + } + Poll::Ready(None) => { + self.finished = true; + if let Some(hasher) = self.hasher.take() { + let computed = hasher.finalize(); + if computed != self.expected_checksum { + return Poll::Ready(Some(Err(io::Error::other(format!( + "Checksum mismatch: expected {}, computed {}", + self.expected_checksum, computed + ))))); + } + } + Poll::Ready(None) + } + Poll::Pending => Poll::Pending, + } + } +} pub struct GetObjectResponse { request: S3Request, headers: HeaderMap, body: Bytes, // Note: not used resp: reqwest::Response, + verify_checksum: bool, } impl_has_s3fields!(GetObjectResponse); @@ -45,11 +217,62 @@ impl HasRegion for GetObjectResponse {} impl HasObject for GetObjectResponse {} impl HasVersion for GetObjectResponse {} impl HasEtagFromHeaders for GetObjectResponse {} +impl HasChecksumHeaders for GetObjectResponse {} impl GetObjectResponse { + /// Checks if the checksum is a composite (multipart) checksum. + /// + /// Composite checksums are returned for objects uploaded via multipart upload. + /// They represent a checksum-of-checksums and cannot be verified by computing + /// a checksum over the full object data. + /// + /// Detection is based solely on the `x-amz-checksum-type: COMPOSITE` header. + /// We intentionally do NOT try to detect composite checksums by parsing the + /// checksum value for a `-N` suffix, as this could cause false positives if + /// the server uses base64url encoding (which includes `-` in its alphabet). + fn is_composite_checksum(&self) -> bool { + if let Some(checksum_type) = self.checksum_type() + && checksum_type.eq_ignore_ascii_case("COMPOSITE") + { + return true; + } + false + } + /// Returns the content of the object as a (streaming) byte buffer. Note: consumes the response. + /// + /// If `verify_checksum` is enabled and the server provided checksums, the stream will + /// automatically verify the checksum incrementally as data is read, maintaining streaming performance. + /// + /// **Note on multipart objects**: Objects uploaded via multipart upload have COMPOSITE checksums + /// (checksum-of-checksums) which cannot be verified by computing a checksum over the downloaded + /// data. For these objects, checksum verification is automatically skipped. pub fn content(self) -> Result { let content_length: u64 = self.object_size()?; + + // Skip verification for composite checksums (multipart uploads) + // Composite checksums are checksum-of-checksums and cannot be verified + // by computing a checksum over the full object data + if self.is_composite_checksum() { + log::debug!( + "Skipping checksum verification for composite checksum (multipart upload). \ + Composite checksums cannot be verified without part boundaries." + ); + let body = self.resp.bytes_stream().map_err(std::io::Error::other); + return Ok(ObjectContent::new_from_stream(body, Some(content_length))); + } + + if let (true, Some(algorithm)) = (self.verify_checksum, self.detect_checksum_algorithm()) + && let Some(expected) = self.get_checksum(algorithm) + { + let stream = self.resp.bytes_stream(); + let verifying_stream = ChecksumVerifyingStream::new(stream, algorithm, expected); + return Ok(ObjectContent::new_from_stream( + verifying_stream, + Some(content_length), + )); + } + let body = self.resp.bytes_stream().map_err(std::io::Error::other); Ok(ObjectContent::new_from_stream(body, Some(content_length))) } @@ -78,12 +301,68 @@ impl GetObjectResponse { .map_err(|e| ValidationErr::HttpError(e).into()) } + /// Sets whether to automatically verify checksums when calling `content()`. + /// Default is `true`. Verification is performed incrementally during streaming with minimal overhead. + /// Set to `false` to disable checksum verification entirely. + pub fn with_verification(mut self, verify: bool) -> Self { + self.verify_checksum = verify; + self + } + /// Returns the content size (in Bytes) of the object. pub fn object_size(&self) -> Result { self.resp .content_length() .ok_or(ValidationErr::ContentLengthUnknown) } + + /// Returns the content with automatic checksum verification. + /// + /// Downloads the full content, computes its checksum, and verifies against server checksum. + /// + /// **Note on multipart objects**: Objects uploaded via multipart upload have COMPOSITE checksums + /// (checksum-of-checksums) which cannot be verified by computing a checksum over the downloaded + /// data. For these objects, checksum verification is automatically skipped and the content is + /// returned without verification. + pub async fn content_verified(self) -> Result { + // Skip verification for composite checksums (multipart uploads) + if self.is_composite_checksum() { + log::debug!( + "Skipping checksum verification for composite checksum (multipart upload). \ + Composite checksums cannot be verified without part boundaries." + ); + return self + .resp + .bytes() + .await + .map_err(|e| ValidationErr::HttpError(e).into()); + } + + let algorithm = self.detect_checksum_algorithm(); + let expected_checksum = algorithm.and_then(|algo| self.get_checksum(algo)); + + let bytes = self.resp.bytes().await.map_err(ValidationErr::HttpError)?; + + if let (Some(algo), Some(expected)) = (algorithm, expected_checksum) { + let computed = compute_checksum(algo, &bytes); + + if computed != expected { + return Err(Error::Validation(ValidationErr::ChecksumMismatch { + expected, + computed, + })); + } + } + + Ok(bytes) + } + + /// Returns whether the object has a composite checksum (from multipart upload). + /// + /// This can be used to check if checksum verification will be skipped. + pub fn has_composite_checksum(&self) -> bool { + self.is_composite_checksum() + } } #[async_trait] @@ -98,6 +377,7 @@ impl FromS3Response for GetObjectResponse { headers: mem::take(resp.headers_mut()), body: Bytes::new(), resp, + verify_checksum: true, // Default to auto-verify }) } } diff --git a/src/s3/response/put_object.rs b/src/s3/response/put_object.rs index 42b8088..b08fb5d 100644 --- a/src/s3/response/put_object.rs +++ b/src/s3/response/put_object.rs @@ -14,7 +14,9 @@ // limitations under the License. use crate::s3::error::ValidationErr; -use crate::s3::response_traits::{HasBucket, HasEtagFromHeaders, HasObject, HasRegion, HasVersion}; +use crate::s3::response_traits::{ + HasBucket, HasChecksumHeaders, HasEtagFromHeaders, HasObject, HasRegion, HasVersion, +}; use crate::s3::types::S3Request; use crate::s3::utils::get_text_result; use crate::{impl_from_s3response, impl_from_s3response_with_size, impl_has_s3fields}; @@ -40,6 +42,7 @@ impl HasObject for S3Response1 {} impl HasRegion for S3Response1 {} impl HasVersion for S3Response1 {} impl HasEtagFromHeaders for S3Response1 {} +impl HasChecksumHeaders for S3Response1 {} /// Extended response struct for operations that need additional data like object size #[derive(Clone, Debug)] @@ -60,6 +63,7 @@ impl HasObject for S3Response1WithSize {} impl HasRegion for S3Response1WithSize {} impl HasVersion for S3Response1WithSize {} impl HasEtagFromHeaders for S3Response1WithSize {} +impl HasChecksumHeaders for S3Response1WithSize {} impl S3Response1WithSize { pub fn new(response: S3Response1, object_size: u64) -> Self { @@ -93,6 +97,7 @@ impl HasObject for S3MultipartResponse {} impl HasRegion for S3MultipartResponse {} impl HasVersion for S3MultipartResponse {} impl HasEtagFromHeaders for S3MultipartResponse {} +impl HasChecksumHeaders for S3MultipartResponse {} impl S3MultipartResponse { /// Returns the upload ID for the multipart upload, while consuming the response. diff --git a/src/s3/response/stat_object.rs b/src/s3/response/stat_object.rs index 7e80f73..a1570c0 100644 --- a/src/s3/response/stat_object.rs +++ b/src/s3/response/stat_object.rs @@ -16,8 +16,8 @@ use crate::s3::error::ValidationErr; use crate::s3::header_constants::*; use crate::s3::response_traits::{ - HasBucket, HasEtagFromHeaders, HasIsDeleteMarker, HasObject, HasObjectSize, HasRegion, - HasS3Fields, HasVersion, + HasBucket, HasChecksumHeaders, HasEtagFromHeaders, HasIsDeleteMarker, HasObject, HasObjectSize, + HasRegion, HasS3Fields, HasVersion, }; use crate::s3::types::S3Request; use crate::s3::types::{RetentionMode, parse_legal_hold}; @@ -46,6 +46,7 @@ impl HasRegion for StatObjectResponse {} impl HasObject for StatObjectResponse {} impl HasEtagFromHeaders for StatObjectResponse {} impl HasIsDeleteMarker for StatObjectResponse {} +impl HasChecksumHeaders for StatObjectResponse {} impl HasVersion for StatObjectResponse {} impl HasObjectSize for StatObjectResponse {} diff --git a/src/s3/response_traits.rs b/src/s3/response_traits.rs index 04a1ae2..dfd0ea3 100644 --- a/src/s3/response_traits.rs +++ b/src/s3/response_traits.rs @@ -67,7 +67,7 @@ use crate::s3::error::ValidationErr; use crate::s3::header_constants::*; use crate::s3::types::S3Request; -use crate::s3::utils::{get_text_result, parse_bool, trim_quotes}; +use crate::s3::utils::{ChecksumAlgorithm, get_text_result, parse_bool, trim_quotes}; use bytes::{Buf, Bytes}; use http::HeaderMap; use std::collections::HashMap; @@ -275,3 +275,90 @@ pub trait HasTagging: HasS3Fields { Ok(tags) } } + +/// Provides checksum-related methods for S3 responses with headers. +/// +/// This trait provides default implementations for extracting and detecting checksums +/// from S3 response headers. Implement this trait for any response type that has +/// `HeaderMap` access via `HasS3Fields`. +pub trait HasChecksumHeaders: HasS3Fields { + /// Extracts the checksum value from response headers for the specified algorithm. + /// + /// Retrieves the base64-encoded checksum value from the appropriate S3 response header + /// (x-amz-checksum-crc32, x-amz-checksum-crc32c, x-amz-checksum-crc64nvme, + /// x-amz-checksum-sha1, or x-amz-checksum-sha256). + /// + /// # Arguments + /// + /// * `algorithm` - The checksum algorithm to retrieve + /// + /// # Returns + /// + /// - `Some(checksum)` if the header is present, containing the base64-encoded checksum value + /// - `None` if the header is not found + /// + /// # Use Cases + /// + /// - Compare with locally computed checksums for manual verification + /// - Store checksum values for audit or compliance records + /// - Verify integrity after downloading to disk + #[inline] + fn get_checksum(&self, algorithm: ChecksumAlgorithm) -> Option { + let header_name = match algorithm { + ChecksumAlgorithm::CRC32 => X_AMZ_CHECKSUM_CRC32, + ChecksumAlgorithm::CRC32C => X_AMZ_CHECKSUM_CRC32C, + ChecksumAlgorithm::SHA1 => X_AMZ_CHECKSUM_SHA1, + ChecksumAlgorithm::SHA256 => X_AMZ_CHECKSUM_SHA256, + ChecksumAlgorithm::CRC64NVME => X_AMZ_CHECKSUM_CRC64NVME, + }; + + self.headers() + .get(header_name) + .and_then(|v| v.to_str().ok()) + .map(|s| s.to_string()) + } + + /// Returns the checksum type from response headers. + /// + /// The checksum type indicates whether the checksum represents: + /// - `FULL_OBJECT` - A checksum computed over the entire object + /// - `COMPOSITE` - A checksum-of-checksums for multipart uploads + /// + /// # Returns + /// + /// - `Some(type_string)` if the `x-amz-checksum-type` header is present + /// - `None` if the header is not found + #[inline] + fn checksum_type(&self) -> Option { + self.headers() + .get(X_AMZ_CHECKSUM_TYPE) + .and_then(|v| v.to_str().ok()) + .map(|s| s.to_string()) + } + + /// Detects which checksum algorithm was used by the server (if any). + /// + /// Examines response headers to determine if the server computed a checksum + /// for this operation. + /// + /// # Returns + /// + /// - `Some(algorithm)` if a checksum header is found (CRC32, CRC32C, CRC64NVME, SHA1, or SHA256) + /// - `None` if no checksum headers are present + #[inline] + fn detect_checksum_algorithm(&self) -> Option { + if self.headers().contains_key(X_AMZ_CHECKSUM_CRC32) { + Some(ChecksumAlgorithm::CRC32) + } else if self.headers().contains_key(X_AMZ_CHECKSUM_CRC32C) { + Some(ChecksumAlgorithm::CRC32C) + } else if self.headers().contains_key(X_AMZ_CHECKSUM_CRC64NVME) { + Some(ChecksumAlgorithm::CRC64NVME) + } else if self.headers().contains_key(X_AMZ_CHECKSUM_SHA1) { + Some(ChecksumAlgorithm::SHA1) + } else if self.headers().contains_key(X_AMZ_CHECKSUM_SHA256) { + Some(ChecksumAlgorithm::SHA256) + } else { + None + } + } +} diff --git a/src/s3/signer.rs b/src/s3/signer.rs index 1a4fa12..10afe51 100644 --- a/src/s3/signer.rs +++ b/src/s3/signer.rs @@ -394,6 +394,155 @@ pub(crate) fn post_presign_v4( get_signature(&signing_key, string_to_sign.as_bytes()) } +// =========================== +// Chunk Signing for STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER +// =========================== + +/// Context required for signing streaming chunks. +/// +/// This struct captures the parameters needed to sign each chunk +/// in STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER uploads. +/// The signing key can be reused for all chunks since it only depends +/// on date/region/service which don't change within a request. +#[derive(Debug, Clone)] +pub struct ChunkSigningContext { + /// The signing key (Arc for zero-copy sharing across chunks) + pub signing_key: Arc<[u8]>, + /// AMZ date format: 20241219T120000Z + pub date_time: String, + /// Credential scope: 20241219/us-east-1/s3/aws4_request + pub scope: String, + /// The seed signature from Authorization header (hex-encoded) + pub seed_signature: String, +} + +/// Signs a single chunk for STREAMING-AWS4-HMAC-SHA256-PAYLOAD streaming. +/// +/// The string-to-sign format is: +/// ```text +/// AWS4-HMAC-SHA256-PAYLOAD +/// +/// +/// +/// +/// +/// ``` +/// +/// # Arguments +/// * `signing_key` - The derived signing key +/// * `date_time` - AMZ date format (e.g., "20130524T000000Z") +/// * `scope` - Credential scope (e.g., "20130524/us-east-1/s3/aws4_request") +/// * `previous_signature` - Previous chunk's signature (or seed signature for first chunk) +/// * `chunk_hash` - Pre-computed SHA256 hash of the chunk data (hex-encoded) +/// +/// # Returns +/// The hex-encoded signature for this chunk. +pub fn sign_chunk( + signing_key: &[u8], + date_time: &str, + scope: &str, + previous_signature: &str, + chunk_hash: &str, +) -> String { + // SHA256 of empty string (constant per AWS spec) + const EMPTY_SHA256: &str = "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"; + + // Build string-to-sign per AWS spec + let string_to_sign = format!( + "AWS4-HMAC-SHA256-PAYLOAD\n{}\n{}\n{}\n{}\n{}", + date_time, scope, previous_signature, EMPTY_SHA256, chunk_hash + ); + + hmac_hash_hex(signing_key, string_to_sign.as_bytes()) +} + +/// Signs the trailer for STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER. +/// +/// The string-to-sign format is: +/// ```text +/// AWS4-HMAC-SHA256-TRAILER +/// +/// +/// +/// +/// ``` +/// +/// # Arguments +/// * `signing_key` - The derived signing key +/// * `date_time` - AMZ date format (e.g., "20130524T000000Z") +/// * `scope` - Credential scope (e.g., "20130524/us-east-1/s3/aws4_request") +/// * `last_chunk_signature` - The signature of the final 0-byte chunk +/// * `canonical_trailers_hash` - Pre-computed SHA256 hash of canonical trailers (hex-encoded) +/// +/// The canonical trailers format is: `:\n` +/// Example: `x-amz-checksum-crc32c:sOO8/Q==\n` +/// +/// **Note**: The canonical form uses LF (`\n`), not CRLF (`\r\n`), even though +/// the HTTP wire format uses CRLF. This is per AWS SigV4 specification. +/// +/// # Returns +/// The hex-encoded trailer signature. +pub fn sign_trailer( + signing_key: &[u8], + date_time: &str, + scope: &str, + last_chunk_signature: &str, + canonical_trailers_hash: &str, +) -> String { + // Build string-to-sign per AWS spec + let string_to_sign = format!( + "AWS4-HMAC-SHA256-TRAILER\n{}\n{}\n{}\n{}", + date_time, scope, last_chunk_signature, canonical_trailers_hash + ); + + hmac_hash_hex(signing_key, string_to_sign.as_bytes()) +} + +/// Signs request and returns context for chunk signing. +/// +/// This is used for STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER uploads +/// where the encoder needs the signing key and seed signature to sign +/// each chunk during streaming. +/// +/// The `cache` parameter should be the per-client `signing_key_cache` from `SharedClientItems`. +pub(crate) fn sign_v4_s3_with_context( + cache: &RwLock, + method: &Method, + uri: &str, + region: &str, + headers: &mut Multimap, + query_params: &Multimap, + access_key: &str, + secret_key: &str, + content_sha256: &str, + date: UtcTime, +) -> ChunkSigningContext { + let scope = get_scope(date, region, "s3"); + let (signed_headers, canonical_headers) = headers.get_canonical_headers(); + let canonical_query_string = query_params.get_canonical_query_string(); + let canonical_request_hash = get_canonical_request_hash( + method, + uri, + &canonical_query_string, + &canonical_headers, + &signed_headers, + content_sha256, + ); + let string_to_sign = get_string_to_sign(date, &scope, &canonical_request_hash); + let signing_key = get_signing_key(cache, secret_key, date, region, "s3"); + let signature = get_signature(&signing_key, string_to_sign.as_bytes()); + let authorization = get_authorization(access_key, &scope, &signed_headers, &signature); + + headers.add(AUTHORIZATION, authorization); + + ChunkSigningContext { + signing_key, + date_time: to_amz_date(date), + scope, + seed_signature: signature, + } +} + #[cfg(test)] mod tests { use super::*; @@ -762,4 +911,274 @@ mod tests { assert_eq!(sig1, sig2); } + + // =========================== + // Chunk Signing Tests + // =========================== + + #[test] + fn test_sign_chunk_produces_valid_signature() { + // Use signing key derived from AWS test credentials + let cache = test_cache(); + let secret_key = "wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let previous_signature = "4f232c4386841ef735655705268965c44a0e4690baa4adea153f7db9fa80a0a9"; + // SHA256 of some test data + let chunk_hash = "bf718b6f653bebc184e1479f1935b8da974d701b893afcf49e701f3e2f9f9c5a"; + + let signature = sign_chunk( + &signing_key, + date_time, + scope, + previous_signature, + chunk_hash, + ); + + // Should produce 64 character hex signature + assert_eq!(signature.len(), 64); + assert!(signature.chars().all(|c| c.is_ascii_hexdigit())); + } + + #[test] + fn test_sign_chunk_deterministic() { + let cache = test_cache(); + let secret_key = "test_secret"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let previous_signature = "abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234"; + let chunk_hash = "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"; + + let sig1 = sign_chunk( + &signing_key, + date_time, + scope, + previous_signature, + chunk_hash, + ); + let sig2 = sign_chunk( + &signing_key, + date_time, + scope, + previous_signature, + chunk_hash, + ); + + assert_eq!(sig1, sig2); + } + + #[test] + fn test_sign_chunk_empty_data() { + let cache = test_cache(); + let secret_key = "test_secret"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let previous_signature = "abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234"; + // SHA256 of empty data + let chunk_hash = "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"; + + let signature = sign_chunk( + &signing_key, + date_time, + scope, + previous_signature, + chunk_hash, + ); + + // Should still produce valid signature for empty chunk + assert_eq!(signature.len(), 64); + assert!(signature.chars().all(|c| c.is_ascii_hexdigit())); + } + + #[test] + fn test_sign_chunk_chaining() { + let cache = test_cache(); + let secret_key = "test_secret"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let seed_signature = "seed1234seed1234seed1234seed1234seed1234seed1234seed1234seed1234"; + let chunk1_hash = "aaaa1111aaaa1111aaaa1111aaaa1111aaaa1111aaaa1111aaaa1111aaaa1111"; + let chunk2_hash = "bbbb2222bbbb2222bbbb2222bbbb2222bbbb2222bbbb2222bbbb2222bbbb2222"; + + // Sign first chunk with seed signature + let chunk1_sig = sign_chunk(&signing_key, date_time, scope, seed_signature, chunk1_hash); + + // Sign second chunk with first chunk's signature + let chunk2_sig = sign_chunk(&signing_key, date_time, scope, &chunk1_sig, chunk2_hash); + + // Signatures should be different + assert_ne!(chunk1_sig, chunk2_sig); + + // Both should be valid hex + assert_eq!(chunk1_sig.len(), 64); + assert_eq!(chunk2_sig.len(), 64); + } + + #[test] + fn test_sign_trailer_produces_valid_signature() { + let cache = test_cache(); + let secret_key = "wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let last_chunk_signature = + "b6c6ea8a5354eaf15b3cb7646744f4275b71ea724fed81ceb9323e279d449df9"; + // SHA256 of "x-amz-checksum-crc32c:sOO8/Q==\n" + let canonical_trailers_hash = + "1e376db7e1a34a8ef1c4bcee131a2d60a1cb62503747488624e10995f448d774"; + + let signature = sign_trailer( + &signing_key, + date_time, + scope, + last_chunk_signature, + canonical_trailers_hash, + ); + + // Should produce 64 character hex signature + assert_eq!(signature.len(), 64); + assert!(signature.chars().all(|c| c.is_ascii_hexdigit())); + } + + #[test] + fn test_sign_trailer_deterministic() { + let cache = test_cache(); + let secret_key = "test_secret"; + let date = get_test_date(); + let region = "us-east-1"; + + let signing_key = get_signing_key(&cache, secret_key, date, region, "s3"); + let date_time = "20130524T000000Z"; + let scope = "20130524/us-east-1/s3/aws4_request"; + let last_chunk_signature = + "abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234"; + let canonical_trailers_hash = + "1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd1234abcd"; + + let sig1 = sign_trailer( + &signing_key, + date_time, + scope, + last_chunk_signature, + canonical_trailers_hash, + ); + let sig2 = sign_trailer( + &signing_key, + date_time, + scope, + last_chunk_signature, + canonical_trailers_hash, + ); + + assert_eq!(sig1, sig2); + } + + #[test] + fn test_sign_v4_s3_with_context_returns_valid_context() { + let cache = test_cache(); + let method = Method::PUT; + let uri = "/examplebucket/chunkObject.txt"; + let region = "us-east-1"; + let mut headers = Multimap::new(); + let date = get_test_date(); + let content_sha256 = "STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER"; + let access_key = "AKIAIOSFODNN7EXAMPLE"; + let secret_key = "wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY"; + + headers.add(HOST, "s3.amazonaws.com"); + headers.add(X_AMZ_CONTENT_SHA256, content_sha256); + headers.add(X_AMZ_DATE, "20130524T000000Z"); + headers.add("Content-Encoding", "aws-chunked"); + headers.add("x-amz-decoded-content-length", "66560"); + headers.add("x-amz-trailer", "x-amz-checksum-crc32c"); + + let query_params = Multimap::new(); + + let context = sign_v4_s3_with_context( + &cache, + &method, + uri, + region, + &mut headers, + &query_params, + access_key, + secret_key, + content_sha256, + date, + ); + + // Authorization header should be added + assert!(headers.contains_key("Authorization")); + let auth_header = headers.get("Authorization").unwrap(); + assert!(auth_header.starts_with("AWS4-HMAC-SHA256")); + + // Context should have valid values + assert!(!context.signing_key.is_empty()); + assert_eq!(context.date_time, "20130524T000000Z"); + assert_eq!(context.scope, "20130524/us-east-1/s3/aws4_request"); + assert_eq!(context.seed_signature.len(), 64); + assert!( + context + .seed_signature + .chars() + .all(|c| c.is_ascii_hexdigit()) + ); + } + + #[test] + fn test_chunk_signing_context_can_be_cloned() { + let cache = test_cache(); + let method = Method::PUT; + let uri = "/test"; + let region = "us-east-1"; + let mut headers = Multimap::new(); + let date = get_test_date(); + let content_sha256 = "STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER"; + let access_key = "test"; + let secret_key = "secret"; + + headers.add(HOST, "s3.amazonaws.com"); + headers.add(X_AMZ_CONTENT_SHA256, content_sha256); + headers.add(X_AMZ_DATE, "20130524T000000Z"); + + let query_params = Multimap::new(); + + let context = sign_v4_s3_with_context( + &cache, + &method, + uri, + region, + &mut headers, + &query_params, + access_key, + secret_key, + content_sha256, + date, + ); + + // Should be cloneable (required for async streams) + let cloned = context.clone(); + assert_eq!(context.date_time, cloned.date_time); + assert_eq!(context.scope, cloned.scope); + assert_eq!(context.seed_signature, cloned.seed_signature); + } } diff --git a/src/s3/types/basic_types.rs b/src/s3/types/basic_types.rs index 80d382c..0656c17 100644 --- a/src/s3/types/basic_types.rs +++ b/src/s3/types/basic_types.rs @@ -16,7 +16,7 @@ //! Basic S3 data types: ListEntry, Bucket, Part, Retention, etc. use crate::s3::error::ValidationErr; -use crate::s3::utils::UtcTime; +use crate::s3::utils::{ChecksumAlgorithm, UtcTime}; use std::collections::HashMap; use std::fmt; @@ -53,11 +53,34 @@ pub struct Part { pub etag: String, } +/// Contains part information for multipart uploads including optional checksum. +/// +/// Only one checksum algorithm is active per upload, so the checksum is stored +/// as an optional tuple of (algorithm, base64-encoded value). #[derive(Clone, Debug)] pub struct PartInfo { pub number: u16, pub etag: String, pub size: u64, + /// Optional checksum for this part: (algorithm, base64-encoded value) + pub checksum: Option<(ChecksumAlgorithm, String)>, +} + +impl PartInfo { + /// Creates a new PartInfo. + pub fn new( + number: u16, + etag: String, + size: u64, + checksum: Option<(ChecksumAlgorithm, String)>, + ) -> Self { + Self { + number, + etag, + size, + checksum, + } + } } #[derive(PartialEq, Clone, Debug)] diff --git a/src/s3/types/header_constants.rs b/src/s3/types/header_constants.rs index 56dbfbf..f5f7e2f 100644 --- a/src/s3/types/header_constants.rs +++ b/src/s3/types/header_constants.rs @@ -105,3 +105,31 @@ pub const X_AMZ_COPY_SOURCE_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY: &str = pub const X_AMZ_COPY_SOURCE_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY_MD5: &str = "X-Amz-Copy-Source-Server-Side-Encryption-Customer-Key-MD5"; + +pub const X_AMZ_CHECKSUM_ALGORITHM: &str = "X-Amz-Checksum-Algorithm"; + +pub const X_AMZ_CHECKSUM_CRC32: &str = "X-Amz-Checksum-CRC32"; + +pub const X_AMZ_CHECKSUM_CRC32C: &str = "X-Amz-Checksum-CRC32C"; + +pub const X_AMZ_CHECKSUM_SHA1: &str = "X-Amz-Checksum-SHA1"; + +pub const X_AMZ_CHECKSUM_SHA256: &str = "X-Amz-Checksum-SHA256"; + +pub const X_AMZ_CHECKSUM_CRC64NVME: &str = "X-Amz-Checksum-CRC64NVME"; + +pub const X_AMZ_CHECKSUM_TYPE: &str = "X-Amz-Checksum-Type"; + +pub const X_AMZ_TRAILER: &str = "X-Amz-Trailer"; + +pub const X_AMZ_DECODED_CONTENT_LENGTH: &str = "X-Amz-Decoded-Content-Length"; + +pub const CONTENT_ENCODING: &str = "Content-Encoding"; + +/// Content-SHA256 value for streaming uploads with unsigned payload and trailing checksum +pub const STREAMING_UNSIGNED_PAYLOAD_TRAILER: &str = "STREAMING-UNSIGNED-PAYLOAD-TRAILER"; + +/// Content-SHA256 value for streaming uploads with signed payload and trailing checksum. +/// Each chunk is signed with AWS Signature V4, and the trailer includes a trailer signature. +pub const STREAMING_AWS4_HMAC_SHA256_PAYLOAD_TRAILER: &str = + "STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER"; diff --git a/src/s3/types/s3_request.rs b/src/s3/types/s3_request.rs index 211dc80..38fbccc 100644 --- a/src/s3/types/s3_request.rs +++ b/src/s3/types/s3_request.rs @@ -19,6 +19,7 @@ use super::super::client::{DEFAULT_REGION, MinioClient}; use crate::s3::error::Error; use crate::s3::multimap_ext::Multimap; use crate::s3::segmented_bytes::SegmentedBytes; +use crate::s3::utils::ChecksumAlgorithm; use http::Method; use std::sync::Arc; use typed_builder::TypedBuilder; @@ -50,6 +51,20 @@ pub struct S3Request { #[builder(default, setter(into))] body: Option>, + /// Optional trailing checksum algorithm for streaming uploads. + /// + /// When set, the request body will be sent using aws-chunked encoding + /// with the checksum computed incrementally and appended as a trailer. + #[builder(default)] + pub(crate) trailing_checksum: Option, + + /// When true and trailing checksums are enabled, signs each chunk with AWS Signature V4. + /// + /// Uses STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER where each chunk is signed + /// and the trailer includes a trailer signature. + #[builder(default = false)] + pub(crate) use_signed_streaming: bool, + /// region computed by [`S3Request::execute`] #[builder(default, setter(skip))] pub(crate) inner_region: String, @@ -76,6 +91,8 @@ impl S3Request { &self.bucket.as_deref(), &self.object.as_deref(), self.body.as_ref().map(Arc::clone), + self.trailing_checksum, + self.use_signed_streaming, ) .await } diff --git a/src/s3/utils.rs b/src/s3/utils.rs index bad58e9..2aff288 100644 --- a/src/s3/utils.rs +++ b/src/s3/utils.rs @@ -20,19 +20,21 @@ use crate::s3::segmented_bytes::SegmentedBytes; use crate::s3::sse::{Sse, SseCustomerKey}; use base64::engine::Engine as _; use chrono::{DateTime, Datelike, NaiveDateTime, Utc}; -use crc::{CRC_32_ISO_HDLC, Crc}; +use crc_fast::{CrcAlgorithm, Digest as CrcFastDigest, checksum as crc_fast_checksum}; use lazy_static::lazy_static; use percent_encoding::{AsciiSet, NON_ALPHANUMERIC, percent_decode_str, utf8_percent_encode}; use regex::Regex; #[cfg(feature = "ring")] use ring::digest::{Context, SHA256}; +use sha1::{Digest as Sha1Digest, Sha1}; #[cfg(not(feature = "ring"))] -use sha2::{Digest, Sha256}; +use sha2::Sha256; use std::collections::HashMap; +use std::str::FromStr; use std::sync::Arc; use xmltree::Element; -/// Date and time with UTC timezone +/// Date and time with UTC timezone. pub type UtcTime = DateTime; // Great stuff to get confused about. @@ -60,10 +62,20 @@ pub fn b64_encode(input: impl AsRef<[u8]>) -> String { base64::engine::general_purpose::STANDARD.encode(input) } -/// Computes CRC32 of given data. +/// Computes CRC32 of given data using hardware-accelerated SIMD implementation. +/// +/// Uses crc-fast which provides hardware acceleration via PCLMULQDQ/CLMUL instructions +/// on modern CPUs, achieving >50 GiB/s throughput (vs ~0.5 GiB/s for software). pub fn crc32(data: &[u8]) -> u32 { - //TODO creating a new Crc object is expensive, we should cache it - Crc::::new(&CRC_32_ISO_HDLC).checksum(data) + crc_fast_checksum(CrcAlgorithm::Crc32IsoHdlc, data) as u32 +} + +/// Computes CRC64-NVME of given data using hardware-accelerated SIMD implementation. +/// +/// Uses crc-fast which provides hardware acceleration via PCLMULQDQ/CLMUL instructions +/// on modern CPUs, achieving >50 GiB/s throughput (vs ~0.5 GiB/s for software). +pub fn crc64nvme(data: &[u8]) -> u64 { + crc_fast_checksum(CrcAlgorithm::Crc64Nvme, data) } /// Converts data array into 32-bit BigEndian unsigned int. @@ -168,9 +180,252 @@ pub fn sha256_hash_sb(sb: Arc) -> String { } } +/// S3 checksum algorithms supported by the API +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum ChecksumAlgorithm { + CRC32, + CRC32C, + SHA1, + SHA256, + CRC64NVME, +} + +impl ChecksumAlgorithm { + /// Returns the AWS header value for this checksum algorithm. + pub fn as_str(&self) -> &'static str { + match self { + ChecksumAlgorithm::CRC32 => "CRC32", + ChecksumAlgorithm::CRC32C => "CRC32C", + ChecksumAlgorithm::SHA1 => "SHA1", + ChecksumAlgorithm::SHA256 => "SHA256", + ChecksumAlgorithm::CRC64NVME => "CRC64NVME", + } + } + + /// Returns the HTTP header name for this checksum algorithm (e.g., "X-Amz-Checksum-CRC32"). + pub fn header_name(&self) -> &'static str { + use crate::s3::types::header_constants::*; + match self { + ChecksumAlgorithm::CRC32 => X_AMZ_CHECKSUM_CRC32, + ChecksumAlgorithm::CRC32C => X_AMZ_CHECKSUM_CRC32C, + ChecksumAlgorithm::SHA1 => X_AMZ_CHECKSUM_SHA1, + ChecksumAlgorithm::SHA256 => X_AMZ_CHECKSUM_SHA256, + ChecksumAlgorithm::CRC64NVME => X_AMZ_CHECKSUM_CRC64NVME, + } + } +} + +/// Parses a checksum algorithm name from a string. +/// +/// Case-insensitive parsing of S3 checksum algorithm names. Useful for parsing +/// header values or configuration strings. +/// +/// # Supported Values +/// +/// - `"CRC32"` / `"crc32"` - Standard CRC32 checksum +/// - `"CRC32C"` / `"crc32c"` - CRC32C (Castagnoli) checksum +/// - `"SHA1"` / `"sha1"` - SHA-1 hash +/// - `"SHA256"` / `"sha256"` - SHA-256 hash +/// - `"CRC64NVME"` / `"crc64nvme"` - CRC-64/NVME checksum +/// +/// # Errors +/// +/// Returns an error string if the algorithm name is not recognized. +impl FromStr for ChecksumAlgorithm { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_uppercase().as_str() { + "CRC32" => Ok(ChecksumAlgorithm::CRC32), + "CRC32C" => Ok(ChecksumAlgorithm::CRC32C), + "SHA1" => Ok(ChecksumAlgorithm::SHA1), + "SHA256" => Ok(ChecksumAlgorithm::SHA256), + "CRC64NVME" => Ok(ChecksumAlgorithm::CRC64NVME), + _ => Err(format!("Unknown checksum algorithm: {}", s)), + } + } +} + +/// Computes CRC32C checksum (Castagnoli polynomial) and returns base64-encoded value. +/// +/// Uses crc-fast which provides hardware acceleration via PCLMULQDQ/CLMUL instructions +/// on modern CPUs, achieving >50 GiB/s throughput. +pub fn crc32c(data: &[u8]) -> String { + let checksum = crc_fast_checksum(CrcAlgorithm::Crc32Iscsi, data) as u32; + b64_encode(checksum.to_be_bytes()) +} + +/// Computes SHA1 hash and returns base64-encoded value +pub fn sha1_hash(data: &[u8]) -> String { + let mut hasher = Sha1::new(); + hasher.update(data); + let result = hasher.finalize(); + b64_encode(&result[..]) +} + +/// Computes SHA256 hash and returns base64-encoded value (for checksums, not authentication) +pub fn sha256_checksum(data: &[u8]) -> String { + #[cfg(feature = "ring")] + { + b64_encode(ring::digest::digest(&SHA256, data).as_ref()) + } + #[cfg(not(feature = "ring"))] + { + let result = Sha256::new_with_prefix(data).finalize(); + b64_encode(&result[..]) + } +} + +/// Computes CRC32 checksum and returns base64-encoded value +pub fn crc32_checksum(data: &[u8]) -> String { + b64_encode(crc32(data).to_be_bytes()) +} + +/// Computes CRC64-NVME checksum and returns base64-encoded value +pub fn crc64nvme_checksum(data: &[u8]) -> String { + b64_encode(crc64nvme(data).to_be_bytes()) +} + +/// Computes checksum based on the specified algorithm for contiguous byte slices. +/// +/// This function computes checksums on already-materialized `&[u8]` data. Use this when: +/// - Data is already in a contiguous buffer (e.g., from `reqwest::Response::bytes()`) +/// - Working with small byte arrays in tests +/// - Data comes from sources other than `SegmentedBytes` +/// +/// **Performance Note**: If you have data in `SegmentedBytes`, use [`compute_checksum_sb`] +/// instead to avoid copying. Calling `.to_bytes()` on `SegmentedBytes` creates a full copy +/// of all segments, which is expensive for large objects (up to 5GB per part). +/// +/// # Arguments +/// +/// * `algorithm` - The checksum algorithm to use (CRC32, CRC32C, CRC64NVME, SHA1, SHA256) +/// * `data` - The contiguous byte slice to compute checksum over +/// +/// # Returns +/// +/// Base64-encoded checksum string suitable for S3 headers +/// +/// # Example +/// +/// ``` +/// use minio::s3::utils::{compute_checksum, ChecksumAlgorithm}; +/// +/// let data = b"hello world"; +/// let checksum = compute_checksum(ChecksumAlgorithm::CRC32C, data); +/// println!("CRC32C: {}", checksum); +/// ``` +pub fn compute_checksum(algorithm: ChecksumAlgorithm, data: &[u8]) -> String { + match algorithm { + ChecksumAlgorithm::CRC32 => crc32_checksum(data), + ChecksumAlgorithm::CRC32C => crc32c(data), + ChecksumAlgorithm::SHA1 => sha1_hash(data), + ChecksumAlgorithm::SHA256 => sha256_checksum(data), + ChecksumAlgorithm::CRC64NVME => crc64nvme_checksum(data), + } +} + +/// Computes checksum for `SegmentedBytes` without copying data (zero-copy streaming). +/// +/// This function computes checksums by iterating over segments incrementally, avoiding +/// the need to materialize the entire buffer in contiguous memory. This is critical for +/// performance when working with large objects (up to 5GB per part in multipart uploads). +/// +/// **Always use this function for `SegmentedBytes` data** instead of calling `.to_bytes()` +/// followed by `compute_checksum()`, which would create an expensive full copy. +/// +/// # Performance Characteristics +/// +/// - **Memory**: Only allocates hasher state (~64 bytes for SHA256, ~4-8 bytes for CRC) +/// - **CPU**: Hardware-accelerated where available (CRC32C with SSE 4.2) +/// - **Streaming**: Processes data incrementally without buffering +/// +/// # Arguments +/// +/// * `algorithm` - The checksum algorithm to use (CRC32, CRC32C, CRC64NVME, SHA1, SHA256) +/// * `sb` - The segmented bytes to compute checksum over (passed by reference, not consumed) +/// +/// # Returns +/// +/// Base64-encoded checksum string suitable for S3 headers +/// +/// # Example +/// +/// ``` +/// use minio::s3::utils::{compute_checksum_sb, ChecksumAlgorithm}; +/// use minio::s3::segmented_bytes::SegmentedBytes; +/// use std::sync::Arc; +/// use bytes::Bytes; +/// +/// let mut sb = SegmentedBytes::new(); +/// sb.append(Bytes::from("hello ")); +/// sb.append(Bytes::from("world")); +/// let sb = Arc::new(sb); +/// +/// let checksum = compute_checksum_sb(ChecksumAlgorithm::CRC32C, &sb); +/// println!("CRC32C: {}", checksum); +/// ``` +/// +/// # See Also +/// +/// - [`compute_checksum`] - For already-contiguous `&[u8]` data +pub fn compute_checksum_sb(algorithm: ChecksumAlgorithm, sb: &Arc) -> String { + match algorithm { + ChecksumAlgorithm::CRC32 => { + let mut digest = CrcFastDigest::new(CrcAlgorithm::Crc32IsoHdlc); + for data in sb.iter() { + digest.update(data.as_ref()); + } + b64_encode((digest.finalize() as u32).to_be_bytes()) + } + ChecksumAlgorithm::CRC32C => { + let mut digest = CrcFastDigest::new(CrcAlgorithm::Crc32Iscsi); + for data in sb.iter() { + digest.update(data.as_ref()); + } + b64_encode((digest.finalize() as u32).to_be_bytes()) + } + ChecksumAlgorithm::SHA1 => { + let mut hasher = Sha1::new(); + for data in sb.iter() { + hasher.update(data.as_ref()); + } + let result = hasher.finalize(); + b64_encode(&result[..]) + } + ChecksumAlgorithm::SHA256 => { + #[cfg(feature = "ring")] + { + let mut context = Context::new(&SHA256); + for data in sb.iter() { + context.update(data.as_ref()); + } + b64_encode(context.finish().as_ref()) + } + #[cfg(not(feature = "ring"))] + { + let mut hasher = Sha256::new(); + for data in sb.iter() { + hasher.update(data.as_ref()); + } + let result = hasher.finalize(); + b64_encode(&result[..]) + } + } + ChecksumAlgorithm::CRC64NVME => { + let mut digest = CrcFastDigest::new(CrcAlgorithm::Crc64Nvme); + for data in sb.iter() { + digest.update(data.as_ref()); + } + b64_encode(digest.finalize().to_be_bytes()) + } + } +} + #[cfg(test)] mod tests { use super::*; + use bytes::Bytes; use std::collections::HashMap; #[test] @@ -290,6 +545,93 @@ mod tests { assert_eq!(empty_hash, "1B2M2Y8AsgTpgAmY7PhCfg=="); } + #[test] + fn test_crc32c() { + let checksum = crc32c(b"hello"); + assert!(!checksum.is_empty()); + let checksum_empty = crc32c(b""); + assert!(!checksum_empty.is_empty()); + let checksum_standard = crc32c(b"123456789"); + assert!(!checksum_standard.is_empty()); + } + + #[test] + fn test_sha1_hash() { + let hash = sha1_hash(b"hello"); + assert!(!hash.is_empty()); + let hash_empty = sha1_hash(b""); + assert!(!hash_empty.is_empty()); + let hash_fox = sha1_hash(b"The quick brown fox jumps over the lazy dog"); + assert!(!hash_fox.is_empty()); + } + + #[test] + fn test_sha256_checksum() { + let checksum = sha256_checksum(b"hello"); + assert!(!checksum.is_empty()); + let checksum_empty = sha256_checksum(b""); + assert!(!checksum_empty.is_empty()); + } + + #[test] + fn test_crc32_checksum() { + let checksum = crc32_checksum(b"hello"); + assert!(!checksum.is_empty()); + let checksum_empty = crc32_checksum(b""); + assert_eq!(checksum_empty, "AAAAAA=="); + let checksum_standard = crc32_checksum(b"123456789"); + assert!(!checksum_standard.is_empty()); + } + + #[test] + fn test_checksum_algorithm_as_str() { + assert_eq!(ChecksumAlgorithm::CRC32.as_str(), "CRC32"); + assert_eq!(ChecksumAlgorithm::CRC32C.as_str(), "CRC32C"); + assert_eq!(ChecksumAlgorithm::SHA1.as_str(), "SHA1"); + assert_eq!(ChecksumAlgorithm::SHA256.as_str(), "SHA256"); + } + + #[test] + fn test_checksum_algorithm_from_str() { + assert_eq!( + "CRC32".parse::().unwrap(), + ChecksumAlgorithm::CRC32 + ); + assert_eq!( + "crc32c".parse::().unwrap(), + ChecksumAlgorithm::CRC32C + ); + assert_eq!( + "SHA1".parse::().unwrap(), + ChecksumAlgorithm::SHA1 + ); + assert_eq!( + "sha256".parse::().unwrap(), + ChecksumAlgorithm::SHA256 + ); + assert!("invalid".parse::().is_err()); + } + + #[test] + fn test_compute_checksum() { + let data = b"hello world"; + + let crc32_result = compute_checksum(ChecksumAlgorithm::CRC32, data); + assert!(!crc32_result.is_empty()); + + let crc32c_result = compute_checksum(ChecksumAlgorithm::CRC32C, data); + assert!(!crc32c_result.is_empty()); + + let sha1_result = compute_checksum(ChecksumAlgorithm::SHA1, data); + assert!(!sha1_result.is_empty()); + + let sha256_result = compute_checksum(ChecksumAlgorithm::SHA256, data); + assert!(!sha256_result.is_empty()); + + assert_ne!(crc32_result, crc32c_result); + assert_ne!(sha1_result, sha256_result); + } + #[test] fn test_parse_bool_true() { assert!(parse_bool("true").unwrap()); @@ -564,6 +906,7 @@ mod tests { #[test] fn test_match_region_basic() { let _result = match_region("us-east-1"); + // TODO consider fixing or removing this test // Test that match_region returns a boolean (always true) } @@ -647,6 +990,36 @@ mod tests { let tags = parse_tags("Environment=Production").unwrap(); assert!(!tags.is_empty()); } + + #[test] + fn test_compute_checksum_sb_matches_compute_checksum() { + // Test data + let test_data = b"The quick brown fox jumps over the lazy dog"; + + // Create SegmentedBytes with multiple segments to test incremental computation + let mut sb = SegmentedBytes::new(); + sb.append(Bytes::from(&test_data[0..10])); + sb.append(Bytes::from(&test_data[10..25])); + sb.append(Bytes::from(&test_data[25..])); + let sb = Arc::new(sb); + + // Test all algorithms + for algo in [ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ] { + let from_bytes = compute_checksum(algo, test_data); + let from_sb = compute_checksum_sb(algo, &sb); + assert_eq!( + from_bytes, from_sb, + "Mismatch for {:?}: bytes='{}' vs sb='{}'", + algo, from_bytes, from_sb + ); + } + } } /// Gets base64-encoded MD5 hash of given data. diff --git a/tests/s3/mod.rs b/tests/s3/mod.rs index c72724f..50651e6 100644 --- a/tests/s3/mod.rs +++ b/tests/s3/mod.rs @@ -21,10 +21,12 @@ mod client_config; // Object operations mod append_object; mod get_object; +mod object_checksums; mod object_compose; mod object_copy; mod object_delete; mod object_put; +mod test_checksums; mod upload_download_object; // Bucket operations diff --git a/tests/s3/object_checksums.rs b/tests/s3/object_checksums.rs new file mode 100644 index 0000000..5c90589 --- /dev/null +++ b/tests/s3/object_checksums.rs @@ -0,0 +1,2031 @@ +// MinIO Rust Library for Amazon S3 Compatible Cloud Storage +// Copyright 2025 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. + +use bytes::Bytes; +use minio::s3::builders::{ComposeSource, CopySource, ObjectContent, PutObject, UploadPart}; +use minio::s3::response::{ + AppendObjectResponse, ComposeObjectResponse, CopyObjectResponse, PutObjectContentResponse, + PutObjectResponse, +}; +use minio::s3::response_traits::{HasBucket, HasChecksumHeaders, HasObject, HasObjectSize}; +use minio::s3::segmented_bytes::SegmentedBytes; +use minio::s3::types::S3Api; +use minio::s3::utils::ChecksumAlgorithm; +use minio_common::rand_src::RandSrc; +use minio_common::test_context::TestContext; +use minio_common::utils::rand_object_name; +use std::sync::Arc; + +/// Helper function to upload an object with a specific checksum algorithm +async fn upload_with_checksum( + ctx: &TestContext, + bucket: &str, + object: &str, + data: &[u8], + algorithm: ChecksumAlgorithm, +) -> PutObjectResponse { + let inner = UploadPart::builder() + .client(ctx.client.clone()) + .bucket(bucket.to_string()) + .object(object.to_string()) + .data(Arc::new(SegmentedBytes::from(Bytes::from(data.to_vec())))) + .checksum_algorithm(algorithm) + .build(); + + PutObject::builder() + .inner(inner) + .build() + .send() + .await + .unwrap() +} + +/// Test uploading an object with CRC32 checksum +#[minio_macros::test] +async fn upload_with_crc32_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing CRC32 checksum."; + + let resp = upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with CRC32C checksum +#[minio_macros::test] +async fn upload_with_crc32c_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing CRC32C checksum."; + + let resp = upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32C, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with SHA1 checksum +#[minio_macros::test] +async fn upload_with_sha1_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing SHA1 checksum."; + + let resp = upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA1, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with SHA256 checksum +#[minio_macros::test] +async fn upload_with_sha256_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing SHA256 checksum."; + + let resp = upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA256, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with CRC64NVME checksum +#[minio_macros::test] +async fn upload_with_crc64nvme_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing CRC64NVME checksum."; + + let resp = upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip: upload with checksum and download with verification +#[minio_macros::test] +async fn upload_download_with_crc32c_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with CRC32C checksum verification."; + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32C, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + // Note: Server may or may not return checksums depending on configuration + // If checksums are available, verify them. If not, just check content matches. + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::CRC32C); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + // No checksum returned, just verify content + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip with SHA256 +#[minio_macros::test] +async fn upload_download_with_sha256_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with SHA256 checksum verification."; + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA256, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + // Note: Server may or may not return checksums depending on configuration + // If checksums are available, verify them. If not, just check content matches. + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::SHA256); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + // No checksum returned, just verify content + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip with CRC64NVME +#[minio_macros::test] +async fn upload_download_with_crc64nvme_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with CRC64NVME checksum verification."; + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + // Note: Server may or may not return checksums depending on configuration + // If checksums are available, verify them. If not, just check content matches. + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::CRC64NVME); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + // No checksum returned, just verify content + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test that downloading without checksum still works +#[minio_macros::test] +async fn upload_download_without_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Upload without checksum, should work fine."; + + ctx.client + .put_object( + &bucket_name, + &object_name, + SegmentedBytes::from(String::from_utf8_lossy(data).to_string()), + ) + .build() + .send() + .await + .unwrap(); + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + assert!( + algorithm.is_none(), + "No checksum algorithm should be detected" + ); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test checksum with larger data +#[minio_macros::test] +async fn upload_download_large_data_with_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = vec![0xAB; 1024 * 100]; // 100KB of data + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + &data, + ChecksumAlgorithm::CRC32C, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), data.len()); + assert_eq!(downloaded.as_ref(), data.as_slice()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test all checksum algorithms in sequence +#[minio_macros::test] +async fn test_all_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let object_name = format!("checksum-test-{:?}-{}", algo, rand_object_name()); + let data = format!("Testing {:?} checksum algorithm", algo); + + upload_with_checksum(&ctx, &bucket_name, &object_name, data.as_bytes(), algo).await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let detected_algo = get_resp.detect_checksum_algorithm(); + // Note: Server may or may not return checksums depending on configuration + // If checksums are available, verify them. If not, just check content matches. + if let Some(detected) = detected_algo { + assert_eq!(detected, algo, "Algorithm mismatch for {:?}", algo); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data.as_bytes()); + } else { + // No checksum returned, just verify content + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data.as_bytes()); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + } +} + +// ============================================================================ +// CRC32 and SHA1 round-trip tests +// ============================================================================ + +/// Test round-trip with CRC32 +#[minio_macros::test] +async fn upload_download_with_crc32_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with CRC32 checksum verification."; + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::CRC32); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip with SHA1 +#[minio_macros::test] +async fn upload_download_with_sha1_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with SHA1 checksum verification."; + + upload_with_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA1, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::SHA1); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +// ============================================================================ +// AppendObject checksum tests +// ============================================================================ + +/// Test AppendObject with CRC32C checksum +#[minio_macros::test(skip_if_not_express)] +async fn append_object_with_crc32c_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let content1 = "Initial content for append test."; + let content2 = "Appended content with checksum."; + + // Create initial object + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content1) + .build() + .send() + .await + .unwrap(); + + // Append with checksum + let data2 = SegmentedBytes::from(content2.to_string()); + let offset = content1.len() as u64; + let resp: AppendObjectResponse = ctx + .client + .append_object(&bucket_name, &object_name, data2, offset) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + assert_eq!(resp.object_size(), (content1.len() + content2.len()) as u64); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test AppendObject with all checksum algorithms +#[minio_macros::test(skip_if_not_express)] +async fn append_object_all_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let object_name = format!("append-checksum-{:?}-{}", algo, rand_object_name()); + let content1 = format!("Initial content for {:?}", algo); + let content2 = format!("Appended with {:?} checksum", algo); + + // Create initial object + let content1_len = content1.len(); + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content1) + .build() + .send() + .await + .unwrap(); + + // Append with checksum + let data2 = SegmentedBytes::from(content2.clone()); + let offset = content1_len as u64; + let resp: AppendObjectResponse = ctx + .client + .append_object(&bucket_name, &object_name, data2, offset) + .checksum_algorithm(algo) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + assert_eq!( + resp.object_size(), + (content1_len + content2.len()) as u64, + "Size mismatch for {:?}", + algo + ); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + } +} + +/// Test AppendObjectContent with checksum +#[minio_macros::test(skip_if_not_express)] +async fn append_object_content_with_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let content1 = "Initial content."; + let content2 = "Appended content with SHA256."; + + // Create initial object + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content1) + .build() + .send() + .await + .unwrap(); + + // Append content with checksum + let resp: AppendObjectResponse = ctx + .client + .append_object_content(&bucket_name, &object_name, content2) + .checksum_algorithm(ChecksumAlgorithm::SHA256) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + assert_eq!(resp.object_size(), (content1.len() + content2.len()) as u64); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +// ============================================================================ +// CopyObject checksum tests +// ============================================================================ + +/// Test CopyObject with CRC32C checksum +#[minio_macros::test(skip_if_express)] +async fn copy_object_with_crc32c_checksum(ctx: TestContext, bucket_name: String) { + let src_object = rand_object_name(); + let dst_object = rand_object_name(); + let data = b"Content to copy with checksum verification."; + + // Create source object + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object, Bytes::from_static(data)) + .build() + .send() + .await + .unwrap(); + + // Copy with checksum + let resp: CopyObjectResponse = ctx + .client + .copy_object(&bucket_name, &dst_object) + .source( + CopySource::builder() + .bucket(&bucket_name) + .object(&src_object) + .build(), + ) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), dst_object); + + // Verify the copy + let get_resp = ctx + .client + .get_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); +} + +/// Test CopyObject with all checksum algorithms +#[minio_macros::test(skip_if_express)] +async fn copy_object_all_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let src_object = format!("copy-src-{:?}-{}", algo, rand_object_name()); + let dst_object = format!("copy-dst-{:?}-{}", algo, rand_object_name()); + let data = format!("Content to copy with {:?}", algo); + + // Create source object + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object, data) + .build() + .send() + .await + .unwrap(); + + // Copy with checksum + let resp: CopyObjectResponse = ctx + .client + .copy_object(&bucket_name, &dst_object) + .source( + CopySource::builder() + .bucket(&bucket_name) + .object(&src_object) + .build(), + ) + .checksum_algorithm(algo) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name, "Bucket mismatch for {:?}", algo); + assert_eq!(resp.object(), dst_object, "Object mismatch for {:?}", algo); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + } +} + +// ============================================================================ +// ComposeObject checksum tests +// ============================================================================ + +/// Test ComposeObject with CRC32C checksum +#[minio_macros::test] +async fn compose_object_with_crc32c_checksum(ctx: TestContext, bucket_name: String) { + let src_object = rand_object_name(); + let dst_object = rand_object_name(); + let data = b"Content to compose with checksum verification."; + + // Create source object + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object, Bytes::from_static(data)) + .build() + .send() + .await + .unwrap(); + assert_eq!(resp.bucket(), bucket_name); + + // Compose with checksum + let sources = vec![ComposeSource::new(&bucket_name, &src_object).unwrap()]; + let resp: ComposeObjectResponse = ctx + .client + .compose_object(&bucket_name, &dst_object, sources) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), dst_object); + + // Verify the composed object + let get_resp = ctx + .client + .get_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); +} + +/// Test ComposeObject with all checksum algorithms +#[minio_macros::test] +async fn compose_object_all_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let src_object = format!("compose-src-{:?}-{}", algo, rand_object_name()); + let dst_object = format!("compose-dst-{:?}-{}", algo, rand_object_name()); + let data = format!("Content to compose with {:?}", algo); + + // Create source object + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object, data) + .build() + .send() + .await + .unwrap(); + + // Compose with checksum + let sources = vec![ComposeSource::new(&bucket_name, &src_object).unwrap()]; + let resp: ComposeObjectResponse = ctx + .client + .compose_object(&bucket_name, &dst_object, sources) + .checksum_algorithm(algo) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name, "Bucket mismatch for {:?}", algo); + assert_eq!(resp.object(), dst_object, "Object mismatch for {:?}", algo); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + } +} + +/// Test ComposeObject with multiple sources (multipart compose) +/// Note: Multi-source compose uses multipart copy which requires 5MB+ per source part +/// Checksum verification on multipart copy requires source objects to have checksums stored, +/// which is complex with streaming uploads. This test validates the basic multipart compose works. +#[minio_macros::test] +async fn compose_object_multiple_sources(ctx: TestContext, bucket_name: String) { + let src_object1 = rand_object_name(); + let src_object2 = rand_object_name(); + let dst_object = rand_object_name(); + + // Each source must be at least 5MB for multipart copy (except last part) + let size1: u64 = 5 * 1024 * 1024; // 5MB + let size2: u64 = 1024; // 1KB for final part (can be smaller) + + // Create source objects + let content1 = ObjectContent::new_from_stream(RandSrc::new(size1), Some(size1)); + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object1, content1) + .build() + .send() + .await + .unwrap(); + + let content2 = ObjectContent::new_from_stream(RandSrc::new(size2), Some(size2)); + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object2, content2) + .build() + .send() + .await + .unwrap(); + + // Compose multiple sources + let sources = vec![ + ComposeSource::new(&bucket_name, &src_object1).unwrap(), + ComposeSource::new(&bucket_name, &src_object2).unwrap(), + ]; + let resp: ComposeObjectResponse = ctx + .client + .compose_object(&bucket_name, &dst_object, sources) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), dst_object); + + // Verify the composed object size + let stat_resp = ctx + .client + .stat_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + assert_eq!(stat_resp.size().unwrap(), size1 + size2); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object1) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &src_object2) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); +} + +// ============================================================================ +// Trailing Checksum tests +// ============================================================================ + +/// Helper function to upload an object with a trailing checksum +async fn upload_with_trailing_checksum( + ctx: &TestContext, + bucket: &str, + object: &str, + data: &[u8], + algorithm: ChecksumAlgorithm, +) -> PutObjectResponse { + let inner = UploadPart::builder() + .client(ctx.client.clone()) + .bucket(bucket.to_string()) + .object(object.to_string()) + .data(Arc::new(SegmentedBytes::from(Bytes::from(data.to_vec())))) + .checksum_algorithm(algorithm) + .use_trailing_checksum(true) + .build(); + + PutObject::builder() + .inner(inner) + .build() + .send() + .await + .unwrap() +} + +/// Test uploading an object with trailing CRC32 checksum +#[minio_macros::test] +async fn upload_with_trailing_crc32_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing trailing CRC32 checksum."; + + let resp = upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + // Verify we can download the object + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with trailing CRC32C checksum +#[minio_macros::test] +async fn upload_with_trailing_crc32c_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing trailing CRC32C checksum."; + + let resp = upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32C, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with trailing CRC64NVME checksum +#[minio_macros::test] +async fn upload_with_trailing_crc64nvme_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing trailing CRC64NVME checksum."; + + let resp = upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with trailing SHA1 checksum +#[minio_macros::test] +async fn upload_with_trailing_sha1_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing trailing SHA1 checksum."; + + let resp = upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA1, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading an object with trailing SHA256 checksum +#[minio_macros::test] +async fn upload_with_trailing_sha256_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing trailing SHA256 checksum."; + + let resp = upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA256, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip with trailing CRC64NVME checksum (the new default in MinIO) +#[minio_macros::test] +async fn upload_download_with_trailing_crc64nvme_verification( + ctx: TestContext, + bucket_name: String, +) { + let object_name = rand_object_name(); + let data = b"Round-trip test with trailing CRC64NVME checksum verification."; + + upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + // Server may or may not return checksums depending on configuration + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::CRC64NVME); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test all checksum algorithms with trailing checksums +#[minio_macros::test] +async fn test_all_trailing_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let object_name = format!("trailing-checksum-test-{:?}-{}", algo, rand_object_name()); + let data = format!("Testing trailing {:?} checksum algorithm", algo); + + upload_with_trailing_checksum(&ctx, &bucket_name, &object_name, data.as_bytes(), algo) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let detected_algo = get_resp.detect_checksum_algorithm(); + if let Some(detected) = detected_algo { + assert_eq!(detected, algo, "Algorithm mismatch for trailing {:?}", algo); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data.as_bytes()); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data.as_bytes()); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + } +} + +/// Test trailing checksum with larger data to exercise chunked encoding. +/// +/// NOTE: This test requires a newer MinIO server that supports trailing checksums. +/// Older servers may fail with "IncompleteBody" errors. +/// Run with `cargo test -- --ignored` to include this test. +#[minio_macros::test(ignore = "Requires newer MinIO server with trailing checksum support")] +async fn upload_download_large_data_with_trailing_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // Use 100KB which is larger than the 64KB default chunk size + let data = vec![0xAB; 1024 * 100]; + + upload_with_trailing_checksum( + &ctx, + &bucket_name, + &object_name, + &data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), data.len()); + assert_eq!(downloaded.as_ref(), data.as_slice()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +// ============================================================================ +// Signed Streaming Checksum tests (STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER) +// ============================================================================ + +/// Helper function to upload an object with signed streaming and trailing checksum. +/// +/// This uses STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER where each chunk is +/// cryptographically signed along with the trailing checksum. +async fn upload_with_signed_streaming( + ctx: &TestContext, + bucket: &str, + object: &str, + data: &[u8], + algorithm: ChecksumAlgorithm, +) -> PutObjectResponse { + let inner = UploadPart::builder() + .client(ctx.client.clone()) + .bucket(bucket.to_string()) + .object(object.to_string()) + .data(Arc::new(SegmentedBytes::from(Bytes::from(data.to_vec())))) + .checksum_algorithm(algorithm) + .use_trailing_checksum(true) + .use_signed_streaming(true) + .build(); + + PutObject::builder() + .inner(inner) + .build() + .send() + .await + .unwrap() +} + +/// Test uploading with signed streaming and CRC32 checksum +#[minio_macros::test] +async fn upload_with_signed_streaming_crc32(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing signed streaming with CRC32 checksum."; + + let resp = upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + // Verify we can download the object + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading with signed streaming and CRC32C checksum +#[minio_macros::test] +async fn upload_with_signed_streaming_crc32c(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing signed streaming with CRC32C checksum."; + + let resp = upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC32C, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading with signed streaming and CRC64NVME checksum +#[minio_macros::test] +async fn upload_with_signed_streaming_crc64nvme(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing signed streaming with CRC64NVME checksum."; + + let resp = upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading with signed streaming and SHA1 checksum +#[minio_macros::test] +async fn upload_with_signed_streaming_sha1(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing signed streaming with SHA1 checksum."; + + let resp = upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA1, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test uploading with signed streaming and SHA256 checksum +#[minio_macros::test] +async fn upload_with_signed_streaming_sha256(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Hello, MinIO! Testing signed streaming with SHA256 checksum."; + + let resp = upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::SHA256, + ) + .await; + + assert_eq!(resp.bucket(), bucket_name.as_str()); + assert_eq!(resp.object(), object_name.as_str()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test round-trip with signed streaming: upload and download with verification +#[minio_macros::test] +async fn upload_download_with_signed_streaming_verification(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let data = b"Round-trip test with signed streaming CRC64NVME checksum verification."; + + upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let algorithm = get_resp.detect_checksum_algorithm(); + // Server may or may not return checksums depending on configuration + if let Some(algo) = algorithm { + assert_eq!(algo, ChecksumAlgorithm::CRC64NVME); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test all checksum algorithms with signed streaming +#[minio_macros::test] +async fn test_all_signed_streaming_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + for algo in algorithms { + let object_name = format!("signed-streaming-{:?}-{}", algo, rand_object_name()); + let data = format!("Testing signed streaming {:?} checksum algorithm", algo); + + upload_with_signed_streaming(&ctx, &bucket_name, &object_name, data.as_bytes(), algo).await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let detected_algo = get_resp.detect_checksum_algorithm(); + if let Some(detected) = detected_algo { + assert_eq!( + detected, algo, + "Algorithm mismatch for signed streaming {:?}", + algo + ); + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.as_ref(), data.as_bytes()); + } else { + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), data.as_bytes()); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + } +} + +/// Test signed streaming with larger data to exercise multiple chunks. +/// +/// This test uses 100KB of data which exceeds the default 64KB chunk size, +/// ensuring that multiple chunk signatures are generated and verified. +#[minio_macros::test] +async fn upload_download_large_data_with_signed_streaming(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // Use 100KB which is larger than the 64KB default chunk size + let data = vec![0xCD; 1024 * 100]; + + upload_with_signed_streaming( + &ctx, + &bucket_name, + &object_name, + &data, + ChecksumAlgorithm::CRC64NVME, + ) + .await; + + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), data.len()); + assert_eq!(downloaded.as_ref(), data.as_slice()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test PutObjectContent with signed streaming. +/// +/// This exercises the high-level put_object_content API with signed streaming enabled. +#[minio_macros::test] +async fn put_object_content_with_signed_streaming(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let content = "Testing PutObjectContent with signed streaming CRC64NVME checksum."; + + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC64NVME) + .use_trailing_checksum(true) + .use_signed_streaming(true) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + + // Verify the object + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), content.as_bytes()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test multipart upload with signed streaming. +/// +/// This uploads an object larger than 5MB to trigger multipart upload, +/// with signed streaming enabled on each part. +#[minio_macros::test] +async fn multipart_upload_with_signed_streaming(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // 6MB to ensure multipart upload (threshold is 5MB) + let size: u64 = 6 * 1024 * 1024; + + let content = ObjectContent::new_from_stream(RandSrc::new(size), Some(size)); + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC64NVME) + .use_trailing_checksum(true) + .use_signed_streaming(true) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + + // Download and verify + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), size as usize); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test PutObjectContent with trailing checksums +#[minio_macros::test] +async fn put_object_content_with_trailing_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + let content = "Testing PutObjectContent with trailing CRC64NVME checksum."; + + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC64NVME) + .use_trailing_checksum(true) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + + // Verify the object + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap().to_bytes(); + assert_eq!(bytes.as_ref(), content.as_bytes()); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +// ============================================================================ +// Multipart Upload Checksum tests +// ============================================================================ +// These tests verify that multipart uploads with checksums work correctly, +// including the handling of COMPOSITE checksums on download. + +/// Test multipart upload with CRC32C checksum and verify download works. +/// +/// This test uploads an object larger than 5MB to trigger multipart upload, +/// with checksums enabled. The resulting object will have a COMPOSITE checksum +/// (checksum-of-checksums) which cannot be verified by computing a hash over +/// the full object. The test verifies that: +/// 1. Upload succeeds with checksums +/// 2. Download works without checksum verification errors +/// 3. Content is correct +#[minio_macros::test] +async fn multipart_upload_with_checksum_crc32c(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // 6MB to ensure multipart upload (threshold is 5MB) + let size: u64 = 6 * 1024 * 1024; + + let content = ObjectContent::new_from_stream(RandSrc::new(size), Some(size)); + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), object_name); + + // Download and verify - should work even with composite checksum + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + // Check if composite checksum is detected (if server returns checksums) + let has_composite = get_resp.has_composite_checksum(); + let algorithm = get_resp.detect_checksum_algorithm(); + + // content_verified() should work without error (skips verification for composite) + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), size as usize); + + // Log for debugging + if algorithm.is_some() { + log::info!( + "Multipart object has checksum algorithm: {:?}, composite: {}", + algorithm, + has_composite + ); + } + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test multipart upload with CRC64NVME checksum (the recommended algorithm). +#[minio_macros::test] +async fn multipart_upload_with_checksum_crc64nvme(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // 6MB to ensure multipart upload + let size: u64 = 6 * 1024 * 1024; + + let content = ObjectContent::new_from_stream(RandSrc::new(size), Some(size)); + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC64NVME) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + + // Download with streaming verification (should skip for composite) + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content().unwrap(); + let bytes = downloaded.to_segmented_bytes().await.unwrap(); + assert_eq!(bytes.len(), size as usize); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test multipart upload with trailing checksums. +/// +/// NOTE: This test requires a newer MinIO server that supports trailing checksums +/// with multipart uploads. Older servers may fail with "IncompleteBody" errors. +/// Run with `cargo test -- --ignored` to include this test. +#[minio_macros::test( + ignore = "Requires newer MinIO server with trailing checksum + multipart support" +)] +async fn multipart_upload_with_trailing_checksum(ctx: TestContext, bucket_name: String) { + let object_name = rand_object_name(); + // 6MB to ensure multipart upload + let size: u64 = 6 * 1024 * 1024; + + let content = ObjectContent::new_from_stream(RandSrc::new(size), Some(size)); + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(ChecksumAlgorithm::CRC64NVME) + .use_trailing_checksum(true) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + + // Download and verify content + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), size as usize); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); +} + +/// Test all checksum algorithms with multipart upload. +#[minio_macros::test] +async fn multipart_upload_all_checksum_algorithms(ctx: TestContext, bucket_name: String) { + let algorithms = vec![ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ]; + + // 6MB to ensure multipart upload + let size: u64 = 6 * 1024 * 1024; + + for algo in algorithms { + let object_name = format!("multipart-{:?}-{}", algo, rand_object_name()); + + let content = ObjectContent::new_from_stream(RandSrc::new(size), Some(size)); + let resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &object_name, content) + .checksum_algorithm(algo) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name, "Bucket mismatch for {:?}", algo); + + // Download and verify - should work for all algorithms + let get_resp = ctx + .client + .get_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + + // content_verified should work (skips for composite) + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!( + downloaded.len(), + size as usize, + "Size mismatch for {:?}", + algo + ); + + ctx.client + .delete_object(&bucket_name, &object_name) + .build() + .send() + .await + .unwrap(); + } +} + +/// Test compose with multiple sources and checksums (creates multipart with composite checksum). +/// +/// NOTE: This test requires a newer MinIO server that supports compose operations +/// with checksum verification. Older servers may fail because they don't properly +/// store/return checksums on source objects needed for compose validation. +/// Run with `cargo test -- --ignored` to include this test. +#[minio_macros::test(ignore = "Requires newer MinIO server with compose + checksum support")] +async fn compose_multiple_sources_with_checksum(ctx: TestContext, bucket_name: String) { + let src_object1 = rand_object_name(); + let src_object2 = rand_object_name(); + let dst_object = rand_object_name(); + + // Each source must be at least 5MB for multipart copy (except last part) + let size1: u64 = 5 * 1024 * 1024; // 5MB + let size2: u64 = 1024; // 1KB for final part + + // Create source objects with checksums + let content1 = ObjectContent::new_from_stream(RandSrc::new(size1), Some(size1)); + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object1, content1) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + let content2 = ObjectContent::new_from_stream(RandSrc::new(size2), Some(size2)); + let _resp: PutObjectContentResponse = ctx + .client + .put_object_content(&bucket_name, &src_object2, content2) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + // Compose multiple sources with checksum + let sources = vec![ + ComposeSource::new(&bucket_name, &src_object1).unwrap(), + ComposeSource::new(&bucket_name, &src_object2).unwrap(), + ]; + let resp: ComposeObjectResponse = ctx + .client + .compose_object(&bucket_name, &dst_object, sources) + .checksum_algorithm(ChecksumAlgorithm::CRC32C) + .build() + .send() + .await + .unwrap(); + + assert_eq!(resp.bucket(), bucket_name); + assert_eq!(resp.object(), dst_object); + + // Download and verify - composite checksum handling + let get_resp = ctx + .client + .get_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); + + // Should work even with composite checksum + let downloaded = get_resp.content_verified().await.unwrap(); + assert_eq!(downloaded.len(), (size1 + size2) as usize); + + // Cleanup + ctx.client + .delete_object(&bucket_name, &src_object1) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &src_object2) + .build() + .send() + .await + .unwrap(); + ctx.client + .delete_object(&bucket_name, &dst_object) + .build() + .send() + .await + .unwrap(); +} diff --git a/tests/s3/test_checksums.rs b/tests/s3/test_checksums.rs new file mode 100644 index 0000000..64e1946 --- /dev/null +++ b/tests/s3/test_checksums.rs @@ -0,0 +1,267 @@ +// MinIO Rust Library for Amazon S3 Compatible Cloud Storage +// Copyright 2025 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. + +use minio::s3::utils::{ + ChecksumAlgorithm, compute_checksum, crc32_checksum, crc32c, crc64nvme_checksum, sha1_hash, + sha256_checksum, +}; + +/// Test CRC32 checksum computation +#[test] +fn test_crc32_checksum() { + let data = b"Hello, World!"; + let checksum = crc32_checksum(data); + + // Verify it's base64 encoded + assert!(!checksum.is_empty()); + assert!(base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok()); +} + +/// Test CRC32C checksum computation +#[test] +fn test_crc32c_checksum() { + let data = b"Hello, World!"; + let checksum = crc32c(data); + + // Verify it's base64 encoded + assert!(!checksum.is_empty()); + assert!(base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok()); +} + +/// Test CRC64-NVME checksum computation +#[test] +fn test_crc64nvme_checksum() { + let data = b"Hello, World!"; + let checksum = crc64nvme_checksum(data); + + // Verify it's base64 encoded + assert!(!checksum.is_empty()); + assert!(base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok()); + + // Verify it's different from CRC32/CRC32C (different algorithms produce different results) + let crc32_result = crc32_checksum(data); + assert_ne!(checksum, crc32_result); +} + +/// Test SHA1 hash computation +#[test] +fn test_sha1_hash() { + let data = b"Hello, World!"; + let hash = sha1_hash(data); + + // Verify it's base64 encoded + assert!(!hash.is_empty()); + assert!(base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &hash).is_ok()); +} + +/// Test SHA256 checksum computation +#[test] +fn test_sha256_checksum() { + let data = b"Hello, World!"; + let checksum = sha256_checksum(data); + + // Verify it's base64 encoded + assert!(!checksum.is_empty()); + assert!(base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok()); +} + +/// Test compute_checksum with all algorithms +#[test] +fn test_compute_checksum_all_algorithms() { + let data = b"Test data for checksums"; + + let crc32 = compute_checksum(ChecksumAlgorithm::CRC32, data); + let crc32c = compute_checksum(ChecksumAlgorithm::CRC32C, data); + let crc64nvme = compute_checksum(ChecksumAlgorithm::CRC64NVME, data); + let sha1 = compute_checksum(ChecksumAlgorithm::SHA1, data); + let sha256 = compute_checksum(ChecksumAlgorithm::SHA256, data); + + // All should be non-empty and valid base64 + for checksum in [&crc32, &crc32c, &crc64nvme, &sha1, &sha256] { + assert!(!checksum.is_empty()); + assert!( + base64::Engine::decode(&base64::engine::general_purpose::STANDARD, checksum).is_ok() + ); + } + + // All should be different (different algorithms) + assert_ne!(crc32, crc32c); + assert_ne!(crc32, crc64nvme); + assert_ne!(crc32, sha1); + assert_ne!(crc32, sha256); + assert_ne!(crc32c, crc64nvme); +} + +/// Test that different data produces different checksums +#[test] +fn test_different_data_different_checksums() { + let data1 = b"First test data"; + let data2 = b"Second test data"; + + // Test with each algorithm + for algorithm in [ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ] { + let checksum1 = compute_checksum(algorithm, data1); + let checksum2 = compute_checksum(algorithm, data2); + assert_ne!( + checksum1, checksum2, + "Algorithm {:?} produced same checksum for different data", + algorithm + ); + } +} + +/// Test that same data produces same checksums (deterministic) +#[test] +fn test_deterministic_checksums() { + let data = b"Deterministic test data"; + + for algorithm in [ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ] { + let checksum1 = compute_checksum(algorithm, data); + let checksum2 = compute_checksum(algorithm, data); + assert_eq!( + checksum1, checksum2, + "Algorithm {:?} is not deterministic", + algorithm + ); + } +} + +/// Test empty data checksums +#[test] +fn test_empty_data_checksums() { + let data = b""; + + for algorithm in [ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ] { + let checksum = compute_checksum(algorithm, data); + // Empty data should still produce a valid checksum + assert!(!checksum.is_empty()); + assert!( + base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok() + ); + } +} + +/// Test large data checksums +#[test] +fn test_large_data_checksums() { + // Test with 1MB of data + let data = vec![0x42u8; 1024 * 1024]; + + for algorithm in [ + ChecksumAlgorithm::CRC32, + ChecksumAlgorithm::CRC32C, + ChecksumAlgorithm::CRC64NVME, + ChecksumAlgorithm::SHA1, + ChecksumAlgorithm::SHA256, + ] { + let checksum = compute_checksum(algorithm, &data); + assert!(!checksum.is_empty()); + assert!( + base64::Engine::decode(&base64::engine::general_purpose::STANDARD, &checksum).is_ok() + ); + } +} + +/// Test ChecksumAlgorithm::as_str() +#[test] +fn test_checksum_algorithm_as_str() { + assert_eq!(ChecksumAlgorithm::CRC32.as_str(), "CRC32"); + assert_eq!(ChecksumAlgorithm::CRC32C.as_str(), "CRC32C"); + assert_eq!(ChecksumAlgorithm::CRC64NVME.as_str(), "CRC64NVME"); + assert_eq!(ChecksumAlgorithm::SHA1.as_str(), "SHA1"); + assert_eq!(ChecksumAlgorithm::SHA256.as_str(), "SHA256"); +} + +/// Test ChecksumAlgorithm::from_str() +#[test] +fn test_checksum_algorithm_from_str() { + use std::str::FromStr; + + // Test uppercase + assert_eq!( + ChecksumAlgorithm::from_str("CRC32").unwrap(), + ChecksumAlgorithm::CRC32 + ); + assert_eq!( + ChecksumAlgorithm::from_str("CRC32C").unwrap(), + ChecksumAlgorithm::CRC32C + ); + assert_eq!( + ChecksumAlgorithm::from_str("CRC64NVME").unwrap(), + ChecksumAlgorithm::CRC64NVME + ); + assert_eq!( + ChecksumAlgorithm::from_str("SHA1").unwrap(), + ChecksumAlgorithm::SHA1 + ); + assert_eq!( + ChecksumAlgorithm::from_str("SHA256").unwrap(), + ChecksumAlgorithm::SHA256 + ); + + // Test lowercase + assert_eq!( + ChecksumAlgorithm::from_str("crc32").unwrap(), + ChecksumAlgorithm::CRC32 + ); + assert_eq!( + ChecksumAlgorithm::from_str("crc32c").unwrap(), + ChecksumAlgorithm::CRC32C + ); + assert_eq!( + ChecksumAlgorithm::from_str("crc64nvme").unwrap(), + ChecksumAlgorithm::CRC64NVME + ); + assert_eq!( + ChecksumAlgorithm::from_str("sha1").unwrap(), + ChecksumAlgorithm::SHA1 + ); + assert_eq!( + ChecksumAlgorithm::from_str("sha256").unwrap(), + ChecksumAlgorithm::SHA256 + ); + + // Test mixed case + assert_eq!( + ChecksumAlgorithm::from_str("Crc32").unwrap(), + ChecksumAlgorithm::CRC32 + ); + assert_eq!( + ChecksumAlgorithm::from_str("Sha256").unwrap(), + ChecksumAlgorithm::SHA256 + ); + + // Test invalid + assert!(ChecksumAlgorithm::from_str("INVALID").is_err()); + assert!(ChecksumAlgorithm::from_str("MD5").is_err()); +}