From 330e0184a5b64d537a374b0990ea40fee0e2ce90 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Fri, 24 Apr 2026 04:02:04 +0900 Subject: [PATCH 01/26] docs: propose SQS-compatible adapter design Add a Milestone-1 design doc for an SQS-compatible HTTP adapter under docs/design/, following the existing adapter pattern used for Redis, DynamoDB, and S3. Covers data model (queue catalog, visibility-indexed message keys, FIFO dedup/group lock), routing (one Raft group per queue), leader-bound mutations with lease-read fast paths for receive scans and read-only APIs, long-poll notifier on the FSM commit stream, and a Jepsen plan for standard + FIFO semantics. Also add docs/design/README.md documenting the YYYY_MM_DD__ filename convention established by the docs/reorg-design-doc-layout refactor, so future design proposals can follow it without reverse- engineering the rule. --- ...6_04_24_proposed_sqs_compatible_adapter.md | 523 ++++++++++++++++++ docs/design/README.md | 71 +++ 2 files changed, 594 insertions(+) create mode 100644 docs/design/2026_04_24_proposed_sqs_compatible_adapter.md create mode 100644 docs/design/README.md diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md new file mode 100644 index 000000000..456f24689 --- /dev/null +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -0,0 +1,523 @@ +# SQS-Compatible Adapter Design for Elastickv + +Status: Proposed +Author: bootjp +Date: 2026-04-24 + +--- + +## 1. Background + +Elastickv exposes four protocol surfaces as of 2026-04-24: + +1. gRPC (`RawKV` / `TransactionalKV`) +2. Redis-compatible commands (`adapter/redis.go`) +3. DynamoDB-compatible HTTP APIs (`adapter/dynamodb.go`) +4. S3-compatible HTTP APIs (`adapter/s3.go`) + +There is no message-queue surface. This document proposes an HTTP adapter that exposes an Amazon SQS-compatible API while reusing the existing Raft, MVCC, HLC, shard-routing, leader-proxy, and SigV4 patterns that the DynamoDB and S3 adapters already established. + +The goal is SQS compatibility for standard AWS SDK/CLI workflows against a self-hosted Elastickv cluster, not AWS feature parity. The adapter should transparently support `aws sqs`, the AWS SDK for Go v2, the JVM SDK, and boto3 against the same endpoint. + +## 2. Goals and Non-goals + +### 2.1 Goals + +1. Add an `adapter/sqs.go` HTTP server that fits the existing adapter model. +2. Support both Standard and FIFO queues with their core semantics: + - Standard: at-least-once delivery, best-effort ordering. + - FIFO: strict order per `MessageGroupId`, exactly-once within a 5-minute deduplication window. +3. Expose visibility-timeout semantics without requiring a background sweeper. +4. Provide durable, Raft-replicated storage of queues and in-flight messages so that leader failover preserves every committed send, receive, and delete. +5. Support long polling (`WaitTimeSeconds` up to 20 seconds) without busy-waiting. +6. Reuse the DynamoDB/S3 adapter conventions for SigV4 auth, leader detection, leader proxying, and metrics. + +### 2.2 Non-goals + +1. Full AWS SQS feature parity. +2. IAM, STS, cross-account policy, or resource-based policies. Static SigV4 key pairs only. +3. Server-side KMS encryption (`SqsManagedSseEnabled`, `KmsMasterKeyId`). +4. Cross-region replication or SQS-managed dead-letter redrive API (`StartMessageMoveTask` / `ListMessageMoveTasks`). DLQ re-drive by `RedrivePolicy` on `ReceiveMessage` **is** in scope. +5. High-throughput FIFO (unlimited per-group throughput via partitioning). Milestone 1 provides single-partition FIFO semantics. +6. Exactly-once pub/sub fan-out (that is SNS, not SQS). + +## 3. Compatibility Scope + +The adapter should focus on the API surface that real clients exercise. Two wire protocols must be accepted because AWS SDKs still split between them: + +1. **JSON-1.0 protocol** (`X-Amz-Target: AmazonSQS.`, JSON body). Used by AWS SDK for Go v2, AWS SDK for JS v3, and `aws sqs` since CLI v2.15. +2. **Query protocol** (`application/x-www-form-urlencoded`, `Action=`, XML response). Used by boto3 < 1.34, JVM SDK v1, and older tooling. + +The adapter should dispatch by request content-type / `X-Amz-Target` header and emit matching JSON or XML responses. + +| API | Phase | Notes | +|---|---|---| +| `CreateQueue` | 1 | Standard + FIFO. Idempotent when attributes match. | +| `DeleteQueue` | 1 | Tombstones metadata; async reclaim. | +| `ListQueues` | 1 | Prefix filter, pagination token. | +| `GetQueueUrl` | 1 | | +| `GetQueueAttributes` | 1 | All core attributes. | +| `SetQueueAttributes` | 1 | VisibilityTimeout, MessageRetentionPeriod, DelaySeconds, ReceiveMessageWaitTimeSeconds, RedrivePolicy, MaximumMessageSize. | +| `PurgeQueue` | 1 | 60-second rate limit per AWS. | +| `SendMessage` | 1 | | +| `SendMessageBatch` | 1 | Up to 10 entries, 256 KiB total. | +| `ReceiveMessage` | 1 | Long polling, up to 10 messages. | +| `DeleteMessage` | 1 | | +| `DeleteMessageBatch` | 1 | | +| `ChangeMessageVisibility` | 1 | | +| `ChangeMessageVisibilityBatch` | 1 | | +| `TagQueue` / `UntagQueue` / `ListQueueTags` | 2 | | +| DLQ redrive on max receives | 1 | | +| `StartMessageMoveTask` and friends | Not planned | | +| Server-side KMS | Not planned | | + +Payload limits match AWS defaults: 256 KiB per message, 120,000 in-flight messages per standard queue, 20,000 in-flight per FIFO queue. These should be enforced at the adapter, not silently truncated. + +## 4. High-Level Architecture + +```mermaid +flowchart TB + subgraph Clients + CLI["aws sqs / SDKs"] + App["Application workers"] + end + + subgraph Node["Elastickv Node"] + SQS["SQS Server (adapter/sqs.go)"] + Auth["SigV4 Verifier"] + Proxy["Leader HTTP Proxy"] + Cat["Queue Catalog"] + Send["Send Pipeline"] + Recv["Receive Pipeline"] + Vis["Visibility Manager"] + Poll["Long-Poll Notifier"] + + Coord["Coordinator / ShardStore"] + Raft["Raft Group(s)"] + MVCC["MVCC Store"] + end + + CLI --> SQS + App --> SQS + SQS --> Auth + SQS --> Proxy + SQS --> Cat + SQS --> Send + SQS --> Recv + Recv --> Vis + Recv --> Poll + Send --> Poll + Cat --> Coord + Send --> Coord + Recv --> Coord + Vis --> Coord + Coord --> Raft + Raft --> MVCC +``` + +The adapter sits beside Redis, DynamoDB, and S3 and reuses the same `store.MVCCStore` + `kv.Coordinator` data plane. Every state-mutating SQS operation (including `ReceiveMessage`, because receive flips visibility state) runs through the Raft leader. Read-only operations (`GetQueueAttributes`, `ListQueues`, `GetQueueUrl`, `ListQueueTags`) are served from any node using the `LeaseRead` fast path introduced in `docs/design/2026_04_20_implemented_lease_read.md`; see §7.8. + +### 4.1 Leader proxy reuse + +The DynamoDB adapter already has the right proxy shape (`adapter/dynamodb.go:proxyToLeader`): small request bodies, JSON envelope, re-emit response. SQS message payloads cap at 256 KiB, so the existing proxy pattern is sufficient and the streaming HTTP proxy that S3 needed (`docs/s3_compatible_adapter_design.md §4.1`) is **not** required here. The SQS adapter should fork the DynamoDB proxy helper rather than reuse the S3 streaming proxy. + +## 5. Data Model + +### 5.1 Queue catalog + +Queue metadata lives in a reserved control-plane keyspace, stored in the default Raft group (same pattern as the S3 bucket catalog and DynamoDB table catalog). + +Suggested keys: + +1. `!sqs|queue|meta|` → queue metadata record +2. `!sqs|queue|gen|` → monotonically increasing `generation` counter (uint64). Incremented on `DeleteQueue` so that stale message keys from the previous incarnation cannot be resurrected by recreating a queue with the same name. +3. `!sqs|queue|tombstone|` → async reclaim marker + +Suggested queue metadata fields: + +1. `queue_name` +2. `queue_url` +3. `generation` +4. `created_at_hlc` +5. `is_fifo` (bool; derived from `.fifo` suffix on `CreateQueue`) +6. `content_based_dedup` +7. `visibility_timeout_seconds` (default 30) +8. `message_retention_seconds` (default 4 days, max 14 days) +9. `delay_seconds` (default 0) +10. `receive_message_wait_seconds` (default 0, max 20) +11. `maximum_message_size` (default 262144) +12. `redrive_policy` (DLQ ARN + `maxReceiveCount`) +13. `redrive_allow_policy` +14. `tags` + +### 5.2 Message keyspace + +Message state is split across three key families so that the hot path (`ReceiveMessage`) is a single bounded prefix scan. + +1. `!sqs|msg|data|` → full message record (body, attributes, send metadata, receive count, current visibility token, current receipt handle). +2. `!sqs|msg|vis|` → visibility index entry. Value is the `message_id`. Ordered by `visible_at`, then by insertion sequence for stable FIFO. +3. `!sqs|msg|dedup|` → FIFO deduplication record, with expiry stored in the value. Populated only for FIFO or `SendMessageBatch` idempotent entries. +4. `!sqs|msg|group|` → FIFO group in-flight lock. Present only when the group currently has an unacknowledged delivered message. Value references the locking `message_id` and `visible_at`. + +Key encoding reuses the DynamoDB adapter's byte-ordered segment encoder (`encodeDynamoKeySegment` in `adapter/dynamodb.go`). Strings are self-delimited with the `0x00 0xFF` / `0x00 0x01` escape-and-terminator scheme; integers are fixed-width big-endian `uint64`. This matches the S3 adapter key layout (`docs/s3_compatible_adapter_design.md §5.2`) so the three adapters agree on ordering and parsing. + +### 5.3 Message record + +Body lives in the `!sqs|msg|data|...` record. The record is written once at `SendMessage` time and updated by `ReceiveMessage` and `ChangeMessageVisibility` via OCC transactions. + +Fields: + +1. `message_id` (server-assigned UUIDv4) +2. `body` (raw bytes) or `body_ref` for larger payloads (reserved; Milestone 1 inlines) +3. `md5_of_body` +4. `attributes` (AWS-defined: `SenderId`, `SentTimestamp`, `ApproximateReceiveCount`, `ApproximateFirstReceiveTimestamp`, etc.) +5. `message_attributes` (user-defined) +6. `message_system_attributes` +7. `send_timestamp_hlc` +8. `available_at_hlc` (send time + `DelaySeconds`) +9. `visible_at_hlc` (current visibility deadline; equals `available_at_hlc` while the message is available) +10. `receive_count` +11. `first_receive_timestamp_hlc` +12. `current_receipt_token` (opaque 16-byte random token; rotated on each receive) +13. `message_group_id` (FIFO only) +14. `message_deduplication_id` (FIFO only) +15. `sequence_number` (FIFO only, monotonic per queue) +16. `source_send_id` (batch sender id for deduped sends) + +### 5.4 Visibility index + +`!sqs|msg|vis|...` is the only key read by `ReceiveMessage`. The adapter must keep it in sync with the message record under every state change: + +1. **Send:** insert `(visible_at = available_at_hlc, seq)` → `message_id`. +2. **Receive (n messages):** for each delivered message, delete the old visibility entry and insert a new `(visible_at = now + VisibilityTimeout, seq)` → `message_id`. The transaction also bumps `visible_at_hlc` and `current_receipt_token` on the data record. +3. **ChangeMessageVisibility:** same delete+insert swap as receive, validated against the supplied receipt handle. +4. **Delete:** remove both the data record and the visibility index entry. + +Because `visible_at` leads the key, the next candidate message is always the first key `>= now` in the prefix. Invisible messages (`visible_at > now`) are naturally skipped without a sweeper. + +### 5.5 Receipt handles + +Receipt handles must be opaque to the client, cheap to validate, and impossible to forge: + +``` +receipt_handle = base64url( + queue_gen_u64 | message_id_16 | receipt_token_16 +) +``` + +`receipt_token` is the `current_receipt_token` field on the message record, rotated on every receive. A valid `DeleteMessage` or `ChangeMessageVisibility` must find the record, confirm the receipt token matches the current one, and — for `ChangeMessageVisibility` — confirm the message is still in flight (`visible_at_hlc > now`). Token mismatch returns `InvalidReceiptHandle` per AWS. No HMAC is needed because the token is a per-delivery shared secret persisted in the Raft log. + +### 5.6 FIFO extensions + +FIFO queues add three invariants on top of the standard schema: + +1. **Group lock**: `!sqs|msg|group|` is held by at most one message at a time. `ReceiveMessage` skips the entire group while the lock is held. The lock is released when the locking message is deleted, expires its visibility (and the next receive within the group rotates ownership), or the message retention expires. +2. **Deduplication**: `!sqs|msg|dedup|` blocks a duplicate `SendMessage` for 5 minutes. For `ContentBasedDeduplication = true` the dedup id is `SHA-256(body)`. +3. **Per-queue sequence number**: `!sqs|queue|seq|` → `uint64`, bumped by every FIFO send. Written into the message record for strict ordering. + +## 6. Routing Model + +### 6.1 Logical route key + +All message keys for a given queue must hash to the same Raft group, so that `ReceiveMessage` only scans one shard and FIFO ordering/locking is single-writer. + +Logical route: + +``` +!sqsroute| +``` + +Every internal prefix must normalize to this route: + +1. `!sqs|msg|data|...` +2. `!sqs|msg|vis|...` +3. `!sqs|msg|dedup|...` +4. `!sqs|msg|group|...` + +### 6.2 Required core changes + +1. `routeKey(...)` (same function touched by the S3 design, see `docs/s3_compatible_adapter_design.md §6.2`) gains an `!sqs|...` branch that trims to the queue route prefix. +2. `kv/ShardStore.routesForScan` gains an SQS-aware internal scan mapping so `ReceiveMessage`'s visibility-index scan lands on the correct shard. +3. The queue catalog (`!sqs|queue|meta|...`, `!sqs|queue|gen|...`) routes to a fixed control-plane group like the DynamoDB table catalog. + +Queue-per-shard routing is the simplest model. A single hot queue that outgrows a shard is a known operational limit in Milestone 1; splitting a queue across shards while preserving FIFO is deferred. + +## 7. Request Flows + +### 7.1 `SendMessage` + +```mermaid +sequenceDiagram + participant C as Client + participant S as SQS Server + participant L as Queue Leader + participant KV as MVCC / Raft + + C->>S: POST AmazonSQS.SendMessage + S->>S: SigV4 verify, parse, size check + S->>S: Resolve queue meta + generation + alt follower + S->>L: Proxy JSON request + L->>KV: OCC Txn: dedup check + data + vis index + L-->>C: 200 (MessageId, MD5, SequenceNumber?) + else local leader + S->>KV: OCC Txn + S-->>C: 200 + end + L->>L: notify long-poll waiters for queue +``` + +Steps: + +1. Validate message size (`MaximumMessageSize` from queue attrs, capped at 262144). +2. Compute `md5_of_body` and `md5_of_message_attributes`. +3. Resolve queue generation; if queue is tombstoned return `QueueDoesNotExist`. +4. Compute `available_at_hlc = now + max(queue.delay, per-message DelaySeconds)`. +5. In one OCC transaction: + - For FIFO: check and insert `!sqs|msg|dedup|...`. On hit, return the existing message's ID (AWS semantics). + - For FIFO: increment `!sqs|queue|seq|...` and write it into the record. + - Write `!sqs|msg|data|...` and `!sqs|msg|vis|...`. +6. On commit, wake any long-poll waiters holding the queue's `sync.Cond` (see §7.3). + +### 7.2 `ReceiveMessage` + +`ReceiveMessage` is a **write** operation: it mutates `visible_at_hlc`, `receive_count`, and `current_receipt_token` on each delivered message and swaps the visibility-index entry. It must run on the leader. + +```mermaid +sequenceDiagram + participant C as Client + participant S as SQS Server + participant KV as MVCC / Raft + participant N as Long-Poll Notifier + + C->>S: ReceiveMessage(MaxNumberOfMessages, WaitTimeSeconds, VisibilityTimeout) + loop until batch >= Max or deadline + S->>KV: Snapshot scan !sqs|msg|vis|| where visible_at <= now + S->>KV: OCC Txn: for each msg {delete old vis, insert new vis, update data record} + alt delivered >= Max or elapsed >= WaitTimeSeconds + S-->>C: 200 with batch + else empty + S->>N: Wait(queue, remaining) + N-->>S: wake on SendMessage commit or timeout + end + end +``` + +Details: + +1. Fence the snapshot scan with `coordinator.LeaseReadForKey(ctx, queueRoute)`. While the lease is valid (≤ `electionTimeout − leaseSafetyMargin`, currently 700 ms per `docs/design/2026_04_20_implemented_lease_read.md §3.2`) this returns immediately with no ReadIndex round-trip. Under sustained receive load this amortizes quorum confirmation across every receive in the lease window instead of paying one ReadIndex per call. +2. Snapshot-scan the visibility index for the first `k` keys with `visible_at ≤ now`, up to `MaxNumberOfMessages` (≤ 10). Use a bounded scan limit (e.g., `2 * k`) to tolerate races. +3. Filter FIFO candidates: skip any message whose group lock is held by another message; acquire the group lock on delivery. +4. Run one OCC transaction per batch: for each candidate, verify the visibility entry still matches, rotate receipt token, set `visible_at_hlc = now + effective_timeout`, insert the new visibility entry, bump `receive_count` and `approximate_first_receive_timestamp`, acquire FIFO group lock if applicable. A successful `Propose` also refreshes the lease (per `§3.3` of the lease-read doc), so a busy queue keeps its fast path warm without any extra work. +5. If the batch is empty and `WaitTimeSeconds > 0`, register a waiter on the queue's in-process condition variable and re-run step 1 on wake or timeout. Do not poll in a tight loop. On re-scan, the `LeaseRead` fence from step 1 stays local for the whole long-poll window — empty long polls cost O(1) quorum operations, not O(ticks). +6. If `receive_count > redrive_policy.maxReceiveCount`, move the message to the DLQ (see §7.6) atomically within the receive transaction and do not return it to the caller. + +Lease invalidation handling: if the leader loses quorum during a long poll, `RegisterLeaderLossCallback` fires, the lease is cleared, and the next `LeaseReadForKey` falls back to `LinearizableRead`. That call fails if no quorum is available, and the receive returns a retryable error rather than quietly serving stale data. + +### 7.3 Long-poll notifier + +Long polling is local to the leader: every node keeps a `map[queueRoute]*cond` and wakes waiters when a `SendMessage` commits on that queue. The notifier lives in the leader process; on failover, waiters on the old leader get `AbortedError` from the coordinator and clients retry against the new leader. AWS SDKs already retry receive on connection errors, so this is acceptable. + +The notifier must subscribe to the local FSM commit stream, not the RPC path, so that a `SendMessage` that committed via another adapter node still wakes local waiters once the Raft log reaches this leader. + +### 7.4 `DeleteMessage` + +1. Parse receipt handle → `(queue_gen, message_id, receipt_token)`. +2. OCC transaction: + - Load `!sqs|msg|data|`. Missing → `ReceiptHandleIsInvalid`. + - Compare `current_receipt_token`. Mismatch → `InvalidReceiptHandle`. + - Delete the data record, the current visibility index entry, and (FIFO) release the group lock. +3. `DeleteMessageBatch` aggregates entries into one multi-row OCC transaction and returns per-entry success/failure. + +### 7.5 `ChangeMessageVisibility` + +1. Validate receipt handle and token as in `DeleteMessage`. +2. Reject if `VisibilityTimeout > 12 hours` or negative. +3. OCC: delete old visibility entry, update `visible_at_hlc`, insert new visibility entry. Receipt token is **not** rotated (AWS does not rotate it for `ChangeMessageVisibility`). + +### 7.6 DLQ redrive + +If the target queue has a `RedrivePolicy`, the `ReceiveMessage` transaction that would push `receive_count` past `maxReceiveCount` should instead: + +1. Read the target DLQ metadata by ARN (must resolve inside the same cluster; cross-cluster DLQ is out of scope). +2. In the same OCC transaction: delete the source data/vis keys, write the message to the DLQ keyspace with a fresh `message_id` and reset `receive_count`, and add `DeadLetterQueueSourceArn` to attributes. +3. The receive response omits the moved message. + +Doing the move inside the receive transaction avoids a two-phase dance and keeps FIFO group locks consistent. + +### 7.7 `PurgeQueue` + +`PurgeQueue` bumps the queue generation counter and writes a tombstone for the old generation. All old message keys become unreachable via routing and are reclaimed asynchronously, mirroring the S3 bucket generation pattern. + +### 7.8 Read-only operations and the follower-read fast path + +Read-only APIs never mutate queue or message state, so they do not need the leader proxy. They use `coordinator.LeaseRead` / `LeaseReadForKey` (`docs/design/2026_04_20_implemented_lease_read.md §3.5`) directly on the node that receives the request: + +| API | Fenced key | Read scope | +|---|---|---| +| `GetQueueUrl` | queue catalog route | `!sqs|queue|meta|` | +| `GetQueueAttributes` | queue catalog route | `!sqs|queue|meta|...` and derived counters | +| `ListQueues` | queue catalog route | prefix scan `!sqs|queue|meta|` | +| `ListQueueTags` | queue catalog route | `!sqs|queue|meta|.tags` | + +The fast path: + +1. Adapter calls `coordinator.LeaseReadForKey(ctx, route)`. +2. If the lease is valid (fast path, local wall-clock compare), the adapter serves the response from the local FSM snapshot at the returned applied index. No Raft traffic, no leader forwarding. +3. If the lease is expired or this node has never held one, `LeaseRead` falls back to `LinearizableRead`, which issues a ReadIndex through etcd/raft. On a follower this forwards to the leader and waits for the local apply to catch up to the returned read index — still one round-trip, but only on the cold path. +4. On any lease or ReadIndex error, return a retryable AWS error code (`ServiceUnavailable`) so SDKs back off. + +This gives three operational properties: + +1. **Followers serve read-only SQS traffic.** Under a healthy cluster, every node can answer `GetQueueAttributes` / `ListQueues` from local state for the entire lease window after it warms up, so read load spreads across the fleet instead of concentrating on the leader. +2. **Receive and read-only share a lease.** When a follower is also running `LeaseReadForKey` from mutating flows proxied to the leader, or when it piggy-backs on replicated applies, the lease stays warm and sustained read load pays essentially zero quorum cost. +3. **Safety bound is explicit.** Stale-read exposure is capped at `electionTimeout − leaseSafetyMargin` (currently 700 ms). This matches the trade-off the Redis and DynamoDB adapters already accept; SQS does not introduce a new class of staleness. + +Mutating operations (`CreateQueue`, `DeleteQueue`, `SetQueueAttributes`, `SendMessage`, `ReceiveMessage`, `DeleteMessage`, `ChangeMessageVisibility`, `PurgeQueue`, `TagQueue`, `UntagQueue`) are unaffected by this fast path: they continue to proxy to the leader because they need a Raft `Propose`, not just a read fence. + +## 8. Consistency Model + +### 8.1 What the adapter guarantees + +1. A successful `SendMessage` is visible to a subsequent `ReceiveMessage` at or after the send commit timestamp. +2. A successful `DeleteMessage` is not re-delivered. +3. A successful `ChangeMessageVisibility` is observed by every subsequent `ReceiveMessage`. +4. FIFO queues preserve strict per-group ordering across any leader failover, because the group lock and sequence counter are both MVCC-committed. +5. Deduplication within the 5-minute FIFO window is exact, backed by the OCC dedup-key insert. + +### 8.2 What the adapter does not promise + +1. Standard queues may deliver a message more than once if a client crashes before `DeleteMessage` — this matches AWS semantics. +2. Ordering across FIFO groups is undefined. +3. `ApproximateNumberOfMessages` is an estimate; Milestone 1 returns the exact count from a snapshot scan, which AWS also does approximately. +4. Cross-queue transactions are not supported (SQS does not offer this either). + +### 8.3 Jepsen coverage + +Given Elastickv's existing Jepsen harness (`jepsen/`) and the prior G1c/G0 work captured in memory, SQS must ship with a dedicated Jepsen workload under `jepsen/sqs/`: + +1. **Standard queue**: check at-least-once delivery and no loss under `partition-majorities-ring`, `kill`, and `clock-skew` faults. +2. **FIFO queue**: per-`MessageGroupId` strict-ordering checker; duplicate delivery inside the 5-minute dedup window is a bug. +3. **Visibility**: assert that a message whose visibility has not expired is never delivered to a second concurrent consumer. + +## 9. Authentication + +SigV4 verification reuses the shared verifier added for the S3 adapter. Credentials come from the same static `--s3CredentialsFile` or a new `--sqsCredentialsFile` flag (both accepted). Region comes from `--sqsRegion` (default `us-east-1`). + +Out of scope for Milestone 1: IAM policies, resource policies, `sqs:SendMessage` per-action allow/deny, temporary credentials via STS. + +Unauthenticated requests are rejected with `MissingAuthenticationToken`. Bad signatures return `SignatureDoesNotMatch`. The SigV4 clock skew window follows AWS (5 minutes). + +## 10. Operational and Configuration Changes + +New server flags (parallel to `--s3*` and DynamoDB flags): + +1. `--sqsAddress` — listener for the SQS HTTP server. +2. `--raftSqsMap` — `nodeID=host:port` pairs used by the leader proxy. +3. `--sqsRegion` +4. `--sqsCredentialsFile` +5. `--sqsEndpointBase` — base URL for emitted `QueueUrl` values (defaults to request `Host`). +6. `--sqsMaxMessageBytes` — cluster-wide ceiling; defaults to 262144. + +New metrics (prefixed `sqs_`): + +1. `sqs_request_total{op, status}` +2. `sqs_request_duration_seconds{op}` +3. `sqs_request_bytes{op}` +4. `sqs_messages_sent_total{queue, is_fifo}` +5. `sqs_messages_received_total{queue, is_fifo}` +6. `sqs_messages_deleted_total{queue}` +7. `sqs_messages_in_flight{queue}` +8. `sqs_longpoll_waiters{queue}` +9. `sqs_longpoll_wake_latency_seconds` +10. `sqs_dlq_redrive_total{queue}` +11. `sqs_proxy_to_leader_total{op}` + +Structured log fields match the rest of the project: `queue`, `message_id`, `receipt_token_prefix`, `group_id`, `dedup_id`, `commit_ts`, `leader`. + +## 11. Failure Handling and Cleanup + +1. **Retention expiry**: a background reaper on the leader scans the visibility index for entries older than `send_timestamp + message_retention_seconds` and deletes them in bounded batches. Because visible_at ≤ send_time + retention implies the message is past retention in the common case, the reaper can mostly piggy-back on `ReceiveMessage` scans. +2. **Old-generation reclaim**: `DeleteQueue` and `PurgeQueue` write `!sqs|queue|tombstone|...` markers. A background task drops remaining keys for the old generation. +3. **Leader failover during long poll**: waiters drain, clients reconnect. Because every state change was Raft-committed before being acknowledged, no send or delete is lost. +4. **Clock skew**: visibility timeouts use the cluster HLC, not wall clock. Sends across nodes remain monotonic because the HLC leader renews the physical ceiling every second (same mechanism the Redis/DynamoDB adapters already depend on). + +## 12. Testing Plan + +### 12.1 Unit tests (`adapter/sqs*_test.go`) + +1. Key encoding round-trips (queue, message, dedup, group). +2. Receipt handle format and validation. +3. SigV4 canonical request for both query and JSON protocols. +4. MD5-of-body and MD5-of-message-attributes vs AWS fixtures. +5. Visibility index update math (delete old + insert new) under concurrent receives. +6. FIFO dedup window boundary (4:59 vs 5:01). +7. Batch entry id uniqueness and partial failure encoding. + +### 12.2 Integration tests + +1. Single-node `CreateQueue` / `SendMessage` / `ReceiveMessage` / `DeleteMessage`. +2. Follower-ingress write proxying to leader (Standard and FIFO). +3. Follower `GetQueueAttributes` / `ListQueues` served via `LeaseRead` with no leader RTT; same call during a leader-loss window falls back and eventually errors out rather than returning stale data. +4. Long-poll wake on cross-node send. +5. Visibility timeout expiry restores delivery. +6. `ChangeMessageVisibility` extends and shortens correctly. +7. FIFO strict per-group order with 16 concurrent producers. +8. DLQ redrive at `maxReceiveCount`. +9. Leader failover mid-receive: no duplicate delivery to a second consumer within visibility timeout. +10. Sustained-load `ReceiveMessage` hits the lease fast path — assert the underlying `LinearizableRead` count grows O(lease-expiries), not O(receives). + +### 12.3 Compatibility tests + +1. `aws sqs` CLI smoke tests (both `create-queue .fifo` and Standard). +2. AWS SDK for Go v2 — `NewFromConfig` with a custom endpoint resolver. +3. boto3 with query protocol. +4. Celery with SQS broker (canonical real-world harness). + +### 12.4 Jepsen workload + +Under `jepsen/sqs/`, add `standard` and `fifo` test variants mirroring the existing Redis/DynamoDB harnesses, with `partition`, `kill`, and `clock-skew` nemeses. Treat duplicate delivery inside the visibility window or within the FIFO dedup window as a failure. + +## 13. Rollout Plan + +### Phase 1 + +1. Queue catalog, `CreateQueue` / `DeleteQueue` / `ListQueues` / `GetQueueUrl`. +2. Standard queue send/receive/delete/change-visibility with long polling. +3. SigV4 + JSON protocol; query protocol added opportunistically in the same phase. +4. Leader proxy. +5. Metrics and structured logs. +6. Unit + integration test coverage. + +### Phase 2 + +1. FIFO queues (group lock, dedup, sequence number). +2. Batch APIs (`SendMessageBatch`, `DeleteMessageBatch`, `ChangeMessageVisibilityBatch`). +3. DLQ redrive. +4. Jepsen standard + FIFO workloads. +5. Tag APIs. + +### Phase 3 + +1. Query-protocol full XML fidelity for older SDKs. +2. `ApproximateNumberOfMessagesDelayed` / `NotVisible` accuracy. +3. Per-queue throttling and fairness across tenants. +4. Split-queue FIFO for very hot queues. + +## 14. Summary + +The design choices are: + +1. **Visibility index keyed by `visible_at`** removes the need for a background visibility sweeper — `ReceiveMessage` naturally skips invisible messages. +2. **Per-receive rotating receipt token persisted in Raft** gives forgery-resistant `DeleteMessage` / `ChangeMessageVisibility` without HMAC or external signing. +3. **One Raft group per queue** keeps FIFO ordering and group locking single-writer without cross-shard coordination. +4. **All mutating operations (including ReceiveMessage) go through the leader** keeps the consistency story identical to the DynamoDB adapter. +5. **Reuse of the DynamoDB adapter's proxy and key-encoding helpers** keeps the adapter small; the streaming S3 proxy path is not needed because SQS payloads are bounded. +6. **Lease-read fast path for read-only APIs and the `ReceiveMessage` scan fence** amortizes ReadIndex across bursts and lets followers serve `GetQueueAttributes` / `ListQueues` without a leader round-trip, reusing the mechanism already merged in `docs/design/2026_04_20_implemented_lease_read.md`. + +The two repository-level prerequisites are: + +1. SQS-aware internal-key routing and scan routing in `kv/`, matching the changes that `docs/s3_compatible_adapter_design.md §6.2` already proposes for S3. +2. A leader-local long-poll notifier subscribed to the FSM commit stream. + +With those in place, SQS compatibility can be added without changing the underlying storage, consensus, or HLC design. diff --git a/docs/design/README.md b/docs/design/README.md new file mode 100644 index 000000000..67345d263 --- /dev/null +++ b/docs/design/README.md @@ -0,0 +1,71 @@ +# Design Documents + +Forward-looking design proposals and implementation plans for elastickv live in this directory. Operational runbooks, reference material, and running TODO lists stay under `docs/` itself. + +## Filename convention + +``` +YYYY_MM_DD__.md +``` + +- `YYYY_MM_DD` — **propose date**: the date the proposal was first written. Use the author date of the first commit that added the file: + + ```sh + git log --follow --diff-filter=A --format='%aI' -- docs/design/ | tail -1 + ``` + + Do not change this date when the doc is later revised or its status is promoted. It is a historical anchor, not a "last updated" marker. + +- `` — current **implementation state**, assigned conservatively from code evidence on `main`: + + | Value | Meaning | + |---|---| + | `proposed` | No matching code on `main`, or the doc itself declares itself a proposal. | + | `partial` | Some components exist but the full scope described in the doc is not yet merged. | + | `implemented` | Concrete Go code exists on `main` that matches the design's central subsystem (the functions, types, or files named in the doc). | + + Promote the status by renaming the file in a dedicated commit (e.g. `docs: promote from proposed to implemented`). Do not introduce new statuses without updating this README. + +- `` — short slug, lowercase with underscores, **without** the `_design` suffix (`s3_compatible_adapter`, not `s3_compatible_adapter_design`). + +### Examples + +``` +2026_04_20_implemented_lease_read.md +2026_04_18_proposed_raft_grpc_streaming_transport.md +2026_02_18_partial_hotspot_shard_split.md +2026_04_24_proposed_sqs_compatible_adapter.md +``` + +## Document header + +New design docs should open with the following metadata block under the title: + +```markdown +# + +Status: <Proposed | Partial | Implemented> +Author: <github handle> +Date: YYYY-MM-DD +``` + +The header `Status` mirrors the filename `<status>` token. The header `Date` matches the filename propose date. + +## What belongs here vs. `docs/` + +Kept under `docs/design/`: + +- Proposals for new subsystems, adapters, or protocol surfaces. +- Concrete implementation plans (milestone PR plans, migration plans). +- Implemented proposals preserved for historical reference. + +Kept under `docs/` (not design proposals): + +- `architecture_overview.md` — reference. +- `docker_multinode_manual_run.md`, `etcd_raft_migration_operations.md`, `redis-proxy-deployment.md` — operations. +- `redis_hotpath_dashboard.md` — observability runbook. +- `review_todo.md` — running TODO list. + +## Moving or renaming + +Always use `git mv` so rename history and `git log --follow` continue to trace prior revisions. When promoting status, keep the propose date and the slug untouched — only the status token changes. From 4f26c07c6cd4ed8eb0c4e5b0fa5ddf6288e3dfdf Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 04:56:18 +0900 Subject: [PATCH 02/26] docs: update stale S3 design doc paths after reorg MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Codex review on #606 flagged that the SQS design cited docs/s3_compatible_adapter_design.md, which was relocated to docs/design/2026_03_22_implemented_s3_compatible_adapter.md by the docs/reorg-design-doc-layout change. Update all four references so the cited §4.1, §5.2, and §6.2 resolve. --- docs/design/2026_04_24_proposed_sqs_compatible_adapter.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 456f24689..4312e993e 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -119,7 +119,7 @@ The adapter sits beside Redis, DynamoDB, and S3 and reuses the same `store.MVCCS ### 4.1 Leader proxy reuse -The DynamoDB adapter already has the right proxy shape (`adapter/dynamodb.go:proxyToLeader`): small request bodies, JSON envelope, re-emit response. SQS message payloads cap at 256 KiB, so the existing proxy pattern is sufficient and the streaming HTTP proxy that S3 needed (`docs/s3_compatible_adapter_design.md §4.1`) is **not** required here. The SQS adapter should fork the DynamoDB proxy helper rather than reuse the S3 streaming proxy. +The DynamoDB adapter already has the right proxy shape (`adapter/dynamodb.go:proxyToLeader`): small request bodies, JSON envelope, re-emit response. SQS message payloads cap at 256 KiB, so the existing proxy pattern is sufficient and the streaming HTTP proxy that S3 needed (`docs/design/2026_03_22_implemented_s3_compatible_adapter.md §4.1`) is **not** required here. The SQS adapter should fork the DynamoDB proxy helper rather than reuse the S3 streaming proxy. ## 5. Data Model @@ -159,7 +159,7 @@ Message state is split across three key families so that the hot path (`ReceiveM 3. `!sqs|msg|dedup|<queue-esc><gen-u64><dedup-id-esc>` → FIFO deduplication record, with expiry stored in the value. Populated only for FIFO or `SendMessageBatch` idempotent entries. 4. `!sqs|msg|group|<queue-esc><gen-u64><group-esc>` → FIFO group in-flight lock. Present only when the group currently has an unacknowledged delivered message. Value references the locking `message_id` and `visible_at`. -Key encoding reuses the DynamoDB adapter's byte-ordered segment encoder (`encodeDynamoKeySegment` in `adapter/dynamodb.go`). Strings are self-delimited with the `0x00 0xFF` / `0x00 0x01` escape-and-terminator scheme; integers are fixed-width big-endian `uint64`. This matches the S3 adapter key layout (`docs/s3_compatible_adapter_design.md §5.2`) so the three adapters agree on ordering and parsing. +Key encoding reuses the DynamoDB adapter's byte-ordered segment encoder (`encodeDynamoKeySegment` in `adapter/dynamodb.go`). Strings are self-delimited with the `0x00 0xFF` / `0x00 0x01` escape-and-terminator scheme; integers are fixed-width big-endian `uint64`. This matches the S3 adapter key layout (`docs/design/2026_03_22_implemented_s3_compatible_adapter.md §5.2`) so the three adapters agree on ordering and parsing. ### 5.3 Message record @@ -236,7 +236,7 @@ Every internal prefix must normalize to this route: ### 6.2 Required core changes -1. `routeKey(...)` (same function touched by the S3 design, see `docs/s3_compatible_adapter_design.md §6.2`) gains an `!sqs|...` branch that trims to the queue route prefix. +1. `routeKey(...)` (same function touched by the S3 design, see `docs/design/2026_03_22_implemented_s3_compatible_adapter.md §6.2`) gains an `!sqs|...` branch that trims to the queue route prefix. 2. `kv/ShardStore.routesForScan` gains an SQS-aware internal scan mapping so `ReceiveMessage`'s visibility-index scan lands on the correct shard. 3. The queue catalog (`!sqs|queue|meta|...`, `!sqs|queue|gen|...`) routes to a fixed control-plane group like the DynamoDB table catalog. @@ -517,7 +517,7 @@ The design choices are: The two repository-level prerequisites are: -1. SQS-aware internal-key routing and scan routing in `kv/`, matching the changes that `docs/s3_compatible_adapter_design.md §6.2` already proposes for S3. +1. SQS-aware internal-key routing and scan routing in `kv/`, matching the changes that `docs/design/2026_03_22_implemented_s3_compatible_adapter.md §6.2` already proposes for S3. 2. A leader-local long-poll notifier subscribed to the FSM commit stream. With those in place, SQS compatibility can be added without changing the underlying storage, consensus, or HLC design. From c6158388af3c681ef24c174e66e387c2882219ed Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 16:10:06 +0900 Subject: [PATCH 03/26] docs(sqs): add operator console UI design section MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add §13 "Console UI" to the SQS-compatible adapter design and thread the companion references into §2.1 goals, §10 operational flags, §14 rollout phases, and §15 summary. What the section covers: - Goals / non-goals: read-only browsing from any node, leader-aware writes, zero JS build toolchain, operator-token auth distinct from SigV4. Explicitly out of scope: RBAC, graphical metrics, edit-in- place. - Serving model: separate --consoleAddress listener (like --metricsAddress / --pprofAddress) so operators can firewall it independently; --consoleToken bearer auth enforced by monitoring.AddressRequiresToken; --consoleReadOnly gate for the mutating path. - Architecture: adapter/console.go + go:embed static bundle under adapter/console_ui/; reuses SQSServer methods for catalog and message state; writes go through the shared proxyHTTPRequestToLeader helper; reads use LeaseReadThrough. - REST API surface: non-AWS JSON with plain status codes; admin peek endpoint that scans the visibility index non-destructively (does not rotate receipt tokens or bump visibility), so the console never competes with workers. - Front-end: single-page HTML + one hand-written JS file, no framework, readable over a slow SSH tunnel. Destructive actions (Purge / Delete) require the operator to type the queue name. - Auth, failure, observability, and testing plans, plus a clear Milestone-1 out-of-scope list (SSO, per-queue ACLs, exports). Rollout phases now include the console: Phase 1 ships read-only views, Phase 2 unlocks mutating actions, Phase 3 adds UI polish (in-flight countdown, dark mode). --- ...6_04_24_proposed_sqs_compatible_adapter.md | 158 +++++++++++++++++- 1 file changed, 156 insertions(+), 2 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 4312e993e..87b3df89c 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -31,6 +31,7 @@ The goal is SQS compatibility for standard AWS SDK/CLI workflows against a self- 4. Provide durable, Raft-replicated storage of queues and in-flight messages so that leader failover preserves every committed send, receive, and delete. 5. Support long polling (`WaitTimeSeconds` up to 20 seconds) without busy-waiting. 6. Reuse the DynamoDB/S3 adapter conventions for SigV4 auth, leader detection, leader proxying, and metrics. +7. Ship an operator-facing web console (§13) for browsing queues, peeking messages, and running administrative actions, on its own listener with token-based auth distinct from SigV4. ### 2.2 Non-goals @@ -418,6 +419,9 @@ New server flags (parallel to `--s3*` and DynamoDB flags): 4. `--sqsCredentialsFile` 5. `--sqsEndpointBase` — base URL for emitted `QueueUrl` values (defaults to request `Host`). 6. `--sqsMaxMessageBytes` — cluster-wide ceiling; defaults to 262144. +7. `--consoleAddress` — listener for the operator web console; empty disables. See §13. +8. `--consoleToken` — bearer token required on the console listener when it is not loopback-only. +9. `--consoleReadOnly` — when true, the console listener refuses mutating actions (Send / Purge / Delete / Create). New metrics (prefixed `sqs_`): @@ -478,7 +482,153 @@ Structured log fields match the rest of the project: `queue`, `message_id`, `rec Under `jepsen/sqs/`, add `standard` and `fifo` test variants mirroring the existing Redis/DynamoDB harnesses, with `partition`, `kill`, and `clock-skew` nemeses. Treat duplicate delivery inside the visibility window or within the FIFO dedup window as a failure. -## 13. Rollout Plan +## 13. Console UI + +A browser-based management console is bundled with the adapter so operators can inspect queues, send test messages, and run administrative actions (purge, delete) without reaching for a CLI. It is a first-class feature of the SQS surface, not a follow-up tool. + +### 13.1 Goals and non-goals + +Goals: + +1. Read-only queue browsing from any node (uses the same `LeaseRead` fast path as §7.8). +2. Send, peek, purge, and delete against any queue, with visible leader routing so writes never quietly land on a follower. +3. Zero external build toolchain — shipped as a `go:embed` static bundle. +4. Operator-friendly auth that is distinct from the SigV4 credentials used by AWS SDK clients, so a human does not need to install `aws sigv4 curl` to load the console. + +Non-goals: + +1. Multi-tenant RBAC. Milestone 1 treats the console as single-role: whoever has the admin token can do anything. +2. Graphical metrics. Grafana already owns that surface; the console links out to the existing dashboards rather than duplicating them. +3. Message editing after send. SQS semantics do not support edit-in-place and neither does this console. + +### 13.2 Serving model + +The console runs on its own listener, controlled by a new `--consoleAddress` flag, exactly like `--metricsAddress` and `--pprofAddress`. Rationale: + +1. Operators usually restrict console access to an internal VLAN or SSH tunnel. A separate port lets them firewall the console without touching the SQS API port. +2. The SQS API port is SigV4-only; the console port uses bearer tokens. Mixing the two authenticators on one port is error-prone. +3. Running the console on loopback by default makes the zero-config development posture safe. + +``` +--consoleAddress "localhost:8090" # TCP host+port; empty disables +--consoleToken "" # bearer token; required for non-loopback +--consoleReadOnly false # if true, the console refuses mutating actions +``` + +`monitoring.AddressRequiresToken(addr)` already enforces "non-loopback requires a token" for metrics/pprof; the console reuses that helper. + +### 13.3 Architecture + +```mermaid +flowchart LR + Browser["Operator browser"] + subgraph Node["Elastickv node"] + Static["Static bundle (go:embed)"] + API["Console REST API\n/console/api/*"] + Auth["Bearer-token middleware"] + Proxy["Leader proxy (shared helper)"] + Internal["Internal queue ops\n(reuse SQSServer methods)"] + Store["MVCC Store"] + Raft["Raft"] + end + Browser -->|GET /console/| Static + Browser -->|JSON fetch| Auth + Auth --> API + API -->|writes| Proxy + Proxy --> Internal + API -->|reads| Internal + Internal --> Store + Internal --> Raft +``` + +Key pieces: + +1. `adapter/console.go` holds `ConsoleServer`, which embeds `*SQSServer` for access to the existing handler methods. It shares the queue catalog and message keyspace; no duplicate storage paths. +2. Static assets live under `adapter/console_ui/` (HTML + a single minimal JS file, both committed) and are served via `go:embed`. No npm/webpack pipeline. +3. The REST API uses JSON with plain status codes — it is **not** AWS-compatible. AWS SDKs do not talk to this endpoint. +4. Writes go through `proxyHTTPRequestToLeader` so a console tab pointed at a follower transparently hits the leader. Reads use `LeaseReadThrough` + local MVCC scans. + +### 13.4 REST API surface + +All paths are rooted at `/console/api/`. Requests carry `Authorization: Bearer <consoleToken>` when the token is configured. + +| Method | Path | Purpose | +|---|---|---| +| `GET` | `/queues` | list queues with attributes + approximate in-flight counts | +| `GET` | `/queues/{name}` | one queue's attributes + live counters | +| `PUT` | `/queues/{name}` | create queue; body is the same Attributes map as `CreateQueue` | +| `DELETE` | `/queues/{name}` | delete queue (same semantics as `DeleteQueue`) | +| `POST` | `/queues/{name}/purge` | purge queue (bumps generation, same as `PurgeQueue`) | +| `POST` | `/queues/{name}/messages` | send a message; body `{ body, delaySeconds?, attributes? }` | +| `GET` | `/queues/{name}/messages?limit=N` | **admin peek**; non-destructive scan of up to N visible messages | +| `GET` | `/queues/{name}/inflight?limit=N` | scan currently-invisible (in-flight) messages for debugging | +| `GET` | `/health` | returns `{ leader: bool, node: "..." }` for the dashboard banner | + +"Admin peek" is the one read path that bypasses the normal SQS surface: it scans `!sqs|msg|vis|` and `!sqs|msg|data|` at a snapshot read and returns body + attributes **without** rotating the receipt token or bumping visibility. This is intentional — the console is for observing state, not competing with workers. Clients that need at-least-once delivery must continue to use `ReceiveMessage`. + +The peek endpoint rejects requests larger than `limit=100` and caps total response bytes so a bad operator click cannot DoS the node. + +Error envelope follows Go idiom, not AWS: + +```json +{ "error": "queue_does_not_exist", "message": "queue \"orders\" not found" } +``` + +### 13.5 Front-end + +The bundled UI is a single-page HTML document plus one JS file. It is deliberately small so operators can read the source, and so it works over a slow SSH tunnel: + +1. Top bar shows the node id, Raft leadership (green "leader" / grey "follower"), and lease state — the `/console/api/health` fields. +2. Left pane lists queues with a name filter, refreshing on a user-controlled interval (default 5s, can be paused). +3. Right pane shows the selected queue's attributes, approximate in-flight count, and three tabs: + - **Peek** — paginated, non-destructive list of visible messages with body + attributes. + - **In-flight** — currently-invisible messages and the wall-clock time until they reappear. + - **Send** — a small form (body + optional delay) for posting a test message. +4. Destructive actions (Purge, Delete) are behind a confirm prompt that types the queue name to proceed, mirroring how the AWS console gates the same actions. +5. When `--consoleReadOnly` is set, the Send / Purge / Delete / PUT endpoints return `403 ReadOnly` and the UI greys out the corresponding buttons. + +No framework, no bundler: the JS file is hand-written, runs in modern browsers (ES2020+), and uses the `fetch` API. This is in the same spirit as the redis-proxy runbook's small internal UIs. + +### 13.6 Authentication + +1. Loopback + no token configured: the console is open. This is the "one-liner dev" case. +2. Non-loopback or token configured: every `/console/*` request must carry `Authorization: Bearer <token>`. Missing or wrong token returns `401 Unauthorized`. No cookie / session; the token is supplied by the operator (typed into a small login screen and stored in `sessionStorage`). +3. The token is compared with `subtle.ConstantTimeCompare`, same pattern as `metricsToken` / `pprofToken`. +4. The console never sees or proxies SigV4 credentials; it uses the coordinator directly via `SQSServer` methods that already run on the authenticated-as-the-server path. + +### 13.7 Failure and safety behaviors + +1. **Lost leader during a mutating call**: the leader proxy returns `503 ServiceUnavailable`; the UI shows a toast with "leader unavailable — retrying…" and a manual retry button. +2. **Peek under leader loss on the local node**: `LeaseReadThrough` falls back to `LinearizableRead`; if that also fails, the API returns `503` and the UI marks the right pane as "read paused". +3. **Purge + Delete confirmations** require the exact queue name typed in a confirmation box. The server additionally rejects Purge when it was issued less than 60 s after the previous Purge, matching AWS's own rate limit. +4. **Large peek requests** are clamped server-side (`limit`, total body bytes). The client additionally enforces `limit<=100` so a URL-bar typo cannot hurt the node. + +### 13.8 Observability + +New metrics (prefixed `sqs_console_`) mirror the SQS API metrics: + +1. `sqs_console_request_total{route, method, status}` +2. `sqs_console_request_duration_seconds{route, method}` +3. `sqs_console_peek_messages_total{queue}` +4. `sqs_console_write_total{queue, action, status}` +5. `sqs_console_auth_failure_total{reason}` + +Structured logs include `route`, `queue`, `action`, `remote_ip`, `token_hash_prefix` (first 8 hex of SHA-256 — enough to identify a token in logs without leaking it). + +### 13.9 Testing + +1. Unit: REST-handler table tests using `httptest.NewRecorder` for auth / validation. +2. Integration: single-node cluster via `createNode(t, 1)`, drive the console API through `http.Client`, assert queue create → peek → purge → delete round-trip. +3. UI smoke: a `testdata/console_ui_smoke_test.go` compiles the static bundle through `httptest.NewServer` and runs a handful of headless HTTP GETs to confirm the assets actually ship. (No Chromium; too heavy for CI.) +4. Auth: token match, token mismatch, loopback-no-token, non-loopback-no-token (rejected at startup). + +### 13.10 Out of scope for Milestone 1 + +1. SSO / OIDC / SAML. A bearer token is enough for the self-hosted operator use case this adapter targets. +2. Per-queue ACLs. Whoever holds the token can touch every queue. +3. Scheduled reports or message export. Observers run against metrics, not the console. + +## 14. Rollout Plan ### Phase 1 @@ -488,6 +638,7 @@ Under `jepsen/sqs/`, add `standard` and `fifo` test variants mirroring the exist 4. Leader proxy. 5. Metrics and structured logs. 6. Unit + integration test coverage. +7. Console UI read-only views (queues list, queue detail, peek, health). ### Phase 2 @@ -496,6 +647,7 @@ Under `jepsen/sqs/`, add `standard` and `fifo` test variants mirroring the exist 3. DLQ redrive. 4. Jepsen standard + FIFO workloads. 5. Tag APIs. +6. Console UI mutating actions (send, purge, delete, create), with the `--consoleReadOnly` gate. ### Phase 3 @@ -503,8 +655,9 @@ Under `jepsen/sqs/`, add `standard` and `fifo` test variants mirroring the exist 2. `ApproximateNumberOfMessagesDelayed` / `NotVisible` accuracy. 3. Per-queue throttling and fairness across tenants. 4. Split-queue FIFO for very hot queues. +5. Console UI polish: in-flight tab with per-message countdown, filtering, dark mode. -## 14. Summary +## 15. Summary The design choices are: @@ -514,6 +667,7 @@ The design choices are: 4. **All mutating operations (including ReceiveMessage) go through the leader** keeps the consistency story identical to the DynamoDB adapter. 5. **Reuse of the DynamoDB adapter's proxy and key-encoding helpers** keeps the adapter small; the streaming S3 proxy path is not needed because SQS payloads are bounded. 6. **Lease-read fast path for read-only APIs and the `ReceiveMessage` scan fence** amortizes ReadIndex across bursts and lets followers serve `GetQueueAttributes` / `ListQueues` without a leader round-trip, reusing the mechanism already merged in `docs/design/2026_04_20_implemented_lease_read.md`. +7. **Embedded operator console on its own listener** (§13) gives read-only visibility from any node and a clearly-gated administrative path, without borrowing the AWS API surface or requiring SigV4 for humans. The two repository-level prerequisites are: From e098bcf74ce04f5368877f0b4849d51528d7ee64 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 16:41:24 +0900 Subject: [PATCH 04/26] docs(sqs): address codex + coderabbit review feedback MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix three active findings from the latest codex review plus the coderabbit markdown polish: - P1 (codex) §5.4: "first key >= now" was inverted; the correct statement is "first N keys with visible_at <= now", which maps to a scan over [prefix|0, prefix|now+1). - P1 (codex) §7.2: drop the blanket "2 * k" FIFO scan cap. FIFO group locks can legitimately hide an entire page, so receive must page forward until MaxNumberOfMessages candidates survive the group-lock/dedup filters. Standard keeps the small page; FIFO uses continuation paging with a soft wall-clock budget. - P2 (codex) §7.3: long-poll notifier now wakes on any transition that makes a message deliverable - send, delay expiry, ChangeMessageVisibility that moves visible_at backward, and natural visibility-timeout expiry - tracked with a per-queue timer wheel over the smallest in-flight visible_at. The sequence-diagram caption is updated accordingly. - Markdown lint: add language tags to unlabeled fenced code blocks (MD040) and escape "|" in example keys inside table cells (MD056) so the table still parses correctly. --- ...6_04_24_proposed_sqs_compatible_adapter.md | 33 +++++++++++-------- docs/design/README.md | 4 +-- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 87b3df89c..82f1346e8 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -194,13 +194,13 @@ Fields: 3. **ChangeMessageVisibility:** same delete+insert swap as receive, validated against the supplied receipt handle. 4. **Delete:** remove both the data record and the visibility index entry. -Because `visible_at` leads the key, the next candidate message is always the first key `>= now` in the prefix. Invisible messages (`visible_at > now`) are naturally skipped without a sweeper. +Because `visible_at` leads the key, the next candidate messages are the first N keys in the prefix whose `visible_at ≤ now` — concretely, a scan over `[prefix|0, prefix|now+1)`. Invisible messages (`visible_at > now`) live past the upper bound of that range and are naturally skipped without a sweeper. ### 5.5 Receipt handles Receipt handles must be opaque to the client, cheap to validate, and impossible to forge: -``` +```text receipt_handle = base64url( queue_gen_u64 | message_id_16 | receipt_token_16 ) @@ -224,7 +224,7 @@ All message keys for a given queue must hash to the same Raft group, so that `Re Logical route: -``` +```text !sqsroute|<queue-esc><gen-u64> ``` @@ -299,7 +299,7 @@ sequenceDiagram S-->>C: 200 with batch else empty S->>N: Wait(queue, remaining) - N-->>S: wake on SendMessage commit or timeout + N-->>S: wake on send / delay / visibility transition / timer end end ``` @@ -307,8 +307,8 @@ sequenceDiagram Details: 1. Fence the snapshot scan with `coordinator.LeaseReadForKey(ctx, queueRoute)`. While the lease is valid (≤ `electionTimeout − leaseSafetyMargin`, currently 700 ms per `docs/design/2026_04_20_implemented_lease_read.md §3.2`) this returns immediately with no ReadIndex round-trip. Under sustained receive load this amortizes quorum confirmation across every receive in the lease window instead of paying one ReadIndex per call. -2. Snapshot-scan the visibility index for the first `k` keys with `visible_at ≤ now`, up to `MaxNumberOfMessages` (≤ 10). Use a bounded scan limit (e.g., `2 * k`) to tolerate races. -3. Filter FIFO candidates: skip any message whose group lock is held by another message; acquire the group lock on delivery. +2. Snapshot-scan the visibility index for keys with `visible_at ≤ now`. Standard queues read in page-sized chunks (page limit 1024) and stop once either `MaxNumberOfMessages` candidates survive the filter or the scan is exhausted; a small `2 * k` page is fine because Standard has no filter that can hide an unbounded prefix. FIFO queues **cannot** cap the scan at `2 * k` because an entire page can legitimately belong to groups whose locks are already held, hiding deliverable messages that sit further along in the index. FIFO receive instead continues paging forward (resuming from the last key returned) until `MaxNumberOfMessages` candidates survive the group-lock and dedup filters, or the end-of-range sentinel `visible_at = now+1` is reached. A soft wall-clock budget (e.g., 100 ms) on the overall scan caps latency when the queue is pathologically group-skewed, in which case the receive returns whatever it has accumulated. +3. Filter FIFO candidates: skip any message whose group lock is held by another message; acquire the group lock on delivery. Candidates rejected by this filter must not consume the `MaxNumberOfMessages` budget. 4. Run one OCC transaction per batch: for each candidate, verify the visibility entry still matches, rotate receipt token, set `visible_at_hlc = now + effective_timeout`, insert the new visibility entry, bump `receive_count` and `approximate_first_receive_timestamp`, acquire FIFO group lock if applicable. A successful `Propose` also refreshes the lease (per `§3.3` of the lease-read doc), so a busy queue keeps its fast path warm without any extra work. 5. If the batch is empty and `WaitTimeSeconds > 0`, register a waiter on the queue's in-process condition variable and re-run step 1 on wake or timeout. Do not poll in a tight loop. On re-scan, the `LeaseRead` fence from step 1 stays local for the whole long-poll window — empty long polls cost O(1) quorum operations, not O(ticks). 6. If `receive_count > redrive_policy.maxReceiveCount`, move the message to the DLQ (see §7.6) atomically within the receive transaction and do not return it to the caller. @@ -317,9 +317,16 @@ Lease invalidation handling: if the leader loses quorum during a long poll, `Reg ### 7.3 Long-poll notifier -Long polling is local to the leader: every node keeps a `map[queueRoute]*cond` and wakes waiters when a `SendMessage` commits on that queue. The notifier lives in the leader process; on failover, waiters on the old leader get `AbortedError` from the coordinator and clients retry against the new leader. AWS SDKs already retry receive on connection errors, so this is acceptable. +Long polling is local to the leader: every node keeps a `map[queueRoute]*cond` and wakes waiters whenever the set of deliverable messages on the queue may have grown. That includes any of these commit-time transitions: + +1. A `SendMessage` (or `SendMessageBatch` entry) commits — the obvious case. +2. A `SendMessage` whose `DelaySeconds` has now elapsed — the notifier schedules a one-shot timer keyed off `available_at` when the delayed message commits, and the timer wakes the queue's waiters when it fires. +3. A `ChangeMessageVisibility` commits a `VisibilityTimeout` that moves the record's `visible_at` backward (including the AWS-permitted value of zero) — the message becomes immediately deliverable even though no send happened. +4. A message's visibility timeout naturally expires. Because each message has a known `visible_at`, the notifier tracks the smallest `visible_at` in-flight per queue and wakes waiters at that deadline (same timer wheel as the delay case). -The notifier must subscribe to the local FSM commit stream, not the RPC path, so that a `SendMessage` that committed via another adapter node still wakes local waiters once the Raft log reaches this leader. +Waking on (1) alone would leave long-polling receivers blocked while deliverable messages already sit in the visibility index — a correctness-adjacent bug, because clients would see latency spikes at exactly the moments they want prompt delivery. The timer-wheel approach keeps this O(in-flight messages on this leader), not O(wall-clock ticks). + +The notifier subscribes to the local FSM commit stream, not the RPC path, so that a mutation committed via another adapter node still wakes local waiters once the Raft log reaches this leader. On failover, waiters on the old leader get `AbortedError` from the coordinator and clients retry against the new leader; AWS SDKs already retry receive on connection errors, so this is acceptable. ### 7.4 `DeleteMessage` @@ -356,10 +363,10 @@ Read-only APIs never mutate queue or message state, so they do not need the lead | API | Fenced key | Read scope | |---|---|---| -| `GetQueueUrl` | queue catalog route | `!sqs|queue|meta|<queue>` | -| `GetQueueAttributes` | queue catalog route | `!sqs|queue|meta|...` and derived counters | -| `ListQueues` | queue catalog route | prefix scan `!sqs|queue|meta|` | -| `ListQueueTags` | queue catalog route | `!sqs|queue|meta|<queue>.tags` | +| `GetQueueUrl` | queue catalog route | `!sqs\|queue\|meta\|<queue>` | +| `GetQueueAttributes` | queue catalog route | `!sqs\|queue\|meta\|...` and derived counters | +| `ListQueues` | queue catalog route | prefix scan `!sqs\|queue\|meta\|` | +| `ListQueueTags` | queue catalog route | `!sqs\|queue\|meta\|<queue>.tags` | The fast path: @@ -509,7 +516,7 @@ The console runs on its own listener, controlled by a new `--consoleAddress` fla 2. The SQS API port is SigV4-only; the console port uses bearer tokens. Mixing the two authenticators on one port is error-prone. 3. Running the console on loopback by default makes the zero-config development posture safe. -``` +```text --consoleAddress "localhost:8090" # TCP host+port; empty disables --consoleToken "" # bearer token; required for non-loopback --consoleReadOnly false # if true, the console refuses mutating actions diff --git a/docs/design/README.md b/docs/design/README.md index 67345d263..183c92730 100644 --- a/docs/design/README.md +++ b/docs/design/README.md @@ -4,7 +4,7 @@ Forward-looking design proposals and implementation plans for elastickv live in ## Filename convention -``` +```text YYYY_MM_DD_<status>_<name>.md ``` @@ -30,7 +30,7 @@ YYYY_MM_DD_<status>_<name>.md ### Examples -``` +```text 2026_04_20_implemented_lease_read.md 2026_04_18_proposed_raft_grpc_streaming_transport.md 2026_02_18_partial_hotspot_shard_split.md From 4206e63eb6992af66ffa5ece1cd90f089c7546d2 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 16:54:27 +0900 Subject: [PATCH 05/26] docs(sqs): address codex review round 3 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two follow-up findings on the SQS design doc: - P1 §7.4: DeleteMessage must be idempotent against stale receipt handles. AWS returns 200 no-op when the handle token no longer matches (a redelivery rotated it) or the record is already gone; SDK retry paths and batch workers rely on this. Rewritten §7.4 to spell out the three cases — structural error vs missing-record no-op vs token-mismatch no-op vs valid delete — and to note that ErrWriteConflict retries must re-apply the same classification. §7.5 explicitly contrasts ChangeMessageVisibility (AWS still errors on stale/missing handles, so we do too). - P2 §7.2 step 4: clarify that approximate_first_receive_timestamp is set only on the FIRST delivery, not bumped on every receive. The attribute is defined as "time of first receive" and overwriting it on redeliveries corrupts age/debug analytics the operators rely on. No behavior change at the storage or routing layer; only clarifies what the adapter must do when it encounters these cases. --- ...6_04_24_proposed_sqs_compatible_adapter.md | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 82f1346e8..58599d0d5 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -309,7 +309,7 @@ Details: 1. Fence the snapshot scan with `coordinator.LeaseReadForKey(ctx, queueRoute)`. While the lease is valid (≤ `electionTimeout − leaseSafetyMargin`, currently 700 ms per `docs/design/2026_04_20_implemented_lease_read.md §3.2`) this returns immediately with no ReadIndex round-trip. Under sustained receive load this amortizes quorum confirmation across every receive in the lease window instead of paying one ReadIndex per call. 2. Snapshot-scan the visibility index for keys with `visible_at ≤ now`. Standard queues read in page-sized chunks (page limit 1024) and stop once either `MaxNumberOfMessages` candidates survive the filter or the scan is exhausted; a small `2 * k` page is fine because Standard has no filter that can hide an unbounded prefix. FIFO queues **cannot** cap the scan at `2 * k` because an entire page can legitimately belong to groups whose locks are already held, hiding deliverable messages that sit further along in the index. FIFO receive instead continues paging forward (resuming from the last key returned) until `MaxNumberOfMessages` candidates survive the group-lock and dedup filters, or the end-of-range sentinel `visible_at = now+1` is reached. A soft wall-clock budget (e.g., 100 ms) on the overall scan caps latency when the queue is pathologically group-skewed, in which case the receive returns whatever it has accumulated. 3. Filter FIFO candidates: skip any message whose group lock is held by another message; acquire the group lock on delivery. Candidates rejected by this filter must not consume the `MaxNumberOfMessages` budget. -4. Run one OCC transaction per batch: for each candidate, verify the visibility entry still matches, rotate receipt token, set `visible_at_hlc = now + effective_timeout`, insert the new visibility entry, bump `receive_count` and `approximate_first_receive_timestamp`, acquire FIFO group lock if applicable. A successful `Propose` also refreshes the lease (per `§3.3` of the lease-read doc), so a busy queue keeps its fast path warm without any extra work. +4. Run one OCC transaction per batch: for each candidate, verify the visibility entry still matches, rotate receipt token, set `visible_at_hlc = now + effective_timeout`, insert the new visibility entry, bump `receive_count`, **set `approximate_first_receive_timestamp` only on the first delivery** (subsequent redeliveries must preserve the original value — it is defined as the time the message was first received), and acquire FIFO group lock if applicable. A successful `Propose` also refreshes the lease (per `§3.3` of the lease-read doc), so a busy queue keeps its fast path warm without any extra work. 5. If the batch is empty and `WaitTimeSeconds > 0`, register a waiter on the queue's in-process condition variable and re-run step 1 on wake or timeout. Do not poll in a tight loop. On re-scan, the `LeaseRead` fence from step 1 stays local for the whole long-poll window — empty long polls cost O(1) quorum operations, not O(ticks). 6. If `receive_count > redrive_policy.maxReceiveCount`, move the message to the DLQ (see §7.6) atomically within the receive transaction and do not return it to the caller. @@ -330,18 +330,28 @@ The notifier subscribes to the local FSM commit stream, not the RPC path, so tha ### 7.4 `DeleteMessage` -1. Parse receipt handle → `(queue_gen, message_id, receipt_token)`. -2. OCC transaction: - - Load `!sqs|msg|data|<queue><gen><message_id>`. Missing → `ReceiptHandleIsInvalid`. - - Compare `current_receipt_token`. Mismatch → `InvalidReceiptHandle`. - - Delete the data record, the current visibility index entry, and (FIFO) release the group lock. -3. `DeleteMessageBatch` aggregates entries into one multi-row OCC transaction and returns per-entry success/failure. +AWS SQS `DeleteMessage` is **idempotent against stale receipt handles**: a caller that retries a delete after the visibility timeout expired (and a new consumer has since received the same message under a rotated token) is not an error case — SQS returns 200 success without deleting the now-in-flight record. Batch workers and SDK retry paths rely on this to avoid double-delivery failures, so this adapter preserves that behavior. + +1. Parse the receipt handle → `(queue_gen, message_id, receipt_token)`. + - If the handle is structurally malformed (bad base64, wrong length, wrong version byte) return `ReceiptHandleIsInvalid`. This is the only error case. +2. Load `!sqs|msg|data|<queue><gen><message_id>` at a snapshot timestamp. + - **Missing record**: return 200 success. The message has already been deleted (either by us on a prior retry or by another consumer); the delete is a no-op. + - **Token mismatch**: return 200 success **without deleting**. The caller is holding a stale handle after a rotation; the current in-flight consumer keeps its message. + - **Token matches**: continue. +3. OCC transaction (with `ReadKeys` covering the data key and the current visibility entry) deletes the data record, the current visibility index entry, and (FIFO) releases the group lock. On `ErrWriteConflict` the whole pass — including step 2's token check — is retried so that a concurrent rotation that landed between our load and our commit is handled as a stale-handle no-op rather than a spurious delete. +4. `DeleteMessageBatch` aggregates entries into one multi-row OCC transaction and returns per-entry success/failure, applying the same stale-handle-is-success rule per entry. ### 7.5 `ChangeMessageVisibility` -1. Validate receipt handle and token as in `DeleteMessage`. +Unlike `DeleteMessage`, `ChangeMessageVisibility` requires an in-flight record and a matching token — AWS returns errors on stale handles because the caller is trying to reach into a specific delivery. The adapter follows suit. + +1. Parse the receipt handle. Malformed → `ReceiptHandleIsInvalid`. 2. Reject if `VisibilityTimeout > 12 hours` or negative. -3. OCC: delete old visibility entry, update `visible_at_hlc`, insert new visibility entry. Receipt token is **not** rotated (AWS does not rotate it for `ChangeMessageVisibility`). +3. Load the data record. + - Missing record → `ReceiptHandleIsInvalid` (SQS `MessageNotInflight` on the record, but we reuse the structural code). + - Token mismatch → `InvalidReceiptHandle`. + - `visible_at_hlc <= now` (already expired back to visible) → `MessageNotInflight`. +4. OCC (with `ReadKeys` on data + old vis entry): delete old visibility entry, update `visible_at_hlc`, insert new visibility entry. The receipt token is **not** rotated (AWS does not rotate it for `ChangeMessageVisibility`). ### 7.6 DLQ redrive From 559d1d4968a28710ed636090e6ca49c4ef76f5e5 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 17:01:11 +0900 Subject: [PATCH 06/26] docs(sqs): address codex review round 4 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two more P2 findings on §7.2 / §7.3: - §7.2 (step 2/3): the earlier revision only removed the 2*k scan cap for FIFO, but Standard queues can also hide an unbounded prefix via DLQ redrive — messages that exceeded maxReceiveCount are moved to the DLQ inside the same receive transaction and not returned, so a poison-message backlog masks deliverable messages deeper in the visibility index. The scan rule is now: both queue types keep paging forward until MaxNumberOfMessages candidates survive the filters OR the end-of-range sentinel is reached, with a soft wall-clock budget to cap latency. Step 3's filter is generalized to list both the FIFO group-lock rule and the DLQ-threshold rule. - §7.3: add a leader-failover bootstrap step. Timer-wheel entries scheduled on the old leader do not Raft-replicate, so the newly elected leader must scan !sqs|msg|vis| at LinearizableRead time and repopulate timers from in-flight visible_at values. Past deadlines fire an immediate wake. Bootstrap is single-shot per leader term with a lazy-populate fallback for extreme-fanout clusters. --- .../2026_04_24_proposed_sqs_compatible_adapter.md | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 58599d0d5..4aeef37fb 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -307,8 +307,11 @@ sequenceDiagram Details: 1. Fence the snapshot scan with `coordinator.LeaseReadForKey(ctx, queueRoute)`. While the lease is valid (≤ `electionTimeout − leaseSafetyMargin`, currently 700 ms per `docs/design/2026_04_20_implemented_lease_read.md §3.2`) this returns immediately with no ReadIndex round-trip. Under sustained receive load this amortizes quorum confirmation across every receive in the lease window instead of paying one ReadIndex per call. -2. Snapshot-scan the visibility index for keys with `visible_at ≤ now`. Standard queues read in page-sized chunks (page limit 1024) and stop once either `MaxNumberOfMessages` candidates survive the filter or the scan is exhausted; a small `2 * k` page is fine because Standard has no filter that can hide an unbounded prefix. FIFO queues **cannot** cap the scan at `2 * k` because an entire page can legitimately belong to groups whose locks are already held, hiding deliverable messages that sit further along in the index. FIFO receive instead continues paging forward (resuming from the last key returned) until `MaxNumberOfMessages` candidates survive the group-lock and dedup filters, or the end-of-range sentinel `visible_at = now+1` is reached. A soft wall-clock budget (e.g., 100 ms) on the overall scan caps latency when the queue is pathologically group-skewed, in which case the receive returns whatever it has accumulated. -3. Filter FIFO candidates: skip any message whose group lock is held by another message; acquire the group lock on delivery. Candidates rejected by this filter must not consume the `MaxNumberOfMessages` budget. +2. Snapshot-scan the visibility index for keys with `visible_at ≤ now`. Both Standard and FIFO queues read in page-sized chunks (page limit 1024) and keep paging forward (resuming from the last key returned) until `MaxNumberOfMessages` candidates survive the per-queue filters, or the end-of-range sentinel `visible_at = now+1` is reached. A small fixed-size first page (e.g., `2 * k`) is adequate for the common case, but the scan **must not** hard-cap at any fixed multiple of `k`, because the per-queue filters can legitimately hide an unbounded visible prefix: + - FIFO queues hide every message whose `MessageGroupId` already holds an in-flight record (group lock). + - Both queue types hide messages that have reached `redrive_policy.maxReceiveCount` — those are moved to the DLQ inside the same receive transaction (see step 6) and **not** returned to the caller, so a poison-message backlog can mask deliverable messages that sit further along in the index. + A soft wall-clock budget (e.g., 100 ms) caps the overall scan latency when the queue is pathologically group-skewed or DLQ-saturated; the receive then returns whatever survived the filters. +3. Filter candidates. FIFO: skip any message whose group lock is held by another message; acquire the group lock on delivery. Standard + FIFO: skip (and DLQ-redrive in step 6) any message whose `receive_count` would exceed `redrive_policy.maxReceiveCount`. Filter-rejected candidates must **not** consume the `MaxNumberOfMessages` budget. 4. Run one OCC transaction per batch: for each candidate, verify the visibility entry still matches, rotate receipt token, set `visible_at_hlc = now + effective_timeout`, insert the new visibility entry, bump `receive_count`, **set `approximate_first_receive_timestamp` only on the first delivery** (subsequent redeliveries must preserve the original value — it is defined as the time the message was first received), and acquire FIFO group lock if applicable. A successful `Propose` also refreshes the lease (per `§3.3` of the lease-read doc), so a busy queue keeps its fast path warm without any extra work. 5. If the batch is empty and `WaitTimeSeconds > 0`, register a waiter on the queue's in-process condition variable and re-run step 1 on wake or timeout. Do not poll in a tight loop. On re-scan, the `LeaseRead` fence from step 1 stays local for the whole long-poll window — empty long polls cost O(1) quorum operations, not O(ticks). 6. If `receive_count > redrive_policy.maxReceiveCount`, move the message to the DLQ (see §7.6) atomically within the receive transaction and do not return it to the caller. @@ -326,6 +329,14 @@ Long polling is local to the leader: every node keeps a `map[queueRoute]*cond` a Waking on (1) alone would leave long-polling receivers blocked while deliverable messages already sit in the visibility index — a correctness-adjacent bug, because clients would see latency spikes at exactly the moments they want prompt delivery. The timer-wheel approach keeps this O(in-flight messages on this leader), not O(wall-clock ticks). +**Leader failover bootstrap**. A newly elected leader inherits in-flight messages whose visibility deadlines were scheduled into timers on the previous leader; those timers do not Raft-replicate. Without a bootstrap step, those preexisting deadlines would pass with no event firing, so long-poll receivers would sleep until their `WaitTimeSeconds` expired even though messages became visible. On the `non-Leader → Leader` transition, the notifier therefore: + +1. Scans `!sqs|msg|vis|` for each queue under this leader's shard(s) at `LinearizableRead` timestamp, extracts every record's `visible_at`, and repopulates the per-queue timer wheel with those deadlines. +2. For deadlines already in the past, fires an immediate wake (since the receive path will find them deliverable on its next scan). +3. Records the bootstrap as a single-shot operation keyed off the leader term so a node that leads, loses, and re-leads runs it again. + +The bootstrap scan is bounded by in-flight count per queue (not total retention), so it is cheap in practice; clusters with extreme in-flight fan-out can lazy-populate the wheel on first receive per queue instead. + The notifier subscribes to the local FSM commit stream, not the RPC path, so that a mutation committed via another adapter node still wakes local waiters once the Raft log reaches this leader. On failover, waiters on the old leader get `AbortedError` from the coordinator and clients retry against the new leader; AWS SDKs already retry receive on connection errors, so this is acceptable. ### 7.4 `DeleteMessage` From feec3782e250c40ab04fac8577f9ef8f77b919d4 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 17:08:00 +0900 Subject: [PATCH 07/26] docs(sqs): address codex review round 5 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Four findings from codex round 5: - §3 (P2): FIFO in-flight quota was stated as 20,000 — the current AWS quota is 120,000. Updated so the adapter does not throttle valid FIFO workloads far earlier than SQS. - §5.6 (P1): FIFO group lock was described as releasing on visibility-timeout expiry, which lets a later message in the same MessageGroupId overtake the head message on the next receive (vis-index scans by visible_at ascending). Rewritten so the lock stays pinned to the head message across visibility expiries; it is only released by delete, DLQ move, or retention expiry. - §7.1 step 4 (P2): FIFO sends now reject per-message DelaySeconds with InvalidParameterValue — AWS only permits queue-level delay on FIFO, and silently accepting a per-message value would let misbehaving clients ship out of order. - §13.6 (P1): console bearer-auth required a header on every /console/* request, which breaks the documented login-UI flow because the browser cannot attach Authorization before the static HTML/JS that renders the login screen has loaded. Split routes: /console/, /console/index.html, /console/assets/** serve unauthenticated so the bundle can bootstrap; /console/api/** requires the bearer token. /console/api/health is also unauthenticated so the login page can show leader/follower status before sign-in. --- ...6_04_24_proposed_sqs_compatible_adapter.md | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index 4aeef37fb..e23438a91 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -72,7 +72,7 @@ The adapter should dispatch by request content-type / `X-Amz-Target` header and | `StartMessageMoveTask` and friends | Not planned | | | Server-side KMS | Not planned | | -Payload limits match AWS defaults: 256 KiB per message, 120,000 in-flight messages per standard queue, 20,000 in-flight per FIFO queue. These should be enforced at the adapter, not silently truncated. +Payload limits match AWS defaults: 256 KiB per message, 120,000 in-flight messages per Standard queue, and 120,000 in-flight per FIFO queue (current AWS quota — the old 20,000 FIFO ceiling was raised). These should be enforced at the adapter, not silently truncated. ## 4. High-Level Architecture @@ -212,7 +212,7 @@ receipt_handle = base64url( FIFO queues add three invariants on top of the standard schema: -1. **Group lock**: `!sqs|msg|group|<queue><gen><group>` is held by at most one message at a time. `ReceiveMessage` skips the entire group while the lock is held. The lock is released when the locking message is deleted, expires its visibility (and the next receive within the group rotates ownership), or the message retention expires. +1. **Group lock**: `!sqs|msg|group|<queue><gen><group>` is held by at most one message at a time. `ReceiveMessage` skips the entire group while the lock is held. Critically, the lock stays **pinned to the same head message** across visibility-timeout expiries: when the head message's visibility expires, the lock is **not** released — it remains held by that message so a later message in the same group cannot overtake it on the next receive. Instead, the next `ReceiveMessage` that hits the group re-delivers the same head message (with a bumped receive count + rotated receipt token) and extends its visibility. The lock is only released when the head message is deleted by the consumer, moved to the DLQ (receive count exceeded `maxReceiveCount`), or expired by `message_retention_seconds`. Releasing on visibility expiry would violate strict per-`MessageGroupId` ordering because a later message with an earlier effective `visible_at` could be scanned first. 2. **Deduplication**: `!sqs|msg|dedup|<queue><gen><dedup-id>` blocks a duplicate `SendMessage` for 5 minutes. For `ContentBasedDeduplication = true` the dedup id is `SHA-256(body)`. 3. **Per-queue sequence number**: `!sqs|queue|seq|<queue-esc><gen-u64>` → `uint64`, bumped by every FIFO send. Written into the message record for strict ordering. @@ -273,7 +273,7 @@ Steps: 1. Validate message size (`MaximumMessageSize` from queue attrs, capped at 262144). 2. Compute `md5_of_body` and `md5_of_message_attributes`. 3. Resolve queue generation; if queue is tombstoned return `QueueDoesNotExist`. -4. Compute `available_at_hlc = now + max(queue.delay, per-message DelaySeconds)`. +4. Compute `available_at_hlc`. Standard queues use `now + max(queue.delay, per-message DelaySeconds)`. **FIFO queues reject per-message `DelaySeconds` with `InvalidParameterValue`** — AWS only allows queue-level `DelaySeconds` on FIFO, and silently accepting a per-message value would diverge from real SQS behavior and let misbehaving clients ship out of order. FIFO sends therefore always compute `available_at_hlc = now + queue.delay`. 5. In one OCC transaction: - For FIFO: check and insert `!sqs|msg|dedup|...`. On hit, return the existing message's ID (AWS semantics). - For FIFO: increment `!sqs|queue|seq|...` and write it into the record. @@ -619,10 +619,16 @@ No framework, no bundler: the JS file is hand-written, runs in modern browsers ( ### 13.6 Authentication -1. Loopback + no token configured: the console is open. This is the "one-liner dev" case. -2. Non-loopback or token configured: every `/console/*` request must carry `Authorization: Bearer <token>`. Missing or wrong token returns `401 Unauthorized`. No cookie / session; the token is supplied by the operator (typed into a small login screen and stored in `sessionStorage`). -3. The token is compared with `subtle.ConstantTimeCompare`, same pattern as `metricsToken` / `pprofToken`. -4. The console never sees or proxies SigV4 credentials; it uses the coordinator directly via `SQSServer` methods that already run on the authenticated-as-the-server path. +The console distinguishes **bootstrap assets** (the static HTML + JS that renders the login screen) from the **REST API** (everything the login screen talks to after the operator types their token). Requiring a bearer header on the bootstrap fetch would create a chicken-and-egg problem: the browser cannot attach `Authorization` before the login UI that collects the token has loaded. + +1. Loopback + no token configured: the console is fully open. This is the "one-liner dev" case. +2. Token configured (any address): the split is: + - `GET /console/`, `GET /console/index.html`, `GET /console/assets/**` — **unauthenticated**. These serve the static bundle (HTML + JS + CSS) so the login UI can render. + - `GET/POST/PUT/DELETE /console/api/**` — **authenticated**. Every API call must carry `Authorization: Bearer <token>`. Missing or wrong token returns `401 Unauthorized`. + - `GET /console/api/health` is the one API endpoint that is also unauthenticated when a token is configured, so the login UI can render "leader: yes/no" before the operator signs in. +3. The login UI submits the typed token to `/console/api/auth/ping` (a trivial authenticated no-op); on 200 it caches the token in `sessionStorage` and uses it for subsequent fetches. On 401 it shows the error and clears state. +4. The token is compared with `subtle.ConstantTimeCompare`, same pattern as `metricsToken` / `pprofToken`. +5. The console never sees or proxies SigV4 credentials; it uses the coordinator directly via `SQSServer` methods that already run on the authenticated-as-the-server path. ### 13.7 Failure and safety behaviors From 1d24525e54fe379363b60e156e932e2cb83ed0bd Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 17:17:30 +0900 Subject: [PATCH 08/26] docs(sqs): address codex review round 6 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three findings: - §7.1 step 4 (P1): per-message DelaySeconds on Standard queues is an OVERRIDE of queue.delay, not max'd with it. AWS lets a client that knows the queue default is 5 min send an immediate message by passing DelaySeconds=0. The previous "max(queue.delay, per-message)" rule broke that compatibility. Rewritten so the effective delay is the per-message value when supplied, else queue.delay. FIFO still rejects per-message DelaySeconds. - §11 item 1 (P1): retention cannot drive off the visibility index because ChangeMessageVisibility can push visible_at past send_timestamp + retention, so the message never enters the scan window. Added a dedicated send-age index !sqs|msg|byage|<queue><gen><send_timestamp><msg-id> that the send path writes in the same txn and the reaper scans by age. ReceiveMessage also skips (and files for reaping) any candidate whose age has elapsed so a slow reaper cannot cause stale-retention deliveries. - §13.9 item 3 (P2): the proposed console UI smoke test lived under testdata/ — `go test` ignores testdata/ so the check would silently not run. The test file now lives at the package root (adapter/console_ui_smoke_test.go); fixtures it consumes can still live under adapter/testdata/console_ui/. --- docs/design/2026_04_24_proposed_sqs_compatible_adapter.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md index e23438a91..ed036ba24 100644 --- a/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md +++ b/docs/design/2026_04_24_proposed_sqs_compatible_adapter.md @@ -273,7 +273,7 @@ Steps: 1. Validate message size (`MaximumMessageSize` from queue attrs, capped at 262144). 2. Compute `md5_of_body` and `md5_of_message_attributes`. 3. Resolve queue generation; if queue is tombstoned return `QueueDoesNotExist`. -4. Compute `available_at_hlc`. Standard queues use `now + max(queue.delay, per-message DelaySeconds)`. **FIFO queues reject per-message `DelaySeconds` with `InvalidParameterValue`** — AWS only allows queue-level `DelaySeconds` on FIFO, and silently accepting a per-message value would diverge from real SQS behavior and let misbehaving clients ship out of order. FIFO sends therefore always compute `available_at_hlc = now + queue.delay`. +4. Compute `available_at_hlc`. Standard queues use `now + effective_delay`, where `effective_delay` is the **per-message `DelaySeconds` if the caller provided one, otherwise `queue.delay`** (the per-message value is an override, not a floor — AWS's timers override the queue default, they are not `max`'d together). A queue configured with a long default delay must still be able to send an immediate message when the caller passes `DelaySeconds=0`. **FIFO queues reject per-message `DelaySeconds` with `InvalidParameterValue`** — AWS only allows queue-level `DelaySeconds` on FIFO, and silently accepting a per-message value would diverge from real SQS behavior and let misbehaving clients ship out of order. FIFO sends therefore always compute `available_at_hlc = now + queue.delay`. 5. In one OCC transaction: - For FIFO: check and insert `!sqs|msg|dedup|...`. On hit, return the existing message's ID (AWS semantics). - For FIFO: increment `!sqs|queue|seq|...` and write it into the record. @@ -469,7 +469,7 @@ Structured log fields match the rest of the project: `queue`, `message_id`, `rec ## 11. Failure Handling and Cleanup -1. **Retention expiry**: a background reaper on the leader scans the visibility index for entries older than `send_timestamp + message_retention_seconds` and deletes them in bounded batches. Because visible_at ≤ send_time + retention implies the message is past retention in the common case, the reaper can mostly piggy-back on `ReceiveMessage` scans. +1. **Retention expiry**: SQS retention is measured by **message age** (`send_timestamp + message_retention_seconds`), including in-flight messages. The reaper therefore cannot drive purely off the visibility index — a message whose `visible_at` has been pushed far into the future via repeated `ChangeMessageVisibility` might never appear in the `visible_at ≤ now` scan even after its retention has elapsed, so it would live forever under a vis-index-only reaper. Instead, retention uses a dedicated **send-age index** `!sqs|msg|byage|<queue-esc><gen-u64><send_timestamp-u64><message-id-esc>` that the send path writes alongside the data + vis entries. A background reaper on the leader scans `!sqs|msg|byage|` for keys whose `send_timestamp ≤ now − retention` and, in one OCC txn per message, deletes the data record, the visibility entry (found via the record's current `visible_at`), the send-age entry, and (FIFO) releases the group lock. `ReceiveMessage` skips any candidate whose send-age stamp has elapsed and files it for reaping instead of delivering it, so a slow reaper cannot cause stale-retention deliveries. 2. **Old-generation reclaim**: `DeleteQueue` and `PurgeQueue` write `!sqs|queue|tombstone|...` markers. A background task drops remaining keys for the old generation. 3. **Leader failover during long poll**: waiters drain, clients reconnect. Because every state change was Raft-committed before being acknowledged, no send or delete is lost. 4. **Clock skew**: visibility timeouts use the cluster HLC, not wall clock. Sends across nodes remain monotonic because the HLC leader renews the physical ceiling every second (same mechanism the Redis/DynamoDB adapters already depend on). @@ -653,7 +653,7 @@ Structured logs include `route`, `queue`, `action`, `remote_ip`, `token_hash_pre 1. Unit: REST-handler table tests using `httptest.NewRecorder` for auth / validation. 2. Integration: single-node cluster via `createNode(t, 1)`, drive the console API through `http.Client`, assert queue create → peek → purge → delete round-trip. -3. UI smoke: a `testdata/console_ui_smoke_test.go` compiles the static bundle through `httptest.NewServer` and runs a handful of headless HTTP GETs to confirm the assets actually ship. (No Chromium; too heavy for CI.) +3. UI smoke: an `adapter/console_ui_smoke_test.go` file (living at the package root, **not** under a `testdata/` directory — `go test` ignores directories named `testdata` so a test there would silently be skipped in CI) compiles the static bundle through `httptest.NewServer` and runs a handful of headless HTTP GETs to confirm the assets actually ship. Fixtures the test loads (HTML / JS expectations) can live under `adapter/testdata/console_ui/`, but the `_test.go` file itself must sit in a regular package directory. 4. Auth: token match, token mismatch, loopback-no-token, non-loopback-no-token (rejected at startup). ### 13.10 Out of scope for Milestone 1 From 9f7e79fda1c10eea18d0a492c48c976c651231cd Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 05:05:44 +0900 Subject: [PATCH 09/26] feat(sqs): scaffold SQS-compatible adapter Start the SQS-compatible HTTP adapter per docs/design/2026_04_24_proposed_sqs_compatible_adapter.md. This first increment wires the server skeleton, dispatch table, error envelope, health endpoints, and leader-proxy path so subsequent PRs can drop real handlers into a tested frame. Included: - adapter/sqs.go: SQSServer struct, NewSQSServer with SQSServerOption, Run/Stop, /sqs_health + /sqs_leader_health, X-Amz-Target dispatch for all Milestone-1/2 SQS operations (CreateQueue ... ListQueueTags), and the JSON-1.0 error envelope (__type + x-amzn-ErrorType + message). Every target currently replies 501 NotImplemented; handlers will be filled in separately. - adapter/sqs_test.go: health endpoints, method gating, error-envelope shape, unknown-target -> InvalidAction, every known target -> NotImplemented, and a Stop-unblocks-Run regression guard. - adapter/leader_http_proxy.go: shared proxyHTTPRequestToLeader helper factored out of the DynamoDB adapter. DynamoDB and SQS both delegate to it via per-adapter error writers, removing the structural dup the linter flagged. No main.go wiring or flag changes in this PR; that lands alongside the first real handler so the binary does not expose a 501-only endpoint. --- adapter/dynamodb.go | 36 ++--- adapter/leader_http_proxy.go | 64 +++++++++ adapter/sqs.go | 257 +++++++++++++++++++++++++++++++++++ adapter/sqs_test.go | 223 ++++++++++++++++++++++++++++++ 4 files changed, 552 insertions(+), 28 deletions(-) create mode 100644 adapter/leader_http_proxy.go create mode 100644 adapter/sqs.go create mode 100644 adapter/sqs_test.go diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 0de6aa393..00a18d981 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -12,8 +12,6 @@ import ( "math/big" "net" "net/http" - "net/http/httputil" - "net/url" "slices" "sort" "strconv" @@ -294,33 +292,15 @@ func (d *DynamoDBServer) Stop() { // (either proxied or an error response was written), false if the request // should be handled locally (i.e. this node is the leader or no leader map is // configured). +// +// Serving reads or writes locally on a follower would expose G2-item-realtime +// stale reads, so every follower request is forwarded to the leader. func (d *DynamoDBServer) proxyToLeader(w http.ResponseWriter, r *http.Request) bool { - if len(d.leaderDynamo) == 0 || d.coordinator == nil { - return false - } - if d.coordinator.IsLeader() { - return false - } - // This node is a follower. All requests must be forwarded to the leader to - // preserve linearizability — serving reads or writes locally on a follower - // causes stale-read anomalies (G2-item-realtime). - leader := d.coordinator.RaftLeader() - if leader == "" { - writeDynamoError(w, http.StatusServiceUnavailable, dynamoErrInternal, "no raft leader currently available") - return true - } - targetAddr, ok := d.leaderDynamo[leader] - if !ok || strings.TrimSpace(targetAddr) == "" { - writeDynamoError(w, http.StatusServiceUnavailable, dynamoErrInternal, "leader dynamo address not found") - return true - } - target := &url.URL{Scheme: "http", Host: targetAddr} - proxy := httputil.NewSingleHostReverseProxy(target) - proxy.ErrorHandler = func(rw http.ResponseWriter, _ *http.Request, err error) { - writeDynamoError(rw, http.StatusServiceUnavailable, dynamoErrInternal, "leader proxy error: "+err.Error()) - } - proxy.ServeHTTP(w, r) - return true + return proxyHTTPRequestToLeader(d.coordinator, d.leaderDynamo, dynamoLeaderProxyErrorWriter, w, r) +} + +func dynamoLeaderProxyErrorWriter(w http.ResponseWriter, status int, message string) { + writeDynamoError(w, status, dynamoErrInternal, message) } func (d *DynamoDBServer) handle(w http.ResponseWriter, r *http.Request) { diff --git a/adapter/leader_http_proxy.go b/adapter/leader_http_proxy.go new file mode 100644 index 000000000..8483698b8 --- /dev/null +++ b/adapter/leader_http_proxy.go @@ -0,0 +1,64 @@ +package adapter + +import ( + "net/http" + "net/http/httputil" + "net/url" + "strings" + + "github.com/bootjp/elastickv/kv" +) + +// httpLeaderErrorWriter writes an adapter-specific error envelope (JSON for +// DynamoDB/SQS, XML for S3, RESP for Redis, ...) when the HTTP leader proxy +// cannot forward a follower request. The adapter owns the shape of the body; +// this helper only decides when to call it. +type httpLeaderErrorWriter func(w http.ResponseWriter, status int, message string) + +// proxyHTTPRequestToLeader forwards r to the current Raft leader's HTTP +// adapter endpoint when this node is a follower. It returns true when the +// request was handled (either proxied or an error body was written) and +// false when the caller should serve it locally (no leader map configured, +// no coordinator, or this node is the leader). +// +// Error paths: +// 1. no known Raft leader → 503 via errWriter("no raft leader currently available") +// 2. leader id missing from leaderMap → 503 via errWriter("leader address not found") +// 3. reverse-proxy dial/copy failure → 503 via errWriter("leader proxy error: <reason>") +// +// leaderMap keys are Raft addresses; values are the matching adapter HTTP +// addresses exported on that same node. +func proxyHTTPRequestToLeader( + coordinator kv.Coordinator, + leaderMap map[string]string, + errWriter httpLeaderErrorWriter, + w http.ResponseWriter, + r *http.Request, +) bool { + if len(leaderMap) == 0 || coordinator == nil { + return false + } + if coordinator.IsLeader() { + return false + } + // Follower ingress: forward to the leader so reads and writes both see a + // single serialization point. Serving locally on a follower would expose + // G2-item-realtime stale reads. + leader := coordinator.RaftLeader() + if leader == "" { + errWriter(w, http.StatusServiceUnavailable, "no raft leader currently available") + return true + } + targetAddr, ok := leaderMap[leader] + if !ok || strings.TrimSpace(targetAddr) == "" { + errWriter(w, http.StatusServiceUnavailable, "leader address not found") + return true + } + target := &url.URL{Scheme: "http", Host: targetAddr} + proxy := httputil.NewSingleHostReverseProxy(target) + proxy.ErrorHandler = func(rw http.ResponseWriter, _ *http.Request, err error) { + errWriter(rw, http.StatusServiceUnavailable, "leader proxy error: "+err.Error()) + } + proxy.ServeHTTP(w, r) + return true +} diff --git a/adapter/sqs.go b/adapter/sqs.go new file mode 100644 index 000000000..545d4fd81 --- /dev/null +++ b/adapter/sqs.go @@ -0,0 +1,257 @@ +package adapter + +import ( + "context" + "io" + "net" + "net/http" + "time" + + "github.com/bootjp/elastickv/kv" + "github.com/bootjp/elastickv/store" + "github.com/cockroachdb/errors" + json "github.com/goccy/go-json" +) + +// SQS target prefix for the JSON-1.0 protocol. Every supported operation is +// dispatched by the X-Amz-Target header, mirroring the DynamoDB adapter. +const sqsTargetPrefix = "AmazonSQS." + +const ( + sqsCreateQueueTarget = sqsTargetPrefix + "CreateQueue" + sqsDeleteQueueTarget = sqsTargetPrefix + "DeleteQueue" + sqsListQueuesTarget = sqsTargetPrefix + "ListQueues" + sqsGetQueueUrlTarget = sqsTargetPrefix + "GetQueueUrl" + sqsGetQueueAttributesTarget = sqsTargetPrefix + "GetQueueAttributes" + sqsSetQueueAttributesTarget = sqsTargetPrefix + "SetQueueAttributes" + sqsPurgeQueueTarget = sqsTargetPrefix + "PurgeQueue" + sqsSendMessageTarget = sqsTargetPrefix + "SendMessage" + sqsSendMessageBatchTarget = sqsTargetPrefix + "SendMessageBatch" + sqsReceiveMessageTarget = sqsTargetPrefix + "ReceiveMessage" + sqsDeleteMessageTarget = sqsTargetPrefix + "DeleteMessage" + sqsDeleteMessageBatchTarget = sqsTargetPrefix + "DeleteMessageBatch" + sqsChangeMessageVisibilityTarget = sqsTargetPrefix + "ChangeMessageVisibility" + sqsChangeMessageVisibilityBatchTgt = sqsTargetPrefix + "ChangeMessageVisibilityBatch" + sqsTagQueueTarget = sqsTargetPrefix + "TagQueue" + sqsUntagQueueTarget = sqsTargetPrefix + "UntagQueue" + sqsListQueueTagsTarget = sqsTargetPrefix + "ListQueueTags" +) + +const ( + sqsHealthPath = "/sqs_health" + sqsLeaderHealthPath = "/sqs_leader_health" +) + +const ( + sqsHealthMaxRequestBodyBytes = 1024 + sqsMaxRequestBodyBytes = 1 << 20 + sqsContentTypeJSON = "application/x-amz-json-1.0" +) + +// AWS SQS error codes used by the JSON protocol. The canonical list is on the +// "Common Errors" page of the SQS API reference. +const ( + sqsErrInvalidAction = "InvalidAction" + sqsErrNotImplemented = "NotImplemented" + sqsErrInternalFailure = "InternalFailure" + sqsErrServiceUnavailable = "ServiceUnavailable" + sqsErrMalformedRequest = "MalformedQueryString" +) + +type SQSServerOption func(*SQSServer) + +type SQSServer struct { + listen net.Listener + store store.MVCCStore + coordinator kv.Coordinator + httpServer *http.Server + targetHandlers map[string]func(http.ResponseWriter, *http.Request) + leaderSQS map[string]string +} + +// WithSQSLeaderMap configures the Raft-address-to-SQS-address mapping used to +// forward requests from followers to the current leader. Format mirrors +// WithDynamoDBLeaderMap / WithS3LeaderMap. +func WithSQSLeaderMap(m map[string]string) SQSServerOption { + return func(s *SQSServer) { + s.leaderSQS = make(map[string]string, len(m)) + for k, v := range m { + s.leaderSQS[k] = v + } + } +} + +func NewSQSServer(listen net.Listener, st store.MVCCStore, coordinate kv.Coordinator, opts ...SQSServerOption) *SQSServer { + s := &SQSServer{ + listen: listen, + store: st, + coordinator: coordinate, + } + s.targetHandlers = map[string]func(http.ResponseWriter, *http.Request){ + sqsCreateQueueTarget: s.notImplemented("CreateQueue"), + sqsDeleteQueueTarget: s.notImplemented("DeleteQueue"), + sqsListQueuesTarget: s.notImplemented("ListQueues"), + sqsGetQueueUrlTarget: s.notImplemented("GetQueueUrl"), + sqsGetQueueAttributesTarget: s.notImplemented("GetQueueAttributes"), + sqsSetQueueAttributesTarget: s.notImplemented("SetQueueAttributes"), + sqsPurgeQueueTarget: s.notImplemented("PurgeQueue"), + sqsSendMessageTarget: s.notImplemented("SendMessage"), + sqsSendMessageBatchTarget: s.notImplemented("SendMessageBatch"), + sqsReceiveMessageTarget: s.notImplemented("ReceiveMessage"), + sqsDeleteMessageTarget: s.notImplemented("DeleteMessage"), + sqsDeleteMessageBatchTarget: s.notImplemented("DeleteMessageBatch"), + sqsChangeMessageVisibilityTarget: s.notImplemented("ChangeMessageVisibility"), + sqsChangeMessageVisibilityBatchTgt: s.notImplemented("ChangeMessageVisibilityBatch"), + sqsTagQueueTarget: s.notImplemented("TagQueue"), + sqsUntagQueueTarget: s.notImplemented("UntagQueue"), + sqsListQueueTagsTarget: s.notImplemented("ListQueueTags"), + } + mux := http.NewServeMux() + mux.HandleFunc("/", s.handle) + s.httpServer = &http.Server{Handler: mux, ReadHeaderTimeout: time.Second} + for _, opt := range opts { + if opt != nil { + opt(s) + } + } + return s +} + +func (s *SQSServer) Run() error { + if err := s.httpServer.Serve(s.listen); err != nil && !errors.Is(err, http.ErrServerClosed) { + return errors.WithStack(err) + } + return nil +} + +func (s *SQSServer) Stop() { + if s.httpServer != nil { + _ = s.httpServer.Shutdown(context.Background()) + } +} + +func (s *SQSServer) handle(w http.ResponseWriter, r *http.Request) { + if s.serveHealthz(w, r) { + return + } + if s.proxyToLeader(w, r) { + return + } + + if r.Method != http.MethodPost { + w.Header().Set("Allow", http.MethodPost) + writeSQSError(w, http.StatusMethodNotAllowed, sqsErrMalformedRequest, "SQS JSON protocol requires POST") + return + } + + target := r.Header.Get("X-Amz-Target") + handler, ok := s.targetHandlers[target] + if !ok { + writeSQSError(w, http.StatusBadRequest, sqsErrInvalidAction, "unsupported SQS target: "+target) + return + } + handler(w, r) +} + +func (s *SQSServer) serveHealthz(w http.ResponseWriter, r *http.Request) bool { + if r == nil || r.URL == nil { + return false + } + switch r.URL.Path { + case sqsHealthPath: + if r.Body != nil { + r.Body = http.MaxBytesReader(w, r.Body, sqsHealthMaxRequestBodyBytes) + } + serveSQSHealthz(w, r) + return true + case sqsLeaderHealthPath: + if r.Body != nil { + r.Body = http.MaxBytesReader(w, r.Body, sqsHealthMaxRequestBodyBytes) + } + s.serveSQSLeaderHealthz(w, r) + return true + default: + return false + } +} + +func serveSQSHealthz(w http.ResponseWriter, r *http.Request) { + if !writeSQSHealthMethod(w, r) { + return + } + writeSQSHealthBody(w, r, http.StatusOK, "ok\n") +} + +func (s *SQSServer) serveSQSLeaderHealthz(w http.ResponseWriter, r *http.Request) { + if !writeSQSHealthMethod(w, r) { + return + } + if isVerifiedSQSLeader(s.coordinator) { + writeSQSHealthBody(w, r, http.StatusOK, "ok\n") + return + } + writeSQSHealthBody(w, r, http.StatusServiceUnavailable, "not leader\n") +} + +func isVerifiedSQSLeader(coordinator kv.Coordinator) bool { + if coordinator == nil || !coordinator.IsLeader() { + return false + } + return coordinator.VerifyLeader() == nil +} + +func writeSQSHealthMethod(w http.ResponseWriter, r *http.Request) bool { + switch r.Method { + case http.MethodGet, http.MethodHead: + return true + default: + w.Header().Set("Allow", "GET, HEAD") + http.Error(w, http.StatusText(http.StatusMethodNotAllowed), http.StatusMethodNotAllowed) + return false + } +} + +func writeSQSHealthBody(w http.ResponseWriter, r *http.Request, statusCode int, body string) { + w.Header().Set("Content-Type", "text/plain; charset=utf-8") + w.WriteHeader(statusCode) + if r.Method == http.MethodHead { + return + } + _, _ = io.WriteString(w, body) +} + +// proxyToLeader forwards the HTTP request to the current SQS leader when this +// node is not the Raft leader. Returns true if the request was handled. +func (s *SQSServer) proxyToLeader(w http.ResponseWriter, r *http.Request) bool { + return proxyHTTPRequestToLeader(s.coordinator, s.leaderSQS, sqsLeaderProxyErrorWriter, w, r) +} + +func sqsLeaderProxyErrorWriter(w http.ResponseWriter, status int, message string) { + writeSQSError(w, status, sqsErrServiceUnavailable, message) +} + +// notImplemented returns a handler that responds with a JSON-protocol +// NotImplemented error so clients get a clean signal while the real handlers +// are still being built out. +func (s *SQSServer) notImplemented(op string) func(http.ResponseWriter, *http.Request) { + return func(w http.ResponseWriter, _ *http.Request) { + writeSQSError(w, http.StatusNotImplemented, sqsErrNotImplemented, op+" is not implemented yet") + } +} + +// writeSQSError emits an SQS JSON-protocol error envelope. AWS returns: +// +// { "__type": "<code>", "message": "<text>" } +// +// with Content-Type application/x-amz-json-1.0 and the x-amzn-ErrorType header +// set to the code. SDKs key off x-amzn-ErrorType first, the body second. +func writeSQSError(w http.ResponseWriter, status int, code string, message string) { + resp := map[string]string{"message": message} + if code != "" { + resp["__type"] = code + w.Header().Set("x-amzn-ErrorType", code) + } + w.Header().Set("Content-Type", sqsContentTypeJSON) + w.WriteHeader(status) + _ = json.NewEncoder(w).Encode(resp) +} diff --git a/adapter/sqs_test.go b/adapter/sqs_test.go new file mode 100644 index 000000000..078d4f05f --- /dev/null +++ b/adapter/sqs_test.go @@ -0,0 +1,223 @@ +package adapter + +import ( + "context" + "io" + "net" + "net/http" + "net/http/httptest" + "strings" + "testing" + "time" + + json "github.com/goccy/go-json" +) + +// startTestSQSServer starts an SQSServer on an ephemeral localhost listener +// with no coordinator or leader map — enough to exercise the health +// endpoints, the dispatch table, and the error envelope. Real flows with Raft +// come in later PRs alongside the handler implementations. +func startTestSQSServer(t *testing.T) string { + t.Helper() + lc := &net.ListenConfig{} + listener, err := lc.Listen(context.Background(), "tcp", "127.0.0.1:0") + if err != nil { + t.Fatalf("listen: %v", err) + } + srv := NewSQSServer(listener, nil, nil) + go func() { + _ = srv.Run() + }() + t.Cleanup(func() { + srv.Stop() + }) + return "http://" + listener.Addr().String() +} + +func doRequest(t *testing.T, method, url string, body io.Reader, headers map[string]string) *http.Response { + t.Helper() + req, err := http.NewRequestWithContext(context.Background(), method, url, body) + if err != nil { + t.Fatalf("request: %v", err) + } + for k, v := range headers { + req.Header.Set(k, v) + } + resp, err := http.DefaultClient.Do(req) + if err != nil { + t.Fatalf("do: %v", err) + } + return resp +} + +func postSQSRequest(t *testing.T, url string, target string, body string) *http.Response { + t.Helper() + headers := map[string]string{"Content-Type": sqsContentTypeJSON} + if target != "" { + headers["X-Amz-Target"] = target + } + return doRequest(t, http.MethodPost, url, strings.NewReader(body), headers) +} + +func TestSQSServer_HealthOK(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + resp := doRequest(t, http.MethodGet, base+sqsHealthPath, nil, nil) + defer resp.Body.Close() + if resp.StatusCode != http.StatusOK { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusOK) + } + body, _ := io.ReadAll(resp.Body) + if got := strings.TrimSpace(string(body)); got != "ok" { + t.Fatalf("body: got %q want %q", got, "ok") + } +} + +func TestSQSServer_LeaderHealthWithoutCoordinatorIsNotLeader(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + resp := doRequest(t, http.MethodGet, base+sqsLeaderHealthPath, nil, nil) + defer resp.Body.Close() + if resp.StatusCode != http.StatusServiceUnavailable { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusServiceUnavailable) + } +} + +func TestSQSServer_HealthRejectsNonGetHead(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + resp := doRequest(t, http.MethodPost, base+sqsHealthPath, nil, nil) + defer resp.Body.Close() + if resp.StatusCode != http.StatusMethodNotAllowed { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusMethodNotAllowed) + } + if allow := resp.Header.Get("Allow"); !strings.Contains(allow, "GET") { + t.Fatalf("allow header: got %q", allow) + } +} + +func TestSQSServer_UnknownTargetReturnsInvalidAction(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + resp := postSQSRequest(t, base+"/", "AmazonSQS.NoSuchOperation", "{}") + defer resp.Body.Close() + if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusBadRequest) + } + if got := resp.Header.Get("x-amzn-ErrorType"); got != sqsErrInvalidAction { + t.Fatalf("error type: got %q want %q", got, sqsErrInvalidAction) + } + var body map[string]string + if err := json.NewDecoder(resp.Body).Decode(&body); err != nil { + t.Fatalf("decode: %v", err) + } + if body["__type"] != sqsErrInvalidAction { + t.Fatalf("body __type: got %q want %q", body["__type"], sqsErrInvalidAction) + } +} + +func TestSQSServer_KnownTargetsReturnNotImplemented(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + targets := []string{ + sqsCreateQueueTarget, + sqsDeleteQueueTarget, + sqsListQueuesTarget, + sqsGetQueueUrlTarget, + sqsGetQueueAttributesTarget, + sqsSetQueueAttributesTarget, + sqsPurgeQueueTarget, + sqsSendMessageTarget, + sqsSendMessageBatchTarget, + sqsReceiveMessageTarget, + sqsDeleteMessageTarget, + sqsDeleteMessageBatchTarget, + sqsChangeMessageVisibilityTarget, + sqsChangeMessageVisibilityBatchTgt, + sqsTagQueueTarget, + sqsUntagQueueTarget, + sqsListQueueTagsTarget, + } + for _, target := range targets { + t.Run(target, func(t *testing.T) { + t.Parallel() + resp := postSQSRequest(t, base+"/", target, "{}") + defer resp.Body.Close() + if resp.StatusCode != http.StatusNotImplemented { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusNotImplemented) + } + if got := resp.Header.Get("x-amzn-ErrorType"); got != sqsErrNotImplemented { + t.Fatalf("error type: got %q want %q", got, sqsErrNotImplemented) + } + }) + } +} + +func TestSQSServer_RejectsNonPostOnRoot(t *testing.T) { + t.Parallel() + base := startTestSQSServer(t) + + resp := doRequest(t, http.MethodGet, base+"/", nil, nil) + defer resp.Body.Close() + if resp.StatusCode != http.StatusMethodNotAllowed { + t.Fatalf("status: got %d want %d", resp.StatusCode, http.StatusMethodNotAllowed) + } + if allow := resp.Header.Get("Allow"); !strings.Contains(allow, http.MethodPost) { + t.Fatalf("allow header: got %q", allow) + } +} + +func TestSQSServer_ErrorEnvelopeShape(t *testing.T) { + t.Parallel() + // We check the precise envelope shape via httptest so that SDK parsers + // that key off x-amzn-ErrorType + __type + message do not regress. + rec := httptest.NewRecorder() + writeSQSError(rec, http.StatusBadRequest, sqsErrInvalidAction, "oops") + if got := rec.Header().Get("Content-Type"); got != sqsContentTypeJSON { + t.Fatalf("content-type: got %q", got) + } + if got := rec.Header().Get("x-amzn-ErrorType"); got != sqsErrInvalidAction { + t.Fatalf("error type header: got %q", got) + } + var body map[string]string + if err := json.Unmarshal(rec.Body.Bytes(), &body); err != nil { + t.Fatalf("decode: %v", err) + } + if body["__type"] != sqsErrInvalidAction { + t.Fatalf("__type: got %q", body["__type"]) + } + if body["message"] != "oops" { + t.Fatalf("message: got %q", body["message"]) + } +} + +// TestSQSServer_StopShutsDown guards against regressions where Stop leaves the +// goroutine leaking; after Stop, Run must return promptly. +func TestSQSServer_StopShutsDown(t *testing.T) { + t.Parallel() + lc := &net.ListenConfig{} + listener, err := lc.Listen(context.Background(), "tcp", "127.0.0.1:0") + if err != nil { + t.Fatalf("listen: %v", err) + } + srv := NewSQSServer(listener, nil, nil) + done := make(chan error, 1) + go func() { done <- srv.Run() }() + // Give Serve a chance to enter its accept loop. + time.Sleep(20 * time.Millisecond) + srv.Stop() + + select { + case err := <-done: + if err != nil { + t.Fatalf("run: %v", err) + } + case <-time.After(2 * time.Second): + t.Fatal("Run did not return within timeout after Stop") + } +} From e46ada90690d47a789aeeb7a18afdb0465d2b9b6 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 14:59:03 +0900 Subject: [PATCH 10/26] feat(sqs): wire SQS adapter into main.go and test harness Add the minimal plumbing to bring an SQSServer online next to the existing DynamoDB and S3 servers. No new handlers yet - every target still returns NotImplemented - but the binary and the test harness can now open an SQS endpoint so subsequent PRs that add real operations have somewhere to land. - shard_config.go: ErrInvalidRaftSQSMapEntry + parseRaftSQSMap. - main.go: --sqsAddress / --raftSqsMap flags, leaderSQS in runtimeConfig, buildLeaderSQS, and startSQSServer wired into runtimeServerRunner.start. sqsAddress defaults to empty. - main_sqs.go: startSQSServer mirrors main_s3.go two-goroutine shutdown pattern. - adapter/test_util.go: portsAdress / listeners / Node / newNode / setupNodes carry an sqsAddress and *SQSServer per node, shutdown calls Stop via an extracted shutdownNode helper, portSQS starts at 29000 so tests do not collide with the dynamo range at 28000. - main_bootstrap_e2e_test.go: update parseRuntimeConfig signature. --- adapter/test_util.go | 82 ++++++++++++++++++++++++++++---------- main.go | 23 ++++++++++- main_bootstrap_e2e_test.go | 2 +- main_sqs.go | 58 +++++++++++++++++++++++++++ shard_config.go | 5 +++ 5 files changed, 146 insertions(+), 24 deletions(-) create mode 100644 main_sqs.go diff --git a/adapter/test_util.go b/adapter/test_util.go index b43f024af..7dcf22ac2 100644 --- a/adapter/test_util.go +++ b/adapter/test_util.go @@ -52,28 +52,35 @@ func newTestFactory() raftengine.Factory { func shutdown(nodes []Node) { for _, n := range nodes { - if n.opsCancel != nil { - n.opsCancel() - } - n.grpcServer.Stop() - if n.grpcService != nil { - if err := n.grpcService.Close(); err != nil { - log.Printf("grpc service close: %v", err) - } - } - n.redisServer.Stop() - if n.dynamoServer != nil { - n.dynamoServer.Stop() + shutdownNode(n) + } +} + +func shutdownNode(n Node) { + if n.opsCancel != nil { + n.opsCancel() + } + n.grpcServer.Stop() + if n.grpcService != nil { + if err := n.grpcService.Close(); err != nil { + log.Printf("grpc service close: %v", err) } - if n.engine != nil { - if err := n.engine.Close(); err != nil { - log.Printf("engine close: %v", err) - } + } + n.redisServer.Stop() + if n.dynamoServer != nil { + n.dynamoServer.Stop() + } + if n.sqsServer != nil { + n.sqsServer.Stop() + } + if n.engine != nil { + if err := n.engine.Close(); err != nil { + log.Printf("engine close: %v", err) } - if n.closeFactory != nil { - if err := n.closeFactory(); err != nil { - log.Printf("factory close: %v", err) - } + } + if n.closeFactory != nil { + if err := n.closeFactory(); err != nil { + log.Printf("factory close: %v", err) } } } @@ -83,10 +90,12 @@ type portsAdress struct { raft int redis int dynamo int + sqs int grpcAddress string raftAddress string redisAddress string dynamoAddress string + sqsAddress string } const ( @@ -95,6 +104,7 @@ const ( raftPort = 50000 redisPort = 63790 dynamoPort = 28000 + sqsPort = 29000 ) var mu sync.Mutex @@ -102,12 +112,14 @@ var portGrpc atomic.Int32 var portRaft atomic.Int32 var portRedis atomic.Int32 var portDynamo atomic.Int32 +var portSQS atomic.Int32 func init() { portGrpc.Store(raftPort) portRaft.Store(grpcPort) portRedis.Store(redisPort) portDynamo.Store(dynamoPort) + portSQS.Store(sqsPort) } func portAssigner() portsAdress { @@ -117,15 +129,18 @@ func portAssigner() portsAdress { rp := portRaft.Add(1) rd := portRedis.Add(1) dn := portDynamo.Add(1) + sq := portSQS.Add(1) return portsAdress{ grpc: int(gp), raft: int(rp), redis: int(rd), dynamo: int(dn), + sqs: int(sq), grpcAddress: net.JoinHostPort("localhost", strconv.Itoa(int(gp))), raftAddress: net.JoinHostPort("localhost", strconv.Itoa(int(rp))), redisAddress: net.JoinHostPort("localhost", strconv.Itoa(int(rd))), dynamoAddress: net.JoinHostPort("localhost", strconv.Itoa(int(dn))), + sqsAddress: net.JoinHostPort("localhost", strconv.Itoa(int(sq))), } } @@ -134,25 +149,29 @@ type Node struct { raftAddress string redisAddress string dynamoAddress string + sqsAddress string grpcServer *grpc.Server grpcService *GRPCServer redisServer *RedisServer dynamoServer *DynamoDBServer + sqsServer *SQSServer opsCancel context.CancelFunc engine raftengine.Engine closeFactory func() error } -func newNode(grpcAddress, raftAddress, redisAddress, dynamoAddress string, engine raftengine.Engine, closeFactory func() error, grpcs *grpc.Server, grpcService *GRPCServer, rd *RedisServer, ds *DynamoDBServer, opsCancel context.CancelFunc) Node { +func newNode(grpcAddress, raftAddress, redisAddress, dynamoAddress, sqsAddress string, engine raftengine.Engine, closeFactory func() error, grpcs *grpc.Server, grpcService *GRPCServer, rd *RedisServer, ds *DynamoDBServer, sq *SQSServer, opsCancel context.CancelFunc) Node { return Node{ grpcAddress: grpcAddress, raftAddress: raftAddress, redisAddress: redisAddress, dynamoAddress: dynamoAddress, + sqsAddress: sqsAddress, grpcServer: grpcs, grpcService: grpcService, redisServer: rd, dynamoServer: ds, + sqsServer: sq, opsCancel: opsCancel, engine: engine, closeFactory: closeFactory, @@ -183,6 +202,7 @@ type listeners struct { grpc net.Listener redis net.Listener dynamo net.Listener + sqs net.Listener } func bindListeners(ctx context.Context, lc *net.ListenConfig, port portsAdress) (portsAdress, listeners, bool, error) { @@ -213,10 +233,22 @@ func bindListeners(ctx context.Context, lc *net.ListenConfig, port portsAdress) return port, listeners{}, false, errors.WithStack(err) } + sqsSock, err := lc.Listen(ctx, "tcp", port.sqsAddress) + if err != nil { + _ = grpcSock.Close() + _ = redisSock.Close() + _ = dynamoSock.Close() + if errors.Is(err, unix.EADDRINUSE) { + return port, listeners{}, true, nil + } + return port, listeners{}, false, errors.WithStack(err) + } + return port, listeners{ grpc: grpcSock, redis: redisSock, dynamo: dynamoSock, + sqs: sqsSock, }, false, nil } @@ -386,6 +418,7 @@ func setupNodes(t *testing.T, ctx context.Context, n int, ports []portsAdress) ( grpcSock := lis[i].grpc redisSock := lis[i].redis dynamoSock := lis[i].dynamo + sqsSock := lis[i].sqs result, err := factory.Create(raftengine.FactoryConfig{ LocalID: strconv.Itoa(i), @@ -433,17 +466,24 @@ func setupNodes(t *testing.T, ctx context.Context, n int, ports []portsAdress) ( assert.NoError(t, ds.Run()) }() + sq := NewSQSServer(sqsSock, routedStore, coordinator) + go func() { + assert.NoError(t, sq.Run()) + }() + nodes = append(nodes, newNode( port.grpcAddress, port.raftAddress, port.redisAddress, port.dynamoAddress, + port.sqsAddress, result.Engine, result.Close, s, gs, rd, ds, + sq, opsCancel, )) } diff --git a/main.go b/main.go index 4306388a8..a9bb6dbac 100644 --- a/main.go +++ b/main.go @@ -79,6 +79,7 @@ var ( s3Region = flag.String("s3Region", "us-east-1", "S3 signing region") s3CredsFile = flag.String("s3CredentialsFile", "", "Path to a JSON file containing static S3 credentials") s3PathStyleOnly = flag.Bool("s3PathStyleOnly", true, "Only accept path-style S3 requests") + sqsAddr = flag.String("sqsAddress", "", "TCP host+port for SQS-compatible API; empty to disable") metricsAddr = flag.String("metricsAddress", "localhost:9090", "TCP host+port for Prometheus metrics") metricsToken = flag.String("metricsToken", "", "Bearer token for Prometheus metrics; required for non-loopback metricsAddress") pprofAddr = flag.String("pprofAddress", "localhost:6060", "TCP host+port for pprof debug endpoints; empty to disable") @@ -93,6 +94,7 @@ var ( raftRedisMap = flag.String("raftRedisMap", "", "Map of Raft address to Redis address (raftAddr=redisAddr,...)") raftS3Map = flag.String("raftS3Map", "", "Map of Raft address to S3 address (raftAddr=s3Addr,...)") raftDynamoMap = flag.String("raftDynamoMap", "", "Map of Raft address to DynamoDB address (raftAddr=dynamoAddr,...)") + raftSqsMap = flag.String("raftSqsMap", "", "Map of Raft address to SQS address (raftAddr=sqsAddr,...)") ) func main() { @@ -212,6 +214,8 @@ func run() error { s3Region: *s3Region, s3CredsFile: *s3CredsFile, s3PathStyleOnly: *s3PathStyleOnly, + sqsAddress: *sqsAddr, + leaderSQS: cfg.leaderSQS, metricsAddress: *metricsAddr, metricsToken: *metricsToken, pprofAddress: *pprofAddr, @@ -238,7 +242,7 @@ func resolveRuntimeInputs() (runtimeConfig, raftEngineType, []raftengine.Server, return runtimeConfig{}, "", nil, false, err } - cfg, err := parseRuntimeConfig(*myAddr, *redisAddr, *s3Addr, *dynamoAddr, *raftGroups, *shardRanges, *raftRedisMap, *raftS3Map, *raftDynamoMap) + cfg, err := parseRuntimeConfig(*myAddr, *redisAddr, *s3Addr, *dynamoAddr, *sqsAddr, *raftGroups, *shardRanges, *raftRedisMap, *raftS3Map, *raftDynamoMap, *raftSqsMap) if err != nil { return runtimeConfig{}, "", nil, false, err } @@ -258,10 +262,11 @@ type runtimeConfig struct { leaderRedis map[string]string leaderS3 map[string]string leaderDynamo map[string]string + leaderSQS map[string]string multi bool } -func parseRuntimeConfig(myAddr, redisAddr, s3Addr, dynamoAddr, raftGroups, shardRanges, raftRedisMap, raftS3Map, raftDynamoMap string) (runtimeConfig, error) { +func parseRuntimeConfig(myAddr, redisAddr, s3Addr, dynamoAddr, sqsAddr, raftGroups, shardRanges, raftRedisMap, raftS3Map, raftDynamoMap, raftSqsMap string) (runtimeConfig, error) { groups, err := parseRaftGroups(raftGroups, myAddr) if err != nil { return runtimeConfig{}, errors.Wrapf(err, "failed to parse raft groups") @@ -288,6 +293,10 @@ func parseRuntimeConfig(myAddr, redisAddr, s3Addr, dynamoAddr, raftGroups, shard if err != nil { return runtimeConfig{}, errors.Wrapf(err, "failed to parse raft dynamo map") } + leaderSQS, err := buildLeaderSQS(groups, sqsAddr, raftSqsMap) + if err != nil { + return runtimeConfig{}, errors.Wrapf(err, "failed to parse raft sqs map") + } return runtimeConfig{ groups: groups, @@ -296,6 +305,7 @@ func parseRuntimeConfig(myAddr, redisAddr, s3Addr, dynamoAddr, raftGroups, shard leaderRedis: leaderRedis, leaderS3: leaderS3, leaderDynamo: leaderDynamo, + leaderSQS: leaderSQS, multi: len(groups) > 1, }, nil } @@ -316,6 +326,10 @@ func buildLeaderS3(groups []groupSpec, s3Addr string, raftS3Map string) (map[str return buildLeaderAddrMap(groups, s3Addr, raftS3Map, parseRaftS3Map) } +func buildLeaderSQS(groups []groupSpec, sqsAddr string, raftSqsMap string) (map[string]string, error) { + return buildLeaderAddrMap(groups, sqsAddr, raftSqsMap, parseRaftSQSMap) +} + func buildLeaderDynamo(groups []groupSpec, dynamoAddr string, raftDynamoMap string) (map[string]string, error) { return buildLeaderAddrMap(groups, dynamoAddr, raftDynamoMap, parseRaftDynamoMap) } @@ -824,6 +838,8 @@ type runtimeServerRunner struct { s3Region string s3CredsFile string s3PathStyleOnly bool + sqsAddress string + leaderSQS map[string]string metricsAddress string metricsToken string pprofAddress string @@ -856,6 +872,9 @@ func (r runtimeServerRunner) start() error { if err := startS3Server(r.ctx, r.lc, r.eg, r.s3Address, r.shardStore, r.coordinate, r.leaderS3, r.s3Region, r.s3CredsFile, r.s3PathStyleOnly, r.readTracker); err != nil { return waitErrgroupAfterStartupFailure(r.cancel, r.eg, err) } + if err := startSQSServer(r.ctx, r.lc, r.eg, r.sqsAddress, r.shardStore, r.coordinate, r.leaderSQS); err != nil { + return waitErrgroupAfterStartupFailure(r.cancel, r.eg, err) + } if err := startMetricsServer(r.ctx, r.lc, r.eg, r.metricsAddress, r.metricsToken, r.metricsRegistry.Handler()); err != nil { return waitErrgroupAfterStartupFailure(r.cancel, r.eg, err) } diff --git a/main_bootstrap_e2e_test.go b/main_bootstrap_e2e_test.go index b6040bb45..92474ddbc 100644 --- a/main_bootstrap_e2e_test.go +++ b/main_bootstrap_e2e_test.go @@ -340,7 +340,7 @@ func startBootstrapE2ENode( bootstrapMembers string, engineType raftEngineType, ) (*bootstrapE2ENode, error) { - cfg, err := parseRuntimeConfig(ep.raftAddr, ep.redisAddr, "", "", "", "", "", "", "") + cfg, err := parseRuntimeConfig(ep.raftAddr, ep.redisAddr, "", "", "", "", "", "", "", "", "") if err != nil { return nil, err } diff --git a/main_sqs.go b/main_sqs.go new file mode 100644 index 000000000..a6371e96e --- /dev/null +++ b/main_sqs.go @@ -0,0 +1,58 @@ +package main + +import ( + "context" + "net" + "strings" + + "github.com/bootjp/elastickv/adapter" + "github.com/bootjp/elastickv/kv" + "github.com/cockroachdb/errors" + "golang.org/x/sync/errgroup" +) + +func startSQSServer( + ctx context.Context, + lc *net.ListenConfig, + eg *errgroup.Group, + sqsAddr string, + shardStore *kv.ShardStore, + coordinate kv.Coordinator, + leaderSQS map[string]string, +) error { + sqsAddr = strings.TrimSpace(sqsAddr) + if sqsAddr == "" { + return nil + } + sqsL, err := lc.Listen(ctx, "tcp", sqsAddr) + if err != nil { + return errors.Wrapf(err, "failed to listen on %s", sqsAddr) + } + sqsServer := adapter.NewSQSServer( + sqsL, + shardStore, + coordinate, + adapter.WithSQSLeaderMap(leaderSQS), + ) + // Two-goroutine shutdown pattern mirrors startS3Server: one goroutine waits + // on either ctx.Done() or Run completion to call Stop, the other runs the + // server and cancels the waiter once it has returned. + runDoneCtx, runDoneCancel := context.WithCancel(context.Background()) + eg.Go(func() error { + select { + case <-ctx.Done(): + sqsServer.Stop() + case <-runDoneCtx.Done(): + } + return nil + }) + eg.Go(func() error { + err := sqsServer.Run() + runDoneCancel() + if err == nil || errors.Is(err, net.ErrClosed) { + return nil + } + return errors.WithStack(err) + }) + return nil +} diff --git a/shard_config.go b/shard_config.go index a7353038b..eee13d204 100644 --- a/shard_config.go +++ b/shard_config.go @@ -32,6 +32,7 @@ var ( ErrInvalidRaftRedisMapEntry = errors.New("invalid raftRedisMap entry") ErrInvalidRaftS3MapEntry = errors.New("invalid raftS3Map entry") ErrInvalidRaftDynamoMapEntry = errors.New("invalid raftDynamoMap entry") + ErrInvalidRaftSQSMapEntry = errors.New("invalid raftSqsMap entry") ErrInvalidRaftBootstrapMembersEntry = errors.New("invalid raftBootstrapMembers entry") ) @@ -128,6 +129,10 @@ func parseRaftDynamoMap(raw string) (map[string]string, error) { return parseRaftAddressMap(raw, ErrInvalidRaftDynamoMapEntry) } +func parseRaftSQSMap(raw string) (map[string]string, error) { + return parseRaftAddressMap(raw, ErrInvalidRaftSQSMapEntry) +} + func parseRaftAddressMap(raw string, invalidEntry error) (map[string]string, error) { out := make(map[string]string) if raw == "" { From 94904526437de895063154e5fd3b92344e288e04 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 15:10:13 +0900 Subject: [PATCH 11/26] feat(sqs): implement queue catalog CRUD Wire up the first real SQS handlers on top of the existing MVCC / Coordinator primitives: - CreateQueue (idempotent when attributes match, QueueNameExists otherwise; FIFO suffix + FifoQueue attribute cross-validated; every supported attribute validated against its AWS range) - DeleteQueue (tombstones the meta record and bumps the generation counter so a subsequent CreateQueue with the same name starts a new incarnation; actual message-keyspace reclaim lands with Stage 4) - ListQueues (prefix-scans !sqs|queue|meta|, supports QueueNamePrefix / MaxResults / NextToken) - GetQueueUrl - GetQueueAttributes / SetQueueAttributes Design choices: - Metadata is stored as JSON with a four-byte magic prefix so future encoding migrations can switch formats without reading back garbage. - Segment encoding uses base64 raw URL for queue names, matching the DynamoDB adapter's encodeDynamoSegment so operators reading raw Pebble keys see the same shape across adapters. - Attribute validation is table-driven via sqsAttributeAppliers so adding an attribute does not grow applyAttributes's cyclomatic complexity. - The OCC retry loop is split into tryCreateQueueOnce so each retry pass is self-contained and the outer loop only deals with backoff. - All mutating operations go through the leader via the shared proxyHTTPRequestToLeader helper, matching the DynamoDB adapter. Tests (adapter/sqs_catalog_test.go) spin up a real single-node cluster via createNode(t, 1) and exercise the end-to-end JSON-protocol path: Create/Get/List, idempotent Create, QueueNameExists on attribute change, Get/SetAttributes, Delete + follow-up NonExistentQueue, FIFO name/attribute validation, and key-encoding round trips. The scaffold "unknown target returns NotImplemented" test was updated to skip the now-implemented catalog targets. --- adapter/sqs.go | 12 +- adapter/sqs_catalog.go | 832 ++++++++++++++++++++++++++++++++++++ adapter/sqs_catalog_test.go | 294 +++++++++++++ adapter/sqs_keys.go | 58 +++ adapter/sqs_test.go | 11 +- 5 files changed, 1195 insertions(+), 12 deletions(-) create mode 100644 adapter/sqs_catalog.go create mode 100644 adapter/sqs_catalog_test.go create mode 100644 adapter/sqs_keys.go diff --git a/adapter/sqs.go b/adapter/sqs.go index 545d4fd81..d0ec3c150 100644 --- a/adapter/sqs.go +++ b/adapter/sqs.go @@ -88,12 +88,12 @@ func NewSQSServer(listen net.Listener, st store.MVCCStore, coordinate kv.Coordin coordinator: coordinate, } s.targetHandlers = map[string]func(http.ResponseWriter, *http.Request){ - sqsCreateQueueTarget: s.notImplemented("CreateQueue"), - sqsDeleteQueueTarget: s.notImplemented("DeleteQueue"), - sqsListQueuesTarget: s.notImplemented("ListQueues"), - sqsGetQueueUrlTarget: s.notImplemented("GetQueueUrl"), - sqsGetQueueAttributesTarget: s.notImplemented("GetQueueAttributes"), - sqsSetQueueAttributesTarget: s.notImplemented("SetQueueAttributes"), + sqsCreateQueueTarget: s.createQueue, + sqsDeleteQueueTarget: s.deleteQueue, + sqsListQueuesTarget: s.listQueues, + sqsGetQueueUrlTarget: s.getQueueUrl, + sqsGetQueueAttributesTarget: s.getQueueAttributes, + sqsSetQueueAttributesTarget: s.setQueueAttributes, sqsPurgeQueueTarget: s.notImplemented("PurgeQueue"), sqsSendMessageTarget: s.notImplemented("SendMessage"), sqsSendMessageBatchTarget: s.notImplemented("SendMessageBatch"), diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go new file mode 100644 index 000000000..010e96802 --- /dev/null +++ b/adapter/sqs_catalog.go @@ -0,0 +1,832 @@ +package adapter + +import ( + "bytes" + "context" + "io" + "net/http" + "net/url" + "regexp" + "sort" + "strconv" + "strings" + "time" + + "github.com/bootjp/elastickv/kv" + "github.com/bootjp/elastickv/store" + "github.com/cockroachdb/errors" + json "github.com/goccy/go-json" +) + +// AWS SQS defaults, reproduced from the public API reference so clients that +// send an empty Attributes map get standard behavior. +const ( + sqsDefaultVisibilityTimeoutSeconds = 30 + sqsDefaultRetentionSeconds = 345600 // 4 days + sqsDefaultDelaySeconds = 0 + sqsDefaultReceiveMessageWaitSeconds = 0 + sqsDefaultMaximumMessageSize = 262144 // 256 KiB + + sqsMaxVisibilityTimeoutSeconds = 43200 // 12 hours + sqsMaxRetentionSeconds = 1209600 // 14 days + sqsMinRetentionSeconds = 60 + sqsMaxDelaySeconds = 900 + sqsMaxReceiveMessageWaitSeconds = 20 + sqsMinMaximumMessageSize = 1024 + sqsMaximumAllowedMaximumMessageSize = 262144 + sqsMaxQueueNameLength = 80 + sqsFIFOQueueNameSuffix = ".fifo" + sqsListQueuesDefaultMaxResults = 1000 + sqsListQueuesHardMaxResults = 1000 + sqsQueueScanPageLimit = 1024 +) + +// AWS error codes specific to the queue catalog. +const ( + sqsErrValidation = "InvalidParameterValue" + sqsErrMissingParameter = "MissingParameter" + sqsErrQueueNameExists = "QueueNameExists" + sqsErrQueueDoesNotExist = "AWS.SimpleQueueService.NonExistentQueue" + sqsErrInvalidAttributeName = "InvalidAttributeName" + sqsErrInvalidAttributeValue = "InvalidAttributeValue" +) + +var sqsQueueNamePattern = regexp.MustCompile(`^[a-zA-Z0-9_-]{1,80}(\.fifo)?$`) + +// sqsQueueMeta is the Go mirror of the queue metadata record persisted at +// !sqs|queue|meta|<queue>. Serialized as JSON with a short magic prefix so +// future schema migrations can switch encoding without reading back garbage. +type sqsQueueMeta struct { + Name string `json:"name"` + Generation uint64 `json:"generation"` + CreatedAtHLC uint64 `json:"created_at_hlc,omitempty"` + IsFIFO bool `json:"is_fifo,omitempty"` + ContentBasedDedup bool `json:"content_based_dedup,omitempty"` + VisibilityTimeoutSeconds int64 `json:"visibility_timeout_seconds"` + MessageRetentionSeconds int64 `json:"message_retention_seconds"` + DelaySeconds int64 `json:"delay_seconds"` + ReceiveMessageWaitSeconds int64 `json:"receive_message_wait_seconds"` + MaximumMessageSize int64 `json:"maximum_message_size"` + RedrivePolicy string `json:"redrive_policy,omitempty"` + Tags map[string]string `json:"tags,omitempty"` +} + +var storedSQSMetaPrefix = []byte{0x00, 'S', 'Q', 0x01} + +func encodeSQSQueueMeta(m *sqsQueueMeta) ([]byte, error) { + body, err := json.Marshal(m) + if err != nil { + return nil, errors.WithStack(err) + } + out := make([]byte, 0, len(storedSQSMetaPrefix)+len(body)) + out = append(out, storedSQSMetaPrefix...) + out = append(out, body...) + return out, nil +} + +func decodeSQSQueueMeta(b []byte) (*sqsQueueMeta, error) { + if !bytes.HasPrefix(b, storedSQSMetaPrefix) { + return nil, errors.New("unrecognized sqs meta format") + } + var m sqsQueueMeta + if err := json.Unmarshal(b[len(storedSQSMetaPrefix):], &m); err != nil { + return nil, errors.WithStack(err) + } + return &m, nil +} + +// sqsAPIError is a typed error that captures the HTTP status and AWS error +// code so handler helpers can fail deep in the call chain and let the top +// level render a consistent envelope via writeSQSErrorFromErr. +type sqsAPIError struct { + status int + errorType string + message string +} + +func (e *sqsAPIError) Error() string { + if e == nil { + return "" + } + if e.message != "" { + return e.message + } + return http.StatusText(e.status) +} + +func newSQSAPIError(status int, errorType string, message string) error { + return &sqsAPIError{status: status, errorType: errorType, message: message} +} + +func writeSQSErrorFromErr(w http.ResponseWriter, err error) { + var apiErr *sqsAPIError + if errors.As(err, &apiErr) { + writeSQSError(w, apiErr.status, apiErr.errorType, apiErr.message) + return + } + writeSQSError(w, http.StatusInternalServerError, sqsErrInternalFailure, err.Error()) +} + +func writeSQSJSON(w http.ResponseWriter, payload any) { + w.Header().Set("Content-Type", sqsContentTypeJSON) + w.WriteHeader(http.StatusOK) + _ = json.NewEncoder(w).Encode(payload) +} + +// ------------------------ input decoding ------------------------ + +type sqsCreateQueueInput struct { + QueueName string `json:"QueueName"` + Attributes map[string]string `json:"Attributes"` + Tags map[string]string `json:"tags"` +} + +type sqsDeleteQueueInput struct { + QueueUrl string `json:"QueueUrl"` +} + +type sqsListQueuesInput struct { + QueueNamePrefix string `json:"QueueNamePrefix"` + MaxResults int `json:"MaxResults"` + NextToken string `json:"NextToken"` +} + +type sqsGetQueueUrlInput struct { + QueueName string `json:"QueueName"` +} + +type sqsGetQueueAttributesInput struct { + QueueUrl string `json:"QueueUrl"` + AttributeNames []string `json:"AttributeNames"` +} + +type sqsSetQueueAttributesInput struct { + QueueUrl string `json:"QueueUrl"` + Attributes map[string]string `json:"Attributes"` +} + +func decodeSQSJSONInput(r *http.Request, v any) error { + body, err := io.ReadAll(http.MaxBytesReader(nil, r.Body, sqsMaxRequestBodyBytes)) + if err != nil { + return newSQSAPIError(http.StatusBadRequest, sqsErrMalformedRequest, err.Error()) + } + if len(bytes.TrimSpace(body)) == 0 { + // Empty body is legal for some actions; leave v at its zero value. + return nil + } + if err := json.Unmarshal(body, v); err != nil { + return newSQSAPIError(http.StatusBadRequest, sqsErrMalformedRequest, err.Error()) + } + return nil +} + +// ------------------------ URL helpers ------------------------ + +// queueURL builds the AWS-compatible URL echoed back to clients. We follow +// the endpoint the client addressed so SDKs that re-sign subsequent requests +// against the same URL keep working behind reverse proxies. +func (s *SQSServer) queueURL(r *http.Request, queueName string) string { + scheme := "http" + if r.TLS != nil { + scheme = "https" + } + host := r.Host + if host == "" && s.listen != nil { + host = s.listen.Addr().String() + } + return scheme + "://" + host + "/" + queueName +} + +// scanContinuationSentinel is appended to a key to produce the exclusive +// upper bound for the next scan page — i.e. the smallest key strictly +// greater than k. +const scanContinuationSentinel = 0x00 + +func nextScanCursorAfter(key []byte) []byte { + return append(bytes.Clone(key), scanContinuationSentinel) +} + +func queueNameFromURL(queueUrl string) (string, error) { + if strings.TrimSpace(queueUrl) == "" { + return "", newSQSAPIError(http.StatusBadRequest, sqsErrMissingParameter, "missing QueueUrl") + } + parsed, err := url.Parse(queueUrl) + if err != nil { + return "", newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "invalid QueueUrl: "+err.Error()) + } + name := strings.TrimPrefix(parsed.Path, "/") + if name == "" { + return "", newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "QueueUrl path is empty") + } + // Strip an AWS-style account-id prefix so http://host/12345/MyQueue works. + if idx := strings.LastIndex(name, "/"); idx >= 0 { + name = name[idx+1:] + } + return name, nil +} + +func validateQueueName(name string) error { + name = strings.TrimSpace(name) + if name == "" { + return newSQSAPIError(http.StatusBadRequest, sqsErrMissingParameter, "missing QueueName") + } + if len(name) > sqsMaxQueueNameLength { + return newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "QueueName too long") + } + if !sqsQueueNamePattern.MatchString(name) { + return newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "QueueName contains invalid characters") + } + return nil +} + +// ------------------------ attribute parsing ------------------------ + +func parseAttributesIntoMeta(name string, attrs map[string]string) (*sqsQueueMeta, error) { + meta := &sqsQueueMeta{ + Name: name, + IsFIFO: strings.HasSuffix(name, sqsFIFOQueueNameSuffix), + VisibilityTimeoutSeconds: sqsDefaultVisibilityTimeoutSeconds, + MessageRetentionSeconds: sqsDefaultRetentionSeconds, + DelaySeconds: sqsDefaultDelaySeconds, + ReceiveMessageWaitSeconds: sqsDefaultReceiveMessageWaitSeconds, + MaximumMessageSize: sqsDefaultMaximumMessageSize, + } + if err := applyAttributes(meta, attrs); err != nil { + return nil, err + } + // FifoQueue attribute is authoritative if explicitly set; otherwise the + // .fifo suffix implies true and callers without the suffix get a + // standard queue. + if v, ok := attrs["FifoQueue"]; ok { + b, err := strconv.ParseBool(v) + if err != nil { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "FifoQueue must be a boolean") + } + if b && !strings.HasSuffix(name, sqsFIFOQueueNameSuffix) { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name must end in .fifo") + } + if !b && strings.HasSuffix(name, sqsFIFOQueueNameSuffix) { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "Queue name ends in .fifo but FifoQueue=false") + } + meta.IsFIFO = b + } + return meta, nil +} + +// attributeApplier writes one attribute into meta. Keeping one applier per +// attribute keeps applyAttributes trivial to dispatch through. +type attributeApplier func(meta *sqsQueueMeta, value string) error + +var sqsAttributeAppliers = map[string]attributeApplier{ + // FifoQueue is applied after applyAttributes returns (see + // parseAttributesIntoMeta) because it needs cross-field validation + // against the queue name. + "FifoQueue": func(_ *sqsQueueMeta, _ string) error { return nil }, + "VisibilityTimeout": func(m *sqsQueueMeta, v string) error { + n, err := parseIntAttr("VisibilityTimeout", v, 0, sqsMaxVisibilityTimeoutSeconds) + if err != nil { + return err + } + m.VisibilityTimeoutSeconds = n + return nil + }, + "MessageRetentionPeriod": func(m *sqsQueueMeta, v string) error { + n, err := parseIntAttr("MessageRetentionPeriod", v, sqsMinRetentionSeconds, sqsMaxRetentionSeconds) + if err != nil { + return err + } + m.MessageRetentionSeconds = n + return nil + }, + "DelaySeconds": func(m *sqsQueueMeta, v string) error { + n, err := parseIntAttr("DelaySeconds", v, 0, sqsMaxDelaySeconds) + if err != nil { + return err + } + m.DelaySeconds = n + return nil + }, + "ReceiveMessageWaitTimeSeconds": func(m *sqsQueueMeta, v string) error { + n, err := parseIntAttr("ReceiveMessageWaitTimeSeconds", v, 0, sqsMaxReceiveMessageWaitSeconds) + if err != nil { + return err + } + m.ReceiveMessageWaitSeconds = n + return nil + }, + "MaximumMessageSize": func(m *sqsQueueMeta, v string) error { + n, err := parseIntAttr("MaximumMessageSize", v, sqsMinMaximumMessageSize, sqsMaximumAllowedMaximumMessageSize) + if err != nil { + return err + } + m.MaximumMessageSize = n + return nil + }, + "ContentBasedDeduplication": func(m *sqsQueueMeta, v string) error { + b, err := strconv.ParseBool(v) + if err != nil { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "ContentBasedDeduplication must be a boolean") + } + m.ContentBasedDedup = b + return nil + }, + "RedrivePolicy": func(m *sqsQueueMeta, v string) error { + m.RedrivePolicy = v + return nil + }, +} + +// applyAttributes writes every entry in attrs into meta, returning a typed +// SQS error on the first unknown key or out-of-range value. +func applyAttributes(meta *sqsQueueMeta, attrs map[string]string) error { + for k, v := range attrs { + apply, ok := sqsAttributeAppliers[k] + if !ok { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeName, "unsupported attribute: "+k) + } + if err := apply(meta, v); err != nil { + return err + } + } + return nil +} + +func parseIntAttr(name, value string, minVal, maxVal int64) (int64, error) { + n, err := strconv.ParseInt(strings.TrimSpace(value), 10, 64) + if err != nil { + return 0, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, name+" must be an integer") + } + if n < minVal || n > maxVal { + return 0, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, name+" is out of range") + } + return n, nil +} + +// attributesEqual is used by CreateQueue for idempotency: calling the same +// CreateQueue twice with identical attributes is a no-op; differing values +// must fail with QueueNameExists. +func attributesEqual(a, b *sqsQueueMeta) bool { + if a == nil || b == nil { + return false + } + return a.IsFIFO == b.IsFIFO && + a.ContentBasedDedup == b.ContentBasedDedup && + a.VisibilityTimeoutSeconds == b.VisibilityTimeoutSeconds && + a.MessageRetentionSeconds == b.MessageRetentionSeconds && + a.DelaySeconds == b.DelaySeconds && + a.ReceiveMessageWaitSeconds == b.ReceiveMessageWaitSeconds && + a.MaximumMessageSize == b.MaximumMessageSize && + a.RedrivePolicy == b.RedrivePolicy +} + +// ------------------------ storage primitives ------------------------ + +func (s *SQSServer) nextTxnReadTS(ctx context.Context) uint64 { + maxTS := uint64(0) + if p, ok := s.store.(globalLastCommitTSProvider); ok { + maxTS = p.GlobalLastCommitTS(ctx) + } else if s.store != nil { + maxTS = s.store.LastCommitTS() + } + if s.coordinator != nil { + if clock := s.coordinator.Clock(); clock != nil && maxTS > 0 { + clock.Observe(maxTS) + } + } + if maxTS == 0 { + return 1 + } + return maxTS +} + +func (s *SQSServer) loadQueueMetaAt(ctx context.Context, queueName string, ts uint64) (*sqsQueueMeta, bool, error) { + b, err := s.store.GetAt(ctx, sqsQueueMetaKey(queueName), ts) + if err != nil { + if errors.Is(err, store.ErrKeyNotFound) { + return nil, false, nil + } + return nil, false, errors.WithStack(err) + } + meta, err := decodeSQSQueueMeta(b) + if err != nil { + return nil, false, err + } + return meta, true, nil +} + +func (s *SQSServer) loadQueueGenerationAt(ctx context.Context, queueName string, ts uint64) (uint64, error) { + b, err := s.store.GetAt(ctx, sqsQueueGenKey(queueName), ts) + if err != nil { + if errors.Is(err, store.ErrKeyNotFound) { + return 0, nil + } + return 0, errors.WithStack(err) + } + n, err := strconv.ParseUint(string(b), 10, 64) + if err != nil { + return 0, errors.WithStack(err) + } + return n, nil +} + +// ------------------------ handlers ------------------------ + +func (s *SQSServer) createQueue(w http.ResponseWriter, r *http.Request) { + var in sqsCreateQueueInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + if err := validateQueueName(in.QueueName); err != nil { + writeSQSErrorFromErr(w, err) + return + } + requested, err := parseAttributesIntoMeta(in.QueueName, in.Attributes) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + requested.Tags = in.Tags + + if err := s.createQueueWithRetry(r.Context(), requested); err != nil { + writeSQSErrorFromErr(w, err) + return + } + writeSQSJSON(w, map[string]string{"QueueUrl": s.queueURL(r, in.QueueName)}) +} + +func (s *SQSServer) createQueueWithRetry(ctx context.Context, requested *sqsQueueMeta) error { + backoff := transactRetryInitialBackoff + deadline := time.Now().Add(transactRetryMaxDuration) + for range transactRetryMaxAttempts { + done, err := s.tryCreateQueueOnce(ctx, requested) + if err == nil && done { + return nil + } + if err != nil && !isRetryableTransactWriteError(err) { + return err + } + if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { + return errors.WithStack(err) + } + backoff = nextTransactRetryBackoff(backoff) + } + return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "create queue retry attempts exhausted") +} + +// tryCreateQueueOnce runs one read/check/dispatch pass. The first bool reports +// whether the caller should stop retrying: true means the queue now exists +// with the requested attributes, false means the dispatch hit a retryable +// conflict and should be retried after backoff. +func (s *SQSServer) tryCreateQueueOnce(ctx context.Context, requested *sqsQueueMeta) (bool, error) { + readTS := s.nextTxnReadTS(ctx) + existing, exists, err := s.loadQueueMetaAt(ctx, requested.Name, readTS) + if err != nil { + return false, errors.WithStack(err) + } + if exists { + if attributesEqual(existing, requested) { + return true, nil + } + return false, newSQSAPIError(http.StatusBadRequest, sqsErrQueueNameExists, "queue already exists with different attributes") + } + lastGen, err := s.loadQueueGenerationAt(ctx, requested.Name, readTS) + if err != nil { + return false, errors.WithStack(err) + } + requested.Generation = lastGen + 1 + if clock := s.coordinator.Clock(); clock != nil { + requested.CreatedAtHLC = clock.Current() + } + metaBytes, err := encodeSQSQueueMeta(requested) + if err != nil { + return false, errors.WithStack(err) + } + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Put, Key: sqsQueueMetaKey(requested.Name), Value: metaBytes}, + {Op: kv.Put, Key: sqsQueueGenKey(requested.Name), Value: []byte(strconv.FormatUint(requested.Generation, 10))}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err != nil { + return false, errors.WithStack(err) + } + return true, nil +} + +func (s *SQSServer) deleteQueue(w http.ResponseWriter, r *http.Request) { + var in sqsDeleteQueueInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + name, err := queueNameFromURL(in.QueueUrl) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + if err := s.deleteQueueWithRetry(r.Context(), name); err != nil { + writeSQSErrorFromErr(w, err) + return + } + // SQS DeleteQueue returns 200 with an empty body. + writeSQSJSON(w, map[string]any{}) +} + +func (s *SQSServer) deleteQueueWithRetry(ctx context.Context, queueName string) error { + backoff := transactRetryInitialBackoff + deadline := time.Now().Add(transactRetryMaxDuration) + for range transactRetryMaxAttempts { + readTS := s.nextTxnReadTS(ctx) + _, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return errors.WithStack(err) + } + if !exists { + return newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + + // Bump the generation counter so any stragglers under the old + // generation are unreachable by routing. Actual message cleanup + // lands in a follow-up PR along with the message keyspace. + lastGen, err := s.loadQueueGenerationAt(ctx, queueName, readTS) + if err != nil { + return errors.WithStack(err) + } + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: sqsQueueMetaKey(queueName)}, + {Op: kv.Put, Key: sqsQueueGenKey(queueName), Value: []byte(strconv.FormatUint(lastGen+1, 10))}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err == nil { + return nil + } else if !isRetryableTransactWriteError(err) { + return errors.WithStack(err) + } + if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { + return errors.WithStack(err) + } + backoff = nextTransactRetryBackoff(backoff) + } + return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "delete queue retry attempts exhausted") +} + +func (s *SQSServer) listQueues(w http.ResponseWriter, r *http.Request) { + var in sqsListQueuesInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + maxResults := clampListQueuesMaxResults(in.MaxResults) + + names, err := s.scanQueueNames(r.Context()) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + sort.Strings(names) + filtered := filterByPrefix(names, in.QueueNamePrefix) + start := resolveListQueuesStart(filtered, in.NextToken) + + end := start + maxResults + truncated := end < len(filtered) + if !truncated { + end = len(filtered) + } + page := filtered[start:end] + + urls := make([]string, 0, len(page)) + for _, n := range page { + urls = append(urls, s.queueURL(r, n)) + } + resp := map[string]any{"QueueUrls": urls} + if truncated && len(page) > 0 { + resp["NextToken"] = encodeSQSSegment(page[len(page)-1]) + } + writeSQSJSON(w, resp) +} + +func clampListQueuesMaxResults(requested int) int { + if requested <= 0 { + return sqsListQueuesDefaultMaxResults + } + if requested > sqsListQueuesHardMaxResults { + return sqsListQueuesHardMaxResults + } + return requested +} + +func filterByPrefix(names []string, prefix string) []string { + if prefix == "" { + return names + } + out := names[:0] + for _, n := range names { + if strings.HasPrefix(n, prefix) { + out = append(out, n) + } + } + return out +} + +// resolveListQueuesStart decodes the NextToken boundary and returns the index +// of the first name strictly greater than it. A malformed token behaves as +// "start from the beginning" to match AWS's lenient behavior. +func resolveListQueuesStart(names []string, token string) int { + if token == "" { + return 0 + } + boundary, err := decodeSQSSegment(token) + if err != nil { + return 0 + } + for i, n := range names { + if n > boundary { + return i + } + } + return len(names) +} + +func (s *SQSServer) scanQueueNames(ctx context.Context) ([]string, error) { + prefix := []byte(SqsQueueMetaPrefix) + end := prefixScanEnd(prefix) + start := bytes.Clone(prefix) + readTS := s.nextTxnReadTS(ctx) + var names []string + for { + kvs, err := s.store.ScanAt(ctx, start, end, sqsQueueScanPageLimit, readTS) + if err != nil { + return nil, errors.WithStack(err) + } + if len(kvs) == 0 { + break + } + for _, kvp := range kvs { + if !bytes.HasPrefix(kvp.Key, prefix) { + return names, nil + } + name, ok := queueNameFromMetaKey(kvp.Key) + if !ok { + continue + } + names = append(names, name) + } + if len(kvs) < sqsQueueScanPageLimit { + break + } + start = nextScanCursorAfter(kvs[len(kvs)-1].Key) + if end != nil && bytes.Compare(start, end) > 0 { + break + } + } + return names, nil +} + +func (s *SQSServer) getQueueUrl(w http.ResponseWriter, r *http.Request) { + var in sqsGetQueueUrlInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + if err := validateQueueName(in.QueueName); err != nil { + writeSQSErrorFromErr(w, err) + return + } + _, exists, err := s.loadQueueMetaAt(r.Context(), in.QueueName, s.nextTxnReadTS(r.Context())) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + if !exists { + writeSQSError(w, http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + return + } + writeSQSJSON(w, map[string]string{"QueueUrl": s.queueURL(r, in.QueueName)}) +} + +func (s *SQSServer) getQueueAttributes(w http.ResponseWriter, r *http.Request) { + var in sqsGetQueueAttributesInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + name, err := queueNameFromURL(in.QueueUrl) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + meta, exists, err := s.loadQueueMetaAt(r.Context(), name, s.nextTxnReadTS(r.Context())) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + if !exists { + writeSQSError(w, http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + return + } + selection := selectedAttributeNames(in.AttributeNames) + attrs := queueMetaToAttributes(meta, selection) + writeSQSJSON(w, map[string]any{"Attributes": attrs}) +} + +// selectedAttributeNames returns a set of attribute names to include in the +// response. An empty selection, or any entry equal to "All", expands to +// every supported attribute. +func selectedAttributeNames(req []string) map[string]bool { + selection := map[string]bool{} + if len(req) == 0 { + return nil + } + for _, n := range req { + if n == "All" { + return nil + } + selection[n] = true + } + return selection +} + +func queueMetaToAttributes(meta *sqsQueueMeta, selection map[string]bool) map[string]string { + all := map[string]string{ + "VisibilityTimeout": strconv.FormatInt(meta.VisibilityTimeoutSeconds, 10), + "MessageRetentionPeriod": strconv.FormatInt(meta.MessageRetentionSeconds, 10), + "DelaySeconds": strconv.FormatInt(meta.DelaySeconds, 10), + "ReceiveMessageWaitTimeSeconds": strconv.FormatInt(meta.ReceiveMessageWaitSeconds, 10), + "MaximumMessageSize": strconv.FormatInt(meta.MaximumMessageSize, 10), + "FifoQueue": strconv.FormatBool(meta.IsFIFO), + "ContentBasedDeduplication": strconv.FormatBool(meta.ContentBasedDedup), + } + if meta.RedrivePolicy != "" { + all["RedrivePolicy"] = meta.RedrivePolicy + } + if selection == nil { + return all + } + out := make(map[string]string, len(selection)) + for k := range selection { + if v, ok := all[k]; ok { + out[k] = v + } + } + return out +} + +func (s *SQSServer) setQueueAttributes(w http.ResponseWriter, r *http.Request) { + var in sqsSetQueueAttributesInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + name, err := queueNameFromURL(in.QueueUrl) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + if err := s.setQueueAttributesWithRetry(r.Context(), name, in.Attributes); err != nil { + writeSQSErrorFromErr(w, err) + return + } + writeSQSJSON(w, map[string]any{}) +} + +func (s *SQSServer) setQueueAttributesWithRetry(ctx context.Context, queueName string, attrs map[string]string) error { + backoff := transactRetryInitialBackoff + deadline := time.Now().Add(transactRetryMaxDuration) + for range transactRetryMaxAttempts { + readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return errors.WithStack(err) + } + if !exists { + return newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + if err := applyAttributes(meta, attrs); err != nil { + return err + } + metaBytes, err := encodeSQSQueueMeta(meta) + if err != nil { + return errors.WithStack(err) + } + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Put, Key: sqsQueueMetaKey(queueName), Value: metaBytes}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err == nil { + return nil + } else if !isRetryableTransactWriteError(err) { + return errors.WithStack(err) + } + if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { + return errors.WithStack(err) + } + backoff = nextTransactRetryBackoff(backoff) + } + return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "set queue attributes retry attempts exhausted") +} diff --git a/adapter/sqs_catalog_test.go b/adapter/sqs_catalog_test.go new file mode 100644 index 000000000..3dac5fd54 --- /dev/null +++ b/adapter/sqs_catalog_test.go @@ -0,0 +1,294 @@ +package adapter + +import ( + "bytes" + "context" + "io" + "net/http" + "strings" + "testing" + + json "github.com/goccy/go-json" +) + +// callSQS routes a JSON-protocol request to the given node's SQS endpoint. +// The helper exists so tests read like "createQueue → 200 with a URL" rather +// than having to hand-build X-Amz-Target envelopes every time. +func callSQS(t *testing.T, node Node, target string, in any) (int, map[string]any) { + t.Helper() + body, err := json.Marshal(in) + if err != nil { + t.Fatalf("marshal: %v", err) + } + req, err := http.NewRequestWithContext(context.Background(), http.MethodPost, + "http://"+node.sqsAddress+"/", bytes.NewReader(body)) + if err != nil { + t.Fatalf("request: %v", err) + } + req.Header.Set("X-Amz-Target", target) + req.Header.Set("Content-Type", sqsContentTypeJSON) + resp, err := http.DefaultClient.Do(req) + if err != nil { + t.Fatalf("do: %v", err) + } + defer resp.Body.Close() + raw, _ := io.ReadAll(resp.Body) + out := map[string]any{} + if len(bytes.TrimSpace(raw)) > 0 { + if err := json.Unmarshal(raw, &out); err != nil { + t.Fatalf("decode %q: %v", string(raw), err) + } + } + return resp.StatusCode, out +} + +func sqsLeaderNode(t *testing.T, nodes []Node) Node { + t.Helper() + for _, n := range nodes { + if n.engine != nil && n.engine.Leader().Address == n.raftAddress { + return n + } + } + return nodes[0] +} + +func TestSQSServer_CatalogCreateGetList(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + // CreateQueue: 200 with a QueueUrl that ends in the queue name. + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "orders", + }) + if status != http.StatusOK { + t.Fatalf("create: status %d body %v", status, out) + } + url, _ := out["QueueUrl"].(string) + if !strings.HasSuffix(url, "/orders") { + t.Fatalf("QueueUrl %q does not end in /orders", url) + } + + // GetQueueUrl returns the same URL. + status, out = callSQS(t, node, sqsGetQueueUrlTarget, map[string]any{ + "QueueName": "orders", + }) + if status != http.StatusOK { + t.Fatalf("getQueueUrl: status %d body %v", status, out) + } + if got, _ := out["QueueUrl"].(string); got != url { + t.Fatalf("GetQueueUrl=%q want %q", got, url) + } + + // ListQueues sees it. + status, out = callSQS(t, node, sqsListQueuesTarget, map[string]any{}) + if status != http.StatusOK { + t.Fatalf("list: status %d body %v", status, out) + } + urls, _ := out["QueueUrls"].([]any) + foundList := false + for _, u := range urls { + if s, _ := u.(string); s == url { + foundList = true + break + } + } + if !foundList { + t.Fatalf("ListQueues did not include %q; got %v", url, urls) + } +} + +func TestSQSServer_CatalogCreateIsIdempotent(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + in := map[string]any{ + "QueueName": "idempotent", + "Attributes": map[string]string{ + "VisibilityTimeout": "60", + }, + } + status1, out1 := callSQS(t, node, sqsCreateQueueTarget, in) + if status1 != http.StatusOK { + t.Fatalf("first create: %d %v", status1, out1) + } + // Second call with the same attributes must succeed with the same URL. + status2, out2 := callSQS(t, node, sqsCreateQueueTarget, in) + if status2 != http.StatusOK { + t.Fatalf("second create (same attrs): %d %v", status2, out2) + } + if out1["QueueUrl"] != out2["QueueUrl"] { + t.Fatalf("idempotent create returned different URLs: %v vs %v", out1, out2) + } + + // Third call with differing attributes must fail with QueueNameExists. + changed := map[string]any{ + "QueueName": "idempotent", + "Attributes": map[string]string{"VisibilityTimeout": "120"}, + } + status3, out3 := callSQS(t, node, sqsCreateQueueTarget, changed) + if status3 != http.StatusBadRequest { + t.Fatalf("differing-attrs create: got %d want 400; body %v", status3, out3) + } + if got, _ := out3["__type"].(string); got != sqsErrQueueNameExists { + t.Fatalf("differing-attrs error type: got %q want %q", got, sqsErrQueueNameExists) + } +} + +func TestSQSServer_CatalogGetAndSetAttributes(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "attrs", + }) + if status != http.StatusOK { + t.Fatalf("create: %d %v", status, out) + } + url, _ := out["QueueUrl"].(string) + + status, out = callSQS(t, node, sqsGetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "AttributeNames": []string{"All"}, + }) + if status != http.StatusOK { + t.Fatalf("getAttrs: %d %v", status, out) + } + attrs, _ := out["Attributes"].(map[string]any) + if attrs["VisibilityTimeout"] != "30" { + t.Fatalf("default VisibilityTimeout = %v, want 30", attrs["VisibilityTimeout"]) + } + + status, out = callSQS(t, node, sqsSetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "Attributes": map[string]string{ + "VisibilityTimeout": "90", + "DelaySeconds": "5", + }, + }) + if status != http.StatusOK { + t.Fatalf("setAttrs: %d %v", status, out) + } + + _, out = callSQS(t, node, sqsGetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "AttributeNames": []string{"VisibilityTimeout", "DelaySeconds"}, + }) + attrs, _ = out["Attributes"].(map[string]any) + if attrs["VisibilityTimeout"] != "90" || attrs["DelaySeconds"] != "5" { + t.Fatalf("updated attrs = %v, want VisibilityTimeout=90 DelaySeconds=5", attrs) + } +} + +func TestSQSServer_CatalogDelete(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + _, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "deleteme", + }) + url, _ := out["QueueUrl"].(string) + + status, out := callSQS(t, node, sqsDeleteQueueTarget, map[string]any{ + "QueueUrl": url, + }) + if status != http.StatusOK { + t.Fatalf("delete: %d %v", status, out) + } + + // GetQueueUrl after delete returns NonExistentQueue. + status, out = callSQS(t, node, sqsGetQueueUrlTarget, map[string]any{ + "QueueName": "deleteme", + }) + if status != http.StatusBadRequest { + t.Fatalf("getQueueUrl after delete: got %d want 400; body %v", status, out) + } + if got, _ := out["__type"].(string); got != sqsErrQueueDoesNotExist { + t.Fatalf("error type: got %q want %q", got, sqsErrQueueDoesNotExist) + } + + // DeleteQueue on an unknown queue also returns NonExistentQueue. + status, _ = callSQS(t, node, sqsDeleteQueueTarget, map[string]any{ + "QueueUrl": url, + }) + if status != http.StatusBadRequest { + t.Fatalf("second delete: got %d want 400", status) + } +} + +func TestSQSServer_CatalogFIFOValidation(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + // FIFO name with FifoQueue=false is rejected. + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "bad.fifo", + "Attributes": map[string]string{"FifoQueue": "false"}, + }) + if status != http.StatusBadRequest { + t.Fatalf("mismatch name/FifoQueue: got %d want 400; body %v", status, out) + } + + // Non-FIFO name with FifoQueue=true is rejected. + status, out = callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "plain", + "Attributes": map[string]string{"FifoQueue": "true"}, + }) + if status != http.StatusBadRequest { + t.Fatalf("plain name + FifoQueue=true: got %d want 400; body %v", status, out) + } + + // Valid FIFO succeeds and the attribute is echoed back. + status, out = callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "events.fifo", + "Attributes": map[string]string{"FifoQueue": "true"}, + }) + if status != http.StatusOK { + t.Fatalf("FIFO create: %d %v", status, out) + } + url, _ := out["QueueUrl"].(string) + _, out = callSQS(t, node, sqsGetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "AttributeNames": []string{"FifoQueue"}, + }) + attrs, _ := out["Attributes"].(map[string]any) + if attrs["FifoQueue"] != "true" { + t.Fatalf("FIFO flag not persisted: %v", attrs) + } +} + +func TestSQSServer_CatalogKeyEncoding(t *testing.T) { + t.Parallel() + for _, name := range []string{"", "a", "hello world", "queue.fifo", strings.Repeat("x", 80)} { + encoded := encodeSQSSegment(name) + decoded, err := decodeSQSSegment(encoded) + if err != nil { + t.Fatalf("decode %q: %v", name, err) + } + if decoded != name { + t.Fatalf("round-trip %q -> %q -> %q", name, encoded, decoded) + } + } + + // queueNameFromMetaKey round-trips sqsQueueMetaKey. + name := "round.trip.fifo" + key := sqsQueueMetaKey(name) + got, ok := queueNameFromMetaKey(key) + if !ok || got != name { + t.Fatalf("queueNameFromMetaKey(sqsQueueMetaKey(%q)) = (%q, %v), want (%q, true)", name, got, ok, name) + } + + // Unknown prefixes are rejected. + if _, ok := queueNameFromMetaKey([]byte("random")); ok { + t.Fatal("queueNameFromMetaKey should reject non-catalog keys") + } +} diff --git a/adapter/sqs_keys.go b/adapter/sqs_keys.go new file mode 100644 index 000000000..f0f402665 --- /dev/null +++ b/adapter/sqs_keys.go @@ -0,0 +1,58 @@ +package adapter + +import ( + "encoding/base64" + "strings" + + "github.com/cockroachdb/errors" +) + +// SQS keyspace prefixes. Kept in sync with the naming in +// docs/design/2026_04_24_proposed_sqs_compatible_adapter.md. +const ( + // SqsQueueMetaPrefix prefixes queue-metadata records. + SqsQueueMetaPrefix = "!sqs|queue|meta|" + // SqsQueueGenPrefix prefixes the per-queue monotonic generation counter. + // Bumped on DeleteQueue / PurgeQueue so keys from an older incarnation of + // the same queue name cannot leak into a newly created queue. + SqsQueueGenPrefix = "!sqs|queue|gen|" +) + +func sqsQueueMetaKey(queueName string) []byte { + return []byte(SqsQueueMetaPrefix + encodeSQSSegment(queueName)) +} + +func sqsQueueGenKey(queueName string) []byte { + return []byte(SqsQueueGenPrefix + encodeSQSSegment(queueName)) +} + +// encodeSQSSegment emits a printable, byte-ordered-unique representation of a +// queue name. Base64 raw URL encoding matches the encoding the DynamoDB +// adapter uses for table segments (see encodeDynamoSegment) so that operators +// reading raw keys from Pebble see the same shape across both adapters. +func encodeSQSSegment(v string) string { + return base64.RawURLEncoding.EncodeToString([]byte(v)) +} + +func decodeSQSSegment(v string) (string, error) { + b, err := base64.RawURLEncoding.DecodeString(v) + if err != nil { + return "", errors.WithStack(err) + } + return string(b), nil +} + +// queueNameFromMetaKey pulls the queue name out of a !sqs|queue|meta|<seg> +// key. The second return reports success so callers can skip keys that were +// not written by this adapter. +func queueNameFromMetaKey(key []byte) (string, bool) { + enc, ok := strings.CutPrefix(string(key), SqsQueueMetaPrefix) + if !ok || enc == "" { + return "", false + } + name, err := decodeSQSSegment(enc) + if err != nil { + return "", false + } + return name, true +} diff --git a/adapter/sqs_test.go b/adapter/sqs_test.go index 078d4f05f..9cf122171 100644 --- a/adapter/sqs_test.go +++ b/adapter/sqs_test.go @@ -124,13 +124,12 @@ func TestSQSServer_KnownTargetsReturnNotImplemented(t *testing.T) { t.Parallel() base := startTestSQSServer(t) + // Targets that still return NotImplemented. The catalog targets + // (CreateQueue/DeleteQueue/ListQueues/GetQueueUrl/GetQueueAttributes/ + // SetQueueAttributes) are covered by TestSQSServer_Catalog* against a + // real single-node cluster because they require a coordinator to + // dispatch transactions. targets := []string{ - sqsCreateQueueTarget, - sqsDeleteQueueTarget, - sqsListQueuesTarget, - sqsGetQueueUrlTarget, - sqsGetQueueAttributesTarget, - sqsSetQueueAttributesTarget, sqsPurgeQueueTarget, sqsSendMessageTarget, sqsSendMessageBatchTarget, From 61fa917675b5d30126c580331f96309490da1084 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 15:20:42 +0900 Subject: [PATCH 12/26] feat(sqs): SigV4 verifier shared with S3, wire static credentials Refactor S3's SigV4 primitives into adapter/sigv4.go and implement the SQS JSON-protocol verifier on top of them. - adapter/sigv4.go: service-agnostic SigV4 helpers (parseSigV4AuthorizationHeader, buildSigV4AuthorizationHeader, buildSigV4AuthorizationHeaderRestricted, extractSigV4Signature / SignedHeaders / Auth fields). The *Restricted builder strips transport-added headers that are not listed in SignedHeaders, so re-signing a server-side request does not fail because Go's http client added Accept-Encoding after the client signed. - adapter/s3_auth.go: delegates parse/build/extract to the shared helpers. Error ordering and the presigned-URL path are unchanged to keep the existing S3 tests green. - adapter/sqs_auth.go: WithSQSRegion / WithSQSStaticCredentials options and the JSON-protocol verifier, split into validateSQSAuthScope / validateSQSAuthDate / drainAndHashSQSBody / verifySQSSignatureMatches so each pass is independently testable. When no credentials are configured the endpoint stays open (matching S3's scaffold-test-friendly behavior). - adapter/sqs.go: SQSServer gains region + staticCreds fields and calls authorizeSQSRequest after the leader-proxy check, before dispatching to the handler map. - adapter/sqs_auth_test.go: happy-path SigV4, missing Authorization, wrong-service scope, unknown access key, tampered body, clock skew, and the no-credentials open-endpoint case. All tests drive requests through the actual AWS SDK v4 signer so the adapter stays compatible with real SDK callers. - main_sigv4_creds.go: loadSigV4StaticCredentialsFile, the shared JSON credentials loader; main_s3.go.loadS3StaticCredentials now delegates to it. - main_sqs.go + main.go: --sqsRegion / --sqsCredentialsFile flags and per-runner fields, passed through to WithSQSRegion / WithSQSStaticCredentials. --- adapter/s3_auth.go | 85 ++------------- adapter/s3_test.go | 2 +- adapter/sigv4.go | 203 ++++++++++++++++++++++++++++++++++ adapter/sqs.go | 7 ++ adapter/sqs_auth.go | 178 ++++++++++++++++++++++++++++++ adapter/sqs_auth_test.go | 230 +++++++++++++++++++++++++++++++++++++++ main.go | 8 +- main_s3.go | 39 +------ main_sigv4_creds.go | 58 ++++++++++ main_sqs.go | 9 ++ 10 files changed, 703 insertions(+), 116 deletions(-) create mode 100644 adapter/sigv4.go create mode 100644 adapter/sqs_auth.go create mode 100644 adapter/sqs_auth_test.go create mode 100644 main_sigv4_creds.go diff --git a/adapter/s3_auth.go b/adapter/s3_auth.go index 1df6070de..15a61bbd1 100644 --- a/adapter/s3_auth.go +++ b/adapter/s3_auth.go @@ -1,7 +1,6 @@ package adapter import ( - "context" "crypto/subtle" "net/http" "net/url" @@ -12,11 +11,12 @@ import ( "github.com/aws/aws-sdk-go-v2/aws" v4 "github.com/aws/aws-sdk-go-v2/aws/signer/v4" "github.com/bootjp/elastickv/kv" - "github.com/cockroachdb/errors" ) const ( - s3SigV4Algorithm = "AWS4-HMAC-SHA256" + // s3SigV4Algorithm is an alias of sigv4Algorithm kept for call-site + // readability in the S3 adapter. + s3SigV4Algorithm = sigv4Algorithm // s3UnsignedPayload / s3Streaming* are sentinel values that AWS SDKs may // place in X-Amz-Content-Sha256. None of them are a literal SHA-256 hash // of the body, so the PUT pipeline must skip hash validation when it sees @@ -27,8 +27,8 @@ const ( s3StreamingSignedPayload = "STREAMING-AWS4-HMAC-SHA256-PAYLOAD" s3StreamingSignedPayloadTrailer = "STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER" s3EmptyPayloadHash = "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855" - s3DateHeaderFormat = "20060102T150405Z" - s3RequestTimeMaxSkew = 15 * time.Minute + s3DateHeaderFormat = sigv4DateHeaderFormat + s3RequestTimeMaxSkew = sigv4RequestTimeMaxSkew ) // isS3PayloadMarker reports whether the given X-Amz-Content-Sha256 value is a @@ -79,14 +79,6 @@ type s3AuthError struct { Message string } -type s3AuthorizationHeader struct { - Algorithm string - AccessKeyID string - Date string - Region string - Service string -} - func WithS3Region(region string) S3ServerOption { return func(server *S3Server) { if server == nil || strings.TrimSpace(region) == "" { @@ -151,7 +143,7 @@ func (s *S3Server) authorizeRequest(r *http.Request) *s3AuthError { } } - parsed, err := parseS3AuthorizationHeader(authHeader) + parsed, err := parseSigV4AuthorizationHeader(authHeader) if err != nil { return &s3AuthError{ Status: http.StatusForbidden, @@ -224,7 +216,7 @@ func (s *S3Server) authorizeRequest(r *http.Request) *s3AuthError { } } - expectedAuth, err := buildS3AuthorizationHeader(r, parsed.AccessKeyID, secretAccessKey, s.effectiveRegion(), signingTime, payloadHash) + expectedAuth, err := buildSigV4AuthorizationHeader(r, parsed.AccessKeyID, secretAccessKey, "s3", s.effectiveRegion(), signingTime, payloadHash) if err != nil { return &s3AuthError{ Status: http.StatusForbidden, @@ -235,8 +227,8 @@ func (s *S3Server) authorizeRequest(r *http.Request) *s3AuthError { // Compare only the Signature component to avoid false rejections caused by // equivalent Authorization headers that differ in whitespace or parameter // ordering (but carry the same cryptographic signature). - gotSig := extractS3Signature(authHeader) - expectedSig := extractS3Signature(expectedAuth) + gotSig := extractSigV4Signature(authHeader) + expectedSig := extractSigV4Signature(expectedAuth) if gotSig == "" || subtle.ConstantTimeCompare([]byte(gotSig), []byte(expectedSig)) != 1 { return &s3AuthError{ Status: http.StatusForbidden, @@ -247,65 +239,6 @@ func (s *S3Server) authorizeRequest(r *http.Request) *s3AuthError { return nil } -func buildS3AuthorizationHeader(r *http.Request, accessKeyID string, secretAccessKey string, region string, signingTime time.Time, payloadHash string) (string, error) { - if r == nil { - return "", errors.New("request is required") - } - clone := r.Clone(context.Background()) - clone.Host = r.Host - clone.Header = clone.Header.Clone() - clone.Header.Del("Authorization") - - signer := v4.NewSigner(func(opts *v4.SignerOptions) { - opts.DisableURIPathEscaping = true - }) - creds := aws.Credentials{ - AccessKeyID: accessKeyID, - SecretAccessKey: secretAccessKey, - Source: "elastickv-s3", - } - if err := signer.SignHTTP(context.Background(), creds, clone, payloadHash, "s3", region, signingTime.UTC()); err != nil { - return "", errors.WithStack(err) - } - return strings.TrimSpace(clone.Header.Get("Authorization")), nil -} - -//nolint:cyclop // AWS authorization parsing is branchy because malformed scopes must be rejected precisely. -func parseS3AuthorizationHeader(raw string) (s3AuthorizationHeader, error) { - raw = strings.TrimSpace(raw) - if raw == "" { - return s3AuthorizationHeader{}, errors.New("authorization header is required") - } - algorithm, rest, ok := strings.Cut(raw, " ") - if !ok { - return s3AuthorizationHeader{}, errors.New("authorization header is malformed") - } - out := s3AuthorizationHeader{Algorithm: strings.TrimSpace(algorithm)} - params := strings.Split(rest, ",") - for _, param := range params { - key, value, ok := strings.Cut(strings.TrimSpace(param), "=") - if !ok { - continue - } - if key != "Credential" { - continue - } - scope := strings.Split(value, "/") - if len(scope) != 5 || scope[4] != "aws4_request" { - return s3AuthorizationHeader{}, errors.New("credential scope is malformed") - } - out.AccessKeyID = scope[0] - out.Date = scope[1] - out.Region = scope[2] - out.Service = scope[3] - break - } - if out.AccessKeyID == "" || out.Date == "" || out.Region == "" || out.Service == "" { - return s3AuthorizationHeader{}, errors.New("credential scope is required") - } - return out, nil -} - func normalizeS3PayloadHash(raw string) string { return strings.TrimSpace(raw) } diff --git a/adapter/s3_test.go b/adapter/s3_test.go index f3f27d2be..457a540bd 100644 --- a/adapter/s3_test.go +++ b/adapter/s3_test.go @@ -874,7 +874,7 @@ func newSignedS3Request( signingTime, ) require.NoError(t, err) - expectedAuth, err := buildS3AuthorizationHeader(req, testS3AccessKey, testS3SecretKey, testS3Region, signingTime, payloadHash) + expectedAuth, err := buildSigV4AuthorizationHeader(req, testS3AccessKey, testS3SecretKey, "s3", testS3Region, signingTime, payloadHash) require.NoError(t, err) require.Equal(t, strings.TrimSpace(req.Header.Get("Authorization")), expectedAuth) return req diff --git a/adapter/sigv4.go b/adapter/sigv4.go new file mode 100644 index 000000000..1f26ff103 --- /dev/null +++ b/adapter/sigv4.go @@ -0,0 +1,203 @@ +package adapter + +import ( + "context" + "net/http" + "strings" + "time" + + "github.com/aws/aws-sdk-go-v2/aws" + v4 "github.com/aws/aws-sdk-go-v2/aws/signer/v4" + "github.com/cockroachdb/errors" +) + +// Service-agnostic SigV4 primitives shared by the S3 and SQS adapters. +// The per-adapter files wrap these with their own error ordering and +// service-specific rules (payload hashing for S3, JSON body hashing for +// SQS, etc.). + +const ( + sigv4Algorithm = "AWS4-HMAC-SHA256" + sigv4DateHeaderFormat = "20060102T150405Z" + sigv4ScopeDateFormat = "20060102" + sigv4RequestTimeMaxSkew = 15 * time.Minute + sigv4ScopeTerminator = "aws4_request" + sigv4ScopeParts = 5 +) + +// sigv4AuthorizationHeader captures the Credential scope fields extracted +// from a SigV4 Authorization header ("AWS4-HMAC-SHA256 Credential=<akid>/ +// <date>/<region>/<service>/aws4_request, SignedHeaders=..., Signature=..."). +type sigv4AuthorizationHeader struct { + Algorithm string + AccessKeyID string + Date string + Region string + Service string +} + +// parseSigV4AuthorizationHeader decodes the Credential scope from an +// Authorization header. Other fields (SignedHeaders, Signature) are not +// parsed here because the adapter-level verifier rebuilds and compares the +// full signature through the AWS SDK v4 signer. +func parseSigV4AuthorizationHeader(raw string) (sigv4AuthorizationHeader, error) { + raw = strings.TrimSpace(raw) + if raw == "" { + return sigv4AuthorizationHeader{}, errors.New("authorization header is required") + } + algorithm, rest, ok := strings.Cut(raw, " ") + if !ok { + return sigv4AuthorizationHeader{}, errors.New("authorization header is malformed") + } + credentialValue, ok := findSigV4Param(rest, "Credential") + if !ok { + return sigv4AuthorizationHeader{}, errors.New("credential scope is required") + } + out, err := parseSigV4CredentialScope(credentialValue) + if err != nil { + return sigv4AuthorizationHeader{}, err + } + out.Algorithm = strings.TrimSpace(algorithm) + return out, nil +} + +// findSigV4Param returns the value of the named parameter from the +// "k=v, k=v, ..." tail of an Authorization header. +func findSigV4Param(params, name string) (string, bool) { + for _, param := range strings.Split(params, ",") { + key, value, ok := strings.Cut(strings.TrimSpace(param), "=") + if ok && key == name { + return strings.TrimSpace(value), true + } + } + return "", false +} + +// parseSigV4CredentialScope validates an +// <akid>/<date>/<region>/<service>/aws4_request scope string. +func parseSigV4CredentialScope(value string) (sigv4AuthorizationHeader, error) { + scope := strings.Split(value, "/") + if len(scope) != sigv4ScopeParts || scope[4] != sigv4ScopeTerminator { + return sigv4AuthorizationHeader{}, errors.New("credential scope is malformed") + } + out := sigv4AuthorizationHeader{ + AccessKeyID: scope[0], + Date: scope[1], + Region: scope[2], + Service: scope[3], + } + if out.AccessKeyID == "" || out.Date == "" || out.Region == "" || out.Service == "" { + return sigv4AuthorizationHeader{}, errors.New("credential scope is required") + } + return out, nil +} + +// buildSigV4AuthorizationHeader re-signs r with the given credentials and +// returns the Authorization header the server expects. Used by adapter +// verifiers to compare against the client-supplied Authorization. +func buildSigV4AuthorizationHeader( + r *http.Request, + accessKeyID, secretAccessKey, service, region string, + signingTime time.Time, + payloadHash string, +) (string, error) { + if r == nil { + return "", errors.New("request is required") + } + clone := r.Clone(context.Background()) + clone.Host = r.Host + clone.Header = clone.Header.Clone() + clone.Header.Del("Authorization") + + signer := v4.NewSigner(func(opts *v4.SignerOptions) { + opts.DisableURIPathEscaping = true + }) + creds := aws.Credentials{ + AccessKeyID: accessKeyID, + SecretAccessKey: secretAccessKey, + Source: "elastickv-" + service, + } + if err := signer.SignHTTP(context.Background(), creds, clone, payloadHash, service, region, signingTime.UTC()); err != nil { + return "", errors.WithStack(err) + } + return strings.TrimSpace(clone.Header.Get("Authorization")), nil +} + +// extractSigV4Signature returns the hex Signature= value from an +// Authorization header, or "" if missing. +func extractSigV4Signature(auth string) string { + return extractSigV4AuthField(auth, "Signature") +} + +// extractSigV4SignedHeaders returns the semicolon-separated SignedHeaders +// list as a []string, or nil if missing. Header names are lowercased. +func extractSigV4SignedHeaders(auth string) []string { + raw := extractSigV4AuthField(auth, "SignedHeaders") + if raw == "" { + return nil + } + parts := strings.Split(raw, ";") + out := make([]string, 0, len(parts)) + for _, p := range parts { + p = strings.TrimSpace(p) + if p == "" { + continue + } + out = append(out, strings.ToLower(p)) + } + return out +} + +func extractSigV4AuthField(auth, field string) string { + _, params, ok := strings.Cut(auth, " ") + if !ok { + return "" + } + v, _ := findSigV4Param(params, field) + return v +} + +// buildSigV4AuthorizationHeaderRestricted is a variant of +// buildSigV4AuthorizationHeader that strips any header not listed in +// signedHeaders before re-signing, so the server's re-computation matches +// the client's canonical request even when Go's transport added headers +// (Accept-Encoding, etc.) after the original signature. +func buildSigV4AuthorizationHeaderRestricted( + r *http.Request, + accessKeyID, secretAccessKey, service, region string, + signingTime time.Time, + payloadHash string, + signedHeaders []string, +) (string, error) { + if r == nil { + return "", errors.New("request is required") + } + clone := r.Clone(context.Background()) + clone.Host = r.Host + clone.Header = clone.Header.Clone() + clone.Header.Del("Authorization") + if len(signedHeaders) > 0 { + allowed := make(map[string]struct{}, len(signedHeaders)) + for _, h := range signedHeaders { + allowed[strings.ToLower(strings.TrimSpace(h))] = struct{}{} + } + for name := range clone.Header { + if _, ok := allowed[strings.ToLower(name)]; !ok { + clone.Header.Del(name) + } + } + } + + signer := v4.NewSigner(func(opts *v4.SignerOptions) { + opts.DisableURIPathEscaping = true + }) + creds := aws.Credentials{ + AccessKeyID: accessKeyID, + SecretAccessKey: secretAccessKey, + Source: "elastickv-" + service, + } + if err := signer.SignHTTP(context.Background(), creds, clone, payloadHash, service, region, signingTime.UTC()); err != nil { + return "", errors.WithStack(err) + } + return strings.TrimSpace(clone.Header.Get("Authorization")), nil +} diff --git a/adapter/sqs.go b/adapter/sqs.go index d0ec3c150..6b30085bc 100644 --- a/adapter/sqs.go +++ b/adapter/sqs.go @@ -67,6 +67,8 @@ type SQSServer struct { httpServer *http.Server targetHandlers map[string]func(http.ResponseWriter, *http.Request) leaderSQS map[string]string + region string + staticCreds map[string]string } // WithSQSLeaderMap configures the Raft-address-to-SQS-address mapping used to @@ -144,6 +146,11 @@ func (s *SQSServer) handle(w http.ResponseWriter, r *http.Request) { return } + if authErr := s.authorizeSQSRequest(r); authErr != nil { + writeSQSError(w, authErr.Status, authErr.Code, authErr.Message) + return + } + target := r.Header.Get("X-Amz-Target") handler, ok := s.targetHandlers[target] if !ok { diff --git a/adapter/sqs_auth.go b/adapter/sqs_auth.go new file mode 100644 index 000000000..c8f49ea77 --- /dev/null +++ b/adapter/sqs_auth.go @@ -0,0 +1,178 @@ +package adapter + +import ( + "bytes" + "crypto/sha256" + "crypto/subtle" + "encoding/hex" + "io" + "net/http" + "strings" + "time" +) + +// Service name used in the SigV4 credential scope. +const sqsSigV4Service = "sqs" + +// sqsAuthError is the SQS-flavored counterpart to s3AuthError. The outer +// handler turns these into the AWS JSON-1.0 error envelope via +// writeSQSError. +type sqsAuthError struct { + Status int + Code string + Message string +} + +// WithSQSRegion configures the signing region the adapter expects inside +// the Credential scope. Empty values retain the previous setting. +func WithSQSRegion(region string) SQSServerOption { + return func(s *SQSServer) { + if s == nil || strings.TrimSpace(region) == "" { + return + } + s.region = strings.TrimSpace(region) + } +} + +// WithSQSStaticCredentials supplies the access-key → secret map the +// adapter will accept. Passing an empty map disables authorization +// entirely (open endpoint), matching the S3 adapter's behavior for +// unit-test friendliness. +func WithSQSStaticCredentials(creds map[string]string) SQSServerOption { + return func(s *SQSServer) { + if s == nil || len(creds) == 0 { + return + } + s.staticCreds = make(map[string]string, len(creds)) + for accessKeyID, secretAccessKey := range creds { + accessKeyID = strings.TrimSpace(accessKeyID) + secretAccessKey = strings.TrimSpace(secretAccessKey) + if accessKeyID == "" || secretAccessKey == "" { + continue + } + s.staticCreds[accessKeyID] = secretAccessKey + } + if len(s.staticCreds) == 0 { + s.staticCreds = nil + } + } +} + +func (s *SQSServer) effectiveRegion() string { + if s == nil || strings.TrimSpace(s.region) == "" { + return "us-east-1" + } + return s.region +} + +// authorizeSQSRequest verifies SigV4 on an SQS JSON-protocol request. The +// flow mirrors S3's header-based path (no presigned URLs or streaming +// payloads) but restricts the credential scope service to "sqs". +// +// Returns nil when authorization is either unconfigured (open endpoint) or +// the signature matches. Returns *sqsAuthError otherwise, suitable for +// writeSQSError. +// +// The function must consume and replace r.Body so that downstream handlers +// can still parse the JSON payload. +func (s *SQSServer) authorizeSQSRequest(r *http.Request) *sqsAuthError { + if s == nil || r == nil || len(s.staticCreds) == 0 { + return nil + } + authHeader := strings.TrimSpace(r.Header.Get("Authorization")) + if authHeader == "" { + return &sqsAuthError{Status: http.StatusForbidden, Code: "MissingAuthenticationToken", Message: "missing Authorization header"} + } + parsed, authErr := s.validateSQSAuthScope(authHeader) + if authErr != nil { + return authErr + } + secretAccessKey, ok := s.staticCreds[parsed.AccessKeyID] + if !ok { + return &sqsAuthError{Status: http.StatusForbidden, Code: "InvalidClientTokenId", Message: "unknown access key"} + } + signingTime, authErr := validateSQSAuthDate(r, parsed) + if authErr != nil { + return authErr + } + payloadHash, authErr := drainAndHashSQSBody(r) + if authErr != nil { + return authErr + } + return verifySQSSignatureMatches(r, authHeader, parsed.AccessKeyID, secretAccessKey, s.effectiveRegion(), signingTime, payloadHash) +} + +// validateSQSAuthScope parses the Credential scope and rejects scopes +// whose algorithm/service/region do not match this adapter. +func (s *SQSServer) validateSQSAuthScope(authHeader string) (sigv4AuthorizationHeader, *sqsAuthError) { + parsed, err := parseSigV4AuthorizationHeader(authHeader) + if err != nil { + return sigv4AuthorizationHeader{}, &sqsAuthError{Status: http.StatusForbidden, Code: "IncompleteSignature", Message: "invalid Authorization header"} + } + if parsed.Algorithm != sigv4Algorithm { + return sigv4AuthorizationHeader{}, &sqsAuthError{Status: http.StatusForbidden, Code: "IncompleteSignature", Message: "unsupported signature algorithm"} + } + if parsed.Service != sqsSigV4Service { + return sigv4AuthorizationHeader{}, &sqsAuthError{Status: http.StatusForbidden, Code: "SignatureDoesNotMatch", Message: "credential scope service must be sqs"} + } + if parsed.Region != s.effectiveRegion() { + return sigv4AuthorizationHeader{}, &sqsAuthError{Status: http.StatusForbidden, Code: "SignatureDoesNotMatch", Message: "credential scope region does not match server region"} + } + return parsed, nil +} + +// validateSQSAuthDate parses X-Amz-Date and verifies it matches the +// Credential scope date and is within the allowed clock-skew window. +func validateSQSAuthDate(r *http.Request, parsed sigv4AuthorizationHeader) (time.Time, *sqsAuthError) { + amzDate := strings.TrimSpace(r.Header.Get("X-Amz-Date")) + if amzDate == "" { + return time.Time{}, &sqsAuthError{Status: http.StatusForbidden, Code: "MissingAuthenticationToken", Message: "missing x-amz-date header"} + } + signingTime, err := time.Parse(sigv4DateHeaderFormat, amzDate) + if err != nil { + return time.Time{}, &sqsAuthError{Status: http.StatusForbidden, Code: "IncompleteSignature", Message: "invalid x-amz-date header"} + } + if parsed.Date != signingTime.UTC().Format(sigv4ScopeDateFormat) { + return time.Time{}, &sqsAuthError{Status: http.StatusForbidden, Code: "SignatureDoesNotMatch", Message: "credential scope date does not match x-amz-date"} + } + skew := time.Now().UTC().Sub(signingTime.UTC()) + if skew < 0 { + skew = -skew + } + if skew > sigv4RequestTimeMaxSkew { + return time.Time{}, &sqsAuthError{Status: http.StatusForbidden, Code: "RequestTimeTooSkewed", Message: "The difference between the request time and the server's time is too large"} + } + return signingTime, nil +} + +// drainAndHashSQSBody reads the request body so the signer reproduces the +// client's payload hash, then replaces r.Body so handler code can re-read +// it afterwards. +func drainAndHashSQSBody(r *http.Request) (string, *sqsAuthError) { + body, err := io.ReadAll(http.MaxBytesReader(nil, r.Body, sqsMaxRequestBodyBytes)) + if err != nil { + return "", &sqsAuthError{Status: http.StatusForbidden, Code: "IncompleteSignature", Message: "failed to read request body for signature verification"} + } + r.Body = io.NopCloser(bytes.NewReader(body)) + sum := sha256.Sum256(body) + return hex.EncodeToString(sum[:]), nil +} + +// verifySQSSignatureMatches rebuilds the expected Authorization header and +// compares its hex signature to the one the client sent. +// +// The restricted builder is used so Go's transport-added headers +// (Accept-Encoding etc.) do not leak into the canonical request. +func verifySQSSignatureMatches(r *http.Request, authHeader, accessKeyID, secretAccessKey, region string, signingTime time.Time, payloadHash string) *sqsAuthError { + signedHeaders := extractSigV4SignedHeaders(authHeader) + expectedAuth, err := buildSigV4AuthorizationHeaderRestricted(r, accessKeyID, secretAccessKey, sqsSigV4Service, region, signingTime, payloadHash, signedHeaders) + if err != nil { + return &sqsAuthError{Status: http.StatusForbidden, Code: "SignatureDoesNotMatch", Message: "failed to verify request signature"} + } + gotSig := extractSigV4Signature(authHeader) + expectedSig := extractSigV4Signature(expectedAuth) + if gotSig == "" || subtle.ConstantTimeCompare([]byte(gotSig), []byte(expectedSig)) != 1 { + return &sqsAuthError{Status: http.StatusForbidden, Code: "SignatureDoesNotMatch", Message: "request signature does not match"} + } + return nil +} diff --git a/adapter/sqs_auth_test.go b/adapter/sqs_auth_test.go new file mode 100644 index 000000000..a8f815513 --- /dev/null +++ b/adapter/sqs_auth_test.go @@ -0,0 +1,230 @@ +package adapter + +import ( + "bytes" + "context" + "crypto/sha256" + "encoding/hex" + "io" + "net" + "net/http" + "testing" + "time" + + "github.com/aws/aws-sdk-go-v2/aws" + v4 "github.com/aws/aws-sdk-go-v2/aws/signer/v4" + json "github.com/goccy/go-json" +) + +const ( + testSQSAccessKey = "AKIATESTSQSAAAAAAAAA" + testSQSSecretKey = "test-secret-key/xxxxxxxxxxxxxxxxxxxxxxxx" + testSQSRegion = "us-west-2" +) + +// startAuthedSQSServer brings up an SQSServer with static credentials so +// we can exercise the full SigV4 verifier. No coordinator is wired in, so +// the server only executes the auth + health paths (handler requests land +// on unknown targets, returning 400 InvalidAction after auth passes). +func startAuthedSQSServer(t *testing.T) string { + t.Helper() + lc := &net.ListenConfig{} + listener, err := lc.Listen(context.Background(), "tcp", "127.0.0.1:0") + if err != nil { + t.Fatalf("listen: %v", err) + } + srv := NewSQSServer(listener, nil, nil, + WithSQSRegion(testSQSRegion), + WithSQSStaticCredentials(map[string]string{testSQSAccessKey: testSQSSecretKey}), + ) + go func() { + _ = srv.Run() + }() + t.Cleanup(func() { + srv.Stop() + }) + return "http://" + listener.Addr().String() +} + +// signSQSRequest signs an SQS request with the AWS SDK v4 signer against +// the supplied credentials, exactly as a real AWS SDK client would. +func signSQSRequest(t *testing.T, base, target string, body []byte, signingTime time.Time, accessKey, secretKey string) *http.Request { + region := testSQSRegion + t.Helper() + req, err := http.NewRequestWithContext(context.Background(), http.MethodPost, base+"/", bytes.NewReader(body)) + if err != nil { + t.Fatalf("request: %v", err) + } + req.Header.Set("Content-Type", sqsContentTypeJSON) + req.Header.Set("X-Amz-Target", target) + sum := sha256.Sum256(body) + payloadHash := hex.EncodeToString(sum[:]) + + signer := v4.NewSigner(func(opts *v4.SignerOptions) { + opts.DisableURIPathEscaping = true + }) + if err := signer.SignHTTP(context.Background(), aws.Credentials{ + AccessKeyID: accessKey, + SecretAccessKey: secretKey, + Source: "test", + }, req, payloadHash, sqsSigV4Service, region, signingTime.UTC()); err != nil { + t.Fatalf("sign: %v", err) + } + return req +} + +func doReq(t *testing.T, req *http.Request) (int, map[string]string) { + t.Helper() + resp, err := http.DefaultClient.Do(req) + if err != nil { + t.Fatalf("do: %v", err) + } + defer resp.Body.Close() + raw, _ := io.ReadAll(resp.Body) + out := map[string]string{} + if len(bytes.TrimSpace(raw)) > 0 { + _ = json.Unmarshal(raw, &out) + } + return resp.StatusCode, out +} + +func TestSQSAuth_ValidSignaturePassesAuth(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + body := []byte(`{}`) + req := signSQSRequest(t, base, "AmazonSQS.NoSuchOperation", body, time.Now().UTC(), + testSQSAccessKey, testSQSSecretKey) + + // Auth should pass; then the target is unknown, so the dispatcher + // returns InvalidAction — *not* SignatureDoesNotMatch. That precise + // ordering is what we care about: auth runs first, dispatch second. + status, body2 := doReq(t, req) + if status != http.StatusBadRequest { + t.Fatalf("status: got %d body %v", status, body2) + } + if body2["__type"] != sqsErrInvalidAction { + t.Fatalf("error type: got %q want %q", body2["__type"], sqsErrInvalidAction) + } +} + +func TestSQSAuth_MissingAuthorizationHeaderRejected(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + req, err := http.NewRequestWithContext(context.Background(), http.MethodPost, base+"/", bytes.NewReader([]byte("{}"))) + if err != nil { + t.Fatalf("request: %v", err) + } + req.Header.Set("Content-Type", sqsContentTypeJSON) + req.Header.Set("X-Amz-Target", sqsListQueuesTarget) + + status, body := doReq(t, req) + if status != http.StatusForbidden { + t.Fatalf("status: got %d body %v", status, body) + } + if body["__type"] != "MissingAuthenticationToken" { + t.Fatalf("error type: got %q", body["__type"]) + } +} + +func TestSQSAuth_WrongServiceScopeRejected(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + body := []byte(`{}`) + req, err := http.NewRequestWithContext(context.Background(), http.MethodPost, base+"/", bytes.NewReader(body)) + if err != nil { + t.Fatalf("request: %v", err) + } + req.Header.Set("Content-Type", sqsContentTypeJSON) + req.Header.Set("X-Amz-Target", sqsListQueuesTarget) + sum := sha256.Sum256(body) + payloadHash := hex.EncodeToString(sum[:]) + signer := v4.NewSigner() + // Sign with service "s3" instead of "sqs". + if err := signer.SignHTTP(context.Background(), aws.Credentials{ + AccessKeyID: testSQSAccessKey, + SecretAccessKey: testSQSSecretKey, + }, req, payloadHash, "s3", testSQSRegion, time.Now().UTC()); err != nil { + t.Fatalf("sign: %v", err) + } + + status, out := doReq(t, req) + if status != http.StatusForbidden { + t.Fatalf("status: got %d body %v", status, out) + } + if out["__type"] != "SignatureDoesNotMatch" { + t.Fatalf("error type: got %q", out["__type"]) + } +} + +func TestSQSAuth_UnknownAccessKeyRejected(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + req := signSQSRequest(t, base, sqsListQueuesTarget, []byte("{}"), time.Now().UTC(), + "AKIAUNKNOWNEEEEEEEEE", "wrong-secret-keyxxxxxxxxxxxxxxxxxxxxxxxx") + + status, out := doReq(t, req) + if status != http.StatusForbidden { + t.Fatalf("status: got %d body %v", status, out) + } + if out["__type"] != "InvalidClientTokenId" { + t.Fatalf("error type: got %q", out["__type"]) + } +} + +func TestSQSAuth_TamperedBodyRejected(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + // Sign the original body, then send a different one — the server must + // hash the body it actually receives and catch the mismatch. + req := signSQSRequest(t, base, sqsListQueuesTarget, []byte(`{"QueueNamePrefix":"a"}`), time.Now().UTC(), + testSQSAccessKey, testSQSSecretKey) + req.Body = io.NopCloser(bytes.NewReader([]byte(`{"QueueNamePrefix":"b"}`))) + req.ContentLength = int64(len(`{"QueueNamePrefix":"b"}`)) + + status, out := doReq(t, req) + if status != http.StatusForbidden { + t.Fatalf("status: got %d body %v", status, out) + } + if out["__type"] != "SignatureDoesNotMatch" { + t.Fatalf("error type: got %q body %v", out["__type"], out) + } +} + +func TestSQSAuth_ClockSkewRejected(t *testing.T) { + t.Parallel() + base := startAuthedSQSServer(t) + old := time.Now().UTC().Add(-30 * time.Minute) + req := signSQSRequest(t, base, sqsListQueuesTarget, []byte("{}"), old, + testSQSAccessKey, testSQSSecretKey) + + status, out := doReq(t, req) + if status != http.StatusForbidden { + t.Fatalf("status: got %d body %v", status, out) + } + if out["__type"] != "RequestTimeTooSkewed" { + t.Fatalf("error type: got %q body %v", out["__type"], out) + } +} + +func TestSQSAuth_NoCredentialsMeansOpenEndpoint(t *testing.T) { + t.Parallel() + // No creds configured → scaffold test server already exercises this + // path (no Authorization header, no 403). This test explicitly + // double-checks: the dispatch runs even without auth, landing on + // InvalidAction for an unknown target. + base := startTestSQSServer(t) + req, err := http.NewRequestWithContext(context.Background(), http.MethodPost, base+"/", bytes.NewReader([]byte("{}"))) + if err != nil { + t.Fatalf("request: %v", err) + } + req.Header.Set("Content-Type", sqsContentTypeJSON) + req.Header.Set("X-Amz-Target", "AmazonSQS.NoSuchOperation") + + status, out := doReq(t, req) + if status != http.StatusBadRequest { + t.Fatalf("status: got %d body %v", status, out) + } + if out["__type"] != sqsErrInvalidAction { + t.Fatalf("error type: got %q", out["__type"]) + } +} diff --git a/main.go b/main.go index a9bb6dbac..2e5e3c4ac 100644 --- a/main.go +++ b/main.go @@ -80,6 +80,8 @@ var ( s3CredsFile = flag.String("s3CredentialsFile", "", "Path to a JSON file containing static S3 credentials") s3PathStyleOnly = flag.Bool("s3PathStyleOnly", true, "Only accept path-style S3 requests") sqsAddr = flag.String("sqsAddress", "", "TCP host+port for SQS-compatible API; empty to disable") + sqsRegion = flag.String("sqsRegion", "us-east-1", "SQS signing region") + sqsCredsFile = flag.String("sqsCredentialsFile", "", "Path to a JSON file containing static SQS credentials") metricsAddr = flag.String("metricsAddress", "localhost:9090", "TCP host+port for Prometheus metrics") metricsToken = flag.String("metricsToken", "", "Bearer token for Prometheus metrics; required for non-loopback metricsAddress") pprofAddr = flag.String("pprofAddress", "localhost:6060", "TCP host+port for pprof debug endpoints; empty to disable") @@ -216,6 +218,8 @@ func run() error { s3PathStyleOnly: *s3PathStyleOnly, sqsAddress: *sqsAddr, leaderSQS: cfg.leaderSQS, + sqsRegion: *sqsRegion, + sqsCredsFile: *sqsCredsFile, metricsAddress: *metricsAddr, metricsToken: *metricsToken, pprofAddress: *pprofAddr, @@ -840,6 +844,8 @@ type runtimeServerRunner struct { s3PathStyleOnly bool sqsAddress string leaderSQS map[string]string + sqsRegion string + sqsCredsFile string metricsAddress string metricsToken string pprofAddress string @@ -872,7 +878,7 @@ func (r runtimeServerRunner) start() error { if err := startS3Server(r.ctx, r.lc, r.eg, r.s3Address, r.shardStore, r.coordinate, r.leaderS3, r.s3Region, r.s3CredsFile, r.s3PathStyleOnly, r.readTracker); err != nil { return waitErrgroupAfterStartupFailure(r.cancel, r.eg, err) } - if err := startSQSServer(r.ctx, r.lc, r.eg, r.sqsAddress, r.shardStore, r.coordinate, r.leaderSQS); err != nil { + if err := startSQSServer(r.ctx, r.lc, r.eg, r.sqsAddress, r.shardStore, r.coordinate, r.leaderSQS, r.sqsRegion, r.sqsCredsFile); err != nil { return waitErrgroupAfterStartupFailure(r.cancel, r.eg, err) } if err := startMetricsServer(r.ctx, r.lc, r.eg, r.metricsAddress, r.metricsToken, r.metricsRegistry.Handler()); err != nil { diff --git a/main_s3.go b/main_s3.go index 7fae6f16d..5de290c7f 100644 --- a/main_s3.go +++ b/main_s3.go @@ -2,10 +2,7 @@ package main import ( "context" - "encoding/json" - "fmt" "net" - "os" "strings" "github.com/bootjp/elastickv/adapter" @@ -14,15 +11,6 @@ import ( "golang.org/x/sync/errgroup" ) -type s3CredentialFile struct { - Credentials []s3CredentialEntry `json:"credentials"` -} - -type s3CredentialEntry struct { - AccessKeyID string `json:"access_key_id"` - SecretAccessKey string `json:"secret_access_key"` -} - func startS3Server( ctx context.Context, lc *net.ListenConfig, @@ -83,30 +71,5 @@ func startS3Server( } func loadS3StaticCredentials(path string) (map[string]string, error) { - path = strings.TrimSpace(path) - if path == "" { - return nil, nil - } - f, err := os.Open(path) - if err != nil { - return nil, errors.WithStack(err) - } - defer f.Close() - file := s3CredentialFile{} - if err := json.NewDecoder(f).Decode(&file); err != nil { - return nil, errors.WithStack(err) - } - out := make(map[string]string, len(file.Credentials)) - for _, cred := range file.Credentials { - accessKeyID := strings.TrimSpace(cred.AccessKeyID) - secretAccessKey := strings.TrimSpace(cred.SecretAccessKey) - if accessKeyID == "" || secretAccessKey == "" { - return nil, errors.New("s3 credentials file contains an empty access key or secret key") - } - if _, exists := out[accessKeyID]; exists { - return nil, errors.WithStack(fmt.Errorf("s3 credentials file contains duplicate access key ID: %q", accessKeyID)) - } - out[accessKeyID] = secretAccessKey - } - return out, nil + return loadSigV4StaticCredentialsFile(path, "s3") } diff --git a/main_sigv4_creds.go b/main_sigv4_creds.go new file mode 100644 index 000000000..9ab517bf7 --- /dev/null +++ b/main_sigv4_creds.go @@ -0,0 +1,58 @@ +package main + +import ( + "encoding/json" + "fmt" + "os" + "strings" + + "github.com/cockroachdb/errors" +) + +// sigV4CredentialsFile is the JSON schema both --s3CredentialsFile and +// --sqsCredentialsFile read. Sharing the schema means operators maintain one +// file per deployment regardless of which adapters are enabled. +type sigV4CredentialsFile struct { + Credentials []sigV4CredentialEntry `json:"credentials"` +} + +type sigV4CredentialEntry struct { + AccessKeyID string `json:"access_key_id"` + SecretAccessKey string `json:"secret_access_key"` +} + +// loadSigV4StaticCredentialsFile parses a credentials file and returns a +// map of access-key → secret suitable for WithS3StaticCredentials or +// WithSQSStaticCredentials. An empty path returns a nil map so the caller +// can leave authorization disabled. +// +// labelForErrors appears in error text ("s3 credentials file ...") so +// operators get context on which flag produced the problem. +func loadSigV4StaticCredentialsFile(path string, labelForErrors string) (map[string]string, error) { + path = strings.TrimSpace(path) + if path == "" { + return nil, nil + } + f, err := os.Open(path) + if err != nil { + return nil, errors.WithStack(err) + } + defer f.Close() + file := sigV4CredentialsFile{} + if err := json.NewDecoder(f).Decode(&file); err != nil { + return nil, errors.WithStack(err) + } + out := make(map[string]string, len(file.Credentials)) + for _, cred := range file.Credentials { + accessKeyID := strings.TrimSpace(cred.AccessKeyID) + secretAccessKey := strings.TrimSpace(cred.SecretAccessKey) + if accessKeyID == "" || secretAccessKey == "" { + return nil, errors.WithStack(fmt.Errorf("%s credentials file contains an empty access key or secret key", labelForErrors)) + } + if _, exists := out[accessKeyID]; exists { + return nil, errors.WithStack(fmt.Errorf("%s credentials file contains duplicate access key ID: %q", labelForErrors, accessKeyID)) + } + out[accessKeyID] = secretAccessKey + } + return out, nil +} diff --git a/main_sqs.go b/main_sqs.go index a6371e96e..55ca41684 100644 --- a/main_sqs.go +++ b/main_sqs.go @@ -19,6 +19,8 @@ func startSQSServer( shardStore *kv.ShardStore, coordinate kv.Coordinator, leaderSQS map[string]string, + region string, + credentialsFile string, ) error { sqsAddr = strings.TrimSpace(sqsAddr) if sqsAddr == "" { @@ -28,11 +30,18 @@ func startSQSServer( if err != nil { return errors.Wrapf(err, "failed to listen on %s", sqsAddr) } + staticCreds, err := loadSigV4StaticCredentialsFile(credentialsFile, "sqs") + if err != nil { + _ = sqsL.Close() + return err + } sqsServer := adapter.NewSQSServer( sqsL, shardStore, coordinate, adapter.WithSQSLeaderMap(leaderSQS), + adapter.WithSQSRegion(region), + adapter.WithSQSStaticCredentials(staticCreds), ) // Two-goroutine shutdown pattern mirrors startS3Server: one goroutine waits // on either ctx.Done() or Run completion to call Stop, the other runs the From 387a19c295418ebf9f768e70edf666c265b7f4bf Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 15:30:07 +0900 Subject: [PATCH 13/26] feat(sqs): SendMessage, ReceiveMessage, DeleteMessage, ChangeMessageVisibility Implement the core message path against the visibility-indexed keyspace described in docs/design/2026_04_24_proposed_sqs_compatible_adapter.md. Storage layout: - !sqs|msg|data|<queue-esc><gen-u64><msg-id-esc> - the full message record: body, MD5, attributes, send / available / visible timestamps, receive count, and the current rotating receipt token. - !sqs|msg|vis|<queue-esc><gen-u64><visible-at-u64><msg-id-esc> - a visibility index keyed by visible_at. ReceiveMessage scans the range [now=0, now+1) to find the next visible messages without any background sweeper. - Receipt handles are base64url(version || queue_gen || message_id || receipt_token) and carry everything Delete / ChangeVisibility need to locate the record and verify ownership. Handlers: - sendMessage: validates body size against MaximumMessageSize, resolves effective DelaySeconds, then writes the data record and the matching visibility entry in one OCC transaction. - receiveMessage: fences the scan with LeaseReadThrough so the snapshot scan stays local on the leader inside the lease window and falls back to LinearizableRead when the lease is cold. For each candidate it runs a single-message OCC transaction that deletes the old vis entry, inserts a new one at (now + visibility timeout), rotates the receipt token, and bumps ReceiveCount. Race losers are skipped so a batch returns whatever it could deliver. - deleteMessage: verifies receipt token on the data record, then atomically drops the data + vis keys. - changeMessageVisibility: swaps the vis entry to a new visible_at and updates the record; rejects messages whose visibility already expired with MessageNotInflight. Tests (adapter/sqs_messages_test.go) spin up a single-node cluster and cover: - Send -> Receive -> Delete happy path, with MD5 and MessageId checks. - Tampered receipt handle rejected with InvalidReceiptHandle. - MaxNumberOfMessages returns a partial batch across two receives. - DelaySeconds defers delivery until the delay elapses. - Visibility-timeout expiry re-delivers the same message with ReceiveCount=2. - ChangeMessageVisibility extends the in-flight window. - Receipt-handle codec round trips for several (queue_gen, message_id, token) combinations. Scaffold "returns NotImplemented" test updated to skip the newly implemented targets (Send/Receive/Delete/ChangeMessageVisibility). PurgeQueue, batch APIs, and tag APIs remain NotImplemented for now. --- adapter/sqs.go | 8 +- adapter/sqs_messages.go | 704 +++++++++++++++++++++++++++++++++++ adapter/sqs_messages_test.go | 388 +++++++++++++++++++ adapter/sqs_test.go | 14 +- 4 files changed, 1101 insertions(+), 13 deletions(-) create mode 100644 adapter/sqs_messages.go create mode 100644 adapter/sqs_messages_test.go diff --git a/adapter/sqs.go b/adapter/sqs.go index 6b30085bc..1a3a06fdf 100644 --- a/adapter/sqs.go +++ b/adapter/sqs.go @@ -97,12 +97,12 @@ func NewSQSServer(listen net.Listener, st store.MVCCStore, coordinate kv.Coordin sqsGetQueueAttributesTarget: s.getQueueAttributes, sqsSetQueueAttributesTarget: s.setQueueAttributes, sqsPurgeQueueTarget: s.notImplemented("PurgeQueue"), - sqsSendMessageTarget: s.notImplemented("SendMessage"), + sqsSendMessageTarget: s.sendMessage, sqsSendMessageBatchTarget: s.notImplemented("SendMessageBatch"), - sqsReceiveMessageTarget: s.notImplemented("ReceiveMessage"), - sqsDeleteMessageTarget: s.notImplemented("DeleteMessage"), + sqsReceiveMessageTarget: s.receiveMessage, + sqsDeleteMessageTarget: s.deleteMessage, sqsDeleteMessageBatchTarget: s.notImplemented("DeleteMessageBatch"), - sqsChangeMessageVisibilityTarget: s.notImplemented("ChangeMessageVisibility"), + sqsChangeMessageVisibilityTarget: s.changeMessageVisibility, sqsChangeMessageVisibilityBatchTgt: s.notImplemented("ChangeMessageVisibilityBatch"), sqsTagQueueTarget: s.notImplemented("TagQueue"), sqsUntagQueueTarget: s.notImplemented("UntagQueue"), diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go new file mode 100644 index 000000000..1c7682eb9 --- /dev/null +++ b/adapter/sqs_messages.go @@ -0,0 +1,704 @@ +package adapter + +import ( + "bytes" + "context" + "crypto/md5" //nolint:gosec // AWS SQS ETag specifies MD5; not used as a cryptographic primitive. + "crypto/rand" + "encoding/base64" + "encoding/binary" + "encoding/hex" + "net/http" + "strconv" + "strings" + "time" + + "github.com/bootjp/elastickv/kv" + "github.com/bootjp/elastickv/store" + "github.com/cockroachdb/errors" + json "github.com/goccy/go-json" +) + +// Message-keyspace prefixes. The data record holds the message body and +// state; the visibility index is a separate, visible_at-sorted key family +// so ReceiveMessage can find the next visible message with a single bounded +// prefix scan. +const ( + SqsMsgDataPrefix = "!sqs|msg|data|" + SqsMsgVisPrefix = "!sqs|msg|vis|" +) + +const ( + sqsMessageIDBytes = 16 + sqsReceiptTokenBytes = 16 + sqsReceiveDefaultMaxMessages = 1 + sqsReceiveHardMaxMessages = 10 + sqsReceiveScanOverfetchFactor = 2 + sqsChangeVisibilityMaxSeconds = sqsMaxVisibilityTimeoutSeconds + sqsVisScanPageLimit = 1024 + // Version byte prefixed to encoded receipt handles. Bumped when the + // on-wire handle format changes so old handles fail to decode loudly. + sqsReceiptHandleVersion = byte(0x01) + // Byte sizes used when pre-sizing key buffers. The exact value is not + // critical; it only avoids one append growth for typical queue/ID + // lengths. + sqsKeyCapSmall = 32 + sqsKeyCapLarge = 64 + // Conversion factors for SQS second-granularity inputs. + sqsMillisPerSecond = 1000 +) + +// AWS error codes specific to message operations. +const ( + sqsErrReceiptHandleInvalid = "ReceiptHandleIsInvalid" + sqsErrInvalidReceiptHandle = "InvalidReceiptHandle" + sqsErrMessageTooLong = "InvalidParameterValue" + sqsErrMessageNotInflight = "MessageNotInflight" +) + +// sqsMessageRecord mirrors !sqs|msg|data|... on disk. Visibility state +// (VisibleAtMillis, CurrentReceiptToken, ReceiveCount) lives here rather +// than in a side-record so a single OCC transaction can rotate it. +type sqsMessageRecord struct { + MessageID string `json:"message_id"` + Body []byte `json:"body"` + MD5OfBody string `json:"md5_of_body"` + MessageAttributes map[string]string `json:"message_attributes,omitempty"` + SenderID string `json:"sender_id,omitempty"` + SendTimestampMillis int64 `json:"send_timestamp_millis"` + AvailableAtMillis int64 `json:"available_at_millis"` + VisibleAtMillis int64 `json:"visible_at_millis"` + ReceiveCount int64 `json:"receive_count"` + FirstReceiveMillis int64 `json:"first_receive_millis,omitempty"` + CurrentReceiptToken []byte `json:"current_receipt_token"` + QueueGeneration uint64 `json:"queue_generation"` +} + +var storedSQSMsgPrefix = []byte{0x00, 'S', 'M', 0x01} + +func encodeSQSMessageRecord(m *sqsMessageRecord) ([]byte, error) { + body, err := json.Marshal(m) + if err != nil { + return nil, errors.WithStack(err) + } + out := make([]byte, 0, len(storedSQSMsgPrefix)+len(body)) + out = append(out, storedSQSMsgPrefix...) + out = append(out, body...) + return out, nil +} + +func decodeSQSMessageRecord(b []byte) (*sqsMessageRecord, error) { + if !bytes.HasPrefix(b, storedSQSMsgPrefix) { + return nil, errors.New("unrecognized sqs message format") + } + var m sqsMessageRecord + if err := json.Unmarshal(b[len(storedSQSMsgPrefix):], &m); err != nil { + return nil, errors.WithStack(err) + } + return &m, nil +} + +// ------------------------ key helpers ------------------------ + +func sqsMsgDataKey(queueName string, gen uint64, messageID string) []byte { + buf := make([]byte, 0, len(SqsMsgDataPrefix)+sqsKeyCapLarge) + buf = append(buf, SqsMsgDataPrefix...) + buf = append(buf, encodeSQSSegment(queueName)...) + buf = appendU64(buf, gen) + buf = append(buf, encodeSQSSegment(messageID)...) + return buf +} + +func sqsMsgVisKey(queueName string, gen uint64, visibleAtMillis int64, messageID string) []byte { + buf := make([]byte, 0, len(SqsMsgVisPrefix)+sqsKeyCapLarge) + buf = append(buf, SqsMsgVisPrefix...) + buf = append(buf, encodeSQSSegment(queueName)...) + buf = appendU64(buf, gen) + buf = appendU64(buf, uint64MaxZero(visibleAtMillis)) + buf = append(buf, encodeSQSSegment(messageID)...) + return buf +} + +func sqsMsgVisPrefixForQueue(queueName string, gen uint64) []byte { + buf := make([]byte, 0, len(SqsMsgVisPrefix)+sqsKeyCapSmall) + buf = append(buf, SqsMsgVisPrefix...) + buf = append(buf, encodeSQSSegment(queueName)...) + buf = appendU64(buf, gen) + return buf +} + +// uint64MaxZero clamps negative int64 (which never happens for wall-clock +// timestamps but would silently overflow under uint64() cast) to zero. +func uint64MaxZero(v int64) uint64 { + if v < 0 { + return 0 + } + return uint64(v) +} + +func sqsMsgVisScanBounds(queueName string, gen uint64, maxVisibleAtMillis int64) (start, end []byte) { + prefix := sqsMsgVisPrefixForQueue(queueName, gen) + start = append(bytes.Clone(prefix), zeroU64()...) + upper := uint64MaxZero(maxVisibleAtMillis) + if upper < ^uint64(0) { + upper++ + } + end = append(bytes.Clone(prefix), encodedU64(upper)...) + return start, end +} + +func appendU64(dst []byte, v uint64) []byte { + var buf [8]byte + binary.BigEndian.PutUint64(buf[:], v) + return append(dst, buf[:]...) +} + +func encodedU64(v uint64) []byte { + var buf [8]byte + binary.BigEndian.PutUint64(buf[:], v) + return buf[:] +} + +func zeroU64() []byte { + var buf [8]byte + return buf[:] +} + +// ------------------------ message id + receipt handle ------------------------ + +func newMessageIDHex() (string, error) { + var buf [sqsMessageIDBytes]byte + if _, err := rand.Read(buf[:]); err != nil { + return "", errors.WithStack(err) + } + return hex.EncodeToString(buf[:]), nil +} + +func newReceiptToken() ([]byte, error) { + buf := make([]byte, sqsReceiptTokenBytes) + if _, err := rand.Read(buf); err != nil { + return nil, errors.WithStack(err) + } + return buf, nil +} + +// encodeReceiptHandle packs (queue_gen, message_id, receipt_token) into a +// single opaque blob. Format: +// +// [ 0 ] byte version = 0x01 +// [ 1..9 ] uint64 queue_gen (BE) +// [ 9..25 ] 16 bytes message_id (raw bytes from hex decode) +// [ 25..41 ] 16 bytes receipt_token +// +// The result is base64-urlsafe (no padding) so it passes through JSON and +// HTTP query parameters untouched. +func encodeReceiptHandle(queueGen uint64, messageIDHex string, receiptToken []byte) (string, error) { + if len(receiptToken) != sqsReceiptTokenBytes { + return "", errors.New("receipt token has wrong length") + } + idBytes, err := hex.DecodeString(messageIDHex) + if err != nil || len(idBytes) != sqsMessageIDBytes { + return "", errors.New("message id has wrong format") + } + buf := make([]byte, 0, 1+8+sqsMessageIDBytes+sqsReceiptTokenBytes) + buf = append(buf, sqsReceiptHandleVersion) + buf = appendU64(buf, queueGen) + buf = append(buf, idBytes...) + buf = append(buf, receiptToken...) + return base64.RawURLEncoding.EncodeToString(buf), nil +} + +type decodedReceiptHandle struct { + QueueGeneration uint64 + MessageIDHex string + ReceiptToken []byte +} + +func decodeReceiptHandle(raw string) (*decodedReceiptHandle, error) { + b, err := base64.RawURLEncoding.DecodeString(raw) + if err != nil { + return nil, errors.WithStack(err) + } + want := 1 + 8 + sqsMessageIDBytes + sqsReceiptTokenBytes + if len(b) != want || b[0] != sqsReceiptHandleVersion { + return nil, errors.New("receipt handle length or version mismatch") + } + out := &decodedReceiptHandle{ + QueueGeneration: binary.BigEndian.Uint64(b[1:9]), + MessageIDHex: hex.EncodeToString(b[9 : 9+sqsMessageIDBytes]), + ReceiptToken: bytes.Clone(b[9+sqsMessageIDBytes:]), + } + return out, nil +} + +// ------------------------ input decoding ------------------------ + +type sqsSendMessageInput struct { + QueueUrl string `json:"QueueUrl"` + MessageBody string `json:"MessageBody"` + DelaySeconds *int64 `json:"DelaySeconds,omitempty"` + MessageAttributes map[string]string `json:"MessageAttributes,omitempty"` +} + +type sqsReceiveMessageInput struct { + QueueUrl string `json:"QueueUrl"` + MaxNumberOfMessages int `json:"MaxNumberOfMessages,omitempty"` + VisibilityTimeout *int64 `json:"VisibilityTimeout,omitempty"` + WaitTimeSeconds *int64 `json:"WaitTimeSeconds,omitempty"` +} + +type sqsDeleteMessageInput struct { + QueueUrl string `json:"QueueUrl"` + ReceiptHandle string `json:"ReceiptHandle"` +} + +type sqsChangeVisibilityInput struct { + QueueUrl string `json:"QueueUrl"` + ReceiptHandle string `json:"ReceiptHandle"` + VisibilityTimeout int64 `json:"VisibilityTimeout"` +} + +// ------------------------ handlers ------------------------ + +func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { + var in sqsSendMessageInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + queueName, err := queueNameFromURL(in.QueueUrl) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + meta, apiErr := s.loadQueueMetaForSend(r.Context(), queueName, []byte(in.MessageBody)) + if apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } + delay, apiErr := resolveSendDelay(meta, in.DelaySeconds) + if apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } + rec, recordBytes, apiErr := buildSendRecord(meta, in, delay) + if apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } + + dataKey := sqsMsgDataKey(queueName, meta.Generation, rec.MessageID) + visKey := sqsMsgVisKey(queueName, meta.Generation, rec.AvailableAtMillis, rec.MessageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Put, Key: dataKey, Value: recordBytes}, + {Op: kv.Put, Key: visKey, Value: []byte(rec.MessageID)}, + }, + } + if _, err := s.coordinator.Dispatch(r.Context(), req); err != nil { + writeSQSErrorFromErr(w, err) + return + } + + writeSQSJSON(w, map[string]string{ + "MessageId": rec.MessageID, + "MD5OfMessageBody": rec.MD5OfBody, + "MD5OfMessageAttributes": md5OfAttributesHex(in.MessageAttributes), + }) +} + +func (s *SQSServer) loadQueueMetaForSend(ctx context.Context, queueName string, body []byte) (*sqsQueueMeta, error) { + readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return nil, errors.WithStack(err) + } + if !exists { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + if int64(len(body)) > meta.MaximumMessageSize { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrMessageTooLong, "message body exceeds MaximumMessageSize") + } + return meta, nil +} + +func resolveSendDelay(meta *sqsQueueMeta, requested *int64) (int64, error) { + delay := meta.DelaySeconds + if requested == nil { + return delay, nil + } + if *requested < 0 || *requested > sqsMaxDelaySeconds { + return 0, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "DelaySeconds out of range") + } + return *requested, nil +} + +func buildSendRecord(meta *sqsQueueMeta, in sqsSendMessageInput, delay int64) (*sqsMessageRecord, []byte, error) { + messageID, err := newMessageIDHex() + if err != nil { + return nil, nil, errors.WithStack(err) + } + token, err := newReceiptToken() + if err != nil { + return nil, nil, errors.WithStack(err) + } + now := time.Now().UnixMilli() + availableAt := now + delay*sqsMillisPerSecond + body := []byte(in.MessageBody) + rec := &sqsMessageRecord{ + MessageID: messageID, + Body: body, + MD5OfBody: sqsMD5Hex(body), + MessageAttributes: in.MessageAttributes, + SendTimestampMillis: now, + AvailableAtMillis: availableAt, + VisibleAtMillis: availableAt, + CurrentReceiptToken: token, + QueueGeneration: meta.Generation, + } + recordBytes, err := encodeSQSMessageRecord(rec) + if err != nil { + return nil, nil, errors.WithStack(err) + } + return rec, recordBytes, nil +} + +//nolint:cyclop // AWS ReceiveMessage branches on per-message eligibility; splitting further just moves the branching around. +func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { + var in sqsReceiveMessageInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + queueName, err := queueNameFromURL(in.QueueUrl) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + ctx := r.Context() + + // Use LeaseRead to fence this scan against a leader that silently lost + // quorum mid-request. When the lease is warm this is a local + // wall-clock compare; when it is cold it falls back to a full + // LinearizableRead. + if _, err := kv.LeaseReadThrough(s.coordinator, ctx); err != nil { + writeSQSErrorFromErr(w, err) + return + } + + readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + if !exists { + writeSQSError(w, http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + return + } + max := clampReceiveMaxMessages(in.MaxNumberOfMessages) + visibilityTimeout := meta.VisibilityTimeoutSeconds + if in.VisibilityTimeout != nil { + if *in.VisibilityTimeout < 0 || *in.VisibilityTimeout > sqsChangeVisibilityMaxSeconds { + writeSQSError(w, http.StatusBadRequest, sqsErrInvalidAttributeValue, "VisibilityTimeout out of range") + return + } + visibilityTimeout = *in.VisibilityTimeout + } + + candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, meta.Generation, max*sqsReceiveScanOverfetchFactor, readTS) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + delivered := s.rotateMessagesForDelivery(ctx, queueName, meta.Generation, candidates, visibilityTimeout, max, readTS) + writeSQSJSON(w, map[string]any{"Messages": delivered}) +} + +func clampReceiveMaxMessages(requested int) int { + if requested <= 0 { + return sqsReceiveDefaultMaxMessages + } + if requested > sqsReceiveHardMaxMessages { + return sqsReceiveHardMaxMessages + } + return requested +} + +// scanVisibleMessageCandidates returns vis-index entries with +// visible_at <= now, up to limit. Each entry carries the key (needed +// for the delete-old-vis step) and the message_id pointed at by its +// value. +type sqsMsgCandidate struct { + visKey []byte + messageID string +} + +func (s *SQSServer) scanVisibleMessageCandidates(ctx context.Context, queueName string, gen uint64, limit int, readTS uint64) ([]sqsMsgCandidate, error) { + if limit <= 0 { + return nil, nil + } + now := time.Now().UnixMilli() + start, end := sqsMsgVisScanBounds(queueName, gen, now) + page := limit + if page > sqsVisScanPageLimit { + page = sqsVisScanPageLimit + } + kvs, err := s.store.ScanAt(ctx, start, end, page, readTS) + if err != nil { + return nil, errors.WithStack(err) + } + out := make([]sqsMsgCandidate, 0, len(kvs)) + for _, kvp := range kvs { + out = append(out, sqsMsgCandidate{visKey: bytes.Clone(kvp.Key), messageID: string(kvp.Value)}) + } + return out, nil +} + +// rotateMessagesForDelivery runs an OCC transaction per candidate to +// rotate its visibility entry + receipt token. Failures on individual +// messages (races, write-conflict) are skipped rather than aborting the +// whole batch — AWS semantics allow ReceiveMessage to return fewer +// messages than requested. +func (s *SQSServer) rotateMessagesForDelivery( + ctx context.Context, + queueName string, + gen uint64, + candidates []sqsMsgCandidate, + visibilityTimeout int64, + max int, + readTS uint64, +) []map[string]any { + delivered := make([]map[string]any, 0, max) + for _, cand := range candidates { + if len(delivered) >= max { + break + } + msg, ok := s.tryDeliverCandidate(ctx, queueName, gen, cand, visibilityTimeout, readTS) + if !ok { + continue + } + delivered = append(delivered, msg) + } + return delivered +} + +func (s *SQSServer) tryDeliverCandidate( + ctx context.Context, + queueName string, + gen uint64, + cand sqsMsgCandidate, + visibilityTimeout int64, + readTS uint64, +) (map[string]any, bool) { + dataKey := sqsMsgDataKey(queueName, gen, cand.messageID) + raw, err := s.store.GetAt(ctx, dataKey, readTS) + if err != nil { + return nil, false + } + rec, err := decodeSQSMessageRecord(raw) + if err != nil { + return nil, false + } + + newToken, err := newReceiptToken() + if err != nil { + return nil, false + } + now := time.Now().UnixMilli() + newVisibleAt := now + visibilityTimeout*sqsMillisPerSecond + rec.VisibleAtMillis = newVisibleAt + rec.CurrentReceiptToken = newToken + rec.ReceiveCount++ + if rec.FirstReceiveMillis == 0 { + rec.FirstReceiveMillis = now + } + recordBytes, err := encodeSQSMessageRecord(rec) + if err != nil { + return nil, false + } + newVisKey := sqsMsgVisKey(queueName, gen, newVisibleAt, cand.messageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: cand.visKey}, + {Op: kv.Put, Key: newVisKey, Value: []byte(cand.messageID)}, + {Op: kv.Put, Key: dataKey, Value: recordBytes}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err != nil { + return nil, false + } + + handle, err := encodeReceiptHandle(gen, cand.messageID, newToken) + if err != nil { + return nil, false + } + return map[string]any{ + "MessageId": cand.messageID, + "ReceiptHandle": handle, + "Body": string(rec.Body), + "MD5OfBody": rec.MD5OfBody, + "Attributes": map[string]string{ + "ApproximateReceiveCount": strconv.FormatInt(rec.ReceiveCount, 10), + "SentTimestamp": strconv.FormatInt(rec.SendTimestampMillis, 10), + "ApproximateFirstReceiveTimestamp": strconv.FormatInt(rec.FirstReceiveMillis, 10), + }, + }, true +} + +func (s *SQSServer) deleteMessage(w http.ResponseWriter, r *http.Request) { + var in sqsDeleteMessageInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + queueName, handle, err := s.parseQueueAndReceipt(in.QueueUrl, in.ReceiptHandle) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + rec, dataKey, apiErr := s.loadAndVerifyMessage(r.Context(), queueName, handle) + if apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } + visKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: dataKey}, + {Op: kv.Del, Key: visKey}, + }, + } + if _, err := s.coordinator.Dispatch(r.Context(), req); err != nil { + writeSQSErrorFromErr(w, err) + return + } + writeSQSJSON(w, map[string]any{}) +} + +func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Request) { + var in sqsChangeVisibilityInput + if err := decodeSQSJSONInput(r, &in); err != nil { + writeSQSErrorFromErr(w, err) + return + } + if in.VisibilityTimeout < 0 || in.VisibilityTimeout > sqsChangeVisibilityMaxSeconds { + writeSQSError(w, http.StatusBadRequest, sqsErrInvalidAttributeValue, "VisibilityTimeout out of range") + return + } + queueName, handle, err := s.parseQueueAndReceipt(in.QueueUrl, in.ReceiptHandle) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + rec, dataKey, apiErr := s.loadAndVerifyMessage(r.Context(), queueName, handle) + if apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } + now := time.Now().UnixMilli() + if rec.VisibleAtMillis <= now { + writeSQSError(w, http.StatusBadRequest, sqsErrMessageNotInflight, "message is not currently in flight") + return + } + + oldVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + rec.VisibleAtMillis = now + in.VisibilityTimeout*sqsMillisPerSecond + recordBytes, err := encodeSQSMessageRecord(rec) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } + newVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: oldVisKey}, + {Op: kv.Put, Key: newVisKey, Value: []byte(rec.MessageID)}, + {Op: kv.Put, Key: dataKey, Value: recordBytes}, + }, + } + if _, err := s.coordinator.Dispatch(r.Context(), req); err != nil { + writeSQSErrorFromErr(w, err) + return + } + writeSQSJSON(w, map[string]any{}) +} + +// parseQueueAndReceipt extracts the queue name and decodes the receipt +// handle from a DeleteMessage / ChangeMessageVisibility input. +func (s *SQSServer) parseQueueAndReceipt(queueUrl, receiptHandle string) (string, *decodedReceiptHandle, error) { + queueName, err := queueNameFromURL(queueUrl) + if err != nil { + return "", nil, err + } + handle, err := decodeReceiptHandle(receiptHandle) + if err != nil { + return "", nil, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "receipt handle is not parseable") + } + return queueName, handle, nil +} + +// loadAndVerifyMessage reads the data record for the given handle and +// verifies that the receipt token matches the current one on record. +// Returns the record, its key, or a typed SQS error. +func (s *SQSServer) loadAndVerifyMessage(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, error) { + readTS := s.nextTxnReadTS(ctx) + dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) + raw, err := s.store.GetAt(ctx, dataKey, readTS) + if err != nil { + if errors.Is(err, store.ErrKeyNotFound) { + return nil, nil, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "message not found") + } + return nil, nil, errors.WithStack(err) + } + rec, err := decodeSQSMessageRecord(raw) + if err != nil { + return nil, nil, errors.WithStack(err) + } + if !bytes.Equal(rec.CurrentReceiptToken, handle.ReceiptToken) { + return nil, nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidReceiptHandle, "receipt handle token does not match") + } + return rec, dataKey, nil +} + +// ------------------------ small helpers ------------------------ + +func sqsMD5Hex(body []byte) string { + sum := md5.Sum(body) //nolint:gosec // AWS-specified ETag hashing, not a crypto primitive. + return hex.EncodeToString(sum[:]) +} + +// md5OfAttributesHex computes AWS's MD5 of a MessageAttributes map. The +// real AWS format canonicalizes names and types; this adapter only +// returns "" on an empty map and a simple concatenated hash otherwise +// (full canonicalization lives in a follow-up PR along with typed +// attribute values). +func md5OfAttributesHex(attrs map[string]string) string { + if len(attrs) == 0 { + return "" + } + keys := make([]string, 0, len(attrs)) + for k := range attrs { + keys = append(keys, k) + } + // Stable order for determinism. + for i := 0; i < len(keys); i++ { + for j := i + 1; j < len(keys); j++ { + if keys[j] < keys[i] { + keys[i], keys[j] = keys[j], keys[i] + } + } + } + var b strings.Builder + for _, k := range keys { + b.WriteString(k) + b.WriteString("=") + b.WriteString(attrs[k]) + b.WriteString(";") + } + return sqsMD5Hex([]byte(b.String())) +} diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go new file mode 100644 index 000000000..688cda999 --- /dev/null +++ b/adapter/sqs_messages_test.go @@ -0,0 +1,388 @@ +package adapter + +import ( + "encoding/hex" + "net/http" + "strconv" + "testing" + "time" +) + +// createSQSQueueForTest is a small helper so every message-path test does +// not have to repeat the "createQueue -> pull URL" dance. +func createSQSQueueForTest(t *testing.T, node Node, name string) string { + t.Helper() + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{"QueueName": name}) + if status != http.StatusOK { + t.Fatalf("createQueue %q: %d %v", name, status, out) + } + url, _ := out["QueueUrl"].(string) + if url == "" { + t.Fatalf("createQueue %q: empty URL", name) + } + return url +} + +func TestSQSServer_SendReceiveDelete(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "work") + + msgID := sendOneMessage(t, node, queueURL, "hello") + receipt := receiveOneMessage(t, node, queueURL, msgID, "hello") + expectNoMessagesVisible(t, node, queueURL, "after-receive") + deleteMessageOK(t, node, queueURL, receipt) + expectNoMessagesVisible(t, node, queueURL, "after-delete") +} + +// sendOneMessage sends a single message and returns the assigned MessageId. +// It asserts the response carries an MD5 over the body that matches what +// sqsMD5Hex would compute locally. +func sendOneMessage(t *testing.T, node Node, queueURL, body string) string { + t.Helper() + status, out := callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": body, + }) + if status != http.StatusOK { + t.Fatalf("send: %d %v", status, out) + } + msgID, _ := out["MessageId"].(string) + if msgID == "" { + t.Fatalf("no MessageId in %v", out) + } + if out["MD5OfMessageBody"] != sqsMD5Hex([]byte(body)) { + t.Fatalf("md5 mismatch: got %v want %q", out["MD5OfMessageBody"], sqsMD5Hex([]byte(body))) + } + return msgID +} + +func receiveOneMessage(t *testing.T, node Node, queueURL, wantID, wantBody string) string { + t.Helper() + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("got %d messages, want 1 (%v)", len(msgs), out) + } + m, _ := msgs[0].(map[string]any) + if m["Body"] != wantBody { + t.Fatalf("Body=%v want %q", m["Body"], wantBody) + } + if m["MessageId"] != wantID { + t.Fatalf("MessageId=%v want %q", m["MessageId"], wantID) + } + receipt, _ := m["ReceiptHandle"].(string) + if receipt == "" { + t.Fatalf("no ReceiptHandle in %v", m) + } + return receipt +} + +func expectNoMessagesVisible(t *testing.T, node Node, queueURL, tag string) { + t.Helper() + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + }) + if status != http.StatusOK { + t.Fatalf("%s receive: %d %v", tag, status, out) + } + if msgs, _ := out["Messages"].([]any); len(msgs) != 0 { + t.Fatalf("%s: got %d messages, want 0", tag, len(msgs)) + } +} + +func deleteMessageOK(t *testing.T, node Node, queueURL, receipt string) { + t.Helper() + status, out := callSQS(t, node, sqsDeleteMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "ReceiptHandle": receipt, + }) + if status != http.StatusOK { + t.Fatalf("delete: %d %v", status, out) + } +} + +func TestSQSServer_DeleteWithWrongReceiptRejected(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "wrong-receipt") + + // Send + receive once to get a real message on the queue. + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "x", + }) + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) == 0 { + t.Fatalf("no messages received") + } + goodHandle, _ := msgs[0].(map[string]any)["ReceiptHandle"].(string) + + // Tamper the receipt token portion of the handle and expect + // InvalidReceiptHandle. + decoded, err := decodeReceiptHandle(goodHandle) + if err != nil { + t.Fatalf("decode: %v", err) + } + decoded.ReceiptToken[0] ^= 0xff + badHandle, err := encodeReceiptHandle(decoded.QueueGeneration, decoded.MessageIDHex, decoded.ReceiptToken) + if err != nil { + t.Fatalf("encode: %v", err) + } + status, out = callSQS(t, node, sqsDeleteMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "ReceiptHandle": badHandle, + }) + if status != http.StatusBadRequest { + t.Fatalf("delete with bad receipt: status=%d body=%v", status, out) + } + if out["__type"] != sqsErrInvalidReceiptHandle { + t.Fatalf("error type: %q want %q", out["__type"], sqsErrInvalidReceiptHandle) + } +} + +func TestSQSServer_ReceiveBatchRespectsMaxMessages(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "batch") + + const sent = 5 + for i := 0; i < sent; i++ { + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "m-" + strconv.Itoa(i), + }) + } + + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 3, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 3 { + t.Fatalf("got %d messages, want 3 (%v)", len(msgs), out) + } + + // A second receive picks up the remaining two. + status, out = callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 10, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("receive#2: %d %v", status, out) + } + msgs, _ = out["Messages"].([]any) + if len(msgs) != 2 { + t.Fatalf("got %d messages on second receive, want 2", len(msgs)) + } +} + +func TestSQSServer_DelaySecondsDefersDelivery(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "delayed") + + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "later", + "DelaySeconds": 2, + }) + + // Immediate receive must return nothing. + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + if msgs, _ := out["Messages"].([]any); len(msgs) != 0 { + t.Fatalf("expected 0 messages before delay elapsed, got %d", len(msgs)) + } + + // After the delay, the message becomes visible. + time.Sleep(2100 * time.Millisecond) + status, out = callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("expected 1 message after delay, got %d (%v)", len(msgs), out) + } +} + +func TestSQSServer_VisibilityTimeoutExpiryMakesMessageVisibleAgain(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "revisible") + + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "retry-me", + }) + + // First receive with a very short visibility so this test doesn't + // sit idle for 30 seconds. + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 1, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("first receive: got %d messages want 1", len(msgs)) + } + first, _ := msgs[0].(map[string]any) + firstAttrs, _ := first["Attributes"].(map[string]any) + if firstAttrs["ApproximateReceiveCount"] != "1" { + t.Fatalf("first receive count = %v, want 1", firstAttrs) + } + + time.Sleep(1200 * time.Millisecond) + + // Second receive should see the same message with ReceiveCount=2. + status, out = callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("second receive: %d %v", status, out) + } + msgs, _ = out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("second receive: got %d messages want 1", len(msgs)) + } + second, _ := msgs[0].(map[string]any) + if first["MessageId"] != second["MessageId"] { + t.Fatalf("second receive returned a different message: %v vs %v", first, second) + } + secondAttrs, _ := second["Attributes"].(map[string]any) + if secondAttrs["ApproximateReceiveCount"] != "2" { + t.Fatalf("second receive count = %v, want 2", secondAttrs) + } +} + +func TestSQSServer_ChangeMessageVisibility(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "chgvis") + + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "bumpy", + }) + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "VisibilityTimeout": 1, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("no message received") + } + receipt, _ := msgs[0].(map[string]any)["ReceiptHandle"].(string) + + // Extend visibility to 60s. + status, out = callSQS(t, node, sqsChangeMessageVisibilityTarget, map[string]any{ + "QueueUrl": queueURL, + "ReceiptHandle": receipt, + "VisibilityTimeout": 60, + }) + if status != http.StatusOK { + t.Fatalf("change visibility: %d %v", status, out) + } + + // After the original 1s expiry, the message must still be hidden. + time.Sleep(1200 * time.Millisecond) + status, out = callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + }) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + if msgs, _ := out["Messages"].([]any); len(msgs) != 0 { + t.Fatalf("expected 0 messages after visibility extended, got %d", len(msgs)) + } +} + +func TestSQSServer_ReceiptHandleCodecRoundTrip(t *testing.T) { + t.Parallel() + for _, tc := range []struct { + gen uint64 + id string + }{ + {0, "00000000000000000000000000000000"}, + {1, "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"}, + {42, "deadbeefdeadbeefdeadbeefdeadbeef"}, + } { + token := make([]byte, sqsReceiptTokenBytes) + for i := range token { + token[i] = byte(i) + } + h, err := encodeReceiptHandle(tc.gen, tc.id, token) + if err != nil { + t.Fatalf("encode: %v", err) + } + back, err := decodeReceiptHandle(h) + if err != nil { + t.Fatalf("decode: %v", err) + } + if back.QueueGeneration != tc.gen || back.MessageIDHex != tc.id { + t.Fatalf("round-trip mismatch: %+v vs %d/%s", back, tc.gen, tc.id) + } + if hex.EncodeToString(back.ReceiptToken) != hex.EncodeToString(token) { + t.Fatalf("token round-trip mismatch: %x vs %x", back.ReceiptToken, token) + } + } + + // A garbage handle must fail to decode, not crash. + if _, err := decodeReceiptHandle("!!!"); err == nil { + t.Fatal("expected decode error for garbage handle") + } +} diff --git a/adapter/sqs_test.go b/adapter/sqs_test.go index 9cf122171..721917174 100644 --- a/adapter/sqs_test.go +++ b/adapter/sqs_test.go @@ -124,19 +124,15 @@ func TestSQSServer_KnownTargetsReturnNotImplemented(t *testing.T) { t.Parallel() base := startTestSQSServer(t) - // Targets that still return NotImplemented. The catalog targets - // (CreateQueue/DeleteQueue/ListQueues/GetQueueUrl/GetQueueAttributes/ - // SetQueueAttributes) are covered by TestSQSServer_Catalog* against a - // real single-node cluster because they require a coordinator to - // dispatch transactions. + // Targets that still return NotImplemented. The catalog and core + // message operations (Create/Delete/List/Get/SetQueue*, SendMessage, + // ReceiveMessage, DeleteMessage, ChangeMessageVisibility) have real + // handlers; they are exercised against a single-node cluster by + // TestSQSServer_Catalog* and TestSQSServer_Send*. targets := []string{ sqsPurgeQueueTarget, - sqsSendMessageTarget, sqsSendMessageBatchTarget, - sqsReceiveMessageTarget, - sqsDeleteMessageTarget, sqsDeleteMessageBatchTarget, - sqsChangeMessageVisibilityTarget, sqsChangeMessageVisibilityBatchTgt, sqsTagQueueTarget, sqsUntagQueueTarget, From 4bae6d05155f2fc907e31c64e02e777fd1fa60bc Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 16:44:54 +0900 Subject: [PATCH 14/26] fix(sqs): close receipt-token and visibility races under OCC Address the P1 findings from the codex review on #610 and the gemini micro-nit: - tryDeliverCandidate now passes ReadKeys=[old_vis_key, data_key] on its commit so a concurrent ReceiveMessage that rotates the same message commits a newer version and our Dispatch returns ErrWriteConflict. Skipping the candidate on conflict prevents the "two workers both think they delivered" duplicate. - deleteMessage and changeMessageVisibility lift their single-shot validate-and-commit into retry loops that mirror the catalog handlers. Both pass ReadKeys covering the data record and the current vis entry; on ErrWriteConflict we re-validate the token, which either succeeds (the state we just saw is still current) or returns InvalidReceiptHandle (someone else rotated the token first, which is the correct AWS semantics for a stale handle). - md5OfAttributesHex: drop the hand-rolled O(n^2) selection sort in favor of sort.Strings. No behavior change on the happy path; these are purely correctness-under-contention fixes. Existing tests still cover the single-writer flows, and the retry loops inherit the same test-wide timeout knobs the catalog code uses. --- adapter/sqs_messages.go | 142 +++++++++++++++++++++++++--------------- 1 file changed, 91 insertions(+), 51 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 1c7682eb9..4bd261c58 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -9,6 +9,7 @@ import ( "encoding/binary" "encoding/hex" "net/http" + "sort" "strconv" "strings" "time" @@ -519,8 +520,13 @@ func (s *SQSServer) tryDeliverCandidate( return nil, false } newVisKey := sqsMsgVisKey(queueName, gen, newVisibleAt, cand.messageID) + // ReadKeys include the visibility entry and the data record so a + // concurrent ReceiveMessage that rotated the same message commits a + // newer version and forces our Dispatch to return ErrWriteConflict. + // Skipping the candidate on conflict prevents duplicate delivery. req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, + IsTxn: true, + ReadKeys: [][]byte{cand.visKey, dataKey}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: cand.visKey}, {Op: kv.Put, Key: newVisKey, Value: []byte(cand.messageID)}, @@ -559,26 +565,49 @@ func (s *SQSServer) deleteMessage(w http.ResponseWriter, r *http.Request) { writeSQSErrorFromErr(w, err) return } - rec, dataKey, apiErr := s.loadAndVerifyMessage(r.Context(), queueName, handle) - if apiErr != nil { - writeSQSErrorFromErr(w, apiErr) - return - } - visKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) - req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, - Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Del, Key: dataKey}, - {Op: kv.Del, Key: visKey}, - }, - } - if _, err := s.coordinator.Dispatch(r.Context(), req); err != nil { + if err := s.deleteMessageWithRetry(r.Context(), queueName, handle); err != nil { writeSQSErrorFromErr(w, err) return } writeSQSJSON(w, map[string]any{}) } +// deleteMessageWithRetry runs the receipt-token check and the delete +// transaction under one OCC budget. A concurrent ReceiveMessage / +// ChangeMessageVisibility that rotates the token between validate and +// commit produces ErrWriteConflict; we retry by re-validating the +// current token against the handle. If the token has actually been +// rotated under us, the next pass returns InvalidReceiptHandle. +func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string, handle *decodedReceiptHandle) error { + backoff := transactRetryInitialBackoff + deadline := time.Now().Add(transactRetryMaxDuration) + for range transactRetryMaxAttempts { + rec, dataKey, apiErr := s.loadAndVerifyMessage(ctx, queueName, handle) + if apiErr != nil { + return apiErr + } + visKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + ReadKeys: [][]byte{dataKey, visKey}, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: dataKey}, + {Op: kv.Del, Key: visKey}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err == nil { + return nil + } else if !isRetryableTransactWriteError(err) { + return errors.WithStack(err) + } + if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { + return errors.WithStack(err) + } + backoff = nextTransactRetryBackoff(backoff) + } + return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "delete message retry attempts exhausted") +} + func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Request) { var in sqsChangeVisibilityInput if err := decodeSQSJSONInput(r, &in); err != nil { @@ -594,40 +623,58 @@ func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Reque writeSQSErrorFromErr(w, err) return } - rec, dataKey, apiErr := s.loadAndVerifyMessage(r.Context(), queueName, handle) - if apiErr != nil { - writeSQSErrorFromErr(w, apiErr) - return - } - now := time.Now().UnixMilli() - if rec.VisibleAtMillis <= now { - writeSQSError(w, http.StatusBadRequest, sqsErrMessageNotInflight, "message is not currently in flight") - return - } - - oldVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) - rec.VisibleAtMillis = now + in.VisibilityTimeout*sqsMillisPerSecond - recordBytes, err := encodeSQSMessageRecord(rec) - if err != nil { - writeSQSErrorFromErr(w, err) - return - } - newVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) - req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, - Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Del, Key: oldVisKey}, - {Op: kv.Put, Key: newVisKey, Value: []byte(rec.MessageID)}, - {Op: kv.Put, Key: dataKey, Value: recordBytes}, - }, - } - if _, err := s.coordinator.Dispatch(r.Context(), req); err != nil { + if err := s.changeVisibilityWithRetry(r.Context(), queueName, handle, in.VisibilityTimeout); err != nil { writeSQSErrorFromErr(w, err) return } writeSQSJSON(w, map[string]any{}) } +// changeVisibilityWithRetry runs the validate-and-swap flow under an OCC +// retry budget. ReadKeys cover the data record and the current vis +// entry; a concurrent receive or delete will bump their commitTS past +// our startTS and we re-validate. +func (s *SQSServer) changeVisibilityWithRetry(ctx context.Context, queueName string, handle *decodedReceiptHandle, newTimeout int64) error { + backoff := transactRetryInitialBackoff + deadline := time.Now().Add(transactRetryMaxDuration) + for range transactRetryMaxAttempts { + rec, dataKey, apiErr := s.loadAndVerifyMessage(ctx, queueName, handle) + if apiErr != nil { + return apiErr + } + now := time.Now().UnixMilli() + if rec.VisibleAtMillis <= now { + return newSQSAPIError(http.StatusBadRequest, sqsErrMessageNotInflight, "message is not currently in flight") + } + oldVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + rec.VisibleAtMillis = now + newTimeout*sqsMillisPerSecond + recordBytes, err := encodeSQSMessageRecord(rec) + if err != nil { + return errors.WithStack(err) + } + newVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + ReadKeys: [][]byte{dataKey, oldVisKey}, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: oldVisKey}, + {Op: kv.Put, Key: newVisKey, Value: []byte(rec.MessageID)}, + {Op: kv.Put, Key: dataKey, Value: recordBytes}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err == nil { + return nil + } else if !isRetryableTransactWriteError(err) { + return errors.WithStack(err) + } + if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { + return errors.WithStack(err) + } + backoff = nextTransactRetryBackoff(backoff) + } + return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "change visibility retry attempts exhausted") +} + // parseQueueAndReceipt extracts the queue name and decodes the receipt // handle from a DeleteMessage / ChangeMessageVisibility input. func (s *SQSServer) parseQueueAndReceipt(queueUrl, receiptHandle string) (string, *decodedReceiptHandle, error) { @@ -685,14 +732,7 @@ func md5OfAttributesHex(attrs map[string]string) string { for k := range attrs { keys = append(keys, k) } - // Stable order for determinism. - for i := 0; i < len(keys); i++ { - for j := i + 1; j < len(keys); j++ { - if keys[j] < keys[i] { - keys[i], keys[j] = keys[j], keys[i] - } - } - } + sort.Strings(keys) var b strings.Builder for _, k := range keys { b.WriteString(k) From 6685fbf530a810c28a743595549eae034b5afdc7 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 16:55:08 +0900 Subject: [PATCH 15/26] fix(sqs): make DeleteMessage idempotent on stale receipt handles MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit AWS SQS semantics: DeleteMessage with a stale receipt handle (token rotated under us, or record already gone) is a 200 success no-op, NOT an InvalidReceiptHandle. This is relied on by SDK retry paths and batch workers, and also by clients that do a visibility-expiry redelivery dance. - deleteMessageWithRetry now calls a new loadMessageForDelete helper that returns a sqsDeleteOutcome tag. Missing record and token mismatch both map to sqsDeleteNoOp -> return nil (handler renders 200 empty body). Token-match proceeds to the OCC delete txn. Only structural errors (malformed handle) and retry-budget exhaustion still propagate as errors. - ChangeMessageVisibility keeps the strict behavior — AWS also errors there when the handle is stale, so changeVisibilityWithRetry is unchanged. - Test rename: TestSQSServer_DeleteWithWrongReceiptRejected -> TestSQSServer_DeleteWithStaleReceiptIsIdempotentNoOp. The test now asserts that a token-mismatched delete returns 200, that the real handle still works afterwards (the stale no-op must not have "stolen" the in-flight message), and that a structurally malformed handle still fails with ReceiptHandleIsInvalid. The ReceiveCount / first-receive-timestamp behavior the same codex review flagged is already implemented correctly (tryDeliverCandidate: "if rec.FirstReceiveMillis == 0 { ... = now }"), so no code change was needed for that finding beyond the design doc clarification that landed on feat/sqs_compatible_adapter. --- adapter/sqs_messages.go | 62 ++++++++++++++++++++++++++++++------ adapter/sqs_messages_test.go | 46 ++++++++++++++++++++------ 2 files changed, 89 insertions(+), 19 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 4bd261c58..614b7f2d6 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -572,19 +572,28 @@ func (s *SQSServer) deleteMessage(w http.ResponseWriter, r *http.Request) { writeSQSJSON(w, map[string]any{}) } -// deleteMessageWithRetry runs the receipt-token check and the delete -// transaction under one OCC budget. A concurrent ReceiveMessage / -// ChangeMessageVisibility that rotates the token between validate and -// commit produces ErrWriteConflict; we retry by re-validating the -// current token against the handle. If the token has actually been -// rotated under us, the next pass returns InvalidReceiptHandle. +// deleteMessageWithRetry runs the load-check-commit flow under one OCC +// budget. AWS SQS semantics: a stale receipt handle (message already +// gone, or token rotated by another consumer) is a 200 no-op, NOT an +// error. The only error cases are structural (malformed handle, caught +// before this function) and infrastructure (retry budget exhausted). +// ErrWriteConflict on the delete Dispatch means a concurrent rotation +// / delete landed between our read and our commit; we retry so the +// next pass either sees the rotated token (no-op success) or the +// missing record (no-op success). func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string, handle *decodedReceiptHandle) error { backoff := transactRetryInitialBackoff deadline := time.Now().Add(transactRetryMaxDuration) for range transactRetryMaxAttempts { - rec, dataKey, apiErr := s.loadAndVerifyMessage(ctx, queueName, handle) - if apiErr != nil { - return apiErr + rec, dataKey, outcome, err := s.loadMessageForDelete(ctx, queueName, handle) + if err != nil { + return err + } + switch outcome { + case sqsDeleteNoOp: + return nil + case sqsDeleteProceed: + // fall through to commit below } visKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) req := &kv.OperationGroup[kv.OP]{ @@ -608,6 +617,41 @@ func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "delete message retry attempts exhausted") } +// sqsDeleteOutcome is a ternary tag returned by loadMessageForDelete so +// the caller can cleanly distinguish the AWS-idempotent no-op case from +// the proceed-to-commit case without conflating them with errors. +type sqsDeleteOutcome int + +const ( + sqsDeleteProceed sqsDeleteOutcome = iota + sqsDeleteNoOp +) + +// loadMessageForDelete reads the message record and classifies the +// outcome for AWS-compatible DeleteMessage semantics: structural errors +// propagate; missing records and token mismatches return +// sqsDeleteNoOp; matching tokens return sqsDeleteProceed with the +// loaded record. +func (s *SQSServer) loadMessageForDelete(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, sqsDeleteOutcome, error) { + readTS := s.nextTxnReadTS(ctx) + dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) + raw, err := s.store.GetAt(ctx, dataKey, readTS) + if err != nil { + if errors.Is(err, store.ErrKeyNotFound) { + return nil, nil, sqsDeleteNoOp, nil + } + return nil, nil, sqsDeleteProceed, errors.WithStack(err) + } + rec, err := decodeSQSMessageRecord(raw) + if err != nil { + return nil, nil, sqsDeleteProceed, errors.WithStack(err) + } + if !bytes.Equal(rec.CurrentReceiptToken, handle.ReceiptToken) { + return nil, nil, sqsDeleteNoOp, nil + } + return rec, dataKey, sqsDeleteProceed, nil +} + func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Request) { var in sqsChangeVisibilityInput if err := decodeSQSJSONInput(r, &in); err != nil { diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go index 688cda999..cc86b6db9 100644 --- a/adapter/sqs_messages_test.go +++ b/adapter/sqs_messages_test.go @@ -112,14 +112,19 @@ func deleteMessageOK(t *testing.T, node Node, queueURL, receipt string) { } } -func TestSQSServer_DeleteWithWrongReceiptRejected(t *testing.T) { +func TestSQSServer_DeleteWithStaleReceiptIsIdempotentNoOp(t *testing.T) { t.Parallel() + // AWS SQS semantics: DeleteMessage with a stale receipt handle (token + // rotated under our feet, or record already gone) must return 200 + // success without deleting. SDK retry paths and batch workers rely on + // this so a retry after a visibility-expiry re-delivery does not fail + // loudly. Structurally malformed handles are still an error; + // token-only mismatches are not. nodes, _, _ := createNode(t, 1) defer shutdown(nodes) node := sqsLeaderNode(t, nodes) - queueURL := createSQSQueueForTest(t, node, "wrong-receipt") + queueURL := createSQSQueueForTest(t, node, "stale-receipt") - // Send + receive once to get a real message on the queue. _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ "QueueUrl": queueURL, "MessageBody": "x", @@ -138,26 +143,47 @@ func TestSQSServer_DeleteWithWrongReceiptRejected(t *testing.T) { } goodHandle, _ := msgs[0].(map[string]any)["ReceiptHandle"].(string) - // Tamper the receipt token portion of the handle and expect - // InvalidReceiptHandle. + // Flip a byte of the token so the stored token != handle token. decoded, err := decodeReceiptHandle(goodHandle) if err != nil { t.Fatalf("decode: %v", err) } decoded.ReceiptToken[0] ^= 0xff - badHandle, err := encodeReceiptHandle(decoded.QueueGeneration, decoded.MessageIDHex, decoded.ReceiptToken) + staleHandle, err := encodeReceiptHandle(decoded.QueueGeneration, decoded.MessageIDHex, decoded.ReceiptToken) if err != nil { t.Fatalf("encode: %v", err) } + + // Delete with the stale handle must succeed (no-op) per AWS. + status, out = callSQS(t, node, sqsDeleteMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "ReceiptHandle": staleHandle, + }) + if status != http.StatusOK { + t.Fatalf("delete with stale receipt: status=%d body=%v", status, out) + } + + // The real handle must still work — the stale delete must not have + // removed the in-flight message out from under the original consumer. + status, out = callSQS(t, node, sqsDeleteMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "ReceiptHandle": goodHandle, + }) + if status != http.StatusOK { + t.Fatalf("delete with good receipt after stale no-op: %d %v", status, out) + } + + // A structurally malformed handle still errors out — only token + // mismatches are the idempotent no-op case. status, out = callSQS(t, node, sqsDeleteMessageTarget, map[string]any{ "QueueUrl": queueURL, - "ReceiptHandle": badHandle, + "ReceiptHandle": "not-base64-!!!", }) if status != http.StatusBadRequest { - t.Fatalf("delete with bad receipt: status=%d body=%v", status, out) + t.Fatalf("malformed handle: status=%d body=%v", status, out) } - if out["__type"] != sqsErrInvalidReceiptHandle { - t.Fatalf("error type: %q want %q", out["__type"], sqsErrInvalidReceiptHandle) + if out["__type"] != sqsErrReceiptHandleInvalid { + t.Fatalf("error type for malformed handle: %q want %q", out["__type"], sqsErrReceiptHandleInvalid) } } From 1846fd018b3cf91d98f3a48833a5ac1492b7ee80 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 17:18:04 +0900 Subject: [PATCH 16/26] fix(sqs): pin OCC StartTS to validated-read snapshot MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit codex round 2 on #610 flagged two P1 OCC safety holes: the ReceiveMessage rotation and the DeleteMessage commit both left StartTS=0, so the coordinator auto-assigned a newer StartTS at Dispatch time. A concurrent rotation that committed between our load and the auto-assigned StartTS would fall below the ReadKeys check and slip through, allowing stale-snapshot writes to land — double delivery in the receive path, stealing an in-flight message in the delete path. Fix: carry the snapshot readTS we actually loaded the record at through to the Dispatch call and set it as StartTS on the OperationGroup. The existing ReadKeys then fence correctly against any commit in (readTS, now]. - tryDeliverCandidate: StartTS=readTS on the rotation txn. - loadMessageForDelete: now returns the readTS it took; caller (deleteMessageWithRetry) threads it into the delete txn. - loadAndVerifyMessage: now returns the readTS too; changeVisibilityWithRetry threads it into the swap txn. No behavior change under no-contention. Existing tests still pass; a Jepsen-style concurrent receive/delete test that exposes the race will land in the follow-up workload PR alongside FIFO. --- adapter/sqs_messages.go | 57 ++++++++++++++++++++++++++--------------- 1 file changed, 37 insertions(+), 20 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 614b7f2d6..fe8ba5443 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -520,12 +520,16 @@ func (s *SQSServer) tryDeliverCandidate( return nil, false } newVisKey := sqsMsgVisKey(queueName, gen, newVisibleAt, cand.messageID) - // ReadKeys include the visibility entry and the data record so a - // concurrent ReceiveMessage that rotated the same message commits a - // newer version and forces our Dispatch to return ErrWriteConflict. - // Skipping the candidate on conflict prevents duplicate delivery. + // StartTS pins the OCC read snapshot to the timestamp we actually + // loaded the record at. Without it, the coordinator assigns a newer + // StartTS at dispatch, so a concurrent rotation that committed + // AFTER our read but BEFORE the assigned StartTS would slip through + // ReadKeys validation and let this transaction double-deliver. + // ReadKeys cover both the visibility entry and the data record so + // any concurrent commit on either produces ErrWriteConflict. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, + StartTS: readTS, ReadKeys: [][]byte{cand.visKey, dataKey}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: cand.visKey}, @@ -585,7 +589,7 @@ func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string backoff := transactRetryInitialBackoff deadline := time.Now().Add(transactRetryMaxDuration) for range transactRetryMaxAttempts { - rec, dataKey, outcome, err := s.loadMessageForDelete(ctx, queueName, handle) + rec, dataKey, readTS, outcome, err := s.loadMessageForDelete(ctx, queueName, handle) if err != nil { return err } @@ -596,8 +600,12 @@ func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string // fall through to commit below } visKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + // StartTS pins OCC to the snapshot we loaded the record at, so a + // concurrent rotation that commits after our load but before a + // coordinator-assigned StartTS cannot slip past ReadKeys. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, + StartTS: readTS, ReadKeys: [][]byte{dataKey, visKey}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: dataKey}, @@ -631,25 +639,27 @@ const ( // outcome for AWS-compatible DeleteMessage semantics: structural errors // propagate; missing records and token mismatches return // sqsDeleteNoOp; matching tokens return sqsDeleteProceed with the -// loaded record. -func (s *SQSServer) loadMessageForDelete(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, sqsDeleteOutcome, error) { +// loaded record. The readTS it took the snapshot at is returned so the +// caller can pass it as StartTS on the OCC dispatch, pinning the +// read-write conflict detection window. +func (s *SQSServer) loadMessageForDelete(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, uint64, sqsDeleteOutcome, error) { readTS := s.nextTxnReadTS(ctx) dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) raw, err := s.store.GetAt(ctx, dataKey, readTS) if err != nil { if errors.Is(err, store.ErrKeyNotFound) { - return nil, nil, sqsDeleteNoOp, nil + return nil, nil, readTS, sqsDeleteNoOp, nil } - return nil, nil, sqsDeleteProceed, errors.WithStack(err) + return nil, nil, readTS, sqsDeleteProceed, errors.WithStack(err) } rec, err := decodeSQSMessageRecord(raw) if err != nil { - return nil, nil, sqsDeleteProceed, errors.WithStack(err) + return nil, nil, readTS, sqsDeleteProceed, errors.WithStack(err) } if !bytes.Equal(rec.CurrentReceiptToken, handle.ReceiptToken) { - return nil, nil, sqsDeleteNoOp, nil + return nil, nil, readTS, sqsDeleteNoOp, nil } - return rec, dataKey, sqsDeleteProceed, nil + return rec, dataKey, readTS, sqsDeleteProceed, nil } func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Request) { @@ -682,7 +692,7 @@ func (s *SQSServer) changeVisibilityWithRetry(ctx context.Context, queueName str backoff := transactRetryInitialBackoff deadline := time.Now().Add(transactRetryMaxDuration) for range transactRetryMaxAttempts { - rec, dataKey, apiErr := s.loadAndVerifyMessage(ctx, queueName, handle) + rec, dataKey, readTS, apiErr := s.loadAndVerifyMessage(ctx, queueName, handle) if apiErr != nil { return apiErr } @@ -697,8 +707,13 @@ func (s *SQSServer) changeVisibilityWithRetry(ctx context.Context, queueName str return errors.WithStack(err) } newVisKey := sqsMsgVisKey(queueName, handle.QueueGeneration, rec.VisibleAtMillis, rec.MessageID) + // StartTS pins OCC to the snapshot; without it the coordinator + // would auto-assign a newer StartTS and a concurrent receive / + // delete that commits between our load and dispatch could slip + // past the ReadKeys validation. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, + StartTS: readTS, ReadKeys: [][]byte{dataKey, oldVisKey}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: oldVisKey}, @@ -735,25 +750,27 @@ func (s *SQSServer) parseQueueAndReceipt(queueUrl, receiptHandle string) (string // loadAndVerifyMessage reads the data record for the given handle and // verifies that the receipt token matches the current one on record. -// Returns the record, its key, or a typed SQS error. -func (s *SQSServer) loadAndVerifyMessage(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, error) { +// Returns the record, its key, the snapshot timestamp the read ran at, +// or a typed SQS error. Callers use the snapshot as StartTS on the +// OCC dispatch so concurrent commits cannot slip past ReadKeys. +func (s *SQSServer) loadAndVerifyMessage(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, uint64, error) { readTS := s.nextTxnReadTS(ctx) dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) raw, err := s.store.GetAt(ctx, dataKey, readTS) if err != nil { if errors.Is(err, store.ErrKeyNotFound) { - return nil, nil, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "message not found") + return nil, nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "message not found") } - return nil, nil, errors.WithStack(err) + return nil, nil, readTS, errors.WithStack(err) } rec, err := decodeSQSMessageRecord(raw) if err != nil { - return nil, nil, errors.WithStack(err) + return nil, nil, readTS, errors.WithStack(err) } if !bytes.Equal(rec.CurrentReceiptToken, handle.ReceiptToken) { - return nil, nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidReceiptHandle, "receipt handle token does not match") + return nil, nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidReceiptHandle, "receipt handle token does not match") } - return rec, dataKey, nil + return rec, dataKey, readTS, nil } // ------------------------ small helpers ------------------------ From 13e75f4e93a8aeef9dea6abc70c5ed5b61f2f0fe Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 17:24:50 +0900 Subject: [PATCH 17/26] fix(sqs): pin catalog OCC StartTS to validated-read snapshot codex round 3 on #610 flagged the same StartTS-missing hole the message path already has fixed on the catalog CRUD path: - tryCreateQueueOnce reads queue absence + generation at readTS, then dispatches Put without StartTS or ReadKeys. Two races can both observe "queue missing" and both commit, and the coordinator auto-assigns StartTS so OCC never rejects the second one. - deleteQueueWithRetry and setQueueAttributesWithRetry have the same pattern (read at readTS, write with no StartTS/ReadKeys), which lets a concurrent create/delete/set clobber a just- committed change. Fix: carry readTS through to the dispatch as StartTS and add ReadKeys covering the meta and (for create/delete) generation records. The existing retry loops already handle ErrWriteConflict, so contention now retries instead of silently committing stale state. No behavior change at the single-writer level; the existing idempotent-create test and delete-generation-bump test still pass. --- adapter/sqs_catalog.go | 39 +++++++++++++++++++++++++++++++-------- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go index 010e96802..9abb74f1b 100644 --- a/adapter/sqs_catalog.go +++ b/adapter/sqs_catalog.go @@ -502,11 +502,22 @@ func (s *SQSServer) tryCreateQueueOnce(ctx context.Context, requested *sqsQueueM if err != nil { return false, errors.WithStack(err) } + metaKey := sqsQueueMetaKey(requested.Name) + genKey := sqsQueueGenKey(requested.Name) + // StartTS pins OCC to the snapshot we took the existence + generation + // read at, and ReadKeys cover both the meta and generation records so + // a concurrent CreateQueue that committed between our read and our + // dispatch causes ErrWriteConflict and the retry loop re-reads. + // Without this, two races could both decide "queue missing" and both + // write their own generation, leaving the later write on top of a + // record the loser never observed. req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{metaKey, genKey}, Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Put, Key: sqsQueueMetaKey(requested.Name), Value: metaBytes}, - {Op: kv.Put, Key: sqsQueueGenKey(requested.Name), Value: []byte(strconv.FormatUint(requested.Generation, 10))}, + {Op: kv.Put, Key: metaKey, Value: metaBytes}, + {Op: kv.Put, Key: genKey, Value: []byte(strconv.FormatUint(requested.Generation, 10))}, }, } if _, err := s.coordinator.Dispatch(ctx, req); err != nil { @@ -554,11 +565,17 @@ func (s *SQSServer) deleteQueueWithRetry(ctx context.Context, queueName string) if err != nil { return errors.WithStack(err) } + metaKey := sqsQueueMetaKey(queueName) + genKey := sqsQueueGenKey(queueName) + // StartTS + ReadKeys fence against a concurrent CreateQueue / + // SetQueueAttributes landing between our load and dispatch. req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{metaKey, genKey}, Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Del, Key: sqsQueueMetaKey(queueName)}, - {Op: kv.Put, Key: sqsQueueGenKey(queueName), Value: []byte(strconv.FormatUint(lastGen+1, 10))}, + {Op: kv.Del, Key: metaKey}, + {Op: kv.Put, Key: genKey, Value: []byte(strconv.FormatUint(lastGen+1, 10))}, }, } if _, err := s.coordinator.Dispatch(ctx, req); err == nil { @@ -812,10 +829,16 @@ func (s *SQSServer) setQueueAttributesWithRetry(ctx context.Context, queueName s if err != nil { return errors.WithStack(err) } + metaKey := sqsQueueMetaKey(queueName) + // StartTS + ReadKeys prevent two concurrent SetQueueAttributes + // from both reading the same old meta and the later dispatch + // clobbering the earlier commit's changes. req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{metaKey}, Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Put, Key: sqsQueueMetaKey(queueName), Value: metaBytes}, + {Op: kv.Put, Key: metaKey, Value: metaBytes}, }, } if _, err := s.coordinator.Dispatch(ctx, req); err == nil { From 3b1d429f8dd96d998deaee565c64b454792824ed Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 18:15:08 +0900 Subject: [PATCH 18/26] fix(sqs): implement long polling + require explicit FifoQueue=true MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit codex round 8 on #610 flagged two findings, both addressed here: - P1 sqs_messages.go: ReceiveMessage parsed WaitTimeSeconds but never used it, so clients asking for long polling got an immediate empty return and hot-polled the server. Milestone 1 now does a polling-based long poll: if the first scan returns nothing and WaitTimeSeconds > 0, the handler re-scans every 200 ms until a message arrives, the wait elapses, or the request context is canceled. The full commit-stream notifier from §7.3 of the design is still deferred, but the observable client contract now matches AWS. resolveReceiveWaitSeconds applies per-request value -> queue default -> clamp to [0, 20]. - P2 sqs_catalog.go: IsFIFO was pre-initialized from the .fifo name suffix before attribute validation ran, so a CreateQueue without FifoQueue=true silently became a FIFO queue. AWS requires FifoQueue=true to be explicitly set. IsFIFO now defaults to false; a .fifo-suffixed name without the attribute returns InvalidParameterValue ("FIFO queue name requires FifoQueue=true attribute"). New tests: TestSQSServer_LongPollWaitsForArrival schedules a send mid-wait and asserts the receive returns promptly after arrival (and waited at all); TestSQSServer_LongPollTimesOutOnEmptyQueue asserts WaitTimeSeconds on an empty queue returns empty at approximately the requested time. --- adapter/sqs_catalog.go | 19 ++++++--- adapter/sqs_messages.go | 75 +++++++++++++++++++++++++++++++++--- adapter/sqs_messages_test.go | 74 +++++++++++++++++++++++++++++++++++ 3 files changed, 157 insertions(+), 11 deletions(-) diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go index 9abb74f1b..b6e28a207 100644 --- a/adapter/sqs_catalog.go +++ b/adapter/sqs_catalog.go @@ -242,9 +242,16 @@ func validateQueueName(name string) error { // ------------------------ attribute parsing ------------------------ func parseAttributesIntoMeta(name string, attrs map[string]string) (*sqsQueueMeta, error) { + // AWS SQS requires FifoQueue=true to be explicitly set to create a + // FIFO queue; the .fifo suffix alone is not enough. IsFIFO therefore + // defaults to false and is only flipped on when the attribute is + // present and true. A .fifo-suffixed name without the attribute is + // an error (InvalidAttributeValue), because silently creating a + // Standard queue with a .fifo suffix would later break real FIFO + // clients that address the queue. meta := &sqsQueueMeta{ Name: name, - IsFIFO: strings.HasSuffix(name, sqsFIFOQueueNameSuffix), + IsFIFO: false, VisibilityTimeoutSeconds: sqsDefaultVisibilityTimeoutSeconds, MessageRetentionSeconds: sqsDefaultRetentionSeconds, DelaySeconds: sqsDefaultDelaySeconds, @@ -254,21 +261,21 @@ func parseAttributesIntoMeta(name string, attrs map[string]string) (*sqsQueueMet if err := applyAttributes(meta, attrs); err != nil { return nil, err } - // FifoQueue attribute is authoritative if explicitly set; otherwise the - // .fifo suffix implies true and callers without the suffix get a - // standard queue. + nameHasFIFOSuffix := strings.HasSuffix(name, sqsFIFOQueueNameSuffix) if v, ok := attrs["FifoQueue"]; ok { b, err := strconv.ParseBool(v) if err != nil { return nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "FifoQueue must be a boolean") } - if b && !strings.HasSuffix(name, sqsFIFOQueueNameSuffix) { + if b && !nameHasFIFOSuffix { return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name must end in .fifo") } - if !b && strings.HasSuffix(name, sqsFIFOQueueNameSuffix) { + if !b && nameHasFIFOSuffix { return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "Queue name ends in .fifo but FifoQueue=false") } meta.IsFIFO = b + } else if nameHasFIFOSuffix { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name requires FifoQueue=true attribute") } return meta, nil } diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index fe8ba5443..220898b22 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -37,6 +37,12 @@ const ( sqsReceiveScanOverfetchFactor = 2 sqsChangeVisibilityMaxSeconds = sqsMaxVisibilityTimeoutSeconds sqsVisScanPageLimit = 1024 + // sqsLongPollInterval is how often the poll loop re-scans the + // visibility index when no messages were deliverable on the first + // scan. 200 ms is small enough that a ~20 s WaitTimeSeconds still + // has <1% tail-latency overhead, large enough that an empty queue + // does not spin. + sqsLongPollInterval = 200 * time.Millisecond // Version byte prefixed to encoded receipt handles. Bumped when the // on-wire handle format changes so old handles fail to decode loudly. sqsReceiptHandleVersion = byte(0x01) @@ -407,14 +413,73 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { } visibilityTimeout = *in.VisibilityTimeout } + waitSeconds := resolveReceiveWaitSeconds(in.WaitTimeSeconds, meta.ReceiveMessageWaitSeconds) - candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, meta.Generation, max*sqsReceiveScanOverfetchFactor, readTS) + delivered := s.longPollReceive(ctx, queueName, meta.Generation, max, visibilityTimeout, waitSeconds) + writeSQSJSON(w, map[string]any{"Messages": delivered}) +} + +// resolveReceiveWaitSeconds picks the effective long-poll duration: the +// per-request WaitTimeSeconds if provided, else the queue default. AWS +// permits 0..20; values outside the range are clamped so a malformed +// client request does not stall the server. +func resolveReceiveWaitSeconds(requested *int64, queueDefault int64) int64 { + var w int64 + if requested != nil { + w = *requested + } else { + w = queueDefault + } + if w < 0 { + w = 0 + } + if w > sqsMaxReceiveMessageWaitSeconds { + w = sqsMaxReceiveMessageWaitSeconds + } + return w +} + +// longPollReceive performs one scan+rotate attempt; if it returned 0 +// messages and the caller asked to wait, it polls the visibility index +// on a fixed interval until a message arrives, WaitTimeSeconds elapses, +// or the request context is canceled. Milestone 1 uses polling rather +// than the commit-stream notifier described in §7.3 of the design; the +// poll interval is short enough (200 ms) to mask the difference for +// typical client-side WaitTimeSeconds values. +func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout, waitSeconds int64) []map[string]any { + delivered := s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) + if len(delivered) > 0 || waitSeconds <= 0 { + return delivered + } + deadline := time.Now().Add(time.Duration(waitSeconds) * time.Second) + ticker := time.NewTicker(sqsLongPollInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return delivered + case <-ticker.C: + } + if time.Now().After(deadline) { + return delivered + } + delivered = s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) + if len(delivered) > 0 { + return delivered + } + } +} + +// scanAndDeliverOnce is the single-pass scan+rotate the long-poll loop +// re-runs. Each pass takes its own snapshot so the OCC StartTS tracks +// the most recent visible_at for the candidates it picked. +func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout int64) []map[string]any { + readTS := s.nextTxnReadTS(ctx) + candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, gen, max*sqsReceiveScanOverfetchFactor, readTS) if err != nil { - writeSQSErrorFromErr(w, err) - return + return nil } - delivered := s.rotateMessagesForDelivery(ctx, queueName, meta.Generation, candidates, visibilityTimeout, max, readTS) - writeSQSJSON(w, map[string]any{"Messages": delivered}) + return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS) } func clampReceiveMaxMessages(requested int) int { diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go index cc86b6db9..b087f9a64 100644 --- a/adapter/sqs_messages_test.go +++ b/adapter/sqs_messages_test.go @@ -377,6 +377,80 @@ func TestSQSServer_ChangeMessageVisibility(t *testing.T) { } } +func TestSQSServer_LongPollWaitsForArrival(t *testing.T) { + t.Parallel() + // WaitTimeSeconds > 0 on an empty queue must block on the server up + // to the requested duration. If a message lands mid-wait, the + // receive returns it promptly instead of after the full wait. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "longpoll-arrival") + + // Schedule a send mid-wait. + sendAt := time.AfterFunc(400*time.Millisecond, func() { + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "late", + }) + }) + defer sendAt.Stop() + + start := time.Now() + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "WaitTimeSeconds": 5, + }) + elapsed := time.Since(start) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + msgs, _ := out["Messages"].([]any) + if len(msgs) != 1 { + t.Fatalf("got %d messages, want 1 (elapsed %v)", len(msgs), elapsed) + } + // Must have actually waited (the send was scheduled 400ms out) but + // must not have waited the full 5 s — the long-poll loop must + // return soon after the message arrives. + if elapsed < 300*time.Millisecond { + t.Fatalf("returned too quickly (%v); long poll should have waited for the send", elapsed) + } + if elapsed > 3*time.Second { + t.Fatalf("returned too slowly (%v); long poll should have returned soon after arrival", elapsed) + } +} + +func TestSQSServer_LongPollTimesOutOnEmptyQueue(t *testing.T) { + t.Parallel() + // WaitTimeSeconds on an empty queue with no arrival must return + // empty after approximately the requested duration. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "longpoll-empty") + + start := time.Now() + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MaxNumberOfMessages": 1, + "WaitTimeSeconds": 1, + }) + elapsed := time.Since(start) + if status != http.StatusOK { + t.Fatalf("receive: %d %v", status, out) + } + if msgs, _ := out["Messages"].([]any); len(msgs) != 0 { + t.Fatalf("expected 0 messages on empty poll, got %d", len(msgs)) + } + if elapsed < 900*time.Millisecond { + t.Fatalf("returned before WaitTimeSeconds elapsed (%v)", elapsed) + } + if elapsed > 2500*time.Millisecond { + t.Fatalf("returned long after WaitTimeSeconds elapsed (%v)", elapsed) + } +} + func TestSQSServer_ReceiptHandleCodecRoundTrip(t *testing.T) { t.Parallel() for _, tc := range []struct { From 0d69c90a8d524403c2448e8c433db2d2264b8130 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 18:32:31 +0900 Subject: [PATCH 19/26] fix(sqs): address codex round 9 findings Three P1s: - sqs_messages.go: scanAndDeliverOnce was swallowing scan errors and returning nil, which the handler serialized as a normal 200 with an empty Messages list. A backend / routing / storage failure would silently stall consumers instead of surfacing. Now scanAndDeliverOnce returns (result, error), longPollReceive propagates the error, and receiveMessage returns an actionable 5xx via writeSQSErrorFromErr. - sqs_messages.go: SendMessage now validates FIFO-specific fields. FIFO queues REQUIRE MessageGroupId, and without ContentBasedDeduplication they REQUIRE MessageDeduplicationId too. Standard queues REJECT both fields. Per-message DelaySeconds is rejected on FIFO (AWS only accepts queue-level delay on FIFO). The record struct and input schema now carry MessageGroupId / MessageDeduplicationId so Milestone 2's group lock has the data it needs. - kv/shard_key.go: added sqsRouteKey so !sqs|... internal keys normalize to a stable !sqs|route|global prefix. Without this, multi-shard deployments with shardRanges defined around user- key ranges could leave SQS internal keys unroutable and fail every queue mutation with "no route for key". Milestone 1 collapses all SQS keys to one route (single control-plane group); per-queue routing lands with FIFO in Milestone 2. Tests: TestSQSServer_SendFIFOValidation exercises every branch of validateSendFIFOParams; TestSQSServer_CreateQueueRequiresExplicitFifoFlag asserts that a .fifo suffix alone is rejected when FifoQueue=true is not set. --- adapter/sqs_messages.go | 141 ++++++++++++++++++++++++++--------- adapter/sqs_messages_test.go | 85 +++++++++++++++++++++ kv/shard_key.go | 36 +++++++++ 3 files changed, 225 insertions(+), 37 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 220898b22..165964935 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -67,18 +67,20 @@ const ( // (VisibleAtMillis, CurrentReceiptToken, ReceiveCount) lives here rather // than in a side-record so a single OCC transaction can rotate it. type sqsMessageRecord struct { - MessageID string `json:"message_id"` - Body []byte `json:"body"` - MD5OfBody string `json:"md5_of_body"` - MessageAttributes map[string]string `json:"message_attributes,omitempty"` - SenderID string `json:"sender_id,omitempty"` - SendTimestampMillis int64 `json:"send_timestamp_millis"` - AvailableAtMillis int64 `json:"available_at_millis"` - VisibleAtMillis int64 `json:"visible_at_millis"` - ReceiveCount int64 `json:"receive_count"` - FirstReceiveMillis int64 `json:"first_receive_millis,omitempty"` - CurrentReceiptToken []byte `json:"current_receipt_token"` - QueueGeneration uint64 `json:"queue_generation"` + MessageID string `json:"message_id"` + Body []byte `json:"body"` + MD5OfBody string `json:"md5_of_body"` + MessageAttributes map[string]string `json:"message_attributes,omitempty"` + SenderID string `json:"sender_id,omitempty"` + SendTimestampMillis int64 `json:"send_timestamp_millis"` + AvailableAtMillis int64 `json:"available_at_millis"` + VisibleAtMillis int64 `json:"visible_at_millis"` + ReceiveCount int64 `json:"receive_count"` + FirstReceiveMillis int64 `json:"first_receive_millis,omitempty"` + CurrentReceiptToken []byte `json:"current_receipt_token"` + QueueGeneration uint64 `json:"queue_generation"` + MessageGroupId string `json:"message_group_id,omitempty"` + MessageDeduplicationId string `json:"message_deduplication_id,omitempty"` } var storedSQSMsgPrefix = []byte{0x00, 'S', 'M', 0x01} @@ -241,10 +243,12 @@ func decodeReceiptHandle(raw string) (*decodedReceiptHandle, error) { // ------------------------ input decoding ------------------------ type sqsSendMessageInput struct { - QueueUrl string `json:"QueueUrl"` - MessageBody string `json:"MessageBody"` - DelaySeconds *int64 `json:"DelaySeconds,omitempty"` - MessageAttributes map[string]string `json:"MessageAttributes,omitempty"` + QueueUrl string `json:"QueueUrl"` + MessageBody string `json:"MessageBody"` + DelaySeconds *int64 `json:"DelaySeconds,omitempty"` + MessageAttributes map[string]string `json:"MessageAttributes,omitempty"` + MessageGroupId string `json:"MessageGroupId,omitempty"` + MessageDeduplicationId string `json:"MessageDeduplicationId,omitempty"` } type sqsReceiveMessageInput struct { @@ -283,6 +287,10 @@ func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { writeSQSErrorFromErr(w, apiErr) return } + if apiErr := validateSendFIFOParams(meta, in); apiErr != nil { + writeSQSErrorFromErr(w, apiErr) + return + } delay, apiErr := resolveSendDelay(meta, in.DelaySeconds) if apiErr != nil { writeSQSErrorFromErr(w, apiErr) @@ -330,6 +338,47 @@ func (s *SQSServer) loadQueueMetaForSend(ctx context.Context, queueName string, return meta, nil } +// validateSendFIFOParams enforces the AWS-compatible rules around +// MessageGroupId and MessageDeduplicationId: +// +// - FIFO queues REQUIRE MessageGroupId on every send. +// - FIFO queues without ContentBasedDeduplication REQUIRE +// MessageDeduplicationId as well. +// - Standard queues REJECT both fields — accepting them silently +// would let misbehaving clients think they are getting FIFO +// semantics. +// - FIFO queues REJECT per-message DelaySeconds (already handled in +// resolveSendDelay below, but we also short-circuit it here so the +// error ordering matches AWS). +// +// Note: Milestone 1 does not yet enforce the per-group ordering / +// dedup invariants — the queue type gate is still useful so clients +// get the right AWS error shape, and the persisted MessageGroupId / +// MessageDeduplicationId fields are there for Milestone 2's group- +// lock implementation. +func validateSendFIFOParams(meta *sqsQueueMeta, in sqsSendMessageInput) error { + if meta.IsFIFO { + if in.MessageGroupId == "" { + return newSQSAPIError(http.StatusBadRequest, sqsErrMissingParameter, "FIFO queue requires MessageGroupId") + } + if !meta.ContentBasedDedup && in.MessageDeduplicationId == "" { + return newSQSAPIError(http.StatusBadRequest, sqsErrMissingParameter, "FIFO queue without ContentBasedDeduplication requires MessageDeduplicationId") + } + if in.DelaySeconds != nil { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "FIFO queue does not accept per-message DelaySeconds") + } + return nil + } + // Standard queue: both FIFO-only fields must be empty. + if in.MessageGroupId != "" { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "MessageGroupId is only valid on FIFO queues") + } + if in.MessageDeduplicationId != "" { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "MessageDeduplicationId is only valid on FIFO queues") + } + return nil +} + func resolveSendDelay(meta *sqsQueueMeta, requested *int64) (int64, error) { delay := meta.DelaySeconds if requested == nil { @@ -354,15 +403,17 @@ func buildSendRecord(meta *sqsQueueMeta, in sqsSendMessageInput, delay int64) (* availableAt := now + delay*sqsMillisPerSecond body := []byte(in.MessageBody) rec := &sqsMessageRecord{ - MessageID: messageID, - Body: body, - MD5OfBody: sqsMD5Hex(body), - MessageAttributes: in.MessageAttributes, - SendTimestampMillis: now, - AvailableAtMillis: availableAt, - VisibleAtMillis: availableAt, - CurrentReceiptToken: token, - QueueGeneration: meta.Generation, + MessageID: messageID, + Body: body, + MD5OfBody: sqsMD5Hex(body), + MessageAttributes: in.MessageAttributes, + SendTimestampMillis: now, + AvailableAtMillis: availableAt, + VisibleAtMillis: availableAt, + CurrentReceiptToken: token, + QueueGeneration: meta.Generation, + MessageGroupId: in.MessageGroupId, + MessageDeduplicationId: in.MessageDeduplicationId, } recordBytes, err := encodeSQSMessageRecord(rec) if err != nil { @@ -415,7 +466,11 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { } waitSeconds := resolveReceiveWaitSeconds(in.WaitTimeSeconds, meta.ReceiveMessageWaitSeconds) - delivered := s.longPollReceive(ctx, queueName, meta.Generation, max, visibilityTimeout, waitSeconds) + delivered, err := s.longPollReceive(ctx, queueName, meta.Generation, max, visibilityTimeout, waitSeconds) + if err != nil { + writeSQSErrorFromErr(w, err) + return + } writeSQSJSON(w, map[string]any{"Messages": delivered}) } @@ -446,10 +501,17 @@ func resolveReceiveWaitSeconds(requested *int64, queueDefault int64) int64 { // than the commit-stream notifier described in §7.3 of the design; the // poll interval is short enough (200 ms) to mask the difference for // typical client-side WaitTimeSeconds values. -func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout, waitSeconds int64) []map[string]any { - delivered := s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) +// +// Scan errors are propagated to the caller so a backend / routing +// failure surfaces as an actionable 5xx instead of a silent empty 200 +// that would stall consumers. +func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout, waitSeconds int64) ([]map[string]any, error) { + delivered, err := s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) + if err != nil { + return nil, err + } if len(delivered) > 0 || waitSeconds <= 0 { - return delivered + return delivered, nil } deadline := time.Now().Add(time.Duration(waitSeconds) * time.Second) ticker := time.NewTicker(sqsLongPollInterval) @@ -457,29 +519,34 @@ func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen u for { select { case <-ctx.Done(): - return delivered + return delivered, nil case <-ticker.C: } if time.Now().After(deadline) { - return delivered + return delivered, nil + } + delivered, err = s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) + if err != nil { + return nil, err } - delivered = s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) if len(delivered) > 0 { - return delivered + return delivered, nil } } } // scanAndDeliverOnce is the single-pass scan+rotate the long-poll loop // re-runs. Each pass takes its own snapshot so the OCC StartTS tracks -// the most recent visible_at for the candidates it picked. -func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout int64) []map[string]any { +// the most recent visible_at for the candidates it picked. Scan errors +// are returned so the caller can fail the receive with an actionable +// status code instead of serializing them as empty success. +func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout int64) ([]map[string]any, error) { readTS := s.nextTxnReadTS(ctx) candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, gen, max*sqsReceiveScanOverfetchFactor, readTS) if err != nil { - return nil + return nil, err } - return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS) + return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS), nil } func clampReceiveMaxMessages(requested int) int { diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go index b087f9a64..7ae89136c 100644 --- a/adapter/sqs_messages_test.go +++ b/adapter/sqs_messages_test.go @@ -451,6 +451,91 @@ func TestSQSServer_LongPollTimesOutOnEmptyQueue(t *testing.T) { } } +func TestSQSServer_SendFIFOValidation(t *testing.T) { + t.Parallel() + // FIFO queues require MessageGroupId, and without + // ContentBasedDeduplication they also require + // MessageDeduplicationId. Standard queues must reject both fields. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + // FIFO queue, ContentBasedDeduplication=false (default). + _, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "fifo-send.fifo", + "Attributes": map[string]string{"FifoQueue": "true"}, + }) + fifoURL, _ := out["QueueUrl"].(string) + + // Missing MessageGroupId must fail. + status, body := callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": fifoURL, + "MessageBody": "x", + }) + if status != http.StatusBadRequest { + t.Fatalf("FIFO send without group: got %d want 400 (%v)", status, body) + } + if body["__type"] != sqsErrMissingParameter { + t.Fatalf("error type: %q want %q", body["__type"], sqsErrMissingParameter) + } + + // MessageGroupId but no dedup id (and no content-based dedup) + // must also fail. + status, body = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": fifoURL, + "MessageBody": "x", + "MessageGroupId": "g1", + }) + if status != http.StatusBadRequest { + t.Fatalf("FIFO send without dedup: got %d (%v)", status, body) + } + + // Both fields present → success. + status, body = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": fifoURL, + "MessageBody": "x", + "MessageGroupId": "g1", + "MessageDeduplicationId": "d1", + }) + if status != http.StatusOK { + t.Fatalf("valid FIFO send: got %d (%v)", status, body) + } + + // Standard queue rejects FIFO-only fields. + stdURL := createSQSQueueForTest(t, node, "std-send") + status, body = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": stdURL, + "MessageBody": "x", + "MessageGroupId": "g1", + }) + if status != http.StatusBadRequest { + t.Fatalf("Standard send with MessageGroupId: got %d (%v)", status, body) + } + if body["__type"] != sqsErrInvalidAttributeValue { + t.Fatalf("error type: %q want %q", body["__type"], sqsErrInvalidAttributeValue) + } +} + +func TestSQSServer_CreateQueueRequiresExplicitFifoFlag(t *testing.T) { + t.Parallel() + // AWS requires FifoQueue=true to be explicitly set to create a + // FIFO queue. A .fifo-suffixed name without the attribute must + // fail, not silently create a FIFO queue. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "implicit-fifo.fifo", + }) + if status != http.StatusBadRequest { + t.Fatalf(".fifo name without FifoQueue=true: got %d (%v)", status, out) + } + if out["__type"] != sqsErrValidation { + t.Fatalf("error type: %q want %q", out["__type"], sqsErrValidation) + } +} + func TestSQSServer_ReceiptHandleCodecRoundTrip(t *testing.T) { t.Parallel() for _, tc := range []struct { diff --git a/kv/shard_key.go b/kv/shard_key.go index 82a028746..6871ce789 100644 --- a/kv/shard_key.go +++ b/kv/shard_key.go @@ -24,12 +24,28 @@ const ( DynamoGSIPrefix = "!ddb|gsi|" ) +const ( + // SqsRoutePrefix is the logical route prefix all SQS internal keys + // normalize to. The adapter stores queue metadata and per-queue + // message keys under several !sqs|... families; routing must map + // every one of them to a single stable prefix so the distribution + // engine colocates a queue's storage on one Raft group. + sqsRoutePrefix = "!sqs|route|" + + // SqsInternalPrefix is the shared prefix of every SQS-owned key + // family (!sqs|queue|meta|, !sqs|msg|vis|, etc.). Used by + // sqsRouteKey to dispatch the routing decision. + sqsInternalPrefix = "!sqs|" +) + var ( dynamoRoutePrefixBytes = []byte(dynamoRoutePrefix) dynamoTableMetaPrefixBytes = []byte(DynamoTableMetaPrefix) dynamoTableGenerationPrefixBytes = []byte(DynamoTableGenerationPrefix) dynamoItemPrefixBytes = []byte(DynamoItemPrefix) dynamoGSIPrefixBytes = []byte(DynamoGSIPrefix) + sqsRoutePrefixBytes = []byte(sqsRoutePrefix) + sqsInternalPrefixBytes = []byte(sqsInternalPrefix) ) // routeKey normalizes internal keys (e.g., list metadata/items) to the logical @@ -51,6 +67,9 @@ func normalizeRouteKey(key []byte) []byte { if table := dynamoRouteKey(key); table != nil { return table } + if route := sqsRouteKey(key); route != nil { + return route + } if user := s3keys.ExtractRouteKey(key); user != nil { return user } @@ -104,3 +123,20 @@ func dynamoRouteTableKey(tableSegment []byte) []byte { out = append(out, tableSegment...) return out } + +// sqsRouteKey maps any !sqs|... internal key to a stable route key so +// multi-shard deployments that partition by user-key range still land +// every SQS mutation on a configured group. Milestone 1 collapses all +// SQS keys to a single !sqs|route|global route — this keeps the +// catalog and every queue's message keyspace on the same group, which +// is the minimum needed for FIFO group-lock semantics (landing later) +// to work. When per-queue sharding is implemented it will live here. +func sqsRouteKey(key []byte) []byte { + if !bytes.HasPrefix(key, sqsInternalPrefixBytes) { + return nil + } + out := make([]byte, 0, len(sqsRoutePrefixBytes)+len("global")) + out = append(out, sqsRoutePrefixBytes...) + out = append(out, []byte("global")...) + return out +} From 698270e444097816d210cba931d31b59b136d245 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Fri, 24 Apr 2026 20:05:37 +0900 Subject: [PATCH 20/26] fix(sqs): address codex + coderabbit round 10 findings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 7 findings across two reviews; all addressed: codex P2: - sqs_messages.go DeleteMessage: loadMessageForDelete no longer treats a cross-queue or deleted-queue handle as a silent no-op. We first resolve the queue by the supplied QueueUrl; missing queue returns AWS.SimpleQueueService.NonExistentQueue; generation mismatch (handle belongs to a prior/different queue incarnation) returns ReceiptHandleIsInvalid. Only same-queue same-generation handles qualify for the AWS-compat stale-handle no-op. - sqs_messages.go ReceiveMessage: resolveReceiveWaitSeconds now rejects out-of-range per-request WaitTimeSeconds with InvalidParameterValue instead of silently clamping; queue-level default is trusted because it was validated at SetQueueAttributes time. - sqs_catalog.go: ContentBasedDeduplication is FIFO-only. Rejected on both CreateQueue and SetQueueAttributes when the queue is not FIFO. resolveFifoQueueFlag extracted so parseAttributesIntoMeta stays below the cyclomatic limit. coderabbit Minor: - sqs_catalog.go writeSQSErrorFromErr: internal errors no longer echo err.Error() (which can wrap Pebble paths, Raft peer ids, and stack frames). Raw detail goes to slog, clients see a generic 500 "internal error". - sqs_messages.go ChangeMessageVisibility: VisibilityTimeout is now *int64 so a missing field returns MissingParameter per AWS, rather than silently meaning "make visible immediately". - sqs_messages.go SendMessage: reject non-empty MessageAttributes with InvalidParameterValue until Milestone 2 ships the canonical AWS MD5-of-attributes encoder. The current hand-rolled hash would produce a non-canonical MD5 and every AWS SDK client would fail SendMessage with MessageAttributeMD5Mismatch — better to fail clearly at the caller with a descriptive error. - sqs_messages_test.go: TestSQSServer_LongPollWaitsForArrival's time.AfterFunc now coordinates with a sync.WaitGroup so the scheduled callSQS cannot call t.Fatalf after the test has returned (go test flags that as misuse). --- adapter/sqs_catalog.go | 132 +++++++++++++++++++++++------------ adapter/sqs_messages.go | 86 +++++++++++++++++------ adapter/sqs_messages_test.go | 18 ++++- 3 files changed, 167 insertions(+), 69 deletions(-) diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go index b6e28a207..0e5f91106 100644 --- a/adapter/sqs_catalog.go +++ b/adapter/sqs_catalog.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "io" + "log/slog" "net/http" "net/url" "regexp" @@ -124,7 +125,14 @@ func writeSQSErrorFromErr(w http.ResponseWriter, err error) { writeSQSError(w, apiErr.status, apiErr.errorType, apiErr.message) return } - writeSQSError(w, http.StatusInternalServerError, sqsErrInternalFailure, err.Error()) + // Internal errors can wrap Pebble file names, Raft peer ids, stack + // frames from errors.WithStack, etc. Never echo the raw error text + // to the client — an authenticated-but-untrusted caller could use + // it to fingerprint the deployment. Log the detail server-side + // (the request metrics / log pipeline already captures the full + // chain) and return a generic 500 body. + slog.Error("sqs adapter internal error", "err", err) + writeSQSError(w, http.StatusInternalServerError, sqsErrInternalFailure, "internal error") } func writeSQSJSON(w http.ResponseWriter, payload any) { @@ -261,23 +269,40 @@ func parseAttributesIntoMeta(name string, attrs map[string]string) (*sqsQueueMet if err := applyAttributes(meta, attrs); err != nil { return nil, err } + if err := resolveFifoQueueFlag(meta, name, attrs); err != nil { + return nil, err + } + if meta.ContentBasedDedup && !meta.IsFIFO { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "ContentBasedDeduplication is only valid on FIFO queues") + } + return meta, nil +} + +// resolveFifoQueueFlag reconciles the FifoQueue attribute with the +// queue name's .fifo suffix. AWS requires both to match, and the +// attribute must be explicitly true for a FIFO queue — the name suffix +// alone is not enough. +func resolveFifoQueueFlag(meta *sqsQueueMeta, name string, attrs map[string]string) error { nameHasFIFOSuffix := strings.HasSuffix(name, sqsFIFOQueueNameSuffix) - if v, ok := attrs["FifoQueue"]; ok { - b, err := strconv.ParseBool(v) - if err != nil { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "FifoQueue must be a boolean") - } - if b && !nameHasFIFOSuffix { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name must end in .fifo") + v, ok := attrs["FifoQueue"] + if !ok { + if nameHasFIFOSuffix { + return newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name requires FifoQueue=true attribute") } - if !b && nameHasFIFOSuffix { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "Queue name ends in .fifo but FifoQueue=false") - } - meta.IsFIFO = b - } else if nameHasFIFOSuffix { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name requires FifoQueue=true attribute") + return nil } - return meta, nil + b, err := strconv.ParseBool(v) + if err != nil { + return newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "FifoQueue must be a boolean") + } + if b && !nameHasFIFOSuffix { + return newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "FIFO queue name must end in .fifo") + } + if !b && nameHasFIFOSuffix { + return newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "Queue name ends in .fifo but FifoQueue=false") + } + meta.IsFIFO = b + return nil } // attributeApplier writes one attribute into meta. Keeping one applier per @@ -821,38 +846,13 @@ func (s *SQSServer) setQueueAttributesWithRetry(ctx context.Context, queueName s backoff := transactRetryInitialBackoff deadline := time.Now().Add(transactRetryMaxDuration) for range transactRetryMaxAttempts { - readTS := s.nextTxnReadTS(ctx) - meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) - if err != nil { - return errors.WithStack(err) - } - if !exists { - return newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + done, err := s.trySetQueueAttributesOnce(ctx, queueName, attrs) + if err == nil && done { + return nil } - if err := applyAttributes(meta, attrs); err != nil { + if err != nil && !isRetryableTransactWriteError(err) { return err } - metaBytes, err := encodeSQSQueueMeta(meta) - if err != nil { - return errors.WithStack(err) - } - metaKey := sqsQueueMetaKey(queueName) - // StartTS + ReadKeys prevent two concurrent SetQueueAttributes - // from both reading the same old meta and the later dispatch - // clobbering the earlier commit's changes. - req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, - StartTS: readTS, - ReadKeys: [][]byte{metaKey}, - Elems: []*kv.Elem[kv.OP]{ - {Op: kv.Put, Key: metaKey, Value: metaBytes}, - }, - } - if _, err := s.coordinator.Dispatch(ctx, req); err == nil { - return nil - } else if !isRetryableTransactWriteError(err) { - return errors.WithStack(err) - } if err := waitRetryWithDeadline(ctx, deadline, backoff); err != nil { return errors.WithStack(err) } @@ -860,3 +860,47 @@ func (s *SQSServer) setQueueAttributesWithRetry(ctx context.Context, queueName s } return newSQSAPIError(http.StatusInternalServerError, sqsErrInternalFailure, "set queue attributes retry attempts exhausted") } + +// trySetQueueAttributesOnce is one read-validate-commit pass. The first +// return reports whether the caller should stop retrying (the attrs +// are now committed); an error means either a non-retryable failure +// (propagate) or a retryable write conflict (retry after backoff). +func (s *SQSServer) trySetQueueAttributesOnce(ctx context.Context, queueName string, attrs map[string]string) (bool, error) { + readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return false, errors.WithStack(err) + } + if !exists { + return false, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + if err := applyAttributes(meta, attrs); err != nil { + return false, err + } + // ContentBasedDeduplication is FIFO-only; a Standard queue + // silently accepting it would advertise unsupported behavior to + // clients. Same rule enforced on CreateQueue. + if meta.ContentBasedDedup && !meta.IsFIFO { + return false, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "ContentBasedDeduplication is only valid on FIFO queues") + } + metaBytes, err := encodeSQSQueueMeta(meta) + if err != nil { + return false, errors.WithStack(err) + } + metaKey := sqsQueueMetaKey(queueName) + // StartTS + ReadKeys prevent two concurrent SetQueueAttributes from + // both reading the same old meta and the later dispatch clobbering + // the earlier commit's changes. + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{metaKey}, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Put, Key: metaKey, Value: metaBytes}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err != nil { + return false, errors.WithStack(err) + } + return true, nil +} diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 165964935..d6727cfb6 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -266,7 +266,7 @@ type sqsDeleteMessageInput struct { type sqsChangeVisibilityInput struct { QueueUrl string `json:"QueueUrl"` ReceiptHandle string `json:"ReceiptHandle"` - VisibilityTimeout int64 `json:"VisibilityTimeout"` + VisibilityTimeout *int64 `json:"VisibilityTimeout"` } // ------------------------ handlers ------------------------ @@ -287,6 +287,19 @@ func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { writeSQSErrorFromErr(w, apiErr) return } + // AWS SDKs verify MD5OfMessageAttributes against the canonical + // binary encoding (sorted, length-prefixed, with transport type + // byte). The Milestone-1 adapter does not yet implement that + // canonical hash, and a non-matching value would make every SDK + // SendMessage call fail with MessageAttributeMD5Mismatch. Until + // Milestone 2 ships the canonical encoder, reject sends that + // actually carry MessageAttributes so clients fail clearly at + // the caller instead of mysteriously in the SDK. + if len(in.MessageAttributes) > 0 { + writeSQSError(w, http.StatusBadRequest, sqsErrInvalidAttributeValue, + "MessageAttributes are not yet supported; omit the field until canonical MD5 lands") + return + } if apiErr := validateSendFIFOParams(meta, in); apiErr != nil { writeSQSErrorFromErr(w, apiErr) return @@ -464,7 +477,11 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { } visibilityTimeout = *in.VisibilityTimeout } - waitSeconds := resolveReceiveWaitSeconds(in.WaitTimeSeconds, meta.ReceiveMessageWaitSeconds) + waitSeconds, waitErr := resolveReceiveWaitSeconds(in.WaitTimeSeconds, meta.ReceiveMessageWaitSeconds) + if waitErr != nil { + writeSQSErrorFromErr(w, waitErr) + return + } delivered, err := s.longPollReceive(ctx, queueName, meta.Generation, max, visibilityTimeout, waitSeconds) if err != nil { @@ -476,22 +493,20 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { // resolveReceiveWaitSeconds picks the effective long-poll duration: the // per-request WaitTimeSeconds if provided, else the queue default. AWS -// permits 0..20; values outside the range are clamped so a malformed -// client request does not stall the server. -func resolveReceiveWaitSeconds(requested *int64, queueDefault int64) int64 { - var w int64 - if requested != nil { - w = *requested - } else { - w = queueDefault - } - if w < 0 { - w = 0 +// permits 0..20 and rejects anything outside with +// InvalidParameterValue; silently clamping a bad client value would +// mask bugs and change behavior (negative becomes immediate polling, +// oversized becomes long polls). The queue default is trusted — it was +// validated at SetQueueAttributes time. +func resolveReceiveWaitSeconds(requested *int64, queueDefault int64) (int64, error) { + if requested == nil { + return queueDefault, nil } - if w > sqsMaxReceiveMessageWaitSeconds { - w = sqsMaxReceiveMessageWaitSeconds + v := *requested + if v < 0 || v > sqsMaxReceiveMessageWaitSeconds { + return 0, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "WaitTimeSeconds must be between 0 and 20") } - return w + return v, nil } // longPollReceive performs one scan+rotate attempt; if it returned 0 @@ -769,13 +784,30 @@ const ( // loadMessageForDelete reads the message record and classifies the // outcome for AWS-compatible DeleteMessage semantics: structural errors -// propagate; missing records and token mismatches return -// sqsDeleteNoOp; matching tokens return sqsDeleteProceed with the -// loaded record. The readTS it took the snapshot at is returned so the -// caller can pass it as StartTS on the OCC dispatch, pinning the -// read-write conflict detection window. +// propagate; missing records and token mismatches on an otherwise-valid +// queue return sqsDeleteNoOp; matching tokens return sqsDeleteProceed +// with the loaded record. The readTS it took the snapshot at is +// returned so the caller can pass it as StartTS on the OCC dispatch, +// pinning the read-write conflict detection window. +// +// The caller-supplied QueueUrl is cross-checked against the handle's +// embedded queue_generation: if the queue does not exist or its current +// generation does not match the handle's generation, the handle refers +// to a different (or recreated) queue and we reject it as a structural +// error — silently succeeding would let misrouted deletes ack messages +// that cannot possibly be deleted on this queue. func (s *SQSServer) loadMessageForDelete(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, uint64, sqsDeleteOutcome, error) { readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return nil, nil, readTS, sqsDeleteProceed, errors.WithStack(err) + } + if !exists { + return nil, nil, readTS, sqsDeleteProceed, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + if meta.Generation != handle.QueueGeneration { + return nil, nil, readTS, sqsDeleteProceed, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "receipt handle does not belong to this queue") + } dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) raw, err := s.store.GetAt(ctx, dataKey, readTS) if err != nil { @@ -800,7 +832,15 @@ func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Reque writeSQSErrorFromErr(w, err) return } - if in.VisibilityTimeout < 0 || in.VisibilityTimeout > sqsChangeVisibilityMaxSeconds { + // AWS requires VisibilityTimeout on ChangeMessageVisibility — + // omitting it returns MissingParameter, not an implicit 0 (which + // would unconditionally make the message visible). + if in.VisibilityTimeout == nil { + writeSQSError(w, http.StatusBadRequest, sqsErrMissingParameter, "VisibilityTimeout is required") + return + } + timeout := *in.VisibilityTimeout + if timeout < 0 || timeout > sqsChangeVisibilityMaxSeconds { writeSQSError(w, http.StatusBadRequest, sqsErrInvalidAttributeValue, "VisibilityTimeout out of range") return } @@ -809,7 +849,7 @@ func (s *SQSServer) changeMessageVisibility(w http.ResponseWriter, r *http.Reque writeSQSErrorFromErr(w, err) return } - if err := s.changeVisibilityWithRetry(r.Context(), queueName, handle, in.VisibilityTimeout); err != nil { + if err := s.changeVisibilityWithRetry(r.Context(), queueName, handle, timeout); err != nil { writeSQSErrorFromErr(w, err) return } diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go index 7ae89136c..eb8f24014 100644 --- a/adapter/sqs_messages_test.go +++ b/adapter/sqs_messages_test.go @@ -4,6 +4,7 @@ import ( "encoding/hex" "net/http" "strconv" + "sync" "testing" "time" ) @@ -387,14 +388,27 @@ func TestSQSServer_LongPollWaitsForArrival(t *testing.T) { node := sqsLeaderNode(t, nodes) queueURL := createSQSQueueForTest(t, node, "longpoll-arrival") - // Schedule a send mid-wait. + // Schedule a send mid-wait. We track the timer goroutine with a + // WaitGroup so the test does not return while callSQS is still + // running — otherwise t.Fatalf inside callSQS could fire on a + // finished *testing.T (go test flags that as a misuse). + var sendWG sync.WaitGroup + sendWG.Add(1) sendAt := time.AfterFunc(400*time.Millisecond, func() { + defer sendWG.Done() _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ "QueueUrl": queueURL, "MessageBody": "late", }) }) - defer sendAt.Stop() + defer func() { + if sendAt.Stop() { + // Timer stopped before firing — func won't run, so balance + // the Add with a manual Done. + sendWG.Done() + } + sendWG.Wait() + }() start := time.Now() status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ From d2d5696cd1967a61f8dc808f38cd0d3d8191f499 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 00:36:56 +0900 Subject: [PATCH 21/26] fix(sqs): fence SendMessage and ChangeMessageVisibility against queue-state races MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two new codex findings on the "concurrent DeleteQueue race" axis: - SendMessage (P1): the transaction wrote data + vis but had no StartTS or ReadKeys. If DeleteQueue committed between loadQueueMetaForSend's read and this dispatch, the send still returned 200 and wrote the message under the old generation — which the generation bump had just made unreachable via routing, effectively silent acknowledged loss. loadQueueMetaForSend now returns the snapshot readTS alongside the metadata, and sendMessage pins StartTS=readTS and ReadKeys=[metaKey, genKey] on its dispatch. A concurrent DeleteQueue / PurgeQueue commits before ours → ErrWriteConflict, which surfaces to the client as a retryable error. - ChangeMessageVisibility (P2): loadAndVerifyMessage previously trusted handle.QueueGeneration and jumped straight to the message key. If the queue was deleted (or deleted-and-recreated with a fresh generation), a stale handle could still mutate the orphan in-flight record left under the old generation. loadAndVerifyMessage now mirrors loadMessageForDelete: it resolves queue meta first, rejects a missing queue with QueueDoesNotExist, and rejects a generation mismatch with ReceiptHandleIsInvalid. Matching generations then proceed exactly as before. No behavior change on the happy path. Single-writer tests still pass; the added checks only fire when a concurrent catalog mutation is in flight. --- adapter/sqs_messages.go | 48 +++++++++++++++++++++++++++++++++++------ 1 file changed, 41 insertions(+), 7 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index d6727cfb6..2f3362250 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -282,7 +282,7 @@ func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { writeSQSErrorFromErr(w, err) return } - meta, apiErr := s.loadQueueMetaForSend(r.Context(), queueName, []byte(in.MessageBody)) + meta, readTS, apiErr := s.loadQueueMetaForSend(r.Context(), queueName, []byte(in.MessageBody)) if apiErr != nil { writeSQSErrorFromErr(w, apiErr) return @@ -317,8 +317,20 @@ func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { dataKey := sqsMsgDataKey(queueName, meta.Generation, rec.MessageID) visKey := sqsMsgVisKey(queueName, meta.Generation, rec.AvailableAtMillis, rec.MessageID) + metaKey := sqsQueueMetaKey(queueName) + genKey := sqsQueueGenKey(queueName) + // StartTS + ReadKeys fence against a concurrent DeleteQueue / + // PurgeQueue / SetQueueAttributes that commits between our meta + // read and this dispatch. Without the fence, a DeleteQueue that + // bumps the generation would land first, and this send would then + // commit under the old generation — silently storing a message + // that is no longer reachable via routing (acknowledged loss). + // ErrWriteConflict surfaces via writeSQSErrorFromErr so clients + // retry against the current queue state. req := &kv.OperationGroup[kv.OP]{ - IsTxn: true, + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{metaKey, genKey}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Put, Key: dataKey, Value: recordBytes}, {Op: kv.Put, Key: visKey, Value: []byte(rec.MessageID)}, @@ -336,19 +348,24 @@ func (s *SQSServer) sendMessage(w http.ResponseWriter, r *http.Request) { }) } -func (s *SQSServer) loadQueueMetaForSend(ctx context.Context, queueName string, body []byte) (*sqsQueueMeta, error) { +// loadQueueMetaForSend reads the queue metadata and body-size-gates the +// send. Returns the snapshot read timestamp alongside the metadata so +// the caller can pin its OCC dispatch to it; without that fence a +// concurrent DeleteQueue / PurgeQueue could slip in between our read +// and the write, storing a message under a dead generation. +func (s *SQSServer) loadQueueMetaForSend(ctx context.Context, queueName string, body []byte) (*sqsQueueMeta, uint64, error) { readTS := s.nextTxnReadTS(ctx) meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) if err != nil { - return nil, errors.WithStack(err) + return nil, readTS, errors.WithStack(err) } if !exists { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + return nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") } if int64(len(body)) > meta.MaximumMessageSize { - return nil, newSQSAPIError(http.StatusBadRequest, sqsErrMessageTooLong, "message body exceeds MaximumMessageSize") + return nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrMessageTooLong, "message body exceeds MaximumMessageSize") } - return meta, nil + return meta, readTS, nil } // validateSendFIFOParams enforces the AWS-compatible rules around @@ -925,8 +942,25 @@ func (s *SQSServer) parseQueueAndReceipt(queueUrl, receiptHandle string) (string // Returns the record, its key, the snapshot timestamp the read ran at, // or a typed SQS error. Callers use the snapshot as StartTS on the // OCC dispatch so concurrent commits cannot slip past ReadKeys. +// +// The caller-supplied QueueUrl is cross-checked against the handle's +// embedded queue_generation, mirroring loadMessageForDelete: an +// existing DeleteQueue leaves orphan message keys until retention +// cleans them up, so a handle from a deleted / recreated queue must +// be rejected with ReceiptHandleIsInvalid instead of silently +// mutating the orphan record. func (s *SQSServer) loadAndVerifyMessage(ctx context.Context, queueName string, handle *decodedReceiptHandle) (*sqsMessageRecord, []byte, uint64, error) { readTS := s.nextTxnReadTS(ctx) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return nil, nil, readTS, errors.WithStack(err) + } + if !exists { + return nil, nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + if meta.Generation != handle.QueueGeneration { + return nil, nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrReceiptHandleInvalid, "receipt handle does not belong to this queue") + } dataKey := sqsMsgDataKey(queueName, handle.QueueGeneration, handle.MessageIDHex) raw, err := s.store.GetAt(ctx, dataKey, readTS) if err != nil { From b9d472711c10ed9eebd25bb94a2eb77af68c84ed Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 00:53:30 +0900 Subject: [PATCH 22/26] fix(sqs): address codex review 4171728561 (round 12) Three P2 findings on AWS-compat request-validation semantics: - ReceiveMessage.MaxNumberOfMessages: was silently clamping <=0 to 1 and >10 to 10, which hides caller bugs (0 becoming active polling, 11+ becoming 10). AWS rejects out-of-range with InvalidParameterValue. Input struct changed to *int so omission can still default to 1; explicit values must be in [1, 10] or the request fails. Same policy already applied to WaitTimeSeconds in round 10. - GetQueueAttributes: omitting AttributeNames was treated as "All", over-returning every queue attribute. AWS returns NO attributes on omission; "All" must be explicit. Replaced the map-or-nil signal with a tri-state sqsAttributeSelection { expandAll bool; names map[string]bool } so the semantics (empty / All / specific names) are each a distinct branch. - SetQueueAttributes.Attributes: was optional and a missing field silently 200'd with no change, hiding automation bugs. AWS marks the field as required. Omission (or an empty map) now returns MissingParameter. New tests: TestSQSServer_GetQueueAttributesOmittedReturnsEmpty asserts the two AWS cases (omitted -> empty, ["All"] -> full); TestSQSServer_SetQueueAttributesRequiresAttributes covers both the omitted-field and empty-map paths; TestSQSServer_ReceiveMessageRejectsOutOfRangeMax iterates {0, -1, 11, 100} expecting 400, plus the omitted-and-in-range happy paths. --- adapter/sqs_catalog.go | 53 ++++++++++++++----- adapter/sqs_catalog_test.go | 103 ++++++++++++++++++++++++++++++++++++ adapter/sqs_messages.go | 27 +++++++--- 3 files changed, 163 insertions(+), 20 deletions(-) diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go index 0e5f91106..e25e5b745 100644 --- a/adapter/sqs_catalog.go +++ b/adapter/sqs_catalog.go @@ -782,24 +782,45 @@ func (s *SQSServer) getQueueAttributes(w http.ResponseWriter, r *http.Request) { writeSQSJSON(w, map[string]any{"Attributes": attrs}) } -// selectedAttributeNames returns a set of attribute names to include in the -// response. An empty selection, or any entry equal to "All", expands to -// every supported attribute. -func selectedAttributeNames(req []string) map[string]bool { - selection := map[string]bool{} +// sqsAttributeSelection is a tri-state result from selectedAttributeNames: +// expandAll = AWS "All" (or any entry equals "All"); a non-nil map lists +// the specific attribute names the caller asked for; and an empty +// selection (no AttributeNames supplied at all) maps to "return no +// attributes" per AWS semantics — which is the opposite of treating +// omission as "All", and matters because real clients that omit +// AttributeNames specifically do NOT want the server to echo every +// piece of queue metadata back. +type sqsAttributeSelection struct { + expandAll bool + names map[string]bool +} + +// selectedAttributeNames parses the AttributeNames array according to +// AWS GetQueueAttributes semantics: +// - missing / empty array: return NO attributes (empty result). +// - any element equal to "All": return every supported attribute. +// - otherwise: return only the listed names. +func selectedAttributeNames(req []string) sqsAttributeSelection { if len(req) == 0 { - return nil + return sqsAttributeSelection{} } + selection := map[string]bool{} for _, n := range req { if n == "All" { - return nil + return sqsAttributeSelection{expandAll: true} } selection[n] = true } - return selection + return sqsAttributeSelection{names: selection} } -func queueMetaToAttributes(meta *sqsQueueMeta, selection map[string]bool) map[string]string { +func queueMetaToAttributes(meta *sqsQueueMeta, selection sqsAttributeSelection) map[string]string { + // No AttributeNames supplied and no "All" → AWS returns nothing. + // The handler still emits "Attributes" as an empty map so the + // response shape is stable. + if !selection.expandAll && len(selection.names) == 0 { + return map[string]string{} + } all := map[string]string{ "VisibilityTimeout": strconv.FormatInt(meta.VisibilityTimeoutSeconds, 10), "MessageRetentionPeriod": strconv.FormatInt(meta.MessageRetentionSeconds, 10), @@ -812,11 +833,11 @@ func queueMetaToAttributes(meta *sqsQueueMeta, selection map[string]bool) map[st if meta.RedrivePolicy != "" { all["RedrivePolicy"] = meta.RedrivePolicy } - if selection == nil { + if selection.expandAll { return all } - out := make(map[string]string, len(selection)) - for k := range selection { + out := make(map[string]string, len(selection.names)) + for k := range selection.names { if v, ok := all[k]; ok { out[k] = v } @@ -835,6 +856,14 @@ func (s *SQSServer) setQueueAttributes(w http.ResponseWriter, r *http.Request) { writeSQSErrorFromErr(w, err) return } + // AWS marks Attributes as a required parameter on SetQueueAttributes. + // Without this check, a client that forgets (or mis-serializes) the + // field gets a 200 success and no change — a silent failure that hides + // automation bugs. Reject omission with MissingParameter. + if len(in.Attributes) == 0 { + writeSQSError(w, http.StatusBadRequest, sqsErrMissingParameter, "Attributes is required") + return + } if err := s.setQueueAttributesWithRetry(r.Context(), name, in.Attributes); err != nil { writeSQSErrorFromErr(w, err) return diff --git a/adapter/sqs_catalog_test.go b/adapter/sqs_catalog_test.go index 3dac5fd54..e19ddab6c 100644 --- a/adapter/sqs_catalog_test.go +++ b/adapter/sqs_catalog_test.go @@ -292,3 +292,106 @@ func TestSQSServer_CatalogKeyEncoding(t *testing.T) { t.Fatal("queueNameFromMetaKey should reject non-catalog keys") } } + +func TestSQSServer_GetQueueAttributesOmittedReturnsEmpty(t *testing.T) { + t.Parallel() + // AWS semantics: an omitted AttributeNames list returns NO + // attributes, not every attribute. Callers that want "everything" + // must pass ["All"] explicitly. Over-returning metadata on + // omission (our previous behavior) would hand operators data + // they did not ask for and differ from real SQS. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + url := createSQSQueueForTest(t, node, "attrs-empty-select") + + // No AttributeNames → empty Attributes map. + status, out := callSQS(t, node, sqsGetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + }) + if status != http.StatusOK { + t.Fatalf("getAttrs omitted: %d %v", status, out) + } + attrs, _ := out["Attributes"].(map[string]any) + if len(attrs) != 0 { + t.Fatalf("omitted AttributeNames should return empty map; got %v", attrs) + } + + // Explicit All → every attribute. + status, out = callSQS(t, node, sqsGetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "AttributeNames": []string{"All"}, + }) + if status != http.StatusOK { + t.Fatalf("getAttrs All: %d %v", status, out) + } + attrs, _ = out["Attributes"].(map[string]any) + if attrs["VisibilityTimeout"] == nil { + t.Fatalf("All should include VisibilityTimeout; got %v", attrs) + } +} + +func TestSQSServer_SetQueueAttributesRequiresAttributes(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + url := createSQSQueueForTest(t, node, "setattrs-required") + + status, out := callSQS(t, node, sqsSetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + }) + if status != http.StatusBadRequest { + t.Fatalf("setAttrs without Attributes: got %d want 400 (%v)", status, out) + } + if got, _ := out["__type"].(string); got != sqsErrMissingParameter { + t.Fatalf("error type: %q want %q", got, sqsErrMissingParameter) + } + + // Empty Attributes map is also treated as omitted. + status, out = callSQS(t, node, sqsSetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "Attributes": map[string]string{}, + }) + if status != http.StatusBadRequest { + t.Fatalf("setAttrs empty Attributes: got %d want 400 (%v)", status, out) + } +} + +func TestSQSServer_ReceiveMessageRejectsOutOfRangeMax(t *testing.T) { + t.Parallel() + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + url := createSQSQueueForTest(t, node, "max-range") + + for _, bad := range []int{0, -1, 11, 100} { + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": url, + "MaxNumberOfMessages": bad, + }) + if status != http.StatusBadRequest { + t.Fatalf("MaxNumberOfMessages=%d: got %d want 400 (%v)", bad, status, out) + } + if got, _ := out["__type"].(string); got != sqsErrInvalidAttributeValue { + t.Fatalf("MaxNumberOfMessages=%d error type: %q want %q", bad, got, sqsErrInvalidAttributeValue) + } + } + + // Omitted → defaults to 1, succeeds (empty queue returns 0 messages). + status, out := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": url, + }) + if status != http.StatusOK { + t.Fatalf("omitted MaxNumberOfMessages: %d %v", status, out) + } + + // In-range → succeeds. + status, out = callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": url, + "MaxNumberOfMessages": 10, + }) + if status != http.StatusOK { + t.Fatalf("MaxNumberOfMessages=10: %d %v", status, out) + } +} diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 2f3362250..a28aab712 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -253,7 +253,7 @@ type sqsSendMessageInput struct { type sqsReceiveMessageInput struct { QueueUrl string `json:"QueueUrl"` - MaxNumberOfMessages int `json:"MaxNumberOfMessages,omitempty"` + MaxNumberOfMessages *int `json:"MaxNumberOfMessages,omitempty"` VisibilityTimeout *int64 `json:"VisibilityTimeout,omitempty"` WaitTimeSeconds *int64 `json:"WaitTimeSeconds,omitempty"` } @@ -485,7 +485,11 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { writeSQSError(w, http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") return } - max := clampReceiveMaxMessages(in.MaxNumberOfMessages) + max, maxErr := resolveReceiveMaxMessages(in.MaxNumberOfMessages) + if maxErr != nil { + writeSQSErrorFromErr(w, maxErr) + return + } visibilityTimeout := meta.VisibilityTimeoutSeconds if in.VisibilityTimeout != nil { if *in.VisibilityTimeout < 0 || *in.VisibilityTimeout > sqsChangeVisibilityMaxSeconds { @@ -581,14 +585,21 @@ func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, ge return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS), nil } -func clampReceiveMaxMessages(requested int) int { - if requested <= 0 { - return sqsReceiveDefaultMaxMessages +// resolveReceiveMaxMessages validates MaxNumberOfMessages against the +// AWS-documented range [1, 10]. An omitted value defaults to 1. +// Anything explicitly outside the range is an InvalidParameterValue +// — silently clamping would let a caller bug (e.g. passing 0 or a +// negative value) change to active polling behavior without surfacing +// the error, matching the same policy we apply to WaitTimeSeconds. +func resolveReceiveMaxMessages(requested *int) (int, error) { + if requested == nil { + return sqsReceiveDefaultMaxMessages, nil } - if requested > sqsReceiveHardMaxMessages { - return sqsReceiveHardMaxMessages + v := *requested + if v < 1 || v > sqsReceiveHardMaxMessages { + return 0, newSQSAPIError(http.StatusBadRequest, sqsErrInvalidAttributeValue, "MaxNumberOfMessages must be between 1 and 10") } - return requested + return v, nil } // scanVisibleMessageCandidates returns vis-index entries with From 94e3a3feecc3ca2cbe4b0c6d9ee7e5f49031d8fe Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 01:12:22 +0900 Subject: [PATCH 23/26] fix(sqs): propagate non-retryable receive rotation failures MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit codex P1 (new): tryDeliverCandidate collapsed every Dispatch failure into a silent candidate skip, so ReceiveMessage returned 200 with an empty or undersized Messages array even when the coordinator had stepped down, a shard was unroutable, or storage was failing. That hides production incidents and stalls consumers. Refactor the delivery path to tri-state returns: - tryDeliverCandidate returns (msg, skip bool, err error): * (msg, false, nil) — delivered. * (nil, true, nil) — expected race; skip this candidate. Covers only ErrKeyNotFound on the initial GetAt (another worker deleted the record between our scan and load) and ErrWriteConflict on dispatch (another receive rotated the same record first). * (nil, false, err) — non-retryable; propagate. - rotateMessagesForDelivery returns ([]msgs, error). On non-retryable error it returns the messages already rotated in this batch (committed, so they must be delivered) together with the error. - scanAndDeliverOnce propagates, longPollReceive propagates, receiveMessage renders the error through writeSQSErrorFromErr (generic 500 body; full detail logged via slog as round 10). No behavior change on the happy path or on genuine races; only coordinator / routing / storage outages stop being masked as false-empty 200s. --- adapter/sqs_messages.go | 63 ++++++++++++++++++++++++++++++----------- 1 file changed, 46 insertions(+), 17 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index a28aab712..8d5b75be4 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -582,7 +582,7 @@ func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, ge if err != nil { return nil, err } - return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS), nil + return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS) } // resolveReceiveMaxMessages validates MaxNumberOfMessages against the @@ -633,10 +633,15 @@ func (s *SQSServer) scanVisibleMessageCandidates(ctx context.Context, queueName } // rotateMessagesForDelivery runs an OCC transaction per candidate to -// rotate its visibility entry + receipt token. Failures on individual -// messages (races, write-conflict) are skipped rather than aborting the -// whole batch — AWS semantics allow ReceiveMessage to return fewer -// messages than requested. +// rotate its visibility entry + receipt token. Expected race +// conditions (the message was deleted between scan and load, or +// another worker already rotated the same candidate — ErrWriteConflict) +// skip the candidate rather than aborting the whole batch; AWS lets +// ReceiveMessage return fewer messages than requested. But any +// non-retryable dispatch error (coordinator outage, shard routing +// failure, storage failure) propagates, because silently returning +// an empty 200 in those cases would stall consumers and hide the +// incident. func (s *SQSServer) rotateMessagesForDelivery( ctx context.Context, queueName string, @@ -645,21 +650,35 @@ func (s *SQSServer) rotateMessagesForDelivery( visibilityTimeout int64, max int, readTS uint64, -) []map[string]any { +) ([]map[string]any, error) { delivered := make([]map[string]any, 0, max) for _, cand := range candidates { if len(delivered) >= max { break } - msg, ok := s.tryDeliverCandidate(ctx, queueName, gen, cand, visibilityTimeout, readTS) - if !ok { + msg, skip, err := s.tryDeliverCandidate(ctx, queueName, gen, cand, visibilityTimeout, readTS) + if err != nil { + return delivered, err + } + if skip { continue } delivered = append(delivered, msg) } - return delivered + return delivered, nil } +// tryDeliverCandidate attempts one scan→load→rotate for a single +// candidate. The return triple is: +// +// - (msg, false, nil) → delivered, caller appends. +// - (nil, true, nil) → expected race; skip this candidate only. +// Covers ErrKeyNotFound (someone deleted the record between the +// vis-index scan and our GetAt) and ErrWriteConflict on dispatch +// (another receive rotated the same record). +// - (nil, false, err) → non-retryable failure; propagate up the +// stack so ReceiveMessage returns an actionable 5xx instead of +// a false-empty 200. func (s *SQSServer) tryDeliverCandidate( ctx context.Context, queueName string, @@ -667,20 +686,25 @@ func (s *SQSServer) tryDeliverCandidate( cand sqsMsgCandidate, visibilityTimeout int64, readTS uint64, -) (map[string]any, bool) { +) (map[string]any, bool, error) { dataKey := sqsMsgDataKey(queueName, gen, cand.messageID) raw, err := s.store.GetAt(ctx, dataKey, readTS) if err != nil { - return nil, false + if errors.Is(err, store.ErrKeyNotFound) { + // Message gone mid-flight (probably deleted by another + // worker). Skip; don't fail the batch. + return nil, true, nil + } + return nil, false, errors.WithStack(err) } rec, err := decodeSQSMessageRecord(raw) if err != nil { - return nil, false + return nil, false, err } newToken, err := newReceiptToken() if err != nil { - return nil, false + return nil, false, err } now := time.Now().UnixMilli() newVisibleAt := now + visibilityTimeout*sqsMillisPerSecond @@ -692,7 +716,7 @@ func (s *SQSServer) tryDeliverCandidate( } recordBytes, err := encodeSQSMessageRecord(rec) if err != nil { - return nil, false + return nil, false, err } newVisKey := sqsMsgVisKey(queueName, gen, newVisibleAt, cand.messageID) // StartTS pins the OCC read snapshot to the timestamp we actually @@ -713,12 +737,17 @@ func (s *SQSServer) tryDeliverCandidate( }, } if _, err := s.coordinator.Dispatch(ctx, req); err != nil { - return nil, false + if isRetryableTransactWriteError(err) { + // Another concurrent receive rotated the same message; + // this candidate is no longer ours to deliver. + return nil, true, nil + } + return nil, false, errors.WithStack(err) } handle, err := encodeReceiptHandle(gen, cand.messageID, newToken) if err != nil { - return nil, false + return nil, false, err } return map[string]any{ "MessageId": cand.messageID, @@ -730,7 +759,7 @@ func (s *SQSServer) tryDeliverCandidate( "SentTimestamp": strconv.FormatInt(rec.SendTimestampMillis, 10), "ApproximateFirstReceiveTimestamp": strconv.FormatInt(rec.FirstReceiveMillis, 10), }, - }, true + }, false, nil } func (s *SQSServer) deleteMessage(w http.ResponseWriter, r *http.Request) { From 55fed695a9dfc9f664aadedf424c79071fe56928 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 01:24:53 +0900 Subject: [PATCH 24/26] fix(sqs): address codex round 14 findings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two new P1s in codex review 4171957563: - tryDeliverCandidate's OCC ReadKeys covered only cand.visKey + dataKey, but DeleteQueue only mutates queue meta + generation keys — so a DeleteQueue that raced with an in-flight receive would NOT produce a write conflict, letting the rotation commit and return a message from an already-deleted queue generation. ReadKeys now additionally include sqsQueueMetaKey / sqsQueueGenKey so concurrent DeleteQueue / PurgeQueue aborts the receive with ErrWriteConflict, which propagates as a retryable 5xx rather than surfacing stale messages from a dead queue. - RedrivePolicy was silently accepted as a queue attribute, but receive has no DLQ-redrive logic — messages that hit maxReceiveCount never move to the DLQ and redeliver indefinitely. Milestone 1 cannot honor the contract, so the attribute now returns 501 NotImplemented ("DLQ redrive is tracked for Milestone 2") from both CreateQueue and SetQueueAttributes, matching the earlier decision to reject MessageAttributes until the canonical MD5 encoder lands. New test TestSQSServer_CreateQueueRejectsRedrivePolicy covers the attribute-rejection behavior on both Create and SetAttributes paths. --- adapter/sqs_catalog.go | 15 ++++++++++++--- adapter/sqs_catalog_test.go | 38 +++++++++++++++++++++++++++++++++++++ adapter/sqs_messages.go | 11 ++++++++--- 3 files changed, 58 insertions(+), 6 deletions(-) diff --git a/adapter/sqs_catalog.go b/adapter/sqs_catalog.go index e25e5b745..13c9d02f9 100644 --- a/adapter/sqs_catalog.go +++ b/adapter/sqs_catalog.go @@ -362,9 +362,18 @@ var sqsAttributeAppliers = map[string]attributeApplier{ m.ContentBasedDedup = b return nil }, - "RedrivePolicy": func(m *sqsQueueMeta, v string) error { - m.RedrivePolicy = v - return nil + "RedrivePolicy": func(_ *sqsQueueMeta, _ string) error { + // Milestone 1 does not enforce DLQ redrive at receive time, + // so silently accepting RedrivePolicy would advertise a + // feature clients rely on (poison messages moving to the + // DLQ after maxReceiveCount) that this adapter does not + // actually provide — receivers would see infinite + // redelivery instead. Reject the attribute until the + // Milestone-2 receive path that actually performs the DLQ + // move lands, so operators get a clear signal instead of + // a silently-broken queue. + return newSQSAPIError(http.StatusNotImplemented, sqsErrNotImplemented, + "RedrivePolicy is not yet supported; DLQ redrive is tracked for Milestone 2") }, } diff --git a/adapter/sqs_catalog_test.go b/adapter/sqs_catalog_test.go index e19ddab6c..d57cd8648 100644 --- a/adapter/sqs_catalog_test.go +++ b/adapter/sqs_catalog_test.go @@ -358,6 +358,44 @@ func TestSQSServer_SetQueueAttributesRequiresAttributes(t *testing.T) { } } +func TestSQSServer_CreateQueueRejectsRedrivePolicy(t *testing.T) { + t.Parallel() + // Milestone 1 does not enforce DLQ redrive on the receive path, so + // accepting RedrivePolicy would silently advertise a feature the + // adapter can't deliver — poison messages would redeliver + // indefinitely instead of moving to the DLQ. Until the Milestone-2 + // receive-side DLQ move lands, reject the attribute loudly. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + status, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "with-redrive", + "Attributes": map[string]string{ + "RedrivePolicy": `{"deadLetterTargetArn":"arn:aws:sqs:us-east-1:000000000000:dlq","maxReceiveCount":"5"}`, + }, + }) + if status != http.StatusNotImplemented { + t.Fatalf("CreateQueue with RedrivePolicy: got %d want 501 (%v)", status, out) + } + if got, _ := out["__type"].(string); got != sqsErrNotImplemented { + t.Fatalf("error type: %q want %q", got, sqsErrNotImplemented) + } + + // SetQueueAttributes rejects the same attribute on an existing + // queue. + url := createSQSQueueForTest(t, node, "no-redrive") + status, out = callSQS(t, node, sqsSetQueueAttributesTarget, map[string]any{ + "QueueUrl": url, + "Attributes": map[string]string{ + "RedrivePolicy": `{"maxReceiveCount":"3"}`, + }, + }) + if status != http.StatusNotImplemented { + t.Fatalf("SetQueueAttributes with RedrivePolicy: got %d want 501 (%v)", status, out) + } +} + func TestSQSServer_ReceiveMessageRejectsOutOfRangeMax(t *testing.T) { t.Parallel() nodes, _, _ := createNode(t, 1) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 8d5b75be4..f40b3a706 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -724,12 +724,17 @@ func (s *SQSServer) tryDeliverCandidate( // StartTS at dispatch, so a concurrent rotation that committed // AFTER our read but BEFORE the assigned StartTS would slip through // ReadKeys validation and let this transaction double-deliver. - // ReadKeys cover both the visibility entry and the data record so - // any concurrent commit on either produces ErrWriteConflict. + // ReadKeys cover: + // - cand.visKey + dataKey: concurrent receive rotation → conflict. + // - sqsQueueMetaKey / sqsQueueGenKey: concurrent DeleteQueue / + // PurgeQueue → conflict. DeleteQueue only mutates the meta / + // generation records and would otherwise slip past an + // unqualified ReadKeys set, letting this rotation commit a + // message under a dead generation. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, StartTS: readTS, - ReadKeys: [][]byte{cand.visKey, dataKey}, + ReadKeys: [][]byte{cand.visKey, dataKey, sqsQueueMetaKey(queueName), sqsQueueGenKey(queueName)}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: cand.visKey}, {Op: kv.Put, Key: newVisKey, Value: []byte(cand.messageID)}, From fa7fdbfaf9cd2f8d0aac913d5e0c147198f7b963 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 01:36:56 +0900 Subject: [PATCH 25/26] fix(sqs): close DeleteQueue races on receive loop and ChangeMessageVisibility MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit codex round 15 P1×2: - receiveMessage loaded queue metadata once and reused the cached generation for every long-poll scan pass. A DeleteQueue that committed mid-wait would leave orphan !sqs|msg|vis| entries behind under the old generation (DeleteQueue only mutates meta + generation, never the message keys), and the cached-gen scan would happily find and deliver them even after ack. scan AndDeliverOnce now re-reads queue metadata on every pass — the fresh read sees the generation bump immediately and returns QueueDoesNotExist instead of scanning stale keys. The outer handler no longer passes a gen parameter; it only passes the queue name. - changeVisibilityWithRetry and deleteMessageWithRetry's OCC ReadKeys covered only the message record and its vis entry, but DeleteQueue / PurgeQueue races mutate queue meta + generation keys. A delete that commits between loadMessageFor* and Dispatch would slip past the conflict check and mutate orphan records on an already-deleted queue. Both dispatches now include sqsQueueMetaKey + sqsQueueGenKey in ReadKeys (same fix tryDeliverCandidate got one round earlier), so a DeleteQueue race aborts the in-flight delete / change-visibility with ErrWriteConflict which the retry loop classifies as ReceiptHandleIsInvalid on the next pass. No behavior change for uncontended callers. The race windows these findings flagged now close via OCC conflict instead of returning success against a dead generation. --- adapter/sqs_messages.go | 46 ++++++++++++++++++++++++++++------------- 1 file changed, 32 insertions(+), 14 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index f40b3a706..5654d6341 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -504,7 +504,7 @@ func (s *SQSServer) receiveMessage(w http.ResponseWriter, r *http.Request) { return } - delivered, err := s.longPollReceive(ctx, queueName, meta.Generation, max, visibilityTimeout, waitSeconds) + delivered, err := s.longPollReceive(ctx, queueName, max, visibilityTimeout, waitSeconds) if err != nil { writeSQSErrorFromErr(w, err) return @@ -540,9 +540,12 @@ func resolveReceiveWaitSeconds(requested *int64, queueDefault int64) (int64, err // // Scan errors are propagated to the caller so a backend / routing // failure surfaces as an actionable 5xx instead of a silent empty 200 -// that would stall consumers. -func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout, waitSeconds int64) ([]map[string]any, error) { - delivered, err := s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) +// that would stall consumers. Each pass re-resolves queue metadata so +// a DeleteQueue / PurgeQueue that commits during a long wait is +// observed on the very next scan — otherwise we'd keep scanning +// orphan keys under the old generation. +func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, max int, visibilityTimeout, waitSeconds int64) ([]map[string]any, error) { + delivered, err := s.scanAndDeliverOnce(ctx, queueName, max, visibilityTimeout) if err != nil { return nil, err } @@ -561,7 +564,7 @@ func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen u if time.Now().After(deadline) { return delivered, nil } - delivered, err = s.scanAndDeliverOnce(ctx, queueName, gen, max, visibilityTimeout) + delivered, err = s.scanAndDeliverOnce(ctx, queueName, max, visibilityTimeout) if err != nil { return nil, err } @@ -573,16 +576,25 @@ func (s *SQSServer) longPollReceive(ctx context.Context, queueName string, gen u // scanAndDeliverOnce is the single-pass scan+rotate the long-poll loop // re-runs. Each pass takes its own snapshot so the OCC StartTS tracks -// the most recent visible_at for the candidates it picked. Scan errors -// are returned so the caller can fail the receive with an actionable -// status code instead of serializing them as empty success. -func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, gen uint64, max int, visibilityTimeout int64) ([]map[string]any, error) { +// the most recent visible_at for the candidates it picked, AND each +// pass re-reads queue metadata so a concurrent DeleteQueue / +// PurgeQueue that bumps the generation is observed immediately. If +// the queue has been deleted the method returns QueueDoesNotExist; +// scan errors and other non-retryable failures propagate. +func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, max int, visibilityTimeout int64) ([]map[string]any, error) { readTS := s.nextTxnReadTS(ctx) - candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, gen, max*sqsReceiveScanOverfetchFactor, readTS) + meta, exists, err := s.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil { + return nil, errors.WithStack(err) + } + if !exists { + return nil, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") + } + candidates, err := s.scanVisibleMessageCandidates(ctx, queueName, meta.Generation, max*sqsReceiveScanOverfetchFactor, readTS) if err != nil { return nil, err } - return s.rotateMessagesForDelivery(ctx, queueName, gen, candidates, visibilityTimeout, max, readTS) + return s.rotateMessagesForDelivery(ctx, queueName, meta.Generation, candidates, visibilityTimeout, max, readTS) } // resolveReceiveMaxMessages validates MaxNumberOfMessages against the @@ -812,10 +824,14 @@ func (s *SQSServer) deleteMessageWithRetry(ctx context.Context, queueName string // StartTS pins OCC to the snapshot we loaded the record at, so a // concurrent rotation that commits after our load but before a // coordinator-assigned StartTS cannot slip past ReadKeys. + // ReadKeys also include the queue meta + generation keys so a + // concurrent DeleteQueue (which only touches those two keys) + // forces this delete to abort with ErrWriteConflict rather + // than committing against an orphan record. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, StartTS: readTS, - ReadKeys: [][]byte{dataKey, visKey}, + ReadKeys: [][]byte{dataKey, visKey, sqsQueueMetaKey(queueName), sqsQueueGenKey(queueName)}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: dataKey}, {Op: kv.Del, Key: visKey}, @@ -944,11 +960,13 @@ func (s *SQSServer) changeVisibilityWithRetry(ctx context.Context, queueName str // StartTS pins OCC to the snapshot; without it the coordinator // would auto-assign a newer StartTS and a concurrent receive / // delete that commits between our load and dispatch could slip - // past the ReadKeys validation. + // past the ReadKeys validation. Meta + generation keys are + // included so a DeleteQueue race (which only mutates those + // two keys) also forces this visibility change to abort. req := &kv.OperationGroup[kv.OP]{ IsTxn: true, StartTS: readTS, - ReadKeys: [][]byte{dataKey, oldVisKey}, + ReadKeys: [][]byte{dataKey, oldVisKey, sqsQueueMetaKey(queueName), sqsQueueGenKey(queueName)}, Elems: []*kv.Elem[kv.OP]{ {Op: kv.Del, Key: oldVisKey}, {Op: kv.Put, Key: newVisKey, Value: []byte(rec.MessageID)}, From 818c27cb66e3ce7d4853e16498306cb63bb0f430 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" <contact@bootjp.me> Date: Sat, 25 Apr 2026 02:00:22 +0900 Subject: [PATCH 26/26] fix(sqs): reject empty MessageBody + enforce retention on receive MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit codex round 16, two findings: - P2 SendMessage: loadQueueMetaForSend only checked the upper body size bound; an omitted or empty MessageBody would be accepted and enqueued silently. AWS rejects empty bodies as InvalidParameterValue, and silently admitting them would let producer serialization bugs land on the queue disguised as real messages. Added the lower-bound check right next to the size cap. - P1 ReceiveMessage: the receive path loaded MessageRetentionPeriod via queue metadata but never applied it, so messages older than send_timestamp + retention could be re-delivered indefinitely on every visibility-timeout cycle. tryDeliverCandidate now takes the retention seconds, compares against rec.SendTimestampMillis, and when expired runs a dedicated expireMessage OCC txn that drops the data + vis entries (idempotent on ErrWriteConflict — another worker racing us to delete or rotate is fine). The candidate is then skipped for this batch, so expired messages never reach clients. A Milestone-2 send-age reaper can still sweep in bulk when the queue is idle. tryDeliverCandidate is split into loadCandidateRecord / handleRetentionExpiry / commitReceiveRotation so the retention branch stays under the cyclomatic limit. Tests: TestSQSServer_SendMessageRejectsEmptyBody covers the two empty-body request shapes; TestSQSServer_ReceiveDropsRetentionExpiredMessages seeds a queue with MessageRetentionPeriod=60 s, backdates the stored SendTimestampMillis via a new backdateSQSMessageForTest helper, and asserts that two consecutive receives return empty (message was reaped inline, not just hidden). --- adapter/sqs_messages.go | 120 ++++++++++++++++++++++++++-------- adapter/sqs_messages_test.go | 122 +++++++++++++++++++++++++++++++++++ 2 files changed, 215 insertions(+), 27 deletions(-) diff --git a/adapter/sqs_messages.go b/adapter/sqs_messages.go index 5654d6341..05742b862 100644 --- a/adapter/sqs_messages.go +++ b/adapter/sqs_messages.go @@ -362,6 +362,14 @@ func (s *SQSServer) loadQueueMetaForSend(ctx context.Context, queueName string, if !exists { return nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrQueueDoesNotExist, "queue does not exist") } + // AWS rejects SendMessage when MessageBody is empty (InvalidParameterValue + // "Message body cannot be empty"). Silently enqueuing a zero-length + // message would let producer bugs (missing serialization, a forgotten + // body parameter) land in the queue where they later look like real + // messages to consumers. + if len(body) == 0 { + return nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrValidation, "MessageBody is required") + } if int64(len(body)) > meta.MaximumMessageSize { return nil, readTS, newSQSAPIError(http.StatusBadRequest, sqsErrMessageTooLong, "message body exceeds MaximumMessageSize") } @@ -594,7 +602,7 @@ func (s *SQSServer) scanAndDeliverOnce(ctx context.Context, queueName string, ma if err != nil { return nil, err } - return s.rotateMessagesForDelivery(ctx, queueName, meta.Generation, candidates, visibilityTimeout, max, readTS) + return s.rotateMessagesForDelivery(ctx, queueName, meta.Generation, candidates, visibilityTimeout, meta.MessageRetentionSeconds, max, readTS) } // resolveReceiveMaxMessages validates MaxNumberOfMessages against the @@ -654,12 +662,61 @@ func (s *SQSServer) scanVisibleMessageCandidates(ctx context.Context, queueName // failure, storage failure) propagates, because silently returning // an empty 200 in those cases would stall consumers and hide the // incident. +// loadCandidateRecord fetches and decodes the message record for a +// receive candidate. Returns (rec, dataKey, skip, err): +// - skip=true, err=nil : ErrKeyNotFound race; caller skips this one. +// - skip=false, err!=nil : non-retryable; propagate. +// - skip=false, err=nil : record loaded. +func (s *SQSServer) loadCandidateRecord(ctx context.Context, queueName string, gen uint64, cand sqsMsgCandidate, readTS uint64) (*sqsMessageRecord, []byte, bool, error) { + dataKey := sqsMsgDataKey(queueName, gen, cand.messageID) + raw, err := s.store.GetAt(ctx, dataKey, readTS) + if err != nil { + if errors.Is(err, store.ErrKeyNotFound) { + return nil, dataKey, true, nil + } + return nil, dataKey, false, errors.WithStack(err) + } + rec, err := decodeSQSMessageRecord(raw) + if err != nil { + return nil, dataKey, false, err + } + return rec, dataKey, false, nil +} + +// expireMessage removes a retention-expired record and its current +// visibility index entry in a single OCC transaction. On +// ErrWriteConflict (another worker raced us to delete or rotate +// this same message) we treat it as success: the message is no +// longer our responsibility either way. Any other error propagates +// so a coordinator / storage failure does not silently fall through +// to "delivered empty", matching the receive-error policy. +func (s *SQSServer) expireMessage(ctx context.Context, queueName string, gen uint64, visKey, dataKey []byte, readTS uint64) error { + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + StartTS: readTS, + ReadKeys: [][]byte{visKey, dataKey, sqsQueueMetaKey(queueName), sqsQueueGenKey(queueName)}, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Del, Key: visKey}, + {Op: kv.Del, Key: dataKey}, + }, + } + if _, err := s.coordinator.Dispatch(ctx, req); err != nil { + if isRetryableTransactWriteError(err) { + return nil + } + return errors.WithStack(err) + } + _ = gen // reserved for future per-queue expiry metrics + return nil +} + func (s *SQSServer) rotateMessagesForDelivery( ctx context.Context, queueName string, gen uint64, candidates []sqsMsgCandidate, visibilityTimeout int64, + retentionSeconds int64, max int, readTS uint64, ) ([]map[string]any, error) { @@ -668,7 +725,7 @@ func (s *SQSServer) rotateMessagesForDelivery( if len(delivered) >= max { break } - msg, skip, err := s.tryDeliverCandidate(ctx, queueName, gen, cand, visibilityTimeout, readTS) + msg, skip, err := s.tryDeliverCandidate(ctx, queueName, gen, cand, visibilityTimeout, retentionSeconds, readTS) if err != nil { return delivered, err } @@ -697,23 +754,41 @@ func (s *SQSServer) tryDeliverCandidate( gen uint64, cand sqsMsgCandidate, visibilityTimeout int64, + retentionSeconds int64, readTS uint64, ) (map[string]any, bool, error) { - dataKey := sqsMsgDataKey(queueName, gen, cand.messageID) - raw, err := s.store.GetAt(ctx, dataKey, readTS) - if err != nil { - if errors.Is(err, store.ErrKeyNotFound) { - // Message gone mid-flight (probably deleted by another - // worker). Skip; don't fail the batch. - return nil, true, nil - } - return nil, false, errors.WithStack(err) + rec, dataKey, skip, err := s.loadCandidateRecord(ctx, queueName, gen, cand, readTS) + if skip || err != nil { + return nil, skip, err } - rec, err := decodeSQSMessageRecord(raw) - if err != nil { - return nil, false, err + if expired, err := s.handleRetentionExpiry(ctx, queueName, gen, cand, dataKey, rec, retentionSeconds, readTS); expired || err != nil { + return nil, expired, err + } + return s.commitReceiveRotation(ctx, queueName, gen, cand, dataKey, rec, visibilityTimeout, readTS) +} + +// handleRetentionExpiry deletes the candidate inline when its +// send age exceeds MessageRetentionPeriod, so the vis-index scan +// does not keep re-finding it. Returns (expired, err): expired=true +// means the candidate has been (or is being) reaped and the caller +// must skip. +func (s *SQSServer) handleRetentionExpiry(ctx context.Context, queueName string, gen uint64, cand sqsMsgCandidate, dataKey []byte, rec *sqsMessageRecord, retentionSeconds int64, readTS uint64) (bool, error) { + if retentionSeconds <= 0 { + return false, nil + } + now := time.Now().UnixMilli() + if now-rec.SendTimestampMillis <= retentionSeconds*sqsMillisPerSecond { + return false, nil } + if err := s.expireMessage(ctx, queueName, gen, cand.visKey, dataKey, readTS); err != nil { + return false, err + } + return true, nil +} +// commitReceiveRotation runs the final OCC dispatch that rotates +// receipt token + visibility index for a non-expired candidate. +func (s *SQSServer) commitReceiveRotation(ctx context.Context, queueName string, gen uint64, cand sqsMsgCandidate, dataKey []byte, rec *sqsMessageRecord, visibilityTimeout int64, readTS uint64) (map[string]any, bool, error) { newToken, err := newReceiptToken() if err != nil { return nil, false, err @@ -732,17 +807,10 @@ func (s *SQSServer) tryDeliverCandidate( } newVisKey := sqsMsgVisKey(queueName, gen, newVisibleAt, cand.messageID) // StartTS pins the OCC read snapshot to the timestamp we actually - // loaded the record at. Without it, the coordinator assigns a newer - // StartTS at dispatch, so a concurrent rotation that committed - // AFTER our read but BEFORE the assigned StartTS would slip through - // ReadKeys validation and let this transaction double-deliver. - // ReadKeys cover: - // - cand.visKey + dataKey: concurrent receive rotation → conflict. - // - sqsQueueMetaKey / sqsQueueGenKey: concurrent DeleteQueue / - // PurgeQueue → conflict. DeleteQueue only mutates the meta / - // generation records and would otherwise slip past an - // unqualified ReadKeys set, letting this rotation commit a - // message under a dead generation. + // loaded the record at. ReadKeys cover: cand.visKey + dataKey so a + // concurrent rotation → conflict; sqsQueueMetaKey / sqsQueueGenKey + // so a concurrent DeleteQueue / PurgeQueue → conflict (DeleteQueue + // only mutates meta + generation and would otherwise slip through). req := &kv.OperationGroup[kv.OP]{ IsTxn: true, StartTS: readTS, @@ -755,8 +823,6 @@ func (s *SQSServer) tryDeliverCandidate( } if _, err := s.coordinator.Dispatch(ctx, req); err != nil { if isRetryableTransactWriteError(err) { - // Another concurrent receive rotated the same message; - // this candidate is no longer ours to deliver. return nil, true, nil } return nil, false, errors.WithStack(err) diff --git a/adapter/sqs_messages_test.go b/adapter/sqs_messages_test.go index eb8f24014..cf8291a8a 100644 --- a/adapter/sqs_messages_test.go +++ b/adapter/sqs_messages_test.go @@ -1,12 +1,16 @@ package adapter import ( + "bytes" + "context" "encoding/hex" "net/http" "strconv" "sync" "testing" "time" + + "github.com/bootjp/elastickv/kv" ) // createSQSQueueForTest is a small helper so every message-path test does @@ -550,6 +554,124 @@ func TestSQSServer_CreateQueueRequiresExplicitFifoFlag(t *testing.T) { } } +func TestSQSServer_SendMessageRejectsEmptyBody(t *testing.T) { + t.Parallel() + // AWS rejects SendMessage with an empty MessageBody + // (InvalidParameterValue). Accepting an empty body silently would + // let producer bugs (missing serialization, forgotten body + // parameter) land on the queue as fake-but-real messages. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + queueURL := createSQSQueueForTest(t, node, "empty-body") + + // Omitted MessageBody. + status, body := callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + }) + if status != http.StatusBadRequest { + t.Fatalf("omitted body: got %d want 400 (%v)", status, body) + } + + // Explicit empty body. + status, body = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": queueURL, + "MessageBody": "", + }) + if status != http.StatusBadRequest { + t.Fatalf("empty body: got %d want 400 (%v)", status, body) + } +} + +func TestSQSServer_ReceiveDropsRetentionExpiredMessages(t *testing.T) { + t.Parallel() + // Set MessageRetentionPeriod to the minimum (60 s), send a + // message, then manually backdate its send_timestamp past + // retention. The next ReceiveMessage must NOT return the message + // and must also clean it up so a subsequent scan does not + // reappear it. + nodes, _, _ := createNode(t, 1) + defer shutdown(nodes) + node := sqsLeaderNode(t, nodes) + + _, out := callSQS(t, node, sqsCreateQueueTarget, map[string]any{ + "QueueName": "retention-drop", + "Attributes": map[string]string{ + "MessageRetentionPeriod": "60", + }, + }) + url, _ := out["QueueUrl"].(string) + _, _ = callSQS(t, node, sqsSendMessageTarget, map[string]any{ + "QueueUrl": url, + "MessageBody": "stale", + }) + + // Cheat: reach into the store and backdate the just-sent record + // to simulate a long-retention-expired message. We could not + // just time.Sleep(60s) in a unit test. + backdateSQSMessageForTest(t, nodes[0], "retention-drop", 120*time.Second) + + // Receive must return nothing and must not replay the message. + for i := 0; i < 2; i++ { + status, body := callSQS(t, node, sqsReceiveMessageTarget, map[string]any{ + "QueueUrl": url, + "MaxNumberOfMessages": 1, + }) + if status != http.StatusOK { + t.Fatalf("receive #%d: %d %v", i, status, body) + } + if msgs, _ := body["Messages"].([]any); len(msgs) != 0 { + t.Fatalf("receive #%d delivered an expired message: %v", i, msgs) + } + } +} + +// backdateSQSMessageForTest walks the data-key prefix for the queue, +// rewrites the single message record it finds so SendTimestampMillis +// is `age` in the past, and commits the change through the adapter's +// coordinator. That simulates a retention-expired message without +// requiring a time.Sleep in tests. +func backdateSQSMessageForTest(t *testing.T, node Node, queueName string, age time.Duration) { + t.Helper() + ctx := context.Background() + srv := node.sqsServer + readTS := srv.nextTxnReadTS(ctx) + meta, exists, err := srv.loadQueueMetaAt(ctx, queueName, readTS) + if err != nil || !exists { + t.Fatalf("loadQueueMetaAt: err=%v exists=%v", err, exists) + } + prefix := []byte(SqsMsgDataPrefix + encodeSQSSegment(queueName)) + // Append gen-u64 so we only scan this generation. + prefix = appendU64(prefix, meta.Generation) + end := append(bytes.Clone(prefix), 0xff) + kvs, err := srv.store.ScanAt(ctx, prefix, end, 16, readTS) + if err != nil { + t.Fatalf("scan data: %v", err) + } + if len(kvs) == 0 { + t.Fatalf("no data records found for queue %q", queueName) + } + rec, err := decodeSQSMessageRecord(kvs[0].Value) + if err != nil { + t.Fatalf("decode: %v", err) + } + rec.SendTimestampMillis = time.Now().Add(-age).UnixMilli() + body, err := encodeSQSMessageRecord(rec) + if err != nil { + t.Fatalf("encode: %v", err) + } + req := &kv.OperationGroup[kv.OP]{ + IsTxn: true, + StartTS: readTS, + Elems: []*kv.Elem[kv.OP]{ + {Op: kv.Put, Key: bytes.Clone(kvs[0].Key), Value: body}, + }, + } + if _, err := srv.coordinator.Dispatch(ctx, req); err != nil { + t.Fatalf("dispatch: %v", err) + } +} + func TestSQSServer_ReceiptHandleCodecRoundTrip(t *testing.T) { t.Parallel() for _, tc := range []struct {